chore: fetch contact with `StoreNodeRequestManager` (#4484)

This commit is contained in:
Igor Sirotin 2023-12-20 12:49:12 +00:00 committed by GitHub
parent 959dcbdea5
commit 92f0479a0c
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
8 changed files with 233 additions and 201 deletions

View File

@ -59,7 +59,7 @@ func (u *StatusUnfurler) buildContactData(publicKey string) (*common.StatusConta
// If no contact found locally, fetch it from waku // If no contact found locally, fetch it from waku
if contact == nil { if contact == nil {
if contact, err = u.m.RequestContactInfoFromMailserver(contactID, true); err != nil { if contact, err = u.m.FetchContact(contactID, true); err != nil {
return nil, fmt.Errorf("failed to request contact info from mailserver for public key '%s': %w", publicKey, err) return nil, fmt.Errorf("failed to request contact info from mailserver for public key '%s': %w", publicKey, err)
} }
} }

View File

@ -154,9 +154,6 @@ type Messenger struct {
once sync.Once once sync.Once
} }
requestedContactsLock sync.RWMutex
requestedContacts map[string]*transport.Filter
connectionState connection.State connectionState connection.State
telemetryClient *telemetry.Client telemetryClient *telemetry.Client
contractMaker *contracts.ContractMaker contractMaker *contracts.ContractMaker
@ -536,8 +533,6 @@ func NewMessenger(
quit: make(chan struct{}), quit: make(chan struct{}),
ctx: ctx, ctx: ctx,
cancel: cancel, cancel: cancel,
requestedContactsLock: sync.RWMutex{},
requestedContacts: make(map[string]*transport.Filter),
importingCommunities: make(map[string]bool), importingCommunities: make(map[string]bool),
importingChannels: make(map[string]bool), importingChannels: make(map[string]bool),
importRateLimiter: rate.NewLimiter(rate.Every(importSlowRate), 1), importRateLimiter: rate.NewLimiter(rate.Every(importSlowRate), 1),
@ -583,7 +578,7 @@ func NewMessenger(
} }
messenger.mentionsManager = NewMentionManager(messenger) messenger.mentionsManager = NewMentionManager(messenger)
messenger.storeNodeRequestsManager = NewCommunityRequestsManager(messenger) messenger.storeNodeRequestsManager = NewStoreNodeRequestManager(messenger)
if c.walletService != nil { if c.walletService != nil {
messenger.walletAPI = walletAPI messenger.walletAPI = walletAPI
@ -3745,12 +3740,10 @@ func (m *Messenger) handleRetrievedMessages(chatWithMessages map[transport.Filte
contact, contactFound := messageState.AllContacts.Load(senderID) contact, contactFound := messageState.AllContacts.Load(senderID)
if _, ok := m.requestedContacts[senderID]; !ok {
// Check for messages from blocked users // Check for messages from blocked users
if contactFound && contact.Blocked { if contactFound && contact.Blocked {
continue continue
} }
}
// Don't process duplicates // Don't process duplicates
messageID := types.EncodeHex(msg.ApplicationLayer.ID) messageID := types.EncodeHex(msg.ApplicationLayer.ID)
@ -3773,7 +3766,6 @@ func (m *Messenger) handleRetrievedMessages(chatWithMessages map[transport.Filte
contact = c contact = c
if msg.ApplicationLayer.Type != protobuf.ApplicationMetadataMessage_PUSH_NOTIFICATION_QUERY { if msg.ApplicationLayer.Type != protobuf.ApplicationMetadataMessage_PUSH_NOTIFICATION_QUERY {
messageState.AllContacts.Store(senderID, contact) messageState.AllContacts.Store(senderID, contact)
m.forgetContactInfoRequest(senderID)
} }
} }
messageState.CurrentMessageState = &CurrentMessageState{ messageState.CurrentMessageState = &CurrentMessageState{

View File

@ -5,7 +5,6 @@ import (
"crypto/ecdsa" "crypto/ecdsa"
"errors" "errors"
"fmt" "fmt"
"time"
"github.com/golang/protobuf/proto" "github.com/golang/protobuf/proto"
"go.uber.org/zap" "go.uber.org/zap"
@ -1218,93 +1217,9 @@ func (m *Messenger) scheduleSyncFiltersForContact(publicKey *ecdsa.PublicKey) (*
return filter, nil return filter, nil
} }
func (m *Messenger) RequestContactInfoFromMailserver(pubkey string, waitForResponse bool) (*Contact, error) { func (m *Messenger) FetchContact(contactID string, waitForResponse bool) (*Contact, error) {
contact, _, err := m.storeNodeRequestsManager.FetchContact(contactID, waitForResponse)
err := m.requestContactInfoFromMailserver(pubkey) return contact, err
if err != nil {
return nil, err
}
if !waitForResponse {
return nil, nil
}
ctx, cancel := context.WithTimeout(context.Background(), 15*time.Second)
defer func() {
cancel()
m.forgetContactInfoRequest(pubkey)
}()
for {
select {
case <-time.After(200 * time.Millisecond):
contact, ok := m.allContacts.Load(pubkey)
if ok && contact != nil && contact.DisplayName != "" {
return contact, nil
}
case <-ctx.Done():
return nil, fmt.Errorf("failed to request contact info from mailserver: %w", ctx.Err())
}
}
}
func (m *Messenger) requestContactInfoFromMailserver(pubkey string) error {
m.requestedContactsLock.Lock()
defer m.requestedContactsLock.Unlock()
if _, ok := m.requestedContacts[pubkey]; ok {
return nil
}
m.logger.Debug("requesting contact info from mailserver", zap.String("publicKey", pubkey))
c, err := buildContactFromPkString(pubkey)
if err != nil {
return err
}
publicKey, err := c.PublicKey()
if err != nil {
return err
}
var filter *transport.Filter
filter, err = m.scheduleSyncFiltersForContact(publicKey)
if err != nil {
return err
}
m.requestedContacts[pubkey] = filter
return nil
}
func (m *Messenger) forgetContactInfoRequest(publicKey string) {
m.requestedContactsLock.Lock()
defer m.requestedContactsLock.Unlock()
filter, ok := m.requestedContacts[publicKey]
if !ok {
return
}
m.logger.Debug("forgetting contact info request", zap.String("publicKey", publicKey))
err := m.transport.RemoveFilters([]*transport.Filter{filter})
if err != nil {
m.logger.Warn("failed to remove filter", zap.Error(err))
}
delete(m.requestedContacts, publicKey)
} }
func (m *Messenger) SubscribeToSelfContactChanges() chan *SelfContactChangeEvent { func (m *Messenger) SubscribeToSelfContactChanges() chan *SelfContactChangeEvent {

View File

@ -6,6 +6,9 @@ import (
"sync" "sync"
"time" "time"
"github.com/status-im/status-go/eth-node/crypto"
"github.com/status-im/status-go/protocol/common/shard"
"go.uber.org/zap" "go.uber.org/zap"
"github.com/status-im/status-go/eth-node/types" "github.com/status-im/status-go/eth-node/types"
@ -17,20 +20,25 @@ const (
storeNodeAvailableTimeout = 30 * time.Second storeNodeAvailableTimeout = 30 * time.Second
) )
// FetchCommunityStats is used in tests // StoreNodeRequestStats is used in tests
type FetchCommunityStats struct { type StoreNodeRequestStats struct {
FetchedEnvelopesCount int FetchedEnvelopesCount int
FetchedPagesCount int FetchedPagesCount int
} }
type storeNodeRequestID struct {
RequestType storeNodeRequestType `json:"requestType"`
DataID string `json:"dataID"`
}
type StoreNodeRequestManager struct { type StoreNodeRequestManager struct {
messenger *Messenger messenger *Messenger
logger *zap.Logger logger *zap.Logger
// activeRequests contain all ongoing store node requests. // activeRequests contain all ongoing store node requests.
// Map is indexed with `CommunityID`. // Map is indexed with `DataID`.
// Request might be duplicated in the map if the request is for multiple communities. // Request might be duplicated in the map in case of contentType collisions.
activeRequests map[string]*storeNodeRequest activeRequests map[storeNodeRequestID]*storeNodeRequest
// activeRequestsLock should be locked each time activeRequests is being accessed or changed. // activeRequestsLock should be locked each time activeRequests is being accessed or changed.
activeRequestsLock sync.RWMutex activeRequestsLock sync.RWMutex
@ -38,11 +46,11 @@ type StoreNodeRequestManager struct {
onPerformingBatch func(MailserverBatch) onPerformingBatch func(MailserverBatch)
} }
func NewCommunityRequestsManager(m *Messenger) *StoreNodeRequestManager { func NewStoreNodeRequestManager(m *Messenger) *StoreNodeRequestManager {
return &StoreNodeRequestManager{ return &StoreNodeRequestManager{
messenger: m, messenger: m,
logger: m.logger.Named("StoreNodeRequestManager"), logger: m.logger.Named("StoreNodeRequestManager"),
activeRequests: map[string]*storeNodeRequest{}, activeRequests: map[storeNodeRequestID]*storeNodeRequest{},
activeRequestsLock: sync.RWMutex{}, activeRequestsLock: sync.RWMutex{},
onPerformingBatch: nil, onPerformingBatch: nil,
} }
@ -53,18 +61,18 @@ func NewCommunityRequestsManager(m *Messenger) *StoreNodeRequestManager {
// the function will also wait for the store node response and return the fetched community. // the function will also wait for the store node response and return the fetched community.
// Automatically waits for an available store node. // Automatically waits for an available store node.
// When a `nil` community and `nil` error is returned, that means the community wasn't found at the store node. // When a `nil` community and `nil` error is returned, that means the community wasn't found at the store node.
func (m *StoreNodeRequestManager) FetchCommunity(community communities.CommunityShard, waitForResponse bool) (*communities.Community, FetchCommunityStats, error) { func (m *StoreNodeRequestManager) FetchCommunity(community communities.CommunityShard, waitForResponse bool) (*communities.Community, StoreNodeRequestStats, error) {
m.logger.Info("requesting community from store node", m.logger.Info("requesting community from store node",
zap.Any("community", community), zap.Any("community", community),
zap.Bool("waitForResponse", waitForResponse)) zap.Bool("waitForResponse", waitForResponse))
channel, err := m.subscribeToCommunityRequest(community) channel, err := m.subscribeToRequest(storeNodeCommunityRequest, community.CommunityID, community.Shard)
if err != nil { if err != nil {
return nil, FetchCommunityStats{}, fmt.Errorf("failed to create a request for community: %w", err) return nil, StoreNodeRequestStats{}, fmt.Errorf("failed to create a request for community: %w", err)
} }
if !waitForResponse { if !waitForResponse {
return nil, FetchCommunityStats{}, nil return nil, StoreNodeRequestStats{}, nil
} }
result := <-channel result := <-channel
@ -96,34 +104,64 @@ func (m *StoreNodeRequestManager) FetchCommunities(communities []communities.Com
return outErr return outErr
} }
// subscribeToCommunityRequest checks if a request for given community is already in progress, creates and installs func (m *StoreNodeRequestManager) FetchContact(contactID string, waitForResponse bool) (*Contact, StoreNodeRequestStats, error) {
m.logger.Info("requesting contact from store node",
zap.Any("contactID", contactID),
zap.Bool("waitForResponse", waitForResponse))
channel, err := m.subscribeToRequest(storeNodeContactRequest, contactID, nil)
if err != nil {
return nil, StoreNodeRequestStats{}, fmt.Errorf("failed to create a request for community: %w", err)
}
if !waitForResponse {
return nil, StoreNodeRequestStats{}, nil
}
result := <-channel
return result.contact, result.stats, result.err
}
// 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. // 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 or an error. // 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) subscribeToCommunityRequest(community communities.CommunityShard) (communitySubscriptionChannel, error) { func (m *StoreNodeRequestManager) subscribeToRequest(requestType storeNodeRequestType, dataID string, shard *shard.Shard) (storeNodeResponseSubscription, error) {
// It's important to unlock only after getting the subscription channel. // It's important to unlock only after getting the subscription channel.
// We also lock `activeRequestsLock` during finalizing the requests. This ensures that the subscription // 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. // created in this function will get the result even if the requests proceeds faster than this function ends.
m.activeRequestsLock.Lock() m.activeRequestsLock.Lock()
defer m.activeRequestsLock.Unlock() defer m.activeRequestsLock.Unlock()
request, requestFound := m.activeRequests[community.CommunityID] requestID := storeNodeRequestID{
RequestType: requestType,
DataID: dataID,
}
request, requestFound := m.activeRequests[requestID]
if !requestFound { if !requestFound {
// Create corresponding filter // Create corresponding filter
filter, filterCreated, err := m.getFilter(community) var err error
var filter *transport.Filter
filterCreated := false
filter, filterCreated, err = m.getFilter(requestType, dataID, shard)
if err != nil { if err != nil {
if filterCreated {
m.forgetFilter(filter)
}
return nil, fmt.Errorf("failed to create community filter: %w", err) return nil, fmt.Errorf("failed to create community filter: %w", err)
} }
request = m.newStoreNodeRequest() request = m.newStoreNodeRequest()
request.pubsubTopic = filter.PubsubTopic request.pubsubTopic = filter.PubsubTopic
request.communityID = community.CommunityID request.requestID = requestID
request.contentTopic = filter.ContentTopic request.contentTopic = filter.ContentTopic
if filterCreated { if filterCreated {
request.filterToForget = filter request.filterToForget = filter
} }
m.activeRequests[community.CommunityID] = request m.activeRequests[requestID] = request
request.start() request.start()
} }
@ -134,30 +172,32 @@ func (m *StoreNodeRequestManager) subscribeToCommunityRequest(community communit
func (m *StoreNodeRequestManager) newStoreNodeRequest() *storeNodeRequest { func (m *StoreNodeRequestManager) newStoreNodeRequest() *storeNodeRequest {
return &storeNodeRequest{ return &storeNodeRequest{
manager: m, manager: m,
subscriptions: make([]communitySubscriptionChannel, 0), subscriptions: make([]storeNodeResponseSubscription, 0),
} }
} }
// getFilter checks if a filter for a given community is already created and creates one of not found. // 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. // Returns the found/created filter, a flag if the filter was created by the function and an error.
func (m *StoreNodeRequestManager) getFilter(c communities.CommunityShard) (*transport.Filter, bool, error) { func (m *StoreNodeRequestManager) getFilter(requestType storeNodeRequestType, dataID string, shard *shard.Shard) (*transport.Filter, bool, error) {
// First check if such filter already exists. // First check if such filter already exists.
filter := m.messenger.transport.FilterByChatID(c.CommunityID) filter := m.messenger.transport.FilterByChatID(dataID)
if filter != nil { if filter != nil {
//we don't remember filter id associated with community because it was already installed //we don't remember filter id associated with community because it was already installed
return filter, false, nil return filter, false, nil
} }
switch requestType {
case storeNodeCommunityRequest:
// If filter wasn't installed we create it and // If filter wasn't installed we create it and
// remember for uninstalling after response is received // remember for uninstalling after response is received
filters, err := m.messenger.transport.InitPublicFilters([]transport.FiltersToInitialize{{ filters, err := m.messenger.transport.InitPublicFilters([]transport.FiltersToInitialize{{
ChatID: c.CommunityID, ChatID: dataID,
PubsubTopic: c.Shard.PubsubTopic(), PubsubTopic: shard.PubsubTopic(),
}}) }})
if err != nil { if err != nil {
m.logger.Error("can't install filter for community", zap.Error(err)) m.logger.Error("failed to install filter for community", zap.Error(err))
return nil, true, err return nil, false, err
} }
if len(filters) != 1 { if len(filters) != 1 {
@ -165,7 +205,29 @@ func (m *StoreNodeRequestManager) getFilter(c communities.CommunityShard) (*tran
return nil, true, fmt.Errorf("unexepcted number of filters created") return nil, true, fmt.Errorf("unexepcted number of filters created")
} }
return filters[0], true, nil filter = filters[0]
case storeNodeContactRequest:
publicKeyBytes, err := types.DecodeHex(dataID)
if err != nil {
return nil, false, fmt.Errorf("failed to decode contact id: %w", err)
}
publicKey, err := crypto.UnmarshalPubkey(publicKeyBytes)
if err != nil {
return nil, false, fmt.Errorf("failed to unmarshal public key: %w", err)
}
filter, err = m.messenger.transport.JoinPrivate(publicKey)
if err != nil {
return nil, false, fmt.Errorf("failed to install filter for contact: %w", err)
}
default:
return nil, false, fmt.Errorf("invalid store node request type: %d", requestType)
}
return filter, true, nil
} }
// forgetFilter uninstalls the given filter // forgetFilter uninstalls the given filter
@ -176,38 +238,49 @@ func (m *StoreNodeRequestManager) forgetFilter(filter *transport.Filter) {
} }
} }
type storeNodeRequestType int
const (
storeNodeCommunityRequest storeNodeRequestType = iota
storeNodeContactRequest
)
// storeNodeRequest represents a single store node batch request. // storeNodeRequest represents a single store node batch request.
// For a valid storeNodeRequest to be performed, the user must set all the struct fields and call start method. // For a valid storeNodeRequest to be performed, the user must set all the struct fields and call start method.
type storeNodeRequest struct { type storeNodeRequest struct {
requestID storeNodeRequestID
// request parameters // request parameters
pubsubTopic string pubsubTopic string
contentTopic types.TopicType contentTopic types.TopicType
// request corresponding metadata to be used in finalize // request corresponding metadata to be used in finalize
filterToForget *transport.Filter filterToForget *transport.Filter
communityID string
// internal fields // internal fields
manager *StoreNodeRequestManager manager *StoreNodeRequestManager
subscriptions []communitySubscriptionChannel subscriptions []storeNodeResponseSubscription
result fetchCommunityResult result storeNodeRequestResult
} }
// fetchCommunityResult contains result of a single storeNodeRequest // storeNodeRequestResult contains result of a single storeNodeRequest
// Further by using `data` we mean community/contact, depending on request type.
// If any error occurs during the request, err field will be set. // If any error occurs during the request, err field will be set.
// If a community was successfully fetched, community field will contain the fetched information. // If data was successfully fetched, data field will contain the fetched information.
// If a community wasn't found in store node, then a community will be set to `nil`. // If data wasn't found in store node, then a data will be set to `nil`.
// stats will contain information about the performed request that might be useful for testing. // stats will contain information about the performed request that might be useful for testing.
type fetchCommunityResult struct { type storeNodeRequestResult struct {
err error err error
stats StoreNodeRequestStats
// One of data fields (community or contact) will be present depending on request type
community *communities.Community community *communities.Community
stats FetchCommunityStats contact *Contact
} }
type communitySubscriptionChannel = chan fetchCommunityResult type storeNodeResponseSubscription = chan storeNodeRequestResult
func (r *storeNodeRequest) subscribe() communitySubscriptionChannel { func (r *storeNodeRequest) subscribe() storeNodeResponseSubscription {
channel := make(communitySubscriptionChannel, 100) channel := make(storeNodeResponseSubscription, 100)
r.subscriptions = append(r.subscriptions, channel) r.subscriptions = append(r.subscriptions, channel)
return channel return channel
} }
@ -217,8 +290,9 @@ func (r *storeNodeRequest) finalize() {
defer r.manager.activeRequestsLock.Unlock() defer r.manager.activeRequestsLock.Unlock()
r.manager.logger.Info("request finished", r.manager.logger.Info("request finished",
zap.String("communityID", r.communityID), zap.Any("requestID", r.requestID),
zap.Bool("communityFound", r.result.community != nil), zap.Bool("communityFound", r.result.community != nil),
zap.Bool("contactFound", r.result.contact != nil),
zap.Error(r.result.err)) zap.Error(r.result.err))
// Send the result to subscribers // Send the result to subscribers
@ -232,7 +306,7 @@ func (r *storeNodeRequest) finalize() {
r.manager.messenger.passStoredCommunityInfoToSignalHandler(r.result.community) r.manager.messenger.passStoredCommunityInfoToSignalHandler(r.result.community)
} }
delete(r.manager.activeRequests, r.communityID) delete(r.manager.activeRequests, r.requestID)
if r.filterToForget != nil { if r.filterToForget != nil {
r.manager.forgetFilter(r.filterToForget) r.manager.forgetFilter(r.filterToForget)
@ -241,7 +315,7 @@ func (r *storeNodeRequest) finalize() {
func (r *storeNodeRequest) shouldFetchNextPage(envelopesCount int) (bool, uint32) { func (r *storeNodeRequest) shouldFetchNextPage(envelopesCount int) (bool, uint32) {
logger := r.manager.logger.With( logger := r.manager.logger.With(
zap.String("communityID", r.communityID), zap.Any("requestID", r.requestID),
zap.Int("envelopesCount", envelopesCount)) zap.Int("envelopesCount", envelopesCount))
r.result.stats.FetchedEnvelopesCount += envelopesCount r.result.stats.FetchedEnvelopesCount += envelopesCount
@ -251,13 +325,15 @@ func (r *storeNodeRequest) shouldFetchNextPage(envelopesCount int) (bool, uint32
r.manager.messenger.ProcessAllMessages() r.manager.messenger.ProcessAllMessages()
// Try to get community from database // Try to get community from database
community, err := r.manager.messenger.communitiesManager.GetByIDString(r.communityID) switch r.requestID.RequestType {
case storeNodeCommunityRequest:
community, err := r.manager.messenger.communitiesManager.GetByIDString(r.requestID.DataID)
if err != nil { if err != nil {
logger.Error("failed to read from database", logger.Error("failed to read from database",
zap.String("communityID", r.communityID), zap.String("communityID", r.requestID.DataID),
zap.Error(err)) zap.Error(err))
r.result = fetchCommunityResult{ r.result = storeNodeRequestResult{
community: nil, community: nil,
err: fmt.Errorf("failed to read from database: %w", err), err: fmt.Errorf("failed to read from database: %w", err),
} }
@ -275,21 +351,37 @@ func (r *storeNodeRequest) shouldFetchNextPage(envelopesCount int) (bool, uint32
r.result.community = community r.result.community = community
case storeNodeContactRequest:
contact := r.manager.messenger.GetContactByID(r.requestID.DataID)
if contact == nil {
// contact not found in the database, request next page
logger.Debug("contact still not fetched")
return true, defaultStoreNodeRequestPageSize
}
logger.Debug("contact found",
zap.String("displayName", contact.DisplayName))
r.result.contact = contact
}
return false, 0 return false, 0
} }
func (r *storeNodeRequest) routine() { func (r *storeNodeRequest) routine() {
r.manager.logger.Info("starting store node request", r.manager.logger.Info("starting store node request",
zap.String("communityID", r.communityID), zap.Any("requestID", r.requestID),
zap.String("pubsubTopic", r.pubsubTopic), zap.String("pubsubTopic", r.pubsubTopic),
zap.Any("contentTopic", r.contentTopic), zap.Any("contentTopic", r.contentTopic),
) )
// Return a nil community and no error when request was // Return a nil community and no error when request was
// performed successfully, but no community found. // performed successfully, but no community/contact found.
r.result = fetchCommunityResult{ r.result = storeNodeRequestResult{
community: nil,
err: nil, err: nil,
community: nil,
contact: nil,
} }
defer func() { defer func() {
@ -297,7 +389,6 @@ func (r *storeNodeRequest) routine() {
}() }()
if !r.manager.messenger.waitForAvailableStoreNode(storeNodeAvailableTimeout) { if !r.manager.messenger.waitForAvailableStoreNode(storeNodeAvailableTimeout) {
r.result.community = nil
r.result.err = fmt.Errorf("store node is not available") r.result.err = fmt.Errorf("store node is not available")
return return
} }
@ -324,11 +415,5 @@ func (r *storeNodeRequest) routine() {
} }
func (r *storeNodeRequest) start() { func (r *storeNodeRequest) start() {
r.manager.logger.Debug("starting new community request",
zap.Any("communities", r.communityID),
zap.String("pubsubTopic", r.pubsubTopic),
zap.Any("contentTopic", r.contentTopic),
)
go r.routine() go r.routine()
} }

View File

@ -6,6 +6,8 @@ import (
"testing" "testing"
"time" "time"
"github.com/status-im/status-go/multiaccounts/accounts"
gethbridge "github.com/status-im/status-go/eth-node/bridge/geth" gethbridge "github.com/status-im/status-go/eth-node/bridge/geth"
"github.com/status-im/status-go/protocol/common" "github.com/status-im/status-go/protocol/common"
"github.com/status-im/status-go/protocol/communities" "github.com/status-im/status-go/protocol/communities"
@ -63,7 +65,7 @@ func (s *MessengerStoreNodeRequestSuite) SetupTest() {
s.cancel = make(chan struct{}, 10) s.cancel = make(chan struct{}, 10)
storeNodeLogger := s.logger.With(zap.String("name", "store-node-waku")) storeNodeLogger := s.logger.Named("store-node-waku")
s.wakuStoreNode = NewWakuV2(&s.Suite, storeNodeLogger, true, true) s.wakuStoreNode = NewWakuV2(&s.Suite, storeNodeLogger, true, true)
storeNodeListenAddresses := s.wakuStoreNode.ListenAddresses() storeNodeListenAddresses := s.wakuStoreNode.ListenAddresses()
@ -149,14 +151,20 @@ func (s *MessengerStoreNodeRequestSuite) createCommunity(m *Messenger) *communit
return response.Communities()[0] return response.Communities()[0]
} }
func (s *MessengerStoreNodeRequestSuite) requireCommunitiesEqual(fetchedCommunity *communities.Community, expectedCommunityInfo *communities.Community) { func (s *MessengerStoreNodeRequestSuite) requireCommunitiesEqual(c *communities.Community, expected *communities.Community) {
s.Require().Equal(expectedCommunityInfo.Name(), fetchedCommunity.Name()) s.Require().Equal(expected.Name(), c.Name())
s.Require().Equal(expectedCommunityInfo.Identity().Description, fetchedCommunity.Identity().Description) s.Require().Equal(expected.Identity().Description, c.Identity().Description)
s.Require().Equal(expectedCommunityInfo.Color(), fetchedCommunity.Color()) s.Require().Equal(expected.Color(), c.Color())
s.Require().Equal(expectedCommunityInfo.Tags(), fetchedCommunity.Tags()) s.Require().Equal(expected.Tags(), c.Tags())
} }
func (s *MessengerStoreNodeRequestSuite) fetchCommunity(m *Messenger, communityShard communities.CommunityShard, expectedCommunityInfo *communities.Community) FetchCommunityStats { func (s *MessengerStoreNodeRequestSuite) requireContactsEqual(c *Contact, expected *Contact) {
s.Require().Equal(expected.DisplayName, c.DisplayName)
s.Require().Equal(expected.Bio, c.Bio)
s.Require().Equal(expected.SocialLinks, c.SocialLinks)
}
func (s *MessengerStoreNodeRequestSuite) fetchCommunity(m *Messenger, communityShard communities.CommunityShard, expectedCommunityInfo *communities.Community) StoreNodeRequestStats {
fetchedCommunity, stats, err := m.storeNodeRequestsManager.FetchCommunity(communityShard, true) fetchedCommunity, stats, err := m.storeNodeRequestsManager.FetchCommunity(communityShard, true)
s.Require().NoError(err) s.Require().NoError(err)
@ -170,6 +178,17 @@ func (s *MessengerStoreNodeRequestSuite) fetchCommunity(m *Messenger, communityS
return stats return stats
} }
func (s *MessengerStoreNodeRequestSuite) fetchProfile(m *Messenger, contactID string, expectedContact *Contact) {
fetchedContact, err := m.FetchContact(contactID, true)
s.Require().NoError(err)
s.Require().NotNil(fetchedContact)
s.Require().Equal(contactID, fetchedContact.ID)
if expectedContact != nil {
s.requireContactsEqual(fetchedContact, expectedContact)
}
}
func (s *MessengerStoreNodeRequestSuite) waitForAvailableStoreNode(messenger *Messenger) { func (s *MessengerStoreNodeRequestSuite) waitForAvailableStoreNode(messenger *Messenger) {
WaitForAvailableStoreNode(&s.Suite, messenger, storeNodeConnectTimeout) WaitForAvailableStoreNode(&s.Suite, messenger, storeNodeConnectTimeout)
} }
@ -438,3 +457,22 @@ func (s *MessengerStoreNodeRequestSuite) TestRequestWithoutWaitingResponse() {
s.requireCommunitiesEqual(fetchedCommunities[community.IDString()], community) s.requireCommunitiesEqual(fetchedCommunities[community.IDString()], community)
} }
func (s *MessengerStoreNodeRequestSuite) TestRequestProfileInfo() {
s.createOwner()
// Set keypair (to be able to set displayName)
ownerProfileKp := accounts.GetProfileKeypairForTest(true, false, false)
ownerProfileKp.KeyUID = s.owner.account.KeyUID
ownerProfileKp.Accounts[0].KeyUID = s.owner.account.KeyUID
err := s.owner.settings.SaveOrUpdateKeypair(ownerProfileKp)
s.Require().NoError(err)
// Set display name, this will also publish contact code
err = s.owner.SetDisplayName("super-owner")
s.Require().NoError(err)
s.createBob()
s.fetchProfile(s.bob, s.owner.selfContact.ID, s.owner.selfContact)
}

View File

@ -353,7 +353,7 @@ func (api *PublicAPI) GetContactByID(parent context.Context, id string) *protoco
} }
func (api *PublicAPI) RequestContactInfoFromMailserver(pubkey string) (*protocol.Contact, error) { func (api *PublicAPI) RequestContactInfoFromMailserver(pubkey string) (*protocol.Contact, error) {
return api.service.messenger.RequestContactInfoFromMailserver(pubkey, true) return api.service.messenger.FetchContact(pubkey, true)
} }
func (api *PublicAPI) RemoveFilters(parent context.Context, chats []*transport.Filter) error { func (api *PublicAPI) RemoveFilters(parent context.Context, chats []*transport.Filter) error {

View File

@ -19,9 +19,10 @@
package wakuv2 package wakuv2
import ( import (
ethdisc "github.com/ethereum/go-ethereum/p2p/dnsdisc"
"github.com/waku-org/go-waku/waku/v2/protocol/relay" "github.com/waku-org/go-waku/waku/v2/protocol/relay"
ethdisc "github.com/ethereum/go-ethereum/p2p/dnsdisc"
"github.com/status-im/status-go/wakuv2/common" "github.com/status-im/status-go/wakuv2/common"
) )

View File

@ -54,7 +54,6 @@ import (
pubsub "github.com/libp2p/go-libp2p-pubsub" pubsub "github.com/libp2p/go-libp2p-pubsub"
"github.com/libp2p/go-libp2p/core/metrics" "github.com/libp2p/go-libp2p/core/metrics"
ethdisc "github.com/ethereum/go-ethereum/p2p/dnsdisc"
"github.com/waku-org/go-waku/waku/v2/dnsdisc" "github.com/waku-org/go-waku/waku/v2/dnsdisc"
wps "github.com/waku-org/go-waku/waku/v2/peerstore" 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"
@ -62,6 +61,8 @@ import (
"github.com/waku-org/go-waku/waku/v2/protocol/peer_exchange" "github.com/waku-org/go-waku/waku/v2/protocol/peer_exchange"
"github.com/waku-org/go-waku/waku/v2/protocol/relay" "github.com/waku-org/go-waku/waku/v2/protocol/relay"
ethdisc "github.com/ethereum/go-ethereum/p2p/dnsdisc"
"github.com/status-im/status-go/connection" "github.com/status-im/status-go/connection"
"github.com/status-im/status-go/eth-node/types" "github.com/status-im/status-go/eth-node/types"
"github.com/status-im/status-go/timesource" "github.com/status-im/status-go/timesource"