From a06eb72c012834100feafc1a90b7d950b1d4d57d Mon Sep 17 00:00:00 2001 From: Richard Ramos Date: Thu, 19 Sep 2024 16:16:53 -0400 Subject: [PATCH 01/14] feat_: extract storenode cycle to go-waku api --- eth-node/bridge/geth/waku.go | 62 +- eth-node/bridge/geth/wakuv2.go | 109 ++-- eth-node/types/mailserver.go | 54 -- eth-node/types/topic.go | 4 + eth-node/types/waku.go | 66 ++- go.mod | 2 +- go.sum | 4 +- protocol/messenger.go | 81 +-- protocol/messenger_communities.go | 31 +- protocol/messenger_mailserver.go | 443 ++++----------- protocol/messenger_mailserver_cycle.go | 530 +++--------------- ..._mailserver_processMailserverBatch_test.go | 167 ------ .../messenger_store_node_request_manager.go | 18 +- ...enger_store_node_request_manager_config.go | 8 +- protocol/messenger_storenode_comunity_test.go | 4 +- protocol/messenger_storenode_request_test.go | 16 +- protocol/messenger_testing_utils.go | 2 +- protocol/storenodes/storenodes.go | 13 +- protocol/transport/transport.go | 148 ++--- services/ext/api.go | 4 - .../waku-org/go-waku/logging/logging.go | 8 + .../go-waku/waku/v2/api/history/cycle.go | 524 +++++++++++++++++ .../go-waku/waku/v2/api/history/emitters.go | 48 ++ .../go-waku/waku/v2/api/history/history.go | 296 ++++++++++ .../go-waku/waku/v2/api/history/sort.go | 32 ++ .../waku/v2/api/missing/missing_messages.go | 10 +- .../waku/v2/api/publish/message_check.go | 15 +- .../waku/v2/api/publish/message_sender.go | 7 - .../go-waku/waku/v2/protocol/store/client.go | 28 +- .../go-waku/waku/v2/protocol/store/options.go | 4 + .../go-waku/waku/v2/protocol/store/result.go | 26 +- vendor/modules.txt | 3 +- wakuv2/history_processor_wrapper.go | 25 + wakuv2/waku.go | 81 +-- wakuv2/waku_test.go | 18 +- 35 files changed, 1506 insertions(+), 1385 deletions(-) delete mode 100644 protocol/messenger_mailserver_processMailserverBatch_test.go create mode 100644 vendor/github.com/waku-org/go-waku/waku/v2/api/history/cycle.go create mode 100644 vendor/github.com/waku-org/go-waku/waku/v2/api/history/emitters.go create mode 100644 vendor/github.com/waku-org/go-waku/waku/v2/api/history/history.go create mode 100644 vendor/github.com/waku-org/go-waku/waku/v2/api/history/sort.go create mode 100644 wakuv2/history_processor_wrapper.go diff --git a/eth-node/bridge/geth/waku.go b/eth-node/bridge/geth/waku.go index 1cf2c2feec3..5d4a8bc50dd 100644 --- a/eth-node/bridge/geth/waku.go +++ b/eth-node/bridge/geth/waku.go @@ -9,6 +9,8 @@ import ( "github.com/libp2p/go-libp2p/core/peer" "github.com/multiformats/go-multiaddr" + "github.com/waku-org/go-waku/waku/v2/api/history" + "github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/p2p/enode" gocommon "github.com/status-im/status-go/common" @@ -274,10 +276,6 @@ func (w *GethWakuWrapper) MarkP2PMessageAsProcessed(hash common.Hash) { w.waku.MarkP2PMessageAsProcessed(hash) } -func (w *GethWakuWrapper) RequestStoreMessages(ctx context.Context, peerID peer.ID, r types.MessagesRequest, processEnvelopes bool) (types.StoreRequestCursor, int, error) { - return nil, 0, errors.New("not implemented") -} - func (w *GethWakuWrapper) ConnectionChanged(_ connection.State) {} func (w *GethWakuWrapper) ClearEnvelopesCache() { @@ -314,13 +312,59 @@ func (w *wakuFilterWrapper) ID() string { func (w *GethWakuWrapper) ConfirmMessageDelivered(hashes []common.Hash) { } -func (w *GethWakuWrapper) SetStorePeerID(peerID peer.ID) { +func (w *GethWakuWrapper) PeerID() peer.ID { + panic("not available in WakuV1") } -func (w *GethWakuWrapper) PeerID() peer.ID { - panic("not implemented") +func (w *GethWakuWrapper) GetActiveStorenode() peer.ID { + panic("not available in WakuV1") +} + +func (w *GethWakuWrapper) OnStorenodeAvailableOneShot() <-chan struct{} { + panic("not available in WakuV1") +} + +func (w *GethWakuWrapper) OnStorenodeChanged() <-chan peer.ID { + panic("not available in WakuV1") +} + +func (w *GethWakuWrapper) OnStorenodeNotWorking() <-chan struct{} { + panic("not available in WakuV1") +} + +func (w *GethWakuWrapper) OnStorenodeAvailable() <-chan peer.ID { + panic("not available in WakuV1") +} + +func (w *GethWakuWrapper) WaitForAvailableStoreNode(timeout time.Duration) bool { + return false +} + +func (w *GethWakuWrapper) SetStorenodeConfigProvider(c history.StorenodeConfigProvider) { + panic("not available in WakuV1") +} + +func (w *GethWakuWrapper) ProcessMailserverBatch( + ctx context.Context, + batch types.MailserverBatch, + storenodeID peer.ID, + pageLimit uint64, + shouldProcessNextPage func(int) (bool, uint64), + processEnvelopes bool, +) error { + return errors.New("not available in WakuV1") +} + +func (w *GethWakuWrapper) IsStorenodeAvailable(peerID peer.ID) bool { + panic("not available in WakuV1") + +} + +func (w *GethWakuWrapper) PerformStorenodeTask(fn func() error, opts ...history.StorenodeTaskOption) error { + panic("not available in WakuV1") + } -func (w *GethWakuWrapper) PingPeer(context.Context, peer.ID) (time.Duration, error) { - return 0, errors.New("not available in WakuV1") +func (w *GethWakuWrapper) DisconnectActiveStorenode(ctx context.Context, backoff time.Duration, shouldCycle bool) { + panic("not available in WakuV1") } diff --git a/eth-node/bridge/geth/wakuv2.go b/eth-node/bridge/geth/wakuv2.go index 2a28dd38d07..a301698396f 100644 --- a/eth-node/bridge/geth/wakuv2.go +++ b/eth-node/bridge/geth/wakuv2.go @@ -9,6 +9,7 @@ import ( "github.com/multiformats/go-multiaddr" "google.golang.org/protobuf/proto" + "github.com/waku-org/go-waku/waku/v2/api/history" "github.com/waku-org/go-waku/waku/v2/protocol" "github.com/waku-org/go-waku/waku/v2/protocol/store" @@ -176,39 +177,6 @@ func (w *gethWakuV2Wrapper) createFilterWrapper(id string, keyAsym *ecdsa.Privat }, id), nil } -func (w *gethWakuV2Wrapper) RequestStoreMessages(ctx context.Context, peerID peer.ID, r types.MessagesRequest, processEnvelopes bool) (types.StoreRequestCursor, int, error) { - options := []store.RequestOption{ - store.WithPaging(false, uint64(r.Limit)), - } - - var cursor []byte - if r.StoreCursor != nil { - cursor = r.StoreCursor - } - - contentTopics := []string{} - for _, topic := range r.ContentTopics { - contentTopics = append(contentTopics, wakucommon.BytesToTopic(topic).ContentTopic()) - } - - query := store.FilterCriteria{ - TimeStart: proto.Int64(int64(r.From) * int64(time.Second)), - TimeEnd: proto.Int64(int64(r.To) * int64(time.Second)), - ContentFilter: protocol.NewContentFilter(w.waku.GetPubsubTopic(r.PubsubTopic), contentTopics...), - } - - pbCursor, envelopesCount, err := w.waku.Query(ctx, peerID, query, cursor, options, processEnvelopes) - if err != nil { - return nil, 0, err - } - - if pbCursor != nil { - return pbCursor, envelopesCount, nil - } - - return nil, envelopesCount, nil -} - func (w *gethWakuV2Wrapper) StartDiscV5() error { return w.waku.StartDiscV5() } @@ -289,7 +257,7 @@ func (w *gethWakuV2Wrapper) SubscribeToConnStatusChanges() (*types.ConnStatusSub func (w *gethWakuV2Wrapper) SetCriteriaForMissingMessageVerification(peerID peer.ID, pubsubTopic string, contentTopics []types.TopicType) error { var cTopics []string for _, ct := range contentTopics { - cTopics = append(cTopics, wakucommon.TopicType(ct).ContentTopic()) + cTopics = append(cTopics, wakucommon.BytesToTopic(ct.Bytes()).ContentTopic()) } pubsubTopic = w.waku.GetPubsubTopic(pubsubTopic) w.waku.SetTopicsToVerifyForMissingMessages(peerID, pubsubTopic, cTopics) @@ -338,14 +306,75 @@ func (w *gethWakuV2Wrapper) ConfirmMessageDelivered(hashes []common.Hash) { w.waku.ConfirmMessageDelivered(hashes) } -func (w *gethWakuV2Wrapper) SetStorePeerID(peerID peer.ID) { - w.waku.SetStorePeerID(peerID) -} - func (w *gethWakuV2Wrapper) PeerID() peer.ID { return w.waku.PeerID() } -func (w *gethWakuV2Wrapper) PingPeer(ctx context.Context, peerID peer.ID) (time.Duration, error) { - return w.waku.PingPeer(ctx, peerID) +func (w *gethWakuV2Wrapper) GetActiveStorenode() peer.ID { + return w.waku.StorenodeCycle.GetActiveStorenode() +} + +func (w *gethWakuV2Wrapper) OnStorenodeAvailableOneShot() <-chan struct{} { + return w.waku.StorenodeCycle.StorenodeAvailableOneshotEmitter.Subscribe() +} + +func (w *gethWakuV2Wrapper) OnStorenodeChanged() <-chan peer.ID { + return w.waku.StorenodeCycle.StorenodeChangedEmitter.Subscribe() +} + +func (w *gethWakuV2Wrapper) OnStorenodeNotWorking() <-chan struct{} { + return w.waku.StorenodeCycle.StorenodeNotWorkingEmitter.Subscribe() +} + +func (w *gethWakuV2Wrapper) OnStorenodeAvailable() <-chan peer.ID { + return w.waku.StorenodeCycle.StorenodeAvailableEmitter.Subscribe() +} + +func (w *gethWakuV2Wrapper) WaitForAvailableStoreNode(timeout time.Duration) bool { + return w.waku.StorenodeCycle.WaitForAvailableStoreNode(context.TODO(), timeout) +} + +func (w *gethWakuV2Wrapper) SetStorenodeConfigProvider(c history.StorenodeConfigProvider) { + w.waku.StorenodeCycle.SetStorenodeConfigProvider(c) +} + +func (w *gethWakuV2Wrapper) ProcessMailserverBatch( + ctx context.Context, + batch types.MailserverBatch, + storenodeID peer.ID, + pageLimit uint64, + shouldProcessNextPage func(int) (bool, uint64), + processEnvelopes bool, +) error { + pubsubTopic := w.waku.GetPubsubTopic(batch.PubsubTopic) + contentTopics := []string{} + for _, topic := range batch.Topics { + contentTopics = append(contentTopics, wakucommon.BytesToTopic(topic.Bytes()).ContentTopic()) + } + + criteria := store.FilterCriteria{ + TimeStart: proto.Int64(int64(batch.From) * int64(time.Second)), + TimeEnd: proto.Int64(int64(batch.To) * int64(time.Second)), + ContentFilter: protocol.NewContentFilter(pubsubTopic, contentTopics...), + } + + return w.waku.HistoryRetriever.Query(ctx, criteria, storenodeID, pageLimit, shouldProcessNextPage, processEnvelopes) +} + +func (w *gethWakuV2Wrapper) IsStorenodeAvailable(peerID peer.ID) bool { + return w.waku.StorenodeCycle.IsStorenodeAvailable(peerID) +} + +func (w *gethWakuV2Wrapper) PerformStorenodeTask(fn func() error, opts ...history.StorenodeTaskOption) error { + return w.waku.StorenodeCycle.PerformStorenodeTask(fn, opts...) +} + +func (w *gethWakuV2Wrapper) DisconnectActiveStorenode(ctx context.Context, backoff time.Duration, shouldCycle bool) { + w.waku.StorenodeCycle.Lock() + defer w.waku.StorenodeCycle.Unlock() + + w.waku.StorenodeCycle.DisconnectActiveStorenode(backoff) + if shouldCycle { + w.waku.StorenodeCycle.Cycle(ctx) + } } diff --git a/eth-node/types/mailserver.go b/eth-node/types/mailserver.go index 2ae7877ceac..dd24824867d 100644 --- a/eth-node/types/mailserver.go +++ b/eth-node/types/mailserver.go @@ -1,59 +1,5 @@ package types -import ( - "time" -) - -const ( - // MaxLimitInMessagesRequest represents the maximum number of messages - // that can be requested from the mailserver - MaxLimitInMessagesRequest = 1000 -) - -// MessagesRequest contains details of a request of historic messages. -type MessagesRequest struct { - // ID of the request. The current implementation requires ID to be 32-byte array, - // however, it's not enforced for future implementation. - ID []byte `json:"id"` - // From is a lower bound of time range. - From uint32 `json:"from"` - // To is a upper bound of time range. - To uint32 `json:"to"` - // Limit determines the number of messages sent by the mail server - // for the current paginated request. - Limit uint32 `json:"limit"` - // Cursor is used as starting point for paginated requests. - Cursor []byte `json:"cursor"` - // StoreCursor is used as starting point for WAKUV2 paginatedRequests - StoreCursor StoreRequestCursor `json:"storeCursor"` - // Bloom is a filter to match requested messages. - Bloom []byte `json:"bloom"` - // PubsubTopic is the gossipsub topic on which the message was broadcasted - PubsubTopic string `json:"pubsubTopic"` - // ContentTopics is a list of topics. A returned message should - // belong to one of the topics from the list. - ContentTopics [][]byte `json:"contentTopics"` -} - -type StoreRequestCursor []byte - -// SetDefaults sets the From and To defaults -func (r *MessagesRequest) SetDefaults(now time.Time) { - // set From and To defaults - if r.To == 0 { - r.To = uint32(now.UTC().Unix()) - } - - if r.From == 0 { - oneDay := uint32(86400) // -24 hours - if r.To < oneDay { - r.From = 0 - } else { - r.From = r.To - oneDay - } - } -} - // MailServerResponse is the response payload sent by the mailserver. type MailServerResponse struct { LastEnvelopeHash Hash diff --git a/eth-node/types/topic.go b/eth-node/types/topic.go index faeeb809ec7..c8fbdeca71e 100644 --- a/eth-node/types/topic.go +++ b/eth-node/types/topic.go @@ -34,6 +34,10 @@ func (t TopicType) String() string { return EncodeHex(t[:]) } +func (t TopicType) Bytes() []byte { + return TopicTypeToByteArray(t) +} + // MarshalText returns the hex representation of t. func (t TopicType) MarshalText() ([]byte, error) { return HexBytes(t[:]).MarshalText() diff --git a/eth-node/types/waku.go b/eth-node/types/waku.go index 1900505f34b..d38c28bbd68 100644 --- a/eth-node/types/waku.go +++ b/eth-node/types/waku.go @@ -3,7 +3,10 @@ package types import ( "context" "crypto/ecdsa" + "crypto/sha256" + "encoding/hex" "encoding/json" + "fmt" "sync" "time" @@ -12,6 +15,8 @@ import ( "github.com/multiformats/go-multiaddr" "github.com/pborman/uuid" + "github.com/waku-org/go-waku/waku/v2/api/history" + "github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/p2p/enode" "github.com/status-im/status-go/connection" @@ -176,9 +181,6 @@ type Waku interface { Unsubscribe(ctx context.Context, id string) error UnsubscribeMany(ids []string) error - // RequestStoreMessages uses the WAKU2-STORE protocol to request historic messages - RequestStoreMessages(ctx context.Context, peerID peer.ID, request MessagesRequest, processEnvelopes bool) (StoreRequestCursor, int, error) - // ProcessingP2PMessages indicates whether there are in-flight p2p messages ProcessingP2PMessages() bool @@ -194,12 +196,60 @@ type Waku interface { // ConfirmMessageDelivered updates a message has been delivered in waku ConfirmMessageDelivered(hash []common.Hash) - // SetStorePeerID updates the peer id of store node - SetStorePeerID(peerID peer.ID) - // PeerID returns node's PeerID PeerID() peer.ID - // PingPeer returns the reply time - PingPeer(ctx context.Context, peerID peer.ID) (time.Duration, error) + // GetActiveStorenode returns the peer ID of the currently active storenode. It will be empty if no storenode is active + GetActiveStorenode() peer.ID + + // OnStorenodeAvailableOneShot returns a channel that will be triggered only once when a storenode becomes available + OnStorenodeAvailableOneShot() <-chan struct{} + + // OnStorenodeChanged is triggered when a new storenode is promoted to become the active storenode or when the active storenode is removed + OnStorenodeChanged() <-chan peer.ID + + // OnStorenodeNotWorking is triggered when the last active storenode fails to return results consistently + OnStorenodeNotWorking() <-chan struct{} + + // OnStorenodeAvailable is triggered when there is a new active storenode selected + OnStorenodeAvailable() <-chan peer.ID + + // WaitForAvailableStoreNode will wait for a storenode to be available until `timeout` happens + WaitForAvailableStoreNode(timeout time.Duration) bool + + // SetStorenodeConfigProvider will set the configuration provider for the storenode cycle + SetStorenodeConfigProvider(c history.StorenodeConfigProvider) + + // ProcessMailserverBatch will receive a criteria and storenode and execute a query + ProcessMailserverBatch( + ctx context.Context, + batch MailserverBatch, + storenodeID peer.ID, + pageLimit uint64, + shouldProcessNextPage func(int) (bool, uint64), + processEnvelopes bool, + ) error + + // IsStorenodeAvailable is used to determine whether a storenode is available or not + IsStorenodeAvailable(peerID peer.ID) bool + + PerformStorenodeTask(fn func() error, opts ...history.StorenodeTaskOption) error + + // DisconnectActiveStorenode will trigger a disconnection of the active storenode, and potentially execute a cycling so a new storenode is promoted + DisconnectActiveStorenode(ctx context.Context, backoff time.Duration, shouldCycle bool) +} + +type MailserverBatch struct { + From uint32 + To uint32 + Cursor string + PubsubTopic string + Topics []TopicType + ChatIDs []string +} + +func (mb *MailserverBatch) Hash() string { + data := fmt.Sprintf("%d%d%s%s%v%v", mb.From, mb.To, mb.Cursor, mb.PubsubTopic, mb.Topics, mb.ChatIDs) + hash := sha256.Sum256([]byte(data)) + return hex.EncodeToString(hash[:4]) } diff --git a/go.mod b/go.mod index b3abfac22bc..e6107a66175 100644 --- a/go.mod +++ b/go.mod @@ -95,7 +95,7 @@ require ( github.com/schollz/peerdiscovery v1.7.0 github.com/siphiuel/lc-proxy-wrapper v0.0.0-20230516150924-246507cee8c7 github.com/urfave/cli/v2 v2.27.2 - github.com/waku-org/go-waku v0.8.1-0.20241004054019-0ed94ce0b1cb + github.com/waku-org/go-waku v0.8.1-0.20241014185851-76275f6fb835 github.com/wk8/go-ordered-map/v2 v2.1.7 github.com/yeqown/go-qrcode/v2 v2.2.1 github.com/yeqown/go-qrcode/writer/standard v1.2.1 diff --git a/go.sum b/go.sum index 9112614af33..4fefc668975 100644 --- a/go.sum +++ b/go.sum @@ -2136,8 +2136,8 @@ github.com/waku-org/go-libp2p-pubsub v0.12.0-gowaku.0.20240823143342-b0f2429ca27 github.com/waku-org/go-libp2p-pubsub v0.12.0-gowaku.0.20240823143342-b0f2429ca27f/go.mod h1:Oi0zw9aw8/Y5GC99zt+Ef2gYAl+0nZlwdJonDyOz/sE= github.com/waku-org/go-libp2p-rendezvous v0.0.0-20240110193335-a67d1cc760a0 h1:R4YYx2QamhBRl/moIxkDCNW+OP7AHbyWLBygDc/xIMo= github.com/waku-org/go-libp2p-rendezvous v0.0.0-20240110193335-a67d1cc760a0/go.mod h1:EhZP9fee0DYjKH/IOQvoNSy1tSHp2iZadsHGphcAJgY= -github.com/waku-org/go-waku v0.8.1-0.20241004054019-0ed94ce0b1cb h1:E3J49PH9iXpjaOOI/VrEX/VhSk3obKjxVehGEDzZgXI= -github.com/waku-org/go-waku v0.8.1-0.20241004054019-0ed94ce0b1cb/go.mod h1:1BRnyg2mQ2aBNLTBaPq6vEvobzywGykPOhGQFbHGf74= +github.com/waku-org/go-waku v0.8.1-0.20241014185851-76275f6fb835 h1:Vp6BhXiDEilmchHy8OLMZVhugudsnvveNkAKD5nhAGk= +github.com/waku-org/go-waku v0.8.1-0.20241014185851-76275f6fb835/go.mod h1:1BRnyg2mQ2aBNLTBaPq6vEvobzywGykPOhGQFbHGf74= github.com/waku-org/go-zerokit-rln v0.1.14-0.20240102145250-fa738c0bdf59 h1:jisj+OCI6QydLtFq3Pyhu49wl9ytPN7oAHjMfepHDrA= github.com/waku-org/go-zerokit-rln v0.1.14-0.20240102145250-fa738c0bdf59/go.mod h1:1PdBdPzyTaKt3VnpAHk3zj+r9dXPFOr3IHZP9nFle6E= github.com/waku-org/go-zerokit-rln-apple v0.0.0-20230916172309-ee0ee61dde2b h1:KgZVhsLkxsj5gb/FfndSCQu6VYwALrCOgYI3poR95yE= diff --git a/protocol/messenger.go b/protocol/messenger.go index 7a5aad832db..7ea39f012c7 100644 --- a/protocol/messenger.go +++ b/protocol/messenger.go @@ -139,7 +139,6 @@ type Messenger struct { allInstallations *installationMap modifiedInstallations *stringBoolMap installationID string - mailserverCycle mailserverCycle communityStorenodes *storenodes.CommunityStorenodes database *sql.DB multiAccounts *multiaccounts.Database @@ -172,7 +171,6 @@ type Messenger struct { // TODO(samyoul) Determine if/how the remaining usage of this mutex can be removed mutex sync.Mutex - mailPeersMutex sync.RWMutex handleMessagesMutex sync.Mutex handleImportMessagesMutex sync.Mutex @@ -199,50 +197,6 @@ type Messenger struct { mvdsStatusChangeEvent chan datasyncnode.PeerStatusChangeEvent } -type connStatus int - -const ( - disconnected connStatus = iota + 1 - connected -) - -type peerStatus struct { - status connStatus - canConnectAfter time.Time - lastConnectionAttempt time.Time - mailserver mailserversDB.Mailserver -} -type mailserverCycle struct { - sync.RWMutex - allMailservers []mailserversDB.Mailserver - activeMailserver *mailserversDB.Mailserver - peers map[string]peerStatus - availabilitySubscriptions *availabilitySubscriptions -} - -type availabilitySubscriptions struct { - sync.Mutex - subscriptions []chan struct{} -} - -func (s *availabilitySubscriptions) Subscribe() <-chan struct{} { - s.Lock() - defer s.Unlock() - c := make(chan struct{}) - s.subscriptions = append(s.subscriptions, c) - return c -} - -func (s *availabilitySubscriptions) EmitMailserverAvailable() { - s.Lock() - defer s.Unlock() - - for _, subs := range s.subscriptions { - close(subs) - } - s.subscriptions = nil -} - type EnvelopeEventsInterceptor struct { EnvelopeEventsHandler transport.EnvelopeEventsHandler Messenger *Messenger @@ -624,19 +578,15 @@ func NewMessenger( peerStore: peerStore, mvdsStatusChangeEvent: make(chan datasyncnode.PeerStatusChangeEvent, 5), verificationDatabase: verification.NewPersistence(database), - mailserverCycle: mailserverCycle{ - peers: make(map[string]peerStatus), - availabilitySubscriptions: &availabilitySubscriptions{}, - }, - mailserversDatabase: c.mailserversDatabase, - communityStorenodes: storenodes.NewCommunityStorenodes(storenodes.NewDB(database), logger), - account: c.account, - quit: make(chan struct{}), - ctx: ctx, - cancel: cancel, - importingCommunities: make(map[string]bool), - importingChannels: make(map[string]bool), - importRateLimiter: rate.NewLimiter(rate.Every(importSlowRate), 1), + mailserversDatabase: c.mailserversDatabase, + communityStorenodes: storenodes.NewCommunityStorenodes(storenodes.NewDB(database), logger), + account: c.account, + quit: make(chan struct{}), + ctx: ctx, + cancel: cancel, + importingCommunities: make(map[string]bool), + importingChannels: make(map[string]bool), + importRateLimiter: rate.NewLimiter(rate.Every(importSlowRate), 1), importDelayer: struct { wait chan struct{} once sync.Once @@ -883,22 +833,26 @@ func (m *Messenger) Start() (*MessengerResponse, error) { } response := &MessengerResponse{} - mailservers, err := m.allMailservers() + storenodes, err := m.AllMailservers() if err != nil { return nil, err } - response.Mailservers = mailservers - err = m.StartMailserverCycle(mailservers) + err = m.setupStorenodes(storenodes) if err != nil { return nil, err } + response.Mailservers = storenodes + + m.transport.SetStorenodeConfigProvider(m) + if err := m.communityStorenodes.ReloadFromDB(); err != nil { return nil, err } go m.checkForMissingMessagesLoop() + go m.checkForStorenodeCycleSignals() controlledCommunities, err := m.communitiesManager.Controlled() if err != nil { @@ -906,10 +860,9 @@ func (m *Messenger) Start() (*MessengerResponse, error) { } if m.archiveManager.IsReady() { - available := m.mailserverCycle.availabilitySubscriptions.Subscribe() go func() { defer gocommon.LogOnPanic() - <-available + <-m.transport.OnStorenodeAvailableOneShot() m.InitHistoryArchiveTasks(controlledCommunities) }() } diff --git a/protocol/messenger_communities.go b/protocol/messenger_communities.go index ad70388589a..9d7ad082b91 100644 --- a/protocol/messenger_communities.go +++ b/protocol/messenger_communities.go @@ -16,6 +16,7 @@ import ( "github.com/golang/protobuf/proto" "github.com/google/uuid" + "github.com/libp2p/go-libp2p/core/peer" gethcommon "github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/common/hexutil" @@ -40,6 +41,7 @@ import ( "github.com/status-im/status-go/protocol/encryption" "github.com/status-im/status-go/protocol/protobuf" "github.com/status-im/status-go/protocol/requests" + "github.com/status-im/status-go/protocol/storenodes" "github.com/status-im/status-go/protocol/transport" v1protocol "github.com/status-im/status-go/protocol/v1" localnotifications "github.com/status-im/status-go/services/local-notifications" @@ -3970,8 +3972,8 @@ func (m *Messenger) InitHistoryArchiveTasks(communities []*communities.Community } // Request possibly missed waku messages for community - ms := m.getActiveMailserver(c.ID().String()) - _, err = m.syncFiltersFrom(*ms, filters, uint32(latestWakuMessageTimestamp)) + ms := m.getCommunityMailserver(c.ID().String()) + _, err = m.syncFiltersFrom(ms, filters, uint32(latestWakuMessageTimestamp)) if err != nil { m.logger.Error("failed to request missing messages", zap.Error(err)) continue @@ -5155,3 +5157,28 @@ func (m *Messenger) startRequestMissingCommunityChannelsHRKeysLoop() { } }() } + +// getCommunityMailserver returns the active mailserver if a communityID is present then it'll return the mailserver +// for that community if it has a mailserver setup otherwise it'll return the global mailserver +func (m *Messenger) getCommunityMailserver(communityID ...string) peer.ID { + if m.transport.WakuVersion() != 2 { + return "" + } + + if len(communityID) == 0 || communityID[0] == "" { + return m.transport.GetActiveStorenode() + } + + ms, err := m.communityStorenodes.GetStorenodeByCommunityID(communityID[0]) + if err != nil { + if !errors.Is(err, storenodes.ErrNotFound) { + m.logger.Error("getting storenode for community, using global", zap.String("communityID", communityID[0]), zap.Error(err)) + } + // if we don't find a specific mailserver for the community, we just use the regular mailserverCycle's one + return m.transport.GetActiveStorenode() + } + + peerID, _ := ms.PeerID() + + return peerID +} diff --git a/protocol/messenger_mailserver.go b/protocol/messenger_mailserver.go index 4af9129f28e..9bb7b7fefc7 100644 --- a/protocol/messenger_mailserver.go +++ b/protocol/messenger_mailserver.go @@ -1,19 +1,17 @@ package protocol import ( - "context" - "crypto/sha256" - "encoding/hex" "fmt" "math" "sort" - "sync" "time" "github.com/libp2p/go-libp2p/core/peer" "github.com/pkg/errors" "go.uber.org/zap" + "github.com/waku-org/go-waku/waku/v2/api/history" + gocommon "github.com/status-im/status-go/common" "github.com/status-im/status-go/connection" "github.com/status-im/status-go/eth-node/crypto" @@ -31,22 +29,21 @@ const ( // tolerance is how many seconds of potentially out-of-order messages we want to fetch tolerance uint32 = 60 - mailserverRequestTimeout = 30 * time.Second - mailserverMaxTries uint = 2 - mailserverMaxFailedRequests uint = 2 - oneDayDuration = 24 * time.Hour oneMonthDuration = 31 * oneDayDuration -) -// maxTopicsPerRequest sets the batch size to limit the number of topics per store query -var maxTopicsPerRequest int = 10 + backoffByUserAction = 0 * time.Second +) var ErrNoFiltersForChat = errors.New("no filter registered for given chat") func (m *Messenger) shouldSync() (bool, error) { + if m.transport.WakuVersion() != 2 { + return false, nil + } + // TODO (pablo) support community store node as well - if m.mailserverCycle.activeMailserver == nil || !m.Online() { + if m.transport.GetActiveStorenode() == "" || !m.Online() { return false, nil } @@ -72,9 +69,9 @@ func (m *Messenger) scheduleSyncChat(chat *Chat) (bool, error) { go func() { defer gocommon.LogOnPanic() - ms := m.getActiveMailserver(chat.CommunityID) - _, err = m.performMailserverRequest(ms, func(mailServer mailservers.Mailserver) (*MessengerResponse, error) { - response, err := m.syncChatWithFilters(mailServer, chat.ID) + peerID := m.getCommunityMailserver(chat.CommunityID) + _, err = m.performStorenodeTask(func() (*MessengerResponse, error) { + response, err := m.syncChatWithFilters(peerID, chat.ID) if err != nil { m.logger.Error("failed to sync chat", zap.Error(err)) @@ -85,7 +82,7 @@ func (m *Messenger) scheduleSyncChat(chat *Chat) (bool, error) { m.config.messengerSignalsHandler.MessengerResponse(response) } return response, nil - }) + }, history.WithPeerID(peerID)) if err != nil { m.logger.Error("failed to perform mailserver request", zap.Error(err)) } @@ -93,65 +90,41 @@ func (m *Messenger) scheduleSyncChat(chat *Chat) (bool, error) { return true, nil } -func (m *Messenger) connectToNewMailserverAndWait() error { - // Handle pinned mailservers - m.logger.Info("disconnecting mailserver") - pinnedMailserver, err := m.getPinnedMailserver() - if err != nil { - m.logger.Error("could not obtain the pinned mailserver", zap.Error(err)) - return err - } - // If pinned mailserver is not nil, no need to disconnect and wait for it to be available - if pinnedMailserver == nil { - m.disconnectActiveMailserver(graylistBackoff) - } +func (m *Messenger) performStorenodeTask(task func() (*MessengerResponse, error), opts ...history.StorenodeTaskOption) (*MessengerResponse, error) { + responseCh := make(chan *MessengerResponse) + errCh := make(chan error) - return m.findNewMailserver() -} + go func() { + err := m.transport.PerformStorenodeTask(func() error { + r, err := task() + if err != nil { + return err + } -func (m *Messenger) performMailserverRequest(ms *mailservers.Mailserver, fn func(mailServer mailservers.Mailserver) (*MessengerResponse, error)) (*MessengerResponse, error) { - if ms == nil { - return nil, errors.New("mailserver not available") - } + select { + case responseCh <- r: + default: + // + } - m.mailserverCycle.RLock() - defer m.mailserverCycle.RUnlock() - var tries uint = 0 - for tries < mailserverMaxTries { - if !m.communityStorenodes.IsCommunityStoreNode(ms.ID) && !m.isMailserverAvailable(ms.ID) { - return nil, errors.New("storenode not available") - } - m.logger.Info("trying performing mailserver requests", zap.Uint("try", tries), zap.String("mailserverID", ms.ID)) - - // Peform request - response, err := fn(*ms) // pass by value because we don't want the fn to modify the mailserver - if err == nil { - // Reset failed requests - m.logger.Debug("mailserver request performed successfully", - zap.String("mailserverID", ms.ID)) - ms.FailedRequests = 0 - return response, nil + return nil + }, opts...) + if err != nil { + errCh <- err } + }() - m.logger.Error("failed to perform mailserver request", - zap.String("mailserverID", ms.ID), - zap.Uint("tries", tries), - zap.Error(err), - ) - - tries++ - // Increment failed requests - ms.FailedRequests++ - - // Change mailserver - if ms.FailedRequests >= mailserverMaxFailedRequests { - return nil, errors.New("too many failed requests") + select { + case err := <-errCh: + return nil, err + case r := <-responseCh: + if r != nil { + return r, nil } - // Wait a couple of second not to spam - time.Sleep(2 * time.Second) - + return nil, errors.New("no response available") + case <-m.ctx.Done(): + return nil, m.ctx.Err() } - return nil, errors.New("failed to perform mailserver request") } func (m *Messenger) scheduleSyncFilters(filters []*transport.Filter) (bool, error) { @@ -170,9 +143,9 @@ func (m *Messenger) scheduleSyncFilters(filters []*transport.Filter) (bool, erro // split filters by community store node so we can request the filters to the correct mailserver filtersByMs := m.SplitFiltersByStoreNode(filters) for communityID, filtersForMs := range filtersByMs { - ms := m.getActiveMailserver(communityID) - _, err := m.performMailserverRequest(ms, func(ms mailservers.Mailserver) (*MessengerResponse, error) { - response, err := m.syncFilters(ms, filtersForMs) + peerID := m.getCommunityMailserver(communityID) + _, err := m.performStorenodeTask(func() (*MessengerResponse, error) { + response, err := m.syncFilters(peerID, filtersForMs) if err != nil { m.logger.Error("failed to sync filter", zap.Error(err)) @@ -183,7 +156,7 @@ func (m *Messenger) scheduleSyncFilters(filters []*transport.Filter) (bool, erro m.config.messengerSignalsHandler.MessengerResponse(response) } return response, nil - }) + }, history.WithPeerID(peerID)) if err != nil { m.logger.Error("failed to perform mailserver request", zap.Error(err)) } @@ -252,13 +225,13 @@ func (m *Messenger) topicsForChat(chatID string) (string, []types.TopicType, err return filters[0].PubsubTopic, contentTopics, nil } -func (m *Messenger) syncChatWithFilters(ms mailservers.Mailserver, chatID string) (*MessengerResponse, error) { +func (m *Messenger) syncChatWithFilters(peerID peer.ID, chatID string) (*MessengerResponse, error) { filters, err := m.filtersForChat(chatID) if err != nil { return nil, err } - return m.syncFilters(ms, filters) + return m.syncFilters(peerID, filters) } func (m *Messenger) syncBackup() error { @@ -277,9 +250,9 @@ func (m *Messenger) syncBackup() error { from, to := m.calculateMailserverTimeBounds(oneMonthDuration) - batch := MailserverBatch{From: from, To: to, Topics: []types.TopicType{filter.ContentTopic}} - ms := m.getActiveMailserver(filter.ChatID) - err = m.processMailserverBatch(*ms, batch) + batch := types.MailserverBatch{From: from, To: to, Topics: []types.TopicType{filter.ContentTopic}} + ms := m.getCommunityMailserver(filter.ChatID) + err = m.processMailserverBatch(ms, batch) if err != nil { return err } @@ -374,11 +347,11 @@ func (m *Messenger) RequestAllHistoricMessages(forceFetchingBackup, withRetries filtersByMs := m.SplitFiltersByStoreNode(filters) allResponses := &MessengerResponse{} for communityID, filtersForMs := range filtersByMs { - ms := m.getActiveMailserver(communityID) + peerID := m.getCommunityMailserver(communityID) if withRetries { - response, err := m.performMailserverRequest(ms, func(ms mailservers.Mailserver) (*MessengerResponse, error) { - return m.syncFilters(ms, filtersForMs) - }) + response, err := m.performStorenodeTask(func() (*MessengerResponse, error) { + return m.syncFilters(peerID, filtersForMs) + }, history.WithPeerID(peerID)) if err != nil { return nil, err } @@ -388,7 +361,7 @@ func (m *Messenger) RequestAllHistoricMessages(forceFetchingBackup, withRetries } continue } - response, err := m.syncFilters(*ms, filtersForMs) + response, err := m.syncFilters(peerID, filtersForMs) if err != nil { return nil, err } @@ -404,10 +377,15 @@ const missingMessageCheckPeriod = 30 * time.Second func (m *Messenger) checkForMissingMessagesLoop() { defer gocommon.LogOnPanic() + + if m.transport.WakuVersion() != 2 { + return + } + t := time.NewTicker(missingMessageCheckPeriod) defer t.Stop() - mailserverAvailableSignal := m.mailserverCycle.availabilitySubscriptions.Subscribe() + mailserverAvailableSignal := m.transport.OnStorenodeAvailable() for { select { @@ -416,7 +394,6 @@ func (m *Messenger) checkForMissingMessagesLoop() { // Wait for mailserver available, also triggered on mailserver change case <-mailserverAvailableSignal: - mailserverAvailableSignal = m.mailserverCycle.availabilitySubscriptions.Subscribe() case <-t.C: @@ -425,16 +402,11 @@ func (m *Messenger) checkForMissingMessagesLoop() { filters := m.transport.Filters() filtersByMs := m.SplitFiltersByStoreNode(filters) for communityID, filtersForMs := range filtersByMs { - ms := m.getActiveMailserver(communityID) - if ms == nil { + peerID := m.getCommunityMailserver(communityID) + if peerID == "" { continue } - peerID, err := ms.PeerID() - if err != nil { - m.logger.Error("could not obtain the peerID") - return - } m.transport.SetCriteriaForMissingMessageVerification(peerID, filtersForMs) } } @@ -444,7 +416,7 @@ func getPrioritizedBatches() []int { return []int{1, 5, 10} } -func (m *Messenger) syncFiltersFrom(ms mailservers.Mailserver, filters []*transport.Filter, lastRequest uint32) (*MessengerResponse, error) { +func (m *Messenger) syncFiltersFrom(peerID peer.ID, filters []*transport.Filter, lastRequest uint32) (*MessengerResponse, error) { canSync, err := m.canSyncWithStoreNodes() if err != nil { return nil, err @@ -464,7 +436,7 @@ func (m *Messenger) syncFiltersFrom(ms mailservers.Mailserver, filters []*transp topicsData[fmt.Sprintf("%s-%s", topic.PubsubTopic, topic.ContentTopic)] = topic } - batches := make(map[string]map[int]MailserverBatch) + batches := make(map[string]map[int]types.MailserverBatch) to := m.calculateMailserverTo() var syncedTopics []mailservers.MailserverTopic @@ -502,7 +474,7 @@ func (m *Messenger) syncFiltersFrom(ms mailservers.Mailserver, filters []*transp for pubsubTopic, contentTopics := range contentTopicsPerPubsubTopic { if _, ok := batches[pubsubTopic]; !ok { - batches[pubsubTopic] = make(map[int]MailserverBatch) + batches[pubsubTopic] = make(map[int]types.MailserverBatch) } for _, filter := range contentTopics { @@ -561,7 +533,7 @@ func (m *Messenger) syncFiltersFrom(ms mailservers.Mailserver, filters []*transp return nil, err } } - batch = MailserverBatch{From: from, To: to} + batch = types.MailserverBatch{From: from, To: to} } batch.ChatIDs = append(batch.ChatIDs, chatID) @@ -579,7 +551,7 @@ func (m *Messenger) syncFiltersFrom(ms mailservers.Mailserver, filters []*transp m.config.messengerSignalsHandler.HistoryRequestStarted(len(batches)) } - var batches24h []MailserverBatch + var batches24h []types.MailserverBatch for pubsubTopic := range batches { batchKeys := make([]int, 0, len(batches[pubsubTopic])) for k := range batches[pubsubTopic] { @@ -594,7 +566,7 @@ func (m *Messenger) syncFiltersFrom(ms mailservers.Mailserver, filters []*transp for _, k := range keysToIterate { batch := batches[pubsubTopic][k] - dayBatch := MailserverBatch{ + dayBatch := types.MailserverBatch{ To: batch.To, Cursor: batch.Cursor, PubsubTopic: batch.PubsubTopic, @@ -624,7 +596,7 @@ func (m *Messenger) syncFiltersFrom(ms mailservers.Mailserver, filters []*transp } for _, batch := range batches24h { - err := m.processMailserverBatch(ms, batch) + err := m.processMailserverBatch(peerID, batch) if err != nil { m.logger.Error("error syncing topics", zap.Error(err)) return nil, err @@ -682,8 +654,8 @@ func (m *Messenger) syncFiltersFrom(ms mailservers.Mailserver, filters []*transp return response, nil } -func (m *Messenger) syncFilters(ms mailservers.Mailserver, filters []*transport.Filter) (*MessengerResponse, error) { - return m.syncFiltersFrom(ms, filters, 0) +func (m *Messenger) syncFilters(peerID peer.ID, filters []*transport.Filter) (*MessengerResponse, error) { + return m.syncFiltersFrom(peerID, filters, 0) } func (m *Messenger) calculateGapForChat(chat *Chat, from uint32) (*common.Message, error) { @@ -722,187 +694,6 @@ func (m *Messenger) calculateGapForChat(chat *Chat, from uint32) (*common.Messag return message, m.persistence.SaveMessages([]*common.Message{message}) } -type work struct { - pubsubTopic string - contentTopics []types.TopicType - cursor types.StoreRequestCursor - limit uint32 -} - -type messageRequester interface { - SendMessagesRequestForTopics( - ctx context.Context, - peerID peer.ID, - from, to uint32, - previousStoreCursor types.StoreRequestCursor, - pubsubTopic string, - contentTopics []types.TopicType, - limit uint32, - waitForResponse bool, - processEnvelopes bool, - ) (cursor types.StoreRequestCursor, envelopesCount int, err error) -} - -func processMailserverBatch( - ctx context.Context, - messageRequester messageRequester, - batch MailserverBatch, - storenodeID peer.ID, - logger *zap.Logger, - pageLimit uint32, - shouldProcessNextPage func(int) (bool, uint32), - processEnvelopes bool, -) error { - - var topicStrings []string - for _, t := range batch.Topics { - topicStrings = append(topicStrings, t.String()) - } - logger = logger.With(zap.String("batch hash", batch.Hash())) - logger.Info("syncing topic", - zap.Any("chatIDs", batch.ChatIDs), - zap.String("fromString", time.Unix(int64(batch.From), 0).Format(time.RFC3339)), - zap.String("toString", time.Unix(int64(batch.To), 0).Format(time.RFC3339)), - zap.Any("topic", topicStrings), - zap.Int64("from", int64(batch.From)), - zap.Int64("to", int64(batch.To))) - - wg := sync.WaitGroup{} - workWg := sync.WaitGroup{} - workCh := make(chan work, 1000) // each batch item is split in 10 topics bunch and sent to this channel - workCompleteCh := make(chan struct{}) // once all batch items are processed, this channel is triggered - semaphore := make(chan int, 3) // limit the number of concurrent queries - errCh := make(chan error) - - ctx, cancel := context.WithCancel(ctx) - defer cancel() - - // Producer - wg.Add(1) - go func() { - defer gocommon.LogOnPanic() - defer func() { - logger.Debug("mailserver batch producer complete") - wg.Done() - }() - - allWorks := int(math.Ceil(float64(len(batch.Topics)) / float64(maxTopicsPerRequest))) - workWg.Add(allWorks) - - for i := 0; i < len(batch.Topics); i += maxTopicsPerRequest { - j := i + maxTopicsPerRequest - if j > len(batch.Topics) { - j = len(batch.Topics) - } - - select { - case <-ctx.Done(): - logger.Debug("processBatch producer - context done") - return - default: - logger.Debug("processBatch producer - creating work") - workCh <- work{ - pubsubTopic: batch.PubsubTopic, - contentTopics: batch.Topics[i:j], - limit: pageLimit, - } - time.Sleep(50 * time.Millisecond) - } - } - - go func() { - defer gocommon.LogOnPanic() - workWg.Wait() - workCompleteCh <- struct{}{} - }() - - logger.Debug("processBatch producer complete") - }() - - var result error - -loop: - for { - select { - case <-ctx.Done(): - logger.Debug("processBatch cleanup - context done") - result = ctx.Err() - if errors.Is(result, context.Canceled) { - result = nil - } - break loop - case w, ok := <-workCh: - if !ok { - continue - } - - logger.Debug("processBatch - received work") - semaphore <- 1 - go func(w work) { // Consumer - defer gocommon.LogOnPanic() - defer func() { - workWg.Done() - <-semaphore - }() - - queryCtx, queryCancel := context.WithTimeout(ctx, mailserverRequestTimeout) - cursor, envelopesCount, err := messageRequester.SendMessagesRequestForTopics(queryCtx, storenodeID, batch.From, batch.To, w.cursor, w.pubsubTopic, w.contentTopics, w.limit, true, processEnvelopes) - queryCancel() - - if err != nil { - logger.Debug("failed to send request", zap.Error(err)) - errCh <- err - return - } - - processNextPage := true - nextPageLimit := pageLimit - - if shouldProcessNextPage != nil { - processNextPage, nextPageLimit = shouldProcessNextPage(envelopesCount) - } - - if !processNextPage { - return - } - - // Check the cursor after calling `shouldProcessNextPage`. - // The app might use process the fetched envelopes in the callback for own needs. - if cursor == nil { - return - } - - logger.Debug("processBatch producer - creating work (cursor)") - - workWg.Add(1) - workCh <- work{ - pubsubTopic: w.pubsubTopic, - contentTopics: w.contentTopics, - cursor: cursor, - limit: nextPageLimit, - } - }(w) - case err := <-errCh: - logger.Debug("processBatch - received error", zap.Error(err)) - cancel() // Kill go routines - return err - case <-workCompleteCh: - logger.Debug("processBatch - all jobs complete") - cancel() // Kill go routines - } - } - - wg.Wait() - - // NOTE(camellos): Disabling for now, not critical and I'd rather take a bit more time - // to test it - //logger.Info("waiting until message processed") - //m.waitUntilP2PMessagesProcessed() - - logger.Info("synced topic", zap.NamedError("hasError", result)) - return result -} - func (m *Messenger) canSyncWithStoreNodes() (bool, error) { if m.featureFlags.StoreNodesDisabled { return false, nil @@ -918,7 +709,7 @@ func (m *Messenger) DisableStoreNodes() { m.featureFlags.StoreNodesDisabled = true } -func (m *Messenger) processMailserverBatch(ms mailservers.Mailserver, batch MailserverBatch) error { +func (m *Messenger) processMailserverBatch(peerID peer.ID, batch types.MailserverBatch) error { canSync, err := m.canSyncWithStoreNodes() if err != nil { return err @@ -927,15 +718,10 @@ func (m *Messenger) processMailserverBatch(ms mailservers.Mailserver, batch Mail return nil } - mailserverID, err := ms.PeerID() - if err != nil { - return err - } - logger := m.logger.With(zap.String("mailserverID", ms.ID)) - return processMailserverBatch(m.ctx, m.transport, batch, mailserverID, logger, defaultStoreNodeRequestPageSize, nil, false) + return m.transport.ProcessMailserverBatch(m.ctx, batch, peerID, defaultStoreNodeRequestPageSize, nil, false) } -func (m *Messenger) processMailserverBatchWithOptions(ms mailservers.Mailserver, batch MailserverBatch, pageLimit uint32, shouldProcessNextPage func(int) (bool, uint32), processEnvelopes bool) error { +func (m *Messenger) processMailserverBatchWithOptions(peerID peer.ID, batch types.MailserverBatch, pageLimit uint64, shouldProcessNextPage func(int) (bool, uint64), processEnvelopes bool) error { canSync, err := m.canSyncWithStoreNodes() if err != nil { return err @@ -944,27 +730,7 @@ func (m *Messenger) processMailserverBatchWithOptions(ms mailservers.Mailserver, return nil } - mailserverID, err := ms.PeerID() - if err != nil { - return err - } - logger := m.logger.With(zap.String("mailserverID", ms.ID)) - return processMailserverBatch(m.ctx, m.transport, batch, mailserverID, logger, pageLimit, shouldProcessNextPage, processEnvelopes) -} - -type MailserverBatch struct { - From uint32 - To uint32 - Cursor string - PubsubTopic string - Topics []types.TopicType - ChatIDs []string -} - -func (mb *MailserverBatch) Hash() string { - data := fmt.Sprintf("%d%d%s%s%v%v", mb.From, mb.To, mb.Cursor, mb.PubsubTopic, mb.Topics, mb.ChatIDs) - hash := sha256.Sum256([]byte(data)) - return hex.EncodeToString(hash[:4]) + return m.transport.ProcessMailserverBatch(m.ctx, batch, peerID, pageLimit, shouldProcessNextPage, processEnvelopes) } func (m *Messenger) SyncChatFromSyncedFrom(chatID string) (uint32, error) { @@ -973,9 +739,9 @@ func (m *Messenger) SyncChatFromSyncedFrom(chatID string) (uint32, error) { return 0, ErrChatNotFound } - ms := m.getActiveMailserver(chat.CommunityID) + peerID := m.getCommunityMailserver(chat.CommunityID) var from uint32 - _, err := m.performMailserverRequest(ms, func(ms mailservers.Mailserver) (*MessengerResponse, error) { + _, err := m.performStorenodeTask(func() (*MessengerResponse, error) { canSync, err := m.canSyncWithStoreNodes() if err != nil { return nil, err @@ -994,7 +760,7 @@ func (m *Messenger) SyncChatFromSyncedFrom(chatID string) (uint32, error) { return nil, err } - batch := MailserverBatch{ + batch := types.MailserverBatch{ ChatIDs: []string{chatID}, To: chat.SyncedFrom, From: chat.SyncedFrom - defaultSyncPeriod, @@ -1005,7 +771,7 @@ func (m *Messenger) SyncChatFromSyncedFrom(chatID string) (uint32, error) { m.config.messengerSignalsHandler.HistoryRequestStarted(1) } - err = m.processMailserverBatch(ms, batch) + err = m.processMailserverBatch(peerID, batch) if err != nil { return nil, err } @@ -1022,7 +788,7 @@ func (m *Messenger) SyncChatFromSyncedFrom(chatID string) (uint32, error) { err = m.persistence.SetSyncTimestamps(batch.From, chat.SyncedTo, chat.ID) from = batch.From return nil, err - }) + }, history.WithPeerID(peerID)) if err != nil { return 0, err } @@ -1062,7 +828,7 @@ func (m *Messenger) FillGaps(chatID string, messageIDs []string) error { } } - batch := MailserverBatch{ + batch := types.MailserverBatch{ ChatIDs: []string{chatID}, To: highestTo, From: lowestFrom, @@ -1074,8 +840,8 @@ func (m *Messenger) FillGaps(chatID string, messageIDs []string) error { m.config.messengerSignalsHandler.HistoryRequestStarted(1) } - ms := m.getActiveMailserver(chat.CommunityID) - err = m.processMailserverBatch(*ms, batch) + peerID := m.getCommunityMailserver(chat.CommunityID) + err = m.processMailserverBatch(peerID, batch) if err != nil { return err } @@ -1087,39 +853,18 @@ func (m *Messenger) FillGaps(chatID string, messageIDs []string) error { return m.persistence.DeleteMessages(messageIDs) } -func (m *Messenger) waitUntilP2PMessagesProcessed() { // nolint: unused - - ticker := time.NewTicker(50 * time.Millisecond) - - for { //nolint: gosimple - select { - case <-ticker.C: - if !m.transport.ProcessingP2PMessages() { - ticker.Stop() - return - } - } - } -} - func (m *Messenger) LoadFilters(filters []*transport.Filter) ([]*transport.Filter, error) { return m.transport.LoadFilters(filters) } func (m *Messenger) ToggleUseMailservers(value bool) error { - m.mailserverCycle.Lock() - defer m.mailserverCycle.Unlock() - err := m.settings.SetUseMailservers(value) if err != nil { return err } - m.disconnectActiveMailserver(backoffByUserAction) - if value { - m.cycleMailservers() - return nil - } + m.transport.DisconnectActiveStorenode(m.ctx, backoffByUserAction, value) + return nil } @@ -1129,8 +874,8 @@ func (m *Messenger) SetPinnedMailservers(mailservers map[string]string) error { return err } - m.disconnectActiveMailserver(backoffByUserAction) - m.cycleMailservers() + m.transport.DisconnectActiveStorenode(m.ctx, backoffByUserAction, true) + return nil } @@ -1162,8 +907,8 @@ func (m *Messenger) fetchMessages(chatID string, duration time.Duration) (uint32 return 0, ErrChatNotFound } - ms := m.getActiveMailserver(chat.CommunityID) - _, err := m.performMailserverRequest(ms, func(ms mailservers.Mailserver) (*MessengerResponse, error) { + peerID := m.getCommunityMailserver(chat.CommunityID) + _, err := m.performStorenodeTask(func() (*MessengerResponse, error) { canSync, err := m.canSyncWithStoreNodes() if err != nil { return nil, err @@ -1172,13 +917,13 @@ func (m *Messenger) fetchMessages(chatID string, duration time.Duration) (uint32 return nil, nil } - m.logger.Debug("fetching messages", zap.String("chatID", chatID), zap.String("mailserver", ms.Name)) + m.logger.Debug("fetching messages", zap.String("chatID", chatID), zap.Stringer("storenodeID", peerID)) pubsubTopic, topics, err := m.topicsForChat(chatID) if err != nil { return nil, nil } - batch := MailserverBatch{ + batch := types.MailserverBatch{ ChatIDs: []string{chatID}, From: from, To: to, @@ -1189,7 +934,7 @@ func (m *Messenger) fetchMessages(chatID string, duration time.Duration) (uint32 m.config.messengerSignalsHandler.HistoryRequestStarted(1) } - err = m.processMailserverBatch(ms, batch) + err = m.processMailserverBatch(peerID, batch) if err != nil { return nil, err } @@ -1206,7 +951,7 @@ func (m *Messenger) fetchMessages(chatID string, duration time.Duration) (uint32 err = m.persistence.SetSyncTimestamps(batch.From, chat.SyncedTo, chat.ID) from = batch.From return nil, err - }) + }, history.WithPeerID(peerID)) if err != nil { return 0, err } diff --git a/protocol/messenger_mailserver_cycle.go b/protocol/messenger_mailserver_cycle.go index c1a8b159104..befeee89fe1 100644 --- a/protocol/messenger_mailserver_cycle.go +++ b/protocol/messenger_mailserver_cycle.go @@ -1,178 +1,18 @@ package protocol import ( - "context" - "crypto/rand" - "math" - "math/big" - "net" - "runtime" - "sort" - "sync" - "time" - - "github.com/pkg/errors" + "github.com/libp2p/go-libp2p/core/peer" "go.uber.org/zap" "github.com/waku-org/go-waku/waku/v2/utils" - "github.com/status-im/status-go/common" gocommon "github.com/status-im/status-go/common" "github.com/status-im/status-go/params" - "github.com/status-im/status-go/protocol/storenodes" "github.com/status-im/status-go/services/mailservers" "github.com/status-im/status-go/signal" ) -const defaultBackoff = 10 * time.Second -const graylistBackoff = 3 * time.Minute -const backoffByUserAction = 0 -const isAndroidEmulator = runtime.GOOS == "android" && runtime.GOARCH == "amd64" -const findNearestMailServer = !isAndroidEmulator -const overrideDNS = runtime.GOOS == "android" || runtime.GOOS == "ios" -const bootstrapDNS = "8.8.8.8:53" - -type byRTTMsAndCanConnectBefore []SortedMailserver - -func (s byRTTMsAndCanConnectBefore) Len() int { - return len(s) -} - -func (s byRTTMsAndCanConnectBefore) Swap(i, j int) { - s[i], s[j] = s[j], s[i] -} - -func (s byRTTMsAndCanConnectBefore) Less(i, j int) bool { - // Slightly inaccurate as time sensitive sorting, but it does not matter so much - now := time.Now() - if s[i].CanConnectAfter.Before(now) && s[j].CanConnectAfter.Before(now) { - return s[i].RTT < s[j].RTT - } - return s[i].CanConnectAfter.Before(s[j].CanConnectAfter) -} - -func (m *Messenger) StartMailserverCycle(mailservers []mailservers.Mailserver) error { - if m.transport.WakuVersion() != 2 { - m.logger.Warn("not starting mailserver cycle: requires wakuv2") - return nil - } - - m.mailserverCycle.allMailservers = mailservers - - if len(mailservers) == 0 { - m.logger.Warn("not starting mailserver cycle: empty mailservers list") - return nil - } - - for _, storenode := range mailservers { - - peerInfo, err := storenode.PeerInfo() - if err != nil { - return err - } - - for _, addr := range utils.EncapsulatePeerID(peerInfo.ID, peerInfo.Addrs...) { - _, err := m.transport.AddStorePeer(addr) - if err != nil { - return err - } - } - } - go m.verifyStorenodeStatus() - - m.logger.Debug("starting mailserver cycle", - zap.Uint("WakuVersion", m.transport.WakuVersion()), - zap.Any("mailservers", mailservers), - ) - - return nil -} - -func (m *Messenger) DisconnectActiveMailserver() { - m.mailserverCycle.Lock() - defer m.mailserverCycle.Unlock() - m.disconnectActiveMailserver(graylistBackoff) -} - -func (m *Messenger) disconnectMailserver(backoffDuration time.Duration) error { - if m.mailserverCycle.activeMailserver == nil { - m.logger.Info("no active mailserver") - return nil - } - m.logger.Info("disconnecting active mailserver", zap.String("nodeID", m.mailserverCycle.activeMailserver.ID)) - m.mailPeersMutex.Lock() - pInfo, ok := m.mailserverCycle.peers[m.mailserverCycle.activeMailserver.ID] - if ok { - pInfo.status = disconnected - - pInfo.canConnectAfter = time.Now().Add(backoffDuration) - m.mailserverCycle.peers[m.mailserverCycle.activeMailserver.ID] = pInfo - } else { - m.mailserverCycle.peers[m.mailserverCycle.activeMailserver.ID] = peerStatus{ - status: disconnected, - mailserver: *m.mailserverCycle.activeMailserver, - canConnectAfter: time.Now().Add(backoffDuration), - } - } - m.mailPeersMutex.Unlock() - - m.mailserverCycle.activeMailserver = nil - return nil -} - -func (m *Messenger) disconnectActiveMailserver(backoffDuration time.Duration) { - err := m.disconnectMailserver(backoffDuration) - if err != nil { - m.logger.Error("failed to disconnect mailserver", zap.Error(err)) - } - signal.SendMailserverChanged(nil) -} - -func (m *Messenger) cycleMailservers() { - m.logger.Info("Automatically switching mailserver") - - if m.mailserverCycle.activeMailserver != nil { - m.disconnectActiveMailserver(graylistBackoff) - } - - useMailserver, err := m.settings.CanUseMailservers() - if err != nil { - m.logger.Error("failed to get use mailservers", zap.Error(err)) - return - } - - if !useMailserver { - m.logger.Info("Skipping mailserver search due to useMailserver being false") - return - } - - err = m.findNewMailserver() - if err != nil { - m.logger.Error("Error getting new mailserver", zap.Error(err)) - } -} - -func poolSize(fleetSize int) int { - return int(math.Ceil(float64(fleetSize) / 4)) -} - -func (m *Messenger) getFleet() (string, error) { - var fleet string - dbFleet, err := m.settings.GetFleet() - if err != nil { - return "", err - } - if dbFleet != "" { - fleet = dbFleet - } else if m.config.clusterConfig.Fleet != "" { - fleet = m.config.clusterConfig.Fleet - } else { - fleet = params.FleetStatusProd - } - return fleet, nil -} - -func (m *Messenger) allMailservers() ([]mailservers.Mailserver, error) { +func (m *Messenger) AllMailservers() ([]mailservers.Mailserver, error) { // Get configured fleet fleet, err := m.getFleet() if err != nil { @@ -199,221 +39,46 @@ func (m *Messenger) allMailservers() ([]mailservers.Mailserver, error) { return allMailservers, nil } -type SortedMailserver struct { - Mailserver mailservers.Mailserver - RTT time.Duration - CanConnectAfter time.Time -} - -func (m *Messenger) getAvailableMailserversSortedByRTT(allMailservers []mailservers.Mailserver) []mailservers.Mailserver { - // TODO: this can be replaced by peer selector once code is moved to go-waku api - availableMailservers := make(map[string]time.Duration) - availableMailserversMutex := sync.Mutex{} - availableMailserversWg := sync.WaitGroup{} - for _, mailserver := range allMailservers { - availableMailserversWg.Add(1) - go func(mailserver mailservers.Mailserver) { - defer gocommon.LogOnPanic() - defer availableMailserversWg.Done() - - peerID, err := mailserver.PeerID() - if err != nil { - return - } - - ctx, cancel := context.WithTimeout(m.ctx, 4*time.Second) - defer cancel() - - rtt, err := m.transport.PingPeer(ctx, peerID) - if err == nil { // pinging mailservers might fail, but we don't care - availableMailserversMutex.Lock() - availableMailservers[mailserver.ID] = rtt - availableMailserversMutex.Unlock() - } - }(mailserver) - } - availableMailserversWg.Wait() - - if len(availableMailservers) == 0 { - m.logger.Warn("No mailservers available") // Do nothing... +func (m *Messenger) setupStorenodes(storenodes []mailservers.Mailserver) error { + if m.transport.WakuVersion() != 2 { return nil } - mailserversByID := make(map[string]mailservers.Mailserver) - for idx := range allMailservers { - mailserversByID[allMailservers[idx].ID] = allMailservers[idx] - } - var sortedMailservers []SortedMailserver - for mailserverID, rtt := range availableMailservers { - ms := mailserversByID[mailserverID] - sortedMailserver := SortedMailserver{ - Mailserver: ms, - RTT: rtt, - } - m.mailPeersMutex.Lock() - pInfo, ok := m.mailserverCycle.peers[ms.ID] - m.mailPeersMutex.Unlock() - if ok { - if time.Now().Before(pInfo.canConnectAfter) { - continue // We can't connect to this node yet - } - } - sortedMailservers = append(sortedMailservers, sortedMailserver) - } - sort.Sort(byRTTMsAndCanConnectBefore(sortedMailservers)) - - result := make([]mailservers.Mailserver, len(sortedMailservers)) - for i, s := range sortedMailservers { - result[i] = s.Mailserver - } + for _, storenode := range storenodes { - return result -} - -func (m *Messenger) findNewMailserver() error { - // we have to override DNS manually because of https://github.com/status-im/status-mobile/issues/19581 - if overrideDNS { - var dialer net.Dialer - net.DefaultResolver = &net.Resolver{ - PreferGo: false, - Dial: func(context context.Context, _, _ string) (net.Conn, error) { - conn, err := dialer.DialContext(context, "udp", bootstrapDNS) - if err != nil { - return nil, err - } - return conn, nil - }, - } - } - - pinnedMailserver, err := m.getPinnedMailserver() - if err != nil { - m.logger.Error("Could not obtain the pinned mailserver", zap.Error(err)) - return err - } - if pinnedMailserver != nil { - return m.connectToMailserver(*pinnedMailserver) - } - - m.logger.Info("Finding a new mailserver...") - - allMailservers := m.mailserverCycle.allMailservers - - // TODO: remove this check once sockets are stable on x86_64 emulators - if findNearestMailServer { - allMailservers = m.getAvailableMailserversSortedByRTT(allMailservers) - } - - // Picks a random mailserver amongs the ones with the lowest latency - // The pool size is 1/4 of the mailservers were pinged successfully - pSize := poolSize(len(allMailservers) - 1) - if pSize <= 0 { - pSize = len(allMailservers) - if pSize <= 0 { - m.logger.Warn("No storenodes available") // Do nothing... - return nil - } - } - - r, err := rand.Int(rand.Reader, big.NewInt(int64(pSize))) - if err != nil { - return err - } - - ms := allMailservers[r.Int64()] - return m.connectToMailserver(ms) -} - -func (m *Messenger) mailserverStatus(mailserverID string) connStatus { - m.mailPeersMutex.RLock() - defer m.mailPeersMutex.RUnlock() - peer, ok := m.mailserverCycle.peers[mailserverID] - if !ok { - return disconnected - } - return peer.status -} - -func (m *Messenger) connectToMailserver(ms mailservers.Mailserver) error { - - m.logger.Info("connecting to mailserver", zap.String("mailserverID", ms.ID)) - - m.mailserverCycle.activeMailserver = &ms - signal.SendMailserverChanged(m.mailserverCycle.activeMailserver) - - mailserverStatus := m.mailserverStatus(ms.ID) - if mailserverStatus != connected { - m.mailPeersMutex.Lock() - m.mailserverCycle.peers[ms.ID] = peerStatus{ - status: connected, - lastConnectionAttempt: time.Now(), - canConnectAfter: time.Now().Add(defaultBackoff), - mailserver: ms, - } - m.mailPeersMutex.Unlock() - - m.mailserverCycle.activeMailserver.FailedRequests = 0 - peerID, err := m.mailserverCycle.activeMailserver.PeerID() + peerInfo, err := storenode.PeerInfo() if err != nil { - m.logger.Error("could not decode the peer id of mailserver", zap.Error(err)) return err } - m.logger.Info("mailserver available", zap.String("mailserverID", m.mailserverCycle.activeMailserver.ID)) - m.mailserverCycle.availabilitySubscriptions.EmitMailserverAvailable() - signal.SendMailserverAvailable(m.mailserverCycle.activeMailserver) - - m.transport.SetStorePeerID(peerID) - - // Query mailserver - m.asyncRequestAllHistoricMessages() + for _, addr := range utils.EncapsulatePeerID(peerInfo.ID, peerInfo.Addrs...) { + _, err := m.transport.AddStorePeer(addr) + if err != nil { + return err + } + } } return nil } -// getActiveMailserver returns the active mailserver if a communityID is present then it'll return the mailserver -// for that community if it has a mailserver setup otherwise it'll return the global mailserver -func (m *Messenger) getActiveMailserver(communityID ...string) *mailservers.Mailserver { - if len(communityID) == 0 || communityID[0] == "" { - return m.mailserverCycle.activeMailserver - } - ms, err := m.communityStorenodes.GetStorenodeByCommunityID(communityID[0]) +func (m *Messenger) getFleet() (string, error) { + var fleet string + dbFleet, err := m.settings.GetFleet() if err != nil { - if !errors.Is(err, storenodes.ErrNotFound) { - m.logger.Error("getting storenode for community, using global", zap.String("communityID", communityID[0]), zap.Error(err)) - } - // if we don't find a specific mailserver for the community, we just use the regular mailserverCycle's one - return m.mailserverCycle.activeMailserver - } - return &ms -} - -func (m *Messenger) getActiveMailserverID(communityID ...string) string { - ms := m.getActiveMailserver(communityID...) - if ms == nil { - return "" + return "", err } - return ms.ID -} - -func (m *Messenger) isMailserverAvailable(mailserverID string) bool { - return m.mailserverStatus(mailserverID) == connected -} - -func (m *Messenger) penalizeMailserver(id string) { - m.mailPeersMutex.Lock() - defer m.mailPeersMutex.Unlock() - pInfo, ok := m.mailserverCycle.peers[id] - if !ok { - pInfo.status = disconnected + if dbFleet != "" { + fleet = dbFleet + } else if m.config.clusterConfig.Fleet != "" { + fleet = m.config.clusterConfig.Fleet + } else { + fleet = params.FleetStatusProd } - - pInfo.canConnectAfter = time.Now().Add(graylistBackoff) - m.mailserverCycle.peers[id] = pInfo + return fleet, nil } func (m *Messenger) asyncRequestAllHistoricMessages() { - if !m.config.codeControlFlags.AutoRequestHistoricMessages { + if !m.config.codeControlFlags.AutoRequestHistoricMessages || m.transport.WakuVersion() == 1 { return } @@ -428,128 +93,117 @@ func (m *Messenger) asyncRequestAllHistoricMessages() { }() } -func (m *Messenger) verifyStorenodeStatus() { - defer common.LogOnPanic() - ticker := time.NewTicker(1 * time.Second) - defer ticker.Stop() - - for { - select { - case <-ticker.C: - err := m.disconnectStorenodeIfRequired() - if err != nil { - m.logger.Error("failed to handle mailserver cycle event", zap.Error(err)) - continue - } - - case <-m.quit: - return - } - } -} - -func (m *Messenger) getPinnedMailserver() (*mailservers.Mailserver, error) { +func (m *Messenger) GetPinnedStorenode() (peer.ID, error) { fleet, err := m.getFleet() if err != nil { - return nil, err + return "", err } pinnedMailservers, err := m.settings.GetPinnedMailservers() if err != nil { - return nil, err + return "", err } pinnedMailserver, ok := pinnedMailservers[fleet] if !ok { - return nil, nil + return "", nil } fleetMailservers := mailservers.DefaultMailservers() for _, c := range fleetMailservers { if c.Fleet == fleet && c.ID == pinnedMailserver { - return &c, nil + return c.PeerID() } } if m.mailserversDatabase != nil { customMailservers, err := m.mailserversDatabase.Mailservers() if err != nil { - return nil, err + return "", err } for _, c := range customMailservers { if c.Fleet == fleet && c.ID == pinnedMailserver { - return &c, nil + return c.PeerID() } } } - return nil, nil + return "", nil } -func (m *Messenger) disconnectStorenodeIfRequired() error { - m.logger.Debug("wakuV2 storenode status verification") +func (m *Messenger) UseStorenodes() (bool, error) { + return m.settings.CanUseMailservers() +} - if m.mailserverCycle.activeMailserver == nil { - // No active storenode, find a new one - m.cycleMailservers() - return nil +func (m *Messenger) Storenodes() ([]peer.ID, error) { + mailservers, err := m.AllMailservers() + if err != nil { + return nil, err } - // Check whether we want to disconnect the active storenode - if m.mailserverCycle.activeMailserver.FailedRequests >= mailserverMaxFailedRequests { - m.penalizeMailserver(m.mailserverCycle.activeMailserver.ID) - signal.SendMailserverNotWorking() - m.logger.Info("too many failed requests", zap.String("storenode", m.mailserverCycle.activeMailserver.ID)) - m.mailserverCycle.activeMailserver.FailedRequests = 0 - return m.connectToNewMailserverAndWait() + var result []peer.ID + for _, m := range mailservers { + peerID, err := m.PeerID() + if err != nil { + return nil, err + } + result = append(result, peerID) } - return nil + return result, nil } -func (m *Messenger) waitForAvailableStoreNode(timeout time.Duration) bool { - // Add 1 second to timeout, because the mailserver cycle has 1 second ticker, which doesn't tick on start. - // This can be improved after merging https://github.com/status-im/status-go/pull/4380. - // NOTE: https://stackoverflow.com/questions/32705582/how-to-get-time-tick-to-tick-immediately - timeout += time.Second +func (m *Messenger) checkForStorenodeCycleSignals() { + if m.transport.WakuVersion() != 2 { + return + } - finish := make(chan struct{}) - cancel := make(chan struct{}) + changed := m.transport.OnStorenodeChanged() + notWorking := m.transport.OnStorenodeNotWorking() + available := m.transport.OnStorenodeAvailable() - wg := sync.WaitGroup{} - wg.Add(1) + allMailservers, err := m.AllMailservers() + if err != nil { + m.logger.Error("Could not retrieve mailserver list", zap.Error(err)) + return + } - go func() { - defer gocommon.LogOnPanic() - defer func() { - wg.Done() - }() - for !m.isMailserverAvailable(m.getActiveMailserverID()) { - select { - case <-m.mailserverCycle.availabilitySubscriptions.Subscribe(): - case <-cancel: - return - } + mailserverMap := make(map[peer.ID]mailservers.Mailserver) + for _, ms := range allMailservers { + peerID, err := ms.PeerID() + if err != nil { + m.logger.Error("could not retrieve peerID", zap.Error(err)) + return } - }() - - go func() { - defer gocommon.LogOnPanic() - defer func() { - close(finish) - }() - wg.Wait() - }() - - select { - case <-finish: - case <-time.After(timeout): - close(cancel) - case <-m.ctx.Done(): - close(cancel) + mailserverMap[peerID] = ms } - return m.isMailserverAvailable(m.getActiveMailserverID()) + for { + select { + case <-m.ctx.Done(): + return + case <-notWorking: + signal.SendMailserverNotWorking() + + case activeMailserver := <-changed: + if activeMailserver != "" { + ms, ok := mailserverMap[activeMailserver] + if ok { + signal.SendMailserverChanged(&ms) + } + } else { + signal.SendMailserverChanged(nil) + } + case activeMailserver := <-available: + if activeMailserver != "" { + ms, ok := mailserverMap[activeMailserver] + if ok { + signal.SendMailserverAvailable(&ms) + } + m.asyncRequestAllHistoricMessages() + } + } + } } diff --git a/protocol/messenger_mailserver_processMailserverBatch_test.go b/protocol/messenger_mailserver_processMailserverBatch_test.go deleted file mode 100644 index 71ea759a409..00000000000 --- a/protocol/messenger_mailserver_processMailserverBatch_test.go +++ /dev/null @@ -1,167 +0,0 @@ -package protocol - -import ( - "context" - "crypto/rand" - "encoding/hex" - "errors" - "math/big" - "testing" - "time" - - "github.com/google/uuid" - "github.com/libp2p/go-libp2p/core/peer" - "github.com/stretchr/testify/require" - - "github.com/status-im/status-go/eth-node/types" - "github.com/status-im/status-go/protocol/tt" -) - -type queryResponse struct { - topics []types.TopicType - err error // Indicates if this response will simulate an error returned by SendMessagesRequestForTopics - cursor []byte -} - -type mockTransport struct { - queryResponses map[string]queryResponse -} - -func newMockTransport() *mockTransport { - return &mockTransport{ - queryResponses: make(map[string]queryResponse), - } -} - -func getInitialResponseKey(topics []types.TopicType) string { - return hex.EncodeToString(append([]byte("start"), topics[0][:]...)) -} - -func (t *mockTransport) SendMessagesRequestForTopics( - ctx context.Context, - peerID peer.ID, - from, to uint32, - prevCursor types.StoreRequestCursor, - pubsubTopic string, - contentTopics []types.TopicType, - limit uint32, - waitForResponse bool, - processEnvelopes bool, -) (cursor types.StoreRequestCursor, envelopesCount int, err error) { - var response queryResponse - if prevCursor == nil { - initialResponse := getInitialResponseKey(contentTopics) - response = t.queryResponses[initialResponse] - } else { - response = t.queryResponses[hex.EncodeToString(prevCursor)] - } - return response.cursor, 0, response.err -} - -func (t *mockTransport) Populate(topics []types.TopicType, responses int, includeRandomError bool) error { - if responses <= 0 || len(topics) == 0 { - return errors.New("invalid input parameters") - } - - var topicBatches [][]types.TopicType - - for i := 0; i < len(topics); i += maxTopicsPerRequest { - // Split batch in 10-contentTopic subbatches - j := i + maxTopicsPerRequest - if j > len(topics) { - j = len(topics) - } - topicBatches = append(topicBatches, topics[i:j]) - } - - randomErrIdx, err := rand.Int(rand.Reader, big.NewInt(int64(len(topicBatches)))) - if err != nil { - return err - } - randomErrIdxInt := int(randomErrIdx.Int64()) - - for i, topicBatch := range topicBatches { - // Setup initial response - initialResponseKey := getInitialResponseKey(topicBatch) - t.queryResponses[initialResponseKey] = queryResponse{ - topics: topicBatch, - err: nil, - } - - prevKey := initialResponseKey - for x := 0; x < responses-1; x++ { - newResponseCursor := []byte(uuid.New().String()) - newResponseKey := hex.EncodeToString(newResponseCursor) - - var err error - if includeRandomError && i == randomErrIdxInt && x == responses-2 { // Include an error in last request - err = errors.New("random error") - } - - t.queryResponses[newResponseKey] = queryResponse{ - topics: topicBatch, - err: err, - } - - // Updating prev response cursor to point to the new response - prevResponse := t.queryResponses[prevKey] - prevResponse.cursor = newResponseCursor - t.queryResponses[prevKey] = prevResponse - - prevKey = newResponseKey - } - - } - - return nil -} - -func TestProcessMailserverBatchHappyPath(t *testing.T) { - ctx, cancel := context.WithTimeout(context.TODO(), 5*time.Second) - defer cancel() - - logger := tt.MustCreateTestLogger() - - mailserverID, err := peer.Decode("16Uiu2HAkw3x97MbbZSWHbdF5bob45vcZvPPK4s4Mjyv2mxyB9GS3") - require.NoError(t, err) - topics := []types.TopicType{} - for i := 0; i < 22; i++ { - topics = append(topics, types.BytesToTopic([]byte{0, 0, 0, byte(i)})) - } - - testTransport := newMockTransport() - err = testTransport.Populate(topics, 10, false) - require.NoError(t, err) - - testBatch := MailserverBatch{ - Topics: topics, - } - - err = processMailserverBatch(ctx, testTransport, testBatch, mailserverID, logger, defaultStoreNodeRequestPageSize, nil, false) - require.NoError(t, err) -} - -func TestProcessMailserverBatchFailure(t *testing.T) { - ctx, cancel := context.WithTimeout(context.TODO(), 5*time.Second) - defer cancel() - - logger := tt.MustCreateTestLogger() - - mailserverID, err := peer.Decode("16Uiu2HAkw3x97MbbZSWHbdF5bob45vcZvPPK4s4Mjyv2mxyB9GS3") - require.NoError(t, err) - topics := []types.TopicType{} - for i := 0; i < 5; i++ { - topics = append(topics, types.BytesToTopic([]byte{0, 0, 0, byte(i)})) - } - - testTransport := newMockTransport() - err = testTransport.Populate(topics, 4, true) - require.NoError(t, err) - - testBatch := MailserverBatch{ - Topics: topics, - } - - err = processMailserverBatch(ctx, testTransport, testBatch, mailserverID, logger, defaultStoreNodeRequestPageSize, nil, false) - require.Error(t, err) -} diff --git a/protocol/messenger_store_node_request_manager.go b/protocol/messenger_store_node_request_manager.go index 84401003f2f..1db7388a507 100644 --- a/protocol/messenger_store_node_request_manager.go +++ b/protocol/messenger_store_node_request_manager.go @@ -10,13 +10,13 @@ import ( gocommon "github.com/status-im/status-go/common" "github.com/status-im/status-go/eth-node/crypto" "github.com/status-im/status-go/protocol/common/shard" + "github.com/waku-org/go-waku/waku/v2/api/history" "go.uber.org/zap" "github.com/status-im/status-go/eth-node/types" "github.com/status-im/status-go/protocol/communities" "github.com/status-im/status-go/protocol/transport" - "github.com/status-im/status-go/services/mailservers" ) const ( @@ -57,7 +57,7 @@ type StoreNodeRequestManager struct { // activeRequestsLock should be locked each time activeRequests is being accessed or changed. activeRequestsLock sync.RWMutex - onPerformingBatch func(MailserverBatch) + onPerformingBatch func(types.MailserverBatch) } func NewStoreNodeRequestManager(m *Messenger) *StoreNodeRequestManager { @@ -374,7 +374,7 @@ func (r *storeNodeRequest) finalize() { } } -func (r *storeNodeRequest) shouldFetchNextPage(envelopesCount int) (bool, uint32) { +func (r *storeNodeRequest) shouldFetchNextPage(envelopesCount int) (bool, uint64) { logger := r.manager.logger.With( zap.Any("requestID", r.requestID), zap.Int("envelopesCount", envelopesCount)) @@ -522,13 +522,13 @@ func (r *storeNodeRequest) routine() { communityID := r.requestID.getCommunityID() if r.requestID.RequestType != storeNodeCommunityRequest || !r.manager.messenger.communityStorenodes.HasStorenodeSetup(communityID) { - if !r.manager.messenger.waitForAvailableStoreNode(storeNodeAvailableTimeout) { + if !r.manager.messenger.transport.WaitForAvailableStoreNode(storeNodeAvailableTimeout) { r.result.err = fmt.Errorf("store node is not available") return } } - storeNode := r.manager.messenger.getActiveMailserver(communityID) + storeNode := r.manager.messenger.getCommunityMailserver(communityID) // Check if community already exists locally and get Clock. if r.requestID.RequestType == storeNodeCommunityRequest { @@ -541,8 +541,8 @@ func (r *storeNodeRequest) routine() { // Start store node request from, to := r.manager.messenger.calculateMailserverTimeBounds(oneMonthDuration) - _, err := r.manager.messenger.performMailserverRequest(storeNode, func(ms mailservers.Mailserver) (*MessengerResponse, error) { - batch := MailserverBatch{ + _, err := r.manager.messenger.performStorenodeTask(func() (*MessengerResponse, error) { + batch := types.MailserverBatch{ From: from, To: to, PubsubTopic: r.pubsubTopic, @@ -553,8 +553,8 @@ func (r *storeNodeRequest) routine() { r.manager.onPerformingBatch(batch) } - return nil, r.manager.messenger.processMailserverBatchWithOptions(ms, batch, r.config.InitialPageSize, r.shouldFetchNextPage, true) - }) + return nil, r.manager.messenger.processMailserverBatchWithOptions(storeNode, batch, r.config.InitialPageSize, r.shouldFetchNextPage, true) + }, history.WithPeerID(storeNode)) r.result.err = err } diff --git a/protocol/messenger_store_node_request_manager_config.go b/protocol/messenger_store_node_request_manager_config.go index cffcc76d70e..56faf835fdb 100644 --- a/protocol/messenger_store_node_request_manager_config.go +++ b/protocol/messenger_store_node_request_manager_config.go @@ -3,8 +3,8 @@ package protocol type StoreNodeRequestConfig struct { WaitForResponse bool StopWhenDataFound bool - InitialPageSize uint32 - FurtherPageSize uint32 + InitialPageSize uint64 + FurtherPageSize uint64 } type StoreNodeRequestOption func(*StoreNodeRequestConfig) @@ -40,13 +40,13 @@ func WithStopWhenDataFound(stopWhenDataFound bool) StoreNodeRequestOption { } } -func WithInitialPageSize(initialPageSize uint32) StoreNodeRequestOption { +func WithInitialPageSize(initialPageSize uint64) StoreNodeRequestOption { return func(c *StoreNodeRequestConfig) { c.InitialPageSize = initialPageSize } } -func WithFurtherPageSize(furtherPageSize uint32) StoreNodeRequestOption { +func WithFurtherPageSize(furtherPageSize uint64) StoreNodeRequestOption { return func(c *StoreNodeRequestConfig) { c.FurtherPageSize = furtherPageSize } diff --git a/protocol/messenger_storenode_comunity_test.go b/protocol/messenger_storenode_comunity_test.go index 746951413fe..51cbf712ff4 100644 --- a/protocol/messenger_storenode_comunity_test.go +++ b/protocol/messenger_storenode_comunity_test.go @@ -351,10 +351,10 @@ func (s *MessengerStoreNodeCommunitySuite) TestToggleUseMailservers() { // Enable use of mailservers err := s.owner.ToggleUseMailservers(true) s.Require().NoError(err) - s.Require().NotNil(s.owner.mailserverCycle.activeMailserver) + s.Require().NotNil(s.owner.transport.GetActiveStorenode()) // Disable use of mailservers err = s.owner.ToggleUseMailservers(false) s.Require().NoError(err) - s.Require().Nil(s.owner.mailserverCycle.activeMailserver) + s.Require().Nil(s.owner.transport.GetActiveStorenode()) } diff --git a/protocol/messenger_storenode_request_test.go b/protocol/messenger_storenode_request_test.go index d8b684d7488..84bede934e9 100644 --- a/protocol/messenger_storenode_request_test.go +++ b/protocol/messenger_storenode_request_test.go @@ -235,7 +235,7 @@ func (s *MessengerStoreNodeRequestSuite) newMessenger(shh types.Waku, logger *za } func (s *MessengerStoreNodeRequestSuite) createCommunity(m *Messenger) *communities.Community { - s.waitForAvailableStoreNode(m) + s.WaitForAvailableStoreNode(m) storeNodeSubscription := s.setupStoreNodeEnvelopesWatcher(nil) @@ -309,7 +309,7 @@ func (s *MessengerStoreNodeRequestSuite) fetchProfile(m *Messenger, contactID st } } -func (s *MessengerStoreNodeRequestSuite) waitForAvailableStoreNode(messenger *Messenger) { +func (s *MessengerStoreNodeRequestSuite) WaitForAvailableStoreNode(messenger *Messenger) { WaitForAvailableStoreNode(&s.Suite, messenger, storeNodeConnectTimeout) } @@ -419,11 +419,11 @@ func (s *MessengerStoreNodeRequestSuite) TestSimultaneousCommunityInfoRequests() community := s.createCommunity(s.owner) storeNodeRequestsCount := 0 - s.bob.storeNodeRequestsManager.onPerformingBatch = func(batch MailserverBatch) { + s.bob.storeNodeRequestsManager.onPerformingBatch = func(batch types.MailserverBatch) { storeNodeRequestsCount++ } - s.waitForAvailableStoreNode(s.bob) + s.WaitForAvailableStoreNode(s.bob) wg := sync.WaitGroup{} @@ -453,7 +453,7 @@ func (s *MessengerStoreNodeRequestSuite) TestRequestNonExistentCommunity() { s.createBob() - s.waitForAvailableStoreNode(s.bob) + s.WaitForAvailableStoreNode(s.bob) fetchedCommunity, err := s.bob.FetchCommunity(&request) s.Require().NoError(err) @@ -722,7 +722,7 @@ func (s *MessengerStoreNodeRequestSuite) TestRequestShardAndCommunityInfo() { s.waitForEnvelopes(storeNodeSubscription, 1) - s.waitForAvailableStoreNode(s.bob) + s.WaitForAvailableStoreNode(s.bob) communityShard := community.CommunityShard() @@ -1195,7 +1195,7 @@ func (s *MessengerStoreNodeRequestSuite) TestFetchingCommunityWithOwnerToken() { s.createOwner() s.createBob() - s.waitForAvailableStoreNode(s.owner) + s.WaitForAvailableStoreNode(s.owner) community := s.createCommunity(s.owner) // owner mints owner token @@ -1228,7 +1228,7 @@ func (s *MessengerStoreNodeRequestSuite) TestFetchingCommunityWithOwnerToken() { s.Require().NoError(err) s.Require().Len(community.TokenPermissions(), 1) - s.waitForAvailableStoreNode(s.bob) + s.WaitForAvailableStoreNode(s.bob) s.fetchCommunity(s.bob, community.CommunityShard(), community) } diff --git a/protocol/messenger_testing_utils.go b/protocol/messenger_testing_utils.go index d78c8d77b06..efed71fc0df 100644 --- a/protocol/messenger_testing_utils.go +++ b/protocol/messenger_testing_utils.go @@ -365,7 +365,7 @@ func SetIdentityImagesAndWaitForChange(s *suite.Suite, messenger *Messenger, tim } func WaitForAvailableStoreNode(s *suite.Suite, m *Messenger, timeout time.Duration) { - available := m.waitForAvailableStoreNode(timeout) + available := m.transport.WaitForAvailableStoreNode(timeout) s.Require().True(available) } diff --git a/protocol/storenodes/storenodes.go b/protocol/storenodes/storenodes.go index 66a6f4bc35a..c6188cfe16e 100644 --- a/protocol/storenodes/storenodes.go +++ b/protocol/storenodes/storenodes.go @@ -6,6 +6,10 @@ import ( "go.uber.org/zap" + "github.com/libp2p/go-libp2p/core/peer" + + "github.com/waku-org/go-waku/waku/v2/utils" + "github.com/status-im/status-go/eth-node/types" "github.com/status-im/status-go/services/mailservers" ) @@ -51,14 +55,17 @@ func (m *CommunityStorenodes) GetStorenodeByCommunityID(communityID string) (mai return toMailserver(msData.storenodes[0]), nil } -func (m *CommunityStorenodes) IsCommunityStoreNode(id string) bool { +func (m *CommunityStorenodes) IsCommunityStoreNode(peerID peer.ID) bool { m.storenodesByCommunityIDMutex.RLock() defer m.storenodesByCommunityIDMutex.RUnlock() for _, data := range m.storenodesByCommunityID { for _, snode := range data.storenodes { - if snode.StorenodeID == id { - return true + commStorenodeID, err := utils.GetPeerID(snode.Address) + if err == nil { + if commStorenodeID == peerID { + return true + } } } } diff --git a/protocol/transport/transport.go b/protocol/transport/transport.go index d7254045712..3780639b2f2 100644 --- a/protocol/transport/transport.go +++ b/protocol/transport/transport.go @@ -4,17 +4,17 @@ import ( "context" "crypto/ecdsa" "database/sql" - "encoding/hex" "sync" "time" - "github.com/google/uuid" "github.com/libp2p/go-libp2p/core/peer" "github.com/multiformats/go-multiaddr" "github.com/pkg/errors" "go.uber.org/zap" "golang.org/x/exp/maps" + "github.com/waku-org/go-waku/waku/v2/api/history" + "github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/p2p/enode" gocommon "github.com/status-im/status-go/common" @@ -462,89 +462,6 @@ func (t *Transport) Peers() types.PeerStats { return t.waku.Peers() } -func (t *Transport) createMessagesRequest( - ctx context.Context, - peerID peer.ID, - from, to uint32, - previousStoreCursor types.StoreRequestCursor, - pubsubTopic string, - contentTopics []types.TopicType, - limit uint32, - waitForResponse bool, - processEnvelopes bool, -) (storeCursor types.StoreRequestCursor, envelopesCount int, err error) { - r := createMessagesRequest(from, to, nil, previousStoreCursor, pubsubTopic, contentTopics, limit) - - if waitForResponse { - resultCh := make(chan struct { - storeCursor types.StoreRequestCursor - envelopesCount int - err error - }) - - go func() { - defer gocommon.LogOnPanic() - storeCursor, envelopesCount, err = t.waku.RequestStoreMessages(ctx, peerID, r, processEnvelopes) - resultCh <- struct { - storeCursor types.StoreRequestCursor - envelopesCount int - err error - }{storeCursor, envelopesCount, err} - }() - - select { - case result := <-resultCh: - return result.storeCursor, result.envelopesCount, result.err - case <-ctx.Done(): - return nil, 0, ctx.Err() - } - } else { - go func() { - defer gocommon.LogOnPanic() - _, _, err = t.waku.RequestStoreMessages(ctx, peerID, r, false) - if err != nil { - t.logger.Error("failed to request store messages", zap.Error(err)) - } - }() - } - - return -} - -func (t *Transport) SendMessagesRequestForTopics( - ctx context.Context, - peerID peer.ID, - from, to uint32, - prevCursor types.StoreRequestCursor, - pubsubTopic string, - contentTopics []types.TopicType, - limit uint32, - waitForResponse bool, - processEnvelopes bool, -) (cursor types.StoreRequestCursor, envelopesCount int, err error) { - return t.createMessagesRequest(ctx, peerID, from, to, prevCursor, pubsubTopic, contentTopics, limit, waitForResponse, processEnvelopes) -} - -func createMessagesRequest(from, to uint32, cursor []byte, storeCursor types.StoreRequestCursor, pubsubTopic string, topics []types.TopicType, limit uint32) types.MessagesRequest { - aUUID := uuid.New() - // uuid is 16 bytes, converted to hex it's 32 bytes as expected by types.MessagesRequest - id := []byte(hex.EncodeToString(aUUID[:])) - var topicBytes [][]byte - for idx := range topics { - topicBytes = append(topicBytes, topics[idx][:]) - } - return types.MessagesRequest{ - ID: id, - From: from, - To: to, - Limit: limit, - Cursor: cursor, - PubsubTopic: pubsubTopic, - ContentTopics: topicBytes, - StoreCursor: storeCursor, - } -} - // ConfirmMessagesProcessed marks the messages as processed in the cache so // they won't be passed to the next layer anymore func (t *Transport) ConfirmMessagesProcessed(ids []string, timestamp uint64) error { @@ -635,10 +552,6 @@ func (t *Transport) ConnectionChanged(state connection.State) { t.waku.ConnectionChanged(state) } -func (t *Transport) PingPeer(ctx context.Context, peerID peer.ID) (time.Duration, error) { - return t.waku.PingPeer(ctx, peerID) -} - // Subscribe to a pubsub topic, passing an optional public key if the pubsub topic is protected func (t *Transport) SubscribeToPubsubTopic(topic string, optPublicKey *ecdsa.PublicKey) error { if t.waku.Version() == 2 { @@ -685,10 +598,6 @@ func (t *Transport) ConfirmMessageDelivered(messageID string) { t.waku.ConfirmMessageDelivered(commHashes) } -func (t *Transport) SetStorePeerID(peerID peer.ID) { - t.waku.SetStorePeerID(peerID) -} - func (t *Transport) SetCriteriaForMissingMessageVerification(peerID peer.ID, filters []*Filter) { if t.waku.Version() != 2 { return @@ -721,3 +630,56 @@ func (t *Transport) SetCriteriaForMissingMessageVerification(peerID peer.ID, fil } } } + +func (t *Transport) GetActiveStorenode() peer.ID { + return t.waku.GetActiveStorenode() +} + +func (t *Transport) DisconnectActiveStorenode(ctx context.Context, backoffReason time.Duration, shouldCycle bool) { + t.waku.DisconnectActiveStorenode(ctx, backoffReason, shouldCycle) +} + +func (t *Transport) OnStorenodeAvailableOneShot() <-chan struct{} { + return t.waku.OnStorenodeAvailableOneShot() +} + +func (t *Transport) OnStorenodeChanged() <-chan peer.ID { + return t.waku.OnStorenodeChanged() +} + +func (t *Transport) OnStorenodeNotWorking() <-chan struct{} { + return t.waku.OnStorenodeNotWorking() +} + +func (t *Transport) OnStorenodeAvailable() <-chan peer.ID { + return t.waku.OnStorenodeAvailable() +} + +func (t *Transport) WaitForAvailableStoreNode(timeout time.Duration) bool { + return t.waku.WaitForAvailableStoreNode(timeout) +} + +func (t *Transport) IsStorenodeAvailable(peerID peer.ID) bool { + return t.waku.IsStorenodeAvailable(peerID) +} + +func (t *Transport) PerformStorenodeTask(fn func() error, opts ...history.StorenodeTaskOption) error { + return t.waku.PerformStorenodeTask(fn, opts...) +} + +func (t *Transport) ProcessMailserverBatch( + ctx context.Context, + batch types.MailserverBatch, + storenodeID peer.ID, + pageLimit uint64, + shouldProcessNextPage func(int) (bool, uint64), + processEnvelopes bool, +) error { + return t.waku.ProcessMailserverBatch(ctx, batch, storenodeID, pageLimit, shouldProcessNextPage, processEnvelopes) +} + +func (t *Transport) SetStorenodeConfigProvider(c history.StorenodeConfigProvider) { + if t.WakuVersion() == 2 { + t.waku.SetStorenodeConfigProvider(c) + } +} diff --git a/services/ext/api.go b/services/ext/api.go index d7918fb4dd0..156b50ca8b0 100644 --- a/services/ext/api.go +++ b/services/ext/api.go @@ -1408,10 +1408,6 @@ func (api *PublicAPI) RequestAllHistoricMessagesWithRetries(forceFetchingBackup return api.service.messenger.RequestAllHistoricMessages(forceFetchingBackup, true) } -func (api *PublicAPI) DisconnectActiveMailserver() { - api.service.messenger.DisconnectActiveMailserver() -} - // Echo is a method for testing purposes. func (api *PublicAPI) Echo(ctx context.Context, message string) (string, error) { return message, nil diff --git a/vendor/github.com/waku-org/go-waku/logging/logging.go b/vendor/github.com/waku-org/go-waku/logging/logging.go index ac895e9d58a..f8742ac5165 100644 --- a/vendor/github.com/waku-org/go-waku/logging/logging.go +++ b/vendor/github.com/waku-org/go-waku/logging/logging.go @@ -74,6 +74,14 @@ func (t timestamp) String() string { return time.Unix(0, int64(t)).Format(time.RFC3339) } +func Timep(key string, time *int64) zapcore.Field { + if time == nil { + return zap.String(key, "-") + } else { + return Time(key, *time) + } +} + func Epoch(key string, time time.Time) zap.Field { return zap.String(key, fmt.Sprintf("%d", time.UnixNano())) } diff --git a/vendor/github.com/waku-org/go-waku/waku/v2/api/history/cycle.go b/vendor/github.com/waku-org/go-waku/waku/v2/api/history/cycle.go new file mode 100644 index 00000000000..313ee0a45b0 --- /dev/null +++ b/vendor/github.com/waku-org/go-waku/waku/v2/api/history/cycle.go @@ -0,0 +1,524 @@ +package history + +import ( + "context" + "crypto/rand" + "errors" + "fmt" + "math" + "math/big" + "net" + "net/http" + "runtime" + "sort" + "sync" + "time" + + "github.com/libp2p/go-libp2p/core/host" + "github.com/libp2p/go-libp2p/core/peer" + "github.com/libp2p/go-libp2p/p2p/protocol/ping" + "github.com/waku-org/go-waku/waku/v2/protocol/store" + "go.uber.org/zap" +) + +const defaultBackoff = 10 * time.Second +const graylistBackoff = 3 * time.Minute +const storenodeVerificationInterval = time.Second +const storenodeMaxFailedRequests uint = 2 +const minStorenodesToChooseFrom = 3 +const isAndroidEmulator = runtime.GOOS == "android" && runtime.GOARCH == "amd64" +const findNearestMailServer = !isAndroidEmulator +const overrideDNS = runtime.GOOS == "android" || runtime.GOOS == "ios" +const bootstrapDNS = "8.8.8.8:53" + +type connStatus int + +const ( + disconnected connStatus = iota + 1 + connected +) + +type peerStatus struct { + status connStatus + canConnectAfter time.Time + lastConnectionAttempt time.Time +} + +type StorenodeConfigProvider interface { + UseStorenodes() (bool, error) + GetPinnedStorenode() (peer.ID, error) + Storenodes() ([]peer.ID, error) +} + +type StorenodeCycle struct { + sync.RWMutex + + logger *zap.Logger + + host host.Host + + storenodeConfigProvider StorenodeConfigProvider + + StorenodeAvailableOneshotEmitter *OneShotEmitter[struct{}] + StorenodeChangedEmitter *Emitter[peer.ID] + StorenodeNotWorkingEmitter *Emitter[struct{}] + StorenodeAvailableEmitter *Emitter[peer.ID] + + failedRequests map[peer.ID]uint + + peersMutex sync.RWMutex + activeStorenode peer.ID + peers map[peer.ID]peerStatus +} + +func NewStorenodeCycle(logger *zap.Logger) *StorenodeCycle { + return &StorenodeCycle{ + StorenodeAvailableOneshotEmitter: NewOneshotEmitter[struct{}](), + StorenodeChangedEmitter: NewEmitter[peer.ID](), + StorenodeNotWorkingEmitter: NewEmitter[struct{}](), + StorenodeAvailableEmitter: NewEmitter[peer.ID](), + logger: logger.Named("storenode-cycle"), + } +} + +func (m *StorenodeCycle) Start(ctx context.Context, h host.Host) { + m.logger.Debug("starting storenode cycle") + m.host = h + m.failedRequests = make(map[peer.ID]uint) + m.peers = make(map[peer.ID]peerStatus) + + go m.verifyStorenodeStatus(ctx) +} + +func (m *StorenodeCycle) DisconnectActiveStorenode(backoff time.Duration) { + m.Lock() + defer m.Unlock() + + m.disconnectActiveStorenode(backoff) +} + +func (m *StorenodeCycle) connectToNewStorenodeAndWait(ctx context.Context) error { + // Handle pinned storenodes + m.logger.Info("disconnecting storenode") + pinnedStorenode, err := m.storenodeConfigProvider.GetPinnedStorenode() + if err != nil { + m.logger.Error("could not obtain the pinned storenode", zap.Error(err)) + return err + } + + // If no pinned storenode, no need to disconnect and wait for it to be available + if pinnedStorenode == "" { + m.disconnectActiveStorenode(graylistBackoff) + } + + return m.findNewStorenode(ctx) +} + +func (m *StorenodeCycle) disconnectStorenode(backoffDuration time.Duration) error { + if m.activeStorenode == "" { + m.logger.Info("no active storenode") + return nil + } + + m.logger.Info("disconnecting active storenode", zap.Stringer("peerID", m.activeStorenode)) + + m.peersMutex.Lock() + pInfo, ok := m.peers[m.activeStorenode] + if ok { + pInfo.status = disconnected + pInfo.canConnectAfter = time.Now().Add(backoffDuration) + m.peers[m.activeStorenode] = pInfo + } else { + m.peers[m.activeStorenode] = peerStatus{ + status: disconnected, + canConnectAfter: time.Now().Add(backoffDuration), + } + } + m.peersMutex.Unlock() + + m.activeStorenode = "" + + return nil +} + +func (m *StorenodeCycle) disconnectActiveStorenode(backoffDuration time.Duration) { + err := m.disconnectStorenode(backoffDuration) + if err != nil { + m.logger.Error("failed to disconnect storenode", zap.Error(err)) + } + + m.StorenodeChangedEmitter.Emit("") +} + +func (m *StorenodeCycle) Cycle(ctx context.Context) { + if m.storenodeConfigProvider == nil { + m.logger.Debug("storenodeConfigProvider not yet setup") + return + } + + m.logger.Info("Automatically switching storenode") + + if m.activeStorenode != "" { + m.disconnectActiveStorenode(graylistBackoff) + } + + useStorenode, err := m.storenodeConfigProvider.UseStorenodes() + if err != nil { + m.logger.Error("failed to get use storenodes", zap.Error(err)) + return + } + + if !useStorenode { + m.logger.Info("Skipping storenode search due to useStorenode being false") + return + } + + err = m.findNewStorenode(ctx) + if err != nil { + m.logger.Error("Error getting new storenode", zap.Error(err)) + } +} + +func poolSize(fleetSize int) int { + return int(math.Ceil(float64(fleetSize) / 4)) +} + +func (m *StorenodeCycle) getAvailableStorenodesSortedByRTT(ctx context.Context, allStorenodes []peer.ID) []peer.ID { + availableStorenodes := make(map[peer.ID]time.Duration) + availableStorenodesMutex := sync.Mutex{} + availableStorenodesWg := sync.WaitGroup{} + for _, storenode := range allStorenodes { + availableStorenodesWg.Add(1) + go func(peerID peer.ID) { + defer availableStorenodesWg.Done() + ctx, cancel := context.WithTimeout(ctx, 4*time.Second) + defer cancel() + + rtt, err := m.pingPeer(ctx, peerID) + if err == nil { // pinging storenodes might fail, but we don't care + availableStorenodesMutex.Lock() + availableStorenodes[peerID] = rtt + availableStorenodesMutex.Unlock() + } + }(storenode) + } + availableStorenodesWg.Wait() + + if len(availableStorenodes) == 0 { + m.logger.Warn("No storenodes available") // Do nothing.. + return nil + } + + var sortedStorenodes []SortedStorenode + for storenodeID, rtt := range availableStorenodes { + sortedStorenode := SortedStorenode{ + Storenode: storenodeID, + RTT: rtt, + } + m.peersMutex.Lock() + pInfo, ok := m.peers[storenodeID] + m.peersMutex.Unlock() + if ok && time.Now().Before(pInfo.canConnectAfter) { + continue // We can't connect to this node yet + } + sortedStorenodes = append(sortedStorenodes, sortedStorenode) + } + sort.Sort(byRTTMsAndCanConnectBefore(sortedStorenodes)) + + result := make([]peer.ID, len(sortedStorenodes)) + for i, s := range sortedStorenodes { + result[i] = s.Storenode + } + + return result +} + +func (m *StorenodeCycle) pingPeer(ctx context.Context, peerID peer.ID) (time.Duration, error) { + pingResultCh := ping.Ping(ctx, m.host, peerID) + select { + case <-ctx.Done(): + return 0, ctx.Err() + case r := <-pingResultCh: + if r.Error != nil { + return 0, r.Error + } + return r.RTT, nil + } +} + +func (m *StorenodeCycle) findNewStorenode(ctx context.Context) error { + // we have to override DNS manually because of https://github.com/status-im/status-mobile/issues/19581 + if overrideDNS { + var dialer net.Dialer + net.DefaultResolver = &net.Resolver{ + PreferGo: false, + Dial: func(context context.Context, _, _ string) (net.Conn, error) { + conn, err := dialer.DialContext(context, "udp", bootstrapDNS) + if err != nil { + return nil, err + } + return conn, nil + }, + } + } + + pinnedStorenode, err := m.storenodeConfigProvider.GetPinnedStorenode() + if err != nil { + m.logger.Error("Could not obtain the pinned storenode", zap.Error(err)) + return err + } + + if pinnedStorenode != "" { + return m.setActiveStorenode(pinnedStorenode) + } + + m.logger.Info("Finding a new storenode..") + + allStorenodes, err := m.storenodeConfigProvider.Storenodes() + if err != nil { + return err + } + + // TODO: remove this check once sockets are stable on x86_64 emulators + if findNearestMailServer { + allStorenodes = m.getAvailableStorenodesSortedByRTT(ctx, allStorenodes) + } + + // Picks a random storenode amongs the ones with the lowest latency + // The pool size is 1/4 of the storenodes were pinged successfully + // If the pool size is less than `minStorenodesToChooseFrom`, it will + // pick a storenode fromm all the available storenodes + pSize := poolSize(len(allStorenodes) - 1) + if pSize <= minStorenodesToChooseFrom { + pSize = len(allStorenodes) + if pSize <= 0 { + m.logger.Warn("No storenodes available") // Do nothing.. + return nil + } + } + + r, err := rand.Int(rand.Reader, big.NewInt(int64(pSize))) + if err != nil { + return err + } + + ms := allStorenodes[r.Int64()] + return m.setActiveStorenode(ms) +} + +func (m *StorenodeCycle) storenodeStatus(peerID peer.ID) connStatus { + m.peersMutex.RLock() + defer m.peersMutex.RUnlock() + + peer, ok := m.peers[peerID] + if !ok { + return disconnected + } + return peer.status +} + +func (m *StorenodeCycle) setActiveStorenode(peerID peer.ID) error { + m.activeStorenode = peerID + + m.StorenodeChangedEmitter.Emit(m.activeStorenode) + + storenodeStatus := m.storenodeStatus(peerID) + if storenodeStatus != connected { + m.peersMutex.Lock() + m.peers[peerID] = peerStatus{ + status: connected, + lastConnectionAttempt: time.Now(), + canConnectAfter: time.Now().Add(defaultBackoff), + } + m.peersMutex.Unlock() + + m.failedRequests[peerID] = 0 + m.logger.Info("storenode available", zap.Stringer("peerID", m.activeStorenode)) + + m.StorenodeAvailableOneshotEmitter.Emit(struct{}{}) // Maybe can be refactored away? + m.StorenodeAvailableEmitter.Emit(m.activeStorenode) + } + return nil +} + +func (m *StorenodeCycle) GetActiveStorenode() peer.ID { + m.RLock() + defer m.RUnlock() + + return m.activeStorenode +} + +func (m *StorenodeCycle) IsStorenodeAvailable(peerID peer.ID) bool { + return m.storenodeStatus(peerID) == connected +} + +func (m *StorenodeCycle) penalizeStorenode(id peer.ID) { + m.peersMutex.Lock() + defer m.peersMutex.Unlock() + pInfo, ok := m.peers[id] + if !ok { + pInfo.status = disconnected + } + + pInfo.canConnectAfter = time.Now().Add(graylistBackoff) + m.peers[id] = pInfo +} + +func (m *StorenodeCycle) verifyStorenodeStatus(ctx context.Context) { + ticker := time.NewTicker(storenodeVerificationInterval) + defer ticker.Stop() + + for { + select { + case <-ticker.C: + err := m.disconnectStorenodeIfRequired(ctx) + if err != nil { + m.logger.Error("failed to handle storenode cycle event", zap.Error(err)) + continue + } + + case <-ctx.Done(): + return + } + } +} + +func (m *StorenodeCycle) disconnectStorenodeIfRequired(ctx context.Context) error { + m.logger.Debug("wakuV2 storenode status verification") + + if m.activeStorenode == "" { + // No active storenode, find a new one + m.Cycle(ctx) + return nil + } + + // Check whether we want to disconnect the active storenode + if m.failedRequests[m.activeStorenode] >= storenodeMaxFailedRequests { + m.penalizeStorenode(m.activeStorenode) + m.StorenodeNotWorkingEmitter.Emit(struct{}{}) + + m.logger.Info("too many failed requests", zap.Stringer("storenode", m.activeStorenode)) + m.failedRequests[m.activeStorenode] = 0 + return m.connectToNewStorenodeAndWait(ctx) + } + + return nil +} + +func (m *StorenodeCycle) SetStorenodeConfigProvider(provider StorenodeConfigProvider) { + m.storenodeConfigProvider = provider +} + +func (m *StorenodeCycle) WaitForAvailableStoreNode(ctx context.Context, timeout time.Duration) bool { + // Add 1 second to timeout, because the storenode cycle has 1 second ticker, which doesn't tick on start. + // This can be improved after merging https://github.com/status-im/status-go/pull/4380. + // NOTE: https://stackoverflow.com/questions/32705582/how-to-get-time-tick-to-tick-immediately + timeout += time.Second + + ctx, cancel := context.WithTimeout(ctx, timeout) + defer cancel() + + wg := sync.WaitGroup{} + wg.Add(1) + go func() { + defer wg.Done() + for !m.IsStorenodeAvailable(m.activeStorenode) { + select { + case <-m.StorenodeAvailableOneshotEmitter.Subscribe(): + case <-ctx.Done(): + return + } + } + }() + + select { + case <-waitForWaitGroup(&wg): + case <-ctx.Done(): + } + + return m.IsStorenodeAvailable(m.activeStorenode) +} + +func waitForWaitGroup(wg *sync.WaitGroup) <-chan struct{} { + ch := make(chan struct{}) + go func() { + wg.Wait() + close(ch) + }() + return ch +} + +type storenodeTaskParameters struct { + customPeerID peer.ID +} + +type StorenodeTaskOption func(*storenodeTaskParameters) + +func WithPeerID(peerID peer.ID) StorenodeTaskOption { + return func(stp *storenodeTaskParameters) { + stp.customPeerID = peerID + } +} + +func (m *StorenodeCycle) PerformStorenodeTask(fn func() error, options ...StorenodeTaskOption) error { + params := storenodeTaskParameters{} + for _, opt := range options { + opt(¶ms) + } + + peerID := params.customPeerID + if peerID == "" { + peerID = m.GetActiveStorenode() + } + + if peerID == "" { + return errors.New("storenode not available") + } + + m.RLock() + defer m.RUnlock() + + var tries uint = 0 + for tries < storenodeMaxFailedRequests { + if params.customPeerID == "" && m.storenodeStatus(peerID) != connected { + return errors.New("storenode not available") + } + m.logger.Info("trying performing history requests", zap.Uint("try", tries), zap.Stringer("peerID", peerID)) + + // Peform request + err := fn() + if err == nil { + // Reset failed requests + m.logger.Debug("history request performed successfully", zap.Stringer("peerID", peerID)) + m.failedRequests[peerID] = 0 + return nil + } + + m.logger.Error("failed to perform history request", + zap.Stringer("peerID", peerID), + zap.Uint("tries", tries), + zap.Error(err), + ) + + tries++ + + if storeErr, ok := err.(*store.StoreError); ok { + if storeErr.Code == http.StatusTooManyRequests { + m.disconnectActiveStorenode(defaultBackoff) + return fmt.Errorf("ratelimited at storenode %s: %w", peerID, err) + } + } + + // Increment failed requests + m.failedRequests[peerID]++ + + // Change storenode + if m.failedRequests[peerID] >= storenodeMaxFailedRequests { + return errors.New("too many failed requests") + } + // Wait a couple of second not to spam + time.Sleep(2 * time.Second) + + } + return errors.New("failed to perform history request") +} diff --git a/vendor/github.com/waku-org/go-waku/waku/v2/api/history/emitters.go b/vendor/github.com/waku-org/go-waku/waku/v2/api/history/emitters.go new file mode 100644 index 00000000000..a12d4db9076 --- /dev/null +++ b/vendor/github.com/waku-org/go-waku/waku/v2/api/history/emitters.go @@ -0,0 +1,48 @@ +package history + +import "sync" + +type Emitter[T any] struct { + sync.Mutex + subscriptions []chan T +} + +func NewEmitter[T any]() *Emitter[T] { + return &Emitter[T]{} +} + +func (s *Emitter[T]) Subscribe() <-chan T { + s.Lock() + defer s.Unlock() + c := make(chan T) + s.subscriptions = append(s.subscriptions, c) + return c +} + +func (s *Emitter[T]) Emit(value T) { + s.Lock() + defer s.Unlock() + + for _, sub := range s.subscriptions { + sub <- value + } +} + +type OneShotEmitter[T any] struct { + Emitter[T] +} + +func NewOneshotEmitter[T any]() *OneShotEmitter[T] { + return &OneShotEmitter[T]{} +} + +func (s *OneShotEmitter[T]) Emit(value T) { + s.Lock() + defer s.Unlock() + + for _, subs := range s.subscriptions { + subs <- value + close(subs) + } + s.subscriptions = nil +} diff --git a/vendor/github.com/waku-org/go-waku/waku/v2/api/history/history.go b/vendor/github.com/waku-org/go-waku/waku/v2/api/history/history.go new file mode 100644 index 00000000000..e95f01a5763 --- /dev/null +++ b/vendor/github.com/waku-org/go-waku/waku/v2/api/history/history.go @@ -0,0 +1,296 @@ +package history + +import ( + "context" + "errors" + "math" + "sync" + "time" + + "github.com/ethereum/go-ethereum/common/hexutil" + "github.com/libp2p/go-libp2p/core/peer" + "github.com/waku-org/go-waku/logging" + "github.com/waku-org/go-waku/waku/v2/protocol" + "github.com/waku-org/go-waku/waku/v2/protocol/store" + "go.uber.org/zap" +) + +const maxTopicsPerRequest int = 10 +const mailserverRequestTimeout = 30 * time.Second + +type work struct { + criteria store.FilterCriteria + cursor []byte + limit uint64 +} + +type HistoryRetriever struct { + store Store + logger *zap.Logger + historyProcessor HistoryProcessor +} + +type HistoryProcessor interface { + OnEnvelope(env *protocol.Envelope, processEnvelopes bool) error + OnRequestFailed(requestID []byte, peerID peer.ID, err error) +} + +type Store interface { + Query(ctx context.Context, criteria store.FilterCriteria, opts ...store.RequestOption) (store.Result, error) +} + +func NewHistoryRetriever(store Store, historyProcessor HistoryProcessor, logger *zap.Logger) *HistoryRetriever { + return &HistoryRetriever{ + store: store, + logger: logger.Named("history-retriever"), + historyProcessor: historyProcessor, + } +} + +func (hr *HistoryRetriever) Query( + ctx context.Context, + criteria store.FilterCriteria, + storenodeID peer.ID, + pageLimit uint64, + shouldProcessNextPage func(int) (bool, uint64), + processEnvelopes bool, +) error { + logger := hr.logger.With( + logging.Timep("fromString", criteria.TimeStart), + logging.Timep("toString", criteria.TimeEnd), + zap.String("pubsubTopic", criteria.PubsubTopic), + zap.Strings("contentTopics", criteria.ContentTopicsList()), + zap.Int64p("from", criteria.TimeStart), + zap.Int64p("to", criteria.TimeEnd), + ) + + logger.Info("syncing") + + wg := sync.WaitGroup{} + workWg := sync.WaitGroup{} + workCh := make(chan work, 1000) // each batch item is split in 10 topics bunch and sent to this channel + workCompleteCh := make(chan struct{}) // once all batch items are processed, this channel is triggered + semaphore := make(chan struct{}, 3) // limit the number of concurrent queries + errCh := make(chan error) + + ctx, cancel := context.WithCancel(ctx) + defer cancel() + + // TODO: refactor this by extracting the consumer into a separate go routine. + + // Producer + wg.Add(1) + go func() { + defer func() { + logger.Debug("mailserver batch producer complete") + wg.Done() + }() + + contentTopicList := criteria.ContentTopics.ToList() + + // TODO: split into 24h batches + + allWorks := int(math.Ceil(float64(len(contentTopicList)) / float64(maxTopicsPerRequest))) + workWg.Add(allWorks) + + for i := 0; i < len(contentTopicList); i += maxTopicsPerRequest { + j := i + maxTopicsPerRequest + if j > len(contentTopicList) { + j = len(contentTopicList) + } + + select { + case <-ctx.Done(): + logger.Debug("processBatch producer - context done") + return + default: + logger.Debug("processBatch producer - creating work") + workCh <- work{ + criteria: store.FilterCriteria{ + ContentFilter: protocol.NewContentFilter(criteria.PubsubTopic, contentTopicList[i:j]...), + TimeStart: criteria.TimeStart, + TimeEnd: criteria.TimeEnd, + }, + limit: pageLimit, + } + } + } + + go func() { + workWg.Wait() + workCompleteCh <- struct{}{} + }() + + logger.Debug("processBatch producer complete") + }() + + var result error + +loop: + for { + select { + case <-ctx.Done(): + logger.Debug("processBatch cleanup - context done") + result = ctx.Err() + if errors.Is(result, context.Canceled) { + result = nil + } + break loop + case w, ok := <-workCh: + if !ok { + continue + } + + select { + case <-ctx.Done(): + return ctx.Err() + default: + // continue... + } + + logger.Debug("processBatch - received work") + + semaphore <- struct{}{} + go func(w work) { // Consumer + defer func() { + workWg.Done() + <-semaphore + }() + + queryCtx, queryCancel := context.WithTimeout(ctx, mailserverRequestTimeout) + cursor, envelopesCount, err := hr.createMessagesRequest(queryCtx, storenodeID, w.criteria, w.cursor, w.limit, true, processEnvelopes, logger) + queryCancel() + + if err != nil { + logger.Debug("failed to send request", zap.Error(err)) + errCh <- err + return + } + + processNextPage := true + nextPageLimit := pageLimit + if shouldProcessNextPage != nil { + processNextPage, nextPageLimit = shouldProcessNextPage(envelopesCount) + } + + if !processNextPage { + return + } + + // Check the cursor after calling `shouldProcessNextPage`. + // The app might use process the fetched envelopes in the callback for own needs. + if cursor == nil { + return + } + + logger.Debug("processBatch producer - creating work (cursor)") + + workWg.Add(1) + workCh <- work{ + criteria: w.criteria, + cursor: cursor, + limit: nextPageLimit, + } + }(w) + case err := <-errCh: + logger.Debug("processBatch - received error", zap.Error(err)) + cancel() // Kill go routines + return err + case <-workCompleteCh: + logger.Debug("processBatch - all jobs complete") + cancel() // Kill go routines + } + } + + wg.Wait() + + logger.Info("synced topic", zap.NamedError("hasError", result)) + + return result +} + +func (hr *HistoryRetriever) createMessagesRequest( + ctx context.Context, + peerID peer.ID, + criteria store.FilterCriteria, + cursor []byte, + limit uint64, + waitForResponse bool, + processEnvelopes bool, + logger *zap.Logger, +) (storeCursor []byte, envelopesCount int, err error) { + if waitForResponse { + resultCh := make(chan struct { + storeCursor []byte + envelopesCount int + err error + }) + + go func() { + storeCursor, envelopesCount, err = hr.requestStoreMessages(ctx, peerID, criteria, cursor, limit, processEnvelopes) + resultCh <- struct { + storeCursor []byte + envelopesCount int + err error + }{storeCursor, envelopesCount, err} + }() + + select { + case result := <-resultCh: + return result.storeCursor, result.envelopesCount, result.err + case <-ctx.Done(): + return nil, 0, ctx.Err() + } + } else { + go func() { + _, _, err = hr.requestStoreMessages(ctx, peerID, criteria, cursor, limit, false) + if err != nil { + logger.Error("failed to request store messages", zap.Error(err)) + } + }() + } + + return +} + +func (hr *HistoryRetriever) requestStoreMessages(ctx context.Context, peerID peer.ID, criteria store.FilterCriteria, cursor []byte, limit uint64, processEnvelopes bool) ([]byte, int, error) { + requestID := protocol.GenerateRequestID() + logger := hr.logger.With(zap.String("requestID", hexutil.Encode(requestID)), zap.Stringer("peerID", peerID)) + + opts := []store.RequestOption{ + store.WithPaging(false, limit), + store.WithRequestID(requestID), + store.WithPeer(peerID), + store.WithCursor(cursor)} + + logger.Debug("store.query", + logging.Timep("startTime", criteria.TimeStart), + logging.Timep("endTime", criteria.TimeEnd), + zap.Strings("contentTopics", criteria.ContentTopics.ToList()), + zap.String("pubsubTopic", criteria.PubsubTopic), + zap.String("cursor", hexutil.Encode(cursor)), + ) + + queryStart := time.Now() + result, err := hr.store.Query(ctx, criteria, opts...) + queryDuration := time.Since(queryStart) + if err != nil { + logger.Error("error querying storenode", zap.Error(err)) + + hr.historyProcessor.OnRequestFailed(requestID, peerID, err) + + return nil, 0, err + } + + messages := result.Messages() + envelopesCount := len(messages) + logger.Debug("store.query response", zap.Duration("queryDuration", queryDuration), zap.Int("numMessages", envelopesCount), zap.Bool("hasCursor", result.IsComplete() && result.Cursor() != nil)) + for _, mkv := range messages { + envelope := protocol.NewEnvelope(mkv.Message, mkv.Message.GetTimestamp(), mkv.GetPubsubTopic()) + err := hr.historyProcessor.OnEnvelope(envelope, processEnvelopes) + if err != nil { + return nil, 0, err + } + } + return result.Cursor(), envelopesCount, nil +} diff --git a/vendor/github.com/waku-org/go-waku/waku/v2/api/history/sort.go b/vendor/github.com/waku-org/go-waku/waku/v2/api/history/sort.go new file mode 100644 index 00000000000..22e94c571f9 --- /dev/null +++ b/vendor/github.com/waku-org/go-waku/waku/v2/api/history/sort.go @@ -0,0 +1,32 @@ +package history + +import ( + "time" + + "github.com/libp2p/go-libp2p/core/peer" +) + +type SortedStorenode struct { + Storenode peer.ID + RTT time.Duration + CanConnectAfter time.Time +} + +type byRTTMsAndCanConnectBefore []SortedStorenode + +func (s byRTTMsAndCanConnectBefore) Len() int { + return len(s) +} + +func (s byRTTMsAndCanConnectBefore) Swap(i, j int) { + s[i], s[j] = s[j], s[i] +} + +func (s byRTTMsAndCanConnectBefore) Less(i, j int) bool { + // Slightly inaccurate as time sensitive sorting, but it does not matter so much + now := time.Now() + if s[i].CanConnectAfter.Before(now) && s[j].CanConnectAfter.Before(now) { + return s[i].RTT < s[j].RTT + } + return s[i].CanConnectAfter.Before(s[j].CanConnectAfter) +} diff --git a/vendor/github.com/waku-org/go-waku/waku/v2/api/missing/missing_messages.go b/vendor/github.com/waku-org/go-waku/waku/v2/api/missing/missing_messages.go index ca8b63fb79b..095e32419f7 100644 --- a/vendor/github.com/waku-org/go-waku/waku/v2/api/missing/missing_messages.go +++ b/vendor/github.com/waku-org/go-waku/waku/v2/api/missing/missing_messages.go @@ -178,7 +178,7 @@ func (m *MissingMessageVerifier) fetchHistory(c chan<- *protocol.Envelope, inter } } -func (m *MissingMessageVerifier) storeQueryWithRetry(ctx context.Context, queryFunc func(ctx context.Context) (*store.Result, error), logger *zap.Logger, logMsg string) (*store.Result, error) { +func (m *MissingMessageVerifier) storeQueryWithRetry(ctx context.Context, queryFunc func(ctx context.Context) (store.Result, error), logger *zap.Logger, logMsg string) (store.Result, error) { retry := true count := 1 for retry && count <= m.params.maxAttemptsToRetrieveHistory { @@ -212,7 +212,7 @@ func (m *MissingMessageVerifier) fetchMessagesBatch(c chan<- *protocol.Envelope, logging.Epoch("to", now), ) - result, err := m.storeQueryWithRetry(interest.ctx, func(ctx context.Context) (*store.Result, error) { + result, err := m.storeQueryWithRetry(interest.ctx, func(ctx context.Context) (store.Result, error) { return m.store.Query(ctx, store.FilterCriteria{ ContentFilter: protocol.NewContentFilter(interest.contentFilter.PubsubTopic, contentTopics[batchFrom:batchTo]...), TimeStart: proto.Int64(interest.lastChecked.Add(-m.params.delay).UnixNano()), @@ -243,7 +243,7 @@ func (m *MissingMessageVerifier) fetchMessagesBatch(c chan<- *protocol.Envelope, missingHashes = append(missingHashes, hash) } - result, err = m.storeQueryWithRetry(interest.ctx, func(ctx context.Context) (*store.Result, error) { + result, err = m.storeQueryWithRetry(interest.ctx, func(ctx context.Context) (store.Result, error) { if err = result.Next(ctx); err != nil { return nil, err } @@ -282,7 +282,7 @@ func (m *MissingMessageVerifier) fetchMessagesBatch(c chan<- *protocol.Envelope, defer utils.LogOnPanic() defer wg.Wait() - result, err := m.storeQueryWithRetry(interest.ctx, func(ctx context.Context) (*store.Result, error) { + result, err := m.storeQueryWithRetry(interest.ctx, func(ctx context.Context) (store.Result, error) { queryCtx, cancel := context.WithTimeout(ctx, m.params.storeQueryTimeout) defer cancel() return m.store.QueryByHash(queryCtx, messageHashes, store.WithPeer(interest.peerID), store.WithPaging(false, maxMsgHashesPerRequest)) @@ -303,7 +303,7 @@ func (m *MissingMessageVerifier) fetchMessagesBatch(c chan<- *protocol.Envelope, } } - result, err = m.storeQueryWithRetry(interest.ctx, func(ctx context.Context) (*store.Result, error) { + result, err = m.storeQueryWithRetry(interest.ctx, func(ctx context.Context) (store.Result, error) { if err = result.Next(ctx); err != nil { return nil, err } diff --git a/vendor/github.com/waku-org/go-waku/waku/v2/api/publish/message_check.go b/vendor/github.com/waku-org/go-waku/waku/v2/api/publish/message_check.go index c6df0f29a60..0591657401e 100644 --- a/vendor/github.com/waku-org/go-waku/waku/v2/api/publish/message_check.go +++ b/vendor/github.com/waku-org/go-waku/waku/v2/api/publish/message_check.go @@ -8,8 +8,8 @@ import ( "github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/common/hexutil" - "github.com/libp2p/go-libp2p/core/peer" apicommon "github.com/waku-org/go-waku/waku/v2/api/common" + "github.com/waku-org/go-waku/waku/v2/api/history" "github.com/waku-org/go-waku/waku/v2/protocol" "github.com/waku-org/go-waku/waku/v2/protocol/pb" "github.com/waku-org/go-waku/waku/v2/protocol/store" @@ -29,7 +29,6 @@ type ISentCheck interface { Start() Add(topic string, messageID common.Hash, sentTime uint32) DeleteByMessageIDs(messageIDs []common.Hash) - SetStorePeerID(peerID peer.ID) } // MessageSentCheck tracks the outgoing messages and check against store node @@ -38,11 +37,11 @@ type ISentCheck interface { type MessageSentCheck struct { messageIDs map[string]map[common.Hash]uint32 messageIDsMu sync.RWMutex - storePeerID peer.ID messageStoredChan chan common.Hash messageExpiredChan chan common.Hash ctx context.Context store *store.WakuStore + storenodeCycle *history.StorenodeCycle timesource timesource.Timesource logger *zap.Logger maxHashQueryLength uint64 @@ -53,7 +52,7 @@ type MessageSentCheck struct { } // NewMessageSentCheck creates a new instance of MessageSentCheck with default parameters -func NewMessageSentCheck(ctx context.Context, store *store.WakuStore, timesource timesource.Timesource, msgStoredChan chan common.Hash, msgExpiredChan chan common.Hash, logger *zap.Logger) *MessageSentCheck { +func NewMessageSentCheck(ctx context.Context, store *store.WakuStore, cycle *history.StorenodeCycle, timesource timesource.Timesource, msgStoredChan chan common.Hash, msgExpiredChan chan common.Hash, logger *zap.Logger) *MessageSentCheck { return &MessageSentCheck{ messageIDs: make(map[string]map[common.Hash]uint32), messageIDsMu: sync.RWMutex{}, @@ -61,6 +60,7 @@ func NewMessageSentCheck(ctx context.Context, store *store.WakuStore, timesource messageExpiredChan: msgExpiredChan, ctx: ctx, store: store, + storenodeCycle: cycle, timesource: timesource, logger: logger, maxHashQueryLength: DefaultMaxHashQueryLength, @@ -139,11 +139,6 @@ func (m *MessageSentCheck) DeleteByMessageIDs(messageIDs []common.Hash) { } } -// SetStorePeerID sets the peer id of store node -func (m *MessageSentCheck) SetStorePeerID(peerID peer.ID) { - m.storePeerID = peerID -} - // Start checks if the tracked outgoing messages are stored periodically func (m *MessageSentCheck) Start() { defer utils.LogOnPanic() @@ -211,7 +206,7 @@ func (m *MessageSentCheck) Start() { } func (m *MessageSentCheck) messageHashBasedQuery(ctx context.Context, hashes []common.Hash, relayTime []uint32, pubsubTopic string) []common.Hash { - selectedPeer := m.storePeerID + selectedPeer := m.storenodeCycle.GetActiveStorenode() if selectedPeer == "" { m.logger.Error("no store peer id available", zap.String("pubsubTopic", pubsubTopic)) return []common.Hash{} diff --git a/vendor/github.com/waku-org/go-waku/waku/v2/api/publish/message_sender.go b/vendor/github.com/waku-org/go-waku/waku/v2/api/publish/message_sender.go index 479d894ad58..c1e9a4ca783 100644 --- a/vendor/github.com/waku-org/go-waku/waku/v2/api/publish/message_sender.go +++ b/vendor/github.com/waku-org/go-waku/waku/v2/api/publish/message_sender.go @@ -6,7 +6,6 @@ import ( "time" "github.com/ethereum/go-ethereum/common" - "github.com/libp2p/go-libp2p/core/peer" "github.com/waku-org/go-waku/waku/v2/protocol" "github.com/waku-org/go-waku/waku/v2/protocol/lightpush" "github.com/waku-org/go-waku/waku/v2/protocol/relay" @@ -162,9 +161,3 @@ func (ms *MessageSender) MessagesDelivered(messageIDs []common.Hash) { ms.messageSentCheck.DeleteByMessageIDs(messageIDs) } } - -func (ms *MessageSender) SetStorePeerID(peerID peer.ID) { - if ms.messageSentCheck != nil { - ms.messageSentCheck.SetStorePeerID(peerID) - } -} diff --git a/vendor/github.com/waku-org/go-waku/waku/v2/protocol/store/client.go b/vendor/github.com/waku-org/go-waku/waku/v2/protocol/store/client.go index f7427b9790f..090ef8f0497 100644 --- a/vendor/github.com/waku-org/go-waku/waku/v2/protocol/store/client.go +++ b/vendor/github.com/waku-org/go-waku/waku/v2/protocol/store/client.go @@ -50,8 +50,8 @@ type StoreError struct { } // NewStoreError creates a new instance of StoreError -func NewStoreError(code int, message string) StoreError { - return StoreError{ +func NewStoreError(code int, message string) *StoreError { + return &StoreError{ Code: code, Message: message, } @@ -99,7 +99,7 @@ func (s *WakuStore) SetHost(h host.Host) { // Request is used to send a store query. This function requires understanding how to prepare a store query // and most of the time you can use `Query`, `QueryByHash` and `Exists` instead, as they provide // a simpler API -func (s *WakuStore) Request(ctx context.Context, criteria Criteria, opts ...RequestOption) (*Result, error) { +func (s *WakuStore) Request(ctx context.Context, criteria Criteria, opts ...RequestOption) (Result, error) { params := new(Parameters) optList := DefaultOptions() @@ -182,7 +182,7 @@ func (s *WakuStore) Request(ctx context.Context, criteria Criteria, opts ...Requ return nil, err } - result := &Result{ + result := &resultImpl{ store: s, messages: response.Messages, storeRequest: storeRequest, @@ -195,12 +195,12 @@ func (s *WakuStore) Request(ctx context.Context, criteria Criteria, opts ...Requ } // Query retrieves all the messages that match a criteria. Use the options to indicate whether to return the message themselves or not. -func (s *WakuStore) Query(ctx context.Context, criteria FilterCriteria, opts ...RequestOption) (*Result, error) { +func (s *WakuStore) Query(ctx context.Context, criteria FilterCriteria, opts ...RequestOption) (Result, error) { return s.Request(ctx, criteria, opts...) } // Query retrieves all the messages with specific message hashes -func (s *WakuStore) QueryByHash(ctx context.Context, messageHashes []wpb.MessageHash, opts ...RequestOption) (*Result, error) { +func (s *WakuStore) QueryByHash(ctx context.Context, messageHashes []wpb.MessageHash, opts ...RequestOption) (Result, error) { return s.Request(ctx, MessageHashCriteria{messageHashes}, opts...) } @@ -214,17 +214,17 @@ func (s *WakuStore) Exists(ctx context.Context, messageHash wpb.MessageHash, opt return false, err } - return len(result.messages) != 0, nil + return len(result.Messages()) != 0, nil } -func (s *WakuStore) next(ctx context.Context, r *Result, opts ...RequestOption) (*Result, error) { +func (s *WakuStore) next(ctx context.Context, r Result, opts ...RequestOption) (*resultImpl, error) { if r.IsComplete() { - return &Result{ + return &resultImpl{ store: s, messages: nil, cursor: nil, - storeRequest: r.storeRequest, - storeResponse: r.storeResponse, + storeRequest: r.Query(), + storeResponse: r.Response(), peerID: r.PeerID(), }, nil } @@ -240,7 +240,7 @@ func (s *WakuStore) next(ctx context.Context, r *Result, opts ...RequestOption) } } - storeRequest := proto.Clone(r.storeRequest).(*pb.StoreQueryRequest) + storeRequest := proto.Clone(r.Query()).(*pb.StoreQueryRequest) storeRequest.RequestId = hex.EncodeToString(protocol.GenerateRequestID()) storeRequest.PaginationCursor = r.Cursor() @@ -249,7 +249,7 @@ func (s *WakuStore) next(ctx context.Context, r *Result, opts ...RequestOption) return nil, err } - result := &Result{ + result := &resultImpl{ store: s, messages: response.Messages, storeRequest: storeRequest, @@ -317,7 +317,7 @@ func (s *WakuStore) queryFrom(ctx context.Context, storeRequest *pb.StoreQueryRe if storeResponse.GetStatusCode() != ok { err := NewStoreError(int(storeResponse.GetStatusCode()), storeResponse.GetStatusDesc()) - return nil, &err + return nil, err } return storeResponse, nil } diff --git a/vendor/github.com/waku-org/go-waku/waku/v2/protocol/store/options.go b/vendor/github.com/waku-org/go-waku/waku/v2/protocol/store/options.go index b8deba47cdf..facb3f54f8c 100644 --- a/vendor/github.com/waku-org/go-waku/waku/v2/protocol/store/options.go +++ b/vendor/github.com/waku-org/go-waku/waku/v2/protocol/store/options.go @@ -22,6 +22,10 @@ type Parameters struct { skipRatelimit bool } +func (p *Parameters) Cursor() []byte { + return p.cursor +} + type RequestOption func(*Parameters) error // WithPeer is an option used to specify the peerID to request the message history. diff --git a/vendor/github.com/waku-org/go-waku/waku/v2/protocol/store/result.go b/vendor/github.com/waku-org/go-waku/waku/v2/protocol/store/result.go index 604d6453c2d..f5f70106da1 100644 --- a/vendor/github.com/waku-org/go-waku/waku/v2/protocol/store/result.go +++ b/vendor/github.com/waku-org/go-waku/waku/v2/protocol/store/result.go @@ -8,7 +8,17 @@ import ( ) // Result represents a valid response from a store node -type Result struct { +type Result interface { + Cursor() []byte + IsComplete() bool + PeerID() peer.ID + Query() *pb.StoreQueryRequest + Response() *pb.StoreQueryResponse + Next(ctx context.Context, opts ...RequestOption) error + Messages() []*pb.WakuMessageKeyValue +} + +type resultImpl struct { done bool messages []*pb.WakuMessageKeyValue @@ -19,27 +29,27 @@ type Result struct { peerID peer.ID } -func (r *Result) Cursor() []byte { +func (r *resultImpl) Cursor() []byte { return r.cursor } -func (r *Result) IsComplete() bool { +func (r *resultImpl) IsComplete() bool { return r.done } -func (r *Result) PeerID() peer.ID { +func (r *resultImpl) PeerID() peer.ID { return r.peerID } -func (r *Result) Query() *pb.StoreQueryRequest { +func (r *resultImpl) Query() *pb.StoreQueryRequest { return r.storeRequest } -func (r *Result) Response() *pb.StoreQueryResponse { +func (r *resultImpl) Response() *pb.StoreQueryResponse { return r.storeResponse } -func (r *Result) Next(ctx context.Context, opts ...RequestOption) error { +func (r *resultImpl) Next(ctx context.Context, opts ...RequestOption) error { if r.cursor == nil { r.done = true r.messages = nil @@ -57,6 +67,6 @@ func (r *Result) Next(ctx context.Context, opts ...RequestOption) error { return nil } -func (r *Result) Messages() []*pb.WakuMessageKeyValue { +func (r *resultImpl) Messages() []*pb.WakuMessageKeyValue { return r.messages } diff --git a/vendor/modules.txt b/vendor/modules.txt index 897a583d5da..4af35ef32b6 100644 --- a/vendor/modules.txt +++ b/vendor/modules.txt @@ -1015,13 +1015,14 @@ github.com/waku-org/go-discover/discover/v5wire github.com/waku-org/go-libp2p-rendezvous github.com/waku-org/go-libp2p-rendezvous/db github.com/waku-org/go-libp2p-rendezvous/pb -# github.com/waku-org/go-waku v0.8.1-0.20241004054019-0ed94ce0b1cb +# github.com/waku-org/go-waku v0.8.1-0.20241014185851-76275f6fb835 ## explicit; go 1.21 github.com/waku-org/go-waku/logging github.com/waku-org/go-waku/tests github.com/waku-org/go-waku/waku/persistence github.com/waku-org/go-waku/waku/v2/api/common github.com/waku-org/go-waku/waku/v2/api/filter +github.com/waku-org/go-waku/waku/v2/api/history github.com/waku-org/go-waku/waku/v2/api/missing github.com/waku-org/go-waku/waku/v2/api/publish github.com/waku-org/go-waku/waku/v2/discv5 diff --git a/wakuv2/history_processor_wrapper.go b/wakuv2/history_processor_wrapper.go new file mode 100644 index 00000000000..25cee9e8f88 --- /dev/null +++ b/wakuv2/history_processor_wrapper.go @@ -0,0 +1,25 @@ +package wakuv2 + +import ( + "github.com/libp2p/go-libp2p/core/peer" + + "github.com/status-im/status-go/wakuv2/common" + "github.com/waku-org/go-waku/waku/v2/api/history" + "github.com/waku-org/go-waku/waku/v2/protocol" +) + +type HistoryProcessorWrapper struct { + waku *Waku +} + +func NewHistoryProcessorWrapper(waku *Waku) history.HistoryProcessor { + return &HistoryProcessorWrapper{waku} +} + +func (hr *HistoryProcessorWrapper) OnEnvelope(env *protocol.Envelope, processEnvelopes bool) error { + return hr.waku.OnNewEnvelopes(env, common.StoreMessageType, processEnvelopes) +} + +func (hr *HistoryProcessorWrapper) OnRequestFailed(requestID []byte, peerID peer.ID, err error) { + hr.waku.onHistoricMessagesRequestFailed(requestID, peerID, err) +} diff --git a/wakuv2/waku.go b/wakuv2/waku.go index 43c0626bc91..3e2319752f3 100644 --- a/wakuv2/waku.go +++ b/wakuv2/waku.go @@ -54,9 +54,9 @@ import ( "github.com/libp2p/go-libp2p" pubsub "github.com/libp2p/go-libp2p-pubsub" "github.com/libp2p/go-libp2p/core/metrics" - "github.com/libp2p/go-libp2p/p2p/protocol/ping" filterapi "github.com/waku-org/go-waku/waku/v2/api/filter" + "github.com/waku-org/go-waku/waku/v2/api/history" "github.com/waku-org/go-waku/waku/v2/api/missing" "github.com/waku-org/go-waku/waku/v2/api/publish" "github.com/waku-org/go-waku/waku/v2/dnsdisc" @@ -167,6 +167,9 @@ type Waku struct { onlineChecker *onlinechecker.DefaultOnlineChecker state connection.State + StorenodeCycle *history.StorenodeCycle + HistoryRetriever *history.HistoryRetriever + logger *zap.Logger // NTP Synced timesource @@ -352,6 +355,7 @@ func New(nodeKey *ecdsa.PrivateKey, fleet string, cfg *Config, logger *zap.Logge } waku.options = opts + waku.logger.Info("setup the go-waku node successfully") return waku, nil @@ -1025,61 +1029,6 @@ func (w *Waku) ConfirmMessageDelivered(hashes []gethcommon.Hash) { w.messageSender.MessagesDelivered(hashes) } -func (w *Waku) SetStorePeerID(peerID peer.ID) { - w.messageSender.SetStorePeerID(peerID) -} - -func (w *Waku) Query(ctx context.Context, peerID peer.ID, query store.FilterCriteria, cursor []byte, opts []store.RequestOption, processEnvelopes bool) ([]byte, int, error) { - requestID := protocol.GenerateRequestID() - - opts = append(opts, - store.WithRequestID(requestID), - store.WithPeer(peerID), - store.WithCursor(cursor)) - - logger := w.logger.With(zap.String("requestID", hexutil.Encode(requestID)), zap.Stringer("peerID", peerID)) - - logger.Debug("store.query", - logutils.WakuMessageTimestamp("startTime", query.TimeStart), - logutils.WakuMessageTimestamp("endTime", query.TimeEnd), - zap.Strings("contentTopics", query.ContentTopics.ToList()), - zap.String("pubsubTopic", query.PubsubTopic), - zap.String("cursor", hexutil.Encode(cursor)), - ) - - queryStart := time.Now() - result, err := w.node.Store().Query(ctx, query, opts...) - queryDuration := time.Since(queryStart) - if err != nil { - logger.Error("error querying storenode", zap.Error(err)) - - if w.onHistoricMessagesRequestFailed != nil { - w.onHistoricMessagesRequestFailed(requestID, peerID, err) - } - return nil, 0, err - } - - messages := result.Messages() - envelopesCount := len(messages) - w.logger.Debug("store.query response", zap.Duration("queryDuration", queryDuration), zap.Int("numMessages", envelopesCount), zap.Bool("hasCursor", result.IsComplete() && result.Cursor() != nil)) - for _, mkv := range messages { - msg := mkv.Message - - // Temporarily setting RateLimitProof to nil so it matches the WakuMessage protobuffer we are sending - // See https://github.com/vacp2p/rfc/issues/563 - mkv.Message.RateLimitProof = nil - - envelope := protocol.NewEnvelope(msg, msg.GetTimestamp(), query.PubsubTopic) - - err = w.OnNewEnvelopes(envelope, common.StoreMessageType, processEnvelopes) - if err != nil { - return nil, 0, err - } - } - - return result.Cursor(), envelopesCount, nil -} - // OnNewEnvelope is an interface from Waku FilterManager API that gets invoked when any new message is received by Filter. func (w *Waku) OnNewEnvelope(env *protocol.Envelope) error { return w.OnNewEnvelopes(env, common.RelayedMessageType, false) @@ -1103,6 +1052,11 @@ func (w *Waku) Start() error { return fmt.Errorf("failed to start go-waku node: %v", err) } + w.StorenodeCycle = history.NewStorenodeCycle(w.logger) + w.HistoryRetriever = history.NewHistoryRetriever(w.node.Store(), NewHistoryProcessorWrapper(w), w.logger) + + w.StorenodeCycle.Start(w.ctx, w.node.Host()) + w.logger.Info("WakuV2 PeerID", zap.Stringer("id", w.node.Host().ID())) w.discoverAndConnectPeers() @@ -1322,7 +1276,7 @@ func (w *Waku) startMessageSender() error { if w.cfg.EnableStoreConfirmationForMessagesSent { msgStoredChan := make(chan gethcommon.Hash, 1000) msgExpiredChan := make(chan gethcommon.Hash, 1000) - messageSentCheck := publish.NewMessageSentCheck(w.ctx, w.node.Store(), w.node.Timesource(), msgStoredChan, msgExpiredChan, w.logger) + messageSentCheck := publish.NewMessageSentCheck(w.ctx, w.node.Store(), w.StorenodeCycle, w.node.Timesource(), msgStoredChan, msgExpiredChan, w.logger) sender.WithMessageSentCheck(messageSentCheck) w.wg.Add(1) @@ -1924,19 +1878,6 @@ func (w *Waku) PeerID() peer.ID { return w.node.Host().ID() } -func (w *Waku) PingPeer(ctx context.Context, peerID peer.ID) (time.Duration, error) { - pingResultCh := ping.Ping(ctx, w.node.Host(), peerID) - select { - case <-ctx.Done(): - return 0, ctx.Err() - case r := <-pingResultCh: - if r.Error != nil { - return 0, r.Error - } - return r.RTT, nil - } -} - func (w *Waku) Peerstore() peerstore.Peerstore { return w.node.Host().Peerstore() } diff --git a/wakuv2/waku_test.go b/wakuv2/waku_test.go index 9792e50828a..36b05d8fbde 100644 --- a/wakuv2/waku_test.go +++ b/wakuv2/waku_test.go @@ -282,19 +282,16 @@ func TestBasicWakuV2(t *testing.T) { b.InitialInterval = 500 * time.Millisecond } err = tt.RetryWithBackOff(func() error { - _, envelopeCount, err := w.Query( + result, err := w.node.Store().Query( context.Background(), - storeNode.PeerID, store.FilterCriteria{ ContentFilter: protocol.NewContentFilter(config.DefaultShardPubsubTopic, contentTopic.ContentTopic()), TimeStart: proto.Int64((timestampInSeconds - int64(marginInSeconds)) * int64(time.Second)), TimeEnd: proto.Int64((timestampInSeconds + int64(marginInSeconds)) * int64(time.Second)), }, - nil, - nil, - false, + store.WithPeer(storeNode.PeerID), ) - if err != nil || envelopeCount == 0 { + if err != nil || len(result.Messages()) == 0 { // in case of failure extend timestamp margin up to 40secs if marginInSeconds < 40 { marginInSeconds += 5 @@ -589,20 +586,17 @@ func TestWakuV2Store(t *testing.T) { timestampInSeconds := msgTimestamp / int64(time.Second) marginInSeconds := 5 // Query the second node's store for the message - _, envelopeCount, err := w1.Query( + result, err := w1.node.Store().Query( context.Background(), - w2.node.Host().ID(), store.FilterCriteria{ TimeStart: proto.Int64((timestampInSeconds - int64(marginInSeconds)) * int64(time.Second)), TimeEnd: proto.Int64((timestampInSeconds + int64(marginInSeconds)) * int64(time.Second)), ContentFilter: protocol.NewContentFilter(config1.DefaultShardPubsubTopic, contentTopic.ContentTopic()), }, - nil, - nil, - false, + store.WithPeer(w2.node.Host().ID()), ) require.NoError(t, err) - require.True(t, envelopeCount > 0, "no messages received from store node") + require.True(t, len(result.Messages()) > 0, "no messages received from store node") } func waitForPeerConnection(t *testing.T, peerID peer.ID, peerCh chan peer.IDSlice) { From d657edffc431d3dc8dd3bddc5e4104b1cdaaa5ea Mon Sep 17 00:00:00 2001 From: Richard Ramos Date: Tue, 15 Oct 2024 19:18:27 -0400 Subject: [PATCH 02/14] refactor_: decouple go-waku api from go-waku wakunode --- go.mod | 2 +- go.sum | 4 +- .../go-waku/waku/v2/api/common/result.go | 17 +++++++ .../waku/v2/api/missing/default_requestor.go | 33 ++++++++++++ .../waku/v2/api/missing/missing_messages.go | 50 +++++++++++-------- .../waku/v2/api/publish/default_publisher.go | 50 +++++++++++++++++++ .../waku/v2/api/publish/default_verifier.go | 39 +++++++++++++++ .../waku/v2/api/publish/message_check.go | 26 +++++----- .../waku/v2/api/publish/message_sender.go | 42 +++++++++------- vendor/modules.txt | 2 +- wakuv2/waku.go | 6 +-- 11 files changed, 213 insertions(+), 58 deletions(-) create mode 100644 vendor/github.com/waku-org/go-waku/waku/v2/api/common/result.go create mode 100644 vendor/github.com/waku-org/go-waku/waku/v2/api/missing/default_requestor.go create mode 100644 vendor/github.com/waku-org/go-waku/waku/v2/api/publish/default_publisher.go create mode 100644 vendor/github.com/waku-org/go-waku/waku/v2/api/publish/default_verifier.go diff --git a/go.mod b/go.mod index e6107a66175..5123db98df9 100644 --- a/go.mod +++ b/go.mod @@ -95,7 +95,7 @@ require ( github.com/schollz/peerdiscovery v1.7.0 github.com/siphiuel/lc-proxy-wrapper v0.0.0-20230516150924-246507cee8c7 github.com/urfave/cli/v2 v2.27.2 - github.com/waku-org/go-waku v0.8.1-0.20241014185851-76275f6fb835 + github.com/waku-org/go-waku v0.8.1-0.20241015194815-37f936d74705 github.com/wk8/go-ordered-map/v2 v2.1.7 github.com/yeqown/go-qrcode/v2 v2.2.1 github.com/yeqown/go-qrcode/writer/standard v1.2.1 diff --git a/go.sum b/go.sum index 4fefc668975..4dc19638627 100644 --- a/go.sum +++ b/go.sum @@ -2136,8 +2136,8 @@ github.com/waku-org/go-libp2p-pubsub v0.12.0-gowaku.0.20240823143342-b0f2429ca27 github.com/waku-org/go-libp2p-pubsub v0.12.0-gowaku.0.20240823143342-b0f2429ca27f/go.mod h1:Oi0zw9aw8/Y5GC99zt+Ef2gYAl+0nZlwdJonDyOz/sE= github.com/waku-org/go-libp2p-rendezvous v0.0.0-20240110193335-a67d1cc760a0 h1:R4YYx2QamhBRl/moIxkDCNW+OP7AHbyWLBygDc/xIMo= github.com/waku-org/go-libp2p-rendezvous v0.0.0-20240110193335-a67d1cc760a0/go.mod h1:EhZP9fee0DYjKH/IOQvoNSy1tSHp2iZadsHGphcAJgY= -github.com/waku-org/go-waku v0.8.1-0.20241014185851-76275f6fb835 h1:Vp6BhXiDEilmchHy8OLMZVhugudsnvveNkAKD5nhAGk= -github.com/waku-org/go-waku v0.8.1-0.20241014185851-76275f6fb835/go.mod h1:1BRnyg2mQ2aBNLTBaPq6vEvobzywGykPOhGQFbHGf74= +github.com/waku-org/go-waku v0.8.1-0.20241015194815-37f936d74705 h1:i1vIOgWIQn0jing5jxqO9rG676jPoShiTLknE/pRaWc= +github.com/waku-org/go-waku v0.8.1-0.20241015194815-37f936d74705/go.mod h1:1BRnyg2mQ2aBNLTBaPq6vEvobzywGykPOhGQFbHGf74= github.com/waku-org/go-zerokit-rln v0.1.14-0.20240102145250-fa738c0bdf59 h1:jisj+OCI6QydLtFq3Pyhu49wl9ytPN7oAHjMfepHDrA= github.com/waku-org/go-zerokit-rln v0.1.14-0.20240102145250-fa738c0bdf59/go.mod h1:1PdBdPzyTaKt3VnpAHk3zj+r9dXPFOr3IHZP9nFle6E= github.com/waku-org/go-zerokit-rln-apple v0.0.0-20230916172309-ee0ee61dde2b h1:KgZVhsLkxsj5gb/FfndSCQu6VYwALrCOgYI3poR95yE= diff --git a/vendor/github.com/waku-org/go-waku/waku/v2/api/common/result.go b/vendor/github.com/waku-org/go-waku/waku/v2/api/common/result.go new file mode 100644 index 00000000000..d8ce175a9ec --- /dev/null +++ b/vendor/github.com/waku-org/go-waku/waku/v2/api/common/result.go @@ -0,0 +1,17 @@ +package common + +import ( + "context" + + "github.com/libp2p/go-libp2p/core/peer" + "github.com/waku-org/go-waku/waku/v2/protocol/store" + "github.com/waku-org/go-waku/waku/v2/protocol/store/pb" +) + +type StoreRequestResult interface { + Cursor() []byte + IsComplete() bool + PeerID() peer.ID + Next(ctx context.Context, opts ...store.RequestOption) error // TODO: see how to decouple store.RequestOption + Messages() []*pb.WakuMessageKeyValue +} diff --git a/vendor/github.com/waku-org/go-waku/waku/v2/api/missing/default_requestor.go b/vendor/github.com/waku-org/go-waku/waku/v2/api/missing/default_requestor.go new file mode 100644 index 00000000000..248d61c6dc0 --- /dev/null +++ b/vendor/github.com/waku-org/go-waku/waku/v2/api/missing/default_requestor.go @@ -0,0 +1,33 @@ +package missing + +import ( + "context" + + "github.com/libp2p/go-libp2p/core/peer" + "github.com/waku-org/go-waku/waku/v2/api/common" + "github.com/waku-org/go-waku/waku/v2/protocol" + "github.com/waku-org/go-waku/waku/v2/protocol/pb" + "github.com/waku-org/go-waku/waku/v2/protocol/store" +) + +func NewDefaultStorenodeRequestor(store *store.WakuStore) StorenodeRequestor { + return &defaultStorenodeRequestor{ + store: store, + } +} + +type defaultStorenodeRequestor struct { + store *store.WakuStore +} + +func (d *defaultStorenodeRequestor) GetMessagesByHash(ctx context.Context, peerID peer.ID, pageSize uint64, messageHashes []pb.MessageHash) (common.StoreRequestResult, error) { + return d.store.QueryByHash(ctx, messageHashes, store.WithPeer(peerID), store.WithPaging(false, pageSize)) +} + +func (d *defaultStorenodeRequestor) QueryWithCriteria(ctx context.Context, peerID peer.ID, pageSize uint64, pubsubTopic string, contentTopics []string, from *int64, to *int64) (common.StoreRequestResult, error) { + return d.store.Query(ctx, store.FilterCriteria{ + ContentFilter: protocol.NewContentFilter(pubsubTopic, contentTopics...), + TimeStart: from, + TimeEnd: to, + }, store.WithPeer(peerID), store.WithPaging(false, pageSize), store.IncludeData(false)) +} diff --git a/vendor/github.com/waku-org/go-waku/waku/v2/api/missing/missing_messages.go b/vendor/github.com/waku-org/go-waku/waku/v2/api/missing/missing_messages.go index 095e32419f7..1af991eb51f 100644 --- a/vendor/github.com/waku-org/go-waku/waku/v2/api/missing/missing_messages.go +++ b/vendor/github.com/waku-org/go-waku/waku/v2/api/missing/missing_messages.go @@ -11,9 +11,9 @@ import ( "github.com/libp2p/go-libp2p/core/peer" "github.com/waku-org/go-waku/logging" + "github.com/waku-org/go-waku/waku/v2/api/common" "github.com/waku-org/go-waku/waku/v2/protocol" "github.com/waku-org/go-waku/waku/v2/protocol/pb" - "github.com/waku-org/go-waku/waku/v2/protocol/store" "github.com/waku-org/go-waku/waku/v2/timesource" "github.com/waku-org/go-waku/waku/v2/utils" "go.uber.org/zap" @@ -22,6 +22,7 @@ import ( const maxContentTopicsPerRequest = 10 const maxMsgHashesPerRequest = 50 +const messageFetchPageSize = 100 // MessageTracker should keep track of messages it has seen before and // provide a way to determine whether a message exists or not. This @@ -30,25 +31,30 @@ type MessageTracker interface { MessageExists(pb.MessageHash) (bool, error) } +type StorenodeRequestor interface { + GetMessagesByHash(ctx context.Context, peerID peer.ID, pageSize uint64, messageHashes []pb.MessageHash) (common.StoreRequestResult, error) + QueryWithCriteria(ctx context.Context, peerID peer.ID, pageSize uint64, pubsubTopic string, contentTopics []string, from *int64, to *int64) (common.StoreRequestResult, error) +} + // MissingMessageVerifier is used to periodically retrieve missing messages from store nodes that have some specific criteria type MissingMessageVerifier struct { ctx context.Context params missingMessageVerifierParams - messageTracker MessageTracker + storenodeRequestor StorenodeRequestor + messageTracker MessageTracker criteriaInterest map[string]criteriaInterest // Track message verification requests and when was the last time a pubsub topic was verified for missing messages criteriaInterestMu sync.RWMutex C <-chan *protocol.Envelope - store *store.WakuStore timesource timesource.Timesource logger *zap.Logger } // NewMissingMessageVerifier creates an instance of a MissingMessageVerifier -func NewMissingMessageVerifier(store *store.WakuStore, messageTracker MessageTracker, timesource timesource.Timesource, logger *zap.Logger, options ...MissingMessageVerifierOption) *MissingMessageVerifier { +func NewMissingMessageVerifier(storenodeRequester StorenodeRequestor, messageTracker MessageTracker, timesource timesource.Timesource, logger *zap.Logger, options ...MissingMessageVerifierOption) *MissingMessageVerifier { options = append(defaultMissingMessagesVerifierOptions, options...) params := missingMessageVerifierParams{} for _, opt := range options { @@ -56,11 +62,11 @@ func NewMissingMessageVerifier(store *store.WakuStore, messageTracker MessageTra } return &MissingMessageVerifier{ - store: store, - timesource: timesource, - messageTracker: messageTracker, - logger: logger.Named("missing-msg-verifier"), - params: params, + storenodeRequestor: storenodeRequester, + timesource: timesource, + messageTracker: messageTracker, + logger: logger.Named("missing-msg-verifier"), + params: params, } } @@ -178,7 +184,7 @@ func (m *MissingMessageVerifier) fetchHistory(c chan<- *protocol.Envelope, inter } } -func (m *MissingMessageVerifier) storeQueryWithRetry(ctx context.Context, queryFunc func(ctx context.Context) (store.Result, error), logger *zap.Logger, logMsg string) (store.Result, error) { +func (m *MissingMessageVerifier) storeQueryWithRetry(ctx context.Context, queryFunc func(ctx context.Context) (common.StoreRequestResult, error), logger *zap.Logger, logMsg string) (common.StoreRequestResult, error) { retry := true count := 1 for retry && count <= m.params.maxAttemptsToRetrieveHistory { @@ -212,12 +218,16 @@ func (m *MissingMessageVerifier) fetchMessagesBatch(c chan<- *protocol.Envelope, logging.Epoch("to", now), ) - result, err := m.storeQueryWithRetry(interest.ctx, func(ctx context.Context) (store.Result, error) { - return m.store.Query(ctx, store.FilterCriteria{ - ContentFilter: protocol.NewContentFilter(interest.contentFilter.PubsubTopic, contentTopics[batchFrom:batchTo]...), - TimeStart: proto.Int64(interest.lastChecked.Add(-m.params.delay).UnixNano()), - TimeEnd: proto.Int64(now.Add(-m.params.delay).UnixNano()), - }, store.WithPeer(interest.peerID), store.WithPaging(false, 100), store.IncludeData(false)) + result, err := m.storeQueryWithRetry(interest.ctx, func(ctx context.Context) (common.StoreRequestResult, error) { + return m.storenodeRequestor.QueryWithCriteria( + ctx, + interest.peerID, + messageFetchPageSize, + interest.contentFilter.PubsubTopic, + contentTopics[batchFrom:batchTo], + proto.Int64(interest.lastChecked.Add(-m.params.delay).UnixNano()), + proto.Int64(now.Add(-m.params.delay).UnixNano()), + ) }, logger, "retrieving history to check for missing messages") if err != nil { if !errors.Is(err, context.Canceled) { @@ -243,7 +253,7 @@ func (m *MissingMessageVerifier) fetchMessagesBatch(c chan<- *protocol.Envelope, missingHashes = append(missingHashes, hash) } - result, err = m.storeQueryWithRetry(interest.ctx, func(ctx context.Context) (store.Result, error) { + result, err = m.storeQueryWithRetry(interest.ctx, func(ctx context.Context) (common.StoreRequestResult, error) { if err = result.Next(ctx); err != nil { return nil, err } @@ -282,10 +292,10 @@ func (m *MissingMessageVerifier) fetchMessagesBatch(c chan<- *protocol.Envelope, defer utils.LogOnPanic() defer wg.Wait() - result, err := m.storeQueryWithRetry(interest.ctx, func(ctx context.Context) (store.Result, error) { + result, err := m.storeQueryWithRetry(interest.ctx, func(ctx context.Context) (common.StoreRequestResult, error) { queryCtx, cancel := context.WithTimeout(ctx, m.params.storeQueryTimeout) defer cancel() - return m.store.QueryByHash(queryCtx, messageHashes, store.WithPeer(interest.peerID), store.WithPaging(false, maxMsgHashesPerRequest)) + return m.storenodeRequestor.GetMessagesByHash(queryCtx, interest.peerID, maxMsgHashesPerRequest, messageHashes) }, logger, "retrieving missing messages") if err != nil { if !errors.Is(err, context.Canceled) { @@ -303,7 +313,7 @@ func (m *MissingMessageVerifier) fetchMessagesBatch(c chan<- *protocol.Envelope, } } - result, err = m.storeQueryWithRetry(interest.ctx, func(ctx context.Context) (store.Result, error) { + result, err = m.storeQueryWithRetry(interest.ctx, func(ctx context.Context) (common.StoreRequestResult, error) { if err = result.Next(ctx); err != nil { return nil, err } diff --git a/vendor/github.com/waku-org/go-waku/waku/v2/api/publish/default_publisher.go b/vendor/github.com/waku-org/go-waku/waku/v2/api/publish/default_publisher.go new file mode 100644 index 00000000000..4ca940cef3b --- /dev/null +++ b/vendor/github.com/waku-org/go-waku/waku/v2/api/publish/default_publisher.go @@ -0,0 +1,50 @@ +package publish + +import ( + "context" + "errors" + + "github.com/libp2p/go-libp2p/core/peer" + "github.com/waku-org/go-waku/waku/v2/protocol/lightpush" + "github.com/waku-org/go-waku/waku/v2/protocol/pb" + "github.com/waku-org/go-waku/waku/v2/protocol/relay" +) + +var ErrRelayNotAvailable = errors.New("relay is not available") +var ErrLightpushNotAvailable = errors.New("lightpush is not available") + +func NewDefaultPublisher(lightpush *lightpush.WakuLightPush, relay *relay.WakuRelay) Publisher { + return &defaultPublisher{ + lightpush: lightpush, + relay: relay, + } +} + +type defaultPublisher struct { + lightpush *lightpush.WakuLightPush + relay *relay.WakuRelay +} + +func (d *defaultPublisher) RelayListPeers(pubsubTopic string) ([]peer.ID, error) { + if d.relay == nil { + return nil, ErrRelayNotAvailable + } + + return d.relay.PubSub().ListPeers(pubsubTopic), nil +} + +func (d *defaultPublisher) RelayPublish(ctx context.Context, message *pb.WakuMessage, pubsubTopic string) (pb.MessageHash, error) { + if d.relay == nil { + return pb.MessageHash{}, ErrRelayNotAvailable + } + + return d.relay.Publish(ctx, message, relay.WithPubSubTopic(pubsubTopic)) +} + +func (d *defaultPublisher) LightpushPublish(ctx context.Context, message *pb.WakuMessage, pubsubTopic string, maxPeers int) (pb.MessageHash, error) { + if d.lightpush == nil { + return pb.MessageHash{}, ErrLightpushNotAvailable + } + + return d.lightpush.Publish(ctx, message, lightpush.WithPubSubTopic(pubsubTopic), lightpush.WithMaxPeers(maxPeers)) +} diff --git a/vendor/github.com/waku-org/go-waku/waku/v2/api/publish/default_verifier.go b/vendor/github.com/waku-org/go-waku/waku/v2/api/publish/default_verifier.go new file mode 100644 index 00000000000..68eca0304d1 --- /dev/null +++ b/vendor/github.com/waku-org/go-waku/waku/v2/api/publish/default_verifier.go @@ -0,0 +1,39 @@ +package publish + +import ( + "context" + + "github.com/libp2p/go-libp2p/core/peer" + "github.com/waku-org/go-waku/waku/v2/protocol/pb" + "github.com/waku-org/go-waku/waku/v2/protocol/store" +) + +func NewDefaultStorenodeMessageVerifier(store *store.WakuStore) StorenodeMessageVerifier { + return &defaultStorenodeMessageVerifier{ + store: store, + } +} + +type defaultStorenodeMessageVerifier struct { + store *store.WakuStore +} + +func (d *defaultStorenodeMessageVerifier) MessageHashesExist(ctx context.Context, requestID []byte, peerID peer.ID, pageSize uint64, messageHashes []pb.MessageHash) ([]pb.MessageHash, error) { + var opts []store.RequestOption + opts = append(opts, store.WithRequestID(requestID)) + opts = append(opts, store.WithPeer(peerID)) + opts = append(opts, store.WithPaging(false, pageSize)) + opts = append(opts, store.IncludeData(false)) + + response, err := d.store.QueryByHash(ctx, messageHashes, opts...) + if err != nil { + return nil, err + } + + result := make([]pb.MessageHash, len(response.Messages())) + for i, msg := range response.Messages() { + result[i] = msg.WakuMessageHash() + } + + return result, nil +} diff --git a/vendor/github.com/waku-org/go-waku/waku/v2/api/publish/message_check.go b/vendor/github.com/waku-org/go-waku/waku/v2/api/publish/message_check.go index 0591657401e..8a37e20ce3f 100644 --- a/vendor/github.com/waku-org/go-waku/waku/v2/api/publish/message_check.go +++ b/vendor/github.com/waku-org/go-waku/waku/v2/api/publish/message_check.go @@ -8,11 +8,11 @@ import ( "github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/common/hexutil" + "github.com/libp2p/go-libp2p/core/peer" apicommon "github.com/waku-org/go-waku/waku/v2/api/common" "github.com/waku-org/go-waku/waku/v2/api/history" "github.com/waku-org/go-waku/waku/v2/protocol" "github.com/waku-org/go-waku/waku/v2/protocol/pb" - "github.com/waku-org/go-waku/waku/v2/protocol/store" "github.com/waku-org/go-waku/waku/v2/timesource" "github.com/waku-org/go-waku/waku/v2/utils" "go.uber.org/zap" @@ -31,6 +31,11 @@ type ISentCheck interface { DeleteByMessageIDs(messageIDs []common.Hash) } +type StorenodeMessageVerifier interface { + // MessagesExist returns a list of the messages it found from a list of message hashes + MessageHashesExist(ctx context.Context, requestID []byte, peerID peer.ID, pageSize uint64, messageHashes []pb.MessageHash) ([]pb.MessageHash, error) +} + // MessageSentCheck tracks the outgoing messages and check against store node // if the message sent time has passed the `messageSentPeriod`, the message id will be includes for the next query // if the message keeps missing after `messageExpiredPerid`, the message id will be expired @@ -40,7 +45,7 @@ type MessageSentCheck struct { messageStoredChan chan common.Hash messageExpiredChan chan common.Hash ctx context.Context - store *store.WakuStore + messageVerifier StorenodeMessageVerifier storenodeCycle *history.StorenodeCycle timesource timesource.Timesource logger *zap.Logger @@ -52,14 +57,14 @@ type MessageSentCheck struct { } // NewMessageSentCheck creates a new instance of MessageSentCheck with default parameters -func NewMessageSentCheck(ctx context.Context, store *store.WakuStore, cycle *history.StorenodeCycle, timesource timesource.Timesource, msgStoredChan chan common.Hash, msgExpiredChan chan common.Hash, logger *zap.Logger) *MessageSentCheck { +func NewMessageSentCheck(ctx context.Context, messageVerifier StorenodeMessageVerifier, cycle *history.StorenodeCycle, timesource timesource.Timesource, msgStoredChan chan common.Hash, msgExpiredChan chan common.Hash, logger *zap.Logger) *MessageSentCheck { return &MessageSentCheck{ messageIDs: make(map[string]map[common.Hash]uint32), messageIDsMu: sync.RWMutex{}, messageStoredChan: msgStoredChan, messageExpiredChan: msgExpiredChan, ctx: ctx, - store: store, + messageVerifier: messageVerifier, storenodeCycle: cycle, timesource: timesource, logger: logger, @@ -212,12 +217,7 @@ func (m *MessageSentCheck) messageHashBasedQuery(ctx context.Context, hashes []c return []common.Hash{} } - var opts []store.RequestOption requestID := protocol.GenerateRequestID() - opts = append(opts, store.WithRequestID(requestID)) - opts = append(opts, store.WithPeer(selectedPeer)) - opts = append(opts, store.WithPaging(false, m.maxHashQueryLength)) - opts = append(opts, store.IncludeData(false)) messageHashes := make([]pb.MessageHash, len(hashes)) for i, hash := range hashes { @@ -228,20 +228,20 @@ func (m *MessageSentCheck) messageHashBasedQuery(ctx context.Context, hashes []c queryCtx, cancel := context.WithTimeout(ctx, m.storeQueryTimeout) defer cancel() - result, err := m.store.QueryByHash(queryCtx, messageHashes, opts...) + result, err := m.messageVerifier.MessageHashesExist(queryCtx, requestID, selectedPeer, m.maxHashQueryLength, messageHashes) if err != nil { m.logger.Error("store.queryByHash failed", zap.String("requestID", hexutil.Encode(requestID)), zap.Stringer("peerID", selectedPeer), zap.Error(err)) return []common.Hash{} } - m.logger.Debug("store.queryByHash result", zap.String("requestID", hexutil.Encode(requestID)), zap.Int("messages", len(result.Messages()))) + m.logger.Debug("store.queryByHash result", zap.String("requestID", hexutil.Encode(requestID)), zap.Int("messages", len(result))) var ackHashes []common.Hash var missedHashes []common.Hash for i, hash := range hashes { found := false - for _, msg := range result.Messages() { - if bytes.Equal(msg.GetMessageHash(), hash.Bytes()) { + for _, msgHash := range result { + if bytes.Equal(msgHash.Bytes(), hash.Bytes()) { found = true break } diff --git a/vendor/github.com/waku-org/go-waku/waku/v2/api/publish/message_sender.go b/vendor/github.com/waku-org/go-waku/waku/v2/api/publish/message_sender.go index c1e9a4ca783..c457589e76b 100644 --- a/vendor/github.com/waku-org/go-waku/waku/v2/api/publish/message_sender.go +++ b/vendor/github.com/waku-org/go-waku/waku/v2/api/publish/message_sender.go @@ -6,9 +6,9 @@ import ( "time" "github.com/ethereum/go-ethereum/common" + "github.com/libp2p/go-libp2p/core/peer" "github.com/waku-org/go-waku/waku/v2/protocol" - "github.com/waku-org/go-waku/waku/v2/protocol/lightpush" - "github.com/waku-org/go-waku/waku/v2/protocol/relay" + "github.com/waku-org/go-waku/waku/v2/protocol/pb" "go.uber.org/zap" "golang.org/x/time/rate" ) @@ -36,10 +36,20 @@ func (pm PublishMethod) String() string { } } +type Publisher interface { + // RelayListPeers returns the list of peers for a pubsub topic + RelayListPeers(pubsubTopic string) ([]peer.ID, error) + + // RelayPublish publishes a message via WakuRelay + RelayPublish(ctx context.Context, message *pb.WakuMessage, pubsubTopic string) (pb.MessageHash, error) + + // LightpushPublish publishes a message via WakuLightPush + LightpushPublish(ctx context.Context, message *pb.WakuMessage, pubsubTopic string, maxPeers int) (pb.MessageHash, error) +} + type MessageSender struct { publishMethod PublishMethod - lightPush *lightpush.WakuLightPush - relay *relay.WakuRelay + publisher Publisher messageSentCheck ISentCheck rateLimiter *PublishRateLimiter logger *zap.Logger @@ -64,14 +74,13 @@ func (r *Request) WithPublishMethod(publishMethod PublishMethod) *Request { return r } -func NewMessageSender(publishMethod PublishMethod, lightPush *lightpush.WakuLightPush, relay *relay.WakuRelay, logger *zap.Logger) (*MessageSender, error) { +func NewMessageSender(publishMethod PublishMethod, publisher Publisher, logger *zap.Logger) (*MessageSender, error) { if publishMethod == UnknownMethod { return nil, errors.New("publish method is required") } return &MessageSender{ publishMethod: publishMethod, - lightPush: lightPush, - relay: relay, + publisher: publisher, rateLimiter: NewPublishRateLimiter(DefaultPublishingLimiterRate, DefaultPublishingLimitBurst), logger: logger, }, nil @@ -108,26 +117,23 @@ func (ms *MessageSender) Send(req *Request) error { switch publishMethod { case LightPush: - if ms.lightPush == nil { - return errors.New("lightpush is not available") - } logger.Info("publishing message via lightpush") - _, err := ms.lightPush.Publish( + _, err := ms.publisher.LightpushPublish( req.ctx, req.envelope.Message(), - lightpush.WithPubSubTopic(req.envelope.PubsubTopic()), - lightpush.WithMaxPeers(DefaultPeersToPublishForLightpush), + req.envelope.PubsubTopic(), + DefaultPeersToPublishForLightpush, ) if err != nil { return err } case Relay: - if ms.relay == nil { - return errors.New("relay is not available") + peers, err := ms.publisher.RelayListPeers(req.envelope.PubsubTopic()) + if err != nil { + return err } - peerCnt := len(ms.relay.PubSub().ListPeers(req.envelope.PubsubTopic())) - logger.Info("publishing message via relay", zap.Int("peerCnt", peerCnt)) - _, err := ms.relay.Publish(req.ctx, req.envelope.Message(), relay.WithPubSubTopic(req.envelope.PubsubTopic())) + logger.Info("publishing message via relay", zap.Int("peerCnt", len(peers))) + _, err = ms.publisher.RelayPublish(req.ctx, req.envelope.Message(), req.envelope.PubsubTopic()) if err != nil { return err } diff --git a/vendor/modules.txt b/vendor/modules.txt index 4af35ef32b6..7d15bf851b2 100644 --- a/vendor/modules.txt +++ b/vendor/modules.txt @@ -1015,7 +1015,7 @@ github.com/waku-org/go-discover/discover/v5wire github.com/waku-org/go-libp2p-rendezvous github.com/waku-org/go-libp2p-rendezvous/db github.com/waku-org/go-libp2p-rendezvous/pb -# github.com/waku-org/go-waku v0.8.1-0.20241014185851-76275f6fb835 +# github.com/waku-org/go-waku v0.8.1-0.20241015194815-37f936d74705 ## explicit; go 1.21 github.com/waku-org/go-waku/logging github.com/waku-org/go-waku/tests diff --git a/wakuv2/waku.go b/wakuv2/waku.go index 3e2319752f3..1869f3a427d 100644 --- a/wakuv2/waku.go +++ b/wakuv2/waku.go @@ -1122,7 +1122,7 @@ func (w *Waku) Start() error { if w.cfg.EnableMissingMessageVerification { w.missingMsgVerifier = missing.NewMissingMessageVerifier( - w.node.Store(), + missing.NewDefaultStorenodeRequestor(w.node.Store()), w, w.node.Timesource(), w.logger) @@ -1267,7 +1267,7 @@ func (w *Waku) startMessageSender() error { publishMethod = publish.LightPush } - sender, err := publish.NewMessageSender(publishMethod, w.node.Lightpush(), w.node.Relay(), w.logger) + sender, err := publish.NewMessageSender(publishMethod, publish.NewDefaultPublisher(w.node.Lightpush(), w.node.Relay()), w.logger) if err != nil { w.logger.Error("failed to create message sender", zap.Error(err)) return err @@ -1276,7 +1276,7 @@ func (w *Waku) startMessageSender() error { if w.cfg.EnableStoreConfirmationForMessagesSent { msgStoredChan := make(chan gethcommon.Hash, 1000) msgExpiredChan := make(chan gethcommon.Hash, 1000) - messageSentCheck := publish.NewMessageSentCheck(w.ctx, w.node.Store(), w.StorenodeCycle, w.node.Timesource(), msgStoredChan, msgExpiredChan, w.logger) + messageSentCheck := publish.NewMessageSentCheck(w.ctx, publish.NewDefaultStorenodeMessageVerifier(w.node.Store()), w.StorenodeCycle, w.node.Timesource(), msgStoredChan, msgExpiredChan, w.logger) sender.WithMessageSentCheck(messageSentCheck) w.wg.Add(1) From 3107c1eb0bf3852e478d8bbf378842a6bf5af468 Mon Sep 17 00:00:00 2001 From: Ivan Folgueira Bande Date: Tue, 27 Aug 2024 15:14:36 +0200 Subject: [PATCH 03/14] refactor_: start using nwaku - some minor progress to add nwaku in status-go - nwaku.go: GetNumConnectedPeers controls when passed pubsub is empty - waku_test.go: adapt TestWakuV2Store - add missing shard.go - feat_: build nwaku with nix and use build tags to choose between go-waku and nwaku (#5896) - chore_: update nwaku - nwaku bump (#5911) - bump: nwaku - chore: add USE_NWAKU env flag - fix: build libwaku only if needed - feat: testing discovery and dialing with nwaku integration (#5940) --- .gitignore | 1 - .gitmodules | 3 + .vscode/settings.json | 3 + Makefile | 49 +- cmd/ping-community/main.go | 10 +- eth-node/bridge/geth/wakuv2.go | 2 +- nix/shell.nix | 2 +- node/status_node_services.go | 3 +- protocol/communities/community.go | 10 +- protocol/communities/manager.go | 22 +- protocol/communities/persistence.go | 8 +- protocol/communities/persistence_mapping.go | 6 +- protocol/communities/persistence_test.go | 4 +- ...nities_messenger_token_permissions_test.go | 7 +- protocol/linkpreview_unfurler_status.go | 6 +- protocol/messenger.go | 4 +- protocol/messenger_communities.go | 32 +- .../messenger_communities_sharding_test.go | 20 +- protocol/messenger_community_shard.go | 6 +- protocol/messenger_share_urls.go | 8 +- .../messenger_store_node_request_manager.go | 12 +- protocol/messenger_storenode_comunity_test.go | 9 +- protocol/messenger_storenode_request_test.go | 27 +- protocol/messenger_testing_utils.go | 6 +- protocol/requests/set_community_shard.go | 6 +- protocol/transport/filters_manager.go | 6 +- protocol/waku_builder_test.go | 10 +- services/ext/api.go | 6 +- services/mailservers/api_test.go | 18 +- services/status/service.go | 4 +- third_party/nwaku | 1 + wakuv2/api.go | 3 +- wakuv2/api_test.go | 6 +- wakuv2/config.go | 8 +- wakuv2/{waku.go => gowaku.go} | 28 +- wakuv2/nwaku.go | 2596 ++++++++++++++++- wakuv2/nwaku_test.go | 807 +++++ wakuv2/nwaku_test_utils.go | 58 + {protocol/common/shard => wakuv2}/shard.go | 2 +- wakuv2/waku_test.go | 16 +- 40 files changed, 3660 insertions(+), 175 deletions(-) create mode 100644 .gitmodules create mode 160000 third_party/nwaku rename wakuv2/{waku.go => gowaku.go} (98%) create mode 100644 wakuv2/nwaku_test.go create mode 100644 wakuv2/nwaku_test_utils.go rename {protocol/common/shard => wakuv2}/shard.go (98%) diff --git a/.gitignore b/.gitignore index d7b9b412f3f..c92f416c1dc 100644 --- a/.gitignore +++ b/.gitignore @@ -64,7 +64,6 @@ coverage.html Session.vim .undodir/* /.idea/ -/.vscode/ /cmd/*/.ethereum/ *.iml diff --git a/.gitmodules b/.gitmodules new file mode 100644 index 00000000000..13c88e400e0 --- /dev/null +++ b/.gitmodules @@ -0,0 +1,3 @@ +[submodule "third_party/nwaku"] + path = third_party/nwaku + url = https://github.com/waku-org/nwaku diff --git a/.vscode/settings.json b/.vscode/settings.json index 16fdfe542a3..d83958300e2 100644 --- a/.vscode/settings.json +++ b/.vscode/settings.json @@ -8,4 +8,7 @@ "cSpell.words": [ "unmarshalling" ], + "gopls":{ + "buildFlags": ["-tags=use_nwaku,gowaku_skip_migrations,gowaku_no_rln"] + } } diff --git a/Makefile b/Makefile index d4c93cfed23..4c1823ec398 100644 --- a/Makefile +++ b/Makefile @@ -1,5 +1,6 @@ .PHONY: statusgo statusd-prune all test clean help .PHONY: statusgo-android statusgo-ios +.PHONY: build-libwaku test-libwaku clean-libwaku rebuild-libwaku # Clear any GOROOT set outside of the Nix shell export GOROOT= @@ -61,6 +62,10 @@ GIT_AUTHOR ?= $(shell git config user.email || echo $$USER) ENABLE_METRICS ?= true BUILD_TAGS ?= gowaku_no_rln +ifeq ($(USE_NWAKU), true) +BUILD_TAGS += use_nwaku +endif + BUILD_FLAGS ?= -ldflags="-X github.com/status-im/status-go/params.Version=$(RELEASE_TAG:v%=%) \ -X github.com/status-im/status-go/params.GitCommit=$(GIT_COMMIT) \ -X github.com/status-im/status-go/params.IpfsGatewayURL=$(IPFS_GATEWAY_URL) \ @@ -234,8 +239,19 @@ statusgo-library: ##@cross-compile Build status-go as static library for current @echo "Static library built:" @ls -la build/bin/libstatus.* -statusgo-shared-library: generate -statusgo-shared-library: ##@cross-compile Build status-go as shared library for current platform + +LIBWAKU := third_party/nwaku/build/libwaku.$(GOBIN_SHARED_LIB_EXT) +$(LIBWAKU): + @echo "Building libwaku" + $(MAKE) -C third_party/nwaku update || { echo "nwaku make update failed"; exit 1; } + $(MAKE) -C ./third_party/nwaku libwaku + +build-libwaku: $(LIBWAKU) + +statusgo-shared-library: generate ##@cross-compile Build status-go as shared library for current platform +ifeq ($(USE_NWAKU),true) + $(MAKE) $(LIBWAKU) +endif ## cmd/library/README.md explains the magic incantation behind this mkdir -p build/bin/statusgo-lib go run cmd/library/*.go > build/bin/statusgo-lib/main.go @@ -356,9 +372,38 @@ lint-fix: -w {} \; $(MAKE) vendor +mock: ##@other Regenerate mocks + mockgen -package=fake -destination=transactions/fake/mock.go -source=transactions/fake/txservice.go + mockgen -package=status -destination=services/status/account_mock.go -source=services/status/service.go + mockgen -package=peer -destination=services/peer/discoverer_mock.go -source=services/peer/service.go + mockgen -package=mock_transactor -destination=transactions/mock_transactor/transactor.go -source=transactions/transactor.go + mockgen -package=mock_pathprocessor -destination=services/wallet/router/pathprocessor/mock_pathprocessor/processor.go -source=services/wallet/router/pathprocessor/processor.go + mockgen -package=mock_bridge -destination=services/wallet/bridge/mock_bridge/bridge.go -source=services/wallet/bridge/bridge.go + mockgen -package=mock_client -destination=rpc/chain/mock/client/client.go -source=rpc/chain/client.go + mockgen -package=mock_token -destination=services/wallet/token/mock/token/tokenmanager.go -source=services/wallet/token/token.go + mockgen -package=mock_thirdparty -destination=services/wallet/thirdparty/mock/types.go -source=services/wallet/thirdparty/types.go + mockgen -package=mock_balance_persistence -destination=services/wallet/token/mock/balance_persistence/balance_persistence.go -source=services/wallet/token/balance_persistence.go + mockgen -package=mock_network -destination=rpc/network/mock/network.go -source=rpc/network/network.go + mockgen -package=mock_rpcclient -destination=rpc/mock/client/client.go -source=rpc/client.go + mockgen -package=mock_collectibles -destination=services/wallet/collectibles/mock/collection_data_db.go -source=services/wallet/collectibles/collection_data_db.go + mockgen -package=mock_collectibles -destination=services/wallet/collectibles/mock/collectible_data_db.go -source=services/wallet/collectibles/collectible_data_db.go + mockgen -package=mock_thirdparty -destination=services/wallet/thirdparty/mock/collectible_types.go -source=services/wallet/thirdparty/collectible_types.go + mockgen -package=mock_paraswap -destination=services/wallet/thirdparty/paraswap/mock/types.go -source=services/wallet/thirdparty/paraswap/types.go + mockgen -package=mock_onramp -destination=services/wallet/onramp/mock/types.go -source=services/wallet/onramp/types.go + + docker-test: ##@tests Run tests in a docker container with golang. docker run --privileged --rm -it -v "$(PWD):$(DOCKER_TEST_WORKDIR)" -w "$(DOCKER_TEST_WORKDIR)" $(DOCKER_TEST_IMAGE) go test ${ARGS} +test-libwaku: | $(LIBWAKU) + go test -tags '$(BUILD_TAGS) use_nwaku' -run TestBasicWakuV2 ./wakuv2/... -count 1 -v -json | jq -r '.Output' + +clean-libwaku: + @echo "Removing libwaku" + rm $(LIBWAKU) + +rebuild-libwaku: | clean-libwaku $(LIBWAKU) + test: test-unit ##@tests Run basic, short tests during development test-unit: generate diff --git a/cmd/ping-community/main.go b/cmd/ping-community/main.go index 7866515bd31..e3cd846fa61 100644 --- a/cmd/ping-community/main.go +++ b/cmd/ping-community/main.go @@ -24,12 +24,12 @@ import ( "github.com/status-im/status-go/multiaccounts" "github.com/status-im/status-go/multiaccounts/accounts" "github.com/status-im/status-go/multiaccounts/settings" + "github.com/status-im/status-go/wakuv2" "github.com/status-im/status-go/logutils" "github.com/status-im/status-go/params" "github.com/status-im/status-go/protocol" "github.com/status-im/status-go/protocol/common" - "github.com/status-im/status-go/protocol/common/shard" "github.com/status-im/status-go/protocol/identity/alias" "github.com/status-im/status-go/protocol/protobuf" wakuextn "github.com/status-im/status-go/services/wakuext" @@ -48,8 +48,8 @@ var ( seedPhrase = flag.String("seed-phrase", "", "Seed phrase") version = flag.Bool("version", false, "Print version and dump configuration") communityID = flag.String("community-id", "", "The id of the community") - shardCluster = flag.Int("shard-cluster", shard.MainStatusShardCluster, "The shard cluster in which the of the community is published") - shardIndex = flag.Int("shard-index", shard.DefaultShardIndex, "The shard index in which the community is published") + shardCluster = flag.Int("shard-cluster", wakuv2.MainStatusShardCluster, "The shard cluster in which the of the community is published") + shardIndex = flag.Int("shard-index", wakuv2.DefaultShardIndex, "The shard index in which the community is published") chatID = flag.String("chat-id", "", "The id of the chat") dataDir = flag.String("dir", getDefaultDataDir(), "Directory used by node to store data") @@ -148,9 +148,9 @@ func main() { messenger := wakuextservice.Messenger() - var s *shard.Shard = nil + var s *wakuv2.Shard = nil if shardCluster != nil && shardIndex != nil { - s = &shard.Shard{ + s = &wakuv2.Shard{ Cluster: uint16(*shardCluster), Index: uint16(*shardIndex), } diff --git a/eth-node/bridge/geth/wakuv2.go b/eth-node/bridge/geth/wakuv2.go index a301698396f..c35a18e8a1f 100644 --- a/eth-node/bridge/geth/wakuv2.go +++ b/eth-node/bridge/geth/wakuv2.go @@ -227,7 +227,7 @@ func (w *gethWakuV2Wrapper) DialPeerByID(peerID peer.ID) error { } func (w *gethWakuV2Wrapper) ListenAddresses() ([]multiaddr.Multiaddr, error) { - return w.waku.ListenAddresses(), nil + return w.waku.ListenAddresses() } func (w *gethWakuV2Wrapper) RelayPeersByTopic(topic string) (*types.PeerList, error) { diff --git a/nix/shell.nix b/nix/shell.nix index 10f2647c77e..c53c80c6edc 100644 --- a/nix/shell.nix +++ b/nix/shell.nix @@ -21,7 +21,7 @@ in pkgs.mkShell { buildInputs = with pkgs; [ git jq which - go golangci-lint go-junit-report gopls go-bindata gomobileMod codecov-cli go-generate-fast + go golangci-lint go-junit-report gopls go-bindata gomobileMod codecov-cli go-generate-fast openssl mockgen protobuf3_20 protoc-gen-go gotestsum go-modvendor openjdk cc-test-reporter ] ++ lib.optionals (stdenv.isDarwin) [ xcodeWrapper ]; diff --git a/node/status_node_services.go b/node/status_node_services.go index 16539d21f77..f5e6fada365 100644 --- a/node/status_node_services.go +++ b/node/status_node_services.go @@ -10,7 +10,6 @@ import ( "reflect" "time" - "github.com/status-im/status-go/protocol/common/shard" "github.com/status-im/status-go/server" "github.com/status-im/status-go/signal" "github.com/status-im/status-go/transactions" @@ -336,7 +335,7 @@ func (b *StatusNode) wakuV2Service(nodeConfig *params.NodeConfig) (*wakuv2.Waku, Nameserver: nodeConfig.WakuV2Config.Nameserver, UDPPort: nodeConfig.WakuV2Config.UDPPort, AutoUpdate: nodeConfig.WakuV2Config.AutoUpdate, - DefaultShardPubsubTopic: shard.DefaultShardPubsubTopic(), + DefaultShardPubsubTopic: wakuv2.DefaultShardPubsubTopic(), TelemetryServerURL: nodeConfig.WakuV2Config.TelemetryServerURL, ClusterID: nodeConfig.ClusterConfig.ClusterID, EnableMissingMessageVerification: nodeConfig.WakuV2Config.EnableMissingMessageVerification, diff --git a/protocol/communities/community.go b/protocol/communities/community.go index 566be333629..40f57820d00 100644 --- a/protocol/communities/community.go +++ b/protocol/communities/community.go @@ -23,12 +23,12 @@ import ( "github.com/status-im/status-go/eth-node/types" "github.com/status-im/status-go/images" "github.com/status-im/status-go/protocol/common" - "github.com/status-im/status-go/protocol/common/shard" community_token "github.com/status-im/status-go/protocol/communities/token" "github.com/status-im/status-go/protocol/protobuf" "github.com/status-im/status-go/protocol/requests" "github.com/status-im/status-go/protocol/v1" "github.com/status-im/status-go/server" + "github.com/status-im/status-go/wakuv2" ) const signatureLength = 65 @@ -55,7 +55,7 @@ type Config struct { RequestsToJoin []*RequestToJoin MemberIdentity *ecdsa.PrivateKey EventsData *EventsData - Shard *shard.Shard + Shard *wakuv2.Shard PubsubTopicPrivateKey *ecdsa.PrivateKey LastOpenedAt int64 } @@ -172,7 +172,7 @@ func (o *Community) MarshalPublicAPIJSON() ([]byte, error) { ActiveMembersCount uint64 `json:"activeMembersCount"` PubsubTopic string `json:"pubsubTopic"` PubsubTopicKey string `json:"pubsubTopicKey"` - Shard *shard.Shard `json:"shard"` + Shard *wakuv2.Shard `json:"shard"` }{ ID: o.ID(), Verified: o.config.Verified, @@ -308,7 +308,7 @@ func (o *Community) MarshalJSON() ([]byte, error) { ActiveMembersCount uint64 `json:"activeMembersCount"` PubsubTopic string `json:"pubsubTopic"` PubsubTopicKey string `json:"pubsubTopicKey"` - Shard *shard.Shard `json:"shard"` + Shard *wakuv2.Shard `json:"shard"` LastOpenedAt int64 `json:"lastOpenedAt"` Clock uint64 `json:"clock"` }{ @@ -461,7 +461,7 @@ func (o *Community) DescriptionText() string { return "" } -func (o *Community) Shard() *shard.Shard { +func (o *Community) Shard() *wakuv2.Shard { if o != nil && o.config != nil { return o.config.Shard } diff --git a/protocol/communities/manager.go b/protocol/communities/manager.go index 23f3abb4b74..87b14616535 100644 --- a/protocol/communities/manager.go +++ b/protocol/communities/manager.go @@ -30,7 +30,6 @@ import ( multiaccountscommon "github.com/status-im/status-go/multiaccounts/common" "github.com/status-im/status-go/params" "github.com/status-im/status-go/protocol/common" - "github.com/status-im/status-go/protocol/common/shard" community_token "github.com/status-im/status-go/protocol/communities/token" "github.com/status-im/status-go/protocol/encryption" "github.com/status-im/status-go/protocol/ens" @@ -45,6 +44,7 @@ import ( "github.com/status-im/status-go/services/wallet/token" "github.com/status-im/status-go/signal" "github.com/status-im/status-go/transactions" + "github.com/status-im/status-go/wakuv2" ) type Publisher interface { @@ -740,8 +740,8 @@ func (m *Manager) All() ([]*Community, error) { } type CommunityShard struct { - CommunityID string `json:"communityID"` - Shard *shard.Shard `json:"shard"` + CommunityID string `json:"communityID"` + Shard *wakuv2.Shard `json:"shard"` } type CuratedCommunities struct { @@ -1549,7 +1549,7 @@ func (m *Manager) DeleteCommunity(id types.HexBytes) error { return m.persistence.DeleteCommunitySettings(id) } -func (m *Manager) updateShard(community *Community, shard *shard.Shard, clock uint64) error { +func (m *Manager) updateShard(community *Community, shard *wakuv2.Shard, clock uint64) error { community.config.Shard = shard if shard == nil { return m.persistence.DeleteCommunityShard(community.ID()) @@ -1558,7 +1558,7 @@ func (m *Manager) updateShard(community *Community, shard *shard.Shard, clock ui return m.persistence.SaveCommunityShard(community.ID(), shard, clock) } -func (m *Manager) UpdateShard(community *Community, shard *shard.Shard, clock uint64) error { +func (m *Manager) UpdateShard(community *Community, shard *wakuv2.Shard, clock uint64) error { m.communityLock.Lock(community.ID()) defer m.communityLock.Unlock(community.ID()) @@ -1566,7 +1566,7 @@ func (m *Manager) UpdateShard(community *Community, shard *shard.Shard, clock ui } // SetShard assigns a shard to a community -func (m *Manager) SetShard(communityID types.HexBytes, shard *shard.Shard) (*Community, error) { +func (m *Manager) SetShard(communityID types.HexBytes, shard *wakuv2.Shard) (*Community, error) { m.communityLock.Lock(communityID) defer m.communityLock.Unlock(communityID) @@ -2158,11 +2158,11 @@ func (m *Manager) HandleCommunityDescriptionMessage(signer *ecdsa.PublicKey, des if err != nil { return nil, err } - var cShard *shard.Shard + var cShard *wakuv2.Shard if communityShard == nil { - cShard = &shard.Shard{Cluster: shard.MainStatusShardCluster, Index: shard.DefaultShardIndex} + cShard = &wakuv2.Shard{Cluster: wakuv2.MainStatusShardCluster, Index: wakuv2.DefaultShardIndex} } else { - cShard = shard.FromProtobuff(communityShard) + cShard = wakuv2.FromProtobuff(communityShard) } config := Config{ CommunityDescription: processedDescription, @@ -3940,11 +3940,11 @@ func (m *Manager) GetByIDString(idString string) (*Community, error) { return m.GetByID(id) } -func (m *Manager) GetCommunityShard(communityID types.HexBytes) (*shard.Shard, error) { +func (m *Manager) GetCommunityShard(communityID types.HexBytes) (*wakuv2.Shard, error) { return m.persistence.GetCommunityShard(communityID) } -func (m *Manager) SaveCommunityShard(communityID types.HexBytes, shard *shard.Shard, clock uint64) error { +func (m *Manager) SaveCommunityShard(communityID types.HexBytes, shard *wakuv2.Shard, clock uint64) error { m.communityLock.Lock(communityID) defer m.communityLock.Unlock(communityID) diff --git a/protocol/communities/persistence.go b/protocol/communities/persistence.go index 84e20645e05..a417b81a233 100644 --- a/protocol/communities/persistence.go +++ b/protocol/communities/persistence.go @@ -16,11 +16,11 @@ import ( "github.com/status-im/status-go/eth-node/crypto" "github.com/status-im/status-go/eth-node/types" "github.com/status-im/status-go/protocol/common" - "github.com/status-im/status-go/protocol/common/shard" "github.com/status-im/status-go/protocol/communities/token" "github.com/status-im/status-go/protocol/encryption" "github.com/status-im/status-go/protocol/protobuf" "github.com/status-im/status-go/services/wallet/bigint" + "github.com/status-im/status-go/wakuv2" ) type Persistence struct { @@ -1766,7 +1766,7 @@ func (p *Persistence) AllNonApprovedCommunitiesRequestsToJoin() ([]*RequestToJoi return nonApprovedRequestsToJoin, nil } -func (p *Persistence) SaveCommunityShard(communityID types.HexBytes, shard *shard.Shard, clock uint64) error { +func (p *Persistence) SaveCommunityShard(communityID types.HexBytes, shard *wakuv2.Shard, clock uint64) error { var cluster, index *uint16 if shard != nil { @@ -1801,7 +1801,7 @@ func (p *Persistence) SaveCommunityShard(communityID types.HexBytes, shard *shar } // if data will not be found, will return sql.ErrNoRows. Must be handled on the caller side -func (p *Persistence) GetCommunityShard(communityID types.HexBytes) (*shard.Shard, error) { +func (p *Persistence) GetCommunityShard(communityID types.HexBytes) (*wakuv2.Shard, error) { var cluster sql.NullInt64 var index sql.NullInt64 err := p.db.QueryRow(`SELECT shard_cluster, shard_index FROM communities_shards WHERE community_id = ?`, @@ -1815,7 +1815,7 @@ func (p *Persistence) GetCommunityShard(communityID types.HexBytes) (*shard.Shar return nil, nil } - return &shard.Shard{ + return &wakuv2.Shard{ Cluster: uint16(cluster.Int64), Index: uint16(index.Int64), }, nil diff --git a/protocol/communities/persistence_mapping.go b/protocol/communities/persistence_mapping.go index a8964187eea..8e743c57141 100644 --- a/protocol/communities/persistence_mapping.go +++ b/protocol/communities/persistence_mapping.go @@ -7,8 +7,8 @@ import ( "github.com/status-im/status-go/eth-node/crypto" "github.com/status-im/status-go/protocol/common" - "github.com/status-im/status-go/protocol/common/shard" "github.com/status-im/status-go/server" + "github.com/status-im/status-go/wakuv2" ) func communityToRecord(community *Community) (*CommunityRecord, error) { @@ -118,9 +118,9 @@ func recordBundleToCommunity( } } - var s *shard.Shard = nil + var s *wakuv2.Shard = nil if r.community.shardCluster != nil && r.community.shardIndex != nil { - s = &shard.Shard{ + s = &wakuv2.Shard{ Cluster: uint16(*r.community.shardCluster), Index: uint16(*r.community.shardIndex), } diff --git a/protocol/communities/persistence_test.go b/protocol/communities/persistence_test.go index 42053ecb966..19947b18153 100644 --- a/protocol/communities/persistence_test.go +++ b/protocol/communities/persistence_test.go @@ -15,13 +15,13 @@ import ( "github.com/status-im/status-go/eth-node/crypto" "github.com/status-im/status-go/eth-node/types" "github.com/status-im/status-go/protocol/common" - "github.com/status-im/status-go/protocol/common/shard" "github.com/status-im/status-go/protocol/communities/token" "github.com/status-im/status-go/protocol/encryption" "github.com/status-im/status-go/protocol/protobuf" "github.com/status-im/status-go/protocol/sqlite" "github.com/status-im/status-go/services/wallet/bigint" "github.com/status-im/status-go/t/helpers" + "github.com/status-im/status-go/wakuv2" ) func TestPersistenceSuite(t *testing.T) { @@ -787,7 +787,7 @@ func (s *PersistenceSuite) TestSaveShardInfo() { s.Require().Nil(resultShard) // not nil shard - expectedShard := &shard.Shard{ + expectedShard := &wakuv2.Shard{ Cluster: 1, Index: 2, } diff --git a/protocol/communities_messenger_token_permissions_test.go b/protocol/communities_messenger_token_permissions_test.go index bbf05984c67..8cfe2b74a30 100644 --- a/protocol/communities_messenger_token_permissions_test.go +++ b/protocol/communities_messenger_token_permissions_test.go @@ -25,13 +25,13 @@ import ( "github.com/status-im/status-go/eth-node/types" "github.com/status-im/status-go/params" "github.com/status-im/status-go/protocol/common" - "github.com/status-im/status-go/protocol/common/shard" "github.com/status-im/status-go/protocol/communities" "github.com/status-im/status-go/protocol/protobuf" "github.com/status-im/status-go/protocol/requests" "github.com/status-im/status-go/protocol/transport" "github.com/status-im/status-go/protocol/tt" "github.com/status-im/status-go/services/wallet/thirdparty" + "github.com/status-im/status-go/wakuv2" ) const testChainID1 = 1 @@ -488,11 +488,12 @@ func (s *MessengerCommunitiesTokenPermissionsSuite) TestBecomeMemberPermissions( cfg := testWakuV2Config{ logger: s.logger.Named("store-node-waku"), enableStore: false, - clusterID: shard.MainStatusShardCluster, + clusterID: wakuv2.MainStatusShardCluster, } wakuStoreNode := NewTestWakuV2(&s.Suite, cfg) - storeNodeListenAddresses := wakuStoreNode.ListenAddresses() + storeNodeListenAddresses, err := wakuStoreNode.ListenAddresses() + s.Require().NoError(err) s.Require().LessOrEqual(1, len(storeNodeListenAddresses)) storeNodeAddress := storeNodeListenAddresses[0] diff --git a/protocol/linkpreview_unfurler_status.go b/protocol/linkpreview_unfurler_status.go index d4e6e8e4dd9..6da0194ee25 100644 --- a/protocol/linkpreview_unfurler_status.go +++ b/protocol/linkpreview_unfurler_status.go @@ -8,8 +8,8 @@ import ( "github.com/status-im/status-go/api/multiformat" "github.com/status-im/status-go/images" "github.com/status-im/status-go/protocol/common" - "github.com/status-im/status-go/protocol/common/shard" "github.com/status-im/status-go/protocol/communities" + "github.com/status-im/status-go/wakuv2" ) type StatusUnfurler struct { @@ -83,7 +83,7 @@ func (u *StatusUnfurler) buildContactData(publicKey string) (*common.StatusConta return c, nil } -func (u *StatusUnfurler) buildCommunityData(communityID string, shard *shard.Shard) (*communities.Community, *common.StatusCommunityLinkPreview, error) { +func (u *StatusUnfurler) buildCommunityData(communityID string, shard *wakuv2.Shard) (*communities.Community, *common.StatusCommunityLinkPreview, error) { // This automatically checks the database community, err := u.m.FetchCommunity(&FetchCommunityRequest{ CommunityKey: communityID, @@ -108,7 +108,7 @@ func (u *StatusUnfurler) buildCommunityData(communityID string, shard *shard.Sha return community, statusCommunityLinkPreviews, nil } -func (u *StatusUnfurler) buildChannelData(channelUUID string, communityID string, communityShard *shard.Shard) (*common.StatusCommunityChannelLinkPreview, error) { +func (u *StatusUnfurler) buildChannelData(channelUUID string, communityID string, communityShard *wakuv2.Shard) (*common.StatusCommunityChannelLinkPreview, error) { community, communityData, err := u.buildCommunityData(communityID, communityShard) if err != nil { return nil, fmt.Errorf("failed to build channel community data: %w", err) diff --git a/protocol/messenger.go b/protocol/messenger.go index 7ea39f012c7..36b65976ca8 100644 --- a/protocol/messenger.go +++ b/protocol/messenger.go @@ -38,13 +38,13 @@ import ( "github.com/status-im/status-go/eth-node/types" "github.com/status-im/status-go/images" multiaccountscommon "github.com/status-im/status-go/multiaccounts/common" + "github.com/status-im/status-go/wakuv2" "github.com/status-im/status-go/multiaccounts" "github.com/status-im/status-go/multiaccounts/accounts" "github.com/status-im/status-go/multiaccounts/settings" "github.com/status-im/status-go/protocol/anonmetrics" "github.com/status-im/status-go/protocol/common" - "github.com/status-im/status-go/protocol/common/shard" "github.com/status-im/status-go/protocol/communities" "github.com/status-im/status-go/protocol/encryption" "github.com/status-im/status-go/protocol/encryption/multidevice" @@ -1738,7 +1738,7 @@ func (m *Messenger) InitFilters() error { logger := m.logger.With(zap.String("site", "Init")) // Community requests will arrive in this pubsub topic - err := m.SubscribeToPubsubTopic(shard.DefaultNonProtectedPubsubTopic(), nil) + err := m.SubscribeToPubsubTopic(wakuv2.DefaultNonProtectedPubsubTopic(), nil) if err != nil { return err } diff --git a/protocol/messenger_communities.go b/protocol/messenger_communities.go index 9d7ad082b91..cd8cb9944d7 100644 --- a/protocol/messenger_communities.go +++ b/protocol/messenger_communities.go @@ -25,6 +25,7 @@ import ( gocommon "github.com/status-im/status-go/common" utils "github.com/status-im/status-go/common" + "github.com/status-im/status-go/wakuv2" "github.com/status-im/status-go/account" multiaccountscommon "github.com/status-im/status-go/multiaccounts/common" @@ -34,7 +35,6 @@ import ( "github.com/status-im/status-go/images" "github.com/status-im/status-go/multiaccounts/accounts" "github.com/status-im/status-go/protocol/common" - "github.com/status-im/status-go/protocol/common/shard" "github.com/status-im/status-go/protocol/communities" "github.com/status-im/status-go/protocol/communities/token" "github.com/status-im/status-go/protocol/discord" @@ -89,10 +89,10 @@ const ( type FetchCommunityRequest struct { // CommunityKey should be either a public or a private community key - CommunityKey string `json:"communityKey"` - Shard *shard.Shard `json:"shard"` - TryDatabase bool `json:"tryDatabase"` - WaitForResponse bool `json:"waitForResponse"` + CommunityKey string `json:"communityKey"` + Shard *wakuv2.Shard `json:"shard"` + TryDatabase bool `json:"tryDatabase"` + WaitForResponse bool `json:"waitForResponse"` } func (r *FetchCommunityRequest) Validate() error { @@ -346,7 +346,7 @@ func (m *Messenger) handleCommunitiesSubscription(c chan *communities.Subscripti Sender: community.PrivateKey(), SkipEncryptionLayer: true, MessageType: protobuf.ApplicationMetadataMessage_COMMUNITY_USER_KICKED, - PubsubTopic: shard.DefaultNonProtectedPubsubTopic(), + PubsubTopic: wakuv2.DefaultNonProtectedPubsubTopic(), } _, err = m.sender.SendPrivate(context.Background(), pk, rawMessage) @@ -681,7 +681,7 @@ func (m *Messenger) handleCommunitySharedAddressesRequest(state *ReceivedMessage CommunityID: community.ID(), SkipEncryptionLayer: true, MessageType: protobuf.ApplicationMetadataMessage_COMMUNITY_SHARED_ADDRESSES_RESPONSE, - PubsubTopic: shard.DefaultNonProtectedPubsubTopic(), + PubsubTopic: wakuv2.DefaultNonProtectedPubsubTopic(), ResendType: common.ResendTypeRawMessage, ResendMethod: common.ResendMethodSendPrivate, Recipients: []*ecdsa.PublicKey{signer}, @@ -1044,7 +1044,7 @@ func (m *Messenger) JoinCommunity(ctx context.Context, communityID types.HexByte return mr, nil } -func (m *Messenger) subscribeToCommunityShard(communityID []byte, shard *shard.Shard) error { +func (m *Messenger) subscribeToCommunityShard(communityID []byte, shard *wakuv2.Shard) error { if m.transport.WakuVersion() != 2 { return nil } @@ -1065,7 +1065,7 @@ func (m *Messenger) subscribeToCommunityShard(communityID []byte, shard *shard.S return m.transport.SubscribeToPubsubTopic(pubsubTopic, pubK) } -func (m *Messenger) unsubscribeFromShard(shard *shard.Shard) error { +func (m *Messenger) unsubscribeFromShard(shard *wakuv2.Shard) error { if m.transport.WakuVersion() != 2 { return nil } @@ -1494,7 +1494,7 @@ func (m *Messenger) RequestToJoinCommunity(request *requests.RequestToJoinCommun ResendType: common.ResendTypeRawMessage, SkipEncryptionLayer: true, MessageType: protobuf.ApplicationMetadataMessage_COMMUNITY_REQUEST_TO_JOIN, - PubsubTopic: shard.DefaultNonProtectedPubsubTopic(), + PubsubTopic: wakuv2.DefaultNonProtectedPubsubTopic(), Priority: &common.HighPriority, } @@ -1872,7 +1872,7 @@ func (m *Messenger) CancelRequestToJoinCommunity(ctx context.Context, request *r CommunityID: community.ID(), SkipEncryptionLayer: true, MessageType: protobuf.ApplicationMetadataMessage_COMMUNITY_CANCEL_REQUEST_TO_JOIN, - PubsubTopic: shard.DefaultNonProtectedPubsubTopic(), + PubsubTopic: wakuv2.DefaultNonProtectedPubsubTopic(), ResendType: common.ResendTypeRawMessage, Priority: &common.HighPriority, } @@ -2028,7 +2028,7 @@ func (m *Messenger) acceptRequestToJoinCommunity(requestToJoin *communities.Requ CommunityID: community.ID(), SkipEncryptionLayer: true, MessageType: protobuf.ApplicationMetadataMessage_COMMUNITY_REQUEST_TO_JOIN_RESPONSE, - PubsubTopic: shard.DefaultNonProtectedPubsubTopic(), + PubsubTopic: wakuv2.DefaultNonProtectedPubsubTopic(), ResendType: common.ResendTypeRawMessage, ResendMethod: common.ResendMethodSendPrivate, Recipients: []*ecdsa.PublicKey{pk}, @@ -2503,7 +2503,7 @@ func (m *Messenger) DefaultFilters(o *communities.Community) []transport.Filters {ChatID: updatesChannelID, PubsubTopic: communityPubsubTopic}, {ChatID: mlChannelID, PubsubTopic: communityPubsubTopic}, {ChatID: memberUpdateChannelID, PubsubTopic: communityPubsubTopic}, - {ChatID: uncompressedPubKey, PubsubTopic: shard.DefaultNonProtectedPubsubTopic()}, + {ChatID: uncompressedPubKey, PubsubTopic: wakuv2.DefaultNonProtectedPubsubTopic()}, } return filters @@ -3562,7 +3562,7 @@ func (m *Messenger) HandleCommunityShardKey(state *ReceivedMessageState, message } func (m *Messenger) handleCommunityShardAndFiltersFromProto(community *communities.Community, message *protobuf.CommunityShardKey) error { - err := m.communitiesManager.UpdateShard(community, shard.FromProtobuff(message.Shard), message.Clock) + err := m.communitiesManager.UpdateShard(community, wakuv2.FromProtobuff(message.Shard), message.Clock) if err != nil { return err } @@ -3584,7 +3584,7 @@ func (m *Messenger) handleCommunityShardAndFiltersFromProto(community *communiti } // Unsubscribing from existing shard - if community.Shard() != nil && community.Shard() != shard.FromProtobuff(message.GetShard()) { + if community.Shard() != nil && community.Shard() != wakuv2.FromProtobuff(message.GetShard()) { err := m.unsubscribeFromShard(community.Shard()) if err != nil { return err @@ -3598,7 +3598,7 @@ func (m *Messenger) handleCommunityShardAndFiltersFromProto(community *communiti return err } // Update community filters in case of change of shard - if community.Shard() != shard.FromProtobuff(message.GetShard()) { + if community.Shard() != wakuv2.FromProtobuff(message.GetShard()) { err = m.UpdateCommunityFilters(community) if err != nil { return err diff --git a/protocol/messenger_communities_sharding_test.go b/protocol/messenger_communities_sharding_test.go index 307df61ad47..962cd62b1bb 100644 --- a/protocol/messenger_communities_sharding_test.go +++ b/protocol/messenger_communities_sharding_test.go @@ -12,11 +12,11 @@ import ( gethbridge "github.com/status-im/status-go/eth-node/bridge/geth" "github.com/status-im/status-go/eth-node/types" "github.com/status-im/status-go/protocol/common" - "github.com/status-im/status-go/protocol/common/shard" "github.com/status-im/status-go/protocol/communities" "github.com/status-im/status-go/protocol/protobuf" "github.com/status-im/status-go/protocol/requests" "github.com/status-im/status-go/protocol/tt" + "github.com/status-im/status-go/wakuv2" ) func TestMessengerCommunitiesShardingSuite(t *testing.T) { @@ -108,7 +108,7 @@ func (s *MessengerCommunitiesShardingSuite) TearDownTest() { _ = s.logger.Sync() } -func (s *MessengerCommunitiesShardingSuite) testPostToCommunityChat(shard *shard.Shard, community *communities.Community, chat *Chat) { +func (s *MessengerCommunitiesShardingSuite) testPostToCommunityChat(shard *wakuv2.Shard, community *communities.Community, chat *Chat) { _, err := s.owner.SetCommunityShard(&requests.SetCommunityShard{ CommunityID: community.ID(), Shard: shard, @@ -144,8 +144,8 @@ func (s *MessengerCommunitiesShardingSuite) TestPostToCommunityChat() { // Members should be able to receive messages in a community with sharding enabled. { - shard := &shard.Shard{ - Cluster: shard.MainStatusShardCluster, + shard := &wakuv2.Shard{ + Cluster: wakuv2.MainStatusShardCluster, Index: 128, } s.testPostToCommunityChat(shard, community, chat) @@ -153,8 +153,8 @@ func (s *MessengerCommunitiesShardingSuite) TestPostToCommunityChat() { // Members should be able to receive messages in a community where the sharding configuration has been edited. { - shard := &shard.Shard{ - Cluster: shard.MainStatusShardCluster, + shard := &wakuv2.Shard{ + Cluster: wakuv2.MainStatusShardCluster, Index: 256, } s.testPostToCommunityChat(shard, community, chat) @@ -162,8 +162,8 @@ func (s *MessengerCommunitiesShardingSuite) TestPostToCommunityChat() { // Members should continue to receive messages in a community if it is moved back to default shard. { - shard := &shard.Shard{ - Cluster: shard.MainStatusShardCluster, + shard := &wakuv2.Shard{ + Cluster: wakuv2.MainStatusShardCluster, Index: 32, } s.testPostToCommunityChat(shard, community, chat) @@ -176,8 +176,8 @@ func (s *MessengerCommunitiesShardingSuite) TestIgnoreOutdatedShardKey() { advertiseCommunityToUserOldWay(&s.Suite, community, s.owner, s.alice) joinCommunity(&s.Suite, community.ID(), s.owner, s.alice, alicePassword, []string{aliceAddress1}) - shard := &shard.Shard{ - Cluster: shard.MainStatusShardCluster, + shard := &wakuv2.Shard{ + Cluster: wakuv2.MainStatusShardCluster, Index: 128, } diff --git a/protocol/messenger_community_shard.go b/protocol/messenger_community_shard.go index 8fdda061d70..ca8f6abda02 100644 --- a/protocol/messenger_community_shard.go +++ b/protocol/messenger_community_shard.go @@ -12,11 +12,11 @@ import ( "github.com/status-im/status-go/eth-node/crypto" "github.com/status-im/status-go/eth-node/types" "github.com/status-im/status-go/protocol/common" - "github.com/status-im/status-go/protocol/common/shard" "github.com/status-im/status-go/protocol/communities" "github.com/status-im/status-go/protocol/protobuf" "github.com/status-im/status-go/protocol/transport" v1protocol "github.com/status-im/status-go/protocol/v1" + "github.com/status-im/status-go/wakuv2" ) func (m *Messenger) sendPublicCommunityShardInfo(community *communities.Community) error { @@ -57,7 +57,7 @@ func (m *Messenger) sendPublicCommunityShardInfo(community *communities.Communit // we don't want to wrap in an encryption layer message SkipEncryptionLayer: true, MessageType: protobuf.ApplicationMetadataMessage_COMMUNITY_PUBLIC_SHARD_INFO, - PubsubTopic: shard.DefaultNonProtectedPubsubTopic(), // it must be sent always to default shard pubsub topic + PubsubTopic: wakuv2.DefaultNonProtectedPubsubTopic(), // it must be sent always to default shard pubsub topic Priority: &common.HighPriority, } @@ -89,7 +89,7 @@ func (m *Messenger) HandleCommunityPublicShardInfo(state *ReceivedMessageState, return err } - err = m.communitiesManager.SaveCommunityShard(publicShardInfo.CommunityId, shard.FromProtobuff(publicShardInfo.Shard), publicShardInfo.Clock) + err = m.communitiesManager.SaveCommunityShard(publicShardInfo.CommunityId, wakuv2.FromProtobuff(publicShardInfo.Shard), publicShardInfo.Clock) if err != nil && err != communities.ErrOldShardInfo { logError(err) return err diff --git a/protocol/messenger_share_urls.go b/protocol/messenger_share_urls.go index 238f06dbdc3..812cd00e4b3 100644 --- a/protocol/messenger_share_urls.go +++ b/protocol/messenger_share_urls.go @@ -15,11 +15,11 @@ import ( "github.com/status-im/status-go/eth-node/crypto" "github.com/status-im/status-go/eth-node/types" "github.com/status-im/status-go/protocol/common" - "github.com/status-im/status-go/protocol/common/shard" "github.com/status-im/status-go/protocol/communities" "github.com/status-im/status-go/protocol/protobuf" "github.com/status-im/status-go/protocol/requests" "github.com/status-im/status-go/services/utils" + "github.com/status-im/status-go/wakuv2" ) type CommunityURLData struct { @@ -49,7 +49,7 @@ type URLDataResponse struct { Community *CommunityURLData `json:"community"` Channel *CommunityChannelURLData `json:"channel"` Contact *ContactURLData `json:"contact"` - Shard *shard.Shard `json:"shard,omitempty"` + Shard *wakuv2.Shard `json:"shard,omitempty"` } const baseShareURL = "https://status.app" @@ -204,7 +204,7 @@ func parseCommunityURLWithData(data string, chatKey string) (*URLDataResponse, e TagIndices: tagIndices, CommunityID: types.EncodeHex(communityID), }, - Shard: shard.FromProtobuff(urlDataProto.Shard), + Shard: wakuv2.FromProtobuff(urlDataProto.Shard), }, nil } @@ -380,7 +380,7 @@ func parseCommunityChannelURLWithData(data string, chatKey string) (*URLDataResp Color: channelProto.Color, ChannelUUID: channelProto.Uuid, }, - Shard: shard.FromProtobuff(urlDataProto.Shard), + Shard: wakuv2.FromProtobuff(urlDataProto.Shard), }, nil } diff --git a/protocol/messenger_store_node_request_manager.go b/protocol/messenger_store_node_request_manager.go index 1db7388a507..bf8da08e3fa 100644 --- a/protocol/messenger_store_node_request_manager.go +++ b/protocol/messenger_store_node_request_manager.go @@ -9,7 +9,6 @@ import ( gocommon "github.com/status-im/status-go/common" "github.com/status-im/status-go/eth-node/crypto" - "github.com/status-im/status-go/protocol/common/shard" "github.com/waku-org/go-waku/waku/v2/api/history" "go.uber.org/zap" @@ -17,6 +16,7 @@ import ( "github.com/status-im/status-go/eth-node/types" "github.com/status-im/status-go/protocol/communities" "github.com/status-im/status-go/protocol/transport" + "github.com/status-im/status-go/wakuv2" ) const ( @@ -82,7 +82,7 @@ func (m *StoreNodeRequestManager) FetchCommunity(community communities.Community zap.Any("community", community), zap.Any("config", cfg)) - requestCommunity := func(communityID string, shard *shard.Shard) (*communities.Community, StoreNodeRequestStats, error) { + requestCommunity := func(communityID string, shard *wakuv2.Shard) (*communities.Community, StoreNodeRequestStats, error) { channel, err := m.subscribeToRequest(storeNodeCommunityRequest, communityID, shard, cfg) if err != nil { return nil, StoreNodeRequestStats{}, fmt.Errorf("failed to create a request for community: %w", err) @@ -100,7 +100,7 @@ func (m *StoreNodeRequestManager) FetchCommunity(community communities.Community communityShard := community.Shard if communityShard == nil { id := transport.CommunityShardInfoTopic(community.CommunityID) - fetchedShard, err := m.subscribeToRequest(storeNodeShardRequest, id, shard.DefaultNonProtectedShard(), cfg) + fetchedShard, err := m.subscribeToRequest(storeNodeShardRequest, id, wakuv2.DefaultNonProtectedShard(), cfg) if err != nil { return nil, StoreNodeRequestStats{}, fmt.Errorf("failed to create a shard info request: %w", err) } @@ -178,7 +178,7 @@ func (m *StoreNodeRequestManager) FetchContact(contactID string, opts []StoreNod // subscribeToRequest checks if a request for given community/contact is already in progress, creates and installs // a new one if not found, and returns a subscription to the result of the found/started request. // The subscription can then be used to get the result of the request, this could be either a community/contact or an error. -func (m *StoreNodeRequestManager) subscribeToRequest(requestType storeNodeRequestType, dataID string, shard *shard.Shard, cfg StoreNodeRequestConfig) (storeNodeResponseSubscription, error) { +func (m *StoreNodeRequestManager) subscribeToRequest(requestType storeNodeRequestType, dataID string, shard *wakuv2.Shard, cfg StoreNodeRequestConfig) (storeNodeResponseSubscription, error) { // It's important to unlock only after getting the subscription channel. // We also lock `activeRequestsLock` during finalizing the requests. This ensures that the subscription // created in this function will get the result even if the requests proceeds faster than this function ends. @@ -232,7 +232,7 @@ func (m *StoreNodeRequestManager) newStoreNodeRequest() *storeNodeRequest { // getFilter checks if a filter for a given community is already created and creates one of not found. // Returns the found/created filter, a flag if the filter was created by the function and an error. -func (m *StoreNodeRequestManager) getFilter(requestType storeNodeRequestType, dataID string, shard *shard.Shard) (*transport.Filter, bool, error) { +func (m *StoreNodeRequestManager) getFilter(requestType storeNodeRequestType, dataID string, shard *wakuv2.Shard) (*transport.Filter, bool, error) { // First check if such filter already exists. filter := m.messenger.transport.FilterByChatID(dataID) if filter != nil { @@ -334,7 +334,7 @@ type storeNodeRequestResult struct { // One of data fields (community or contact) will be present depending on request type community *communities.Community contact *Contact - shard *shard.Shard + shard *wakuv2.Shard } type storeNodeResponseSubscription = chan storeNodeRequestResult diff --git a/protocol/messenger_storenode_comunity_test.go b/protocol/messenger_storenode_comunity_test.go index 51cbf712ff4..74b796b538c 100644 --- a/protocol/messenger_storenode_comunity_test.go +++ b/protocol/messenger_storenode_comunity_test.go @@ -10,9 +10,9 @@ import ( "github.com/multiformats/go-multiaddr" "github.com/status-im/status-go/protocol/storenodes" + "github.com/status-im/status-go/wakuv2" gethbridge "github.com/status-im/status-go/eth-node/bridge/geth" - "github.com/status-im/status-go/protocol/common/shard" "github.com/status-im/status-go/protocol/communities" "github.com/status-im/status-go/protocol/tt" @@ -92,11 +92,12 @@ func (s *MessengerStoreNodeCommunitySuite) createStore(name string) (*waku2.Waku cfg := testWakuV2Config{ logger: s.logger.Named(name), enableStore: true, - clusterID: shard.MainStatusShardCluster, + clusterID: wakuv2.MainStatusShardCluster, } storeNode := NewTestWakuV2(&s.Suite, cfg) - addresses := storeNode.ListenAddresses() + addresses, err := storeNode.ListenAddresses() + s.Require().NoError(err) s.Require().GreaterOrEqual(len(addresses), 1, "no storenode listen address") return storeNode, addresses[0] } @@ -109,7 +110,7 @@ func (s *MessengerStoreNodeCommunitySuite) newMessenger(name string, storenodeAd cfg := testWakuV2Config{ logger: logger, enableStore: false, - clusterID: shard.MainStatusShardCluster, + clusterID: wakuv2.MainStatusShardCluster, } wakuV2 := NewTestWakuV2(&s.Suite, cfg) wakuV2Wrapper := gethbridge.NewGethWakuV2Wrapper(wakuV2) diff --git a/protocol/messenger_storenode_request_test.go b/protocol/messenger_storenode_request_test.go index 84bede934e9..7764b4a8a68 100644 --- a/protocol/messenger_storenode_request_test.go +++ b/protocol/messenger_storenode_request_test.go @@ -24,7 +24,6 @@ import ( "github.com/status-im/status-go/multiaccounts/accounts" "github.com/status-im/status-go/params" "github.com/status-im/status-go/protocol/common" - "github.com/status-im/status-go/protocol/common/shard" "github.com/status-im/status-go/protocol/communities" "github.com/status-im/status-go/protocol/communities/token" "github.com/status-im/status-go/protocol/protobuf" @@ -34,6 +33,7 @@ import ( mailserversDB "github.com/status-im/status-go/services/mailservers" "github.com/status-im/status-go/services/wallet/bigint" "github.com/status-im/status-go/t/helpers" + "github.com/status-im/status-go/wakuv2" waku2 "github.com/status-im/status-go/wakuv2" wakuV2common "github.com/status-im/status-go/wakuv2/common" ) @@ -160,7 +160,7 @@ func (s *MessengerStoreNodeRequestSuite) createStore() { cfg := testWakuV2Config{ logger: s.logger.Named("store-waku"), enableStore: true, - clusterID: shard.MainStatusShardCluster, + clusterID: wakuv2.MainStatusShardCluster, } s.wakuStoreNode = NewTestWakuV2(&s.Suite, cfg) @@ -178,7 +178,7 @@ func (s *MessengerStoreNodeRequestSuite) createOwner() { cfg := testWakuV2Config{ logger: s.logger.Named("owner-waku"), enableStore: false, - clusterID: shard.MainStatusShardCluster, + clusterID: wakuv2.MainStatusShardCluster, } wakuV2 := NewTestWakuV2(&s.Suite, cfg) @@ -199,7 +199,7 @@ func (s *MessengerStoreNodeRequestSuite) createBob() { cfg := testWakuV2Config{ logger: s.logger.Named("bob-waku"), enableStore: false, - clusterID: shard.MainStatusShardCluster, + clusterID: wakuv2.MainStatusShardCluster, } wakuV2 := NewTestWakuV2(&s.Suite, cfg) s.bobWaku = gethbridge.NewGethWakuV2Wrapper(wakuV2) @@ -366,7 +366,8 @@ func (s *MessengerStoreNodeRequestSuite) waitForEnvelopes(subscription <-chan st } func (s *MessengerStoreNodeRequestSuite) wakuListenAddress(waku *waku2.Waku) multiaddr.Multiaddr { - addresses := waku.ListenAddresses() + addresses, err := waku.ListenAddresses() + s.Require().NoError(err) s.Require().LessOrEqual(1, len(addresses)) return addresses[0] } @@ -696,8 +697,8 @@ func (s *MessengerStoreNodeRequestSuite) TestRequestShardAndCommunityInfo() { topicPrivKey, err := crypto.GenerateKey() s.Require().NoError(err) - expectedShard := &shard.Shard{ - Cluster: shard.MainStatusShardCluster, + expectedShard := &wakuv2.Shard{ + Cluster: wakuv2.MainStatusShardCluster, Index: 23, } @@ -841,8 +842,8 @@ type testFetchRealCommunityExampleTokenInfo struct { var testFetchRealCommunityExample = []struct { CommunityID string - CommunityURL string // If set, takes precedence over CommunityID - CommunityShard *shard.Shard // WARNING: I didn't test a sharded community + CommunityURL string // If set, takes precedence over CommunityID + CommunityShard *wakuv2.Shard // WARNING: I didn't test a sharded community Fleet string ClusterID uint16 UserPrivateKeyString string // When empty a new user will be created @@ -863,14 +864,14 @@ var testFetchRealCommunityExample = []struct { CommunityID: "0x03073514d4c14a7d10ae9fc9b0f05abc904d84166a6ac80add58bf6a3542a4e50a", CommunityShard: nil, Fleet: params.FleetStatusProd, - ClusterID: shard.MainStatusShardCluster, + ClusterID: wakuv2.MainStatusShardCluster, }, { // Example 3, // https://status.app/c/CxiACi8KFGFwIHJlcSAxIHN0dCBiZWMgbWVtEgdkc2Fkc2FkGAMiByM0MzYwREYqAxkrHAM=#zQ3shwDYZHtrLE7NqoTGjTWzWUu6hom5D4qxfskLZfgfyGRyL CommunityID: "0x03f64be95ed5c925022265f9250f538f65ed3dcf6e4ef6c139803dc02a3487ae7b", Fleet: params.FleetStatusProd, - ClusterID: shard.MainStatusShardCluster, + ClusterID: wakuv2.MainStatusShardCluster, CheckExpectedEnvelopes: true, ExpectedShardEnvelopes: []string{ @@ -973,7 +974,7 @@ var testFetchRealCommunityExample = []struct { //Example 1, CommunityID: "0x02471dd922756a3a50b623e59cf3b99355d6587e43d5c517eb55f9aea9d3fe9fe9", Fleet: params.FleetStatusProd, - ClusterID: shard.MainStatusShardCluster, + ClusterID: wakuv2.MainStatusShardCluster, CheckExpectedEnvelopes: true, ExpectedShardEnvelopes: []string{ "0xc3e68e838d09e0117b3f3fd27aabe5f5a509d13e9045263c78e6890953d43547", @@ -1013,7 +1014,7 @@ var testFetchRealCommunityExample = []struct { ContractAddress: "0x21F6F5Cb75E81e5104D890D750270eD6538C50cb", }, }, - ClusterID: shard.MainStatusShardCluster, + ClusterID: wakuv2.MainStatusShardCluster, CheckExpectedEnvelopes: false, CustomOptions: []StoreNodeRequestOption{ WithInitialPageSize(1), diff --git a/protocol/messenger_testing_utils.go b/protocol/messenger_testing_utils.go index efed71fc0df..a3ab62330bb 100644 --- a/protocol/messenger_testing_utils.go +++ b/protocol/messenger_testing_utils.go @@ -14,11 +14,11 @@ import ( gocommon "github.com/status-im/status-go/common" "github.com/status-im/status-go/protocol/wakusync" + "github.com/status-im/status-go/wakuv2" "github.com/status-im/status-go/protocol/identity" "github.com/status-im/status-go/eth-node/types" - waku2 "github.com/status-im/status-go/wakuv2" "github.com/stretchr/testify/suite" @@ -206,7 +206,7 @@ func WaitOnSignaledCommunityFound(m *Messenger, action func(), condition func(co } } -func WaitForConnectionStatus(s *suite.Suite, waku *waku2.Waku, action func() bool) { +func WaitForConnectionStatus(s *suite.Suite, waku *wakuv2.Waku, action func() bool) { subscription := waku.SubscribeToConnStatusChanges() defer subscription.Unsubscribe() @@ -238,7 +238,7 @@ func hasAllPeers(m map[peer.ID]types.WakuV2Peer, checkSlice peer.IDSlice) bool { return true } -func WaitForPeersConnected(s *suite.Suite, waku *waku2.Waku, action func() peer.IDSlice) { +func WaitForPeersConnected(s *suite.Suite, waku *wakuv2.Waku, action func() peer.IDSlice) { subscription := waku.SubscribeToConnStatusChanges() defer subscription.Unsubscribe() diff --git a/protocol/requests/set_community_shard.go b/protocol/requests/set_community_shard.go index 0b1240b56d3..6ae52d1bd3d 100644 --- a/protocol/requests/set_community_shard.go +++ b/protocol/requests/set_community_shard.go @@ -4,12 +4,12 @@ import ( "errors" "github.com/status-im/status-go/eth-node/types" - "github.com/status-im/status-go/protocol/common/shard" + "github.com/status-im/status-go/wakuv2" ) type SetCommunityShard struct { CommunityID types.HexBytes `json:"communityId"` - Shard *shard.Shard `json:"shard,omitempty"` + Shard *wakuv2.Shard `json:"shard,omitempty"` PrivateKey *types.HexBytes `json:"privateKey,omitempty"` } @@ -19,7 +19,7 @@ func (s *SetCommunityShard) Validate() error { } if s.Shard != nil { // TODO: for now only MainStatusShard(16) is accepted - if s.Shard.Cluster != shard.MainStatusShardCluster { + if s.Shard.Cluster != wakuv2.MainStatusShardCluster { return errors.New("invalid shard cluster") } if s.Shard.Index > 1023 { diff --git a/protocol/transport/filters_manager.go b/protocol/transport/filters_manager.go index 5393d63bf78..acf3002d73f 100644 --- a/protocol/transport/filters_manager.go +++ b/protocol/transport/filters_manager.go @@ -11,7 +11,7 @@ import ( "go.uber.org/zap" "github.com/status-im/status-go/eth-node/types" - "github.com/status-im/status-go/protocol/common/shard" + "github.com/status-im/status-go/wakuv2" ) const ( @@ -141,7 +141,7 @@ func (f *FiltersManager) InitPublicFilters(publicFiltersToInit []FiltersToInitia } type CommunityFilterToInitialize struct { - Shard *shard.Shard + Shard *wakuv2.Shard PrivKey *ecdsa.PrivateKey } @@ -158,7 +158,7 @@ func (f *FiltersManager) InitCommunityFilters(communityFiltersToInitialize []Com } topics := make([]string, 0) - topics = append(topics, shard.DefaultNonProtectedPubsubTopic()) + topics = append(topics, wakuv2.DefaultNonProtectedPubsubTopic()) topics = append(topics, communityFilter.Shard.PubsubTopic()) for _, pubsubTopic := range topics { diff --git a/protocol/waku_builder_test.go b/protocol/waku_builder_test.go index 7c8adab32d1..1134d5eece9 100644 --- a/protocol/waku_builder_test.go +++ b/protocol/waku_builder_test.go @@ -12,7 +12,6 @@ import ( "github.com/status-im/status-go/appdatabase" gethbridge "github.com/status-im/status-go/eth-node/bridge/geth" "github.com/status-im/status-go/eth-node/types" - "github.com/status-im/status-go/protocol/common/shard" "github.com/status-im/status-go/t/helpers" waku2 "github.com/status-im/status-go/wakuv2" ) @@ -62,7 +61,7 @@ func NewTestWakuV2(s *suite.Suite, cfg testWakuV2Config) *waku2.Waku { err = wakuNode.Start() if cfg.enableStore { - err := wakuNode.SubscribeToPubsubTopic(shard.DefaultNonProtectedPubsubTopic(), nil) + err := wakuNode.SubscribeToPubsubTopic(waku2.DefaultNonProtectedPubsubTopic(), nil) s.Require().NoError(err) } s.Require().NoError(err) @@ -78,7 +77,7 @@ func CreateWakuV2Network(s *suite.Suite, parentLogger *zap.Logger, nodeNames []s nodes[i] = NewTestWakuV2(s, testWakuV2Config{ logger: parentLogger.Named("waku-" + name), enableStore: false, - clusterID: shard.MainStatusShardCluster, + clusterID: waku2.MainStatusShardCluster, }) } @@ -89,9 +88,10 @@ func CreateWakuV2Network(s *suite.Suite, parentLogger *zap.Logger, nodeNames []s continue } - addrs := nodes[j].ListenAddresses() + addrs, err := nodes[j].ListenAddresses() + s.Require().NoError(err) s.Require().Greater(len(addrs), 0) - _, err := nodes[i].AddRelayPeer(addrs[0]) + _, err = nodes[i].AddRelayPeer(addrs[0]) s.Require().NoError(err) err = nodes[i].DialPeer(addrs[0]) s.Require().NoError(err) diff --git a/services/ext/api.go b/services/ext/api.go index 156b50ca8b0..1aa0808153d 100644 --- a/services/ext/api.go +++ b/services/ext/api.go @@ -16,6 +16,7 @@ import ( "github.com/status-im/status-go/services/browsers" "github.com/status-im/status-go/services/wallet" "github.com/status-im/status-go/services/wallet/bigint" + "github.com/status-im/status-go/wakuv2" "github.com/ethereum/go-ethereum/common/hexutil" "github.com/ethereum/go-ethereum/log" @@ -32,7 +33,6 @@ import ( "github.com/status-im/status-go/multiaccounts/settings" "github.com/status-im/status-go/protocol" "github.com/status-im/status-go/protocol/common" - "github.com/status-im/status-go/protocol/common/shard" "github.com/status-im/status-go/protocol/communities" "github.com/status-im/status-go/protocol/communities/token" "github.com/status-im/status-go/protocol/discord" @@ -1308,7 +1308,7 @@ func (api *PublicAPI) RequestCommunityInfoFromMailserver(communityID string) (*c // Deprecated: RequestCommunityInfoFromMailserverWithShard is deprecated in favor of // configurable FetchCommunity. -func (api *PublicAPI) RequestCommunityInfoFromMailserverWithShard(communityID string, shard *shard.Shard) (*communities.Community, error) { +func (api *PublicAPI) RequestCommunityInfoFromMailserverWithShard(communityID string, shard *wakuv2.Shard) (*communities.Community, error) { request := &protocol.FetchCommunityRequest{ CommunityKey: communityID, Shard: shard, @@ -1333,7 +1333,7 @@ func (api *PublicAPI) RequestCommunityInfoFromMailserverAsync(communityID string // Deprecated: RequestCommunityInfoFromMailserverAsyncWithShard is deprecated in favor of // configurable FetchCommunity. -func (api *PublicAPI) RequestCommunityInfoFromMailserverAsyncWithShard(communityID string, shard *shard.Shard) error { +func (api *PublicAPI) RequestCommunityInfoFromMailserverAsyncWithShard(communityID string, shard *wakuv2.Shard) error { request := &protocol.FetchCommunityRequest{ CommunityKey: communityID, Shard: shard, diff --git a/services/mailservers/api_test.go b/services/mailservers/api_test.go index c0d847b981b..c830a15e3b9 100644 --- a/services/mailservers/api_test.go +++ b/services/mailservers/api_test.go @@ -8,10 +8,10 @@ import ( "github.com/status-im/status-go/appdatabase" "github.com/status-im/status-go/eth-node/types" - "github.com/status-im/status-go/protocol/common/shard" "github.com/status-im/status-go/protocol/sqlite" "github.com/status-im/status-go/protocol/transport" "github.com/status-im/status-go/t/helpers" + "github.com/status-im/status-go/wakuv2" ) func setupTestDB(t *testing.T) (*Database, func()) { @@ -62,9 +62,9 @@ func TestTopic(t *testing.T) { defer close() topicA := "0x61000000" topicD := "0x64000000" - topic1 := MailserverTopic{PubsubTopic: shard.DefaultShardPubsubTopic(), ContentTopic: topicA, LastRequest: 1} - topic2 := MailserverTopic{PubsubTopic: shard.DefaultShardPubsubTopic(), ContentTopic: "0x6200000", LastRequest: 2} - topic3 := MailserverTopic{PubsubTopic: shard.DefaultShardPubsubTopic(), ContentTopic: "0x6300000", LastRequest: 3} + topic1 := MailserverTopic{PubsubTopic: wakuv2.DefaultShardPubsubTopic(), ContentTopic: topicA, LastRequest: 1} + topic2 := MailserverTopic{PubsubTopic: wakuv2.DefaultShardPubsubTopic(), ContentTopic: "0x6200000", LastRequest: 2} + topic3 := MailserverTopic{PubsubTopic: wakuv2.DefaultShardPubsubTopic(), ContentTopic: "0x6300000", LastRequest: 3} require.NoError(t, db.AddTopic(topic1)) require.NoError(t, db.AddTopic(topic2)) @@ -77,14 +77,14 @@ func TestTopic(t *testing.T) { filters := []*transport.Filter{ // Existing topic, is not updated { - PubsubTopic: shard.DefaultShardPubsubTopic(), + PubsubTopic: wakuv2.DefaultShardPubsubTopic(), ContentTopic: types.BytesToTopic([]byte{0x61}), }, // Non existing topic is not inserted { Discovery: true, Negotiated: true, - PubsubTopic: shard.DefaultShardPubsubTopic(), + PubsubTopic: wakuv2.DefaultShardPubsubTopic(), ContentTopic: types.BytesToTopic([]byte{0x64}), }, } @@ -160,7 +160,7 @@ func TestAddGetDeleteMailserverTopics(t *testing.T) { defer close() api := &API{db: db} testTopic := MailserverTopic{ - PubsubTopic: shard.DefaultShardPubsubTopic(), + PubsubTopic: wakuv2.DefaultShardPubsubTopic(), ContentTopic: "topic-001", ChatIDs: []string{"chatID01", "chatID02"}, LastRequest: 10, @@ -173,14 +173,14 @@ func TestAddGetDeleteMailserverTopics(t *testing.T) { require.NoError(t, err) require.EqualValues(t, []MailserverTopic{testTopic}, topics) - err = api.DeleteMailserverTopic(context.Background(), shard.DefaultShardPubsubTopic(), testTopic.ContentTopic) + err = api.DeleteMailserverTopic(context.Background(), wakuv2.DefaultShardPubsubTopic(), testTopic.ContentTopic) require.NoError(t, err) topics, err = api.GetMailserverTopics(context.Background()) require.NoError(t, err) require.EqualValues(t, ([]MailserverTopic)(nil), topics) // Delete non-existing topic. - err = api.DeleteMailserverTopic(context.Background(), shard.DefaultShardPubsubTopic(), "non-existing-topic") + err = api.DeleteMailserverTopic(context.Background(), wakuv2.DefaultShardPubsubTopic(), "non-existing-topic") require.NoError(t, err) } diff --git a/services/status/service.go b/services/status/service.go index abfa1531ebe..d8fac8bd47d 100644 --- a/services/status/service.go +++ b/services/status/service.go @@ -10,7 +10,7 @@ import ( "github.com/status-im/status-go/eth-node/types" "github.com/status-im/status-go/protocol" - "github.com/status-im/status-go/protocol/common/shard" + "github.com/status-im/status-go/wakuv2" ) // Make sure that Service implements node.Lifecycle interface. @@ -70,7 +70,7 @@ type PublicAPI struct { service *Service } -func (p *PublicAPI) CommunityInfo(communityID types.HexBytes, shard *shard.Shard) (json.RawMessage, error) { +func (p *PublicAPI) CommunityInfo(communityID types.HexBytes, shard *wakuv2.Shard) (json.RawMessage, error) { if p.service.messenger == nil { return nil, ErrNotInitialized } diff --git a/third_party/nwaku b/third_party/nwaku new file mode 160000 index 00000000000..c861fa9f756 --- /dev/null +++ b/third_party/nwaku @@ -0,0 +1 @@ +Subproject commit c861fa9f7560068874570598c81b7a1425a9e931 diff --git a/wakuv2/api.go b/wakuv2/api.go index 49be14a32c1..f24e98f1883 100644 --- a/wakuv2/api.go +++ b/wakuv2/api.go @@ -18,6 +18,7 @@ package wakuv2 +/* TODO-nwaku import ( "context" "crypto/ecdsa" @@ -512,4 +513,4 @@ func (api *PublicWakuAPI) NewMessageFilter(req Criteria) (string, error) { api.mu.Unlock() return id, nil -} +} */ \ No newline at end of file diff --git a/wakuv2/api_test.go b/wakuv2/api_test.go index ef8c7ab5e7d..10d16a4c332 100644 --- a/wakuv2/api_test.go +++ b/wakuv2/api_test.go @@ -18,13 +18,13 @@ package wakuv2 +/* TODO-nwaku import ( "testing" "time" "golang.org/x/exp/maps" - "github.com/status-im/status-go/protocol/common/shard" "github.com/status-im/status-go/wakuv2/common" ) @@ -57,7 +57,7 @@ func TestMultipleTopicCopyInNewMessageFilter(t *testing.T) { } found := false - candidates := w.filters.GetWatchersByTopic(shard.DefaultShardPubsubTopic(), t1) + candidates := w.filters.GetWatchersByTopic(DefaultShardPubsubTopic(), t1) for _, f := range candidates { if maps.Equal(f.ContentTopics, common.NewTopicSet(crit.ContentTopics)) { found = true @@ -68,4 +68,4 @@ func TestMultipleTopicCopyInNewMessageFilter(t *testing.T) { if !found { t.Fatalf("Could not find filter with both topics") } -} +} */ \ No newline at end of file diff --git a/wakuv2/config.go b/wakuv2/config.go index 0d73fe98a85..7bad536b38c 100644 --- a/wakuv2/config.go +++ b/wakuv2/config.go @@ -23,8 +23,6 @@ import ( "go.uber.org/zap" - "github.com/status-im/status-go/protocol/common/shard" - ethdisc "github.com/ethereum/go-ethereum/p2p/dnsdisc" "github.com/status-im/status-go/wakuv2/common" @@ -117,10 +115,10 @@ func setDefaults(cfg *Config) *Config { } if cfg.DefaultShardPubsubTopic == "" { - cfg.DefaultShardPubsubTopic = shard.DefaultShardPubsubTopic() + cfg.DefaultShardPubsubTopic = DefaultShardPubsubTopic() //For now populating with both used shards, but this can be populated from user subscribed communities etc once community sharding is implemented - cfg.DefaultShardedPubsubTopics = append(cfg.DefaultShardedPubsubTopics, shard.DefaultShardPubsubTopic()) - cfg.DefaultShardedPubsubTopics = append(cfg.DefaultShardedPubsubTopics, shard.DefaultNonProtectedPubsubTopic()) + cfg.DefaultShardedPubsubTopics = append(cfg.DefaultShardedPubsubTopics, DefaultShardPubsubTopic()) + cfg.DefaultShardedPubsubTopics = append(cfg.DefaultShardedPubsubTopics, DefaultNonProtectedPubsubTopic()) } return cfg diff --git a/wakuv2/waku.go b/wakuv2/gowaku.go similarity index 98% rename from wakuv2/waku.go rename to wakuv2/gowaku.go index 1869f3a427d..8976f44905e 100644 --- a/wakuv2/waku.go +++ b/wakuv2/gowaku.go @@ -1,3 +1,6 @@ +//go:build !use_nwaku +// +build !use_nwaku + // Copyright 2019 The Waku Library Authors. // // The Waku library is free software: you can redistribute it and/or modify @@ -1575,8 +1578,8 @@ func (w *Waku) RelayPeersByTopic(topic string) (*types.PeerList, error) { }, nil } -func (w *Waku) ListenAddresses() []multiaddr.Multiaddr { - return w.node.ListenAddresses() +func (w *Waku) ListenAddresses() ([]multiaddr.Multiaddr, error) { + return w.node.ListenAddresses(), nil } func (w *Waku) ENR() (*enode.Node, error) { @@ -1947,3 +1950,24 @@ func FormatPeerConnFailures(wakuNode *node.WakuNode) map[string]int { func (w *Waku) LegacyStoreNode() legacy_store.Store { return w.node.LegacyStore() } + +func (w *Waku) WakuLightpushPublish(message *pb.WakuMessage, pubsubTopic string) (string, error) { + msgHash, err := w.node.Lightpush().Publish(w.ctx, message, lightpush.WithPubSubTopic(pubsubTopic)) + if err != nil { + return "", err + } + return msgHash.String(), nil +} + +func (w *Waku) WakuRelayPublish(message *pb.WakuMessage, pubsubTopic string) (string, error) { + msgHash, err := w.node.Relay().Publish(w.ctx, message, relay.WithPubSubTopic(pubsubTopic)) + if err != nil { + return "", err + } + return msgHash.String(), nil +} + +func (w *Waku) ListPeersInMesh(pubsubTopic string) (int, error) { + listPeers := w.node.Relay().PubSub().ListPeers(pubsubTopic) + return len(listPeers), nil +} diff --git a/wakuv2/nwaku.go b/wakuv2/nwaku.go index ed9f3e80ce3..a1d70379817 100644 --- a/wakuv2/nwaku.go +++ b/wakuv2/nwaku.go @@ -1,58 +1,2594 @@ +//go:build use_nwaku +// +build use_nwaku + package wakuv2 +/* + #cgo LDFLAGS: -L../third_party/nwaku/build/ -lnegentropy -lwaku + #cgo LDFLAGS: -L../third_party/nwaku -Wl,-rpath,../third_party/nwaku/build/ + + #include "../third_party/nwaku/library/libwaku.h" + #include + #include + + extern void globalEventCallback(int ret, char* msg, size_t len, void* userData); + + typedef struct { + int ret; + char* msg; + size_t len; + } Resp; + + static void* allocResp() { + return calloc(1, sizeof(Resp)); + } + + static void freeResp(void* resp) { + if (resp != NULL) { + free(resp); + } + } + + static char* getMyCharPtr(void* resp) { + if (resp == NULL) { + return NULL; + } + Resp* m = (Resp*) resp; + return m->msg; + } + + static size_t getMyCharLen(void* resp) { + if (resp == NULL) { + return 0; + } + Resp* m = (Resp*) resp; + return m->len; + } + + static int getRet(void* resp) { + if (resp == NULL) { + return 0; + } + Resp* m = (Resp*) resp; + return m->ret; + } + + // resp must be set != NULL in case interest on retrieving data from the callback + static void callback(int ret, char* msg, size_t len, void* resp) { + if (resp != NULL) { + Resp* m = (Resp*) resp; + m->ret = ret; + m->msg = msg; + m->len = len; + } + } + + #define WAKU_CALL(call) \ + do { \ + int ret = call; \ + if (ret != 0) { \ + printf("Failed the call to: %s. Returned code: %d\n", #call, ret); \ + exit(1); \ + } \ + } while (0) + + static void* cGoWakuNew(const char* configJson, void* resp) { + // We pass NULL because we are not interested in retrieving data from this callback + void* ret = waku_new(configJson, (WakuCallBack) callback, resp); + return ret; + } + + static void cGoWakuStart(void* wakuCtx, void* resp) { + WAKU_CALL(waku_start(wakuCtx, (WakuCallBack) callback, resp)); + } + + static void cGoWakuStop(void* wakuCtx, void* resp) { + WAKU_CALL(waku_stop(wakuCtx, (WakuCallBack) callback, resp)); + } + + static void cGoWakuDestroy(void* wakuCtx, void* resp) { + WAKU_CALL(waku_destroy(wakuCtx, (WakuCallBack) callback, resp)); + } + + static void cGoWakuStartDiscV5(void* wakuCtx, void* resp) { + WAKU_CALL(waku_start_discv5(wakuCtx, (WakuCallBack) callback, resp)); + } + + static void cGoWakuStopDiscV5(void* wakuCtx, void* resp) { + WAKU_CALL(waku_stop_discv5(wakuCtx, (WakuCallBack) callback, resp)); + } + + static void cGoWakuVersion(void* wakuCtx, void* resp) { + WAKU_CALL(waku_version(wakuCtx, (WakuCallBack) callback, resp)); + } + + static void cGoWakuSetEventCallback(void* wakuCtx) { + // The 'globalEventCallback' Go function is shared amongst all possible Waku instances. + + // Given that the 'globalEventCallback' is shared, we pass again the + // wakuCtx instance but in this case is needed to pick up the correct method + // that will handle the event. + + // In other words, for every call the libwaku makes to globalEventCallback, + // the 'userData' parameter will bring the context of the node that registered + // that globalEventCallback. + + // This technique is needed because cgo only allows to export Go functions and not methods. + + waku_set_event_callback(wakuCtx, (WakuCallBack) globalEventCallback, wakuCtx); + } + + static void cGoWakuContentTopic(void* wakuCtx, + char* appName, + int appVersion, + char* contentTopicName, + char* encoding, + void* resp) { + + WAKU_CALL( waku_content_topic(wakuCtx, + appName, + appVersion, + contentTopicName, + encoding, + (WakuCallBack) callback, + resp) ); + } + + static void cGoWakuPubsubTopic(void* wakuCtx, char* topicName, void* resp) { + WAKU_CALL( waku_pubsub_topic(wakuCtx, topicName, (WakuCallBack) callback, resp) ); + } + + static void cGoWakuDefaultPubsubTopic(void* wakuCtx, void* resp) { + WAKU_CALL (waku_default_pubsub_topic(wakuCtx, (WakuCallBack) callback, resp)); + } + + static void cGoWakuRelayPublish(void* wakuCtx, + const char* pubSubTopic, + const char* jsonWakuMessage, + int timeoutMs, + void* resp) { + + WAKU_CALL (waku_relay_publish(wakuCtx, + pubSubTopic, + jsonWakuMessage, + timeoutMs, + (WakuCallBack) callback, + resp)); + } + + static void cGoWakuRelaySubscribe(void* wakuCtx, char* pubSubTopic, void* resp) { + WAKU_CALL ( waku_relay_subscribe(wakuCtx, + pubSubTopic, + (WakuCallBack) callback, + resp) ); + } + + static void cGoWakuRelayUnsubscribe(void* wakuCtx, char* pubSubTopic, void* resp) { + + WAKU_CALL ( waku_relay_unsubscribe(wakuCtx, + pubSubTopic, + (WakuCallBack) callback, + resp) ); + } + + static void cGoWakuConnect(void* wakuCtx, char* peerMultiAddr, int timeoutMs, void* resp) { + WAKU_CALL( waku_connect(wakuCtx, + peerMultiAddr, + timeoutMs, + (WakuCallBack) callback, + resp) ); + } + + static void cGoWakuDialPeerById(void* wakuCtx, + char* peerId, + char* protocol, + int timeoutMs, + void* resp) { + + WAKU_CALL( waku_dial_peer_by_id(wakuCtx, + peerId, + protocol, + timeoutMs, + (WakuCallBack) callback, + resp) ); + } + + static void cGoWakuDisconnectPeerById(void* wakuCtx, char* peerId, void* resp) { + WAKU_CALL( waku_disconnect_peer_by_id(wakuCtx, + peerId, + (WakuCallBack) callback, + resp) ); + } + + static void cGoWakuListenAddresses(void* wakuCtx, void* resp) { + WAKU_CALL (waku_listen_addresses(wakuCtx, (WakuCallBack) callback, resp) ); + } + + static void cGoWakuGetMyENR(void* ctx, void* resp) { + WAKU_CALL (waku_get_my_enr(ctx, (WakuCallBack) callback, resp) ); + } + + static void cGoWakuListPeersInMesh(void* ctx, char* pubSubTopic, void* resp) { + WAKU_CALL (waku_relay_get_num_peers_in_mesh(ctx, pubSubTopic, (WakuCallBack) callback, resp) ); + } + + static void cGoWakuGetNumConnectedPeers(void* ctx, char* pubSubTopic, void* resp) { + WAKU_CALL (waku_relay_get_num_connected_peers(ctx, pubSubTopic, (WakuCallBack) callback, resp) ); + } + + static void cGoWakuLightpushPublish(void* wakuCtx, + const char* pubSubTopic, + const char* jsonWakuMessage, + void* resp) { + + WAKU_CALL (waku_lightpush_publish(wakuCtx, + pubSubTopic, + jsonWakuMessage, + (WakuCallBack) callback, + resp)); + } + + static void cGoWakuStoreQuery(void* wakuCtx, + const char* jsonQuery, + const char* peerAddr, + int timeoutMs, + void* resp) { + + WAKU_CALL (waku_store_query(wakuCtx, + jsonQuery, + peerAddr, + timeoutMs, + (WakuCallBack) callback, + resp)); + } + + static void cGoWakuPeerExchangeQuery(void* wakuCtx, + uint64_t numPeers, + void* resp) { + + WAKU_CALL (waku_peer_exchange_request(wakuCtx, + numPeers, + (WakuCallBack) callback, + resp)); + } + + static void cGoWakuGetPeerIdsByProtocol(void* wakuCtx, + const char* protocol, + void* resp) { + + WAKU_CALL (waku_get_peerids_by_protocol(wakuCtx, + protocol, + (WakuCallBack) callback, + resp)); + } + +*/ +import "C" + import ( + "context" + "crypto/ecdsa" + "crypto/sha256" + "database/sql" + "encoding/hex" "encoding/json" + "errors" "fmt" - "io" - "net/http" - "os" + "runtime" "strconv" + "strings" + "sync" + "time" + "unsafe" + + gethcommon "github.com/ethereum/go-ethereum/common" + "github.com/ethereum/go-ethereum/common/hexutil" + "github.com/ethereum/go-ethereum/crypto" + "github.com/ethereum/go-ethereum/event" + "github.com/ethereum/go-ethereum/p2p" + "github.com/ethereum/go-ethereum/p2p/enode" + "github.com/jellydator/ttlcache/v3" + "github.com/libp2p/go-libp2p/core/metrics" + "github.com/libp2p/go-libp2p/core/peer" + peermod "github.com/libp2p/go-libp2p/core/peer" + "github.com/libp2p/go-libp2p/core/peerstore" + "github.com/multiformats/go-multiaddr" + ma "github.com/multiformats/go-multiaddr" + "github.com/status-im/status-go/connection" + "github.com/status-im/status-go/eth-node/types" + "github.com/status-im/status-go/logutils" + "github.com/status-im/status-go/timesource" + "github.com/status-im/status-go/wakuv2/common" + "github.com/status-im/status-go/wakuv2/persistence" + filterapi "github.com/waku-org/go-waku/waku/v2/api/filter" + "github.com/waku-org/go-waku/waku/v2/api/missing" + "github.com/waku-org/go-waku/waku/v2/api/publish" + "github.com/waku-org/go-waku/waku/v2/dnsdisc" + node "github.com/waku-org/go-waku/waku/v2/node" + "github.com/waku-org/go-waku/waku/v2/onlinechecker" + "github.com/waku-org/go-waku/waku/v2/peermanager" + wps "github.com/waku-org/go-waku/waku/v2/peerstore" + "github.com/waku-org/go-waku/waku/v2/protocol" + "github.com/waku-org/go-waku/waku/v2/protocol/legacy_store" + "github.com/waku-org/go-waku/waku/v2/protocol/pb" + "github.com/waku-org/go-waku/waku/v2/protocol/relay" + "github.com/waku-org/go-waku/waku/v2/protocol/store" + storepb "github.com/waku-org/go-waku/waku/v2/protocol/store/pb" + "github.com/waku-org/go-waku/waku/v2/utils" + "go.uber.org/zap" + "golang.org/x/crypto/pbkdf2" + "golang.org/x/time/rate" ) -type NwakuInfo struct { - ListenAddresses []string `json:"listenAddresses"` - EnrUri string `json:"enrUri"` +const messageQueueLimit = 1024 +const requestTimeout = 30 * time.Second +const bootnodesQueryBackoffMs = 200 +const bootnodesMaxRetries = 7 +const cacheTTL = 20 * time.Minute +const maxRelayPeers = 300 +const randomPeersKeepAliveInterval = 5 * time.Second +const allPeersKeepAliveInterval = 5 * time.Minute +const peersToPublishForLightpush = 2 +const publishingLimiterRate = rate.Limit(2) +const publishingLimitBurst = 4 + +/* TODO-nwaku +type SentEnvelope struct { + Envelope *protocol.Envelope + PublishMethod PublishMethod +} + +type ErrorSendingEnvelope struct { + Error error + SentEnvelope SentEnvelope +} + +type ITelemetryClient interface { + PushReceivedEnvelope(ctx context.Context, receivedEnvelope *protocol.Envelope) + PushSentEnvelope(ctx context.Context, sentEnvelope SentEnvelope) + PushErrorSendingEnvelope(ctx context.Context, errorSendingEnvelope ErrorSendingEnvelope) + PushPeerCount(ctx context.Context, peerCount int) + PushPeerConnFailures(ctx context.Context, peerConnFailures map[string]int) +} + +func (w *Waku) SetStatusTelemetryClient(client ITelemetryClient) { + w.statusTelemetryClient = client +} */ + +func newTTLCache() *ttlcache.Cache[gethcommon.Hash, *common.ReceivedMessage] { + cache := ttlcache.New[gethcommon.Hash, *common.ReceivedMessage](ttlcache.WithTTL[gethcommon.Hash, *common.ReceivedMessage](cacheTTL)) + go cache.Start() + return cache +} + +func (w *Waku) SubscribeToConnStatusChanges() *types.ConnStatusSubscription { + w.connStatusMu.Lock() + defer w.connStatusMu.Unlock() + subscription := types.NewConnStatusSubscription() + w.connStatusSubscriptions[subscription.ID] = subscription + return subscription } -func GetNwakuInfo(host *string, port *int) (NwakuInfo, error) { - nwakuRestPort := 8645 - if port != nil { - nwakuRestPort = *port +/* TODO-nwaku +func (w *Waku) getDiscV5BootstrapNodes(ctx context.Context, addresses []string) ([]*enode.Node, error) { + wg := sync.WaitGroup{} + mu := sync.Mutex{} + var result []*enode.Node + + w.seededBootnodesForDiscV5 = true + + retrieveENR := func(d dnsdisc.DiscoveredNode, wg *sync.WaitGroup) { + mu.Lock() + defer mu.Unlock() + defer wg.Done() + if d.ENR != nil { + result = append(result, d.ENR) + } + } + + for _, addrString := range addresses { + if addrString == "" { + continue + } + + if strings.HasPrefix(addrString, "enrtree://") { + // Use DNS Discovery + wg.Add(1) + go func(addr string) { + defer wg.Done() + if err := w.dnsDiscover(ctx, addr, retrieveENR); err != nil { + mu.Lock() + w.seededBootnodesForDiscV5 = false + mu.Unlock() + } + }(addrString) + } else { + // It's a normal enr + bootnode, err := enode.Parse(enode.ValidSchemes, addrString) + if err != nil { + return nil, err + } + result = append(result, bootnode) + } } - envNwakuRestPort := os.Getenv("NWAKU_REST_PORT") - if envNwakuRestPort != "" { - v, err := strconv.Atoi(envNwakuRestPort) + wg.Wait() + + return result, nil +} + +type fnApplyToEachPeer func(d dnsdisc.DiscoveredNode, wg *sync.WaitGroup) + +func (w *Waku) dnsDiscover(ctx context.Context, enrtreeAddress string, apply fnApplyToEachPeer) error { + w.logger.Info("retrieving nodes", zap.String("enr", enrtreeAddress)) + ctx, cancel := context.WithTimeout(ctx, requestTimeout) + defer cancel() + + w.dnsAddressCacheLock.Lock() + defer w.dnsAddressCacheLock.Unlock() + + discNodes, ok := w.dnsAddressCache[enrtreeAddress] + if !ok { + nameserver := w.cfg.Nameserver + resolver := w.cfg.Resolver + + var opts []dnsdisc.DNSDiscoveryOption + if nameserver != "" { + opts = append(opts, dnsdisc.WithNameserver(nameserver)) + } + if resolver != nil { + opts = append(opts, dnsdisc.WithResolver(resolver)) + } + + discoveredNodes, err := dnsdisc.RetrieveNodes(ctx, enrtreeAddress, opts...) if err != nil { - return NwakuInfo{}, err + w.logger.Warn("dns discovery error ", zap.Error(err)) + return err + } + + if len(discoveredNodes) != 0 { + w.dnsAddressCache[enrtreeAddress] = append(w.dnsAddressCache[enrtreeAddress], discoveredNodes...) + discNodes = w.dnsAddressCache[enrtreeAddress] + } + } + + wg := &sync.WaitGroup{} + wg.Add(len(discNodes)) + for _, d := range discNodes { + apply(d, wg) + } + wg.Wait() + + return nil +} + +func (w *Waku) discoverAndConnectPeers() { + fnApply := func(d dnsdisc.DiscoveredNode, wg *sync.WaitGroup) { + defer wg.Done() + if len(d.PeerInfo.Addrs) != 0 { + go w.connect(d.PeerInfo, d.ENR, wps.DNSDiscovery) + } + } + + for _, addrString := range w.cfg.WakuNodes { + addrString := addrString + if strings.HasPrefix(addrString, "enrtree://") { + // Use DNS Discovery + go func() { + if err := w.dnsDiscover(w.ctx, addrString, fnApply); err != nil { + w.logger.Error("could not obtain dns discovery peers for ClusterConfig.WakuNodes", zap.Error(err), zap.String("dnsDiscURL", addrString)) + } + }() + } else { + // It is a normal multiaddress + addr, err := multiaddr.NewMultiaddr(addrString) + if err != nil { + w.logger.Warn("invalid peer multiaddress", zap.String("ma", addrString), zap.Error(err)) + continue + } + + peerInfo, err := peer.AddrInfoFromP2pAddr(addr) + if err != nil { + w.logger.Warn("invalid peer multiaddress", zap.Stringer("addr", addr), zap.Error(err)) + continue + } + + go w.connect(*peerInfo, nil, wps.Static) + } + } +} */ + +func (w *Waku) connect(peerInfo peer.AddrInfo, enr *enode.Node, origin wps.Origin) { + // Connection will be prunned eventually by the connection manager if needed + // The peer connector in go-waku uses Connect, so it will execute identify as part of its + addr := peerInfo.Addrs[0] + w.WakuConnect(addr.String(), 1000) +} + +/* TODO-nwaku +func (w *Waku) telemetryBandwidthStats(telemetryServerURL string) { + w.wg.Add(1) + defer w.wg.Done() + + if telemetryServerURL == "" { + return + } + + telemetry := NewBandwidthTelemetryClient(w.logger, telemetryServerURL) + + ticker := time.NewTicker(time.Second * 20) + defer ticker.Stop() + + today := time.Now() + + for { + select { + case <-w.ctx.Done(): + return + case now := <-ticker.C: + // Reset totals when day changes + if now.Day() != today.Day() { + today = now + w.bandwidthCounter.Reset() + } + + go telemetry.PushProtocolStats(w.bandwidthCounter.GetBandwidthByProtocol()) + } + } +} + +func (w *Waku) GetStats() types.StatsSummary { + stats := w.bandwidthCounter.GetBandwidthTotals() + return types.StatsSummary{ + UploadRate: uint64(stats.RateOut), + DownloadRate: uint64(stats.RateIn), + } +} + +func (w *Waku) runPeerExchangeLoop() { + w.wg.Add(1) + defer w.wg.Done() + + if !w.cfg.EnablePeerExchangeClient { + // Currently peer exchange client is only used for light nodes + return + } + + ticker := time.NewTicker(time.Second * 5) + defer ticker.Stop() + + for { + select { + case <-w.ctx.Done(): + w.logger.Debug("Peer exchange loop stopped") + return + case <-ticker.C: + w.logger.Info("Running peer exchange loop") + + // We select only the nodes discovered via DNS Discovery that support peer exchange + // We assume that those peers are running peer exchange according to infra config, + // If not, the peer selection process in go-waku will filter them out anyway + w.dnsAddressCacheLock.RLock() + var peers peer.IDSlice + for _, record := range w.dnsAddressCache { + for _, discoveredNode := range record { + if len(discoveredNode.PeerInfo.Addrs) == 0 { + continue + } + // Attempt to connect to the peers. + // Peers will be added to the libp2p peer store thanks to identify + go w.connect(discoveredNode.PeerInfo, discoveredNode.ENR, wps.DNSDiscovery) + peers = append(peers, discoveredNode.PeerID) + } + } + w.dnsAddressCacheLock.RUnlock() + + if len(peers) != 0 { + // TODO + // err := w.node.PeerExchange().Request(w.ctx, w.cfg.DiscoveryLimit, peer_exchange.WithAutomaticPeerSelection(peers...), + // peer_exchange.FilterByShard(int(w.defaultShardInfo.ClusterID), int(w.defaultShardInfo.ShardIDs[0]))) + // if err != nil { + // w.logger.Error("couldnt request peers via peer exchange", zap.Error(err)) + // } + } } - nwakuRestPort = v + } +} + +func (w *Waku) GetPubsubTopic(topic string) string { + if topic == "" { + topic = w.cfg.DefaultShardPubsubTopic + } + + return topic +} + +// CurrentTime returns current time. +func (w *Waku) CurrentTime() time.Time { + return w.timesource.Now() +} + +// APIs returns the RPC descriptors the Waku implementation offers +func (w *Waku) APIs() []rpc.API { + return []rpc.API{ + { + Namespace: Name, + Version: VersionStr, + Service: NewPublicWakuAPI(w), + Public: false, + }, + } +} */ + +// Protocols returns the waku sub-protocols ran by this particular client. +func (w *Waku) Protocols() []p2p.Protocol { + return []p2p.Protocol{} +} + +func (w *Waku) SendEnvelopeEvent(event common.EnvelopeEvent) int { + return w.envelopeFeed.Send(event) +} + +// SubscribeEnvelopeEvents subscribes to envelopes feed. +// In order to prevent blocking waku producers events must be amply buffered. +func (w *Waku) SubscribeEnvelopeEvents(events chan<- common.EnvelopeEvent) event.Subscription { + return w.envelopeFeed.Subscribe(events) +} + +// NewKeyPair generates a new cryptographic identity for the client, and injects +// it into the known identities for message decryption. Returns ID of the new key pair. +func (w *Waku) NewKeyPair() (string, error) { + key, err := crypto.GenerateKey() + if err != nil || !validatePrivateKey(key) { + key, err = crypto.GenerateKey() // retry once + } + if err != nil { + return "", err + } + if !validatePrivateKey(key) { + return "", fmt.Errorf("failed to generate valid key") + } + + id, err := toDeterministicID(hexutil.Encode(crypto.FromECDSAPub(&key.PublicKey)), common.KeyIDSize) + if err != nil { + return "", err + } + + w.keyMu.Lock() + defer w.keyMu.Unlock() + + if w.privateKeys[id] != nil { + return "", fmt.Errorf("failed to generate unique ID") + } + w.privateKeys[id] = key + return id, nil +} + +// DeleteKeyPair deletes the specified key if it exists. +func (w *Waku) DeleteKeyPair(key string) bool { + deterministicID, err := toDeterministicID(key, common.KeyIDSize) + if err != nil { + return false + } + + w.keyMu.Lock() + defer w.keyMu.Unlock() + + if w.privateKeys[deterministicID] != nil { + delete(w.privateKeys, deterministicID) + return true + } + return false +} + +// AddKeyPair imports a asymmetric private key and returns it identifier. +func (w *Waku) AddKeyPair(key *ecdsa.PrivateKey) (string, error) { + id, err := makeDeterministicID(hexutil.Encode(crypto.FromECDSAPub(&key.PublicKey)), common.KeyIDSize) + if err != nil { + return "", err + } + if w.HasKeyPair(id) { + return id, nil // no need to re-inject } - nwakuRestHost := "localhost" - if host != nil { - nwakuRestHost = *host + w.keyMu.Lock() + w.privateKeys[id] = key + w.keyMu.Unlock() + + return id, nil +} + +// SelectKeyPair adds cryptographic identity, and makes sure +// that it is the only private key known to the node. +func (w *Waku) SelectKeyPair(key *ecdsa.PrivateKey) error { + id, err := makeDeterministicID(hexutil.Encode(crypto.FromECDSAPub(&key.PublicKey)), common.KeyIDSize) + if err != nil { + return err + } + + w.keyMu.Lock() + defer w.keyMu.Unlock() + + w.privateKeys = make(map[string]*ecdsa.PrivateKey) // reset key store + w.privateKeys[id] = key + + return nil +} + +// DeleteKeyPairs removes all cryptographic identities known to the node +func (w *Waku) DeleteKeyPairs() error { + w.keyMu.Lock() + defer w.keyMu.Unlock() + + w.privateKeys = make(map[string]*ecdsa.PrivateKey) + + return nil +} + +// HasKeyPair checks if the waku node is configured with the private key +// of the specified public pair. +func (w *Waku) HasKeyPair(id string) bool { + deterministicID, err := toDeterministicID(id, common.KeyIDSize) + if err != nil { + return false + } + + w.keyMu.RLock() + defer w.keyMu.RUnlock() + return w.privateKeys[deterministicID] != nil +} + +// GetPrivateKey retrieves the private key of the specified identity. +func (w *Waku) GetPrivateKey(id string) (*ecdsa.PrivateKey, error) { + deterministicID, err := toDeterministicID(id, common.KeyIDSize) + if err != nil { + return nil, err + } + + w.keyMu.RLock() + defer w.keyMu.RUnlock() + key := w.privateKeys[deterministicID] + if key == nil { + return nil, fmt.Errorf("invalid id") + } + return key, nil +} + +// GenerateSymKey generates a random symmetric key and stores it under id, +// which is then returned. Will be used in the future for session key exchange. +func (w *Waku) GenerateSymKey() (string, error) { + key, err := common.GenerateSecureRandomData(common.AESKeyLength) + if err != nil { + return "", err + } else if !common.ValidateDataIntegrity(key, common.AESKeyLength) { + return "", fmt.Errorf("error in GenerateSymKey: crypto/rand failed to generate random data") + } + + id, err := common.GenerateRandomID() + if err != nil { + return "", fmt.Errorf("failed to generate ID: %s", err) + } + + w.keyMu.Lock() + defer w.keyMu.Unlock() + + if w.symKeys[id] != nil { + return "", fmt.Errorf("failed to generate unique ID") } - envNwakuRestHost := os.Getenv("NWAKU_REST_HOST") - if envNwakuRestHost != "" { - nwakuRestHost = envNwakuRestHost + w.symKeys[id] = key + return id, nil +} + +// AddSymKey stores the key with a given id. +func (w *Waku) AddSymKey(id string, key []byte) (string, error) { + deterministicID, err := toDeterministicID(id, common.KeyIDSize) + if err != nil { + return "", err + } + + w.keyMu.Lock() + defer w.keyMu.Unlock() + + if w.symKeys[deterministicID] != nil { + return "", fmt.Errorf("key already exists: %v", id) + } + w.symKeys[deterministicID] = key + return deterministicID, nil +} + +// AddSymKeyDirect stores the key, and returns its id. +func (w *Waku) AddSymKeyDirect(key []byte) (string, error) { + if len(key) != common.AESKeyLength { + return "", fmt.Errorf("wrong key size: %d", len(key)) } - resp, err := http.Get(fmt.Sprintf("http://%s:%d/debug/v1/info", nwakuRestHost, nwakuRestPort)) + id, err := common.GenerateRandomID() if err != nil { - return NwakuInfo{}, err + return "", fmt.Errorf("failed to generate ID: %s", err) + } + + w.keyMu.Lock() + defer w.keyMu.Unlock() + + if w.symKeys[id] != nil { + return "", fmt.Errorf("failed to generate unique ID") } - defer resp.Body.Close() + w.symKeys[id] = key + return id, nil +} - body, err := io.ReadAll(resp.Body) +// AddSymKeyFromPassword generates the key from password, stores it, and returns its id. +func (w *Waku) AddSymKeyFromPassword(password string) (string, error) { + id, err := common.GenerateRandomID() if err != nil { - return NwakuInfo{}, err + return "", fmt.Errorf("failed to generate ID: %s", err) + } + if w.HasSymKey(id) { + return "", fmt.Errorf("failed to generate unique ID") + } + + // kdf should run no less than 0.1 seconds on an average computer, + // because it's an once in a session experience + derived := pbkdf2.Key([]byte(password), nil, 65356, common.AESKeyLength, sha256.New) + + w.keyMu.Lock() + defer w.keyMu.Unlock() + + // double check is necessary, because deriveKeyMaterial() is very slow + if w.symKeys[id] != nil { + return "", fmt.Errorf("critical error: failed to generate unique ID") + } + w.symKeys[id] = derived + return id, nil +} + +// HasSymKey returns true if there is a key associated with the given id. +// Otherwise returns false. +func (w *Waku) HasSymKey(id string) bool { + w.keyMu.RLock() + defer w.keyMu.RUnlock() + return w.symKeys[id] != nil +} + +// DeleteSymKey deletes the key associated with the name string if it exists. +func (w *Waku) DeleteSymKey(id string) bool { + w.keyMu.Lock() + defer w.keyMu.Unlock() + if w.symKeys[id] != nil { + delete(w.symKeys, id) + return true + } + return false +} + +// GetSymKey returns the symmetric key associated with the given id. +func (w *Waku) GetSymKey(id string) ([]byte, error) { + w.keyMu.RLock() + defer w.keyMu.RUnlock() + if w.symKeys[id] != nil { + return w.symKeys[id], nil } + return nil, fmt.Errorf("non-existent key ID") +} - var data NwakuInfo - err = json.Unmarshal(body, &data) +/* TODO-nwaku +// Subscribe installs a new message handler used for filtering, decrypting +// and subsequent storing of incoming messages. +func (w *Waku) Subscribe(f *common.Filter) (string, error) { + f.PubsubTopic = w.GetPubsubTopic(f.PubsubTopic) + id, err := w.filters.Install(f) if err != nil { - return NwakuInfo{}, err + return id, err + } + + if w.cfg.LightClient { + cf := protocol.NewContentFilter(f.PubsubTopic, f.ContentTopics.ContentTopics()...) + w.filterManager.SubscribeFilter(id, cf) } - return data, nil + return id, nil +} + +// Unsubscribe removes an installed message handler. +func (w *Waku) Unsubscribe(ctx context.Context, id string) error { + ok := w.filters.Uninstall(id) + if !ok { + return fmt.Errorf("failed to unsubscribe: invalid ID '%s'", id) + } + + if w.cfg.LightClient { + w.filterManager.UnsubscribeFilter(id) + } + + return nil +} + +// GetFilter returns the filter by id. +func (w *Waku) GetFilter(id string) *common.Filter { + return w.filters.Get(id) +} + +// Unsubscribe removes an installed message handler. +func (w *Waku) UnsubscribeMany(ids []string) error { + for _, id := range ids { + w.logger.Info("cleaning up filter", zap.String("id", id)) + ok := w.filters.Uninstall(id) + if !ok { + w.logger.Warn("could not remove filter with id", zap.String("id", id)) + } + } + return nil +} + +func (w *Waku) SkipPublishToTopic(value bool) { + w.cfg.SkipPublishToTopic = value +} + +func (w *Waku) ConfirmMessageDelivered(hashes []gethcommon.Hash) { + if !w.cfg.EnableStoreConfirmationForMessagesSent { + return + } + w.messageSentCheck.DeleteByMessageIDs(hashes) +} */ + +func (w *Waku) SetStorePeerID(peerID peer.ID) { + if w.messageSentCheck != nil { + w.messageSentCheck.SetStorePeerID(peerID) + } +} + +func (w *Waku) Query(ctx context.Context, + peerID peer.ID, + query store.FilterCriteria, + cursor []byte, + opts []store.RequestOption, + processEnvelopes bool) ([]byte, int, error) { + requestID := protocol.GenerateRequestID() + + opts = append(opts, + store.WithRequestID(requestID), + store.WithPeer(peerID), + store.WithCursor(cursor)) + + logger := w.logger.With(zap.String("requestID", hexutil.Encode(requestID)), zap.Stringer("peerID", peerID)) + + logger.Debug("store.query", + logutils.WakuMessageTimestamp("startTime", query.TimeStart), + logutils.WakuMessageTimestamp("endTime", query.TimeEnd), + zap.Strings("contentTopics", query.ContentTopics.ToList()), + zap.String("pubsubTopic", query.PubsubTopic), + zap.String("cursor", hexutil.Encode(cursor)), + ) + + // queryStart := time.Now() + + params := new(store.Parameters) + + optList := store.DefaultOptions() + optList = append(optList, opts...) + for _, opt := range optList { + err := opt(params) + if err != nil { + return nil, 0, err + } + } + + storeRequest := &storepb.StoreQueryRequest{ + RequestId: hex.EncodeToString(requestID), + IncludeData: params.IncludeData, + PaginationForward: params.Forward, + PaginationLimit: ¶ms.PageLimit, + } + + jsonStoreRequest, err := json.Marshal(storeRequest) + if err != nil { + return nil, 0, err + } + + result, err := w.wakuStoreQuery(string(jsonStoreRequest), string(peerID), 10000) + + fmt.Println("Store result ", result) + + // result, err := w.node.Store().Query(ctx, query, opts...) + // queryDuration := time.Since(queryStart) + // if err != nil { + // logger.Error("error querying storenode", zap.Error(err)) + + // if w.onHistoricMessagesRequestFailed != nil { + // w.onHistoricMessagesRequestFailed(requestID, peerID, err) + // } + // return nil, 0, err + // } + + // messages := result.Messages() + // envelopesCount := len(messages) + // w.logger.Debug("store.query response", zap.Duration("queryDuration", queryDuration), zap.Int("numMessages", envelopesCount), zap.Bool("hasCursor", result.IsComplete() && result.Cursor() != nil)) + // for _, mkv := range messages { + // msg := mkv.Message + + // // Temporarily setting RateLimitProof to nil so it matches the WakuMessage protobuffer we are sending + // // See https://github.com/vacp2p/rfc/issues/563 + // mkv.Message.RateLimitProof = nil + + // envelope := protocol.NewEnvelope(msg, msg.GetTimestamp(), query.PubsubTopic) + + // err = w.OnNewEnvelopes(envelope, common.StoreMessageType, processEnvelopes) + // if err != nil { + // return nil, 0, err + // } + // } + + // return result.Cursor(), envelopesCount, nil + return nil, 0, nil +} + +/* TODO-nwaku +// OnNewEnvelope is an interface from Waku FilterManager API that gets invoked when any new message is received by Filter. +func (w *Waku) OnNewEnvelope(env *protocol.Envelope) error { + return w.OnNewEnvelopes(env, common.RelayedMessageType, false) +} */ + +// Start implements node.Service, starting the background data propagation thread +// of the Waku protocol. +func (w *Waku) Start() error { + // if w.ctx == nil { + // w.ctx, w.cancel = context.WithCancel(context.Background()) + // } + + // var err error + // if w.node, err = node.New(w.options...); err != nil { + // return fmt.Errorf("failed to create a go-waku node: %v", err) + // } + + // w.goingOnline = make(chan struct{}) + + err := w.WakuStart() + if err != nil { + fmt.Println("Error happened:", err.Error()) + return err + } + + // if err = w.node.Start(w.ctx); err != nil { + // return fmt.Errorf("failed to start go-waku node: %v", err) + // } + + // w.logger.Info("WakuV2 PeerID", zap.Stringer("id", w.node.Host().ID())) + + // w.discoverAndConnectPeers() + + // if w.cfg.EnableDiscV5 { + // err := w.node.DiscV5().Start(w.ctx) + // if err != nil { + // return err + // } + // } + + // w.wg.Add(1) + // go func() { + // defer w.wg.Done() + // ticker := time.NewTicker(5 * time.Second) + // defer ticker.Stop() + // for { + // select { + // case <-w.ctx.Done(): + // return + // case <-ticker.C: + // w.checkForConnectionChanges() + // case <-w.topicHealthStatusChan: + // // TODO: https://github.com/status-im/status-go/issues/4628 + // case <-w.connectionNotifChan: + // w.checkForConnectionChanges() + // } + // } + // }() + + // go w.telemetryBandwidthStats(w.cfg.TelemetryServerURL) + //TODO: commenting for now so that only fleet nodes are used. + //Need to uncomment once filter peer scoring etc is implemented. + // go w.runPeerExchangeLoop() + + // if w.cfg.EnableMissingMessageVerification { + + // w.missingMsgVerifier = missing.NewMissingMessageVerifier( + // w.node.Store(), + // w, + // w.node.Timesource(), + // w.logger) + + // w.missingMsgVerifier.Start(w.ctx) + + // w.wg.Add(1) + // go func() { + // w.wg.Done() + // for { + // select { + // case <-w.ctx.Done(): + // return + // case envelope := <-w.missingMsgVerifier.C: + // err = w.OnNewEnvelopes(envelope, common.MissingMessageType, false) + // if err != nil { + // w.logger.Error("OnNewEnvelopes error", zap.Error(err)) + // } + // } + // } + // }() + // } + + // if w.cfg.LightClient { + // // Create FilterManager that will main peer connectivity + // // for installed filters + // w.filterManager = filterapi.NewFilterManager(w.ctx, w.logger, w.cfg.MinPeersForFilter, + // w, + // w.node.FilterLightnode()) + // } + + // err = w.setupRelaySubscriptions() + // if err != nil { + // return err + // } + + // numCPU := runtime.NumCPU() + // for i := 0; i < numCPU; i++ { + // go w.processQueueLoop() + // } + + // go w.broadcast() + + // go w.sendQueue.Start(w.ctx) + + // if w.cfg.EnableStoreConfirmationForMessagesSent { + // w.confirmMessagesSent() + // } + + // we should wait `seedBootnodesForDiscV5` shutdown smoothly before set w.ctx to nil within `w.Stop()` + // go w.seedBootnodesForDiscV5() + + return nil +} + +func (w *Waku) checkForConnectionChanges() { + + // isOnline := len(w.node.Host().Network().Peers()) > 0 + + // w.connStatusMu.Lock() + + // latestConnStatus := types.ConnStatus{ + // IsOnline: isOnline, + // Peers: FormatPeerStats(w.node), + // } + + // w.logger.Debug("peer stats", + // zap.Int("peersCount", len(latestConnStatus.Peers)), + // zap.Any("stats", latestConnStatus)) + // for k, subs := range w.connStatusSubscriptions { + // if !subs.Send(latestConnStatus) { + // delete(w.connStatusSubscriptions, k) + // } + // } + + // w.connStatusMu.Unlock() + + // if w.onPeerStats != nil { + // w.onPeerStats(latestConnStatus) + // } + + // if w.statusTelemetryClient != nil { + // connFailures := FormatPeerConnFailures(w.node) + // w.statusTelemetryClient.PushPeerCount(w.ctx, w.PeerCount()) + // w.statusTelemetryClient.PushPeerConnFailures(w.ctx, connFailures) + // } + + // w.ConnectionChanged(connection.State{ + // Type: w.state.Type, //setting state type as previous one since there won't be a change here + // Offline: !latestConnStatus.IsOnline, + // }) +} + +// func (w *Waku) confirmMessagesSent() { +// w.messageSentCheck = publish.NewMessageSentCheck(w.ctx, w.node.Store(), w.node.Timesource(), w.logger) +// go w.messageSentCheck.Start() + +// go func() { +// for { +// select { +// case <-w.ctx.Done(): +// return +// case hash := <-w.messageSentCheck.MessageStoredChan: +// w.SendEnvelopeEvent(common.EnvelopeEvent{ +// Hash: hash, +// Event: common.EventEnvelopeSent, +// }) +// case hash := <-w.messageSentCheck.MessageExpiredChan: +// w.SendEnvelopeEvent(common.EnvelopeEvent{ +// Hash: hash, +// Event: common.EventEnvelopeExpired, +// }) +// } +// } +// }() +// } + +func (w *Waku) MessageExists(mh pb.MessageHash) (bool, error) { + w.poolMu.Lock() + defer w.poolMu.Unlock() + return w.envelopeCache.Has(gethcommon.Hash(mh)), nil +} + +/* TODO-nwaku +func (w *Waku) SetTopicsToVerifyForMissingMessages(peerID peer.ID, pubsubTopic string, contentTopics []string) { + if !w.cfg.EnableMissingMessageVerification { + return + } + + w.missingMsgVerifier.SetCriteriaInterest(peerID, protocol.NewContentFilter(pubsubTopic, contentTopics...)) +} + +func (w *Waku) setupRelaySubscriptions() error { + if w.cfg.LightClient { + return nil + } + + if w.protectedTopicStore != nil { + protectedTopics, err := w.protectedTopicStore.ProtectedTopics() + if err != nil { + return err + } + + for _, pt := range protectedTopics { + // Adding subscription to protected topics + // err = w.subscribeToPubsubTopicWithWakuRelay(pt.Topic, pt.PubKey) + // if err != nil { + // return err + // } + + fmt.Println("Subscription to topic: ", pt.Topic) + err = w.WakuRelaySubscribe(pt.Topic) + if err != nil { + return err + } + } + } + + err := w.WakuRelaySubscribe(w.cfg.DefaultShardPubsubTopic) + if err != nil { + return err + } + + return nil +} */ + +/* TODO-nwaku +func (w *Waku) OnNewEnvelopes(envelope *protocol.Envelope, msgType common.MessageType, processImmediately bool) error { + if envelope == nil { + return nil + } + + recvMessage := common.NewReceivedMessage(envelope, msgType) + if recvMessage == nil { + return nil + } + + if w.statusTelemetryClient != nil { + w.statusTelemetryClient.PushReceivedEnvelope(w.ctx, envelope) + } + + logger := w.logger.With( + zap.String("messageType", msgType), + zap.Stringer("envelopeHash", envelope.Hash()), + zap.String("pubsubTopic", envelope.PubsubTopic()), + zap.String("contentTopic", envelope.Message().ContentTopic), + logutils.WakuMessageTimestamp("timestamp", envelope.Message().Timestamp), + ) + + logger.Debug("received new envelope") + trouble := false + + _, err := w.add(recvMessage, processImmediately) + if err != nil { + logger.Info("invalid envelope received", zap.Error(err)) + trouble = true + } + + common.EnvelopesValidatedCounter.Inc() + + if trouble { + return errors.New("received invalid envelope") + } + + return nil +} + +// addEnvelope adds an envelope to the envelope map, used for sending +func (w *Waku) addEnvelope(envelope *common.ReceivedMessage) { + w.poolMu.Lock() + w.envelopeCache.Set(envelope.Hash(), envelope, ttlcache.DefaultTTL) + w.poolMu.Unlock() +} + +func (w *Waku) add(recvMessage *common.ReceivedMessage, processImmediately bool) (bool, error) { + common.EnvelopesReceivedCounter.Inc() + + w.poolMu.Lock() + envelope := w.envelopeCache.Get(recvMessage.Hash()) + alreadyCached := envelope != nil + w.poolMu.Unlock() + + if !alreadyCached { + recvMessage.Processed.Store(false) + w.addEnvelope(recvMessage) + } + + logger := w.logger.With(zap.String("envelopeHash", recvMessage.Hash().Hex())) + + if alreadyCached { + logger.Debug("w envelope already cached") + common.EnvelopesCachedCounter.WithLabelValues("hit").Inc() + } else { + logger.Debug("cached w envelope") + common.EnvelopesCachedCounter.WithLabelValues("miss").Inc() + common.EnvelopesSizeMeter.Observe(float64(len(recvMessage.Envelope.Message().Payload))) + } + + if !alreadyCached || !envelope.Value().Processed.Load() { + if processImmediately { + logger.Debug("immediately processing envelope") + w.processMessage(recvMessage) + } else { + logger.Debug("posting event") + w.postEvent(recvMessage) // notify the local node about the new message + } + } + + return true, nil +} */ + +// postEvent queues the message for further processing. +func (w *Waku) postEvent(envelope *common.ReceivedMessage) { + w.msgQueue <- envelope +} + +/* TODO-nwaku +// processQueueLoop delivers the messages to the watchers during the lifetime of the waku node. +func (w *Waku) processQueueLoop() { + if w.ctx == nil { + return + } + for { + select { + case <-w.ctx.Done(): + return + case e := <-w.msgQueue: + w.processMessage(e) + } + } +} + +func (w *Waku) processMessage(e *common.ReceivedMessage) { + logger := w.logger.With( + zap.Stringer("envelopeHash", e.Envelope.Hash()), + zap.String("pubsubTopic", e.PubsubTopic), + zap.String("contentTopic", e.ContentTopic.ContentTopic()), + zap.Int64("timestamp", e.Envelope.Message().GetTimestamp()), + ) + + if e.MsgType == common.StoreMessageType { + // We need to insert it first, and then remove it if not matched, + // as messages are processed asynchronously + w.storeMsgIDsMu.Lock() + w.storeMsgIDs[e.Hash()] = true + w.storeMsgIDsMu.Unlock() + } + + ephemeral := e.Envelope.Message().Ephemeral + if w.cfg.EnableStoreConfirmationForMessagesSent && e.MsgType == common.SendMessageType && (ephemeral == nil || !*ephemeral) { + w.messageSentCheck.Add(e.PubsubTopic, e.Hash(), e.Sent) + } + + matched := w.filters.NotifyWatchers(e) + + // If not matched we remove it + if !matched { + logger.Debug("filters did not match") + w.storeMsgIDsMu.Lock() + delete(w.storeMsgIDs, e.Hash()) + w.storeMsgIDsMu.Unlock() + } else { + logger.Debug("filters did match") + e.Processed.Store(true) + } + + w.envelopeFeed.Send(common.EnvelopeEvent{ + Topic: e.ContentTopic, + Hash: e.Hash(), + Event: common.EventEnvelopeAvailable, + }) +} */ + +// GetEnvelope retrieves an envelope from the message queue by its hash. +// It returns nil if the envelope can not be found. +func (w *Waku) GetEnvelope(hash gethcommon.Hash) *common.ReceivedMessage { + w.poolMu.RLock() + defer w.poolMu.RUnlock() + + envelope := w.envelopeCache.Get(hash) + if envelope == nil { + return nil + } + + return envelope.Value() +} + +// isEnvelopeCached checks if envelope with specific hash has already been received and cached. +func (w *Waku) IsEnvelopeCached(hash gethcommon.Hash) bool { + w.poolMu.Lock() + defer w.poolMu.Unlock() + + return w.envelopeCache.Has(hash) +} + +func (w *Waku) ClearEnvelopesCache() { + w.poolMu.Lock() + defer w.poolMu.Unlock() + + w.envelopeCache.Stop() + w.envelopeCache = newTTLCache() +} + +func (w *Waku) PeerCount() int { + return 0 + // return w.node.PeerCount() +} + +func (w *Waku) Peers() types.PeerStats { + return nil + // return FormatPeerStats(w.node) +} + +/* TODO-nwaku +func (w *Waku) RelayPeersByTopic(topic string) (*types.PeerList, error) { + if w.cfg.LightClient { + return nil, errors.New("only available in relay mode") + } + + // return &types.PeerList{ + // FullMeshPeers: w.node.Relay().PubSub().MeshPeers(topic), + // AllPeers: w.node.Relay().PubSub().ListPeers(topic), + // }, nil + return nil, nil +} + +func (w *Waku) SubscribeToPubsubTopic(topic string, pubkey *ecdsa.PublicKey) error { + topic = w.GetPubsubTopic(topic) + + if !w.cfg.LightClient { + err := w.WakuRelaySubscribe(topic) + // err := w.subscribeToPubsubTopicWithWakuRelay(topic, pubkey) + if err != nil { + return err + } + } + return nil +} + +func (w *Waku) UnsubscribeFromPubsubTopic(topic string) error { + topic = w.GetPubsubTopic(topic) + + if !w.cfg.LightClient { + err := w.WakuRelayUnsubscribe(topic) + if err != nil { + return err + } + } + return nil +} + +func (w *Waku) RetrievePubsubTopicKey(topic string) (*ecdsa.PrivateKey, error) { + topic = w.GetPubsubTopic(topic) + if w.protectedTopicStore == nil { + return nil, nil + } + + return w.protectedTopicStore.FetchPrivateKey(topic) +} + +func (w *Waku) StorePubsubTopicKey(topic string, privKey *ecdsa.PrivateKey) error { + topic = w.GetPubsubTopic(topic) + if w.protectedTopicStore == nil { + return nil + } + + return w.protectedTopicStore.Insert(topic, privKey, &privKey.PublicKey) +} + +func (w *Waku) RemovePubsubTopicKey(topic string) error { + topic = w.GetPubsubTopic(topic) + if w.protectedTopicStore == nil { + return nil + } + + return w.protectedTopicStore.Delete(topic) +} */ + +func (w *Waku) handleNetworkChangeFromApp(state connection.State) { + //If connection state is reported by something other than peerCount becoming 0 e.g from mobile app, disconnect all peers + // if (state.Offline && len(w.node.Host().Network().Peers()) > 0) || + // (w.state.Type != state.Type && !w.state.Offline && !state.Offline) { // network switched between wifi and cellular + // w.logger.Info("connection switched or offline detected via mobile, disconnecting all peers") + // w.node.DisconnectAllPeers() + // if w.cfg.LightClient { + // w.filterManager.NetworkChange() + // } + // } +} + +/* TODO-nwaku +func (w *Waku) ConnectionChanged(state connection.State) { + isOnline := !state.Offline + if w.cfg.LightClient { + //TODO: Update this as per https://github.com/waku-org/go-waku/issues/1114 + go w.filterManager.OnConnectionStatusChange("", isOnline) + w.handleNetworkChangeFromApp(state) + } else { + // for lightClient state update and onlineChange is handled in filterManager. + // going online + if isOnline && !w.onlineChecker.IsOnline() { + //TODO: analyze if we need to discover and connect to peers for relay. + w.discoverAndConnectPeers() + select { + case w.goingOnline <- struct{}{}: + default: + w.logger.Warn("could not write on connection changed channel") + } + } + // update state + w.onlineChecker.SetOnline(isOnline) + } + w.state = state +} */ + +func (w *Waku) AddStorePeer(address multiaddr.Multiaddr) (peer.ID, error) { + // peerID, err := w.node.AddPeer(address, wps.Static, w.cfg.DefaultShardedPubsubTopics, store.StoreQueryID_v300) + // if err != nil { + // return "", err + // } + // return peerID, nil + return "", nil +} + +func (w *Waku) timestamp() int64 { + return w.timesource.Now().UnixNano() +} + +func (w *Waku) AddRelayPeer(address multiaddr.Multiaddr) (peer.ID, error) { + // peerID, err := w.node.AddPeer(address, wps.Static, w.cfg.DefaultShardedPubsubTopics, relay.WakuRelayID_v200) + // if err != nil { + // return "", err + // } + // return peerID, nil + return "", nil +} + +func (w *Waku) DialPeer(address multiaddr.Multiaddr) error { + // ctx, cancel := context.WithTimeout(w.ctx, requestTimeout) + // defer cancel() + // return w.node.DialPeerWithMultiAddress(ctx, address) + return nil +} + +func (w *Waku) DialPeerByID(peerId peer.ID) error { + return w.WakuDialPeerById(peerId, string(relay.WakuRelayID_v200), 1000) +} + +func (w *Waku) DropPeer(peerID peer.ID) error { + // return w.node.ClosePeerById(peerID) + return nil +} + +func (w *Waku) ProcessingP2PMessages() bool { + w.storeMsgIDsMu.Lock() + defer w.storeMsgIDsMu.Unlock() + return len(w.storeMsgIDs) != 0 +} + +func (w *Waku) MarkP2PMessageAsProcessed(hash gethcommon.Hash) { + w.storeMsgIDsMu.Lock() + defer w.storeMsgIDsMu.Unlock() + delete(w.storeMsgIDs, hash) +} + +func (w *Waku) Clean() error { + w.msgQueue = make(chan *common.ReceivedMessage, messageQueueLimit) + + for _, f := range w.filters.All() { + f.Messages = common.NewMemoryMessageStore() + } + + return nil +} + +func (w *Waku) PeerID() peer.ID { + // return w.node.Host().ID() + return "" +} + +func (w *Waku) Peerstore() peerstore.Peerstore { + // return w.node.Host().Peerstore() + return nil +} + +// validatePrivateKey checks the format of the given private key. +func validatePrivateKey(k *ecdsa.PrivateKey) bool { + if k == nil || k.D == nil || k.D.Sign() == 0 { + return false + } + return common.ValidatePublicKey(&k.PublicKey) +} + +// makeDeterministicID generates a deterministic ID, based on a given input +func makeDeterministicID(input string, keyLen int) (id string, err error) { + buf := pbkdf2.Key([]byte(input), nil, 4096, keyLen, sha256.New) + if !common.ValidateDataIntegrity(buf, common.KeyIDSize) { + return "", fmt.Errorf("error in GenerateDeterministicID: failed to generate key") + } + id = gethcommon.Bytes2Hex(buf) + return id, err +} + +// toDeterministicID reviews incoming id, and transforms it to format +// expected internally be private key store. Originally, public keys +// were used as keys, now random keys are being used. And in order to +// make it easier to consume, we now allow both random IDs and public +// keys to be passed. +func toDeterministicID(id string, expectedLen int) (string, error) { + if len(id) != (expectedLen * 2) { // we received hex key, so number of chars in id is doubled + var err error + id, err = makeDeterministicID(id, expectedLen) + if err != nil { + return "", err + } + } + + return id, nil +} + +func FormatPeerStats(wakuNode *node.WakuNode) types.PeerStats { + p := make(types.PeerStats) + for k, v := range wakuNode.PeerStats() { + p[k] = types.WakuV2Peer{ + Addresses: utils.EncapsulatePeerID(k, wakuNode.Host().Peerstore().PeerInfo(k).Addrs...), + Protocols: v, + } + } + return p +} + +func (w *Waku) StoreNode() *store.WakuStore { + // return w.node.Store() + return nil +} + +func FormatPeerConnFailures(wakuNode *node.WakuNode) map[string]int { + p := make(map[string]int) + for _, peerID := range wakuNode.Host().Network().Peers() { + peerInfo := wakuNode.Host().Peerstore().PeerInfo(peerID) + connFailures := wakuNode.Host().Peerstore().(wps.WakuPeerstore).ConnFailures(peerInfo) + if connFailures > 0 { + p[peerID.String()] = connFailures + } + } + return p +} + +func (w *Waku) LegacyStoreNode() legacy_store.Store { + // return w.node.LegacyStore() + return nil +} + +type WakuMessageHash = string +type WakuPubsubTopic = string +type WakuContentTopic = string + +type WakuConfig struct { + Host string `json:"host,omitempty"` + Port int `json:"port,omitempty"` + NodeKey string `json:"key,omitempty"` + EnableRelay bool `json:"relay"` + LogLevel string `json:"logLevel"` + DnsDiscovery bool `json:"dnsDiscovery,omitempty"` + DnsDiscoveryUrl string `json:"dnsDiscoveryUrl,omitempty"` + MaxMessageSize string `json:"maxMessageSize,omitempty"` + Staticnodes []string `json:"staticnodes,omitempty"` + Discv5BootstrapNodes []string `json:"discv5BootstrapNodes,omitempty"` + Discv5Discovery bool `json:"discv5Discovery,omitempty"` + ClusterID uint16 `json:"clusterId,omitempty"` + Shards []uint16 `json:"shards,omitempty"` +} + +type Waku struct { + wakuCtx unsafe.Pointer + + appDB *sql.DB + + dnsAddressCache map[string][]dnsdisc.DiscoveredNode // Map to store the multiaddresses returned by dns discovery + dnsAddressCacheLock *sync.RWMutex // lock to handle access to the map + + // Filter-related + filters *common.Filters // Message filters installed with Subscribe function + filterManager *filterapi.FilterManager + + privateKeys map[string]*ecdsa.PrivateKey // Private key storage + symKeys map[string][]byte // Symmetric key storage + keyMu sync.RWMutex // Mutex associated with key stores + + envelopeCache *ttlcache.Cache[gethcommon.Hash, *common.ReceivedMessage] // Pool of envelopes currently tracked by this node + poolMu sync.RWMutex // Mutex to sync the message and expiration pools + + bandwidthCounter *metrics.BandwidthCounter + + protectedTopicStore *persistence.ProtectedTopicsStore + + sendQueue *publish.MessageQueue + limiter *publish.PublishRateLimiter + + missingMsgVerifier *missing.MissingMessageVerifier + + msgQueue chan *common.ReceivedMessage // Message queue for waku messages that havent been decoded + + ctx context.Context + cancel context.CancelFunc + wg sync.WaitGroup + + cfg *WakuConfig + options []node.WakuNodeOption + + envelopeFeed event.Feed + + storeMsgIDs map[gethcommon.Hash]bool // Map of the currently processing ids + storeMsgIDsMu sync.RWMutex + + messageSentCheck *publish.MessageSentCheck + + topicHealthStatusChan chan peermanager.TopicHealthStatus + connectionNotifChan chan node.PeerConnection + connStatusSubscriptions map[string]*types.ConnStatusSubscription + connStatusMu sync.Mutex + onlineChecker *onlinechecker.DefaultOnlineChecker + state connection.State + + logger *zap.Logger + + // NTP Synced timesource + timesource *timesource.NTPTimeSource + + // seededBootnodesForDiscV5 indicates whether we manage to retrieve discovery + // bootnodes successfully + seededBootnodesForDiscV5 bool + + // goingOnline is channel that notifies when connectivity has changed from offline to online + goingOnline chan struct{} + + // discV5BootstrapNodes is the ENR to be used to fetch bootstrap nodes for discovery + discV5BootstrapNodes []string + + onHistoricMessagesRequestFailed func([]byte, peer.ID, error) + onPeerStats func(types.ConnStatus) + + // statusTelemetryClient ITelemetryClient // TODO-nwaku + + defaultShardInfo protocol.RelayShards +} + +func (w *Waku) Stop() error { + return w.WakuStop() +} + +func WakuSetup() { + C.waku_setup() +} + +func printStackTrace() { + // Create a buffer to hold the stack trace + buf := make([]byte, 102400) + // Capture the stack trace into the buffer + n := runtime.Stack(buf, false) + // Print the stack trace + fmt.Printf("Current stack trace:\n%s\n", buf[:n]) +} + +func wakuNew(nodeKey *ecdsa.PrivateKey, + fleet string, + cfg *WakuConfig, + logger *zap.Logger, + appDB *sql.DB, + ts *timesource.NTPTimeSource, + onHistoricMessagesRequestFailed func([]byte, peer.ID, error), onPeerStats func(types.ConnStatus)) (*Waku, error) { + + var err error + if logger == nil { + logger, err = zap.NewDevelopment() + if err != nil { + return nil, err + } + } + if ts == nil { + ts = timesource.Default() + } + + /* TODO-nwaku + cfg = setDefaults(cfg) + if err = cfg.Validate(logger); err != nil { + return nil, err + } */ + + ctx, cancel := context.WithCancel(context.Background()) + + jsonConfig, err := json.Marshal(cfg) + if err != nil { + return nil, err + } + + var cJsonConfig = C.CString(string(jsonConfig)) + var resp = C.allocResp() + + defer C.free(unsafe.Pointer(cJsonConfig)) + defer C.freeResp(resp) + + wakuCtx := C.cGoWakuNew(cJsonConfig, resp) + // Notice that the events for self node are handled by the 'MyEventCallback' method + + if C.getRet(resp) == C.RET_OK { + + return &Waku{ + wakuCtx: wakuCtx, + cfg: cfg, + privateKeys: make(map[string]*ecdsa.PrivateKey), + symKeys: make(map[string][]byte), + envelopeCache: newTTLCache(), + msgQueue: make(chan *common.ReceivedMessage, messageQueueLimit), + topicHealthStatusChan: make(chan peermanager.TopicHealthStatus, 100), + connectionNotifChan: make(chan node.PeerConnection, 20), + connStatusSubscriptions: make(map[string]*types.ConnStatusSubscription), + ctx: ctx, + cancel: cancel, + wg: sync.WaitGroup{}, + dnsAddressCache: make(map[string][]dnsdisc.DiscoveredNode), + dnsAddressCacheLock: &sync.RWMutex{}, + storeMsgIDs: make(map[gethcommon.Hash]bool), + timesource: ts, + storeMsgIDsMu: sync.RWMutex{}, + logger: logger, + discV5BootstrapNodes: cfg.Discv5BootstrapNodes, + onHistoricMessagesRequestFailed: onHistoricMessagesRequestFailed, + onPeerStats: onPeerStats, + onlineChecker: onlinechecker.NewDefaultOnlineChecker(false).(*onlinechecker.DefaultOnlineChecker), + //sendQueue: publish.NewMessageQueue(1000, cfg.UseThrottledPublish), // TODO-nwaku + }, nil + } + + errMsg := "error wakuNew: " + C.GoStringN(C.getMyCharPtr(resp), C.int(C.getMyCharLen(resp))) + return nil, errors.New(errMsg) +} + +func (self *Waku) WakuStart() error { + + var resp = C.allocResp() + defer C.freeResp(resp) + C.cGoWakuStart(self.wakuCtx, resp) + + if C.getRet(resp) == C.RET_OK { + return nil + } + errMsg := "error WakuStart: " + C.GoStringN(C.getMyCharPtr(resp), C.int(C.getMyCharLen(resp))) + return errors.New(errMsg) +} + +func (self *Waku) WakuStop() error { + var resp = C.allocResp() + defer C.freeResp(resp) + C.cGoWakuStop(self.wakuCtx, resp) + + if C.getRet(resp) == C.RET_OK { + return nil + } + errMsg := "error WakuStop: " + C.GoStringN(C.getMyCharPtr(resp), C.int(C.getMyCharLen(resp))) + return errors.New(errMsg) +} + +func (self *Waku) WakuDestroy() error { + var resp = C.allocResp() + defer C.freeResp(resp) + C.cGoWakuDestroy(self.wakuCtx, resp) + + if C.getRet(resp) == C.RET_OK { + return nil + } + errMsg := "error WakuDestroy: " + C.GoStringN(C.getMyCharPtr(resp), C.int(C.getMyCharLen(resp))) + return errors.New(errMsg) +} + +func (self *Waku) StartDiscV5() error { + var resp = C.allocResp() + defer C.freeResp(resp) + C.cGoWakuStartDiscV5(self.wakuCtx, resp) + + if C.getRet(resp) == C.RET_OK { + return nil + } + errMsg := "error WakuStartDiscV5: " + C.GoStringN(C.getMyCharPtr(resp), C.int(C.getMyCharLen(resp))) + return errors.New(errMsg) +} + +func (self *Waku) StopDiscV5() error { + var resp = C.allocResp() + defer C.freeResp(resp) + C.cGoWakuStopDiscV5(self.wakuCtx, resp) + + if C.getRet(resp) == C.RET_OK { + return nil + } + errMsg := "error WakuStopDiscV5: " + C.GoStringN(C.getMyCharPtr(resp), C.int(C.getMyCharLen(resp))) + return errors.New(errMsg) +} + +func (self *Waku) WakuVersion() (string, error) { + var resp = C.allocResp() + defer C.freeResp(resp) + + C.cGoWakuVersion(self.wakuCtx, resp) + + if C.getRet(resp) == C.RET_OK { + var version = C.GoStringN(C.getMyCharPtr(resp), C.int(C.getMyCharLen(resp))) + return version, nil + } + + errMsg := "error WakuVersion: " + + C.GoStringN(C.getMyCharPtr(resp), C.int(C.getMyCharLen(resp))) + return "", errors.New(errMsg) +} + +//export globalEventCallback +func globalEventCallback(callerRet C.int, msg *C.char, len C.size_t, userData unsafe.Pointer) { + // This is shared among all Golang instances + self := Waku{wakuCtx: userData} + self.MyEventCallback(callerRet, msg, len) +} + +func (self *Waku) MyEventCallback(callerRet C.int, msg *C.char, len C.size_t) { + fmt.Println("Event received:", C.GoStringN(msg, C.int(len))) +} + +func (self *Waku) WakuSetEventCallback() { + // Notice that the events for self node are handled by the 'MyEventCallback' method + C.cGoWakuSetEventCallback(self.wakuCtx) +} + +func (self *Waku) FormatContentTopic( + appName string, + appVersion int, + contentTopicName string, + encoding string) (WakuContentTopic, error) { + + var cAppName = C.CString(appName) + var cContentTopicName = C.CString(contentTopicName) + var cEncoding = C.CString(encoding) + var resp = C.allocResp() + + defer C.free(unsafe.Pointer(cAppName)) + defer C.free(unsafe.Pointer(cContentTopicName)) + defer C.free(unsafe.Pointer(cEncoding)) + defer C.freeResp(resp) + + C.cGoWakuContentTopic(self.wakuCtx, + cAppName, + C.int(appVersion), + cContentTopicName, + cEncoding, + resp) + + if C.getRet(resp) == C.RET_OK { + var contentTopic = C.GoStringN(C.getMyCharPtr(resp), C.int(C.getMyCharLen(resp))) + return contentTopic, nil + } + + errMsg := "error FormatContentTopic: " + + C.GoStringN(C.getMyCharPtr(resp), C.int(C.getMyCharLen(resp))) + + return "", errors.New(errMsg) +} + +func (self *Waku) FormatPubsubTopic(topicName string) (WakuPubsubTopic, error) { + var cTopicName = C.CString(topicName) + var resp = C.allocResp() + + defer C.free(unsafe.Pointer(cTopicName)) + defer C.freeResp(resp) + + C.cGoWakuPubsubTopic(self.wakuCtx, cTopicName, resp) + if C.getRet(resp) == C.RET_OK { + var pubsubTopic = C.GoStringN(C.getMyCharPtr(resp), C.int(C.getMyCharLen(resp))) + return pubsubTopic, nil + } + + errMsg := "error FormatPubsubTopic: " + + C.GoStringN(C.getMyCharPtr(resp), C.int(C.getMyCharLen(resp))) + + return "", errors.New(errMsg) +} + +func (self *Waku) WakuDefaultPubsubTopic() (WakuPubsubTopic, error) { + var resp = C.allocResp() + defer C.freeResp(resp) + C.cGoWakuDefaultPubsubTopic(self.wakuCtx, resp) + if C.getRet(resp) == C.RET_OK { + var defaultPubsubTopic = C.GoStringN(C.getMyCharPtr(resp), C.int(C.getMyCharLen(resp))) + return defaultPubsubTopic, nil + } + + errMsg := "error WakuDefaultPubsubTopic: " + + C.GoStringN(C.getMyCharPtr(resp), C.int(C.getMyCharLen(resp))) + + return "", errors.New(errMsg) +} + +func (self *Waku) WakuRelayPublish(wakuMsg *pb.WakuMessage, pubsubTopic string) (string, error) { + timeoutMs := 1000 + + message, err := json.Marshal(wakuMsg) + if err != nil { + return "", err + } + + var cPubsubTopic = C.CString(pubsubTopic) + var msg = C.CString(string(message)) + var resp = C.allocResp() + + defer C.freeResp(resp) + defer C.free(unsafe.Pointer(cPubsubTopic)) + defer C.free(unsafe.Pointer(msg)) + + C.cGoWakuRelayPublish(self.wakuCtx, cPubsubTopic, msg, C.int(timeoutMs), resp) + if C.getRet(resp) == C.RET_OK { + msgHash := C.GoStringN(C.getMyCharPtr(resp), C.int(C.getMyCharLen(resp))) + return msgHash, nil + } + errMsg := "error WakuRelayPublish: " + + C.GoStringN(C.getMyCharPtr(resp), C.int(C.getMyCharLen(resp))) + return "", errors.New(errMsg) +} + +func (self *Waku) WakuRelaySubscribe(pubsubTopic string) error { + var resp = C.allocResp() + var cPubsubTopic = C.CString(pubsubTopic) + + defer C.freeResp(resp) + defer C.free(unsafe.Pointer(cPubsubTopic)) + + if self.wakuCtx == nil { + fmt.Println("ctx is nil") + } + // if self.cPubsubTopic == nil { + // fmt.Println("cPubsubTopic is nil") + // } + // if self.resp == nil { + // fmt.Println("resp is nil") + // } + + C.cGoWakuRelaySubscribe(self.wakuCtx, cPubsubTopic, resp) + + if C.getRet(resp) == C.RET_OK { + return nil + } + errMsg := "error WakuRelaySubscribe: " + + C.GoStringN(C.getMyCharPtr(resp), C.int(C.getMyCharLen(resp))) + return errors.New(errMsg) +} + +func (self *Waku) WakuRelayUnsubscribe(pubsubTopic string) error { + var resp = C.allocResp() + var cPubsubTopic = C.CString(pubsubTopic) + defer C.freeResp(resp) + defer C.free(unsafe.Pointer(cPubsubTopic)) + C.cGoWakuRelayUnsubscribe(self.wakuCtx, cPubsubTopic, resp) + + if C.getRet(resp) == C.RET_OK { + return nil + } + errMsg := "error WakuRelayUnsubscribe: " + + C.GoStringN(C.getMyCharPtr(resp), C.int(C.getMyCharLen(resp))) + return errors.New(errMsg) +} + +func (self *Waku) WakuLightpushPublish(message *pb.WakuMessage, pubsubTopic string) (string, error) { + jsonMsg, err := json.Marshal(message) + if err != nil { + return "", err + } + + var cPubsubTopic = C.CString(pubsubTopic) + var msg = C.CString(string(jsonMsg)) + var resp = C.allocResp() + + defer C.freeResp(resp) + defer C.free(unsafe.Pointer(cPubsubTopic)) + defer C.free(unsafe.Pointer(msg)) + + C.cGoWakuLightpushPublish(self.wakuCtx, cPubsubTopic, msg, resp) + if C.getRet(resp) == C.RET_OK { + msg := C.GoStringN(C.getMyCharPtr(resp), C.int(C.getMyCharLen(resp))) + return msg, nil + } + errMsg := "error WakuLightpushPublish: " + + C.GoStringN(C.getMyCharPtr(resp), C.int(C.getMyCharLen(resp))) + return "", errors.New(errMsg) +} + +func (self *Waku) wakuStoreQuery( + jsonQuery string, + peerAddr string, + timeoutMs int) (string, error) { + + var cJsonQuery = C.CString(jsonQuery) + var cPeerAddr = C.CString(peerAddr) + var resp = C.allocResp() + + defer C.free(unsafe.Pointer(cJsonQuery)) + defer C.free(unsafe.Pointer(cPeerAddr)) + defer C.freeResp(resp) + + C.cGoWakuStoreQuery(self.wakuCtx, cJsonQuery, cPeerAddr, C.int(timeoutMs), resp) + if C.getRet(resp) == C.RET_OK { + msg := C.GoStringN(C.getMyCharPtr(resp), C.int(C.getMyCharLen(resp))) + return msg, nil + } + errMsg := "error WakuStoreQuery: " + + C.GoStringN(C.getMyCharPtr(resp), C.int(C.getMyCharLen(resp))) + return "", errors.New(errMsg) +} + +func (self *Waku) WakuPeerExchangeRequest(numPeers uint64) (string, error) { + var resp = C.allocResp() + defer C.freeResp(resp) + + C.cGoWakuPeerExchangeQuery(self.wakuCtx, C.uint64_t(numPeers), resp) + if C.getRet(resp) == C.RET_OK { + msg := C.GoStringN(C.getMyCharPtr(resp), C.int(C.getMyCharLen(resp))) + return msg, nil + } + errMsg := "error WakuPeerExchangeRequest: " + + C.GoStringN(C.getMyCharPtr(resp), C.int(C.getMyCharLen(resp))) + return "", errors.New(errMsg) +} + +func (self *Waku) WakuConnect(peerMultiAddr string, timeoutMs int) error { + var resp = C.allocResp() + var cPeerMultiAddr = C.CString(peerMultiAddr) + defer C.freeResp(resp) + defer C.free(unsafe.Pointer(cPeerMultiAddr)) + + C.cGoWakuConnect(self.wakuCtx, cPeerMultiAddr, C.int(timeoutMs), resp) + + if C.getRet(resp) == C.RET_OK { + return nil + } + errMsg := "error WakuConnect: " + + C.GoStringN(C.getMyCharPtr(resp), C.int(C.getMyCharLen(resp))) + return errors.New(errMsg) +} + +func (self *Waku) WakuDialPeerById(peerId peer.ID, protocol string, timeoutMs int) error { + var resp = C.allocResp() + var cPeerId = C.CString(peerId.String()) + var cProtocol = C.CString(protocol) + defer C.freeResp(resp) + defer C.free(unsafe.Pointer(cPeerId)) + defer C.free(unsafe.Pointer(cProtocol)) + + C.cGoWakuDialPeerById(self.wakuCtx, cPeerId, cProtocol, C.int(timeoutMs), resp) + + if C.getRet(resp) == C.RET_OK { + return nil + } + errMsg := "error DialPeerById: " + + C.GoStringN(C.getMyCharPtr(resp), C.int(C.getMyCharLen(resp))) + return errors.New(errMsg) +} + +func (self *Waku) ListenAddresses() ([]multiaddr.Multiaddr, error) { + var resp = C.allocResp() + defer C.freeResp(resp) + C.cGoWakuListenAddresses(self.wakuCtx, resp) + + if C.getRet(resp) == C.RET_OK { + + var addrsRet []multiaddr.Multiaddr + listenAddresses := C.GoStringN(C.getMyCharPtr(resp), C.int(C.getMyCharLen(resp))) + addrss := strings.Split(listenAddresses, ",") + for _, addr := range addrss { + addr, err := ma.NewMultiaddr(addr) + if err != nil { + return nil, err + } + + addrsRet = append(addrsRet, addr) + } + + fmt.Println("AAAAAA listen addresses: ", listenAddresses) + + return addrsRet, nil + } + errMsg := "error WakuListenAddresses: " + + C.GoStringN(C.getMyCharPtr(resp), C.int(C.getMyCharLen(resp))) + + return nil, errors.New(errMsg) +} + +func (self *Waku) ENR() (*enode.Node, error) { + var resp = C.allocResp() + defer C.freeResp(resp) + C.cGoWakuGetMyENR(self.wakuCtx, resp) + + if C.getRet(resp) == C.RET_OK { + enrStr := C.GoStringN(C.getMyCharPtr(resp), C.int(C.getMyCharLen(resp))) + n, err := enode.Parse(enode.ValidSchemes, enrStr) + if err != nil { + return nil, err + } + return n, nil + } + errMsg := "error WakuGetMyENR: " + + C.GoStringN(C.getMyCharPtr(resp), C.int(C.getMyCharLen(resp))) + return nil, errors.New(errMsg) +} + +func (self *Waku) ListPeersInMesh(pubsubTopic string) (int, error) { + var resp = C.allocResp() + var cPubsubTopic = C.CString(pubsubTopic) + defer C.freeResp(resp) + defer C.free(unsafe.Pointer(cPubsubTopic)) + + C.cGoWakuListPeersInMesh(self.wakuCtx, cPubsubTopic, resp) + + if C.getRet(resp) == C.RET_OK { + numPeersStr := C.GoStringN(C.getMyCharPtr(resp), C.int(C.getMyCharLen(resp))) + numPeers, err := strconv.Atoi(numPeersStr) + if err != nil { + fmt.Println(":", err) + errMsg := "ListPeersInMesh - error converting string to int: " + err.Error() + return 0, errors.New(errMsg) + } + return numPeers, nil + } + errMsg := "error ListPeersInMesh: " + + C.GoStringN(C.getMyCharPtr(resp), C.int(C.getMyCharLen(resp))) + return 0, errors.New(errMsg) +} + +func (self *Waku) GetNumConnectedPeers(paramPubsubTopic ...string) (int, error) { + var pubsubTopic string + if len(paramPubsubTopic) == 0 { + pubsubTopic = "" + } else { + pubsubTopic = paramPubsubTopic[0] + } + + var resp = C.allocResp() + var cPubsubTopic = C.CString(pubsubTopic) + defer C.freeResp(resp) + defer C.free(unsafe.Pointer(cPubsubTopic)) + + C.cGoWakuGetNumConnectedPeers(self.wakuCtx, cPubsubTopic, resp) + + if C.getRet(resp) == C.RET_OK { + numPeersStr := C.GoStringN(C.getMyCharPtr(resp), C.int(C.getMyCharLen(resp))) + numPeers, err := strconv.Atoi(numPeersStr) + if err != nil { + fmt.Println(":", err) + errMsg := "GetNumConnectedPeers - error converting string to int: " + err.Error() + return 0, errors.New(errMsg) + } + return numPeers, nil + } + errMsg := "error GetNumConnectedPeers: " + + C.GoStringN(C.getMyCharPtr(resp), C.int(C.getMyCharLen(resp))) + return 0, errors.New(errMsg) +} + +func (self *Waku) GetPeerIdsByProtocol(protocol string) (peer.IDSlice, error) { + var resp = C.allocResp() + var cProtocol = C.CString(protocol) + defer C.freeResp(resp) + defer C.free(unsafe.Pointer(cProtocol)) + + C.cGoWakuGetPeerIdsByProtocol(self.wakuCtx, cProtocol, resp) + + if C.getRet(resp) == C.RET_OK { + peersStr := C.GoStringN(C.getMyCharPtr(resp), C.int(C.getMyCharLen(resp))) + if peersStr == "" { + return peer.IDSlice{}, nil + } + // peersStr contains a comma-separated list of peer ids + itemsPeerIds := strings.Split(peersStr, ",") + + var peers peer.IDSlice + for _, peer := range itemsPeerIds { + id, err := peermod.Decode(peer) + if err != nil { + errMsg := "GetPeerIdsByProtocol - error converting string to int: " + err.Error() + return nil, errors.New(errMsg) + } + peers = append(peers, id) + } + + return peers, nil + } + errMsg := "error GetPeerIdsByProtocol: " + + C.GoStringN(C.getMyCharPtr(resp), C.int(C.getMyCharLen(resp))) + return nil, errors.New(errMsg) +} + +func (self *Waku) DisconnectPeerById(peerId peer.ID) error { + var resp = C.allocResp() + var cPeerId = C.CString(peerId.String()) + defer C.freeResp(resp) + defer C.free(unsafe.Pointer(cPeerId)) + + C.cGoWakuDisconnectPeerById(self.wakuCtx, cPeerId, resp) + + if C.getRet(resp) == C.RET_OK { + return nil + } + errMsg := "error DisconnectPeerById: " + + C.GoStringN(C.getMyCharPtr(resp), C.int(C.getMyCharLen(resp))) + return errors.New(errMsg) +} + +// func main() { + +// config := WakuConfig{ +// Host: "0.0.0.0", +// Port: 30304, +// NodeKey: "11d0dcea28e86f81937a3bd1163473c7fbc0a0db54fd72914849bc47bdf78710", +// EnableRelay: true, +// LogLevel: "DEBUG", +// } + +// node, err := wakuNew(config) +// if err != nil { +// fmt.Println("Error happened:", err.Error()) +// return +// } + +// node.WakuSetEventCallback() + +// defaultPubsubTopic, err := node.WakuDefaultPubsubTopic() +// if err != nil { +// fmt.Println("Error happened:", err.Error()) +// return +// } + +// err = node.WakuRelaySubscribe(defaultPubsubTopic) +// if err != nil { +// fmt.Println("Error happened:", err.Error()) +// return +// } + +// err = node.WakuConnect( +// // tries to connect to a localhost node with key: 0d714a1fada214dead6dc9c7274585eca0ff292451866e7d6d677dc818e8ccd2 +// "/ip4/0.0.0.0/tcp/60000/p2p/16Uiu2HAmVFXtAfSj4EiR7mL2KvL4EE2wztuQgUSBoj2Jx2KeXFLN", +// 10000) +// if err != nil { +// fmt.Println("Error happened:", err.Error()) +// return +// } + +// err = node.WakuStart() +// if err != nil { +// fmt.Println("Error happened:", err.Error()) +// return +// } + +// version, err := node.WakuVersion() +// if err != nil { +// fmt.Println("Error happened:", err.Error()) +// return +// } + +// formattedContentTopic, err := node.FormatContentTopic("appName", 1, "cTopicName", "enc") +// if err != nil { +// fmt.Println("Error happened:", err.Error()) +// return +// } + +// formattedPubsubTopic, err := node.FormatPubsubTopic("my-ctopic") +// if err != nil { +// fmt.Println("Error happened:", err.Error()) +// return +// } + +// listenAddresses, err := node.WakuListenAddresses() +// if err != nil { +// fmt.Println("Error happened:", err.Error()) +// return +// } + +// fmt.Println("Version:", version) +// fmt.Println("Custom content topic:", formattedContentTopic) +// fmt.Println("Custom pubsub topic:", formattedPubsubTopic) +// fmt.Println("Default pubsub topic:", defaultPubsubTopic) +// fmt.Println("Listen addresses:", listenAddresses) + +// // Wait for a SIGINT or SIGTERM signal +// ch := make(chan os.Signal, 1) +// signal.Notify(ch, syscall.SIGINT, syscall.SIGTERM) +// <-ch + +// err = node.WakuStop() +// if err != nil { +// fmt.Println("Error happened:", err.Error()) +// return +// } + +// err = node.WakuDestroy() +// if err != nil { +// fmt.Println("Error happened:", err.Error()) +// return +// } +// } + +// MaxMessageSize returns the maximum accepted message size. +/* TODO-nwaku +func (w *Waku) MaxMessageSize() uint32 { + return w.cfg.MaxMessageSize +} */ + +// New creates a WakuV2 client ready to communicate through the LibP2P network. +func New(nodeKey *ecdsa.PrivateKey, + fleet string, + cfg *WakuConfig, + logger *zap.Logger, + appDB *sql.DB, + ts *timesource.NTPTimeSource, + onHistoricMessagesRequestFailed func([]byte, peer.ID, error), + onPeerStats func(types.ConnStatus)) (*Waku, error) { + + // Lock the main goroutine to its current OS thread + runtime.LockOSThread() + + WakuSetup() // This should only be called once in the whole app's life + + node, err := wakuNew(nodeKey, + fleet, + cfg, logger, appDB, ts, onHistoricMessagesRequestFailed, + onPeerStats) + if err != nil { + return nil, err + } + + defaultPubsubTopic, err := node.WakuDefaultPubsubTopic() + if err != nil { + fmt.Println("Error happened:", err.Error()) + } + + err = node.WakuRelaySubscribe(defaultPubsubTopic) + if err != nil { + fmt.Println("Error happened:", err.Error()) + } + + node.WakuSetEventCallback() + + return node, nil + + // if !cfg.UseThrottledPublish || testing.Testing() { + // // To avoid delaying the tests, or for when we dont want to rate limit, we set up an infinite rate limiter, + // // basically disabling the rate limit functionality + // waku.limiter = publish.NewPublishRateLimiter(rate.Inf, 1) + + // } else { + // waku.limiter = publish.NewPublishRateLimiter(publishingLimiterRate, publishingLimitBurst) + // } + + // waku.filters = common.NewFilters(waku.cfg.DefaultShardPubsubTopic, waku.logger) + // waku.bandwidthCounter = metrics.NewBandwidthCounter() + + // if nodeKey == nil { + // // No nodekey is provided, create an ephemeral key + // nodeKey, err = crypto.GenerateKey() + // if err != nil { + // return nil, fmt.Errorf("failed to generate a random go-waku private key: %v", err) + // } + // } + + // hostAddr, err := net.ResolveTCPAddr("tcp", fmt.Sprint(cfg.Host, ":", cfg.Port)) + // if err != nil { + // return nil, fmt.Errorf("failed to setup the network interface: %v", err) + // } + + // libp2pOpts := node.DefaultLibP2POptions + // libp2pOpts = append(libp2pOpts, libp2p.BandwidthReporter(waku.bandwidthCounter)) + // libp2pOpts = append(libp2pOpts, libp2p.NATPortMap()) + + // opts := []node.WakuNodeOption{ + // node.WithLibP2POptions(libp2pOpts...), + // node.WithPrivateKey(nodeKey), + // node.WithHostAddress(hostAddr), + // node.WithConnectionNotification(waku.connectionNotifChan), + // node.WithTopicHealthStatusChannel(waku.topicHealthStatusChan), + // node.WithKeepAlive(randomPeersKeepAliveInterval, allPeersKeepAliveInterval), + // node.WithLogger(logger), + // node.WithLogLevel(logger.Level()), + // node.WithClusterID(cfg.ClusterID), + // node.WithMaxMsgSize(1024 * 1024), + // } + + // if cfg.EnableDiscV5 { + // bootnodes, err := waku.getDiscV5BootstrapNodes(waku.ctx, cfg.DiscV5BootstrapNodes) + // if err != nil { + // logger.Error("failed to get bootstrap nodes", zap.Error(err)) + // return nil, err + // } + // opts = append(opts, node.WithDiscoveryV5(uint(cfg.UDPPort), bootnodes, cfg.AutoUpdate)) + // } + // shards, err := protocol.TopicsToRelayShards(cfg.DefaultShardPubsubTopic) + // if err != nil { + // logger.Error("FATAL ERROR: failed to parse relay shards", zap.Error(err)) + // return nil, errors.New("failed to parse relay shard, invalid pubsubTopic configuration") + // } + // if len(shards) == 0 { //Hack so that tests don't fail. TODO: Need to remove this once tests are changed to use proper cluster and shard. + // shardInfo := protocol.RelayShards{ClusterID: 0, ShardIDs: []uint16{0}} + // shards = append(shards, shardInfo) + // } + // waku.defaultShardInfo = shards[0] + // if cfg.LightClient { + // opts = append(opts, node.WithWakuFilterLightNode()) + // waku.defaultShardInfo = shards[0] + // opts = append(opts, node.WithMaxPeerConnections(cfg.DiscoveryLimit)) + // cfg.EnableStoreConfirmationForMessagesSent = false + // //TODO: temporary work-around to improve lightClient connectivity, need to be removed once community sharding is implemented + // opts = append(opts, node.WithPubSubTopics(cfg.DefaultShardedPubsubTopics)) + // } else { + // relayOpts := []pubsub.Option{ + // pubsub.WithMaxMessageSize(int(waku.cfg.MaxMessageSize)), + // } + + // if waku.logger.Level() == zap.DebugLevel { + // relayOpts = append(relayOpts, pubsub.WithEventTracer(waku)) + // } + + // opts = append(opts, node.WithWakuRelayAndMinPeers(waku.cfg.MinPeersForRelay, relayOpts...)) + // opts = append(opts, node.WithMaxPeerConnections(maxRelayPeers)) + // cfg.EnablePeerExchangeClient = true //Enabling this until discv5 issues are resolved. This will enable more peers to be connected for relay mesh. + // cfg.EnableStoreConfirmationForMessagesSent = true + // } + + // if cfg.EnableStore { + // if appDB == nil { + // return nil, errors.New("appDB is required for store") + // } + // opts = append(opts, node.WithWakuStore()) + // dbStore, err := persistence.NewDBStore(logger, persistence.WithDB(appDB), persistence.WithRetentionPolicy(cfg.StoreCapacity, time.Duration(cfg.StoreSeconds)*time.Second)) + // if err != nil { + // return nil, err + // } + // opts = append(opts, node.WithMessageProvider(dbStore)) + // } + + // if !cfg.LightClient { + // opts = append(opts, node.WithWakuFilterFullNode(filter.WithMaxSubscribers(20))) + // opts = append(opts, node.WithLightPush(lightpush.WithRateLimiter(1, 1))) + // } + + // if appDB != nil { + // waku.protectedTopicStore, err = persistence.NewProtectedTopicsStore(logger, appDB) + // if err != nil { + // return nil, err + // } + // } + + // if cfg.EnablePeerExchangeServer { + // opts = append(opts, node.WithPeerExchange(peer_exchange.WithRateLimiter(1, 1))) + // } + + // waku.options = opts + // waku.logger.Info("setup the go-waku node successfully") + + // return waku, nil } diff --git a/wakuv2/nwaku_test.go b/wakuv2/nwaku_test.go new file mode 100644 index 00000000000..8f2ead7bb0a --- /dev/null +++ b/wakuv2/nwaku_test.go @@ -0,0 +1,807 @@ +//go:build use_nwaku +// +build use_nwaku + +package wakuv2 + +import ( + "context" + "errors" + "slices" + "testing" + "time" + + "github.com/cenkalti/backoff/v3" + "github.com/libp2p/go-libp2p/core/peer" + "github.com/waku-org/go-waku/waku/v2/protocol/store" + + "github.com/ethereum/go-ethereum/common/hexutil" + "github.com/ethereum/go-ethereum/crypto" + ethdnsdisc "github.com/ethereum/go-ethereum/p2p/dnsdisc" + "github.com/ethereum/go-ethereum/p2p/enode" + + "github.com/stretchr/testify/require" + + "github.com/status-im/status-go/protocol/tt" +) + +var testStoreENRBootstrap = "enrtree://AI4W5N5IFEUIHF5LESUAOSMV6TKWF2MB6GU2YK7PU4TYUGUNOCEPW@store.staging.status.nodes.status.im" +var testBootENRBootstrap = "enrtree://AMOJVZX4V6EXP7NTJPMAYJYST2QP6AJXYW76IU6VGJS7UVSNDYZG4@boot.staging.status.nodes.status.im" + +func setDefaultConfig(config *Config, lightMode bool) { + config.ClusterID = 16 + + if lightMode { + config.EnablePeerExchangeClient = true + config.LightClient = true + config.EnableDiscV5 = false + } else { + config.EnableDiscV5 = true + config.EnablePeerExchangeServer = true + config.LightClient = false + config.EnablePeerExchangeClient = false + } +} + +/* +func TestDiscoveryV5(t *testing.T) { + config := &Config{} + setDefaultConfig(config, false) + config.DiscV5BootstrapNodes = []string{testStoreENRBootstrap} + config.DiscoveryLimit = 20 + w, err := New(nil, "shards.staging", config, nil, nil, nil, nil, nil) + require.NoError(t, err) + + require.NoError(t, w.Start()) + + err = tt.RetryWithBackOff(func() error { + if len(w.Peers()) == 0 { + return errors.New("no peers discovered") + } + return nil + }) + + require.NoError(t, err) + + require.NotEqual(t, 0, len(w.Peers())) + require.NoError(t, w.Stop()) +} +*/ +/* +func TestRestartDiscoveryV5(t *testing.T) { + config := &Config{} + setDefaultConfig(config, false) + // Use wrong discv5 bootstrap address, to simulate being offline + config.DiscV5BootstrapNodes = []string{"enrtree://AOGECG2SPND25EEFMAJ5WF3KSGJNSGV356DSTL2YVLLZWIV6SAYBM@1.1.1.2"} + config.DiscoveryLimit = 20 + config.UDPPort = 10002 + config.ClusterID = 16 + w, err := New(nil, "", config, nil, nil, nil, nil, nil) + require.NoError(t, err) + + require.NoError(t, w.Start()) + require.False(t, w.seededBootnodesForDiscV5) + + options := func(b *backoff.ExponentialBackOff) { + b.MaxElapsedTime = 2 * time.Second + } + + // Sanity check, not great, but it's probably helpful + err = tt.RetryWithBackOff(func() error { + if len(w.Peers()) == 0 { + return errors.New("no peers discovered") + } + return nil + }, options) + + require.Error(t, err) + + w.discV5BootstrapNodes = []string{testStoreENRBootstrap} + + options = func(b *backoff.ExponentialBackOff) { + b.MaxElapsedTime = 90 * time.Second + } + + err = tt.RetryWithBackOff(func() error { + if len(w.Peers()) == 0 { + return errors.New("no peers discovered") + } + return nil + }, options) + require.NoError(t, err) + + require.True(t, w.seededBootnodesForDiscV5) + require.NotEqual(t, 0, len(w.Peers())) + require.NoError(t, w.Stop()) +} + +func TestRelayPeers(t *testing.T) { + config := &Config{ + EnableMissingMessageVerification: true, + } + setDefaultConfig(config, false) + w, err := New(nil, "", config, nil, nil, nil, nil, nil) + require.NoError(t, err) + require.NoError(t, w.Start()) + _, err = w.RelayPeersByTopic(config.DefaultShardPubsubTopic) + require.NoError(t, err) + + // Ensure function returns an error for lightclient + config = &Config{} + config.ClusterID = 16 + config.LightClient = true + w, err = New(nil, "", config, nil, nil, nil, nil, nil) + require.NoError(t, err) + require.NoError(t, w.Start()) + _, err = w.RelayPeersByTopic(config.DefaultShardPubsubTopic) + require.Error(t, err) +} +*/ +func parseNodes(rec []string) []*enode.Node { + var ns []*enode.Node + for _, r := range rec { + var n enode.Node + if err := n.UnmarshalText([]byte(r)); err != nil { + panic(err) + } + ns = append(ns, &n) + } + return ns +} + +// In order to run these tests, you must run an nwaku node +// +// Using Docker: +// +// IP_ADDRESS=$(hostname -I | awk '{print $1}'); +// docker run \ +// -p 61000:61000/tcp -p 8000:8000/udp -p 8646:8646/tcp harbor.status.im/wakuorg/nwaku:v0.33.0 \ +// --discv5-discovery=true --cluster-id=16 --log-level=DEBUG \ +// --nat=extip:${IP_ADDRESS} --discv5-udp-port=8000 --rest-address=0.0.0.0 --store --rest-port=8646 \ + +func TestBasicWakuV2(t *testing.T) { + extNodeRestPort := 8646 + storeNodeInfo, err := GetNwakuInfo(nil, &extNodeRestPort) + require.NoError(t, err) + + nwakuConfig := WakuConfig{ + Port: 30303, + NodeKey: "11d0dcea28e86f81937a3bd1163473c7fbc0a0db54fd72914849bc47bdf78710", + EnableRelay: true, + LogLevel: "DEBUG", + DnsDiscoveryUrl: "enrtree://AMOJVZX4V6EXP7NTJPMAYJYST2QP6AJXYW76IU6VGJS7UVSNDYZG4@boot.prod.status.nodes.status.im", + DnsDiscovery: true, + Discv5Discovery: true, + Staticnodes: []string{storeNodeInfo.ListenAddresses[0]}, + ClusterID: 16, + Shards: []uint16{64}, + } + + w, err := New(nil, "", &nwakuConfig, nil, nil, nil, nil, nil) + require.NoError(t, err) + require.NoError(t, w.Start()) + + enr, err := w.ENR() + require.NoError(t, err) + require.NotNil(t, enr) + + options := func(b *backoff.ExponentialBackOff) { + b.MaxElapsedTime = 30 * time.Second + } + + // Sanity check, not great, but it's probably helpful + err = tt.RetryWithBackOff(func() error { + + numConnected, err := w.GetNumConnectedPeers() + if err != nil { + return err + } + // Have to be connected to at least 3 nodes: the static node, the bootstrap node, and one discovered node + if numConnected > 2 { + return nil + } + return errors.New("no peers discovered") + }, options) + require.NoError(t, err) + + // Get local store node address + storeNode, err :=peer.AddrInfoFromString(storeNodeInfo.ListenAddresses[0]) + require.NoError(t, err) + require.NoError(t, err) + + // Check that we are indeed connected to the store node + connectedStoreNodes, err := w.GetPeerIdsByProtocol(string(store.StoreQueryID_v300)) + require.NoError(t, err) + require.True(t, slices.Contains(connectedStoreNodes, storeNode.ID), "nwaku should be connected to the store node") + + // Disconnect from the store node + err = w.DisconnectPeerById(storeNode.ID) + require.NoError(t, err) + + // Check that we are indeed disconnected + connectedStoreNodes, err = w.GetPeerIdsByProtocol(string(store.StoreQueryID_v300)) + require.NoError(t, err) + isDisconnected := !slices.Contains(connectedStoreNodes, storeNode.ID) + require.True(t, isDisconnected, "nwaku should be disconnected from the store node") + + // Re-connect + err = w.DialPeerByID(storeNode.ID) + require.NoError(t, err) + + // Check that we are connected again + connectedStoreNodes, err = w.GetPeerIdsByProtocol(string(store.StoreQueryID_v300)) + require.NoError(t, err) + require.True(t, slices.Contains(connectedStoreNodes, storeNode.ID), "nwaku should be connected to the store node") + + /* + filter := &common.Filter{ + PubsubTopic: config.DefaultShardPubsubTopic, + Messages: common.NewMemoryMessageStore(), + ContentTopics: common.NewTopicSetFromBytes([][]byte{{1, 2, 3, 4}}), + } + + _, err = w.Subscribe(filter) + require.NoError(t, err) + + msgTimestamp := w.timestamp() + contentTopic := maps.Keys(filter.ContentTopics)[0] + + time.Sleep(2 * time.Second) + + _, err = w.Send(config.DefaultShardPubsubTopic, &pb.WakuMessage{ + Payload: []byte{1, 2, 3, 4, 5}, + ContentTopic: contentTopic.ContentTopic(), + Version: proto.Uint32(0), + Timestamp: &msgTimestamp, + }, nil) + + require.NoError(t, err) + + time.Sleep(1 * time.Second) + + messages := filter.Retrieve() + require.Len(t, messages, 1) + + timestampInSeconds := msgTimestamp / int64(time.Second) + marginInSeconds := 20 + + options = func(b *backoff.ExponentialBackOff) { + b.MaxElapsedTime = 60 * time.Second + b.InitialInterval = 500 * time.Millisecond + } + err = tt.RetryWithBackOff(func() error { + _, envelopeCount, err := w.Query( + context.Background(), + storeNode.PeerID, + store.FilterCriteria{ + ContentFilter: protocol.NewContentFilter(config.DefaultShardPubsubTopic, contentTopic.ContentTopic()), + TimeStart: proto.Int64((timestampInSeconds - int64(marginInSeconds)) * int64(time.Second)), + TimeEnd: proto.Int64((timestampInSeconds + int64(marginInSeconds)) * int64(time.Second)), + }, + nil, + nil, + false, + ) + if err != nil || envelopeCount == 0 { + // in case of failure extend timestamp margin up to 40secs + if marginInSeconds < 40 { + marginInSeconds += 5 + } + return errors.New("no messages received from store node") + } + return nil + }, options) + require.NoError(t, err) */ + + require.NoError(t, w.Stop()) +} + +type mapResolver map[string]string + +func (mr mapResolver) LookupTXT(ctx context.Context, name string) ([]string, error) { + if record, ok := mr[name]; ok { + return []string{record}, nil + } + return nil, errors.New("not found") +} + +var signingKeyForTesting, _ = crypto.ToECDSA(hexutil.MustDecode("0xdc599867fc513f8f5e2c2c9c489cde5e71362d1d9ec6e693e0de063236ed1240")) + +func makeTestTree(domain string, nodes []*enode.Node, links []string) (*ethdnsdisc.Tree, string) { + tree, err := ethdnsdisc.MakeTree(1, nodes, links) + if err != nil { + panic(err) + } + url, err := tree.Sign(signingKeyForTesting, domain) + if err != nil { + panic(err) + } + return tree, url +} + +/* +func TestPeerExchange(t *testing.T) { + logger, err := zap.NewDevelopment() + require.NoError(t, err) + // start node which serve as PeerExchange server + config := &Config{} + config.ClusterID = 16 + config.EnableDiscV5 = true + config.EnablePeerExchangeServer = true + config.EnablePeerExchangeClient = false + pxServerNode, err := New(nil, "", config, logger.Named("pxServerNode"), nil, nil, nil, nil) + require.NoError(t, err) + require.NoError(t, pxServerNode.Start()) + + time.Sleep(1 * time.Second) + + // start node that will be discovered by PeerExchange + config = &Config{} + config.ClusterID = 16 + config.EnableDiscV5 = true + config.EnablePeerExchangeServer = false + config.EnablePeerExchangeClient = false + enr, err := pxServerNode.ENR() + require.NoError(t, err) + + config.DiscV5BootstrapNodes = []string{enr.String()} + discV5Node, err := New(nil, "", config, logger.Named("discV5Node"), nil, nil, nil, nil) + require.NoError(t, err) + require.NoError(t, discV5Node.Start()) + + time.Sleep(1 * time.Second) + + // start light node which use PeerExchange to discover peers + enrNodes := []*enode.Node{enr} + tree, url := makeTestTree("n", enrNodes, nil) + resolver := mapResolver(tree.ToTXT("n")) + + config = &Config{} + config.ClusterID = 16 + config.EnablePeerExchangeServer = false + config.EnablePeerExchangeClient = true + config.LightClient = true + config.Resolver = resolver + + config.WakuNodes = []string{url} + lightNode, err := New(nil, "", config, logger.Named("lightNode"), nil, nil, nil, nil) + require.NoError(t, err) + require.NoError(t, lightNode.Start()) + + // Sanity check, not great, but it's probably helpful + options := func(b *backoff.ExponentialBackOff) { + b.MaxElapsedTime = 30 * time.Second + } + err = tt.RetryWithBackOff(func() error { + // we should not use lightNode.Peers() here as it only indicates peers that are connected right now, + // in light client mode,the peer will be closed via `w.node.Host().Network().ClosePeer(peerInfo.ID)` + // after invoking identifyAndConnect, instead, we should check the peerStore, peers from peerStore + // won't get deleted especially if they are statically added. + numConnected, err := lightNode.GetNumConnectedPeers() + if err != nil { + return err + } + if numConnected == 2 { + return nil + } + return errors.New("no peers discovered") + }, options) + require.NoError(t, err) + + _, cancel := context.WithCancel(context.Background()) + defer cancel() + _, err = discV5Node.WakuPeerExchangeRequest(1) + require.NoError(t, err) + _, err = discV5Node.WakuPeerExchangeRequest(1) + require.Error(t, err) //should fail due to rate limit + + require.NoError(t, lightNode.Stop()) + require.NoError(t, pxServerNode.Stop()) + require.NoError(t, discV5Node.Stop()) +} + +func TestWakuV2Filter(t *testing.T) { + t.Skip("flaky test") + + enrTreeAddress := testBootENRBootstrap + envEnrTreeAddress := os.Getenv("ENRTREE_ADDRESS") + if envEnrTreeAddress != "" { + enrTreeAddress = envEnrTreeAddress + } + config := &Config{} + setDefaultConfig(config, true) + config.EnablePeerExchangeClient = false + config.Port = 0 + config.MinPeersForFilter = 2 + + config.DiscV5BootstrapNodes = []string{enrTreeAddress} + config.DiscoveryLimit = 20 + config.WakuNodes = []string{enrTreeAddress} + w, err := New(nil, "", config, nil, nil, nil, nil, nil) + require.NoError(t, err) + require.NoError(t, w.Start()) + + options := func(b *backoff.ExponentialBackOff) { + b.MaxElapsedTime = 10 * time.Second + } + time.Sleep(10 * time.Second) //TODO: Check if we can remove this sleep. + + // Sanity check, not great, but it's probably helpful + err = tt.RetryWithBackOff(func() error { + peers, err := w.GetPeerIdsByProtocol(string(filter.FilterSubscribeID_v20beta1)) + if err != nil { + return err + } + if len(peers) < 2 { + return errors.New("no peers discovered") + } + return nil + }, options) + require.NoError(t, err) + testPubsubTopic := "/waku/2/rs/16/32" + contentTopicBytes := make([]byte, 4) + _, err = rand.Read(contentTopicBytes) + require.NoError(t, err) + filter := &common.Filter{ + Messages: common.NewMemoryMessageStore(), + PubsubTopic: testPubsubTopic, + ContentTopics: common.NewTopicSetFromBytes([][]byte{contentTopicBytes}), + } + + fID, err := w.Subscribe(filter) + require.NoError(t, err) + + msgTimestamp := w.timestamp() + contentTopic := maps.Keys(filter.ContentTopics)[0] + + _, err = w.Send(testPubsubTopic, &pb.WakuMessage{ + Payload: []byte{1, 2, 3, 4, 5}, + ContentTopic: contentTopic.ContentTopic(), + Version: proto.Uint32(0), + Timestamp: &msgTimestamp, + }, nil) + require.NoError(t, err) + time.Sleep(5 * time.Second) + + // Ensure there is at least 1 active filter subscription + subscriptions := w.FilterLightnode().Subscriptions() + require.Greater(t, len(subscriptions), 0) + + messages := filter.Retrieve() + require.Len(t, messages, 1) + + // Mock peers going down + _, err = w.FilterLightnode().UnsubscribeWithSubscription(w.ctx, subscriptions[0]) + require.NoError(t, err) + + time.Sleep(10 * time.Second) + + // Ensure there is at least 1 active filter subscription + subscriptions = w.FilterLightnode().Subscriptions() + require.Greater(t, len(subscriptions), 0) + + // Ensure that messages are retrieved with a fresh sub + _, err = w.Send(testPubsubTopic, &pb.WakuMessage{ + Payload: []byte{1, 2, 3, 4, 5, 6}, + ContentTopic: contentTopic.ContentTopic(), + Version: proto.Uint32(0), + Timestamp: &msgTimestamp, + }, nil) + require.NoError(t, err) + time.Sleep(10 * time.Second) + + messages = filter.Retrieve() + require.Len(t, messages, 1) + err = w.Unsubscribe(context.Background(), fID) + require.NoError(t, err) + require.NoError(t, w.Stop()) +} + +func TestWakuV2Store(t *testing.T) { + t.Skip("deprecated. Storenode must use nwaku") + + // Configuration for the first Waku node + config1 := &Config{ + Port: 0, + ClusterID: 16, + EnableDiscV5: false, + DiscoveryLimit: 20, + EnableStore: false, + StoreCapacity: 100, + StoreSeconds: 3600, + EnableMissingMessageVerification: true, + } + w1PeersCh := make(chan peer.IDSlice, 100) // buffered not to block on the send side + + // Start the first Waku node + w1, err := New(nil, "", config1, nil, nil, nil, nil, func(cs types.ConnStatus) { + w1PeersCh <- maps.Keys(cs.Peers) + }) + require.NoError(t, err) + require.NoError(t, w1.Start()) + defer func() { + require.NoError(t, w1.Stop()) + close(w1PeersCh) + }() + + // Configuration for the second Waku node + sql2, err := helpers.SetupTestMemorySQLDB(appdatabase.DbInitializer{}) + require.NoError(t, err) + config2 := &Config{ + Port: 0, + ClusterID: 16, + EnableDiscV5: false, + DiscoveryLimit: 20, + EnableStore: true, + StoreCapacity: 100, + StoreSeconds: 3600, + } + + // Start the second Waku node + w2, err := New(nil, "", config2, nil, sql2, nil, nil, nil) + require.NoError(t, err) + require.NoError(t, w2.Start()) + w2EnvelopeCh := make(chan common.EnvelopeEvent, 100) + w2.SubscribeEnvelopeEvents(w2EnvelopeCh) + defer func() { + require.NoError(t, w2.Stop()) + close(w2EnvelopeCh) + }() + + // Connect the two nodes directly + peer2Addr, err := w2.ListenAddresses() + require.NoError(t, err) + + err = w1.DialPeer(peer2Addr[0]) + require.NoError(t, err) + + // Create a filter for the second node to catch messages + filter := &common.Filter{ + Messages: common.NewMemoryMessageStore(), + PubsubTopic: config2.DefaultShardPubsubTopic, + ContentTopics: common.NewTopicSetFromBytes([][]byte{{1, 2, 3, 4}}), + } + + _, err = w2.Subscribe(filter) + require.NoError(t, err) + + time.Sleep(2 * time.Second) + + // Send a message from the first node + msgTimestamp := w1.CurrentTime().UnixNano() + contentTopic := maps.Keys(filter.ContentTopics)[0] + _, err = w1.Send(config1.DefaultShardPubsubTopic, &pb.WakuMessage{ + Payload: []byte{1, 2, 3, 4, 5}, + ContentTopic: contentTopic.ContentTopic(), + Version: proto.Uint32(0), + Timestamp: &msgTimestamp, + }, nil) + require.NoError(t, err) + + waitForEnvelope(t, contentTopic.ContentTopic(), w2EnvelopeCh) + + // Retrieve the message from the second node's filter + messages := filter.Retrieve() + require.Len(t, messages, 1) + + timestampInSeconds := msgTimestamp / int64(time.Second) + marginInSeconds := 5 + // Query the second node's store for the message + _, envelopeCount, err := w1.Query( + context.Background(), + w2.Host().ID(), + store.FilterCriteria{ + TimeStart: proto.Int64((timestampInSeconds - int64(marginInSeconds)) * int64(time.Second)), + TimeEnd: proto.Int64((timestampInSeconds + int64(marginInSeconds)) * int64(time.Second)), + ContentFilter: protocol.NewContentFilter(config1.DefaultShardPubsubTopic, contentTopic.ContentTopic()), + }, + nil, + nil, + false, + ) + require.NoError(t, err) + require.True(t, envelopeCount > 0, "no messages received from store node") +} + +func waitForPeerConnection(t *testing.T, peerID peer.ID, peerCh chan peer.IDSlice) { + waitForPeerConnectionWithTimeout(t, peerID, peerCh, 3*time.Second) +} + +func waitForPeerConnectionWithTimeout(t *testing.T, peerID peer.ID, peerCh chan peer.IDSlice, timeout time.Duration) { + ctx, cancel := context.WithTimeout(context.Background(), timeout) + defer cancel() + for { + select { + case peers := <-peerCh: + for _, p := range peers { + if p == peerID { + return + } + } + case <-ctx.Done(): + require.Fail(t, "timed out waiting for peer "+peerID.String()) + return + } + } +} + +func waitForEnvelope(t *testing.T, contentTopic string, envCh chan common.EnvelopeEvent) { + ctx, cancel := context.WithTimeout(context.Background(), 2*time.Second) + defer cancel() + for { + select { + case env := <-envCh: + if env.Topic.ContentTopic() == contentTopic { + return + } + case <-ctx.Done(): + require.Fail(t, "timed out waiting for envelope's topic "+contentTopic) + return + } + } +} + +func TestOnlineChecker(t *testing.T) { + w, err := New(nil, "shards.staging", nil, nil, nil, nil, nil, nil) + require.NoError(t, w.Start()) + + require.NoError(t, err) + require.False(t, w.onlineChecker.IsOnline()) + + w.ConnectionChanged(connection.State{Offline: false}) + require.True(t, w.onlineChecker.IsOnline()) + + wg := sync.WaitGroup{} + wg.Add(1) + go func() { + defer wg.Done() + <-w.goingOnline + require.True(t, true) + }() + + time.Sleep(100 * time.Millisecond) + + w.ConnectionChanged(connection.State{Offline: true}) + require.False(t, w.onlineChecker.IsOnline()) + + // Test lightnode online checker + config := &Config{} + config.ClusterID = 16 + config.LightClient = true + lightNode, err := New(nil, "shards.staging", config, nil, nil, nil, nil, nil) + require.NoError(t, err) + + err = lightNode.Start() + require.NoError(t, err) + + require.False(t, lightNode.onlineChecker.IsOnline()) + f := &common.Filter{} + lightNode.filterManager.SubscribeFilter("test", protocol.NewContentFilter(f.PubsubTopic, f.ContentTopics.ContentTopics()...)) + +} + +func TestLightpushRateLimit(t *testing.T) { + logger, err := zap.NewDevelopment() + require.NoError(t, err) + + config0 := &Config{} + setDefaultConfig(config0, false) + w0PeersCh := make(chan peer.IDSlice, 5) // buffered not to block on the send side + + // Start the relayu node + w0, err := New(nil, "", config0, logger.Named("relayNode"), nil, nil, nil, func(cs types.ConnStatus) { + w0PeersCh <- maps.Keys(cs.Peers) + }) + require.NoError(t, err) + require.NoError(t, w0.Start()) + defer func() { + require.NoError(t, w0.Stop()) + close(w0PeersCh) + }() + + contentTopics := common.NewTopicSetFromBytes([][]byte{{1, 2, 3, 4}}) + filter := &common.Filter{ + PubsubTopic: config0.DefaultShardPubsubTopic, + Messages: common.NewMemoryMessageStore(), + ContentTopics: contentTopics, + } + + _, err = w0.Subscribe(filter) + require.NoError(t, err) + + config1 := &Config{} + setDefaultConfig(config1, false) + w1PeersCh := make(chan peer.IDSlice, 5) // buffered not to block on the send side + + // Start the full node + w1, err := New(nil, "", config1, logger.Named("fullNode"), nil, nil, nil, func(cs types.ConnStatus) { + w1PeersCh <- maps.Keys(cs.Peers) + }) + require.NoError(t, err) + require.NoError(t, w1.Start()) + defer func() { + require.NoError(t, w1.Stop()) + close(w1PeersCh) + }() + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + //Connect the relay peer and full node + err = w1.DialPeer(ctx, w0.ListenAddresses()[0].String()) + require.NoError(t, err) + + err = tt.RetryWithBackOff(func() error { + if len(w1.Peers()) == 0 { + return errors.New("no peers discovered") + } + return nil + }) + require.NoError(t, err) + + config2 := &Config{} + setDefaultConfig(config2, true) + w2PeersCh := make(chan peer.IDSlice, 5) // buffered not to block on the send side + + // Start the light node + w2, err := New(nil, "", config2, logger.Named("lightNode"), nil, nil, nil, func(cs types.ConnStatus) { + w2PeersCh <- maps.Keys(cs.Peers) + }) + require.NoError(t, err) + require.NoError(t, w2.Start()) + defer func() { + require.NoError(t, w2.Stop()) + close(w2PeersCh) + }() + + //Use this instead of DialPeer to make sure the peer is added to PeerStore and can be selected for Lighpush + w2.AddDiscoveredPeer(w1.PeerID(), w1.ListenAddresses(), wps.Static, w1.cfg.DefaultShardedPubsubTopics, w1.node.ENR(), true) + + waitForPeerConnectionWithTimeout(t, w2.Host().ID(), w1PeersCh, 5*time.Second) + + event := make(chan common.EnvelopeEvent, 10) + w2.SubscribeEnvelopeEvents(event) + + for i := range [4]int{} { + msgTimestamp := w2.timestamp() + _, err := w2.Send(config2.DefaultShardPubsubTopic, &pb.WakuMessage{ + Payload: []byte{1, 2, 3, 4, 5, 6, byte(i)}, + ContentTopic: maps.Keys(contentTopics)[0].ContentTopic(), + Version: proto.Uint32(0), + Timestamp: &msgTimestamp, + }, nil) + + require.NoError(t, err) + + time.Sleep(550 * time.Millisecond) + + } + + messages := filter.Retrieve() + require.Len(t, messages, 2) + +} + +func TestTelemetryFormat(t *testing.T) { + logger, err := zap.NewDevelopment() + require.NoError(t, err) + + tc := NewBandwidthTelemetryClient(logger, "#") + + s := metrics.Stats{ + TotalIn: 10, + TotalOut: 20, + RateIn: 30, + RateOut: 40, + } + + m := make(map[libp2pprotocol.ID]metrics.Stats) + m[relay.WakuRelayID_v200] = s + m[filter.FilterPushID_v20beta1] = s + m[filter.FilterSubscribeID_v20beta1] = s + m[legacy_store.StoreID_v20beta4] = s + m[lightpush.LightPushID_v20beta1] = s + + requestBody := tc.getTelemetryRequestBody(m) + _, err = json.Marshal(requestBody) + require.NoError(t, err) +} +*/ diff --git a/wakuv2/nwaku_test_utils.go b/wakuv2/nwaku_test_utils.go new file mode 100644 index 00000000000..ed9f3e80ce3 --- /dev/null +++ b/wakuv2/nwaku_test_utils.go @@ -0,0 +1,58 @@ +package wakuv2 + +import ( + "encoding/json" + "fmt" + "io" + "net/http" + "os" + "strconv" +) + +type NwakuInfo struct { + ListenAddresses []string `json:"listenAddresses"` + EnrUri string `json:"enrUri"` +} + +func GetNwakuInfo(host *string, port *int) (NwakuInfo, error) { + nwakuRestPort := 8645 + if port != nil { + nwakuRestPort = *port + } + envNwakuRestPort := os.Getenv("NWAKU_REST_PORT") + if envNwakuRestPort != "" { + v, err := strconv.Atoi(envNwakuRestPort) + if err != nil { + return NwakuInfo{}, err + } + nwakuRestPort = v + } + + nwakuRestHost := "localhost" + if host != nil { + nwakuRestHost = *host + } + envNwakuRestHost := os.Getenv("NWAKU_REST_HOST") + if envNwakuRestHost != "" { + nwakuRestHost = envNwakuRestHost + } + + resp, err := http.Get(fmt.Sprintf("http://%s:%d/debug/v1/info", nwakuRestHost, nwakuRestPort)) + if err != nil { + return NwakuInfo{}, err + } + defer resp.Body.Close() + + body, err := io.ReadAll(resp.Body) + if err != nil { + return NwakuInfo{}, err + } + + var data NwakuInfo + err = json.Unmarshal(body, &data) + if err != nil { + return NwakuInfo{}, err + } + + return data, nil +} diff --git a/protocol/common/shard/shard.go b/wakuv2/shard.go similarity index 98% rename from protocol/common/shard/shard.go rename to wakuv2/shard.go index 011a6f452dc..fc86867550e 100644 --- a/protocol/common/shard/shard.go +++ b/wakuv2/shard.go @@ -1,4 +1,4 @@ -package shard +package wakuv2 import ( wakuproto "github.com/waku-org/go-waku/waku/v2/protocol" diff --git a/wakuv2/waku_test.go b/wakuv2/waku_test.go index 36b05d8fbde..3cdf31a520a 100644 --- a/wakuv2/waku_test.go +++ b/wakuv2/waku_test.go @@ -1,3 +1,6 @@ +//go:build !use_nwaku +// +build !use_nwaku + package wakuv2 import ( @@ -548,8 +551,9 @@ func TestWakuV2Store(t *testing.T) { }() // Connect the two nodes directly - peer2Addr := w2.node.ListenAddresses()[0].String() - err = w1.node.DialPeer(context.Background(), peer2Addr) + peer2Addr, err := w2.ListenAddresses() + require.NoError(t, err) + err = w1.node.DialPeer(context.Background(), peer2Addr[0].String()) require.NoError(t, err) waitForPeerConnection(t, w2.node.Host().ID(), w1PeersCh) @@ -723,7 +727,9 @@ func TestLightpushRateLimit(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() //Connect the relay peer and full node - err = w1.node.DialPeer(ctx, w0.node.ListenAddresses()[0].String()) + peerAddr, err := w0.ListenAddresses() + require.NoError(t, err) + err = w1.node.DialPeer(ctx, peerAddr[0].String()) require.NoError(t, err) err = tt.RetryWithBackOff(func() error { @@ -750,7 +756,9 @@ func TestLightpushRateLimit(t *testing.T) { }() //Use this instead of DialPeer to make sure the peer is added to PeerStore and can be selected for Lighpush - w2.node.AddDiscoveredPeer(w1.PeerID(), w1.node.ListenAddresses(), wps.Static, w1.cfg.DefaultShardedPubsubTopics, w1.node.ENR(), true) + addresses, err := w1.ListenAddresses() + require.NoError(t, err) + w2.node.AddDiscoveredPeer(w1.PeerID(), addresses, wps.Static, w1.cfg.DefaultShardedPubsubTopics, w1.node.ENR(), true) waitForPeerConnectionWithTimeout(t, w2.node.Host().ID(), w1PeersCh, 5*time.Second) From 8198ac19969cdaa199ab92eae115bf136574c0db Mon Sep 17 00:00:00 2001 From: Richard Ramos Date: Thu, 17 Oct 2024 17:54:13 -0400 Subject: [PATCH 04/14] fix_: rebase --- third_party/nwaku | 2 +- wakuv2/api.go | 2 +- wakuv2/history_processor_wrapper.go | 5 +- wakuv2/message_publishing.go | 2 + wakuv2/nwaku.go | 1567 ++++++++++++++++----------- wakuv2/nwaku_test.go | 126 +-- 6 files changed, 1019 insertions(+), 685 deletions(-) diff --git a/third_party/nwaku b/third_party/nwaku index c861fa9f756..c5a825e206c 160000 --- a/third_party/nwaku +++ b/third_party/nwaku @@ -1 +1 @@ -Subproject commit c861fa9f7560068874570598c81b7a1425a9e931 +Subproject commit c5a825e206c1cb3e6e0cf8c01410527804cb76c4 diff --git a/wakuv2/api.go b/wakuv2/api.go index f24e98f1883..c129abdc033 100644 --- a/wakuv2/api.go +++ b/wakuv2/api.go @@ -513,4 +513,4 @@ func (api *PublicWakuAPI) NewMessageFilter(req Criteria) (string, error) { api.mu.Unlock() return id, nil -} */ \ No newline at end of file +} */ diff --git a/wakuv2/history_processor_wrapper.go b/wakuv2/history_processor_wrapper.go index 25cee9e8f88..bdb72745cce 100644 --- a/wakuv2/history_processor_wrapper.go +++ b/wakuv2/history_processor_wrapper.go @@ -3,7 +3,6 @@ package wakuv2 import ( "github.com/libp2p/go-libp2p/core/peer" - "github.com/status-im/status-go/wakuv2/common" "github.com/waku-org/go-waku/waku/v2/api/history" "github.com/waku-org/go-waku/waku/v2/protocol" ) @@ -17,7 +16,9 @@ func NewHistoryProcessorWrapper(waku *Waku) history.HistoryProcessor { } func (hr *HistoryProcessorWrapper) OnEnvelope(env *protocol.Envelope, processEnvelopes bool) error { - return hr.waku.OnNewEnvelopes(env, common.StoreMessageType, processEnvelopes) + // TODO-nwaku + // return hr.waku.OnNewEnvelopes(env, common.StoreMessageType, processEnvelopes) + return nil } func (hr *HistoryProcessorWrapper) OnRequestFailed(requestID []byte, peerID peer.ID, err error) { diff --git a/wakuv2/message_publishing.go b/wakuv2/message_publishing.go index 93543bc6e39..90dce0cae8d 100644 --- a/wakuv2/message_publishing.go +++ b/wakuv2/message_publishing.go @@ -1,5 +1,6 @@ package wakuv2 +/* TODO-nwaku import ( "errors" @@ -116,3 +117,4 @@ func (w *Waku) publishEnvelope(envelope *protocol.Envelope) { }) } } +*/ diff --git a/wakuv2/nwaku.go b/wakuv2/nwaku.go index a1d70379817..f121343e8a3 100644 --- a/wakuv2/nwaku.go +++ b/wakuv2/nwaku.go @@ -270,7 +270,6 @@ import ( "crypto/ecdsa" "crypto/sha256" "database/sql" - "encoding/hex" "encoding/json" "errors" "fmt" @@ -281,43 +280,47 @@ import ( "time" "unsafe" + "github.com/jellydator/ttlcache/v3" + "github.com/libp2p/go-libp2p/core/peer" + "github.com/libp2p/go-libp2p/core/peerstore" + "github.com/multiformats/go-multiaddr" + + "go.uber.org/zap" + + "golang.org/x/crypto/pbkdf2" + gethcommon "github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/common/hexutil" "github.com/ethereum/go-ethereum/crypto" "github.com/ethereum/go-ethereum/event" "github.com/ethereum/go-ethereum/p2p" "github.com/ethereum/go-ethereum/p2p/enode" - "github.com/jellydator/ttlcache/v3" + "github.com/libp2p/go-libp2p/core/metrics" - "github.com/libp2p/go-libp2p/core/peer" - peermod "github.com/libp2p/go-libp2p/core/peer" - "github.com/libp2p/go-libp2p/core/peerstore" - "github.com/multiformats/go-multiaddr" - ma "github.com/multiformats/go-multiaddr" - "github.com/status-im/status-go/connection" - "github.com/status-im/status-go/eth-node/types" - "github.com/status-im/status-go/logutils" - "github.com/status-im/status-go/timesource" - "github.com/status-im/status-go/wakuv2/common" - "github.com/status-im/status-go/wakuv2/persistence" + filterapi "github.com/waku-org/go-waku/waku/v2/api/filter" + "github.com/waku-org/go-waku/waku/v2/api/history" "github.com/waku-org/go-waku/waku/v2/api/missing" "github.com/waku-org/go-waku/waku/v2/api/publish" "github.com/waku-org/go-waku/waku/v2/dnsdisc" - node "github.com/waku-org/go-waku/waku/v2/node" "github.com/waku-org/go-waku/waku/v2/onlinechecker" "github.com/waku-org/go-waku/waku/v2/peermanager" wps "github.com/waku-org/go-waku/waku/v2/peerstore" "github.com/waku-org/go-waku/waku/v2/protocol" "github.com/waku-org/go-waku/waku/v2/protocol/legacy_store" - "github.com/waku-org/go-waku/waku/v2/protocol/pb" "github.com/waku-org/go-waku/waku/v2/protocol/relay" "github.com/waku-org/go-waku/waku/v2/protocol/store" - storepb "github.com/waku-org/go-waku/waku/v2/protocol/store/pb" "github.com/waku-org/go-waku/waku/v2/utils" - "go.uber.org/zap" - "golang.org/x/crypto/pbkdf2" - "golang.org/x/time/rate" + + gocommon "github.com/status-im/status-go/common" + "github.com/status-im/status-go/connection" + "github.com/status-im/status-go/eth-node/types" + "github.com/status-im/status-go/timesource" + "github.com/status-im/status-go/wakuv2/common" + "github.com/status-im/status-go/wakuv2/persistence" + + node "github.com/waku-org/go-waku/waku/v2/node" + "github.com/waku-org/go-waku/waku/v2/protocol/pb" ) const messageQueueLimit = 1024 @@ -328,14 +331,11 @@ const cacheTTL = 20 * time.Minute const maxRelayPeers = 300 const randomPeersKeepAliveInterval = 5 * time.Second const allPeersKeepAliveInterval = 5 * time.Minute -const peersToPublishForLightpush = 2 -const publishingLimiterRate = rate.Limit(2) -const publishingLimitBurst = 4 -/* TODO-nwaku +/* TODO-nwaku type SentEnvelope struct { Envelope *protocol.Envelope - PublishMethod PublishMethod + PublishMethod publish.PublishMethod } type ErrorSendingEnvelope struct { @@ -344,13 +344,118 @@ type ErrorSendingEnvelope struct { } type ITelemetryClient interface { - PushReceivedEnvelope(ctx context.Context, receivedEnvelope *protocol.Envelope) + SetDeviceType(deviceType string) PushSentEnvelope(ctx context.Context, sentEnvelope SentEnvelope) PushErrorSendingEnvelope(ctx context.Context, errorSendingEnvelope ErrorSendingEnvelope) PushPeerCount(ctx context.Context, peerCount int) PushPeerConnFailures(ctx context.Context, peerConnFailures map[string]int) + PushMessageCheckSuccess(ctx context.Context, messageHash string) + PushMessageCheckFailure(ctx context.Context, messageHash string) + PushPeerCountByShard(ctx context.Context, peerCountByShard map[uint16]uint) + PushPeerCountByOrigin(ctx context.Context, peerCountByOrigin map[wps.Origin]uint) +} +*/ + +type WakuMessageHash = string +type WakuPubsubTopic = string +type WakuContentTopic = string + +type WakuConfig struct { + Host string `json:"host,omitempty"` + Port int `json:"port,omitempty"` + NodeKey string `json:"key,omitempty"` + EnableRelay bool `json:"relay"` + LogLevel string `json:"logLevel"` + DnsDiscovery bool `json:"dnsDiscovery,omitempty"` + DnsDiscoveryUrl string `json:"dnsDiscoveryUrl,omitempty"` + MaxMessageSize string `json:"maxMessageSize,omitempty"` + Staticnodes []string `json:"staticnodes,omitempty"` + Discv5BootstrapNodes []string `json:"discv5BootstrapNodes,omitempty"` + Discv5Discovery bool `json:"discv5Discovery,omitempty"` + ClusterID uint16 `json:"clusterId,omitempty"` + Shards []uint16 `json:"shards,omitempty"` +} + +// Waku represents a dark communication interface through the Ethereum +// network, using its very own P2P communication layer. +type Waku struct { + wakuCtx unsafe.Pointer + + appDB *sql.DB + + dnsAddressCache map[string][]dnsdisc.DiscoveredNode // Map to store the multiaddresses returned by dns discovery + dnsAddressCacheLock *sync.RWMutex // lock to handle access to the map + dnsDiscAsyncRetrievedSignal chan struct{} + + // Filter-related + filters *common.Filters // Message filters installed with Subscribe function + filterManager *filterapi.FilterManager + + privateKeys map[string]*ecdsa.PrivateKey // Private key storage + symKeys map[string][]byte // Symmetric key storage + keyMu sync.RWMutex // Mutex associated with key stores + + envelopeCache *ttlcache.Cache[gethcommon.Hash, *common.ReceivedMessage] // Pool of envelopes currently tracked by this node + poolMu sync.RWMutex // Mutex to sync the message and expiration pools + + bandwidthCounter *metrics.BandwidthCounter + + protectedTopicStore *persistence.ProtectedTopicsStore + + sendQueue *publish.MessageQueue + + missingMsgVerifier *missing.MissingMessageVerifier + + msgQueue chan *common.ReceivedMessage // Message queue for waku messages that havent been decoded + + ctx context.Context + cancel context.CancelFunc + wg sync.WaitGroup + + cfg *WakuConfig + options []node.WakuNodeOption + + envelopeFeed event.Feed + + storeMsgIDs map[gethcommon.Hash]bool // Map of the currently processing ids + storeMsgIDsMu sync.RWMutex + + messageSender *publish.MessageSender + + topicHealthStatusChan chan peermanager.TopicHealthStatus + connectionNotifChan chan node.PeerConnection + connStatusSubscriptions map[string]*types.ConnStatusSubscription + connStatusMu sync.Mutex + onlineChecker *onlinechecker.DefaultOnlineChecker + state connection.State + + StorenodeCycle *history.StorenodeCycle + HistoryRetriever *history.HistoryRetriever + + logger *zap.Logger + + // NTP Synced timesource + timesource *timesource.NTPTimeSource + + // seededBootnodesForDiscV5 indicates whether we manage to retrieve discovery + // bootnodes successfully + seededBootnodesForDiscV5 bool + + // goingOnline is channel that notifies when connectivity has changed from offline to online + goingOnline chan struct{} + + // discV5BootstrapNodes is the ENR to be used to fetch bootstrap nodes for discovery + discV5BootstrapNodes []string + + onHistoricMessagesRequestFailed func([]byte, peer.ID, error) + onPeerStats func(types.ConnStatus) + + // statusTelemetryClient ITelemetryClient // TODO-nwaku + + defaultShardInfo protocol.RelayShards } +/* TODO-nwaku func (w *Waku) SetStatusTelemetryClient(client ITelemetryClient) { w.statusTelemetryClient = client } */ @@ -361,6 +466,193 @@ func newTTLCache() *ttlcache.Cache[gethcommon.Hash, *common.ReceivedMessage] { return cache } +// New creates a WakuV2 client ready to communicate through the LibP2P network. +func New(nodeKey *ecdsa.PrivateKey, fleet string, cfg *WakuConfig, logger *zap.Logger, appDB *sql.DB, ts *timesource.NTPTimeSource, onHistoricMessagesRequestFailed func([]byte, peer.ID, error), onPeerStats func(types.ConnStatus)) (*Waku, error) { + // Lock the main goroutine to its current OS thread + runtime.LockOSThread() + + WakuSetup() // This should only be called once in the whole app's life + + node, err := wakuNew(nodeKey, + fleet, + cfg, logger, appDB, ts, onHistoricMessagesRequestFailed, + onPeerStats) + if err != nil { + return nil, err + } + + defaultPubsubTopic, err := node.WakuDefaultPubsubTopic() + if err != nil { + fmt.Println("Error happened:", err.Error()) + } + + err = node.WakuRelaySubscribe(defaultPubsubTopic) + if err != nil { + fmt.Println("Error happened:", err.Error()) + } + + node.WakuSetEventCallback() + + return node, nil + + // TODO-nwaku + /* + var err error + if logger == nil { + logger, err = zap.NewDevelopment() + if err != nil { + return nil, err + } + } + + if ts == nil { + ts = timesource.Default() + } + + cfg = setDefaults(cfg) + if err = cfg.Validate(logger); err != nil { + return nil, err + } + + logger.Info("starting wakuv2 with config", zap.Any("config", cfg)) + + ctx, cancel := context.WithCancel(context.Background()) + + waku := &Waku{ + appDB: appDB, + cfg: cfg, + privateKeys: make(map[string]*ecdsa.PrivateKey), + symKeys: make(map[string][]byte), + envelopeCache: newTTLCache(), + msgQueue: make(chan *common.ReceivedMessage, messageQueueLimit), + topicHealthStatusChan: make(chan peermanager.TopicHealthStatus, 100), + connectionNotifChan: make(chan node.PeerConnection, 20), + connStatusSubscriptions: make(map[string]*types.ConnStatusSubscription), + ctx: ctx, + cancel: cancel, + wg: sync.WaitGroup{}, + dnsAddressCache: make(map[string][]dnsdisc.DiscoveredNode), + dnsAddressCacheLock: &sync.RWMutex{}, + dnsDiscAsyncRetrievedSignal: make(chan struct{}), + storeMsgIDs: make(map[gethcommon.Hash]bool), + timesource: ts, + storeMsgIDsMu: sync.RWMutex{}, + logger: logger, + discV5BootstrapNodes: cfg.DiscV5BootstrapNodes, + onHistoricMessagesRequestFailed: onHistoricMessagesRequestFailed, + onPeerStats: onPeerStats, + onlineChecker: onlinechecker.NewDefaultOnlineChecker(false).(*onlinechecker.DefaultOnlineChecker), + sendQueue: publish.NewMessageQueue(1000, cfg.UseThrottledPublish), + } + + waku.filters = common.NewFilters(waku.cfg.DefaultShardPubsubTopic, waku.logger) + waku.bandwidthCounter = metrics.NewBandwidthCounter() + + if nodeKey == nil { + // No nodekey is provided, create an ephemeral key + nodeKey, err = crypto.GenerateKey() + if err != nil { + return nil, fmt.Errorf("failed to generate a random go-waku private key: %v", err) + } + } + + hostAddr, err := net.ResolveTCPAddr("tcp", fmt.Sprint(cfg.Host, ":", cfg.Port)) + if err != nil { + return nil, fmt.Errorf("failed to setup the network interface: %v", err) + } + + libp2pOpts := node.DefaultLibP2POptions + libp2pOpts = append(libp2pOpts, libp2p.BandwidthReporter(waku.bandwidthCounter)) + libp2pOpts = append(libp2pOpts, libp2p.NATPortMap()) + + opts := []node.WakuNodeOption{ + node.WithLibP2POptions(libp2pOpts...), + node.WithPrivateKey(nodeKey), + node.WithHostAddress(hostAddr), + node.WithConnectionNotification(waku.connectionNotifChan), + node.WithTopicHealthStatusChannel(waku.topicHealthStatusChan), + node.WithKeepAlive(randomPeersKeepAliveInterval, allPeersKeepAliveInterval), + node.WithLogger(logger), + node.WithLogLevel(logger.Level()), + node.WithClusterID(cfg.ClusterID), + node.WithMaxMsgSize(1024 * 1024), + } + + if cfg.EnableDiscV5 { + bootnodes, err := waku.getDiscV5BootstrapNodes(waku.ctx, cfg.DiscV5BootstrapNodes, false) + if err != nil { + logger.Error("failed to get bootstrap nodes", zap.Error(err)) + return nil, err + } + opts = append(opts, node.WithDiscoveryV5(uint(cfg.UDPPort), bootnodes, cfg.AutoUpdate)) + } + shards, err := protocol.TopicsToRelayShards(cfg.DefaultShardPubsubTopic) + if err != nil { + logger.Error("FATAL ERROR: failed to parse relay shards", zap.Error(err)) + return nil, errors.New("failed to parse relay shard, invalid pubsubTopic configuration") + } + if len(shards) == 0 { //Hack so that tests don't fail. TODO: Need to remove this once tests are changed to use proper cluster and shard. + shardInfo := protocol.RelayShards{ClusterID: 0, ShardIDs: []uint16{0}} + shards = append(shards, shardInfo) + } + waku.defaultShardInfo = shards[0] + if cfg.LightClient { + opts = append(opts, node.WithWakuFilterLightNode()) + waku.defaultShardInfo = shards[0] + opts = append(opts, node.WithMaxPeerConnections(cfg.DiscoveryLimit)) + cfg.EnableStoreConfirmationForMessagesSent = false + //TODO: temporary work-around to improve lightClient connectivity, need to be removed once community sharding is implemented + opts = append(opts, node.WithShards(waku.defaultShardInfo.ShardIDs)) + } else { + relayOpts := []pubsub.Option{ + pubsub.WithMaxMessageSize(int(waku.cfg.MaxMessageSize)), + } + + if testing.Testing() { + relayOpts = append(relayOpts, pubsub.WithEventTracer(waku)) + } + + opts = append(opts, node.WithWakuRelayAndMinPeers(waku.cfg.MinPeersForRelay, relayOpts...)) + opts = append(opts, node.WithMaxPeerConnections(maxRelayPeers)) + cfg.EnablePeerExchangeClient = true //Enabling this until discv5 issues are resolved. This will enable more peers to be connected for relay mesh. + cfg.EnableStoreConfirmationForMessagesSent = true + } + + if cfg.EnableStore { + if appDB == nil { + return nil, errors.New("appDB is required for store") + } + opts = append(opts, node.WithWakuStore()) + dbStore, err := persistence.NewDBStore(logger, persistence.WithDB(appDB), persistence.WithRetentionPolicy(cfg.StoreCapacity, time.Duration(cfg.StoreSeconds)*time.Second)) + if err != nil { + return nil, err + } + opts = append(opts, node.WithMessageProvider(dbStore)) + } + + if !cfg.LightClient { + opts = append(opts, node.WithWakuFilterFullNode(filter.WithMaxSubscribers(20))) + opts = append(opts, node.WithLightPush(lightpush.WithRateLimiter(1, 1))) + } + + if appDB != nil { + waku.protectedTopicStore, err = persistence.NewProtectedTopicsStore(logger, appDB) + if err != nil { + return nil, err + } + } + + if cfg.EnablePeerExchangeServer { + opts = append(opts, node.WithPeerExchange(peer_exchange.WithRateLimiter(1, 1))) + } + + waku.options = opts + + waku.logger.Info("setup the go-waku node successfully") + + return waku, nil*/ +} + func (w *Waku) SubscribeToConnStatusChanges() *types.ConnStatusSubscription { w.connStatusMu.Lock() defer w.connStatusMu.Unlock() @@ -369,8 +661,8 @@ func (w *Waku) SubscribeToConnStatusChanges() *types.ConnStatusSubscription { return subscription } -/* TODO-nwaku -func (w *Waku) getDiscV5BootstrapNodes(ctx context.Context, addresses []string) ([]*enode.Node, error) { +/* TODO-nwaku +func (w *Waku) getDiscV5BootstrapNodes(ctx context.Context, addresses []string, useOnlyDnsDiscCache bool) ([]*enode.Node, error) { wg := sync.WaitGroup{} mu := sync.Mutex{} var result []*enode.Node @@ -395,11 +687,13 @@ func (w *Waku) getDiscV5BootstrapNodes(ctx context.Context, addresses []string) // Use DNS Discovery wg.Add(1) go func(addr string) { + defer gocommon.LogOnPanic() defer wg.Done() - if err := w.dnsDiscover(ctx, addr, retrieveENR); err != nil { - mu.Lock() - w.seededBootnodesForDiscV5 = false - mu.Unlock() + if err := w.dnsDiscover(ctx, addr, retrieveENR, useOnlyDnsDiscCache); err != nil { + go func() { + defer gocommon.LogOnPanic() + w.retryDnsDiscoveryWithBackoff(ctx, addr, w.dnsDiscAsyncRetrievedSignal) + }() } }(addrString) } else { @@ -408,17 +702,23 @@ func (w *Waku) getDiscV5BootstrapNodes(ctx context.Context, addresses []string) if err != nil { return nil, err } + mu.Lock() result = append(result, bootnode) + mu.Unlock() } } wg.Wait() + if len(result) == 0 { + w.seededBootnodesForDiscV5 = false + } + return result, nil } type fnApplyToEachPeer func(d dnsdisc.DiscoveredNode, wg *sync.WaitGroup) -func (w *Waku) dnsDiscover(ctx context.Context, enrtreeAddress string, apply fnApplyToEachPeer) error { +func (w *Waku) dnsDiscover(ctx context.Context, enrtreeAddress string, apply fnApplyToEachPeer, useOnlyCache bool) error { w.logger.Info("retrieving nodes", zap.String("enr", enrtreeAddress)) ctx, cancel := context.WithTimeout(ctx, requestTimeout) defer cancel() @@ -427,7 +727,7 @@ func (w *Waku) dnsDiscover(ctx context.Context, enrtreeAddress string, apply fnA defer w.dnsAddressCacheLock.Unlock() discNodes, ok := w.dnsAddressCache[enrtreeAddress] - if !ok { + if !ok && !useOnlyCache { nameserver := w.cfg.Nameserver resolver := w.cfg.Resolver @@ -461,6 +761,36 @@ func (w *Waku) dnsDiscover(ctx context.Context, enrtreeAddress string, apply fnA return nil } +func (w *Waku) retryDnsDiscoveryWithBackoff(ctx context.Context, addr string, successChan chan<- struct{}) { + retries := 0 + for { + err := w.dnsDiscover(ctx, addr, func(d dnsdisc.DiscoveredNode, wg *sync.WaitGroup) {}, false) + if err == nil { + select { + case successChan <- struct{}{}: + default: + } + + break + } + + retries++ + backoff := time.Second * time.Duration(math.Exp2(float64(retries))) + if backoff > time.Minute { + backoff = time.Minute + } + + t := time.NewTimer(backoff) + select { + case <-w.ctx.Done(): + t.Stop() + return + case <-t.C: + t.Stop() + } + } +} + func (w *Waku) discoverAndConnectPeers() { fnApply := func(d dnsdisc.DiscoveredNode, wg *sync.WaitGroup) { defer wg.Done() @@ -474,7 +804,8 @@ func (w *Waku) discoverAndConnectPeers() { if strings.HasPrefix(addrString, "enrtree://") { // Use DNS Discovery go func() { - if err := w.dnsDiscover(w.ctx, addrString, fnApply); err != nil { + defer gocommon.LogOnPanic() + if err := w.dnsDiscover(w.ctx, addrString, fnApply, false); err != nil { w.logger.Error("could not obtain dns discovery peers for ClusterConfig.WakuNodes", zap.Error(err), zap.String("dnsDiscURL", addrString)) } }() @@ -498,15 +829,16 @@ func (w *Waku) discoverAndConnectPeers() { } */ func (w *Waku) connect(peerInfo peer.AddrInfo, enr *enode.Node, origin wps.Origin) { + defer gocommon.LogOnPanic() // Connection will be prunned eventually by the connection manager if needed // The peer connector in go-waku uses Connect, so it will execute identify as part of its addr := peerInfo.Addrs[0] w.WakuConnect(addr.String(), 1000) } -/* TODO-nwaku +/* TODO-nwaku func (w *Waku) telemetryBandwidthStats(telemetryServerURL string) { - w.wg.Add(1) + defer gocommon.LogOnPanic() defer w.wg.Done() if telemetryServerURL == "" { @@ -518,20 +850,14 @@ func (w *Waku) telemetryBandwidthStats(telemetryServerURL string) { ticker := time.NewTicker(time.Second * 20) defer ticker.Stop() - today := time.Now() - for { select { case <-w.ctx.Done(): return - case now := <-ticker.C: - // Reset totals when day changes - if now.Day() != today.Day() { - today = now - w.bandwidthCounter.Reset() - } - - go telemetry.PushProtocolStats(w.bandwidthCounter.GetBandwidthByProtocol()) + case <-ticker.C: + bandwidthPerProtocol := w.bandwidthCounter.GetBandwidthByProtocol() + w.bandwidthCounter.Reset() + go telemetry.PushProtocolStats(bandwidthPerProtocol) } } } @@ -545,7 +871,7 @@ func (w *Waku) GetStats() types.StatsSummary { } func (w *Waku) runPeerExchangeLoop() { - w.wg.Add(1) + defer gocommon.LogOnPanic() defer w.wg.Done() if !w.cfg.EnablePeerExchangeClient { @@ -583,12 +909,11 @@ func (w *Waku) runPeerExchangeLoop() { w.dnsAddressCacheLock.RUnlock() if len(peers) != 0 { - // TODO - // err := w.node.PeerExchange().Request(w.ctx, w.cfg.DiscoveryLimit, peer_exchange.WithAutomaticPeerSelection(peers...), - // peer_exchange.FilterByShard(int(w.defaultShardInfo.ClusterID), int(w.defaultShardInfo.ShardIDs[0]))) - // if err != nil { - // w.logger.Error("couldnt request peers via peer exchange", zap.Error(err)) - // } + err := w.node.PeerExchange().Request(w.ctx, w.cfg.DiscoveryLimit, peer_exchange.WithAutomaticPeerSelection(peers...), + peer_exchange.FilterByShard(int(w.defaultShardInfo.ClusterID), int(w.defaultShardInfo.ShardIDs[0]))) + if err != nil { + w.logger.Error("couldnt request peers via peer exchange", zap.Error(err)) + } } } } @@ -602,6 +927,72 @@ func (w *Waku) GetPubsubTopic(topic string) string { return topic } +func (w *Waku) unsubscribeFromPubsubTopicWithWakuRelay(topic string) error { + topic = w.GetPubsubTopic(topic) + + if !w.node.Relay().IsSubscribed(topic) { + return nil + } + + contentFilter := protocol.NewContentFilter(topic) + + return w.node.Relay().Unsubscribe(w.ctx, contentFilter) +} + +func (w *Waku) subscribeToPubsubTopicWithWakuRelay(topic string, pubkey *ecdsa.PublicKey) error { + if w.cfg.LightClient { + return errors.New("only available for full nodes") + } + + topic = w.GetPubsubTopic(topic) + + if w.node.Relay().IsSubscribed(topic) { + return nil + } + + if pubkey != nil { + err := w.node.Relay().AddSignedTopicValidator(topic, pubkey) + if err != nil { + return err + } + } + + contentFilter := protocol.NewContentFilter(topic) + + sub, err := w.node.Relay().Subscribe(w.ctx, contentFilter) + if err != nil { + return err + } + + w.wg.Add(1) + go func() { + defer gocommon.LogOnPanic() + defer w.wg.Done() + for { + select { + case <-w.ctx.Done(): + err := w.node.Relay().Unsubscribe(w.ctx, contentFilter) + if err != nil && !errors.Is(err, context.Canceled) { + w.logger.Error("could not unsubscribe", zap.Error(err)) + } + return + case env := <-sub[0].Ch: + err := w.OnNewEnvelopes(env, common.RelayedMessageType, false) + if err != nil { + w.logger.Error("OnNewEnvelopes error", zap.Error(err)) + } + } + } + }() + + return nil +} + +// MaxMessageSize returns the maximum accepted message size. +func (w *Waku) MaxMessageSize() uint32 { + return w.cfg.MaxMessageSize +} + // CurrentTime returns current time. func (w *Waku) CurrentTime() time.Time { return w.timesource.Now() @@ -870,7 +1261,7 @@ func (w *Waku) GetSymKey(id string) ([]byte, error) { return nil, fmt.Errorf("non-existent key ID") } -/* TODO-nwaku +/* TODO-nwaku // Subscribe installs a new message handler used for filtering, decrypting // and subsequent storing of incoming messages. func (w *Waku) Subscribe(f *common.Filter) (string, error) { @@ -924,288 +1315,320 @@ func (w *Waku) SkipPublishToTopic(value bool) { } func (w *Waku) ConfirmMessageDelivered(hashes []gethcommon.Hash) { - if !w.cfg.EnableStoreConfirmationForMessagesSent { - return - } - w.messageSentCheck.DeleteByMessageIDs(hashes) + w.messageSender.MessagesDelivered(hashes) } */ -func (w *Waku) SetStorePeerID(peerID peer.ID) { - if w.messageSentCheck != nil { - w.messageSentCheck.SetStorePeerID(peerID) - } -} - -func (w *Waku) Query(ctx context.Context, - peerID peer.ID, - query store.FilterCriteria, - cursor []byte, - opts []store.RequestOption, - processEnvelopes bool) ([]byte, int, error) { - requestID := protocol.GenerateRequestID() - - opts = append(opts, - store.WithRequestID(requestID), - store.WithPeer(peerID), - store.WithCursor(cursor)) - - logger := w.logger.With(zap.String("requestID", hexutil.Encode(requestID)), zap.Stringer("peerID", peerID)) - - logger.Debug("store.query", - logutils.WakuMessageTimestamp("startTime", query.TimeStart), - logutils.WakuMessageTimestamp("endTime", query.TimeEnd), - zap.Strings("contentTopics", query.ContentTopics.ToList()), - zap.String("pubsubTopic", query.PubsubTopic), - zap.String("cursor", hexutil.Encode(cursor)), - ) - - // queryStart := time.Now() +/* TODO-nwaku +// OnNewEnvelope is an interface from Waku FilterManager API that gets invoked when any new message is received by Filter. +func (w *Waku) OnNewEnvelope(env *protocol.Envelope) error { + return w.OnNewEnvelopes(env, common.RelayedMessageType, false) +} */ - params := new(store.Parameters) +// Start implements node.Service, starting the background data propagation thread +// of the Waku protocol. +func (w *Waku) Start() error { + err := w.WakuStart() + if err != nil { + fmt.Println("Error happened:", err.Error()) + return err + } - optList := store.DefaultOptions() - optList = append(optList, opts...) - for _, opt := range optList { - err := opt(params) - if err != nil { - return nil, 0, err - } + /* TODO-nwaku + if w.ctx == nil { + w.ctx, w.cancel = context.WithCancel(context.Background()) } - storeRequest := &storepb.StoreQueryRequest{ - RequestId: hex.EncodeToString(requestID), - IncludeData: params.IncludeData, - PaginationForward: params.Forward, - PaginationLimit: ¶ms.PageLimit, + var err error + if w.node, err = node.New(w.options...); err != nil { + return fmt.Errorf("failed to create a go-waku node: %v", err) } - jsonStoreRequest, err := json.Marshal(storeRequest) - if err != nil { - return nil, 0, err + w.goingOnline = make(chan struct{}) + + if err = w.node.Start(w.ctx); err != nil { + return fmt.Errorf("failed to start go-waku node: %v", err) } - result, err := w.wakuStoreQuery(string(jsonStoreRequest), string(peerID), 10000) + w.StorenodeCycle = history.NewStorenodeCycle(w.logger) + w.HistoryRetriever = history.NewHistoryRetriever(w.node.Store(), NewHistoryProcessorWrapper(w), w.logger) - fmt.Println("Store result ", result) + w.StorenodeCycle.Start(w.ctx, w.node.Host()) - // result, err := w.node.Store().Query(ctx, query, opts...) - // queryDuration := time.Since(queryStart) - // if err != nil { - // logger.Error("error querying storenode", zap.Error(err)) + w.logger.Info("WakuV2 PeerID", zap.Stringer("id", w.node.Host().ID())) - // if w.onHistoricMessagesRequestFailed != nil { - // w.onHistoricMessagesRequestFailed(requestID, peerID, err) - // } - // return nil, 0, err - // } + w.discoverAndConnectPeers() - // messages := result.Messages() - // envelopesCount := len(messages) - // w.logger.Debug("store.query response", zap.Duration("queryDuration", queryDuration), zap.Int("numMessages", envelopesCount), zap.Bool("hasCursor", result.IsComplete() && result.Cursor() != nil)) - // for _, mkv := range messages { - // msg := mkv.Message + if w.cfg.EnableDiscV5 { + err := w.node.DiscV5().Start(w.ctx) + if err != nil { + return err + } + } - // // Temporarily setting RateLimitProof to nil so it matches the WakuMessage protobuffer we are sending - // // See https://github.com/vacp2p/rfc/issues/563 - // mkv.Message.RateLimitProof = nil + w.wg.Add(1) + go func() { + defer gocommon.LogOnPanic() + defer w.wg.Done() + ticker := time.NewTicker(5 * time.Second) + defer ticker.Stop() + for { + select { + case <-w.ctx.Done(): + return + case <-ticker.C: + w.checkForConnectionChanges() + case <-w.topicHealthStatusChan: + // TODO: https://github.com/status-im/status-go/issues/4628 + case <-w.connectionNotifChan: + w.checkForConnectionChanges() + } + } + }() + + if w.cfg.TelemetryServerURL != "" { + w.wg.Add(1) + go func() { + defer gocommon.LogOnPanic() + defer w.wg.Done() + peerTelemetryTickerInterval := time.Duration(w.cfg.TelemetryPeerCountSendPeriod) * time.Millisecond + if peerTelemetryTickerInterval == 0 { + peerTelemetryTickerInterval = 10 * time.Second + } + peerTelemetryTicker := time.NewTicker(peerTelemetryTickerInterval) + defer peerTelemetryTicker.Stop() + + for { + select { + case <-w.ctx.Done(): + return + case <-peerTelemetryTicker.C: + w.reportPeerMetrics() + } + } + }() + } - // envelope := protocol.NewEnvelope(msg, msg.GetTimestamp(), query.PubsubTopic) + w.wg.Add(1) + go w.telemetryBandwidthStats(w.cfg.TelemetryServerURL) + //TODO: commenting for now so that only fleet nodes are used. + //Need to uncomment once filter peer scoring etc is implemented. - // err = w.OnNewEnvelopes(envelope, common.StoreMessageType, processEnvelopes) - // if err != nil { - // return nil, 0, err - // } - // } + w.wg.Add(1) + go w.runPeerExchangeLoop() + + if w.cfg.EnableMissingMessageVerification { + + w.missingMsgVerifier = missing.NewMissingMessageVerifier( + missing.NewDefaultStorenodeRequestor(w.node.Store()), + w, + w.node.Timesource(), + w.logger) + + w.missingMsgVerifier.Start(w.ctx) + + w.wg.Add(1) + go func() { + defer gocommon.LogOnPanic() + w.wg.Done() + for { + select { + case <-w.ctx.Done(): + return + case envelope := <-w.missingMsgVerifier.C: + err = w.OnNewEnvelopes(envelope, common.MissingMessageType, false) + if err != nil { + w.logger.Error("OnNewEnvelopes error", zap.Error(err)) + } + } + } + }() + } - // return result.Cursor(), envelopesCount, nil - return nil, 0, nil -} + if w.cfg.LightClient { + // Create FilterManager that will main peer connectivity + // for installed filters + w.filterManager = filterapi.NewFilterManager( + w.ctx, + w.logger, + w.cfg.MinPeersForFilter, + w, + w.node.FilterLightnode(), + filterapi.WithBatchInterval(300*time.Millisecond)) + } + + err = w.setupRelaySubscriptions() + if err != nil { + return err + } -/* TODO-nwaku -// OnNewEnvelope is an interface from Waku FilterManager API that gets invoked when any new message is received by Filter. -func (w *Waku) OnNewEnvelope(env *protocol.Envelope) error { - return w.OnNewEnvelopes(env, common.RelayedMessageType, false) -} */ + numCPU := runtime.NumCPU() + for i := 0; i < numCPU; i++ { + w.wg.Add(1) + go w.processQueueLoop() + } + */ -// Start implements node.Service, starting the background data propagation thread -// of the Waku protocol. -func (w *Waku) Start() error { - // if w.ctx == nil { - // w.ctx, w.cancel = context.WithCancel(context.Background()) - // } + // w.wg.Add(1) - // var err error - // if w.node, err = node.New(w.options...); err != nil { - // return fmt.Errorf("failed to create a go-waku node: %v", err) - // } + // TODO-nwaku + // go w.broadcast() - // w.goingOnline = make(chan struct{}) + // go w.sendQueue.Start(w.ctx) - err := w.WakuStart() + /* TODO-nwaku + err = w.startMessageSender() if err != nil { - fmt.Println("Error happened:", err.Error()) return err } + */ - // if err = w.node.Start(w.ctx); err != nil { - // return fmt.Errorf("failed to start go-waku node: %v", err) - // } + /* TODO-nwaku + // we should wait `seedBootnodesForDiscV5` shutdown smoothly before set w.ctx to nil within `w.Stop()` + w.wg.Add(1) + go w.seedBootnodesForDiscV5() + */ - // w.logger.Info("WakuV2 PeerID", zap.Stringer("id", w.node.Host().ID())) + return nil +} - // w.discoverAndConnectPeers() +func (w *Waku) checkForConnectionChanges() { - // if w.cfg.EnableDiscV5 { - // err := w.node.DiscV5().Start(w.ctx) - // if err != nil { - // return err - // } - // } + /* TODO-nwaku + isOnline := len(w.node.Host().Network().Peers()) > 0 - // w.wg.Add(1) - // go func() { - // defer w.wg.Done() - // ticker := time.NewTicker(5 * time.Second) - // defer ticker.Stop() - // for { - // select { - // case <-w.ctx.Done(): - // return - // case <-ticker.C: - // w.checkForConnectionChanges() - // case <-w.topicHealthStatusChan: - // // TODO: https://github.com/status-im/status-go/issues/4628 - // case <-w.connectionNotifChan: - // w.checkForConnectionChanges() - // } - // } - // }() - - // go w.telemetryBandwidthStats(w.cfg.TelemetryServerURL) - //TODO: commenting for now so that only fleet nodes are used. - //Need to uncomment once filter peer scoring etc is implemented. - // go w.runPeerExchangeLoop() - - // if w.cfg.EnableMissingMessageVerification { - - // w.missingMsgVerifier = missing.NewMissingMessageVerifier( - // w.node.Store(), - // w, - // w.node.Timesource(), - // w.logger) - - // w.missingMsgVerifier.Start(w.ctx) - - // w.wg.Add(1) - // go func() { - // w.wg.Done() - // for { - // select { - // case <-w.ctx.Done(): - // return - // case envelope := <-w.missingMsgVerifier.C: - // err = w.OnNewEnvelopes(envelope, common.MissingMessageType, false) - // if err != nil { - // w.logger.Error("OnNewEnvelopes error", zap.Error(err)) - // } - // } - // } - // }() - // } + w.connStatusMu.Lock() - // if w.cfg.LightClient { - // // Create FilterManager that will main peer connectivity - // // for installed filters - // w.filterManager = filterapi.NewFilterManager(w.ctx, w.logger, w.cfg.MinPeersForFilter, - // w, - // w.node.FilterLightnode()) - // } + latestConnStatus := types.ConnStatus{ + IsOnline: isOnline, + Peers: FormatPeerStats(w.node), + } - // err = w.setupRelaySubscriptions() - // if err != nil { - // return err - // } + w.logger.Debug("peer stats", + zap.Int("peersCount", len(latestConnStatus.Peers)), + zap.Any("stats", latestConnStatus)) + for k, subs := range w.connStatusSubscriptions { + if !subs.Send(latestConnStatus) { + delete(w.connStatusSubscriptions, k) + } + } - // numCPU := runtime.NumCPU() - // for i := 0; i < numCPU; i++ { - // go w.processQueueLoop() - // } + w.connStatusMu.Unlock() - // go w.broadcast() + if w.onPeerStats != nil { + w.onPeerStats(latestConnStatus) + } - // go w.sendQueue.Start(w.ctx) + w.ConnectionChanged(connection.State{ + Type: w.state.Type, //setting state type as previous one since there won't be a change here + Offline: !latestConnStatus.IsOnline, + }) */ +} - // if w.cfg.EnableStoreConfirmationForMessagesSent { - // w.confirmMessagesSent() - // } +/* TODO: nwaku +func (w *Waku) reportPeerMetrics() { + if w.statusTelemetryClient != nil { + connFailures := FormatPeerConnFailures(w.node) + w.statusTelemetryClient.PushPeerCount(w.ctx, w.PeerCount()) + w.statusTelemetryClient.PushPeerConnFailures(w.ctx, connFailures) - // we should wait `seedBootnodesForDiscV5` shutdown smoothly before set w.ctx to nil within `w.Stop()` - // go w.seedBootnodesForDiscV5() + peerCountByOrigin := make(map[wps.Origin]uint) + peerCountByShard := make(map[uint16]uint) + wakuPeerStore := w.node.Host().Peerstore().(wps.WakuPeerstore) - return nil -} + for _, peerID := range w.node.Host().Network().Peers() { + origin, err := wakuPeerStore.Origin(peerID) + if err != nil { + origin = wps.Unknown + } -func (w *Waku) checkForConnectionChanges() { + peerCountByOrigin[origin]++ + pubsubTopics, err := wakuPeerStore.PubSubTopics(peerID) + if err != nil { + continue + } - // isOnline := len(w.node.Host().Network().Peers()) > 0 + keys := make([]string, 0, len(pubsubTopics)) + for k := range pubsubTopics { + keys = append(keys, k) + } + relayShards, err := protocol.TopicsToRelayShards(keys...) + if err != nil { + continue + } - // w.connStatusMu.Lock() + for _, shards := range relayShards { + for _, shard := range shards.ShardIDs { + peerCountByShard[shard]++ + } + } + } + w.statusTelemetryClient.PushPeerCountByShard(w.ctx, peerCountByShard) + w.statusTelemetryClient.PushPeerCountByOrigin(w.ctx, peerCountByOrigin) + } +} +*/ - // latestConnStatus := types.ConnStatus{ - // IsOnline: isOnline, - // Peers: FormatPeerStats(w.node), - // } +/* TODO-nwaku +func (w *Waku) startMessageSender() error { + publishMethod := publish.Relay + if w.cfg.LightClient { + publishMethod = publish.LightPush + } - // w.logger.Debug("peer stats", - // zap.Int("peersCount", len(latestConnStatus.Peers)), - // zap.Any("stats", latestConnStatus)) - // for k, subs := range w.connStatusSubscriptions { - // if !subs.Send(latestConnStatus) { - // delete(w.connStatusSubscriptions, k) - // } - // } + sender, err := publish.NewMessageSender(publishMethod, publish.NewDefaultPublisher(w.node.Lightpush(), w.node.Relay()), w.logger) + if err != nil { + w.logger.Error("failed to create message sender", zap.Error(err)) + return err + } - // w.connStatusMu.Unlock() + if w.cfg.EnableStoreConfirmationForMessagesSent { + msgStoredChan := make(chan gethcommon.Hash, 1000) + msgExpiredChan := make(chan gethcommon.Hash, 1000) + messageSentCheck := publish.NewMessageSentCheck(w.ctx, publish.NewDefaultStorenodeMessageVerifier(w.node.Store()), w.StorenodeCycle, w.node.Timesource(), msgStoredChan, msgExpiredChan, w.logger) + sender.WithMessageSentCheck(messageSentCheck) + + w.wg.Add(1) + go func() { + defer gocommon.LogOnPanic() + defer w.wg.Done() + for { + select { + case <-w.ctx.Done(): + return + case hash := <-msgStoredChan: + w.SendEnvelopeEvent(common.EnvelopeEvent{ + Hash: hash, + Event: common.EventEnvelopeSent, + }) + if w.statusTelemetryClient != nil { + w.statusTelemetryClient.PushMessageCheckSuccess(w.ctx, hash.Hex()) + } + case hash := <-msgExpiredChan: + w.SendEnvelopeEvent(common.EnvelopeEvent{ + Hash: hash, + Event: common.EventEnvelopeExpired, + }) + if w.statusTelemetryClient != nil { + w.statusTelemetryClient.PushMessageCheckFailure(w.ctx, hash.Hex()) + } + } + } + }() + } - // if w.onPeerStats != nil { - // w.onPeerStats(latestConnStatus) - // } + if !w.cfg.UseThrottledPublish || testing.Testing() { + // To avoid delaying the tests, or for when we dont want to rate limit, we set up an infinite rate limiter, + // basically disabling the rate limit functionality + limiter := publish.NewPublishRateLimiter(rate.Inf, 1) + sender.WithRateLimiting(limiter) + } - // if w.statusTelemetryClient != nil { - // connFailures := FormatPeerConnFailures(w.node) - // w.statusTelemetryClient.PushPeerCount(w.ctx, w.PeerCount()) - // w.statusTelemetryClient.PushPeerConnFailures(w.ctx, connFailures) - // } + w.messageSender = sender + w.messageSender.Start() - // w.ConnectionChanged(connection.State{ - // Type: w.state.Type, //setting state type as previous one since there won't be a change here - // Offline: !latestConnStatus.IsOnline, - // }) -} - -// func (w *Waku) confirmMessagesSent() { -// w.messageSentCheck = publish.NewMessageSentCheck(w.ctx, w.node.Store(), w.node.Timesource(), w.logger) -// go w.messageSentCheck.Start() - -// go func() { -// for { -// select { -// case <-w.ctx.Done(): -// return -// case hash := <-w.messageSentCheck.MessageStoredChan: -// w.SendEnvelopeEvent(common.EnvelopeEvent{ -// Hash: hash, -// Event: common.EventEnvelopeSent, -// }) -// case hash := <-w.messageSentCheck.MessageExpiredChan: -// w.SendEnvelopeEvent(common.EnvelopeEvent{ -// Hash: hash, -// Event: common.EventEnvelopeExpired, -// }) -// } -// } -// }() -// } + return nil +} +*/ func (w *Waku) MessageExists(mh pb.MessageHash) (bool, error) { w.poolMu.Lock() @@ -1213,7 +1636,7 @@ func (w *Waku) MessageExists(mh pb.MessageHash) (bool, error) { return w.envelopeCache.Has(gethcommon.Hash(mh)), nil } -/* TODO-nwaku +/* TODO-nwaku func (w *Waku) SetTopicsToVerifyForMissingMessages(peerID peer.ID, pubsubTopic string, contentTopics []string) { if !w.cfg.EnableMissingMessageVerification { return @@ -1222,41 +1645,65 @@ func (w *Waku) SetTopicsToVerifyForMissingMessages(peerID peer.ID, pubsubTopic s w.missingMsgVerifier.SetCriteriaInterest(peerID, protocol.NewContentFilter(pubsubTopic, contentTopics...)) } -func (w *Waku) setupRelaySubscriptions() error { - if w.cfg.LightClient { - return nil +func (w *Waku) setupRelaySubscriptions() error { + if w.cfg.LightClient { + return nil + } + + if w.protectedTopicStore != nil { + protectedTopics, err := w.protectedTopicStore.ProtectedTopics() + if err != nil { + return err + } + + for _, pt := range protectedTopics { + // Adding subscription to protected topics + err = w.subscribeToPubsubTopicWithWakuRelay(pt.Topic, pt.PubKey) + if err != nil { + return err + } + } + } + + err := w.subscribeToPubsubTopicWithWakuRelay(w.cfg.DefaultShardPubsubTopic, nil) + if err != nil { + return err + } + + return nil +} */ + +// Stop implements node.Service, stopping the background data propagation thread +// of the Waku protocol. +func (w *Waku) Stop() error { + w.cancel() + + w.envelopeCache.Stop() + + err := w.WakuStop() + if err != nil { + return err } + /* TODO-nwaku if w.protectedTopicStore != nil { - protectedTopics, err := w.protectedTopicStore.ProtectedTopics() + err := w.protectedTopicStore.Close() if err != nil { return err } + } - for _, pt := range protectedTopics { - // Adding subscription to protected topics - // err = w.subscribeToPubsubTopicWithWakuRelay(pt.Topic, pt.PubKey) - // if err != nil { - // return err - // } + close(w.goingOnline)*/ - fmt.Println("Subscription to topic: ", pt.Topic) - err = w.WakuRelaySubscribe(pt.Topic) - if err != nil { - return err - } - } - } + w.wg.Wait() - err := w.WakuRelaySubscribe(w.cfg.DefaultShardPubsubTopic) - if err != nil { - return err - } + w.ctx = nil + w.cancel = nil return nil -} */ +} -/* TODO-nwaku +/* TODO-nwaku func (w *Waku) OnNewEnvelopes(envelope *protocol.Envelope, msgType common.MessageType, processImmediately bool) error { if envelope == nil { return nil @@ -1267,10 +1714,6 @@ func (w *Waku) OnNewEnvelopes(envelope *protocol.Envelope, msgType common.Messag return nil } - if w.statusTelemetryClient != nil { - w.statusTelemetryClient.PushReceivedEnvelope(w.ctx, envelope) - } - logger := w.logger.With( zap.String("messageType", msgType), zap.Stringer("envelopeHash", envelope.Hash()), @@ -1296,6 +1739,7 @@ func (w *Waku) OnNewEnvelopes(envelope *protocol.Envelope, msgType common.Messag return nil } +*/ // addEnvelope adds an envelope to the envelope map, used for sending func (w *Waku) addEnvelope(envelope *common.ReceivedMessage) { @@ -1339,7 +1783,7 @@ func (w *Waku) add(recvMessage *common.ReceivedMessage, processImmediately bool) } return true, nil -} */ +} // postEvent queues the message for further processing. func (w *Waku) postEvent(envelope *common.ReceivedMessage) { @@ -1349,6 +1793,8 @@ func (w *Waku) postEvent(envelope *common.ReceivedMessage) { /* TODO-nwaku // processQueueLoop delivers the messages to the watchers during the lifetime of the waku node. func (w *Waku) processQueueLoop() { + defer gocommon.LogOnPanic() + defer w.wg.Done() if w.ctx == nil { return } @@ -1360,7 +1806,7 @@ func (w *Waku) processQueueLoop() { w.processMessage(e) } } -} +}*/ func (w *Waku) processMessage(e *common.ReceivedMessage) { logger := w.logger.With( @@ -1378,11 +1824,6 @@ func (w *Waku) processMessage(e *common.ReceivedMessage) { w.storeMsgIDsMu.Unlock() } - ephemeral := e.Envelope.Message().Ephemeral - if w.cfg.EnableStoreConfirmationForMessagesSent && e.MsgType == common.SendMessageType && (ephemeral == nil || !*ephemeral) { - w.messageSentCheck.Add(e.PubsubTopic, e.Hash(), e.Sent) - } - matched := w.filters.NotifyWatchers(e) // If not matched we remove it @@ -1401,7 +1842,7 @@ func (w *Waku) processMessage(e *common.ReceivedMessage) { Hash: e.Hash(), Event: common.EventEnvelopeAvailable, }) -} */ +} // GetEnvelope retrieves an envelope from the message queue by its hash. // It returns nil if the envelope can not be found. @@ -1433,35 +1874,35 @@ func (w *Waku) ClearEnvelopesCache() { w.envelopeCache = newTTLCache() } +// TODO-nwaku func (w *Waku) PeerCount() int { return 0 // return w.node.PeerCount() } +// TODO-nwaku func (w *Waku) Peers() types.PeerStats { return nil // return FormatPeerStats(w.node) } -/* TODO-nwaku +/* TODO-nwaku func (w *Waku) RelayPeersByTopic(topic string) (*types.PeerList, error) { if w.cfg.LightClient { return nil, errors.New("only available in relay mode") } - // return &types.PeerList{ - // FullMeshPeers: w.node.Relay().PubSub().MeshPeers(topic), - // AllPeers: w.node.Relay().PubSub().ListPeers(topic), - // }, nil - return nil, nil + return &types.PeerList{ + FullMeshPeers: w.node.Relay().PubSub().MeshPeers(topic), + AllPeers: w.node.Relay().PubSub().ListPeers(topic), + }, nil } func (w *Waku) SubscribeToPubsubTopic(topic string, pubkey *ecdsa.PublicKey) error { topic = w.GetPubsubTopic(topic) if !w.cfg.LightClient { - err := w.WakuRelaySubscribe(topic) - // err := w.subscribeToPubsubTopicWithWakuRelay(topic, pubkey) + err := w.subscribeToPubsubTopicWithWakuRelay(topic, pubkey) if err != nil { return err } @@ -1473,7 +1914,7 @@ func (w *Waku) UnsubscribeFromPubsubTopic(topic string) error { topic = w.GetPubsubTopic(topic) if !w.cfg.LightClient { - err := w.WakuRelayUnsubscribe(topic) + err := w.unsubscribeFromPubsubTopicWithWakuRelay(topic) if err != nil { return err } @@ -1509,19 +1950,22 @@ func (w *Waku) RemovePubsubTopicKey(topic string) error { } */ func (w *Waku) handleNetworkChangeFromApp(state connection.State) { - //If connection state is reported by something other than peerCount becoming 0 e.g from mobile app, disconnect all peers - // if (state.Offline && len(w.node.Host().Network().Peers()) > 0) || - // (w.state.Type != state.Type && !w.state.Offline && !state.Offline) { // network switched between wifi and cellular - // w.logger.Info("connection switched or offline detected via mobile, disconnecting all peers") - // w.node.DisconnectAllPeers() - // if w.cfg.LightClient { - // w.filterManager.NetworkChange() - // } - // } + // TODO-nwaku + /* + //If connection state is reported by something other than peerCount becoming 0 e.g from mobile app, disconnect all peers + if (state.Offline && len(w.node.Host().Network().Peers()) > 0) || + (w.state.Type != state.Type && !w.state.Offline && !state.Offline) { // network switched between wifi and cellular + w.logger.Info("connection switched or offline detected via mobile, disconnecting all peers") + w.node.DisconnectAllPeers() + if w.cfg.LightClient { + w.filterManager.NetworkChange() + } + } + */ } -/* TODO-nwaku func (w *Waku) ConnectionChanged(state connection.State) { + /* TODO-nwaku isOnline := !state.Offline if w.cfg.LightClient { //TODO: Update this as per https://github.com/waku-org/go-waku/issues/1114 @@ -1543,14 +1987,143 @@ func (w *Waku) ConnectionChanged(state connection.State) { w.onlineChecker.SetOnline(isOnline) } w.state = state -} */ + */ +} + +/* TODO-nwaku +// seedBootnodesForDiscV5 tries to fetch bootnodes +// from an ENR periodically. +// It backs off exponentially until maxRetries, at which point it restarts from 0 +// It also restarts if there's a connection change signalled from the client +func (w *Waku) seedBootnodesForDiscV5() { + defer gocommon.LogOnPanic() + defer w.wg.Done() + + if !w.cfg.EnableDiscV5 || w.node.DiscV5() == nil { + return + } + + ticker := time.NewTicker(500 * time.Millisecond) + defer ticker.Stop() + var retries = 0 + + now := func() int64 { + return time.Now().UnixNano() / int64(time.Millisecond) + + } + + var lastTry = now() + + canQuery := func() bool { + backoff := bootnodesQueryBackoffMs * int64(math.Exp2(float64(retries))) + + return lastTry+backoff < now() + } + + for { + select { + case <-w.dnsDiscAsyncRetrievedSignal: + if !canQuery() { + continue + } + + err := w.restartDiscV5(true) + if err != nil { + w.logger.Warn("failed to restart discv5", zap.Error(err)) + } + retries = 0 + lastTry = now() + case <-ticker.C: + if w.seededBootnodesForDiscV5 && len(w.node.Host().Network().Peers()) > 3 { + w.logger.Debug("not querying bootnodes", zap.Bool("seeded", w.seededBootnodesForDiscV5), zap.Int("peer-count", len(w.node.Host().Network().Peers()))) + continue + } + + if !canQuery() { + w.logger.Info("can't query bootnodes", zap.Int("peer-count", len(w.node.Host().Network().Peers())), zap.Int64("lastTry", lastTry), zap.Int64("now", now()), zap.Int64("backoff", bootnodesQueryBackoffMs*int64(math.Exp2(float64(retries)))), zap.Int("retries", retries)) + continue + } + + w.logger.Info("querying bootnodes to restore connectivity", zap.Int("peer-count", len(w.node.Host().Network().Peers()))) + err := w.restartDiscV5(false) + if err != nil { + w.logger.Warn("failed to restart discv5", zap.Error(err)) + } + + lastTry = now() + retries++ + // We reset the retries after a while and restart + if retries > bootnodesMaxRetries { + retries = 0 + } + + // If we go online, trigger immediately + case <-w.goingOnline: + if !canQuery() { + continue + } + + err := w.restartDiscV5(false) + if err != nil { + w.logger.Warn("failed to restart discv5", zap.Error(err)) + } + retries = 0 + lastTry = now() + + case <-w.ctx.Done(): + w.logger.Debug("bootnode seeding stopped") + return + } + } +} + +// Restart discv5, re-retrieving bootstrap nodes +func (w *Waku) restartDiscV5(useOnlyDNSDiscCache bool) error { + ctx, cancel := context.WithTimeout(w.ctx, 30*time.Second) + defer cancel() + bootnodes, err := w.getDiscV5BootstrapNodes(ctx, w.discV5BootstrapNodes, useOnlyDNSDiscCache) + if err != nil { + return err + } + if len(bootnodes) == 0 { + return errors.New("failed to fetch bootnodes") + } + + if w.node.DiscV5().ErrOnNotRunning() != nil { + w.logger.Info("is not started restarting") + err := w.node.DiscV5().Start(w.ctx) + if err != nil { + w.logger.Error("Could not start DiscV5", zap.Error(err)) + } + } else { + w.node.DiscV5().Stop() + w.logger.Info("is started restarting") + + select { + case <-w.ctx.Done(): // Don't start discv5 if we are stopping waku + return nil + default: + } + + err := w.node.DiscV5().Start(w.ctx) + if err != nil { + w.logger.Error("Could not start DiscV5", zap.Error(err)) + } + } + + w.logger.Info("restarting discv5 with nodes", zap.Any("nodes", bootnodes)) + return w.node.SetDiscV5Bootnodes(bootnodes) +} +*/ func (w *Waku) AddStorePeer(address multiaddr.Multiaddr) (peer.ID, error) { - // peerID, err := w.node.AddPeer(address, wps.Static, w.cfg.DefaultShardedPubsubTopics, store.StoreQueryID_v300) - // if err != nil { - // return "", err - // } - // return peerID, nil + // TODO-nwaku + /* + peerID, err := w.node.AddPeer(address, wps.Static, w.cfg.DefaultShardedPubsubTopics, store.StoreQueryID_v300) + if err != nil { + return "", err + } + return peerID, nil */ return "", nil } @@ -1559,26 +2132,32 @@ func (w *Waku) timestamp() int64 { } func (w *Waku) AddRelayPeer(address multiaddr.Multiaddr) (peer.ID, error) { - // peerID, err := w.node.AddPeer(address, wps.Static, w.cfg.DefaultShardedPubsubTopics, relay.WakuRelayID_v200) - // if err != nil { - // return "", err - // } - // return peerID, nil + // TODO-nwaku + /* + peerID, err := w.node.AddPeer(address, wps.Static, w.cfg.DefaultShardedPubsubTopics, relay.WakuRelayID_v200) + if err != nil { + return "", err + } + return peerID, nil + */ return "", nil } func (w *Waku) DialPeer(address multiaddr.Multiaddr) error { - // ctx, cancel := context.WithTimeout(w.ctx, requestTimeout) - // defer cancel() - // return w.node.DialPeerWithMultiAddress(ctx, address) + // TODO-nwaku + /* + ctx, cancel := context.WithTimeout(w.ctx, requestTimeout) + defer cancel() + return w.node.DialPeerWithMultiAddress(ctx, address) */ return nil } -func (w *Waku) DialPeerByID(peerId peer.ID) error { - return w.WakuDialPeerById(peerId, string(relay.WakuRelayID_v200), 1000) +func (w *Waku) DialPeerByID(peerID peer.ID) error { + return w.WakuDialPeerById(peerID, string(relay.WakuRelayID_v200), 1000) } func (w *Waku) DropPeer(peerID peer.ID) error { + // TODO-nwaku // return w.node.ClosePeerById(peerID) return nil } @@ -1605,11 +2184,13 @@ func (w *Waku) Clean() error { return nil } +// TODO-nwaku func (w *Waku) PeerID() peer.ID { // return w.node.Host().ID() return "" } +// TODO-nwaku func (w *Waku) Peerstore() peerstore.Peerstore { // return w.node.Host().Peerstore() return nil @@ -1661,6 +2242,7 @@ func FormatPeerStats(wakuNode *node.WakuNode) types.PeerStats { return p } +// TODO-nwaku func (w *Waku) StoreNode() *store.WakuStore { // return w.node.Store() return nil @@ -1670,7 +2252,7 @@ func FormatPeerConnFailures(wakuNode *node.WakuNode) map[string]int { p := make(map[string]int) for _, peerID := range wakuNode.Host().Network().Peers() { peerInfo := wakuNode.Host().Peerstore().PeerInfo(peerID) - connFailures := wakuNode.Host().Peerstore().(wps.WakuPeerstore).ConnFailures(peerInfo) + connFailures := wakuNode.Host().Peerstore().(wps.WakuPeerstore).ConnFailures(peerInfo.ID) if connFailures > 0 { p[peerID.String()] = connFailures } @@ -1678,109 +2260,12 @@ func FormatPeerConnFailures(wakuNode *node.WakuNode) map[string]int { return p } +// TODO-nwaku func (w *Waku) LegacyStoreNode() legacy_store.Store { // return w.node.LegacyStore() return nil } -type WakuMessageHash = string -type WakuPubsubTopic = string -type WakuContentTopic = string - -type WakuConfig struct { - Host string `json:"host,omitempty"` - Port int `json:"port,omitempty"` - NodeKey string `json:"key,omitempty"` - EnableRelay bool `json:"relay"` - LogLevel string `json:"logLevel"` - DnsDiscovery bool `json:"dnsDiscovery,omitempty"` - DnsDiscoveryUrl string `json:"dnsDiscoveryUrl,omitempty"` - MaxMessageSize string `json:"maxMessageSize,omitempty"` - Staticnodes []string `json:"staticnodes,omitempty"` - Discv5BootstrapNodes []string `json:"discv5BootstrapNodes,omitempty"` - Discv5Discovery bool `json:"discv5Discovery,omitempty"` - ClusterID uint16 `json:"clusterId,omitempty"` - Shards []uint16 `json:"shards,omitempty"` -} - -type Waku struct { - wakuCtx unsafe.Pointer - - appDB *sql.DB - - dnsAddressCache map[string][]dnsdisc.DiscoveredNode // Map to store the multiaddresses returned by dns discovery - dnsAddressCacheLock *sync.RWMutex // lock to handle access to the map - - // Filter-related - filters *common.Filters // Message filters installed with Subscribe function - filterManager *filterapi.FilterManager - - privateKeys map[string]*ecdsa.PrivateKey // Private key storage - symKeys map[string][]byte // Symmetric key storage - keyMu sync.RWMutex // Mutex associated with key stores - - envelopeCache *ttlcache.Cache[gethcommon.Hash, *common.ReceivedMessage] // Pool of envelopes currently tracked by this node - poolMu sync.RWMutex // Mutex to sync the message and expiration pools - - bandwidthCounter *metrics.BandwidthCounter - - protectedTopicStore *persistence.ProtectedTopicsStore - - sendQueue *publish.MessageQueue - limiter *publish.PublishRateLimiter - - missingMsgVerifier *missing.MissingMessageVerifier - - msgQueue chan *common.ReceivedMessage // Message queue for waku messages that havent been decoded - - ctx context.Context - cancel context.CancelFunc - wg sync.WaitGroup - - cfg *WakuConfig - options []node.WakuNodeOption - - envelopeFeed event.Feed - - storeMsgIDs map[gethcommon.Hash]bool // Map of the currently processing ids - storeMsgIDsMu sync.RWMutex - - messageSentCheck *publish.MessageSentCheck - - topicHealthStatusChan chan peermanager.TopicHealthStatus - connectionNotifChan chan node.PeerConnection - connStatusSubscriptions map[string]*types.ConnStatusSubscription - connStatusMu sync.Mutex - onlineChecker *onlinechecker.DefaultOnlineChecker - state connection.State - - logger *zap.Logger - - // NTP Synced timesource - timesource *timesource.NTPTimeSource - - // seededBootnodesForDiscV5 indicates whether we manage to retrieve discovery - // bootnodes successfully - seededBootnodesForDiscV5 bool - - // goingOnline is channel that notifies when connectivity has changed from offline to online - goingOnline chan struct{} - - // discV5BootstrapNodes is the ENR to be used to fetch bootstrap nodes for discovery - discV5BootstrapNodes []string - - onHistoricMessagesRequestFailed func([]byte, peer.ID, error) - onPeerStats func(types.ConnStatus) - - // statusTelemetryClient ITelemetryClient // TODO-nwaku - - defaultShardInfo protocol.RelayShards -} - -func (w *Waku) Stop() error { - return w.WakuStop() -} - func WakuSetup() { C.waku_setup() } @@ -1813,7 +2298,7 @@ func wakuNew(nodeKey *ecdsa.PrivateKey, ts = timesource.Default() } - /* TODO-nwaku + /* TODO-nwaku cfg = setDefaults(cfg) if err = cfg.Validate(logger); err != nil { return nil, err @@ -1860,7 +2345,7 @@ func wakuNew(nodeKey *ecdsa.PrivateKey, onHistoricMessagesRequestFailed: onHistoricMessagesRequestFailed, onPeerStats: onPeerStats, onlineChecker: onlinechecker.NewDefaultOnlineChecker(false).(*onlinechecker.DefaultOnlineChecker), - //sendQueue: publish.NewMessageQueue(1000, cfg.UseThrottledPublish), // TODO-nwaku + //sendQueue: publish.NewMessageQueue(1000, cfg.UseThrottledPublish), // TODO-nwaku }, nil } @@ -1892,7 +2377,6 @@ func (self *Waku) WakuStop() error { errMsg := "error WakuStop: " + C.GoStringN(C.getMyCharPtr(resp), C.int(C.getMyCharLen(resp))) return errors.New(errMsg) } - func (self *Waku) WakuDestroy() error { var resp = C.allocResp() defer C.freeResp(resp) @@ -2203,7 +2687,7 @@ func (self *Waku) ListenAddresses() ([]multiaddr.Multiaddr, error) { listenAddresses := C.GoStringN(C.getMyCharPtr(resp), C.int(C.getMyCharLen(resp))) addrss := strings.Split(listenAddresses, ",") for _, addr := range addrss { - addr, err := ma.NewMultiaddr(addr) + addr, err := multiaddr.NewMultiaddr(addr) if err != nil { return nil, err } @@ -2309,8 +2793,8 @@ func (self *Waku) GetPeerIdsByProtocol(protocol string) (peer.IDSlice, error) { itemsPeerIds := strings.Split(peersStr, ",") var peers peer.IDSlice - for _, peer := range itemsPeerIds { - id, err := peermod.Decode(peer) + for _, p := range itemsPeerIds { + id, err := peer.Decode(p) if err != nil { errMsg := "GetPeerIdsByProtocol - error converting string to int: " + err.Error() return nil, errors.New(errMsg) @@ -2435,160 +2919,7 @@ func (self *Waku) DisconnectPeerById(peerId peer.ID) error { // } // MaxMessageSize returns the maximum accepted message size. -/* TODO-nwaku +/* TODO-nwaku func (w *Waku) MaxMessageSize() uint32 { return w.cfg.MaxMessageSize } */ - -// New creates a WakuV2 client ready to communicate through the LibP2P network. -func New(nodeKey *ecdsa.PrivateKey, - fleet string, - cfg *WakuConfig, - logger *zap.Logger, - appDB *sql.DB, - ts *timesource.NTPTimeSource, - onHistoricMessagesRequestFailed func([]byte, peer.ID, error), - onPeerStats func(types.ConnStatus)) (*Waku, error) { - - // Lock the main goroutine to its current OS thread - runtime.LockOSThread() - - WakuSetup() // This should only be called once in the whole app's life - - node, err := wakuNew(nodeKey, - fleet, - cfg, logger, appDB, ts, onHistoricMessagesRequestFailed, - onPeerStats) - if err != nil { - return nil, err - } - - defaultPubsubTopic, err := node.WakuDefaultPubsubTopic() - if err != nil { - fmt.Println("Error happened:", err.Error()) - } - - err = node.WakuRelaySubscribe(defaultPubsubTopic) - if err != nil { - fmt.Println("Error happened:", err.Error()) - } - - node.WakuSetEventCallback() - - return node, nil - - // if !cfg.UseThrottledPublish || testing.Testing() { - // // To avoid delaying the tests, or for when we dont want to rate limit, we set up an infinite rate limiter, - // // basically disabling the rate limit functionality - // waku.limiter = publish.NewPublishRateLimiter(rate.Inf, 1) - - // } else { - // waku.limiter = publish.NewPublishRateLimiter(publishingLimiterRate, publishingLimitBurst) - // } - - // waku.filters = common.NewFilters(waku.cfg.DefaultShardPubsubTopic, waku.logger) - // waku.bandwidthCounter = metrics.NewBandwidthCounter() - - // if nodeKey == nil { - // // No nodekey is provided, create an ephemeral key - // nodeKey, err = crypto.GenerateKey() - // if err != nil { - // return nil, fmt.Errorf("failed to generate a random go-waku private key: %v", err) - // } - // } - - // hostAddr, err := net.ResolveTCPAddr("tcp", fmt.Sprint(cfg.Host, ":", cfg.Port)) - // if err != nil { - // return nil, fmt.Errorf("failed to setup the network interface: %v", err) - // } - - // libp2pOpts := node.DefaultLibP2POptions - // libp2pOpts = append(libp2pOpts, libp2p.BandwidthReporter(waku.bandwidthCounter)) - // libp2pOpts = append(libp2pOpts, libp2p.NATPortMap()) - - // opts := []node.WakuNodeOption{ - // node.WithLibP2POptions(libp2pOpts...), - // node.WithPrivateKey(nodeKey), - // node.WithHostAddress(hostAddr), - // node.WithConnectionNotification(waku.connectionNotifChan), - // node.WithTopicHealthStatusChannel(waku.topicHealthStatusChan), - // node.WithKeepAlive(randomPeersKeepAliveInterval, allPeersKeepAliveInterval), - // node.WithLogger(logger), - // node.WithLogLevel(logger.Level()), - // node.WithClusterID(cfg.ClusterID), - // node.WithMaxMsgSize(1024 * 1024), - // } - - // if cfg.EnableDiscV5 { - // bootnodes, err := waku.getDiscV5BootstrapNodes(waku.ctx, cfg.DiscV5BootstrapNodes) - // if err != nil { - // logger.Error("failed to get bootstrap nodes", zap.Error(err)) - // return nil, err - // } - // opts = append(opts, node.WithDiscoveryV5(uint(cfg.UDPPort), bootnodes, cfg.AutoUpdate)) - // } - // shards, err := protocol.TopicsToRelayShards(cfg.DefaultShardPubsubTopic) - // if err != nil { - // logger.Error("FATAL ERROR: failed to parse relay shards", zap.Error(err)) - // return nil, errors.New("failed to parse relay shard, invalid pubsubTopic configuration") - // } - // if len(shards) == 0 { //Hack so that tests don't fail. TODO: Need to remove this once tests are changed to use proper cluster and shard. - // shardInfo := protocol.RelayShards{ClusterID: 0, ShardIDs: []uint16{0}} - // shards = append(shards, shardInfo) - // } - // waku.defaultShardInfo = shards[0] - // if cfg.LightClient { - // opts = append(opts, node.WithWakuFilterLightNode()) - // waku.defaultShardInfo = shards[0] - // opts = append(opts, node.WithMaxPeerConnections(cfg.DiscoveryLimit)) - // cfg.EnableStoreConfirmationForMessagesSent = false - // //TODO: temporary work-around to improve lightClient connectivity, need to be removed once community sharding is implemented - // opts = append(opts, node.WithPubSubTopics(cfg.DefaultShardedPubsubTopics)) - // } else { - // relayOpts := []pubsub.Option{ - // pubsub.WithMaxMessageSize(int(waku.cfg.MaxMessageSize)), - // } - - // if waku.logger.Level() == zap.DebugLevel { - // relayOpts = append(relayOpts, pubsub.WithEventTracer(waku)) - // } - - // opts = append(opts, node.WithWakuRelayAndMinPeers(waku.cfg.MinPeersForRelay, relayOpts...)) - // opts = append(opts, node.WithMaxPeerConnections(maxRelayPeers)) - // cfg.EnablePeerExchangeClient = true //Enabling this until discv5 issues are resolved. This will enable more peers to be connected for relay mesh. - // cfg.EnableStoreConfirmationForMessagesSent = true - // } - - // if cfg.EnableStore { - // if appDB == nil { - // return nil, errors.New("appDB is required for store") - // } - // opts = append(opts, node.WithWakuStore()) - // dbStore, err := persistence.NewDBStore(logger, persistence.WithDB(appDB), persistence.WithRetentionPolicy(cfg.StoreCapacity, time.Duration(cfg.StoreSeconds)*time.Second)) - // if err != nil { - // return nil, err - // } - // opts = append(opts, node.WithMessageProvider(dbStore)) - // } - - // if !cfg.LightClient { - // opts = append(opts, node.WithWakuFilterFullNode(filter.WithMaxSubscribers(20))) - // opts = append(opts, node.WithLightPush(lightpush.WithRateLimiter(1, 1))) - // } - - // if appDB != nil { - // waku.protectedTopicStore, err = persistence.NewProtectedTopicsStore(logger, appDB) - // if err != nil { - // return nil, err - // } - // } - - // if cfg.EnablePeerExchangeServer { - // opts = append(opts, node.WithPeerExchange(peer_exchange.WithRateLimiter(1, 1))) - // } - - // waku.options = opts - // waku.logger.Info("setup the go-waku node successfully") - - // return waku, nil -} diff --git a/wakuv2/nwaku_test.go b/wakuv2/nwaku_test.go index 8f2ead7bb0a..a8a5366d387 100644 --- a/wakuv2/nwaku_test.go +++ b/wakuv2/nwaku_test.go @@ -162,20 +162,20 @@ func TestBasicWakuV2(t *testing.T) { extNodeRestPort := 8646 storeNodeInfo, err := GetNwakuInfo(nil, &extNodeRestPort) require.NoError(t, err) - + nwakuConfig := WakuConfig{ - Port: 30303, - NodeKey: "11d0dcea28e86f81937a3bd1163473c7fbc0a0db54fd72914849bc47bdf78710", - EnableRelay: true, - LogLevel: "DEBUG", + Port: 30303, + NodeKey: "11d0dcea28e86f81937a3bd1163473c7fbc0a0db54fd72914849bc47bdf78710", + EnableRelay: true, + LogLevel: "DEBUG", DnsDiscoveryUrl: "enrtree://AMOJVZX4V6EXP7NTJPMAYJYST2QP6AJXYW76IU6VGJS7UVSNDYZG4@boot.prod.status.nodes.status.im", - DnsDiscovery: true, + DnsDiscovery: true, Discv5Discovery: true, - Staticnodes: []string{storeNodeInfo.ListenAddresses[0]}, - ClusterID: 16, - Shards: []uint16{64}, + Staticnodes: []string{storeNodeInfo.ListenAddresses[0]}, + ClusterID: 16, + Shards: []uint16{64}, } - + w, err := New(nil, "", &nwakuConfig, nil, nil, nil, nil, nil) require.NoError(t, err) require.NoError(t, w.Start()) @@ -190,7 +190,7 @@ func TestBasicWakuV2(t *testing.T) { // Sanity check, not great, but it's probably helpful err = tt.RetryWithBackOff(func() error { - + numConnected, err := w.GetNumConnectedPeers() if err != nil { return err @@ -204,7 +204,7 @@ func TestBasicWakuV2(t *testing.T) { require.NoError(t, err) // Get local store node address - storeNode, err :=peer.AddrInfoFromString(storeNodeInfo.ListenAddresses[0]) + storeNode, err := peer.AddrInfoFromString(storeNodeInfo.ListenAddresses[0]) require.NoError(t, err) require.NoError(t, err) @@ -222,7 +222,7 @@ func TestBasicWakuV2(t *testing.T) { require.NoError(t, err) isDisconnected := !slices.Contains(connectedStoreNodes, storeNode.ID) require.True(t, isDisconnected, "nwaku should be disconnected from the store node") - + // Re-connect err = w.DialPeerByID(storeNode.ID) require.NoError(t, err) @@ -231,66 +231,66 @@ func TestBasicWakuV2(t *testing.T) { connectedStoreNodes, err = w.GetPeerIdsByProtocol(string(store.StoreQueryID_v300)) require.NoError(t, err) require.True(t, slices.Contains(connectedStoreNodes, storeNode.ID), "nwaku should be connected to the store node") - - /* - filter := &common.Filter{ - PubsubTopic: config.DefaultShardPubsubTopic, - Messages: common.NewMemoryMessageStore(), - ContentTopics: common.NewTopicSetFromBytes([][]byte{{1, 2, 3, 4}}), - } - _, err = w.Subscribe(filter) - require.NoError(t, err) + /* + filter := &common.Filter{ + PubsubTopic: config.DefaultShardPubsubTopic, + Messages: common.NewMemoryMessageStore(), + ContentTopics: common.NewTopicSetFromBytes([][]byte{{1, 2, 3, 4}}), + } - msgTimestamp := w.timestamp() - contentTopic := maps.Keys(filter.ContentTopics)[0] + _, err = w.Subscribe(filter) + require.NoError(t, err) - time.Sleep(2 * time.Second) + msgTimestamp := w.timestamp() + contentTopic := maps.Keys(filter.ContentTopics)[0] - _, err = w.Send(config.DefaultShardPubsubTopic, &pb.WakuMessage{ - Payload: []byte{1, 2, 3, 4, 5}, - ContentTopic: contentTopic.ContentTopic(), - Version: proto.Uint32(0), - Timestamp: &msgTimestamp, - }, nil) + time.Sleep(2 * time.Second) - require.NoError(t, err) + _, err = w.Send(config.DefaultShardPubsubTopic, &pb.WakuMessage{ + Payload: []byte{1, 2, 3, 4, 5}, + ContentTopic: contentTopic.ContentTopic(), + Version: proto.Uint32(0), + Timestamp: &msgTimestamp, + }, nil) - time.Sleep(1 * time.Second) + require.NoError(t, err) - messages := filter.Retrieve() - require.Len(t, messages, 1) + time.Sleep(1 * time.Second) - timestampInSeconds := msgTimestamp / int64(time.Second) - marginInSeconds := 20 + messages := filter.Retrieve() + require.Len(t, messages, 1) - options = func(b *backoff.ExponentialBackOff) { - b.MaxElapsedTime = 60 * time.Second - b.InitialInterval = 500 * time.Millisecond - } - err = tt.RetryWithBackOff(func() error { - _, envelopeCount, err := w.Query( - context.Background(), - storeNode.PeerID, - store.FilterCriteria{ - ContentFilter: protocol.NewContentFilter(config.DefaultShardPubsubTopic, contentTopic.ContentTopic()), - TimeStart: proto.Int64((timestampInSeconds - int64(marginInSeconds)) * int64(time.Second)), - TimeEnd: proto.Int64((timestampInSeconds + int64(marginInSeconds)) * int64(time.Second)), - }, - nil, - nil, - false, - ) - if err != nil || envelopeCount == 0 { - // in case of failure extend timestamp margin up to 40secs - if marginInSeconds < 40 { - marginInSeconds += 5 - } - return errors.New("no messages received from store node") + timestampInSeconds := msgTimestamp / int64(time.Second) + marginInSeconds := 20 + + options = func(b *backoff.ExponentialBackOff) { + b.MaxElapsedTime = 60 * time.Second + b.InitialInterval = 500 * time.Millisecond } - return nil - }, options) - require.NoError(t, err) */ + err = tt.RetryWithBackOff(func() error { + _, envelopeCount, err := w.Query( + context.Background(), + storeNode.PeerID, + store.FilterCriteria{ + ContentFilter: protocol.NewContentFilter(config.DefaultShardPubsubTopic, contentTopic.ContentTopic()), + TimeStart: proto.Int64((timestampInSeconds - int64(marginInSeconds)) * int64(time.Second)), + TimeEnd: proto.Int64((timestampInSeconds + int64(marginInSeconds)) * int64(time.Second)), + }, + nil, + nil, + false, + ) + if err != nil || envelopeCount == 0 { + // in case of failure extend timestamp margin up to 40secs + if marginInSeconds < 40 { + marginInSeconds += 5 + } + return errors.New("no messages received from store node") + } + return nil + }, options) + require.NoError(t, err) */ require.NoError(t, w.Stop()) } From d6079c7bc98ac9624c24063be6880127d3a2a6e1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?rich=CE=9Brd?= Date: Wed, 23 Oct 2024 16:22:51 -0400 Subject: [PATCH 05/14] feat(nwaku)_: message publisher and sent verifier (#5966) --- timesource/timesource.go | 34 +++--- timesource/timesource_test.go | 6 +- wakuv2/gowaku.go | 16 --- wakuv2/message_publishing.go | 4 +- wakuv2/nwaku.go | 214 ++++++++++++++++++++++++---------- wakuv2/nwaku_test.go | 7 +- 6 files changed, 185 insertions(+), 96 deletions(-) diff --git a/timesource/timesource.go b/timesource/timesource.go index 6b85c394822..e6f5510c870 100644 --- a/timesource/timesource.go +++ b/timesource/timesource.go @@ -2,6 +2,7 @@ package timesource import ( "bytes" + "context" "errors" "sort" "sync" @@ -144,8 +145,8 @@ type NTPTimeSource struct { timeQuery ntpQuery // for ease of testing now func() time.Time - quit chan struct{} started bool + cancel context.CancelFunc mu sync.RWMutex latestOffset time.Duration @@ -175,7 +176,7 @@ func (s *NTPTimeSource) updateOffset() error { // runPeriodically runs periodically the given function based on NTPTimeSource // synchronization limits (fastNTPSyncPeriod / slowNTPSyncPeriod) -func (s *NTPTimeSource) runPeriodically(fn func() error, starWithSlowSyncPeriod bool) { +func (s *NTPTimeSource) runPeriodically(ctx context.Context, fn func() error, starWithSlowSyncPeriod bool) { if s.started { return } @@ -184,7 +185,7 @@ func (s *NTPTimeSource) runPeriodically(fn func() error, starWithSlowSyncPeriod if starWithSlowSyncPeriod { period = s.slowNTPSyncPeriod } - s.quit = make(chan struct{}) + go func() { defer common.LogOnPanic() for { @@ -196,7 +197,7 @@ func (s *NTPTimeSource) runPeriodically(fn func() error, starWithSlowSyncPeriod period = s.fastNTPSyncPeriod } - case <-s.quit: + case <-ctx.Done(): return } } @@ -204,11 +205,13 @@ func (s *NTPTimeSource) runPeriodically(fn func() error, starWithSlowSyncPeriod } // Start initializes the local offset and starts a goroutine that periodically updates the local offset. -func (s *NTPTimeSource) Start() { +func (s *NTPTimeSource) Start(ctx context.Context) error { if s.started { - return + return nil } + ctx, cancel := context.WithCancel(ctx) + // Attempt to update the offset synchronously so that user can have reliable messages right away err := s.updateOffset() if err != nil { @@ -217,23 +220,26 @@ func (s *NTPTimeSource) Start() { log.Error("failed to update offset", err) } - s.runPeriodically(s.updateOffset, err == nil) + s.runPeriodically(ctx, s.updateOffset, err == nil) s.started = true + s.cancel = cancel + + return nil } // Stop goroutine that updates time source. -func (s *NTPTimeSource) Stop() error { - if s.quit == nil { - return nil +func (s *NTPTimeSource) Stop() { + if s.cancel == nil { + return } - close(s.quit) + + s.cancel() s.started = false - return nil } func (s *NTPTimeSource) GetCurrentTime() time.Time { - s.Start() + s.Start(context.Background()) return s.Now() } @@ -243,7 +249,7 @@ func (s *NTPTimeSource) GetCurrentTimeInMillis() uint64 { func GetCurrentTime() time.Time { ts := Default() - ts.Start() + ts.Start(context.Background()) return ts.Now() } diff --git a/timesource/timesource_test.go b/timesource/timesource_test.go index 51a75365710..1721eb1d00d 100644 --- a/timesource/timesource_test.go +++ b/timesource/timesource_test.go @@ -277,14 +277,12 @@ func TestGetCurrentTimeInMillis(t *testing.T) { // test repeat invoke GetCurrentTimeInMillis n = ts.GetCurrentTimeInMillis() require.Equal(t, expectedTime, n) - e := ts.Stop() - require.NoError(t, e) + ts.Stop() // test invoke after stop n = ts.GetCurrentTimeInMillis() require.Equal(t, expectedTime, n) - e = ts.Stop() - require.NoError(t, e) + ts.Stop() } func TestGetCurrentTimeOffline(t *testing.T) { diff --git a/wakuv2/gowaku.go b/wakuv2/gowaku.go index 8976f44905e..5a96a0acf26 100644 --- a/wakuv2/gowaku.go +++ b/wakuv2/gowaku.go @@ -1951,22 +1951,6 @@ func (w *Waku) LegacyStoreNode() legacy_store.Store { return w.node.LegacyStore() } -func (w *Waku) WakuLightpushPublish(message *pb.WakuMessage, pubsubTopic string) (string, error) { - msgHash, err := w.node.Lightpush().Publish(w.ctx, message, lightpush.WithPubSubTopic(pubsubTopic)) - if err != nil { - return "", err - } - return msgHash.String(), nil -} - -func (w *Waku) WakuRelayPublish(message *pb.WakuMessage, pubsubTopic string) (string, error) { - msgHash, err := w.node.Relay().Publish(w.ctx, message, relay.WithPubSubTopic(pubsubTopic)) - if err != nil { - return "", err - } - return msgHash.String(), nil -} - func (w *Waku) ListPeersInMesh(pubsubTopic string) (int, error) { listPeers := w.node.Relay().PubSub().ListPeers(pubsubTopic) return len(listPeers), nil diff --git a/wakuv2/message_publishing.go b/wakuv2/message_publishing.go index 90dce0cae8d..3be0ea96d46 100644 --- a/wakuv2/message_publishing.go +++ b/wakuv2/message_publishing.go @@ -1,6 +1,5 @@ package wakuv2 -/* TODO-nwaku import ( "errors" @@ -93,6 +92,7 @@ func (w *Waku) publishEnvelope(envelope *protocol.Envelope) { err = w.messageSender.Send(publish.NewRequest(w.ctx, envelope)) } + /* TODO-nwaku if w.statusTelemetryClient != nil { if err == nil { w.statusTelemetryClient.PushSentEnvelope(w.ctx, SentEnvelope{Envelope: envelope, PublishMethod: w.messageSender.PublishMethod()}) @@ -100,6 +100,7 @@ func (w *Waku) publishEnvelope(envelope *protocol.Envelope) { w.statusTelemetryClient.PushErrorSendingEnvelope(w.ctx, ErrorSendingEnvelope{Error: err, SentEnvelope: SentEnvelope{Envelope: envelope, PublishMethod: w.messageSender.PublishMethod()}}) } } + */ if err != nil { logger.Error("could not send message", zap.Error(err)) @@ -117,4 +118,3 @@ func (w *Waku) publishEnvelope(envelope *protocol.Envelope) { }) } } -*/ diff --git a/wakuv2/nwaku.go b/wakuv2/nwaku.go index f121343e8a3..796630a3bde 100644 --- a/wakuv2/nwaku.go +++ b/wakuv2/nwaku.go @@ -270,16 +270,20 @@ import ( "crypto/ecdsa" "crypto/sha256" "database/sql" + "encoding/hex" "encoding/json" "errors" "fmt" + "net/http" "runtime" "strconv" "strings" "sync" + "testing" "time" "unsafe" + "github.com/golang/protobuf/proto" "github.com/jellydator/ttlcache/v3" "github.com/libp2p/go-libp2p/core/peer" "github.com/libp2p/go-libp2p/core/peerstore" @@ -288,6 +292,7 @@ import ( "go.uber.org/zap" "golang.org/x/crypto/pbkdf2" + "golang.org/x/time/rate" gethcommon "github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/common/hexutil" @@ -307,9 +312,11 @@ import ( "github.com/waku-org/go-waku/waku/v2/peermanager" wps "github.com/waku-org/go-waku/waku/v2/peerstore" "github.com/waku-org/go-waku/waku/v2/protocol" + "github.com/waku-org/go-waku/waku/v2/protocol/legacy_store" "github.com/waku-org/go-waku/waku/v2/protocol/relay" "github.com/waku-org/go-waku/waku/v2/protocol/store" + storepb "github.com/waku-org/go-waku/waku/v2/protocol/store/pb" "github.com/waku-org/go-waku/waku/v2/utils" gocommon "github.com/status-im/status-go/common" @@ -412,7 +419,9 @@ type Waku struct { cancel context.CancelFunc wg sync.WaitGroup - cfg *WakuConfig + cfg *Config + wakuCfg *WakuConfig + options []node.WakuNodeOption envelopeFeed event.Feed @@ -467,7 +476,7 @@ func newTTLCache() *ttlcache.Cache[gethcommon.Hash, *common.ReceivedMessage] { } // New creates a WakuV2 client ready to communicate through the LibP2P network. -func New(nodeKey *ecdsa.PrivateKey, fleet string, cfg *WakuConfig, logger *zap.Logger, appDB *sql.DB, ts *timesource.NTPTimeSource, onHistoricMessagesRequestFailed func([]byte, peer.ID, error), onPeerStats func(types.ConnStatus)) (*Waku, error) { +func New(nodeKey *ecdsa.PrivateKey, fleet string, cfg *Config, nwakuCfg *WakuConfig, logger *zap.Logger, appDB *sql.DB, ts *timesource.NTPTimeSource, onHistoricMessagesRequestFailed func([]byte, peer.ID, error), onPeerStats func(types.ConnStatus)) (*Waku, error) { // Lock the main goroutine to its current OS thread runtime.LockOSThread() @@ -475,7 +484,9 @@ func New(nodeKey *ecdsa.PrivateKey, fleet string, cfg *WakuConfig, logger *zap.L node, err := wakuNew(nodeKey, fleet, - cfg, logger, appDB, ts, onHistoricMessagesRequestFailed, + cfg, + nwakuCfg, + logger, appDB, ts, onHistoricMessagesRequestFailed, onPeerStats) if err != nil { return nil, err @@ -918,6 +929,7 @@ func (w *Waku) runPeerExchangeLoop() { } } } +*/ func (w *Waku) GetPubsubTopic(topic string) string { if topic == "" { @@ -927,6 +939,7 @@ func (w *Waku) GetPubsubTopic(topic string) string { return topic } +/* TODO-nwaku func (w *Waku) unsubscribeFromPubsubTopicWithWakuRelay(topic string) error { topic = w.GetPubsubTopic(topic) @@ -1329,8 +1342,7 @@ func (w *Waku) OnNewEnvelope(env *protocol.Envelope) error { func (w *Waku) Start() error { err := w.WakuStart() if err != nil { - fmt.Println("Error happened:", err.Error()) - return err + return fmt.Errorf("failed to start nwaku node: %v", err) } /* TODO-nwaku @@ -1468,19 +1480,16 @@ func (w *Waku) Start() error { } */ - // w.wg.Add(1) + w.wg.Add(1) - // TODO-nwaku - // go w.broadcast() + go w.broadcast() - // go w.sendQueue.Start(w.ctx) + go w.sendQueue.Start(w.ctx) - /* TODO-nwaku err = w.startMessageSender() if err != nil { return err } - */ /* TODO-nwaku // we should wait `seedBootnodesForDiscV5` shutdown smoothly before set w.ctx to nil within `w.Stop()` @@ -1568,14 +1577,14 @@ func (w *Waku) reportPeerMetrics() { } */ -/* TODO-nwaku func (w *Waku) startMessageSender() error { publishMethod := publish.Relay + /* TODO-nwaku if w.cfg.LightClient { publishMethod = publish.LightPush - } + }*/ - sender, err := publish.NewMessageSender(publishMethod, publish.NewDefaultPublisher(w.node.Lightpush(), w.node.Relay()), w.logger) + sender, err := publish.NewMessageSender(publishMethod, newPublisher(w.wakuCtx), w.logger) if err != nil { w.logger.Error("failed to create message sender", zap.Error(err)) return err @@ -1584,7 +1593,7 @@ func (w *Waku) startMessageSender() error { if w.cfg.EnableStoreConfirmationForMessagesSent { msgStoredChan := make(chan gethcommon.Hash, 1000) msgExpiredChan := make(chan gethcommon.Hash, 1000) - messageSentCheck := publish.NewMessageSentCheck(w.ctx, publish.NewDefaultStorenodeMessageVerifier(w.node.Store()), w.StorenodeCycle, w.node.Timesource(), msgStoredChan, msgExpiredChan, w.logger) + messageSentCheck := publish.NewMessageSentCheck(w.ctx, newStorenodeMessageVerifier(w.wakuCtx), w.StorenodeCycle, w.timesource, msgStoredChan, msgExpiredChan, w.logger) sender.WithMessageSentCheck(messageSentCheck) w.wg.Add(1) @@ -1600,17 +1609,21 @@ func (w *Waku) startMessageSender() error { Hash: hash, Event: common.EventEnvelopeSent, }) - if w.statusTelemetryClient != nil { + + // TODO-nwaku + /*if w.statusTelemetryClient != nil { w.statusTelemetryClient.PushMessageCheckSuccess(w.ctx, hash.Hex()) - } + }*/ case hash := <-msgExpiredChan: w.SendEnvelopeEvent(common.EnvelopeEvent{ Hash: hash, Event: common.EventEnvelopeExpired, }) - if w.statusTelemetryClient != nil { + + // TODO-nwaku + /* if w.statusTelemetryClient != nil { w.statusTelemetryClient.PushMessageCheckFailure(w.ctx, hash.Hex()) - } + }*/ } } }() @@ -1628,7 +1641,6 @@ func (w *Waku) startMessageSender() error { return nil } -*/ func (w *Waku) MessageExists(mh pb.MessageHash) (bool, error) { w.poolMu.Lock() @@ -1790,7 +1802,6 @@ func (w *Waku) postEvent(envelope *common.ReceivedMessage) { w.msgQueue <- envelope } -/* TODO-nwaku // processQueueLoop delivers the messages to the watchers during the lifetime of the waku node. func (w *Waku) processQueueLoop() { defer gocommon.LogOnPanic() @@ -1806,7 +1817,7 @@ func (w *Waku) processQueueLoop() { w.processMessage(e) } } -}*/ +} func (w *Waku) processMessage(e *common.ReceivedMessage) { logger := w.logger.With( @@ -2281,7 +2292,8 @@ func printStackTrace() { func wakuNew(nodeKey *ecdsa.PrivateKey, fleet string, - cfg *WakuConfig, + cfg *Config, // TODO: merge Config and WakuConfig + nwakuCfg *WakuConfig, logger *zap.Logger, appDB *sql.DB, ts *timesource.NTPTimeSource, @@ -2298,15 +2310,17 @@ func wakuNew(nodeKey *ecdsa.PrivateKey, ts = timesource.Default() } - /* TODO-nwaku + nwakuCfg.NodeKey = hex.EncodeToString(crypto.FromECDSA(nodeKey)) + + // TODO-nwaku + // TODO: merge Config and WakuConfig cfg = setDefaults(cfg) if err = cfg.Validate(logger); err != nil { return nil, err - } */ - - ctx, cancel := context.WithCancel(context.Background()) + } + logger.Info("starting wakuv2 with config", zap.Any("nwakuCfg", nwakuCfg), zap.Any("wakuCfg", cfg)) - jsonConfig, err := json.Marshal(cfg) + jsonConfig, err := json.Marshal(nwakuCfg) if err != nil { return nil, err } @@ -2321,9 +2335,10 @@ func wakuNew(nodeKey *ecdsa.PrivateKey, // Notice that the events for self node are handled by the 'MyEventCallback' method if C.getRet(resp) == C.RET_OK { - + ctx, cancel := context.WithCancel(context.Background()) return &Waku{ wakuCtx: wakuCtx, + wakuCfg: nwakuCfg, cfg: cfg, privateKeys: make(map[string]*ecdsa.PrivateKey), symKeys: make(map[string][]byte), @@ -2337,15 +2352,16 @@ func wakuNew(nodeKey *ecdsa.PrivateKey, wg: sync.WaitGroup{}, dnsAddressCache: make(map[string][]dnsdisc.DiscoveredNode), dnsAddressCacheLock: &sync.RWMutex{}, + dnsDiscAsyncRetrievedSignal: make(chan struct{}), storeMsgIDs: make(map[gethcommon.Hash]bool), timesource: ts, storeMsgIDsMu: sync.RWMutex{}, logger: logger, - discV5BootstrapNodes: cfg.Discv5BootstrapNodes, + discV5BootstrapNodes: nwakuCfg.Discv5BootstrapNodes, onHistoricMessagesRequestFailed: onHistoricMessagesRequestFailed, onPeerStats: onPeerStats, onlineChecker: onlinechecker.NewDefaultOnlineChecker(false).(*onlinechecker.DefaultOnlineChecker), - //sendQueue: publish.NewMessageQueue(1000, cfg.UseThrottledPublish), // TODO-nwaku + sendQueue: publish.NewMessageQueue(1000, cfg.UseThrottledPublish), }, nil } @@ -2513,32 +2529,6 @@ func (self *Waku) WakuDefaultPubsubTopic() (WakuPubsubTopic, error) { return "", errors.New(errMsg) } -func (self *Waku) WakuRelayPublish(wakuMsg *pb.WakuMessage, pubsubTopic string) (string, error) { - timeoutMs := 1000 - - message, err := json.Marshal(wakuMsg) - if err != nil { - return "", err - } - - var cPubsubTopic = C.CString(pubsubTopic) - var msg = C.CString(string(message)) - var resp = C.allocResp() - - defer C.freeResp(resp) - defer C.free(unsafe.Pointer(cPubsubTopic)) - defer C.free(unsafe.Pointer(msg)) - - C.cGoWakuRelayPublish(self.wakuCtx, cPubsubTopic, msg, C.int(timeoutMs), resp) - if C.getRet(resp) == C.RET_OK { - msgHash := C.GoStringN(C.getMyCharPtr(resp), C.int(C.getMyCharLen(resp))) - return msgHash, nil - } - errMsg := "error WakuRelayPublish: " + - C.GoStringN(C.getMyCharPtr(resp), C.int(C.getMyCharLen(resp))) - return "", errors.New(errMsg) -} - func (self *Waku) WakuRelaySubscribe(pubsubTopic string) error { var resp = C.allocResp() var cPubsubTopic = C.CString(pubsubTopic) @@ -2605,7 +2595,8 @@ func (self *Waku) WakuLightpushPublish(message *pb.WakuMessage, pubsubTopic stri return "", errors.New(errMsg) } -func (self *Waku) wakuStoreQuery( +func wakuStoreQuery( + wakuCtx unsafe.Pointer, jsonQuery string, peerAddr string, timeoutMs int) (string, error) { @@ -2618,7 +2609,7 @@ func (self *Waku) wakuStoreQuery( defer C.free(unsafe.Pointer(cPeerAddr)) defer C.freeResp(resp) - C.cGoWakuStoreQuery(self.wakuCtx, cJsonQuery, cPeerAddr, C.int(timeoutMs), resp) + C.cGoWakuStoreQuery(wakuCtx, cJsonQuery, cPeerAddr, C.int(timeoutMs), resp) if C.getRet(resp) == C.RET_OK { msg := C.GoStringN(C.getMyCharPtr(resp), C.int(C.getMyCharLen(resp))) return msg, nil @@ -2923,3 +2914,108 @@ func (self *Waku) DisconnectPeerById(peerId peer.ID) error { func (w *Waku) MaxMessageSize() uint32 { return w.cfg.MaxMessageSize } */ + +func newPublisher(wakuCtx unsafe.Pointer) publish.Publisher { + return &nwakuPublisher{ + wakuCtx: wakuCtx, + } +} + +type nwakuPublisher struct { + wakuCtx unsafe.Pointer +} + +func (p *nwakuPublisher) RelayListPeers(pubsubTopic string) ([]peer.ID, error) { + // TODO-nwaku + return nil, nil +} + +func (p *nwakuPublisher) RelayPublish(ctx context.Context, message *pb.WakuMessage, pubsubTopic string) (pb.MessageHash, error) { + timeoutMs := 1000 + + jsonMsg, err := json.Marshal(message) + if err != nil { + return pb.MessageHash{}, err + } + + var cPubsubTopic = C.CString(pubsubTopic) + var msg = C.CString(string(jsonMsg)) + var resp = C.allocResp() + + defer C.freeResp(resp) + defer C.free(unsafe.Pointer(cPubsubTopic)) + defer C.free(unsafe.Pointer(msg)) + + C.cGoWakuRelayPublish(p.wakuCtx, cPubsubTopic, msg, C.int(timeoutMs), resp) + if C.getRet(resp) == C.RET_OK { + msgHash := C.GoStringN(C.getMyCharPtr(resp), C.int(C.getMyCharLen(resp))) + msgHashBytes, err := hexutil.Decode(msgHash) + if err != nil { + return pb.MessageHash{}, err + } + return pb.ToMessageHash(msgHashBytes), nil + } + errMsg := "error WakuRelayPublish: " + + C.GoStringN(C.getMyCharPtr(resp), C.int(C.getMyCharLen(resp))) + return pb.MessageHash{}, errors.New(errMsg) +} + +// LightpushPublish publishes a message via WakuLightPush +func (p *nwakuPublisher) LightpushPublish(ctx context.Context, message *pb.WakuMessage, pubsubTopic string, maxPeers int) (pb.MessageHash, error) { + // TODO-nwaku + return pb.MessageHash{}, errors.New("not implemented yet") +} + +func newStorenodeMessageVerifier(wakuCtx unsafe.Pointer) publish.StorenodeMessageVerifier { + return &defaultStorenodeMessageVerifier{ + wakuCtx: wakuCtx, + } +} + +type defaultStorenodeMessageVerifier struct { + wakuCtx unsafe.Pointer +} + +func (d *defaultStorenodeMessageVerifier) MessageHashesExist(ctx context.Context, requestID []byte, peerID peer.ID, pageSize uint64, messageHashes []pb.MessageHash) ([]pb.MessageHash, error) { + requestIDStr := hexutil.Encode(requestID) + storeRequest := &storepb.StoreQueryRequest{ + RequestId: requestIDStr, + MessageHashes: make([][]byte, len(messageHashes)), + IncludeData: false, + PaginationCursor: nil, + PaginationForward: false, + PaginationLimit: proto.Uint64(pageSize), + } + + for i, mhash := range messageHashes { + storeRequest.MessageHashes[i] = mhash.Bytes() + } + + jsonQuery, err := json.Marshal(storeRequest) + if err != nil { + return nil, err + } + + // TODO: timeouts need to be managed differently. For now we're using a 1m timeout + jsonResponse, err := wakuStoreQuery(d.wakuCtx, string(jsonQuery), peerID.String(), int(time.Minute.Milliseconds())) + if err != nil { + return nil, err + } + + response := &storepb.StoreQueryResponse{} + err = json.Unmarshal([]byte(jsonResponse), response) + if err != nil { + return nil, err + } + + if response.GetStatusCode() != http.StatusOK { + return nil, fmt.Errorf("could not query storenode: %s %d %s", requestIDStr, response.GetStatusCode(), response.GetStatusDesc()) + } + + result := make([]pb.MessageHash, len(response.Messages)) + for i, msg := range response.Messages { + result[i] = pb.ToMessageHash(msg.GetMessageHash()) + } + + return result, nil +} diff --git a/wakuv2/nwaku_test.go b/wakuv2/nwaku_test.go index a8a5366d387..fdf6f5458a9 100644 --- a/wakuv2/nwaku_test.go +++ b/wakuv2/nwaku_test.go @@ -163,6 +163,11 @@ func TestBasicWakuV2(t *testing.T) { storeNodeInfo, err := GetNwakuInfo(nil, &extNodeRestPort) require.NoError(t, err) + wakuConfig := Config{ + UseThrottledPublish: true, + ClusterID: 16, + } + nwakuConfig := WakuConfig{ Port: 30303, NodeKey: "11d0dcea28e86f81937a3bd1163473c7fbc0a0db54fd72914849bc47bdf78710", @@ -176,7 +181,7 @@ func TestBasicWakuV2(t *testing.T) { Shards: []uint16{64}, } - w, err := New(nil, "", &nwakuConfig, nil, nil, nil, nil, nil) + w, err := New(nil, "", &wakuConfig, &nwakuConfig, nil, nil, nil, nil, nil) require.NoError(t, err) require.NoError(t, w.Start()) From 6dd9b203686a4528f928bd207b11c17512cec97d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?rich=CE=9Brd?= Date: Wed, 23 Oct 2024 16:25:28 -0400 Subject: [PATCH 06/14] feat(nwaku)_: storenode requestor for missing message retrieval and result iterator impl (#5971) --- wakuv2/nwaku.go | 186 ++++++++++++++++++++++++++++++++++++++++++++++-- 1 file changed, 180 insertions(+), 6 deletions(-) diff --git a/wakuv2/nwaku.go b/wakuv2/nwaku.go index 796630a3bde..6deda931798 100644 --- a/wakuv2/nwaku.go +++ b/wakuv2/nwaku.go @@ -303,6 +303,8 @@ import ( "github.com/libp2p/go-libp2p/core/metrics" + commonapi "github.com/waku-org/go-waku/waku/v2/api/common" + filterapi "github.com/waku-org/go-waku/waku/v2/api/filter" "github.com/waku-org/go-waku/waku/v2/api/history" "github.com/waku-org/go-waku/waku/v2/api/missing" @@ -322,6 +324,7 @@ import ( gocommon "github.com/status-im/status-go/common" "github.com/status-im/status-go/connection" "github.com/status-im/status-go/eth-node/types" + "github.com/status-im/status-go/logutils" "github.com/status-im/status-go/timesource" "github.com/status-im/status-go/wakuv2/common" "github.com/status-im/status-go/wakuv2/persistence" @@ -1427,13 +1430,13 @@ func (w *Waku) Start() error { w.wg.Add(1) go w.runPeerExchangeLoop() + */ if w.cfg.EnableMissingMessageVerification { - w.missingMsgVerifier = missing.NewMissingMessageVerifier( - missing.NewDefaultStorenodeRequestor(w.node.Store()), + newStorenodeRequestor(w.wakuCtx, w.logger), w, - w.node.Timesource(), + w.timesource, w.logger) w.missingMsgVerifier.Start(w.ctx) @@ -1456,6 +1459,7 @@ func (w *Waku) Start() error { }() } + /* TODO: nwaku if w.cfg.LightClient { // Create FilterManager that will main peer connectivity // for installed filters @@ -1715,7 +1719,6 @@ func (w *Waku) Stop() error { return nil } -/* TODO-nwaku func (w *Waku) OnNewEnvelopes(envelope *protocol.Envelope, msgType common.MessageType, processImmediately bool) error { if envelope == nil { return nil @@ -1751,7 +1754,6 @@ func (w *Waku) OnNewEnvelopes(envelope *protocol.Envelope, msgType common.Messag return nil } -*/ // addEnvelope adds an envelope to the envelope map, used for sending func (w *Waku) addEnvelope(envelope *common.ReceivedMessage) { @@ -2977,7 +2979,7 @@ type defaultStorenodeMessageVerifier struct { } func (d *defaultStorenodeMessageVerifier) MessageHashesExist(ctx context.Context, requestID []byte, peerID peer.ID, pageSize uint64, messageHashes []pb.MessageHash) ([]pb.MessageHash, error) { - requestIDStr := hexutil.Encode(requestID) + requestIDStr := hex.EncodeToString(requestID) storeRequest := &storepb.StoreQueryRequest{ RequestId: requestIDStr, MessageHashes: make([][]byte, len(messageHashes)), @@ -3019,3 +3021,175 @@ func (d *defaultStorenodeMessageVerifier) MessageHashesExist(ctx context.Context return result, nil } + +func newStorenodeRequestor(wakuCtx unsafe.Pointer, logger *zap.Logger) missing.StorenodeRequestor { + return &storenodeRequestor{ + wakuCtx: wakuCtx, + logger: logger.Named("storenodeRequestor"), + } +} + +type storenodeRequestor struct { + wakuCtx unsafe.Pointer + logger *zap.Logger +} + +func (s *storenodeRequestor) GetMessagesByHash(ctx context.Context, peerID peer.ID, pageSize uint64, messageHashes []pb.MessageHash) (commonapi.StoreRequestResult, error) { + requestIDStr := hex.EncodeToString(protocol.GenerateRequestID()) + + logger := s.logger.With(zap.Stringer("peerID", peerID), zap.String("requestID", requestIDStr)) + + logger.Debug("sending store request") + + storeRequest := &storepb.StoreQueryRequest{ + RequestId: requestIDStr, + MessageHashes: make([][]byte, len(messageHashes)), + IncludeData: true, + PaginationCursor: nil, + PaginationForward: false, + PaginationLimit: proto.Uint64(pageSize), + } + + for i, mhash := range messageHashes { + storeRequest.MessageHashes[i] = mhash.Bytes() + } + + jsonQuery, err := json.Marshal(storeRequest) + if err != nil { + return nil, err + } + + // TODO: timeouts need to be managed differently. For now we're using a 1m timeout + jsonResponse, err := wakuStoreQuery(s.wakuCtx, string(jsonQuery), peerID.String(), int(time.Minute.Milliseconds())) + if err != nil { + return nil, err + } + + storeResponse := &storepb.StoreQueryResponse{} + err = json.Unmarshal([]byte(jsonResponse), storeResponse) + if err != nil { + return nil, err + } + + if storeResponse.GetStatusCode() != http.StatusOK { + return nil, fmt.Errorf("could not query storenode: %s %d %s", requestIDStr, storeResponse.GetStatusCode(), storeResponse.GetStatusDesc()) + } + + return newStoreResultImpl(s.wakuCtx, peerID, storeRequest, storeResponse), nil +} + +func (s *storenodeRequestor) QueryWithCriteria(ctx context.Context, peerID peer.ID, pageSize uint64, pubsubTopic string, contentTopics []string, from *int64, to *int64) (commonapi.StoreRequestResult, error) { + requestIDStr := hex.EncodeToString(protocol.GenerateRequestID()) + + logger := s.logger.With(zap.Stringer("peerID", peerID), zap.String("requestID", requestIDStr)) + + logger.Debug("sending store request") + + storeRequest := &storepb.StoreQueryRequest{ + RequestId: requestIDStr, + PubsubTopic: proto.String(pubsubTopic), + ContentTopics: contentTopics, + TimeStart: from, + TimeEnd: to, + IncludeData: false, + PaginationCursor: nil, + PaginationForward: false, + PaginationLimit: proto.Uint64(pageSize), + } + + jsonQuery, err := json.Marshal(storeRequest) + if err != nil { + return nil, err + } + + // TODO: timeouts need to be managed differently. For now we're using a 1m timeout + jsonResponse, err := wakuStoreQuery(s.wakuCtx, string(jsonQuery), peerID.String(), int(time.Minute.Milliseconds())) + if err != nil { + return nil, err + } + + storeResponse := &storepb.StoreQueryResponse{} + err = json.Unmarshal([]byte(jsonResponse), storeResponse) + if err != nil { + return nil, err + } + + if storeResponse.GetStatusCode() != http.StatusOK { + return nil, fmt.Errorf("could not query storenode: %s %d %s", requestIDStr, storeResponse.GetStatusCode(), storeResponse.GetStatusDesc()) + } + + return newStoreResultImpl(s.wakuCtx, peerID, storeRequest, storeResponse), nil +} + +type storeResultImpl struct { + done bool + + wakuCtx unsafe.Pointer + storeRequest *storepb.StoreQueryRequest + storeResponse *storepb.StoreQueryResponse + peerID peer.ID +} + +func newStoreResultImpl(wakuCtx unsafe.Pointer, peerID peer.ID, storeRequest *storepb.StoreQueryRequest, storeResponse *storepb.StoreQueryResponse) *storeResultImpl { + return &storeResultImpl{ + wakuCtx: wakuCtx, + storeRequest: storeRequest, + storeResponse: storeResponse, + peerID: peerID, + } +} + +func (r *storeResultImpl) Cursor() []byte { + return r.storeResponse.GetPaginationCursor() +} + +func (r *storeResultImpl) IsComplete() bool { + return r.done +} + +func (r *storeResultImpl) PeerID() peer.ID { + return r.peerID +} + +func (r *storeResultImpl) Query() *storepb.StoreQueryRequest { + return r.storeRequest +} + +func (r *storeResultImpl) Response() *storepb.StoreQueryResponse { + return r.storeResponse +} + +func (r *storeResultImpl) Next(ctx context.Context, opts ...store.RequestOption) error { + // TODO: opts is being ignored. Will require some changes in go-waku. For now using this + // is not necessary + + if r.storeResponse.GetPaginationCursor() == nil { + r.done = true + return nil + } + + r.storeRequest.RequestId = hex.EncodeToString(protocol.GenerateRequestID()) + r.storeRequest.PaginationCursor = r.storeResponse.PaginationCursor + + jsonQuery, err := json.Marshal(r.storeRequest) + if err != nil { + return err + } + + // TODO: timeouts need to be managed differently. For now we're using a 1m timeout + jsonResponse, err := wakuStoreQuery(r.wakuCtx, string(jsonQuery), r.peerID.String(), int(time.Minute.Milliseconds())) + if err != nil { + return err + } + + err = json.Unmarshal([]byte(jsonResponse), r.storeResponse) + if err != nil { + return err + } + + return nil +} + +func (r *storeResultImpl) Messages() []*storepb.WakuMessageKeyValue { + return r.storeResponse.GetMessages() +} From ed5a9aa46ff6ec3eda6bcea5d67b0124f5615936 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?rich=CE=9Brd?= Date: Thu, 24 Oct 2024 10:35:01 -0400 Subject: [PATCH 07/14] chore_: uncomment code that would allow status-go/go-waku to compile and libwaku test to run (#5986) --- timesource/timesource.go | 12 ++++++-- timesource/timesource_test.go | 3 +- wakuv2/api.go | 3 +- wakuv2/api_test.go | 3 +- wakuv2/history_processor_wrapper.go | 6 ++-- wakuv2/nwaku.go | 44 +++++++++-------------------- 6 files changed, 32 insertions(+), 39 deletions(-) diff --git a/timesource/timesource.go b/timesource/timesource.go index e6f5510c870..57cf9a5bab0 100644 --- a/timesource/timesource.go +++ b/timesource/timesource.go @@ -239,7 +239,11 @@ func (s *NTPTimeSource) Stop() { } func (s *NTPTimeSource) GetCurrentTime() time.Time { - s.Start(context.Background()) + err := s.Start(context.Background()) + if err != nil { + panic("could not obtain timesource") + } + return s.Now() } @@ -249,7 +253,11 @@ func (s *NTPTimeSource) GetCurrentTimeInMillis() uint64 { func GetCurrentTime() time.Time { ts := Default() - ts.Start(context.Background()) + err := ts.Start(context.Background()) + if err != nil { + panic("could not obtain timesource") + } + return ts.Now() } diff --git a/timesource/timesource_test.go b/timesource/timesource_test.go index 1721eb1d00d..35840586a41 100644 --- a/timesource/timesource_test.go +++ b/timesource/timesource_test.go @@ -1,6 +1,7 @@ package timesource import ( + "context" "errors" "sync" "testing" @@ -214,7 +215,7 @@ func TestRunningPeriodically(t *testing.T) { // on NTPTimeSource specified periods (fastNTPSyncPeriod & slowNTPSyncPeriod) wg := sync.WaitGroup{} wg.Add(1) - source.runPeriodically(func() error { + source.runPeriodically(context.TODO(), func() error { mu.Lock() periods = append(periods, time.Since(lastCall)) mu.Unlock() diff --git a/wakuv2/api.go b/wakuv2/api.go index c129abdc033..49be14a32c1 100644 --- a/wakuv2/api.go +++ b/wakuv2/api.go @@ -18,7 +18,6 @@ package wakuv2 -/* TODO-nwaku import ( "context" "crypto/ecdsa" @@ -513,4 +512,4 @@ func (api *PublicWakuAPI) NewMessageFilter(req Criteria) (string, error) { api.mu.Unlock() return id, nil -} */ +} diff --git a/wakuv2/api_test.go b/wakuv2/api_test.go index 10d16a4c332..d3353ffa3a4 100644 --- a/wakuv2/api_test.go +++ b/wakuv2/api_test.go @@ -68,4 +68,5 @@ func TestMultipleTopicCopyInNewMessageFilter(t *testing.T) { if !found { t.Fatalf("Could not find filter with both topics") } -} */ \ No newline at end of file +} +*/ diff --git a/wakuv2/history_processor_wrapper.go b/wakuv2/history_processor_wrapper.go index bdb72745cce..56c3f8cf437 100644 --- a/wakuv2/history_processor_wrapper.go +++ b/wakuv2/history_processor_wrapper.go @@ -3,6 +3,8 @@ package wakuv2 import ( "github.com/libp2p/go-libp2p/core/peer" + "github.com/status-im/status-go/wakuv2/common" + "github.com/waku-org/go-waku/waku/v2/api/history" "github.com/waku-org/go-waku/waku/v2/protocol" ) @@ -16,9 +18,7 @@ func NewHistoryProcessorWrapper(waku *Waku) history.HistoryProcessor { } func (hr *HistoryProcessorWrapper) OnEnvelope(env *protocol.Envelope, processEnvelopes bool) error { - // TODO-nwaku - // return hr.waku.OnNewEnvelopes(env, common.StoreMessageType, processEnvelopes) - return nil + return hr.waku.OnNewEnvelopes(env, common.StoreMessageType, processEnvelopes) } func (hr *HistoryProcessorWrapper) OnRequestFailed(requestID []byte, peerID peer.ID, err error) { diff --git a/wakuv2/nwaku.go b/wakuv2/nwaku.go index 6deda931798..baa4e2e4987 100644 --- a/wakuv2/nwaku.go +++ b/wakuv2/nwaku.go @@ -286,7 +286,6 @@ import ( "github.com/golang/protobuf/proto" "github.com/jellydator/ttlcache/v3" "github.com/libp2p/go-libp2p/core/peer" - "github.com/libp2p/go-libp2p/core/peerstore" "github.com/multiformats/go-multiaddr" "go.uber.org/zap" @@ -300,6 +299,7 @@ import ( "github.com/ethereum/go-ethereum/event" "github.com/ethereum/go-ethereum/p2p" "github.com/ethereum/go-ethereum/p2p/enode" + "github.com/ethereum/go-ethereum/rpc" "github.com/libp2p/go-libp2p/core/metrics" @@ -342,7 +342,6 @@ const maxRelayPeers = 300 const randomPeersKeepAliveInterval = 5 * time.Second const allPeersKeepAliveInterval = 5 * time.Minute -/* TODO-nwaku type SentEnvelope struct { Envelope *protocol.Envelope PublishMethod publish.PublishMethod @@ -364,7 +363,6 @@ type ITelemetryClient interface { PushPeerCountByShard(ctx context.Context, peerCountByShard map[uint16]uint) PushPeerCountByOrigin(ctx context.Context, peerCountByOrigin map[wps.Origin]uint) } -*/ type WakuMessageHash = string type WakuPubsubTopic = string @@ -462,15 +460,14 @@ type Waku struct { onHistoricMessagesRequestFailed func([]byte, peer.ID, error) onPeerStats func(types.ConnStatus) - // statusTelemetryClient ITelemetryClient // TODO-nwaku + statusTelemetryClient ITelemetryClient defaultShardInfo protocol.RelayShards } -/* TODO-nwaku func (w *Waku) SetStatusTelemetryClient(client ITelemetryClient) { w.statusTelemetryClient = client -} */ +} func newTTLCache() *ttlcache.Cache[gethcommon.Hash, *common.ReceivedMessage] { cache := ttlcache.New[gethcommon.Hash, *common.ReceivedMessage](ttlcache.WithTTL[gethcommon.Hash, *common.ReceivedMessage](cacheTTL)) @@ -497,12 +494,12 @@ func New(nodeKey *ecdsa.PrivateKey, fleet string, cfg *Config, nwakuCfg *WakuCon defaultPubsubTopic, err := node.WakuDefaultPubsubTopic() if err != nil { - fmt.Println("Error happened:", err.Error()) + return nil, err } err = node.WakuRelaySubscribe(defaultPubsubTopic) if err != nil { - fmt.Println("Error happened:", err.Error()) + return nil, err } node.WakuSetEventCallback() @@ -1003,6 +1000,7 @@ func (w *Waku) subscribeToPubsubTopicWithWakuRelay(topic string, pubkey *ecdsa.P return nil } +*/ // MaxMessageSize returns the maximum accepted message size. func (w *Waku) MaxMessageSize() uint32 { @@ -1024,7 +1022,7 @@ func (w *Waku) APIs() []rpc.API { Public: false, }, } -} */ +} // Protocols returns the waku sub-protocols ran by this particular client. func (w *Waku) Protocols() []p2p.Protocol { @@ -1277,7 +1275,6 @@ func (w *Waku) GetSymKey(id string) ([]byte, error) { return nil, fmt.Errorf("non-existent key ID") } -/* TODO-nwaku // Subscribe installs a new message handler used for filtering, decrypting // and subsequent storing of incoming messages. func (w *Waku) Subscribe(f *common.Filter) (string, error) { @@ -1332,13 +1329,12 @@ func (w *Waku) SkipPublishToTopic(value bool) { func (w *Waku) ConfirmMessageDelivered(hashes []gethcommon.Hash) { w.messageSender.MessagesDelivered(hashes) -} */ +} -/* TODO-nwaku // OnNewEnvelope is an interface from Waku FilterManager API that gets invoked when any new message is received by Filter. func (w *Waku) OnNewEnvelope(env *protocol.Envelope) error { return w.OnNewEnvelopes(env, common.RelayedMessageType, false) -} */ +} // Start implements node.Service, starting the background data propagation thread // of the Waku protocol. @@ -1614,20 +1610,18 @@ func (w *Waku) startMessageSender() error { Event: common.EventEnvelopeSent, }) - // TODO-nwaku - /*if w.statusTelemetryClient != nil { + if w.statusTelemetryClient != nil { w.statusTelemetryClient.PushMessageCheckSuccess(w.ctx, hash.Hex()) - }*/ + } case hash := <-msgExpiredChan: w.SendEnvelopeEvent(common.EnvelopeEvent{ Hash: hash, Event: common.EventEnvelopeExpired, }) - // TODO-nwaku - /* if w.statusTelemetryClient != nil { + if w.statusTelemetryClient != nil { w.statusTelemetryClient.PushMessageCheckFailure(w.ctx, hash.Hex()) - }*/ + } } } }() @@ -2203,12 +2197,6 @@ func (w *Waku) PeerID() peer.ID { return "" } -// TODO-nwaku -func (w *Waku) Peerstore() peerstore.Peerstore { - // return w.node.Host().Peerstore() - return nil -} - // validatePrivateKey checks the format of the given private key. func validatePrivateKey(k *ecdsa.PrivateKey) bool { if k == nil || k.D == nil || k.D.Sign() == 0 { @@ -2539,7 +2527,7 @@ func (self *Waku) WakuRelaySubscribe(pubsubTopic string) error { defer C.free(unsafe.Pointer(cPubsubTopic)) if self.wakuCtx == nil { - fmt.Println("ctx is nil") + return errors.New("wakuCtx is nil") } // if self.cPubsubTopic == nil { // fmt.Println("cPubsubTopic is nil") @@ -2688,8 +2676,6 @@ func (self *Waku) ListenAddresses() ([]multiaddr.Multiaddr, error) { addrsRet = append(addrsRet, addr) } - fmt.Println("AAAAAA listen addresses: ", listenAddresses) - return addrsRet, nil } errMsg := "error WakuListenAddresses: " + @@ -2728,7 +2714,6 @@ func (self *Waku) ListPeersInMesh(pubsubTopic string) (int, error) { numPeersStr := C.GoStringN(C.getMyCharPtr(resp), C.int(C.getMyCharLen(resp))) numPeers, err := strconv.Atoi(numPeersStr) if err != nil { - fmt.Println(":", err) errMsg := "ListPeersInMesh - error converting string to int: " + err.Error() return 0, errors.New(errMsg) } @@ -2758,7 +2743,6 @@ func (self *Waku) GetNumConnectedPeers(paramPubsubTopic ...string) (int, error) numPeersStr := C.GoStringN(C.getMyCharPtr(resp), C.int(C.getMyCharLen(resp))) numPeers, err := strconv.Atoi(numPeersStr) if err != nil { - fmt.Println(":", err) errMsg := "GetNumConnectedPeers - error converting string to int: " + err.Error() return 0, errors.New(errMsg) } From 8c4c876de663f3e1ccf672f8982dbc6f5aa010a2 Mon Sep 17 00:00:00 2001 From: gabrielmer <101006718+gabrielmer@users.noreply.github.com> Date: Fri, 25 Oct 2024 11:03:07 +0300 Subject: [PATCH 08/14] feat: supporting peer exchange with nwaku (#5983) --- third_party/nwaku | 2 +- wakuv2/nwaku.go | 87 ++++++++++++++++++++----- wakuv2/nwaku_test.go | 151 +++++++++++++++++++++++++++++++++++++++++-- 3 files changed, 220 insertions(+), 20 deletions(-) diff --git a/third_party/nwaku b/third_party/nwaku index c5a825e206c..de11e576f4b 160000 --- a/third_party/nwaku +++ b/third_party/nwaku @@ -1 +1 @@ -Subproject commit c5a825e206c1cb3e6e0cf8c01410527804cb76c4 +Subproject commit de11e576f4b69b63b4135cfb9549ef15cdc1ad34 diff --git a/wakuv2/nwaku.go b/wakuv2/nwaku.go index baa4e2e4987..bcb32460a4c 100644 --- a/wakuv2/nwaku.go +++ b/wakuv2/nwaku.go @@ -208,6 +208,10 @@ package wakuv2 WAKU_CALL (waku_get_my_enr(ctx, (WakuCallBack) callback, resp) ); } + static void cGoWakuGetMyPeerId(void* ctx, void* resp) { + WAKU_CALL (waku_get_my_peerid(ctx, (WakuCallBack) callback, resp) ); + } + static void cGoWakuListPeersInMesh(void* ctx, char* pubSubTopic, void* resp) { WAKU_CALL (waku_relay_get_num_peers_in_mesh(ctx, pubSubTopic, (WakuCallBack) callback, resp) ); } @@ -216,6 +220,10 @@ package wakuv2 WAKU_CALL (waku_relay_get_num_connected_peers(ctx, pubSubTopic, (WakuCallBack) callback, resp) ); } + static void cGoWakuGetPeerIdsFromPeerStore(void* wakuCtx, void* resp) { + WAKU_CALL (waku_get_peerids_from_peerstore(wakuCtx, (WakuCallBack) callback, resp) ); + } + static void cGoWakuLightpushPublish(void* wakuCtx, const char* pubSubTopic, const char* jsonWakuMessage, @@ -380,8 +388,12 @@ type WakuConfig struct { Staticnodes []string `json:"staticnodes,omitempty"` Discv5BootstrapNodes []string `json:"discv5BootstrapNodes,omitempty"` Discv5Discovery bool `json:"discv5Discovery,omitempty"` + Discv5UdpPort uint16 `json:"discv5UdpPort,omitempty"` ClusterID uint16 `json:"clusterId,omitempty"` Shards []uint16 `json:"shards,omitempty"` + PeerExchange bool `json:"peerExchange,omitempty"` + PeerExchangeNode string `json:"peerExchangeNode,omitempty"` + TcpPort uint16 `json:"tcpPort,omitempty"` } // Waku represents a dark communication interface through the Ethereum @@ -497,9 +509,11 @@ func New(nodeKey *ecdsa.PrivateKey, fleet string, cfg *Config, nwakuCfg *WakuCon return nil, err } - err = node.WakuRelaySubscribe(defaultPubsubTopic) - if err != nil { - return nil, err + if nwakuCfg.EnableRelay { + err = node.WakuRelaySubscribe(defaultPubsubTopic) + if err != nil { + return nil, err + } } node.WakuSetEventCallback() @@ -2191,10 +2205,23 @@ func (w *Waku) Clean() error { return nil } -// TODO-nwaku -func (w *Waku) PeerID() peer.ID { - // return w.node.Host().ID() - return "" +func (w *Waku) PeerID() (peer.ID, error) { + var resp = C.allocResp() + defer C.freeResp(resp) + C.cGoWakuGetMyPeerId(w.wakuCtx, resp) + + if C.getRet(resp) == C.RET_OK { + + peerIdStr := C.GoStringN(C.getMyCharPtr(resp), C.int(C.getMyCharLen(resp))) + id, err := peer.Decode(peerIdStr) + if err != nil { + errMsg := "WakuGetMyPeerId - decoding peerId: %w" + return "", fmt.Errorf(errMsg, err) + } + return id, nil + } + errMsg := C.GoStringN(C.getMyCharPtr(resp), C.int(C.getMyCharLen(resp))) + return "", fmt.Errorf("WakuGetMyPeerId: %s", errMsg) } // validatePrivateKey checks the format of the given private key. @@ -2609,18 +2636,21 @@ func wakuStoreQuery( return "", errors.New(errMsg) } -func (self *Waku) WakuPeerExchangeRequest(numPeers uint64) (string, error) { +func (self *Waku) WakuPeerExchangeRequest(numPeers uint64) (uint64, error) { var resp = C.allocResp() defer C.freeResp(resp) C.cGoWakuPeerExchangeQuery(self.wakuCtx, C.uint64_t(numPeers), resp) if C.getRet(resp) == C.RET_OK { - msg := C.GoStringN(C.getMyCharPtr(resp), C.int(C.getMyCharLen(resp))) - return msg, nil + numRecvPeersStr := C.GoStringN(C.getMyCharPtr(resp), C.int(C.getMyCharLen(resp))) + numRecvPeers, err := strconv.ParseUint(numRecvPeersStr, 10, 64) + if err != nil { + return 0, err + } + return numRecvPeers, nil } - errMsg := "error WakuPeerExchangeRequest: " + - C.GoStringN(C.getMyCharPtr(resp), C.int(C.getMyCharLen(resp))) - return "", errors.New(errMsg) + errMsg := C.GoStringN(C.getMyCharPtr(resp), C.int(C.getMyCharLen(resp))) + return 0, fmt.Errorf("WakuPeerExchangeRequest: %s", errMsg) } func (self *Waku) WakuConnect(peerMultiAddr string, timeoutMs int) error { @@ -2753,6 +2783,34 @@ func (self *Waku) GetNumConnectedPeers(paramPubsubTopic ...string) (int, error) return 0, errors.New(errMsg) } +func (self *Waku) GetPeerIdsFromPeerStore() (peer.IDSlice, error) { + var resp = C.allocResp() + defer C.freeResp(resp) + C.cGoWakuGetPeerIdsFromPeerStore(self.wakuCtx, resp) + + if C.getRet(resp) == C.RET_OK { + peersStr := C.GoStringN(C.getMyCharPtr(resp), C.int(C.getMyCharLen(resp))) + if peersStr == "" { + return peer.IDSlice{}, nil + } + // peersStr contains a comma-separated list of peer ids + itemsPeerIds := strings.Split(peersStr, ",") + + var peers peer.IDSlice + for _, peerId := range itemsPeerIds { + id, err := peer.Decode(peerId) + if err != nil { + return nil, fmt.Errorf("GetPeerIdsFromPeerStore - decoding peerId: %w", err) + } + peers = append(peers, id) + } + + return peers, nil + } + errMsg := C.GoStringN(C.getMyCharPtr(resp), C.int(C.getMyCharLen(resp))) + return nil, fmt.Errorf("GetPeerIdsFromPeerStore: %s", errMsg) +} + func (self *Waku) GetPeerIdsByProtocol(protocol string) (peer.IDSlice, error) { var resp = C.allocResp() var cProtocol = C.CString(protocol) @@ -2773,8 +2831,7 @@ func (self *Waku) GetPeerIdsByProtocol(protocol string) (peer.IDSlice, error) { for _, p := range itemsPeerIds { id, err := peer.Decode(p) if err != nil { - errMsg := "GetPeerIdsByProtocol - error converting string to int: " + err.Error() - return nil, errors.New(errMsg) + return nil, fmt.Errorf("GetPeerIdsByProtocol - decoding peerId: %w", err) } peers = append(peers, id) } diff --git a/wakuv2/nwaku_test.go b/wakuv2/nwaku_test.go index fdf6f5458a9..417348e029c 100644 --- a/wakuv2/nwaku_test.go +++ b/wakuv2/nwaku_test.go @@ -13,6 +13,7 @@ import ( "github.com/cenkalti/backoff/v3" "github.com/libp2p/go-libp2p/core/peer" "github.com/waku-org/go-waku/waku/v2/protocol/store" + "go.uber.org/zap" "github.com/ethereum/go-ethereum/common/hexutil" "github.com/ethereum/go-ethereum/crypto" @@ -155,7 +156,7 @@ func parseNodes(rec []string) []*enode.Node { // IP_ADDRESS=$(hostname -I | awk '{print $1}'); // docker run \ // -p 61000:61000/tcp -p 8000:8000/udp -p 8646:8646/tcp harbor.status.im/wakuorg/nwaku:v0.33.0 \ -// --discv5-discovery=true --cluster-id=16 --log-level=DEBUG \ +// --discv5-discovery=true --cluster-id=16 --log-level=DEBUG --shard=64 --tcp-port=61000 \ // --nat=extip:${IP_ADDRESS} --discv5-udp-port=8000 --rest-address=0.0.0.0 --store --rest-port=8646 \ func TestBasicWakuV2(t *testing.T) { @@ -195,7 +196,6 @@ func TestBasicWakuV2(t *testing.T) { // Sanity check, not great, but it's probably helpful err = tt.RetryWithBackOff(func() error { - numConnected, err := w.GetNumConnectedPeers() if err != nil { return err @@ -323,10 +323,151 @@ func makeTestTree(domain string, nodes []*enode.Node, links []string) (*ethdnsdi return tree, url } -/* func TestPeerExchange(t *testing.T) { logger, err := zap.NewDevelopment() require.NoError(t, err) + + discV5NodeConfig := Config{ + UseThrottledPublish: true, + ClusterID: 16, + } + + // start node that will be discovered by PeerExchange + discV5NodeWakuConfig := WakuConfig{ + EnableRelay: true, + LogLevel: "DEBUG", + Discv5Discovery: true, + ClusterID: 16, + Shards: []uint16{64}, + PeerExchange: false, + Discv5UdpPort: 9001, + TcpPort: 60010, + } + + discV5Node, err := New(nil, "", &discV5NodeConfig, &discV5NodeWakuConfig, logger.Named("discV5Node"), nil, nil, nil, nil) + require.NoError(t, err) + require.NoError(t, discV5Node.Start()) + + time.Sleep(1 * time.Second) + + discV5NodePeerId, err := discV5Node.PeerID() + require.NoError(t, err) + + discv5NodeEnr, err := discV5Node.ENR() + require.NoError(t, err) + + pxServerConfig := Config{ + UseThrottledPublish: true, + ClusterID: 16, + } + + // start node which serves as PeerExchange server + pxServerWakuConfig := WakuConfig{ + EnableRelay: true, + LogLevel: "DEBUG", + Discv5Discovery: true, + ClusterID: 16, + Shards: []uint16{64}, + PeerExchange: true, + Discv5UdpPort: 9000, + Discv5BootstrapNodes: []string{discv5NodeEnr.String()}, + TcpPort: 60011, + } + + pxServerNode, err := New(nil, "", &pxServerConfig, &pxServerWakuConfig, logger.Named("pxServerNode"), nil, nil, nil, nil) + require.NoError(t, err) + require.NoError(t, pxServerNode.Start()) + + // Adding an extra second to make sure PX cache is not empty + time.Sleep(2 * time.Second) + + serverNodeMa, err := pxServerNode.ListenAddresses() + require.NoError(t, err) + require.NotNil(t, serverNodeMa) + + // Sanity check, not great, but it's probably helpful + options := func(b *backoff.ExponentialBackOff) { + b.MaxElapsedTime = 30 * time.Second + } + + // Check that pxServerNode has discV5Node in its Peer Store + err = tt.RetryWithBackOff(func() error { + peers, err := pxServerNode.GetPeerIdsFromPeerStore() + + if err != nil { + return err + } + + if slices.Contains(peers, discV5NodePeerId) { + return nil + } + + return errors.New("pxServer is missing the discv5 node in its peer store") + }, options) + require.NoError(t, err) + + pxClientConfig := Config{ + UseThrottledPublish: true, + ClusterID: 16, + } + + // start light node which uses PeerExchange to discover peers + pxClientWakuConfig := WakuConfig{ + EnableRelay: false, + LogLevel: "DEBUG", + Discv5Discovery: false, + ClusterID: 16, + Shards: []uint16{64}, + PeerExchange: true, + Discv5UdpPort: 9002, + TcpPort: 60012, + PeerExchangeNode: serverNodeMa[0].String(), + } + + lightNode, err := New(nil, "", &pxClientConfig, &pxClientWakuConfig, logger.Named("lightNode"), nil, nil, nil, nil) + require.NoError(t, err) + require.NoError(t, lightNode.Start()) + + time.Sleep(1 * time.Second) + + pxServerPeerId, err := pxServerNode.PeerID() + require.NoError(t, err) + + // Check that the light node discovered the discV5Node and has both nodes in its peer store + err = tt.RetryWithBackOff(func() error { + peers, err := lightNode.GetPeerIdsFromPeerStore() + if err != nil { + return err + } + + if slices.Contains(peers, discV5NodePeerId) && slices.Contains(peers, pxServerPeerId) { + return nil + } + return errors.New("lightnode is missing peers") + }, options) + require.NoError(t, err) + + // Now perform the PX request manually to see if it also works + err = tt.RetryWithBackOff(func() error { + numPeersReceived, err := lightNode.WakuPeerExchangeRequest(1) + if err != nil { + return err + } + + if numPeersReceived == 1 { + return nil + } + return errors.New("Peer Exchange is not returning peers") + }, options) + require.NoError(t, err) + + // Stop nodes + require.NoError(t, lightNode.Stop()) + require.NoError(t, pxServerNode.Stop()) + require.NoError(t, discV5Node.Stop()) + + /* logger, err := zap.NewDevelopment() + require.NoError(t, err) // start node which serve as PeerExchange server config := &Config{} config.ClusterID = 16 @@ -401,9 +542,11 @@ func TestPeerExchange(t *testing.T) { require.NoError(t, lightNode.Stop()) require.NoError(t, pxServerNode.Stop()) - require.NoError(t, discV5Node.Stop()) + require.NoError(t, discV5Node.Stop()) */ } +/* + func TestWakuV2Filter(t *testing.T) { t.Skip("flaky test") From bc9b21480b594c7bf8bb621a3437d14418cb0dc0 Mon Sep 17 00:00:00 2001 From: Gabriel mermelstein Date: Fri, 25 Oct 2024 12:04:00 +0300 Subject: [PATCH 09/14] implementing PeerCount and DropPeer --- wakuv2/nwaku.go | 39 ++++++++++++++++----------------------- wakuv2/nwaku_test.go | 2 +- 2 files changed, 17 insertions(+), 24 deletions(-) diff --git a/wakuv2/nwaku.go b/wakuv2/nwaku.go index bcb32460a4c..d53199a4b1e 100644 --- a/wakuv2/nwaku.go +++ b/wakuv2/nwaku.go @@ -1895,10 +1895,9 @@ func (w *Waku) ClearEnvelopesCache() { w.envelopeCache = newTTLCache() } -// TODO-nwaku func (w *Waku) PeerCount() int { - return 0 - // return w.node.PeerCount() + peerCount, _ := w.GetNumConnectedPeers() + return peerCount } // TODO-nwaku @@ -2177,10 +2176,20 @@ func (w *Waku) DialPeerByID(peerID peer.ID) error { return w.WakuDialPeerById(peerID, string(relay.WakuRelayID_v200), 1000) } -func (w *Waku) DropPeer(peerID peer.ID) error { - // TODO-nwaku - // return w.node.ClosePeerById(peerID) - return nil +func (self *Waku) DropPeer(peerId peer.ID) error { + var resp = C.allocResp() + var cPeerId = C.CString(peerId.String()) + defer C.freeResp(resp) + defer C.free(unsafe.Pointer(cPeerId)) + + C.cGoWakuDisconnectPeerById(self.wakuCtx, cPeerId, resp) + + if C.getRet(resp) == C.RET_OK { + return nil + } + errMsg := "error DisconnectPeerById: " + + C.GoStringN(C.getMyCharPtr(resp), C.int(C.getMyCharLen(resp))) + return errors.New(errMsg) } func (w *Waku) ProcessingP2PMessages() bool { @@ -2843,22 +2852,6 @@ func (self *Waku) GetPeerIdsByProtocol(protocol string) (peer.IDSlice, error) { return nil, errors.New(errMsg) } -func (self *Waku) DisconnectPeerById(peerId peer.ID) error { - var resp = C.allocResp() - var cPeerId = C.CString(peerId.String()) - defer C.freeResp(resp) - defer C.free(unsafe.Pointer(cPeerId)) - - C.cGoWakuDisconnectPeerById(self.wakuCtx, cPeerId, resp) - - if C.getRet(resp) == C.RET_OK { - return nil - } - errMsg := "error DisconnectPeerById: " + - C.GoStringN(C.getMyCharPtr(resp), C.int(C.getMyCharLen(resp))) - return errors.New(errMsg) -} - // func main() { // config := WakuConfig{ diff --git a/wakuv2/nwaku_test.go b/wakuv2/nwaku_test.go index 417348e029c..48d11f7aa82 100644 --- a/wakuv2/nwaku_test.go +++ b/wakuv2/nwaku_test.go @@ -219,7 +219,7 @@ func TestBasicWakuV2(t *testing.T) { require.True(t, slices.Contains(connectedStoreNodes, storeNode.ID), "nwaku should be connected to the store node") // Disconnect from the store node - err = w.DisconnectPeerById(storeNode.ID) + err = w.DropPeer(storeNode.ID) require.NoError(t, err) // Check that we are indeed disconnected From 341f915b9f2605b999bd8ca8ad72b8eabe15c710 Mon Sep 17 00:00:00 2001 From: Gabriel mermelstein Date: Fri, 25 Oct 2024 13:33:57 +0300 Subject: [PATCH 10/14] initial dialPeer implementation --- wakuv2/nwaku.go | 39 ++++++++++++++++++---- wakuv2/nwaku_test.go | 78 ++++++++++++++++++++++++++++++++++++++++++++ 2 files changed, 111 insertions(+), 6 deletions(-) diff --git a/wakuv2/nwaku.go b/wakuv2/nwaku.go index d53199a4b1e..6c7e0184d8a 100644 --- a/wakuv2/nwaku.go +++ b/wakuv2/nwaku.go @@ -179,6 +179,20 @@ package wakuv2 resp) ); } + static void cGoWakuDialPeer(void* wakuCtx, + char* peerMultiAddr, + char* protocol, + int timeoutMs, + void* resp) { + + WAKU_CALL( waku_dial_peer(wakuCtx, + peerMultiAddr, + protocol, + timeoutMs, + (WakuCallBack) callback, + resp) ); + } + static void cGoWakuDialPeerById(void* wakuCtx, char* peerId, char* protocol, @@ -2164,12 +2178,7 @@ func (w *Waku) AddRelayPeer(address multiaddr.Multiaddr) (peer.ID, error) { } func (w *Waku) DialPeer(address multiaddr.Multiaddr) error { - // TODO-nwaku - /* - ctx, cancel := context.WithTimeout(w.ctx, requestTimeout) - defer cancel() - return w.node.DialPeerWithMultiAddress(ctx, address) */ - return nil + return w.WakuDialPeer(address, string(relay.WakuRelayID_v200), 1000) } func (w *Waku) DialPeerByID(peerID peer.ID) error { @@ -2678,6 +2687,24 @@ func (self *Waku) WakuConnect(peerMultiAddr string, timeoutMs int) error { return errors.New(errMsg) } +func (self *Waku) WakuDialPeer(peerMultiAddr multiaddr.Multiaddr, protocol string, timeoutMs int) error { + var resp = C.allocResp() + var cPeerMultiAddr = C.CString(peerMultiAddr.String()) + var cProtocol = C.CString(protocol) + defer C.freeResp(resp) + defer C.free(unsafe.Pointer(cPeerMultiAddr)) + defer C.free(unsafe.Pointer(cProtocol)) + + C.cGoWakuDialPeer(self.wakuCtx, cPeerMultiAddr, cProtocol, C.int(timeoutMs), resp) + + if C.getRet(resp) == C.RET_OK { + return nil + } + errMsg := "error DialPeer: " + + C.GoStringN(C.getMyCharPtr(resp), C.int(C.getMyCharLen(resp))) + return errors.New(errMsg) +} + func (self *Waku) WakuDialPeerById(peerId peer.ID, protocol string, timeoutMs int) error { var resp = C.allocResp() var cPeerId = C.CString(peerId.String()) diff --git a/wakuv2/nwaku_test.go b/wakuv2/nwaku_test.go index 48d11f7aa82..85ff0a78683 100644 --- a/wakuv2/nwaku_test.go +++ b/wakuv2/nwaku_test.go @@ -545,6 +545,84 @@ func TestPeerExchange(t *testing.T) { require.NoError(t, discV5Node.Stop()) */ } +func TestDial(t *testing.T) { + logger, err := zap.NewDevelopment() + require.NoError(t, err) + + dialerNodeConfig := Config{ + UseThrottledPublish: true, + ClusterID: 16, + } + + // start node that will initiate the dial + dialerNodeWakuConfig := WakuConfig{ + EnableRelay: true, + LogLevel: "DEBUG", + Discv5Discovery: false, + ClusterID: 16, + Shards: []uint16{64}, + Discv5UdpPort: 9020, + TcpPort: 60020, + } + + dialerNode, err := New(nil, "", &dialerNodeConfig, &dialerNodeWakuConfig, logger.Named("dialerNode"), nil, nil, nil, nil) + require.NoError(t, err) + require.NoError(t, dialerNode.Start()) + + time.Sleep(1 * time.Second) + + receiverNodeConfig := Config{ + UseThrottledPublish: true, + ClusterID: 16, + } + + // start node that will receive the dial + receiverNodeWakuConfig := WakuConfig{ + EnableRelay: true, + LogLevel: "DEBUG", + Discv5Discovery: false, + ClusterID: 16, + Shards: []uint16{64}, + Discv5UdpPort: 9021, + TcpPort: 60021, + } + + receiverNode, err := New(nil, "", &receiverNodeConfig, &receiverNodeWakuConfig, logger.Named("receiverNode"), nil, nil, nil, nil) + require.NoError(t, err) + require.NoError(t, receiverNode.Start()) + + time.Sleep(1 * time.Second) + + receiverMultiaddr, err := receiverNode.ListenAddresses() + require.NoError(t, err) + require.NotNil(t, receiverMultiaddr) + + // Check that both nodes start with no connected peers + dialerPeerCount := dialerNode.PeerCount() + require.True(t, dialerPeerCount == 0, "Dialer node should have no connected peers") + + receiverPeerCount := receiverNode.PeerCount() + require.True(t, receiverPeerCount == 0, "Receiver node should have no connected peers") + + // Dial + err = dialerNode.DialPeer(receiverMultiaddr[0]) + require.NoError(t, err) + + time.Sleep(1 * time.Second) + + // Check that both nodes now have one connected peer + dialerPeerCount = dialerNode.PeerCount() + require.True(t, dialerPeerCount == 1, "Dialer node should have 1 peer") + + receiverPeerCount = receiverNode.PeerCount() + require.True(t, receiverPeerCount == 0, "Receiver node should have 1 peer") + + // Stop nodes + require.NoError(t, dialerNode.Stop()) + require.NoError(t, receiverNode.Stop()) + +} + /* func TestWakuV2Filter(t *testing.T) { From aa230335131cd1f787349ae28fea938f74d91394 Mon Sep 17 00:00:00 2001 From: Gabriel mermelstein Date: Mon, 28 Oct 2024 12:21:56 +0200 Subject: [PATCH 11/14] changing function name to specify that only connected relay peers are returned --- wakuv2/nwaku.go | 15 +++++++-------- wakuv2/nwaku_test.go | 5 ++++- 2 files changed, 11 insertions(+), 9 deletions(-) diff --git a/wakuv2/nwaku.go b/wakuv2/nwaku.go index 6c7e0184d8a..d913b7978dc 100644 --- a/wakuv2/nwaku.go +++ b/wakuv2/nwaku.go @@ -230,7 +230,7 @@ package wakuv2 WAKU_CALL (waku_relay_get_num_peers_in_mesh(ctx, pubSubTopic, (WakuCallBack) callback, resp) ); } - static void cGoWakuGetNumConnectedPeers(void* ctx, char* pubSubTopic, void* resp) { + static void cGoWakuGetNumConnectedRelayPeers(void* ctx, char* pubSubTopic, void* resp) { WAKU_CALL (waku_relay_get_num_connected_peers(ctx, pubSubTopic, (WakuCallBack) callback, resp) ); } @@ -338,7 +338,6 @@ import ( "github.com/waku-org/go-waku/waku/v2/protocol" "github.com/waku-org/go-waku/waku/v2/protocol/legacy_store" - "github.com/waku-org/go-waku/waku/v2/protocol/relay" "github.com/waku-org/go-waku/waku/v2/protocol/store" storepb "github.com/waku-org/go-waku/waku/v2/protocol/store/pb" "github.com/waku-org/go-waku/waku/v2/utils" @@ -2178,11 +2177,11 @@ func (w *Waku) AddRelayPeer(address multiaddr.Multiaddr) (peer.ID, error) { } func (w *Waku) DialPeer(address multiaddr.Multiaddr) error { - return w.WakuDialPeer(address, string(relay.WakuRelayID_v200), 1000) + return w.WakuDialPeer(address, "", 1000) } func (w *Waku) DialPeerByID(peerID peer.ID) error { - return w.WakuDialPeerById(peerID, string(relay.WakuRelayID_v200), 1000) + return w.WakuDialPeerById(peerID, "", 1000) } func (self *Waku) DropPeer(peerId peer.ID) error { @@ -2790,7 +2789,7 @@ func (self *Waku) ListPeersInMesh(pubsubTopic string) (int, error) { return 0, errors.New(errMsg) } -func (self *Waku) GetNumConnectedPeers(paramPubsubTopic ...string) (int, error) { +func (self *Waku) GetNumConnectedRelayPeers(paramPubsubTopic ...string) (int, error) { var pubsubTopic string if len(paramPubsubTopic) == 0 { pubsubTopic = "" @@ -2803,18 +2802,18 @@ func (self *Waku) GetNumConnectedPeers(paramPubsubTopic ...string) (int, error) defer C.freeResp(resp) defer C.free(unsafe.Pointer(cPubsubTopic)) - C.cGoWakuGetNumConnectedPeers(self.wakuCtx, cPubsubTopic, resp) + C.cGoWakuGetNumConnectedRelayPeers(self.wakuCtx, cPubsubTopic, resp) if C.getRet(resp) == C.RET_OK { numPeersStr := C.GoStringN(C.getMyCharPtr(resp), C.int(C.getMyCharLen(resp))) numPeers, err := strconv.Atoi(numPeersStr) if err != nil { - errMsg := "GetNumConnectedPeers - error converting string to int: " + err.Error() + errMsg := "GetNumConnectedRelayPeers - error converting string to int: " + err.Error() return 0, errors.New(errMsg) } return numPeers, nil } - errMsg := "error GetNumConnectedPeers: " + + errMsg := "error GetNumConnectedRelayPeers: " + C.GoStringN(C.getMyCharPtr(resp), C.int(C.getMyCharLen(resp))) return 0, errors.New(errMsg) } diff --git a/wakuv2/nwaku_test.go b/wakuv2/nwaku_test.go index 85ff0a78683..266720a6d6d 100644 --- a/wakuv2/nwaku_test.go +++ b/wakuv2/nwaku_test.go @@ -6,6 +6,7 @@ package wakuv2 import ( "context" "errors" + "fmt" "slices" "testing" "time" @@ -196,7 +197,7 @@ func TestBasicWakuV2(t *testing.T) { // Sanity check, not great, but it's probably helpful err = tt.RetryWithBackOff(func() error { - numConnected, err := w.GetNumConnectedPeers() + numConnected, err := w.GetNumConnectedRelayPeers() if err != nil { return err } @@ -612,6 +613,8 @@ func TestDial(t *testing.T) { // Check that both nodes now have one connected peer dialerPeerCount = dialerNode.PeerCount() + peerIds, _ := dialerNode.GetPeerIdsFromPeerStore() + fmt.Println("------------- dialerNode.GetPeerIdsFromPeerStore(): ", peerIds) require.True(t, dialerPeerCount == 1, "Dialer node should have 1 peer") receiverPeerCount = receiverNode.PeerCount() From d7290d48e2b8f47a9d12cf0666cd1e6f8bd4bca0 Mon Sep 17 00:00:00 2001 From: Gabriel mermelstein Date: Mon, 28 Oct 2024 12:43:57 +0200 Subject: [PATCH 12/14] integrating waku_get_connected_peers and fixing test --- wakuv2/nwaku.go | 40 ++++++++++++++++++++++++++++++++++++++++ wakuv2/nwaku_test.go | 5 +---- 2 files changed, 41 insertions(+), 4 deletions(-) diff --git a/wakuv2/nwaku.go b/wakuv2/nwaku.go index d913b7978dc..2ae3117bd7e 100644 --- a/wakuv2/nwaku.go +++ b/wakuv2/nwaku.go @@ -238,6 +238,10 @@ package wakuv2 WAKU_CALL (waku_get_peerids_from_peerstore(wakuCtx, (WakuCallBack) callback, resp) ); } + static void cGoWakuGetConnectedPeers(void* wakuCtx, void* resp) { + WAKU_CALL (waku_get_connected_peers(wakuCtx, (WakuCallBack) callback, resp) ); + } + static void cGoWakuLightpushPublish(void* wakuCtx, const char* pubSubTopic, const char* jsonWakuMessage, @@ -2818,6 +2822,42 @@ func (self *Waku) GetNumConnectedRelayPeers(paramPubsubTopic ...string) (int, er return 0, errors.New(errMsg) } +func (self *Waku) GetConnectedPeers() (peer.IDSlice, error) { + var resp = C.allocResp() + defer C.freeResp(resp) + C.cGoWakuGetConnectedPeers(self.wakuCtx, resp) + + if C.getRet(resp) == C.RET_OK { + peersStr := C.GoStringN(C.getMyCharPtr(resp), C.int(C.getMyCharLen(resp))) + if peersStr == "" { + return peer.IDSlice{}, nil + } + // peersStr contains a comma-separated list of peer ids + itemsPeerIds := strings.Split(peersStr, ",") + + var peers peer.IDSlice + for _, peerId := range itemsPeerIds { + id, err := peer.Decode(peerId) + if err != nil { + return nil, fmt.Errorf("GetConnectedPeers - decoding peerId: %w", err) + } + peers = append(peers, id) + } + + return peers, nil + } + errMsg := C.GoStringN(C.getMyCharPtr(resp), C.int(C.getMyCharLen(resp))) + return nil, fmt.Errorf("GetConnectedPeers: %s", errMsg) +} + +func (self *Waku) GetNumConnectedPeers() (int, error) { + connecterPeers, err := self.GetConnectedPeers() + if err != nil { + return 0, err + } + return len(connecterPeers), nil +} + func (self *Waku) GetPeerIdsFromPeerStore() (peer.IDSlice, error) { var resp = C.allocResp() defer C.freeResp(resp) diff --git a/wakuv2/nwaku_test.go b/wakuv2/nwaku_test.go index 266720a6d6d..10be1c6e20e 100644 --- a/wakuv2/nwaku_test.go +++ b/wakuv2/nwaku_test.go @@ -6,7 +6,6 @@ package wakuv2 import ( "context" "errors" - "fmt" "slices" "testing" "time" @@ -613,12 +612,10 @@ func TestDial(t *testing.T) { // Check that both nodes now have one connected peer dialerPeerCount = dialerNode.PeerCount() - peerIds, _ := dialerNode.GetPeerIdsFromPeerStore() - fmt.Println("------------- dialerNode.GetPeerIdsFromPeerStore(): ", peerIds) require.True(t, dialerPeerCount == 1, "Dialer node should have 1 peer") receiverPeerCount = receiverNode.PeerCount() - require.True(t, receiverPeerCount == 0, "Receiver node should have 1 peer") + require.True(t, receiverPeerCount == 1, "Receiver node should have 1 peer") // Stop nodes require.NoError(t, dialerNode.Stop()) From e1ab9d374d870f9e5eeb065477f2471a52e66aca Mon Sep 17 00:00:00 2001 From: Gabriel mermelstein Date: Mon, 28 Oct 2024 13:43:05 +0200 Subject: [PATCH 13/14] fix flaky test --- wakuv2/nwaku_test.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/wakuv2/nwaku_test.go b/wakuv2/nwaku_test.go index 10be1c6e20e..07bd6e7034f 100644 --- a/wakuv2/nwaku_test.go +++ b/wakuv2/nwaku_test.go @@ -232,6 +232,8 @@ func TestBasicWakuV2(t *testing.T) { err = w.DialPeerByID(storeNode.ID) require.NoError(t, err) + time.Sleep(1 * time.Second) + // Check that we are connected again connectedStoreNodes, err = w.GetPeerIdsByProtocol(string(store.StoreQueryID_v300)) require.NoError(t, err) From a87b625cf9aac90b5d780ad93457352fb02c2968 Mon Sep 17 00:00:00 2001 From: Gabriel mermelstein Date: Mon, 28 Oct 2024 17:49:40 +0200 Subject: [PATCH 14/14] implementing feedback --- wakuv2/nwaku.go | 14 +++++++++----- wakuv2/nwaku_test.go | 12 ++++++++---- 2 files changed, 17 insertions(+), 9 deletions(-) diff --git a/wakuv2/nwaku.go b/wakuv2/nwaku.go index 2ae3117bd7e..311caa66aa5 100644 --- a/wakuv2/nwaku.go +++ b/wakuv2/nwaku.go @@ -1912,9 +1912,13 @@ func (w *Waku) ClearEnvelopesCache() { w.envelopeCache = newTTLCache() } -func (w *Waku) PeerCount() int { - peerCount, _ := w.GetNumConnectedPeers() - return peerCount +func (w *Waku) PeerCount() (int, error) { + peerCount, err := w.GetNumConnectedPeers() + if err != nil { + return 0, err + } + + return peerCount, nil } // TODO-nwaku @@ -2181,11 +2185,11 @@ func (w *Waku) AddRelayPeer(address multiaddr.Multiaddr) (peer.ID, error) { } func (w *Waku) DialPeer(address multiaddr.Multiaddr) error { - return w.WakuDialPeer(address, "", 1000) + return w.WakuDialPeer(address, "", int(requestTimeout/time.Millisecond)) } func (w *Waku) DialPeerByID(peerID peer.ID) error { - return w.WakuDialPeerById(peerID, "", 1000) + return w.WakuDialPeerById(peerID, "", int(requestTimeout/time.Millisecond)) } func (self *Waku) DropPeer(peerId peer.ID) error { diff --git a/wakuv2/nwaku_test.go b/wakuv2/nwaku_test.go index 07bd6e7034f..8c2d1952ff1 100644 --- a/wakuv2/nwaku_test.go +++ b/wakuv2/nwaku_test.go @@ -600,10 +600,12 @@ func TestDial(t *testing.T) { require.NotNil(t, receiverMultiaddr) // Check that both nodes start with no connected peers - dialerPeerCount := dialerNode.PeerCount() + dialerPeerCount, err := dialerNode.PeerCount() + require.NoError(t, err) require.True(t, dialerPeerCount == 0, "Dialer node should have no connected peers") - receiverPeerCount := receiverNode.PeerCount() + receiverPeerCount, err := receiverNode.PeerCount() + require.NoError(t, err) require.True(t, receiverPeerCount == 0, "Receiver node should have no connected peers") // Dial @@ -613,10 +615,12 @@ func TestDial(t *testing.T) { time.Sleep(1 * time.Second) // Check that both nodes now have one connected peer - dialerPeerCount = dialerNode.PeerCount() + dialerPeerCount, err = dialerNode.PeerCount() + require.NoError(t, err) require.True(t, dialerPeerCount == 1, "Dialer node should have 1 peer") - receiverPeerCount = receiverNode.PeerCount() + receiverPeerCount, err = receiverNode.PeerCount() + require.NoError(t, err) require.True(t, receiverPeerCount == 1, "Receiver node should have 1 peer") // Stop nodes