go-waku/waku/v2/protocol/store/waku_store.go

624 lines
18 KiB
Go
Raw Normal View History

2021-03-18 16:40:47 +00:00
package store
import (
"bytes"
"context"
"crypto/sha256"
"encoding/hex"
"errors"
2021-03-22 16:45:13 +00:00
"fmt"
2021-06-10 13:00:06 +00:00
"math"
2021-03-18 16:40:47 +00:00
"sort"
"sync"
"time"
2021-03-22 16:45:13 +00:00
logging "github.com/ipfs/go-log"
2021-06-16 10:14:22 +00:00
"github.com/libp2p/go-libp2p-core/event"
2021-03-18 16:40:47 +00:00
"github.com/libp2p/go-libp2p-core/host"
"github.com/libp2p/go-libp2p-core/network"
"github.com/libp2p/go-libp2p-core/peer"
libp2pProtocol "github.com/libp2p/go-libp2p-core/protocol"
2021-04-07 21:16:29 +00:00
"github.com/libp2p/go-msgio/protoio"
"go.opencensus.io/stats"
"go.opencensus.io/tag"
2021-03-22 16:45:13 +00:00
"github.com/status-im/go-waku/waku/v2/metrics"
2021-03-18 16:40:47 +00:00
"github.com/status-im/go-waku/waku/v2/protocol"
2021-04-22 00:09:37 +00:00
"github.com/status-im/go-waku/waku/v2/protocol/pb"
2021-06-16 10:14:22 +00:00
"github.com/status-im/go-waku/waku/v2/utils"
2021-03-18 16:40:47 +00:00
)
2021-03-22 16:45:13 +00:00
var log = logging.Logger("wakustore")
const WakuStoreCodec = "/vac/waku/store/2.0.0-beta3"
const WakuStoreProtocolId = libp2pProtocol.ID(WakuStoreCodec)
2021-03-18 16:40:47 +00:00
const MaxPageSize = 100 // Maximum number of waku messages in each page
const DefaultContentTopic = "/waku/2/default-content/proto"
2021-03-18 16:40:47 +00:00
var (
2021-06-10 13:00:06 +00:00
ErrNoPeersAvailable = errors.New("no suitable remote peers")
ErrInvalidId = errors.New("invalid request id")
ErrFailedToResumeHistory = errors.New("failed to resume the history")
ErrFailedQuery = errors.New("failed to resolve the query")
)
2021-03-18 16:40:47 +00:00
func minOf(vars ...int) int {
min := vars[0]
for _, i := range vars {
if min > i {
min = i
}
}
return min
}
2021-04-22 00:09:37 +00:00
func paginateWithIndex(list []IndexedWakuMessage, pinfo *pb.PagingInfo) (resMessages []IndexedWakuMessage, resPagingInfo *pb.PagingInfo) {
2021-03-18 16:40:47 +00:00
// takes list, and performs paging based on pinfo
// returns the page i.e, a sequence of IndexedWakuMessage and the new paging info to be used for the next paging request
cursor := pinfo.Cursor
pageSize := pinfo.PageSize
dir := pinfo.Direction
if pageSize == 0 { // pageSize being zero indicates that no pagination is required
return list, pinfo
}
if len(list) == 0 { // no pagination is needed for an empty list
2021-04-22 00:09:37 +00:00
return list, &pb.PagingInfo{PageSize: 0, Cursor: pinfo.Cursor, Direction: pinfo.Direction}
2021-03-18 16:40:47 +00:00
}
msgList := make([]IndexedWakuMessage, len(list))
_ = copy(msgList, list) // makes a copy of the list
sort.Slice(msgList, func(i, j int) bool { // sorts msgList based on the custom comparison proc indexedWakuMessageComparison
return indexedWakuMessageComparison(msgList[i], msgList[j]) == -1
})
initQuery := false
if cursor == nil {
initQuery = true // an empty cursor means it is an initial query
switch dir {
2021-04-22 00:09:37 +00:00
case pb.PagingInfo_FORWARD:
2021-03-18 16:40:47 +00:00
cursor = list[0].index // perform paging from the begining of the list
2021-04-22 00:09:37 +00:00
case pb.PagingInfo_BACKWARD:
2021-03-18 16:40:47 +00:00
cursor = list[len(list)-1].index // perform paging from the end of the list
}
}
foundIndex := findIndex(msgList, cursor)
2021-03-18 16:40:47 +00:00
if foundIndex == -1 { // the cursor is not valid
2021-04-22 00:09:37 +00:00
return nil, &pb.PagingInfo{PageSize: 0, Cursor: pinfo.Cursor, Direction: pinfo.Direction}
2021-03-18 16:40:47 +00:00
}
var retrievedPageSize, s, e int
2021-04-22 00:09:37 +00:00
var newCursor *pb.Index // to be returned as part of the new paging info
2021-03-18 16:40:47 +00:00
switch dir {
2021-04-22 00:09:37 +00:00
case pb.PagingInfo_FORWARD: // forward pagination
2021-03-18 16:40:47 +00:00
remainingMessages := len(msgList) - foundIndex - 1
if initQuery {
remainingMessages = remainingMessages + 1
2021-03-18 16:40:47 +00:00
foundIndex = foundIndex - 1
}
// the number of queried messages cannot exceed the MaxPageSize and the total remaining messages i.e., msgList.len-foundIndex
retrievedPageSize = minOf(int(pageSize), MaxPageSize, remainingMessages)
2021-03-18 16:40:47 +00:00
s = foundIndex + 1 // non inclusive
e = foundIndex + retrievedPageSize
newCursor = msgList[e].index // the new cursor points to the end of the page
2021-04-22 00:09:37 +00:00
case pb.PagingInfo_BACKWARD: // backward pagination
2021-03-18 16:40:47 +00:00
remainingMessages := foundIndex
if initQuery {
remainingMessages = remainingMessages + 1
2021-03-18 16:40:47 +00:00
foundIndex = foundIndex + 1
}
// the number of queried messages cannot exceed the MaxPageSize and the total remaining messages i.e., foundIndex-0
retrievedPageSize = minOf(int(pageSize), MaxPageSize, remainingMessages)
2021-03-18 16:40:47 +00:00
s = foundIndex - retrievedPageSize
e = foundIndex - 1
newCursor = msgList[s].index // the new cursor points to the begining of the page
}
// retrieve the messages
for i := s; i <= e; i++ {
resMessages = append(resMessages, msgList[i])
}
2021-04-22 00:09:37 +00:00
resPagingInfo = &pb.PagingInfo{PageSize: uint64(retrievedPageSize), Cursor: newCursor, Direction: pinfo.Direction}
2021-03-18 16:40:47 +00:00
return
}
2021-04-22 00:09:37 +00:00
func paginateWithoutIndex(list []IndexedWakuMessage, pinfo *pb.PagingInfo) (resMessages []*pb.WakuMessage, resPinfo *pb.PagingInfo) {
2021-03-18 16:40:47 +00:00
// takes list, and performs paging based on pinfo
// returns the page i.e, a sequence of WakuMessage and the new paging info to be used for the next paging request
indexedData, updatedPagingInfo := paginateWithIndex(list, pinfo)
for _, indexedMsg := range indexedData {
resMessages = append(resMessages, indexedMsg.msg)
}
resPinfo = updatedPagingInfo
return
}
2021-04-22 00:09:37 +00:00
func (w *WakuStore) FindMessages(query *pb.HistoryQuery) *pb.HistoryResponse {
result := new(pb.HistoryResponse)
2021-03-18 16:40:47 +00:00
// data holds IndexedWakuMessage whose topics match the query
var data []IndexedWakuMessage
for _, indexedMsg := range w.messages {
// temporal filtering
// check whether the history query contains a time filter
if query.StartTime != 0 && query.EndTime != 0 {
if indexedMsg.msg.Timestamp < query.StartTime || indexedMsg.msg.Timestamp > query.EndTime {
continue
}
}
// filter based on content filters
// an empty list of contentFilters means no content filter is requested
if len(query.ContentFilters) != 0 {
match := false
for _, cf := range query.ContentFilters {
if cf.ContentTopic == indexedMsg.msg.ContentTopic {
match = true
break
}
}
if !match {
continue
}
}
// filter based on pubsub topic
// an empty pubsub topic means no pubsub topic filter is requested
if query.PubsubTopic != "" {
if indexedMsg.pubsubTopic != query.PubsubTopic {
continue
2021-04-20 21:46:35 +00:00
}
2021-03-18 16:40:47 +00:00
}
2021-04-20 21:46:35 +00:00
// Some criteria matched
data = append(data, indexedMsg)
2021-03-18 16:40:47 +00:00
}
result.Messages, result.PagingInfo = paginateWithoutIndex(data, query.PagingInfo)
return result
}
type StoredMessage struct {
ID []byte
PubsubTopic string
ReceiverTime int64
Message *pb.WakuMessage
}
2021-03-18 16:40:47 +00:00
type MessageProvider interface {
GetAll() ([]StoredMessage, error)
Put(cursor *pb.Index, pubsubTopic string, message *pb.WakuMessage) error
Stop()
2021-03-18 16:40:47 +00:00
}
type IndexedWakuMessage struct {
msg *pb.WakuMessage
index *pb.Index
pubsubTopic string
2021-03-18 16:40:47 +00:00
}
type WakuStore struct {
ctx context.Context
MsgC chan *protocol.Envelope
messages []IndexedWakuMessage
messageSet map[[32]byte]struct{}
2021-03-22 16:45:13 +00:00
messagesMutex sync.Mutex
2021-04-19 00:03:16 +00:00
storeMsgs bool
2021-03-22 16:45:13 +00:00
msgProvider MessageProvider
2021-03-18 16:40:47 +00:00
h host.Host
2021-06-16 10:14:22 +00:00
peerChan chan *event.EvtPeerConnectednessChanged
2021-03-18 16:40:47 +00:00
}
2021-04-22 00:09:37 +00:00
func NewWakuStore(shouldStoreMessages bool, p MessageProvider) *WakuStore {
2021-03-18 16:40:47 +00:00
wakuStore := new(WakuStore)
2021-04-22 00:09:37 +00:00
wakuStore.MsgC = make(chan *protocol.Envelope)
2021-03-18 16:40:47 +00:00
wakuStore.msgProvider = p
2021-04-19 00:03:16 +00:00
wakuStore.storeMsgs = shouldStoreMessages
2021-03-18 16:40:47 +00:00
return wakuStore
}
2021-03-18 16:40:47 +00:00
func (store *WakuStore) SetMsgProvider(p MessageProvider) {
store.msgProvider = p
}
2021-06-16 10:14:22 +00:00
func (store *WakuStore) peerListener() {
for e := range store.peerChan {
if e.Connectedness == network.NotConnected {
log.Info("Notification received ", e.Peer)
}
}
}
func (store *WakuStore) Start(ctx context.Context, h host.Host, peerChan chan *event.EvtPeerConnectednessChanged) {
store.h = h
store.ctx = ctx
2021-06-16 10:14:22 +00:00
store.peerChan = peerChan
2021-04-19 00:03:16 +00:00
if !store.storeMsgs {
log.Info("Store protocol started (messages aren't stored)")
2021-04-07 21:16:29 +00:00
return
}
store.h.SetStreamHandlerMatch(WakuStoreProtocolId, protocol.PrefixTextMatch(WakuStoreCodec), store.onRequest)
2021-03-18 16:40:47 +00:00
go store.storeIncomingMessages(ctx)
2021-04-19 00:03:16 +00:00
if store.msgProvider == nil {
log.Info("Store protocol started (no message provider)")
return
}
storedMessages, err := store.msgProvider.GetAll()
2021-03-22 16:45:13 +00:00
if err != nil {
log.Error("could not load DBProvider messages")
stats.RecordWithTags(ctx, []tag.Mutator{tag.Insert(metrics.KeyStoreErrorType, "store_load_failure")}, metrics.Errors.M(1))
2021-03-22 16:45:13 +00:00
return
}
for _, storedMessage := range storedMessages {
idx := &pb.Index{
Digest: storedMessage.ID,
ReceiverTime: float64(storedMessage.ReceiverTime),
2021-03-22 16:45:13 +00:00
}
store.storeMessageWithIndex(storedMessage.PubsubTopic, idx, storedMessage.Message)
stats.RecordWithTags(ctx, []tag.Mutator{tag.Insert(metrics.KeyType, "stored")}, metrics.StoreMessages.M(int64(len(store.messages))))
2021-03-22 16:45:13 +00:00
}
2021-06-16 10:14:22 +00:00
go store.peerListener()
log.Info("Store protocol started")
2021-03-18 16:40:47 +00:00
}
func (store *WakuStore) storeMessageWithIndex(pubsubTopic string, idx *pb.Index, msg *pb.WakuMessage) {
var k [32]byte
copy(k[:], idx.Digest)
if _, ok := store.messageSet[k]; ok {
return
}
store.messageSet[k] = struct{}{}
store.messages = append(store.messages, IndexedWakuMessage{msg: msg, index: idx, pubsubTopic: pubsubTopic})
}
2021-06-10 13:00:06 +00:00
func (store *WakuStore) storeMessage(pubSubTopic string, msg *pb.WakuMessage) {
index, err := computeIndex(msg)
if err != nil {
log.Error("could not calculate message index", err)
return
}
2021-03-18 16:40:47 +00:00
2021-06-10 13:00:06 +00:00
store.messagesMutex.Lock()
defer store.messagesMutex.Unlock()
store.storeMessageWithIndex(pubSubTopic, index, msg)
2021-03-18 16:40:47 +00:00
2021-06-10 13:00:06 +00:00
if store.msgProvider == nil {
return
}
2021-03-18 16:40:47 +00:00
2021-06-10 13:00:06 +00:00
err = store.msgProvider.Put(index, pubSubTopic, msg) // Should the index be stored?
2021-06-10 13:00:06 +00:00
if err != nil {
log.Error("could not store message", err)
stats.RecordWithTags(store.ctx, []tag.Mutator{tag.Insert(metrics.KeyStoreErrorType, "store_failure")}, metrics.Errors.M(1))
2021-06-10 13:00:06 +00:00
return
}
stats.RecordWithTags(store.ctx, []tag.Mutator{tag.Insert(metrics.KeyType, "stored")}, metrics.StoreMessages.M(int64(len(store.messages))))
2021-06-10 13:00:06 +00:00
}
func (store *WakuStore) storeIncomingMessages(ctx context.Context) {
2021-06-10 13:00:06 +00:00
for envelope := range store.MsgC {
store.storeMessage(envelope.PubsubTopic(), envelope.Message())
2021-03-18 16:40:47 +00:00
}
}
func (store *WakuStore) onRequest(s network.Stream) {
defer s.Close()
2021-04-22 00:09:37 +00:00
historyRPCRequest := &pb.HistoryRPC{}
2021-04-07 21:16:29 +00:00
writer := protoio.NewDelimitedWriter(s)
reader := protoio.NewDelimitedReader(s, 64*1024)
2021-04-07 21:16:29 +00:00
err := reader.ReadMsg(historyRPCRequest)
2021-03-18 16:40:47 +00:00
if err != nil {
2021-03-22 16:45:13 +00:00
log.Error("error reading request", err)
stats.RecordWithTags(store.ctx, []tag.Mutator{tag.Insert(metrics.KeyStoreErrorType, "decodeRPCFailure")}, metrics.Errors.M(1))
2021-03-18 16:40:47 +00:00
return
}
2021-03-22 16:45:13 +00:00
log.Info(fmt.Sprintf("%s: Received query from %s", s.Conn().LocalPeer(), s.Conn().RemotePeer()))
2021-03-18 16:40:47 +00:00
2021-04-22 00:09:37 +00:00
historyResponseRPC := &pb.HistoryRPC{}
2021-03-18 16:40:47 +00:00
historyResponseRPC.RequestId = historyRPCRequest.RequestId
historyResponseRPC.Response = store.FindMessages(historyRPCRequest.Query)
2021-04-07 21:16:29 +00:00
err = writer.WriteMsg(historyResponseRPC)
2021-03-18 16:40:47 +00:00
if err != nil {
2021-03-22 16:45:13 +00:00
log.Error("error writing response", err)
_ = s.Reset()
2021-03-18 16:40:47 +00:00
} else {
2021-03-22 16:45:13 +00:00
log.Info(fmt.Sprintf("%s: Response sent to %s", s.Conn().LocalPeer().String(), s.Conn().RemotePeer().String()))
2021-03-18 16:40:47 +00:00
}
}
2021-04-22 00:09:37 +00:00
func computeIndex(msg *pb.WakuMessage) (*pb.Index, error) {
2021-04-07 21:16:29 +00:00
data, err := msg.Marshal()
2021-03-18 16:40:47 +00:00
if err != nil {
return nil, err
}
digest := sha256.Sum256(data)
2021-04-22 00:09:37 +00:00
return &pb.Index{
2021-03-18 16:40:47 +00:00
Digest: digest[:],
ReceiverTime: float64(time.Now().UnixNano()),
SenderTime: msg.Timestamp,
2021-03-18 16:40:47 +00:00
}, nil
}
2021-04-22 00:09:37 +00:00
func indexComparison(x, y *pb.Index) int {
2021-03-18 16:40:47 +00:00
// compares x and y
// returns 0 if they are equal
// returns -1 if x < y
// returns 1 if x > y
var timecmp int = 0
if x.SenderTime > y.SenderTime {
2021-03-18 16:40:47 +00:00
timecmp = 1
} else if x.SenderTime < y.SenderTime {
2021-03-18 16:40:47 +00:00
timecmp = -1
}
digestcm := bytes.Compare(x.Digest, y.Digest)
if timecmp != 0 {
return timecmp // timestamp has a higher priority for comparison
}
return digestcm
}
func indexedWakuMessageComparison(x, y IndexedWakuMessage) int {
// compares x and y
// returns 0 if they are equal
// returns -1 if x < y
// returns 1 if x > y
return indexComparison(x.index, y.index)
}
2021-04-22 00:09:37 +00:00
func findIndex(msgList []IndexedWakuMessage, index *pb.Index) int {
2021-03-18 16:40:47 +00:00
// returns the position of an IndexedWakuMessage in msgList whose index value matches the given index
// returns -1 if no match is found
for i, indexedWakuMessage := range msgList {
if bytes.Equal(indexedWakuMessage.index.Digest, index.Digest) && indexedWakuMessage.index.ReceiverTime == index.ReceiverTime {
2021-03-18 16:40:47 +00:00
return i
}
}
return -1
}
type HistoryRequestParameters struct {
2021-04-15 17:55:40 +00:00
selectedPeer peer.ID
requestId []byte
2021-04-22 00:09:37 +00:00
cursor *pb.Index
pageSize uint64
asc bool
s *WakuStore
}
type HistoryRequestOption func(*HistoryRequestParameters)
2021-04-15 17:55:40 +00:00
func WithPeer(p peer.ID) HistoryRequestOption {
return func(params *HistoryRequestParameters) {
2021-04-15 17:55:40 +00:00
params.selectedPeer = p
}
}
func WithAutomaticPeerSelection() HistoryRequestOption {
return func(params *HistoryRequestParameters) {
2021-06-16 10:14:22 +00:00
p, err := utils.SelectPeer(params.s.h, string(WakuStoreProtocolId))
if err == nil {
params.selectedPeer = *p
} else {
log.Info("Error selecting peer: ", err)
}
}
}
func WithRequestId(requestId []byte) HistoryRequestOption {
return func(params *HistoryRequestParameters) {
params.requestId = requestId
}
}
func WithAutomaticRequestId() HistoryRequestOption {
return func(params *HistoryRequestParameters) {
params.requestId = protocol.GenerateRequestId()
}
}
2021-04-22 00:09:37 +00:00
func WithCursor(c *pb.Index) HistoryRequestOption {
return func(params *HistoryRequestParameters) {
params.cursor = c
}
}
func WithPaging(asc bool, pageSize uint64) HistoryRequestOption {
return func(params *HistoryRequestParameters) {
params.asc = asc
params.pageSize = pageSize
}
}
func DefaultOptions() []HistoryRequestOption {
return []HistoryRequestOption{
WithAutomaticRequestId(),
WithAutomaticPeerSelection(),
WithPaging(true, 0),
}
}
2021-06-10 13:00:06 +00:00
func (store *WakuStore) queryFrom(ctx context.Context, q *pb.HistoryQuery, selectedPeer peer.ID, requestId []byte) (*pb.HistoryResponse, error) {
connOpt, err := store.h.NewStream(ctx, selectedPeer, WakuStoreProtocolId)
if err != nil {
log.Info("failed to connect to remote peer", err)
return nil, err
}
defer connOpt.Close()
defer func() {
_ = connOpt.Reset()
}()
2021-06-10 13:00:06 +00:00
historyRequest := &pb.HistoryRPC{Query: q, RequestId: hex.EncodeToString(requestId)}
writer := protoio.NewDelimitedWriter(connOpt)
reader := protoio.NewDelimitedReader(connOpt, 64*1024)
err = writer.WriteMsg(historyRequest)
if err != nil {
log.Error("could not write request", err)
return nil, err
}
historyResponseRPC := &pb.HistoryRPC{}
err = reader.ReadMsg(historyResponseRPC)
if err != nil {
log.Error("could not read response", err)
stats.RecordWithTags(store.ctx, []tag.Mutator{tag.Insert(metrics.KeyStoreErrorType, "decodeRPCFailure")}, metrics.Errors.M(1))
2021-06-10 13:00:06 +00:00
return nil, err
}
stats.RecordWithTags(store.ctx, []tag.Mutator{tag.Insert(metrics.KeyType, "retrieved")}, metrics.StoreMessages.M(int64(len(store.messages))))
2021-06-10 13:00:06 +00:00
return historyResponseRPC.Response, nil
}
2021-04-22 00:09:37 +00:00
func (store *WakuStore) Query(ctx context.Context, q *pb.HistoryQuery, opts ...HistoryRequestOption) (*pb.HistoryResponse, error) {
params := new(HistoryRequestParameters)
params.s = store
optList := DefaultOptions()
optList = append(optList, opts...)
for _, opt := range optList {
opt(params)
}
2021-04-15 17:55:40 +00:00
if params.selectedPeer == "" {
return nil, ErrNoPeersAvailable
}
if len(params.requestId) == 0 {
return nil, ErrInvalidId
}
if params.cursor != nil {
q.PagingInfo.Cursor = params.cursor
}
if params.asc {
2021-04-22 00:09:37 +00:00
q.PagingInfo.Direction = pb.PagingInfo_FORWARD
} else {
2021-04-22 00:09:37 +00:00
q.PagingInfo.Direction = pb.PagingInfo_BACKWARD
2021-03-18 16:40:47 +00:00
}
q.PagingInfo.PageSize = params.pageSize
2021-03-18 16:40:47 +00:00
2021-06-10 13:00:06 +00:00
return store.queryFrom(ctx, q, params.selectedPeer, params.requestId)
}
func (store *WakuStore) queryLoop(ctx context.Context, query *pb.HistoryQuery, candidateList []peer.ID) (*pb.HistoryResponse, error) {
// loops through the candidateList in order and sends the query to each until one of the query gets resolved successfully
// returns the number of retrieved messages, or error if all the requests fail
for _, peer := range candidateList {
result, err := store.queryFrom(ctx, query, peer, protocol.GenerateRequestId())
if err != nil {
return result, nil
}
2021-03-18 16:40:47 +00:00
}
2021-06-10 13:00:06 +00:00
return nil, ErrFailedQuery
}
func (store *WakuStore) findLastSeen() float64 {
var lastSeenTime float64 = 0
for _, imsg := range store.messages {
if imsg.msg.Timestamp > lastSeenTime {
lastSeenTime = imsg.msg.Timestamp
}
}
return lastSeenTime
}
2021-03-18 16:40:47 +00:00
2021-06-10 13:00:06 +00:00
// resume proc retrieves the history of waku messages published on the default waku pubsub topic since the last time the waku store node has been online
// messages are stored in the store node's messages field and in the message db
// the offline time window is measured as the difference between the current time and the timestamp of the most recent persisted waku message
// an offset of 20 second is added to the time window to count for nodes asynchrony
// the history is fetched from one of the peers persisted in the waku store node's peer manager unit
// peerList indicates the list of peers to query from. The history is fetched from the first available peer in this list. Such candidates should be found through a discovery method (to be developed).
// if no peerList is passed, one of the peers in the underlying peer manager unit of the store protocol is picked randomly to fetch the history from. The history gets fetched successfully if the dialed peer has been online during the queried time window.
// the resume proc returns the number of retrieved messages if no error occurs, otherwise returns the error string
func (store *WakuStore) Resume(pubsubTopic string, peerList []peer.ID) (int, error) {
2021-06-10 13:00:06 +00:00
currentTime := float64(time.Now().UnixNano())
lastSeenTime := store.findLastSeen()
log.Info("resume ", int64(currentTime))
var offset float64 = 200000
currentTime = currentTime + offset
lastSeenTime = math.Max(lastSeenTime-offset, 0)
rpc := &pb.HistoryQuery{
PubsubTopic: pubsubTopic,
StartTime: lastSeenTime,
EndTime: currentTime,
PagingInfo: &pb.PagingInfo{
PageSize: 0,
Direction: pb.PagingInfo_BACKWARD,
},
}
var response *pb.HistoryResponse
if len(peerList) > 0 {
var err error
response, err = store.queryLoop(store.ctx, rpc, peerList)
2021-06-10 13:00:06 +00:00
if err != nil {
log.Error("failed to resume history", err)
return -1, ErrFailedToResumeHistory
}
} else {
2021-06-16 10:14:22 +00:00
p, err := utils.SelectPeer(store.h, string(WakuStoreProtocolId))
2021-04-07 21:16:29 +00:00
2021-06-16 10:14:22 +00:00
if err != nil {
log.Info("Error selecting peer: ", err)
2021-06-10 13:00:06 +00:00
return -1, ErrNoPeersAvailable
}
2021-03-18 16:40:47 +00:00
response, err = store.queryFrom(store.ctx, rpc, *p, protocol.GenerateRequestId())
2021-06-10 13:00:06 +00:00
if err != nil {
log.Error("failed to resume history", err)
return -1, ErrFailedToResumeHistory
}
2021-03-18 16:40:47 +00:00
}
2021-06-10 13:00:06 +00:00
for _, msg := range response.Messages {
store.storeMessage(pubsubTopic, msg)
2021-03-18 16:40:47 +00:00
}
2021-06-10 13:00:06 +00:00
return len(response.Messages), nil
2021-03-18 16:40:47 +00:00
}
// TODO: queryWithAccounting