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

729 lines
21 KiB
Go
Raw Normal View History

2021-03-18 16:40:47 +00:00
package store
import (
"bytes"
"context"
"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-18 16:40:47 +00:00
2021-03-22 16:45:13 +00:00
logging "github.com/ipfs/go-log"
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"
2021-03-22 16:45:13 +00:00
2021-10-25 19:41:08 +00:00
"github.com/status-im/go-waku/waku/persistence"
"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"
"github.com/status-im/go-waku/waku/v2/protocol/swap"
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")
// StoreID_v20beta3 is the current Waku Store protocol identifier
2021-09-30 15:59:51 +00:00
const StoreID_v20beta3 = libp2pProtocol.ID("/vac/waku/store/2.0.0-beta3")
// MaxPageSize is the maximum number of waku messages to return per page
const MaxPageSize = 100
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-10-25 19:41:08 +00:00
if pinfo == nil {
pinfo = new(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-10-09 18:18:53 +00:00
cursor = list[0].index // perform paging from the beginning 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
2021-10-09 18:18:53 +00:00
newCursor = msgList[s].index // the new cursor points to the beginning of the page
2021-03-18 16:40:47 +00:00
}
// 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
}
func (store *WakuStore) FindMessages(query *pb.HistoryQuery) *pb.HistoryResponse {
2021-04-22 00:09:37 +00:00
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 store.messageQueue.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 MessageProvider interface {
2021-10-25 19:41:08 +00:00
GetAll() ([]persistence.StoredMessage, error)
Put(cursor *pb.Index, pubsubTopic string, message *pb.WakuMessage) error
Stop()
2021-03-18 16:40:47 +00:00
}
type Query struct {
Topic string
ContentTopics []string
StartTime float64
EndTime float64
}
// Result represents a valid response from a store node
type Result struct {
Messages []*pb.WakuMessage
query *pb.HistoryQuery
cursor *pb.Index
peerId peer.ID
}
func (r *Result) Cursor() *pb.Index {
return r.cursor
}
func (r *Result) PeerID() peer.ID {
return r.peerId
}
func (r *Result) Query() *pb.HistoryQuery {
return r.query
}
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
wg *sync.WaitGroup
2021-11-01 12:38:03 +00:00
started bool
messageQueue *MessageQueue
msgProvider MessageProvider
h host.Host
swap *swap.WakuSwap
2021-03-18 16:40:47 +00:00
}
// NewWakuStore creates a WakuStore using an specific MessageProvider for storing the messages
func NewWakuStore(host host.Host, swap *swap.WakuSwap, p MessageProvider, maxNumberOfMessages int, maxRetentionDuration time.Duration) *WakuStore {
2021-03-18 16:40:47 +00:00
wakuStore := new(WakuStore)
wakuStore.msgProvider = p
wakuStore.h = host
wakuStore.swap = swap
wakuStore.wg = &sync.WaitGroup{}
wakuStore.messageQueue = NewMessageQueue(maxNumberOfMessages, maxRetentionDuration)
return wakuStore
}
2021-03-18 16:40:47 +00:00
// SetMessageProvider allows switching the message provider used with a WakuStore
func (store *WakuStore) SetMessageProvider(p MessageProvider) {
store.msgProvider = p
}
// Start initializes the WakuStore by enabling the protocol and fetching records from a message provider
func (store *WakuStore) Start(ctx context.Context) {
2021-11-01 12:38:03 +00:00
if store.started {
2021-04-07 21:16:29 +00:00
return
}
2021-11-01 12:38:03 +00:00
store.started = true
store.ctx = ctx
2021-11-04 16:19:59 +00:00
store.MsgC = make(chan *protocol.Envelope, 1024)
2021-11-01 12:38:03 +00:00
2021-09-30 15:59:51 +00:00
store.h.SetStreamHandlerMatch(StoreID_v20beta3, protocol.PrefixTextMatch(string(StoreID_v20beta3)), store.onRequest)
2021-03-18 16:40:47 +00:00
store.wg.Add(1)
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
}
2021-10-25 19:41:08 +00:00
store.fetchDBRecords(ctx)
log.Info("Store protocol started")
}
func (store *WakuStore) fetchDBRecords(ctx context.Context) {
2021-11-01 12:38:03 +00:00
if store.msgProvider == nil {
return
}
storedMessages, err := (store.msgProvider).GetAll()
2021-03-22 16:45:13 +00:00
if err != nil {
log.Error("could not load DBProvider messages", err)
metrics.RecordStoreError(ctx, "store_load_failure")
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)
metrics.RecordMessage(ctx, "stored", store.messageQueue.Length())
2021-03-22 16:45:13 +00:00
}
2021-03-18 16:40:47 +00:00
}
func (store *WakuStore) storeMessageWithIndex(pubsubTopic string, idx *pb.Index, msg *pb.WakuMessage) {
store.messageQueue.Push(IndexedWakuMessage{msg: msg, index: idx, pubsubTopic: pubsubTopic})
}
func (store *WakuStore) storeMessage(env *protocol.Envelope) {
index, err := computeIndex(env)
2021-06-10 13:00:06 +00:00
if err != nil {
log.Error("could not calculate message index", err)
return
}
2021-03-18 16:40:47 +00:00
store.storeMessageWithIndex(env.PubsubTopic(), index, env.Message())
2021-03-18 16:40:47 +00:00
2021-06-10 13:00:06 +00:00
if store.msgProvider == nil {
metrics.RecordMessage(store.ctx, "stored", store.messageQueue.Length())
2021-06-10 13:00:06 +00:00
return
}
2021-03-18 16:40:47 +00:00
// TODO: Move this to a separate go routine if DB writes becomes a bottleneck
err = store.msgProvider.Put(index, env.PubsubTopic(), env.Message()) // Should the index be stored?
2021-06-10 13:00:06 +00:00
if err != nil {
log.Error("could not store message", err)
metrics.RecordStoreError(store.ctx, "store_failure")
2021-06-10 13:00:06 +00:00
return
}
metrics.RecordMessage(store.ctx, "stored", store.messageQueue.Length())
2021-06-10 13:00:06 +00:00
}
func (store *WakuStore) storeIncomingMessages(ctx context.Context) {
defer store.wg.Done()
2021-06-10 13:00:06 +00:00
for envelope := range store.MsgC {
store.storeMessage(envelope)
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, math.MaxInt32)
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)
metrics.RecordStoreError(store.ctx, "decodeRPCFailure")
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
}
}
func computeIndex(env *protocol.Envelope) (*pb.Index, error) {
2021-04-22 00:09:37 +00:00
return &pb.Index{
Digest: env.Hash(),
2021-10-12 13:12:54 +00:00
ReceiverTime: utils.GetUnixEpoch(),
SenderTime: env.Message().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)
// WithPeer is an option used to specify the peerID to request the message history
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
}
}
// WithAutomaticPeerSelection is an option used to randomly select a peer from the store
// to request the message history
func WithAutomaticPeerSelection() HistoryRequestOption {
return func(params *HistoryRequestParameters) {
2021-09-30 15:59:51 +00:00
p, err := utils.SelectPeer(params.s.h, string(StoreID_v20beta3))
2021-06-16 10:14:22 +00:00
if err == nil {
params.selectedPeer = *p
} else {
log.Info("Error selecting peer: ", err)
}
}
}
func WithFastestPeerSelection(ctx context.Context) HistoryRequestOption {
return func(params *HistoryRequestParameters) {
p, err := utils.SelectPeerWithLowestRTT(ctx, params.s.h, string(StoreID_v20beta3))
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
}
}
// WithPaging is an option used to specify the order and maximum number of records to return
func WithPaging(asc bool, pageSize uint64) HistoryRequestOption {
return func(params *HistoryRequestParameters) {
params.asc = asc
params.pageSize = pageSize
}
}
// Default options to be used when querying a store node for results
func DefaultOptions() []HistoryRequestOption {
return []HistoryRequestOption{
WithAutomaticRequestId(),
WithAutomaticPeerSelection(),
WithPaging(true, MaxPageSize),
}
}
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) {
2021-11-04 16:19:59 +00:00
log.Info(fmt.Sprintf("Querying message history with peer %s", selectedPeer))
2021-09-30 15:59:51 +00:00
connOpt, err := store.h.NewStream(ctx, selectedPeer, StoreID_v20beta3)
2021-06-10 13:00:06 +00:00
if err != nil {
log.Error("Failed to connect to remote peer", err)
2021-06-10 13:00:06 +00:00
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, math.MaxInt32)
2021-06-10 13:00:06 +00:00
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)
metrics.RecordStoreError(store.ctx, "decodeRPCFailure")
2021-06-10 13:00:06 +00:00
return nil, err
}
metrics.RecordMessage(ctx, "retrieved", store.messageQueue.Length())
2021-06-10 13:00:06 +00:00
return historyResponseRPC.Response, nil
}
func (store *WakuStore) Query(ctx context.Context, query Query, opts ...HistoryRequestOption) (*Result, error) {
q := &pb.HistoryQuery{
PubsubTopic: query.Topic,
ContentFilters: []*pb.ContentFilter{},
StartTime: query.StartTime,
EndTime: query.EndTime,
PagingInfo: &pb.PagingInfo{},
}
for _, cf := range query.ContentTopics {
q.ContentFilters = append(q.ContentFilters, &pb.ContentFilter{ContentTopic: cf})
}
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
response, err := store.queryFrom(ctx, q, params.selectedPeer, params.requestId)
if err != nil {
return nil, err
}
if response.Error == pb.HistoryResponse_INVALID_CURSOR {
return nil, errors.New("invalid cursor")
}
return &Result{
Messages: response.Messages,
cursor: response.PagingInfo.Cursor,
query: q,
peerId: params.selectedPeer,
}, nil
}
// Next is used with to retrieve the next page of rows from a query response.
// If no more records are found, the result will not contain any messages.
// This function is useful for iterating over results without having to manually
// specify the cursor and pagination order and max number of results
func (store *WakuStore) Next(ctx context.Context, r *Result) (*Result, error) {
q := &pb.HistoryQuery{
PubsubTopic: r.query.PubsubTopic,
ContentFilters: r.query.ContentFilters,
StartTime: r.query.StartTime,
EndTime: r.query.EndTime,
PagingInfo: &pb.PagingInfo{
PageSize: r.query.PagingInfo.PageSize,
Direction: r.query.PagingInfo.Direction,
Cursor: &pb.Index{
Digest: r.cursor.Digest,
ReceiverTime: r.cursor.ReceiverTime,
SenderTime: r.cursor.SenderTime,
},
},
}
response, err := store.queryFrom(ctx, q, r.peerId, protocol.GenerateRequestId())
if err != nil {
return nil, err
}
if response.Error == pb.HistoryResponse_INVALID_CURSOR {
return nil, errors.New("invalid cursor")
}
return &Result{
Messages: response.Messages,
cursor: response.PagingInfo.Cursor,
query: q,
peerId: r.peerId,
}, nil
2021-06-10 13:00:06 +00:00
}
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 {
2021-06-10 13:00:06 +00:00
return result, nil
}
log.Error(fmt.Errorf("resume history with peer %s failed: %w", peer, err))
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.messageQueue.Messages() {
2021-06-10 13:00:06 +00:00
if imsg.msg.Timestamp > lastSeenTime {
lastSeenTime = imsg.msg.Timestamp
}
}
return lastSeenTime
}
2021-03-18 16:40:47 +00:00
// Resume retrieves the history of waku messages published on the default waku pubsub topic since the last time the waku store node has been online
2021-06-10 13:00:06 +00:00
// 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
2021-10-09 18:18:53 +00:00
func (store *WakuStore) Resume(ctx context.Context, pubsubTopic string, peerList []peer.ID) (int, error) {
2021-11-01 12:38:03 +00:00
if !store.started {
return 0, errors.New("can't resume: store has not started")
}
2021-10-12 13:12:54 +00:00
currentTime := utils.GetUnixEpoch()
2021-06-10 13:00:06 +00:00
lastSeenTime := store.findLastSeen()
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,
},
}
2021-04-07 21:16:29 +00:00
2021-12-08 14:21:30 +00:00
if len(peerList) == 0 {
p, err := utils.SelectPeer(store.h, string(StoreID_v20beta3))
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
2021-12-08 14:21:30 +00:00
peerList = append(peerList, *p)
}
response, err := store.queryLoop(ctx, rpc, peerList)
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(protocol.NewEnvelope(msg, pubsubTopic))
2021-03-18 16:40:47 +00:00
}
2021-11-01 12:38:03 +00:00
log.Info("Retrieved messages since the last online time: ", len(response.Messages))
2021-06-10 13:00:06 +00:00
return len(response.Messages), nil
2021-03-18 16:40:47 +00:00
}
// TODO: queryWithAccounting
// Stop closes the store message channel and removes the protocol stream handler
func (store *WakuStore) Stop() {
store.started = false
2021-11-01 14:42:55 +00:00
if store.MsgC != nil {
close(store.MsgC)
2021-11-01 14:42:55 +00:00
}
if store.h != nil {
store.h.RemoveStreamHandler(StoreID_v20beta3)
2021-11-01 14:42:55 +00:00
}
store.wg.Wait()
}