nwaku/waku/v2/protocol/waku_store/protocol.nim

490 lines
17 KiB
Nim
Raw Normal View History

## Waku Store protocol for historical messaging support.
## See spec for more details:
## https://github.com/vacp2p/specs/blob/master/specs/waku/v2/waku-store.md
2021-06-09 14:37:08 +00:00
{.push raises: [Defect].}
import
std/[tables, times, sequtils, options, algorithm],
stew/results,
chronicles,
chronos,
bearssl/rand,
libp2p/crypto/crypto,
libp2p/protocols/protocol,
libp2p/protobuf/minprotobuf,
libp2p/stream/connection,
metrics
import
../../node/storage/message/message_retention_policy,
../../node/storage/message/waku_store_queue,
../../node/peer_manager/peer_manager,
Refactoring timestamps (#842) * Refactor timestamps type from float64 to int64 (milliseconds resolution) * Revert epochs to float64 * Update 00002_addSenderTimeStamp.up.sql * Update quicksim2.nim * Add files via upload * Delete 00003_convertTimestampsToInts.up.sql * Add files via upload * Rename 00003_convertTimestampsToInts.up.sql to 00003_addTimestampsToInts.up.sql * Delete 00003_addTimestampsToInts.up.sql * Rln-relay integration into chat2 (#835) * adds ProofMetadata * adds EPOCH_INTERVAL * adds messageLog field * adds updateLog, toEpoch, fromEpoch, getEpoch, compareTo * adds unit test for toEpoch and fromEpoch * adds unit test for Epoch comparison * adds result codes for updateLog * adds unit test for update log * renames epoch related consts * modifies updateLog with new return type and new logic of spam detection * adds unit text for the modified updateLog * changes max epoch gap type size * splits updateLog into two procs isSpam and updateLog * updates unittests * fixes a bug, returns false when the message is not spam * renames messageLog to nullifierLog * renames isSpam to hasDuplicate * updates the rln validator, adds comments * adds appendRLNProof proc plus some code beatification * unit test for validate message * adds unhappy test to validateMessage unit test * renames EPOCH_UNIT_SECONDS * renames MAX_CLOCK_GAP_SECONDS * WIP: integration test * fixes compile errors * sets a real epoch value * updates on old unittests * adds comments to the rln relay tests * adds more comments * makes rln import conditional * adds todos * adds more todos * adds rln-relay mount process into chat2 * further todos * logs contentTopic * introduces rln relay configs * changes default pubsub topic * adds contentTopic config * imports rln relay dependencies * consolidates imports * removes module identifier from ContentTopic * adds contentTopic field * adds contentTopic argument to mountRlnRelay calls * appends rln proof to chat2 messages * changes the default chat2 contentTopic * adds missing content topic fields * fixes a bug * adds a new logic about empty content topics * appends proof only when rln flag is active * removes unnecessary todos * fixes an indentation issue * adds log messages * verifies the proof against the concatenation of msg payload and content topic * a bug fix * removes duplicate epoch time calculation * updates log level to trace * updates default rln-relay content topic * adds support for empty content topics * updates changelog * changelog updates * removes a commented code block * updates addRLNRelayValidator string doc * Squashed commit of the following: commit bc36c99ab202d07baa0a5f0100bd10d1d76fdfa1 Merge: dc2b2946 5a77d6e2 Author: G <28568419+s1fr0@users.noreply.github.com> Date: Sat Feb 5 01:10:06 2022 +0100 Merge branch 'master' into int64-timestamps-ns commit dc2b294667bb5770cc32b93cc560638cf5ce7087 Author: s1fr0 <28568419+s1fr0@users.noreply.github.com> Date: Sat Feb 5 00:24:45 2022 +0100 Fix commit f97b95a036a197938df38a5adaea46fca778016d Author: s1fr0 <28568419+s1fr0@users.noreply.github.com> Date: Sat Feb 5 00:13:18 2022 +0100 Missing import commit 060c4f8d64e1b6e7c0593540fa8fa7f4cadf6df7 Author: s1fr0 <28568419+s1fr0@users.noreply.github.com> Date: Sat Feb 5 00:10:36 2022 +0100 Fixed typo commit 08ca99b6f692d3df6d4c7c2312c7cada05fc0041 Author: s1fr0 <28568419+s1fr0@users.noreply.github.com> Date: Fri Feb 4 23:59:20 2022 +0100 Time util file commit 2b5c360746990936dec256e90d08dae3c3e35a94 Author: s1fr0 <28568419+s1fr0@users.noreply.github.com> Date: Fri Feb 4 23:33:20 2022 +0100 Moved time utility functions to utils/time commit fdaf121f089aa011855303cc8dd1ce52aec506ad Author: s1fr0 <28568419+s1fr0@users.noreply.github.com> Date: Fri Feb 4 23:10:25 2022 +0100 Fix comment commit c7e06ab4e7618d9a3fe8aa744dd48bf3f7d8754c Author: s1fr0 <28568419+s1fr0@users.noreply.github.com> Date: Fri Feb 4 23:04:13 2022 +0100 Restore previous migration script commit 80282db1d79df676255d4b8e6e09d9f8a2b00fd3 Author: s1fr0 <28568419+s1fr0@users.noreply.github.com> Date: Fri Feb 4 22:54:15 2022 +0100 Typo commit b9d67f89b0eea11a8362dbb10b5f9d6894343352 Author: s1fr0 <28568419+s1fr0@users.noreply.github.com> Date: Fri Feb 4 22:49:29 2022 +0100 Added utilities to get int64 nanosecond, microsecond, millisecond time resolution from float commit 0130d496e694a01cfc9eeb90b7cbc77764490bf9 Author: s1fr0 <28568419+s1fr0@users.noreply.github.com> Date: Fri Feb 4 22:36:35 2022 +0100 Switched to nanoseconds support. * Update CHANGELOG.md * Create 00003_convertTimestampsToInt64.up.sql Migration script * Moved migration script to right location * Update waku_rln_relay_utils.nim * Update waku_rln_relay_utils.nim * Addressed reviewers' comments * Update default fleet metrics dashboard (#844) * Fix * No need for float * Aligning master to changes in PR * Further fixes Co-authored-by: Sanaz Taheri Boshrooyeh <35961250+staheri14@users.noreply.github.com> Co-authored-by: Hanno Cornelius <68783915+jm-clius@users.noreply.github.com>
2022-02-17 15:00:15 +00:00
../../utils/time,
../../utils/requests,
../waku_message,
../waku_swap/waku_swap,
./rpc,
./rpc_codec,
./pagination,
./message_store
2021-03-04 07:19:21 +00:00
declarePublicGauge waku_store_messages, "number of historical messages", ["type"]
2021-01-29 08:42:41 +00:00
declarePublicGauge waku_store_peers, "number of store peers"
declarePublicGauge waku_store_errors, "number of store protocol errors", ["type"]
declarePublicGauge waku_store_queries, "number of store queries received"
declarePublicHistogram waku_store_insert_duration_seconds, "message insertion duration"
declarePublicHistogram waku_store_query_duration_seconds, "history query duration"
2021-01-29 08:42:41 +00:00
logScope:
topics = "wakustore"
const
WakuStoreCodec* = "/vac/waku/store/2.0.0-beta4"
DefaultTopic* = "/waku/2/default-waku/proto"
MaxMessageTimestampVariance* = Timestamp(20.seconds.nanoseconds) # 20 seconds maximum allowable sender timestamp "drift"
# Error types (metric label values)
const
invalidMessage = "invalid_message"
insertFailure = "insert_failure"
retPolicyFailure = "retpolicy_failure"
dialFailure = "dial_failure"
decodeRpcFailure = "decode_rpc_failure"
peerNotFoundFailure = "peer_not_found_failure"
type
WakuStoreResult*[T] = Result[T, string]
WakuStore* = ref object of LPProtocol
peerManager*: PeerManager
rng*: ref rand.HmacDrbgContext
store*: MessageStore
wakuSwap*: WakuSwap
retentionPolicy: Option[MessageRetentionPolicy]
proc executeMessageRetentionPolicy*(w: WakuStore) =
if w.retentionPolicy.isNone():
return
if w.store.isNil():
return
let policy = w.retentionPolicy.get()
let retPolicyRes = policy.execute(w.store)
if retPolicyRes.isErr():
waku_store_errors.inc(labelValues = [retPolicyFailure])
debug "failed execution of retention policy", error=retPolicyRes.error
proc reportStoredMessagesMetric*(w: WakuStore) =
if w.store.isNil():
return
let resCount = w.store.getMessagesCount()
if resCount.isErr():
return
waku_store_messages.set(resCount.value, labelValues = ["stored"])
2022-02-17 10:00:45 +00:00
proc findMessages(w: WakuStore, query: HistoryQuery): HistoryResponse {.gcsafe.} =
## Query history to return a single page of messages matching the query
# Extract query criteria. All query criteria are optional
let
qContentTopics = if (query.contentFilters.len != 0): some(query.contentFilters.mapIt(it.contentTopic))
else: none(seq[ContentTopic])
qPubSubTopic = if (query.pubsubTopic != ""): some(query.pubsubTopic)
else: none(string)
qCursor = if query.pagingInfo.cursor != PagingIndex(): some(query.pagingInfo.cursor)
else: none(PagingIndex)
Refactoring timestamps (#842) * Refactor timestamps type from float64 to int64 (milliseconds resolution) * Revert epochs to float64 * Update 00002_addSenderTimeStamp.up.sql * Update quicksim2.nim * Add files via upload * Delete 00003_convertTimestampsToInts.up.sql * Add files via upload * Rename 00003_convertTimestampsToInts.up.sql to 00003_addTimestampsToInts.up.sql * Delete 00003_addTimestampsToInts.up.sql * Rln-relay integration into chat2 (#835) * adds ProofMetadata * adds EPOCH_INTERVAL * adds messageLog field * adds updateLog, toEpoch, fromEpoch, getEpoch, compareTo * adds unit test for toEpoch and fromEpoch * adds unit test for Epoch comparison * adds result codes for updateLog * adds unit test for update log * renames epoch related consts * modifies updateLog with new return type and new logic of spam detection * adds unit text for the modified updateLog * changes max epoch gap type size * splits updateLog into two procs isSpam and updateLog * updates unittests * fixes a bug, returns false when the message is not spam * renames messageLog to nullifierLog * renames isSpam to hasDuplicate * updates the rln validator, adds comments * adds appendRLNProof proc plus some code beatification * unit test for validate message * adds unhappy test to validateMessage unit test * renames EPOCH_UNIT_SECONDS * renames MAX_CLOCK_GAP_SECONDS * WIP: integration test * fixes compile errors * sets a real epoch value * updates on old unittests * adds comments to the rln relay tests * adds more comments * makes rln import conditional * adds todos * adds more todos * adds rln-relay mount process into chat2 * further todos * logs contentTopic * introduces rln relay configs * changes default pubsub topic * adds contentTopic config * imports rln relay dependencies * consolidates imports * removes module identifier from ContentTopic * adds contentTopic field * adds contentTopic argument to mountRlnRelay calls * appends rln proof to chat2 messages * changes the default chat2 contentTopic * adds missing content topic fields * fixes a bug * adds a new logic about empty content topics * appends proof only when rln flag is active * removes unnecessary todos * fixes an indentation issue * adds log messages * verifies the proof against the concatenation of msg payload and content topic * a bug fix * removes duplicate epoch time calculation * updates log level to trace * updates default rln-relay content topic * adds support for empty content topics * updates changelog * changelog updates * removes a commented code block * updates addRLNRelayValidator string doc * Squashed commit of the following: commit bc36c99ab202d07baa0a5f0100bd10d1d76fdfa1 Merge: dc2b2946 5a77d6e2 Author: G <28568419+s1fr0@users.noreply.github.com> Date: Sat Feb 5 01:10:06 2022 +0100 Merge branch 'master' into int64-timestamps-ns commit dc2b294667bb5770cc32b93cc560638cf5ce7087 Author: s1fr0 <28568419+s1fr0@users.noreply.github.com> Date: Sat Feb 5 00:24:45 2022 +0100 Fix commit f97b95a036a197938df38a5adaea46fca778016d Author: s1fr0 <28568419+s1fr0@users.noreply.github.com> Date: Sat Feb 5 00:13:18 2022 +0100 Missing import commit 060c4f8d64e1b6e7c0593540fa8fa7f4cadf6df7 Author: s1fr0 <28568419+s1fr0@users.noreply.github.com> Date: Sat Feb 5 00:10:36 2022 +0100 Fixed typo commit 08ca99b6f692d3df6d4c7c2312c7cada05fc0041 Author: s1fr0 <28568419+s1fr0@users.noreply.github.com> Date: Fri Feb 4 23:59:20 2022 +0100 Time util file commit 2b5c360746990936dec256e90d08dae3c3e35a94 Author: s1fr0 <28568419+s1fr0@users.noreply.github.com> Date: Fri Feb 4 23:33:20 2022 +0100 Moved time utility functions to utils/time commit fdaf121f089aa011855303cc8dd1ce52aec506ad Author: s1fr0 <28568419+s1fr0@users.noreply.github.com> Date: Fri Feb 4 23:10:25 2022 +0100 Fix comment commit c7e06ab4e7618d9a3fe8aa744dd48bf3f7d8754c Author: s1fr0 <28568419+s1fr0@users.noreply.github.com> Date: Fri Feb 4 23:04:13 2022 +0100 Restore previous migration script commit 80282db1d79df676255d4b8e6e09d9f8a2b00fd3 Author: s1fr0 <28568419+s1fr0@users.noreply.github.com> Date: Fri Feb 4 22:54:15 2022 +0100 Typo commit b9d67f89b0eea11a8362dbb10b5f9d6894343352 Author: s1fr0 <28568419+s1fr0@users.noreply.github.com> Date: Fri Feb 4 22:49:29 2022 +0100 Added utilities to get int64 nanosecond, microsecond, millisecond time resolution from float commit 0130d496e694a01cfc9eeb90b7cbc77764490bf9 Author: s1fr0 <28568419+s1fr0@users.noreply.github.com> Date: Fri Feb 4 22:36:35 2022 +0100 Switched to nanoseconds support. * Update CHANGELOG.md * Create 00003_convertTimestampsToInt64.up.sql Migration script * Moved migration script to right location * Update waku_rln_relay_utils.nim * Update waku_rln_relay_utils.nim * Addressed reviewers' comments * Update default fleet metrics dashboard (#844) * Fix * No need for float * Aligning master to changes in PR * Further fixes Co-authored-by: Sanaz Taheri Boshrooyeh <35961250+staheri14@users.noreply.github.com> Co-authored-by: Hanno Cornelius <68783915+jm-clius@users.noreply.github.com>
2022-02-17 15:00:15 +00:00
qStartTime = if query.startTime != Timestamp(0): some(query.startTime)
else: none(Timestamp)
qEndTime = if query.endTime != Timestamp(0): some(query.endTime)
else: none(Timestamp)
qMaxPageSize = if query.pagingInfo.pageSize <= 0: DefaultPageSize
else: min(query.pagingInfo.pageSize, MaxPageSize)
qAscendingOrder = query.pagingInfo.direction == PagingDirection.FORWARD
let queryStartTime = getTime().toUnixFloat()
let queryRes = w.store.getMessagesByHistoryQuery(
contentTopic = qContentTopics,
pubsubTopic = qPubSubTopic,
cursor = qCursor,
startTime = qStartTime,
endTime = qEndTime,
maxPageSize = qMaxPageSize + 1,
ascendingOrder = qAscendingOrder
)
let queryDuration = getTime().toUnixFloat() - queryStartTime
waku_store_query_duration_seconds.observe(queryDuration)
# Build response
# TODO: Improve error reporting
if queryRes.isErr():
return HistoryResponse(messages: @[], pagingInfo: PagingInfo(), error: HistoryResponseError.INVALID_CURSOR)
let rows = queryRes.get()
if rows.len <= 0:
return HistoryResponse(messages: @[], error: HistoryResponseError.NONE)
var messages = if rows.len <= int(qMaxPageSize): rows.mapIt(it[1])
else: rows[0..^2].mapIt(it[1])
var pagingInfo = none(PagingInfo)
# The retrieved messages list should always be in chronological order
if not qAscendingOrder:
messages.reverse()
if rows.len > int(qMaxPageSize):
# Build last message cursor
let (pubsubTopic, message, digest, storeTimestamp) = rows[^1]
# TODO: Improve coherence of MessageDigest type
var messageDigest: array[32, byte]
for i in 0..<min(digest.len, 32):
messageDigest[i] = digest[i]
let pagingIndex = PagingIndex(
pubsubTopic: pubsubTopic,
senderTime: message.timestamp,
receiverTime: storeTimestamp,
digest: MessageDigest(data: messageDigest)
)
pagingInfo = some(PagingInfo(
pageSize: uint64(messages.len),
cursor: pagingIndex,
direction: if qAscendingOrder: PagingDirection.FORWARD
else: PagingDirection.BACKWARD
))
HistoryResponse(
messages: messages,
pagingInfo: pagingInfo.get(PagingInfo()),
error: HistoryResponseError.NONE
)
Pagination feature/indexing waku messages (#233) * changes the digest type to MDigest[256] and modifies the computeIndex * fixes formatting issue * adds the pagination with its tests stores and retrieves IndexedWakuMessage adds the paginate proc adds the paginate function fixes some formatting issues minor edits indentation and fixes a bug removes unused imports minor fixes indentations and adds a new testcase adds indexedWakuMessageComparison adds `==` proc for IndexedWakuMessage separates the comparison of index and indexed waku messages adds testcases for the Index comparison and IndexedWakuMessage comparison WIP WIP: adds an decoder for Index removes an unnecessary imports WIP adds findIndex() proc removes the equality check '==' for IndexedWakuMessages edits the code format and adds the pagination test edits paginate() proc to work on a copy of the input list deletes unnecessary echo adds the boundary calculations for forward and backward pagination adds test cases for the page boundaries tests corner cases for the queried cursor and pagesize minor adds some comments adds a proc to extract WakuMessages from a list of IndexedWakuMessages integrates pagination into the findMessages proc adds some comments changes paginate to paginateWithIndex removes some echos modifies paginateWithIndex to handle invalid cursors adds test case for an invalid cursor WIP: adds a `$` proc for IndexedWakuMessages adds some debugging message prints adds an integration test for handling query with pagination * fixes a type mismatch issue in the min proc * replaces boolean direction with their enums and updates contentTopics * adds the unit test for the sorting of the indexed waku messages * fixes a flaky test * fixes a flaky test * removes index equality check proc * handles an initial query with an empty cursor * adds test for the initial query * adds integration test for pagination * adds a test for empty message list * adds comments and fixes an issue * adds comments * code cleanup * removes the content topic validation check * resolves the errors related to the windows CI tests * Update waku/protocol/v2/waku_store.nim Co-authored-by: Oskar Thorén <ot@oskarthoren.com> * Update waku/protocol/v2/waku_store.nim Co-authored-by: Oskar Thorén <ot@oskarthoren.com> * Update tests/v2/test_waku_pagination.nim Co-authored-by: Oskar Thorén <ot@oskarthoren.com> * Update waku/protocol/v2/waku_store.nim Co-authored-by: Oskar Thorén <ot@oskarthoren.com> * Update waku/protocol/v2/waku_store.nim Co-authored-by: Oskar Thorén <ot@oskarthoren.com> * Update waku/protocol/v2/waku_store.nim Co-authored-by: Oskar Thorén <ot@oskarthoren.com> * changes the output type of findIndex to Option * Update tests/v2/test_waku_pagination.nim Co-authored-by: Dean Eigenmann <7621705+decanus@users.noreply.github.com> * Update tests/v2/test_waku_pagination.nim Co-authored-by: Dean Eigenmann <7621705+decanus@users.noreply.github.com> * Update tests/v2/test_waku_pagination.nim Co-authored-by: Dean Eigenmann <7621705+decanus@users.noreply.github.com> * Apply suggestions from code review Co-authored-by: Dean Eigenmann <7621705+decanus@users.noreply.github.com> * Apply suggestions from code review Co-authored-by: Dean Eigenmann <7621705+decanus@users.noreply.github.com> * adds some comments * fixes an indentation issue * some code modification for array initialization * Apply suggestions from code review Co-authored-by: Oskar Thorén <ot@oskarthoren.com> Co-authored-by: Dean Eigenmann <7621705+decanus@users.noreply.github.com> * does some code reorganizations and clean up * CreateSampleList to createSampleList * replaces a byte array literal initialization with a for loop * relocates indexedWakuMessageComparison and indexComparison * minor Co-authored-by: Oskar Thorén <ot@oskarthoren.com> Co-authored-by: Dean Eigenmann <7621705+decanus@users.noreply.github.com>
2020-11-09 04:48:09 +00:00
proc initProtocolHandler*(ws: WakuStore) =
2021-06-09 14:37:08 +00:00
proc handler(conn: Connection, proto: string) {.async.} =
let buf = await conn.readLp(MaxRpcSize.int)
let resReq = HistoryRPC.init(buf)
if resReq.isErr():
error "failed to decode rpc", peerId=conn.peerId
waku_store_errors.inc(labelValues = [decodeRpcFailure])
return
let req = resReq.value
info "received history query", peerId=conn.peerId, requestId=req.requestId, query=req.query
waku_store_queries.inc()
let resp = if not ws.store.isNil(): ws.findMessages(req.query)
# TODO: Improve error reporting
else: HistoryResponse(error: HistoryResponseError.SERVICE_UNAVAILABLE)
if not ws.wakuSwap.isNil():
info "handle store swap", peerId=conn.peerId, requestId=req.requestId, text=ws.wakuSwap.text
# Perform accounting operation
# TODO: Do accounting here, response is HistoryResponse. How do we get node or swap context?
let peerId = conn.peerId
let messages = resp.messages
ws.wakuSwap.credit(peerId, messages.len)
info "sending history response", peerId=conn.peerId, requestId=req.requestId, messages=resp.messages.len
let rpc = HistoryRPC(requestId: req.requestId, response: resp)
await conn.writeLp(rpc.encode().buffer)
2021-06-09 14:37:08 +00:00
ws.handler = handler
ws.codec = WakuStoreCodec
proc init*(T: type WakuStore,
peerManager: PeerManager,
rng: ref rand.HmacDrbgContext,
store: MessageStore,
wakuSwap: WakuSwap = nil,
retentionPolicy=none(MessageRetentionPolicy)): T =
let ws = WakuStore(
rng: rng,
peerManager: peerManager,
store: store,
wakuSwap: wakuSwap,
retentionPolicy: retentionPolicy
)
ws.initProtocolHandler()
return ws
proc init*(T: type WakuStore,
peerManager: PeerManager,
rng: ref rand.HmacDrbgContext,
wakuSwap: WakuSwap = nil,
retentionPolicy=none(MessageRetentionPolicy)): T =
let store = StoreQueueRef.new()
WakuStore.init(peerManager, rng, store, wakuSwap, retentionPolicy)
proc isValidMessage(msg: WakuMessage): bool =
if msg.timestamp == 0:
return true
let
now = getNanosecondTime(getTime().toUnixFloat())
lowerBound = now - MaxMessageTimestampVariance
upperBound = now + MaxMessageTimestampVariance
return lowerBound <= msg.timestamp and msg.timestamp <= upperBound
proc handleMessage*(w: WakuStore, pubsubTopic: string, msg: WakuMessage) =
if w.store.isNil():
# Messages should not be stored
return
if msg.ephemeral:
# The message is ephemeral, should not be stored
return
if not isValidMessage(msg):
waku_store_errors.inc(labelValues = [invalidMessage])
return
let insertStartTime = getTime().toUnixFloat()
block:
let
msgDigest = computeDigest(msg)
msgReceivedTime = if msg.timestamp > 0: msg.timestamp
else: getNanosecondTime(getTime().toUnixFloat())
trace "handling message", pubsubTopic=pubsubTopic, contentTopic=msg.contentTopic, timestamp=msg.timestamp, digest=msgDigest
let putStoreRes = w.store.put(pubsubTopic, msg, msgDigest, msgReceivedTime)
if putStoreRes.isErr():
debug "failed to insert message into the store", err=putStoreRes.error
waku_store_errors.inc(labelValues = [insertFailure])
return
let insertDuration = getTime().toUnixFloat() - insertStartTime
waku_store_insert_duration_seconds.observe(insertDuration)
## CLIENT
# TODO: This should probably be an add function and append the peer to an array
proc setPeer*(ws: WakuStore, peer: RemotePeerInfo) =
ws.peerManager.addPeer(peer, WakuStoreCodec)
waku_store_peers.inc()
proc query(w: WakuStore, req: HistoryQuery, peer: RemotePeerInfo): Future[WakuStoreResult[HistoryResponse]] {.async, gcsafe.} =
let connOpt = await w.peerManager.dialPeer(peer, WakuStoreCodec)
if connOpt.isNone():
waku_store_errors.inc(labelValues = [dialFailure])
return err(dialFailure)
let connection = connOpt.get()
let rpc = HistoryRPC(requestId: generateRequestId(w.rng), query: req)
await connection.writeLP(rpc.encode().buffer)
var message = await connOpt.get().readLp(MaxRpcSize.int)
let response = HistoryRPC.init(message)
if response.isErr():
error "failed to decode response"
waku_store_errors.inc(labelValues = [decodeRpcFailure])
return err(decodeRpcFailure)
waku_store_messages.set(response.value.response.messages.len.int64, labelValues = ["retrieved"])
return ok(response.value.response)
proc query*(w: WakuStore, req: HistoryQuery): Future[WakuStoreResult[HistoryResponse]] {.async, gcsafe.} =
# TODO: We need to be more stratigic about which peers we dial. Right now we just set one on the service.
# Ideally depending on the query and our set of peers we take a subset of ideal peers.
# This will require us to check for various factors such as:
# - which topics they track
# - latency?
# - default store peer?
let peerOpt = w.peerManager.selectPeer(WakuStoreCodec)
if peerOpt.isNone():
error "no suitable remote peers"
waku_store_errors.inc(labelValues = [peerNotFoundFailure])
return err(peerNotFoundFailure)
return await w.query(req, peerOpt.get())
## 21/WAKU2-FAULT-TOLERANT-STORE
const StoreResumeTimeWindowOffset: Timestamp = getNanosecondTime(20) ## Adjust the time window with an offset of 20 seconds
proc queryFromWithPaging*(w: WakuStore, query: HistoryQuery, peer: RemotePeerInfo): Future[WakuStoreResult[seq[WakuMessage]]] {.async, gcsafe.} =
## A thin wrapper for query. Sends the query to the given peer. when the query has a valid pagingInfo,
## it retrieves the historical messages in pages.
## Returns all the fetched messages, if error occurs, returns an error string
# Make a copy of the query
var req = query
var messageList: seq[WakuMessage] = @[]
# Fetch the history in pages
while true:
let res = await w.query(req, peer)
if res.isErr():
return err(res.error)
let response = res.get()
messageList.add(response.messages)
# Check whether it is the last page
if response.pagingInfo.pageSize == 0:
break
# Update paging cursor
req.pagingInfo.cursor = response.pagingInfo.cursor
return ok(messageList)
proc queryLoop(w: WakuStore, req: HistoryQuery, candidateList: seq[RemotePeerInfo]): Future[WakuStoreResult[seq[WakuMessage]]] {.async, gcsafe.} =
## Loops through the peers candidate list in order and sends the query to each
##
## Once all responses have been received, the retrieved messages are consolidated into one deduplicated list.
## if no messages have been retrieved, the returned future will resolve into a result holding an empty seq.
let queriesList = candidateList.mapIt(w.queryFromWithPaging(req, it))
await allFutures(queriesList)
let messagesList = queriesList
.map(proc (fut: Future[WakuStoreResult[seq[WakuMessage]]]): seq[WakuMessage] =
# These futures have been awaited before using allFutures(). Call completed() just as a sanity check.
if not fut.completed() or fut.read().isErr():
return @[]
fut.read().value
)
.concat()
.deduplicate()
if messagesList.len == 0:
return err("failed to resolve the query")
return ok(messagesList)
proc resume*(w: WakuStore,
peerList: Option[seq[RemotePeerInfo]] = none(seq[RemotePeerInfo]),
pageSize: uint64 = DefaultPageSize,
pubsubTopic = DefaultTopic): Future[WakuStoreResult[uint64]] {.async, gcsafe.} =
## 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
## peerList indicates the list of peers to query from.
## The history is fetched from all available peers in this list and then consolidated into one deduplicated 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
# If store has not been provided, don't even try
if w.store.isNil():
return err("store not provided (nil)")
# NOTE: Original implementation is based on the message's sender timestamp. At the moment
# of writing, the sqlite store implementation returns the last message's receiver
# timestamp.
# lastSeenTime = lastSeenItem.get().msg.timestamp
let
lastSeenTime = w.store.getNewestMessageTimestamp().get(Timestamp(0))
now = getNanosecondTime(getTime().toUnixFloat())
debug "resuming with offline time window", lastSeenTime=lastSeenTime, currentTime=now
let
queryEndTime = now + StoreResumeTimeWindowOffset
queryStartTime = max(lastSeenTime - StoreResumeTimeWindowOffset, 0)
let req = HistoryQuery(
pubsubTopic: pubsubTopic,
startTime: queryStartTime,
endTime: queryEndTime,
pagingInfo: PagingInfo(
direction:PagingDirection.FORWARD,
pageSize: pageSize
)
)
var res: WakuStoreResult[seq[WakuMessage]]
if peerList.isSome():
debug "trying the candidate list to fetch the history"
res = await w.queryLoop(req, peerList.get())
else:
debug "no candidate list is provided, selecting a random peer"
# if no peerList is set then query from one of the peers stored in the peer manager
let peerOpt = w.peerManager.selectPeer(WakuStoreCodec)
if peerOpt.isNone():
warn "no suitable remote peers"
waku_store_errors.inc(labelValues = [peerNotFoundFailure])
return err("no suitable remote peers")
debug "a peer is selected from peer manager"
res = await w.queryFromWithPaging(req, peerOpt.get())
if res.isErr():
debug "failed to resume the history"
return err("failed to resume the history")
# Save the retrieved messages in the store
var added: uint = 0
for msg in res.get():
let putStoreRes = w.store.put(pubsubTopic, msg)
if putStoreRes.isErr():
warn "failed to insert resumed message into store", error=putStoreRes.error
continue
added.inc()
return ok(added)
## EXPERIMENTAL
# NOTE: Experimental, maybe incorporate as part of query call
proc queryWithAccounting*(ws: WakuStore, req: HistoryQuery): Future[WakuStoreResult[HistoryResponse]] {.async, gcsafe.} =
let peerOpt = ws.peerManager.selectPeer(WakuStoreCodec)
if peerOpt.isNone():
error "no suitable remote peers"
waku_store_errors.inc(labelValues = [peerNotFoundFailure])
return err(peerNotFoundFailure)
let res = await ws.query(req, peerOpt.get())
if res.isErr():
return err(res.error)
let response = res.get()
# Perform accounting operation. Assumes wakuSwap protocol is mounted
ws.wakuSwap.debit(peerOpt.get().peerId, response.messages.len)
return ok(response)