WIP: Harmonize BlockStore API (remaining functions) (#123) (#130)

BlockStore API got new return types (rationale in https://github.com/status-im/nim-codex/issues/123#issuecomment-1163797753):
- getBlock: Future[?! (?Block)]
- putBlock/delBlock/listBlocks: Future[?!void]
- hasBlock: Future[?!bool]

Plus refactored readOnce(StoreStream) and check received data in its tests.

And replaced local use of AsyncHeapQueue with seq.sort.
This commit is contained in:
Bulat-Ziganshin 2022-07-28 03:39:17 +03:00 committed by GitHub
parent afdb5be2d4
commit 0bfe26440e
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
22 changed files with 366 additions and 315 deletions

View File

@ -83,7 +83,7 @@ proc advertiseQueueLoop*(b: DiscoveryEngine) {.async.} =
trace "Exception listing blocks", exc = exc.msg trace "Exception listing blocks", exc = exc.msg
while b.discEngineRunning: while b.discEngineRunning:
await b.localStore.listBlocks(onBlock) discard await b.localStore.listBlocks(onBlock)
trace "About to sleep advertise loop", sleep = b.advertiseLoopSleep trace "About to sleep advertise loop", sleep = b.advertiseLoopSleep
await sleepAsync(b.advertiseLoopSleep) await sleepAsync(b.advertiseLoopSleep)

View File

@ -9,6 +9,8 @@
import std/sequtils import std/sequtils
import std/sets import std/sets
import std/options
import std/algorithm
import pkg/chronos import pkg/chronos
import pkg/chronicles import pkg/chronicles
@ -200,7 +202,7 @@ proc blockPresenceHandler*(
.filter do(cid: Cid) -> bool: .filter do(cid: Cid) -> bool:
not b.peers.anyIt( cid in it.peerHave )) not b.peers.anyIt( cid in it.peerHave ))
proc scheduleTasks(b: BlockExcEngine, blocks: seq[bt.Block]) = proc scheduleTasks(b: BlockExcEngine, blocks: seq[bt.Block]) {.async.} =
trace "Schedule a task for new blocks" trace "Schedule a task for new blocks"
let let
@ -209,9 +211,10 @@ proc scheduleTasks(b: BlockExcEngine, blocks: seq[bt.Block]) =
# schedule any new peers to provide blocks to # schedule any new peers to provide blocks to
for p in b.peers: for p in b.peers:
for c in cids: # for each cid for c in cids: # for each cid
# schedule a peer if it wants at least one # schedule a peer if it wants at least one cid
# cid and we have it in our local store # and we have it in our local store
if c in p.peerWants and c in b.localStore: if c in p.peerWants:
if await (c in b.localStore):
if b.scheduleTask(p): if b.scheduleTask(p):
trace "Task scheduled for peer", peer = p.id trace "Task scheduled for peer", peer = p.id
else: else:
@ -219,7 +222,7 @@ proc scheduleTasks(b: BlockExcEngine, blocks: seq[bt.Block]) =
break # do next peer break # do next peer
proc resolveBlocks*(b: BlockExcEngine, blocks: seq[bt.Block]) = proc resolveBlocks*(b: BlockExcEngine, blocks: seq[bt.Block]) {.async.} =
## Resolve pending blocks from the pending blocks manager ## Resolve pending blocks from the pending blocks manager
## and schedule any new task to be ran ## and schedule any new task to be ran
## ##
@ -227,7 +230,7 @@ proc resolveBlocks*(b: BlockExcEngine, blocks: seq[bt.Block]) =
trace "Resolving blocks", blocks = blocks.len trace "Resolving blocks", blocks = blocks.len
b.pendingBlocks.resolve(blocks) b.pendingBlocks.resolve(blocks)
b.scheduleTasks(blocks) await b.scheduleTasks(blocks)
b.discovery.queueProvideBlocksReq(blocks.mapIt( it.cid )) b.discovery.queueProvideBlocksReq(blocks.mapIt( it.cid ))
proc payForBlocks(engine: BlockExcEngine, proc payForBlocks(engine: BlockExcEngine,
@ -250,11 +253,10 @@ proc blocksHandler*(
trace "Got blocks from peer", peer, len = blocks.len trace "Got blocks from peer", peer, len = blocks.len
for blk in blocks: for blk in blocks:
if not (await b.localStore.putBlock(blk)): if isErr (await b.localStore.putBlock(blk)):
trace "Unable to store block", cid = blk.cid trace "Unable to store block", cid = blk.cid
continue
b.resolveBlocks(blocks) await b.resolveBlocks(blocks)
let peerCtx = b.peers.get(peer) let peerCtx = b.peers.get(peer)
if peerCtx != nil: if peerCtx != nil:
b.payForBlocks(peerCtx, blocks) b.payForBlocks(peerCtx, blocks)
@ -289,7 +291,8 @@ proc wantListHandler*(
# peer might want to ask for the same cid with # peer might want to ask for the same cid with
# different want params # different want params
if e.sendDontHave and e.cid notin b.localStore: if e.sendDontHave:
if not(await e.cid in b.localStore):
dontHaves.add(e.cid) dontHaves.add(e.cid)
# send don't have's to remote # send don't have's to remote
@ -358,22 +361,25 @@ proc dropPeer*(b: BlockExcEngine, peer: PeerID) =
proc taskHandler*(b: BlockExcEngine, task: BlockExcPeerCtx) {.gcsafe, async.} = proc taskHandler*(b: BlockExcEngine, task: BlockExcPeerCtx) {.gcsafe, async.} =
trace "Handling task for peer", peer = task.id trace "Handling task for peer", peer = task.id
var wantsBlocks = newAsyncHeapQueue[Entry](queueType = QueueType.Max) # PART 1: Send to the peer blocks he wants to get,
# get blocks and wants to send to the remote # if they present in our local store
for e in task.peerWants:
if e.wantType == WantType.wantBlock:
await wantsBlocks.push(e)
# TODO: There should be all sorts of accounting of # TODO: There should be all sorts of accounting of
# bytes sent/received here # bytes sent/received here
var wantsBlocks = task.peerWants.filterIt(it.wantType == WantType.wantBlock)
if wantsBlocks.len > 0: if wantsBlocks.len > 0:
wantsBlocks.sort(SortOrder.Descending)
let blockFuts = await allFinished(wantsBlocks.mapIt( let blockFuts = await allFinished(wantsBlocks.mapIt(
b.localStore.getBlock(it.cid) b.localStore.getBlock(it.cid)
)) ))
# Extract succesfully received blocks
let blocks = blockFuts let blocks = blockFuts
.filterIt((not it.failed) and it.read.isOk) .filterIt(it.completed and it.read.isOk and it.read.get.isSome)
.mapIt(!it.read) .mapIt(it.read.get.get)
if blocks.len > 0: if blocks.len > 0:
trace "Sending blocks to peer", peer = task.id, blocks = blocks.len trace "Sending blocks to peer", peer = task.id, blocks = blocks.len
@ -387,13 +393,17 @@ proc taskHandler*(b: BlockExcEngine, task: BlockExcPeerCtx) {.gcsafe, async.} =
not blocks.anyIt( it.cid == e.cid ) not blocks.anyIt( it.cid == e.cid )
) )
# PART 2: Send to the peer prices of the blocks he wants to discover,
# if they present in our local store
var wants: seq[BlockPresence] var wants: seq[BlockPresence]
# do not remove wants from the queue unless # do not remove wants from the queue unless
# we send the block or get a cancel # we send the block or get a cancel
for e in task.peerWants: for e in task.peerWants:
if e.wantType == WantType.wantHave: if e.wantType == WantType.wantHave:
var presence = Presence(cid: e.cid) var presence = Presence(cid: e.cid)
presence.have = b.localStore.hasblock(presence.cid) presence.have = await (presence.cid in b.localStore)
if presence.have and price =? b.pricing.?price: if presence.have and price =? b.pricing.?price:
presence.price = price presence.price = price
wants.add(BlockPresence.init(presence)) wants.add(BlockPresence.init(presence))

View File

@ -21,6 +21,14 @@ import pkg/questionable/results
import ./errors import ./errors
const
BlockSize* = 31 * 64 * 4 # block size
type
Block* = ref object of RootObj
cid*: Cid
data*: seq[byte]
template EmptyCid*: untyped = template EmptyCid*: untyped =
var var
emptyCid {.global, threadvar.}: emptyCid {.global, threadvar.}:
@ -90,14 +98,6 @@ template EmptyBlock*: untyped =
emptyBlock emptyBlock
const
BlockSize* = 31 * 64 * 4 # block size
type
Block* = ref object of RootObj
cid*: Cid
data*: seq[byte]
proc isEmpty*(cid: Cid): bool = proc isEmpty*(cid: Cid): bool =
cid == EmptyCid[cid.cidver] cid == EmptyCid[cid.cidver]
.catch .catch

View File

@ -114,10 +114,14 @@ proc encode*(
for j in 0..<blocks: for j in 0..<blocks:
let idx = blockIdx[j] let idx = blockIdx[j]
if idx < manifest.len: if idx < manifest.len:
without var blk =? await dataBlocks[j], error: without blkOrNone =? await dataBlocks[j], error:
trace "Unable to retrieve block", msg = error.msg trace "Unable to retrieve block", error = error.msg
return error.failure return error.failure
without blk =? blkOrNone:
trace "Block not found", cid = encoded[idx]
return failure("Block not found")
trace "Encoding block", cid = blk.cid, pos = idx trace "Encoding block", cid = blk.cid, pos = idx
shallowCopy(data[j], blk.data) shallowCopy(data[j], blk.data)
else: else:
@ -125,11 +129,11 @@ proc encode*(
data[j] = newSeq[byte](manifest.blockSize) data[j] = newSeq[byte](manifest.blockSize)
trace "Erasure coding data", data = data.len, parity = parityData.len trace "Erasure coding data", data = data.len, parity = parityData.len
if (
let err = encoder.encode(data, parityData); let res = encoder.encode(data, parityData);
err.isErr): if res.isErr:
trace "Unable to encode manifest!", err = $err.error trace "Unable to encode manifest!", error = $res.error
return failure($err.error) return failure($res.error)
for j in 0..<parity: for j in 0..<parity:
let idx = encoded.rounded + blockIdx[j] let idx = encoded.rounded + blockIdx[j]
@ -139,7 +143,7 @@ proc encode*(
trace "Adding parity block", cid = blk.cid, pos = idx trace "Adding parity block", cid = blk.cid, pos = idx
encoded[idx] = blk.cid encoded[idx] = blk.cid
if not (await self.store.putBlock(blk)): if isErr (await self.store.putBlock(blk)):
trace "Unable to store block!", cid = blk.cid trace "Unable to store block!", cid = blk.cid
return failure("Unable to store block!") return failure("Unable to store block!")
except CancelledError as exc: except CancelledError as exc:
@ -205,10 +209,12 @@ proc decode*(
idxPendingBlocks.del(idxPendingBlocks.find(done)) idxPendingBlocks.del(idxPendingBlocks.find(done))
without blk =? (await done), error: without blkOrNone =? (await done), error:
trace "Failed retrieving block", exc = error.msg trace "Failed retrieving block", exc = error.msg
return error.failure
if blk.isNil: without blk =? blkOrNone:
trace "Block not found"
continue continue
if idx >= encoded.K: if idx >= encoded.K:
@ -242,7 +248,7 @@ proc decode*(
return failure(error) return failure(error)
trace "Recovered block", cid = blk.cid trace "Recovered block", cid = blk.cid
if not (await self.store.putBlock(blk)): if isErr (await self.store.putBlock(blk)):
trace "Unable to store block!", cid = blk.cid trace "Unable to store block!", cid = blk.cid
return failure("Unable to store block!") return failure("Unable to store block!")
except CancelledError as exc: except CancelledError as exc:

View File

@ -66,9 +66,12 @@ proc retrieve*(
cid: Cid): Future[?!LPStream] {.async.} = cid: Cid): Future[?!LPStream] {.async.} =
trace "Received retrieval request", cid trace "Received retrieval request", cid
without blk =? await node.blockStore.getBlock(cid): without blkOrNone =? await node.blockStore.getBlock(cid), error:
return failure( return failure(error)
newException(CodexError, "Couldn't retrieve block for Cid!"))
without blk =? blkOrNone:
trace "Block not found", cid
return failure("Block not found")
if manifest =? Manifest.decode(blk.data, blk.cid): if manifest =? Manifest.decode(blk.data, blk.cid):
@ -134,7 +137,7 @@ proc store*(
return failure("Unable to init block from chunk!") return failure("Unable to init block from chunk!")
blockManifest.add(blk.cid) blockManifest.add(blk.cid)
if not (await node.blockStore.putBlock(blk)): if isErr (await node.blockStore.putBlock(blk)):
# trace "Unable to store block", cid = blk.cid # trace "Unable to store block", cid = blk.cid
return failure("Unable to store block " & $blk.cid) return failure("Unable to store block " & $blk.cid)
@ -155,7 +158,7 @@ proc store*(
trace "Unable to init block from manifest data!" trace "Unable to init block from manifest data!"
return failure("Unable to init block from manifest data!") return failure("Unable to init block from manifest data!")
if not (await node.blockStore.putBlock(manifest)): if isErr (await node.blockStore.putBlock(manifest)):
trace "Unable to store manifest", cid = manifest.cid trace "Unable to store manifest", cid = manifest.cid
return failure("Unable to store manifest " & $manifest.cid) return failure("Unable to store manifest " & $manifest.cid)
@ -187,13 +190,17 @@ proc store(node: CodexNodeRef, cids: seq[Cid]): Future[?!void] {.async.} =
proc store(node: CodexNodeRef, cid: Cid): Future[?!void] {.async.} = proc store(node: CodexNodeRef, cid: Cid): Future[?!void] {.async.} =
## Retrieves dataset from the network, and stores it locally ## Retrieves dataset from the network, and stores it locally
if node.blockstore.hasBlock(cid): without present =? await node.blockstore.hasBlock(cid):
return failure newException(CodexError, "Unable to find block " & $cid)
if present:
return success() return success()
without blk =? await node.blockstore.getBlock(cid): without blkOrNone =? await node.blockstore.getBlock(cid):
return failure newException(CodexError, "Unable to retrieve block " & $cid)
without blk =? blkOrNone:
return failure newException(CodexError, "Unable to retrieve block " & $cid) return failure newException(CodexError, "Unable to retrieve block " & $cid)
if not (await node.blockstore.putBlock(blk)): if isErr (await node.blockstore.putBlock(blk)):
return failure newException(CodexError, "Unable to store block " & $cid) return failure newException(CodexError, "Unable to store block " & $cid)
if manifest =? Manifest.decode(blk.data, blk.cid): if manifest =? Manifest.decode(blk.data, blk.cid):
@ -224,10 +231,14 @@ proc requestStorage*(self: CodexNodeRef,
trace "Purchasing not available" trace "Purchasing not available"
return failure "Purchasing not available" return failure "Purchasing not available"
without blk =? (await self.blockStore.getBlock(cid)), error: without blkOrNone =? (await self.blockStore.getBlock(cid)), error:
trace "Unable to retrieve manifest block", cid trace "Unable to retrieve manifest block", cid
return failure(error) return failure(error)
without blk =? blkOrNone:
trace "Manifest block not found", cid
return failure("Manifest block not found")
without mc =? blk.cid.contentType(): without mc =? blk.cid.contentType():
trace "Couldn't identify Cid!", cid trace "Couldn't identify Cid!", cid
return failure("Couldn't identify Cid! " & $cid) return failure("Couldn't identify Cid! " & $cid)
@ -254,7 +265,7 @@ proc requestStorage*(self: CodexNodeRef,
trace "Unable to create block from encoded manifest" trace "Unable to create block from encoded manifest"
return failure(error) return failure(error)
if not (await self.blockStore.putBlock(encodedBlk)): if isErr (await self.blockStore.putBlock(encodedBlk)):
trace "Unable to store encoded manifest block", cid = encodedBlk.cid trace "Unable to store encoded manifest block", cid = encodedBlk.cid
return failure("Unable to store encoded manifest block") return failure("Unable to store encoded manifest block")

View File

@ -13,6 +13,7 @@ push: {.upraises: [].}
import pkg/chronos import pkg/chronos
import pkg/libp2p import pkg/libp2p
import pkg/questionable
import pkg/questionable/results import pkg/questionable/results
import ../blocktype import ../blocktype
@ -23,41 +24,39 @@ type
OnBlock* = proc(cid: Cid): Future[void] {.upraises: [], gcsafe.} OnBlock* = proc(cid: Cid): Future[void] {.upraises: [], gcsafe.}
BlockStore* = ref object of RootObj BlockStore* = ref object of RootObj
method getBlock*( method getBlock*(self: BlockStore, cid: Cid): Future[?! (? Block)] {.base.} =
b: BlockStore, ## Get a block from the blockstore
cid: Cid): Future[?!Block] {.base.} =
## Get a block from the stores
## ##
raiseAssert("Not implemented!") raiseAssert("Not implemented!")
method putBlock*( method putBlock*(self: BlockStore, blk: Block): Future[?!void] {.base.} =
s: BlockStore,
blk: Block): Future[bool] {.base.} =
## Put a block to the blockstore ## Put a block to the blockstore
## ##
raiseAssert("Not implemented!") raiseAssert("Not implemented!")
method delBlock*( method delBlock*(self: BlockStore, cid: Cid): Future[?!void] {.base.} =
s: BlockStore,
cid: Cid): Future[?!void] {.base.} =
## Delete a block from the blockstore ## Delete a block from the blockstore
## ##
raiseAssert("Not implemented!") raiseAssert("Not implemented!")
method hasBlock*(s: BlockStore, cid: Cid): bool {.base.} = method hasBlock*(self: BlockStore, cid: Cid): Future[?!bool] {.base.} =
## Check if the block exists in the blockstore ## Check if the block exists in the blockstore
## ##
return false raiseAssert("Not implemented!")
method listBlocks*(s: BlockStore, onBlock: OnBlock): Future[void] {.base.} = method listBlocks*(self: BlockStore, onBlock: OnBlock): Future[?!void] {.base.} =
## Get the list of blocks in the BlockStore. This is an intensive operation ## Get the list of blocks in the BlockStore. This is an intensive operation
## ##
raiseAssert("Not implemented!") raiseAssert("Not implemented!")
proc contains*(s: BlockStore, blk: Cid): bool = proc contains*(self: BlockStore, blk: Cid): Future[bool] {.async.} =
s.hasBlock(blk) ## Check if the block exists in the blockstore.
## Return false if error encountered
##
return (await self.hasBlock(blk)) |? false

View File

@ -44,34 +44,45 @@ const
DefaultCacheSizeMiB* = 100 DefaultCacheSizeMiB* = 100
DefaultCacheSize* = DefaultCacheSizeMiB * MiB # bytes DefaultCacheSize* = DefaultCacheSizeMiB * MiB # bytes
method getBlock*( method getBlock*(self: CacheStore, cid: Cid): Future[?! (? Block)] {.async.} =
self: CacheStore,
cid: Cid): Future[?!Block] {.async.} =
## Get a block from the stores ## Get a block from the stores
## ##
trace "Getting block from cache", cid trace "Getting block from cache", cid
if cid.isEmpty: if cid.isEmpty:
trace "Empty block, ignoring" trace "Empty block, ignoring"
return cid.emptyBlock.success return cid.emptyBlock.some.success
return self.cache[cid].catch() if cid notin self.cache:
return Block.none.success
method hasBlock*(self: CacheStore, cid: Cid): bool = try:
## check if the block exists let blk = self.cache[cid]
return blk.some.success
except CatchableError as exc:
trace "Exception requesting block", cid, exc = exc.msg
return failure(exc)
method hasBlock*(self: CacheStore, cid: Cid): Future[?!bool] {.async.} =
## Check if the block exists in the blockstore
## ##
trace "Checking for block presence in cache", cid trace "Checking CacheStore for block presence", cid
if cid.isEmpty: if cid.isEmpty:
trace "Empty block, ignoring" trace "Empty block, ignoring"
return true return true.success
cid in self.cache return (cid in self.cache).success
method listBlocks*(s: CacheStore, onBlock: OnBlock): Future[?!void] {.async.} =
## Get the list of blocks in the BlockStore. This is an intensive operation
##
method listBlocks*(s: CacheStore, onBlock: OnBlock) {.async.} =
for cid in toSeq(s.cache.keys): for cid in toSeq(s.cache.keys):
await onBlock(cid) await onBlock(cid)
return success()
func putBlockSync(self: CacheStore, blk: Block): bool = func putBlockSync(self: CacheStore, blk: Block): bool =
let blkSize = blk.data.len # in bytes let blkSize = blk.data.len # in bytes
@ -94,22 +105,19 @@ func putBlockSync(self: CacheStore, blk: Block): bool =
self.currentSize += blkSize self.currentSize += blkSize
return true return true
method putBlock*( method putBlock*(self: CacheStore, blk: Block): Future[?!void] {.async.} =
self: CacheStore,
blk: Block): Future[bool] {.async.} =
## Put a block to the blockstore ## Put a block to the blockstore
## ##
trace "Storing block in cache", cid = blk.cid trace "Storing block in cache", cid = blk.cid
if blk.isEmpty: if blk.isEmpty:
trace "Empty block, ignoring" trace "Empty block, ignoring"
return true return success()
return self.putBlockSync(blk) discard self.putBlockSync(blk)
return success()
method delBlock*( method delBlock*(self: CacheStore, cid: Cid): Future[?!void] {.async.} =
self: CacheStore,
cid: Cid): Future[?!void] {.async.} =
## Delete a block from the blockstore ## Delete a block from the blockstore
## ##

View File

@ -37,67 +37,73 @@ type
template blockPath*(self: FSStore, cid: Cid): string = template blockPath*(self: FSStore, cid: Cid): string =
self.repoDir / ($cid)[^self.postfixLen..^1] / $cid self.repoDir / ($cid)[^self.postfixLen..^1] / $cid
method getBlock*( method getBlock*(self: FSStore, cid: Cid): Future[?! (? Block)] {.async.} =
self: FSStore,
cid: Cid): Future[?!Block] {.async.} =
## Get a block from the stores ## Get a block from the stores
## ##
trace "Getting block from filestore", cid
if cid.isEmpty: if cid.isEmpty:
trace "Empty block, ignoring" trace "Empty block, ignoring"
return cid.emptyBlock.success return cid.emptyBlock.some.success
if cid in self.cache: let cachedBlock = await self.cache.getBlock(cid)
return await self.cache.getBlock(cid) if cachedBlock.isErr:
return cachedBlock
if cid notin self: if cachedBlock.get.isSome:
return Block.failure("Couldn't find block in fs store") trace "Retrieved block from cache", cid
return cachedBlock
# Read file contents
var data: seq[byte] var data: seq[byte]
let path = self.blockPath(cid) let
if ( path = self.blockPath(cid)
let res = io2.readFile(path, data); res = io2.readFile(path, data)
res.isErr):
if res.isErr:
if not isFile(path): # May be, check instead that "res.error == ERROR_FILE_NOT_FOUND" ?
return Block.none.success
else:
let error = io2.ioErrorMsg(res.error) let error = io2.ioErrorMsg(res.error)
trace "Cannot read file from fs store", path , error trace "Cannot read file from filestore", path, error
return Block.failure("Cannot read file from fs store") return failure("Cannot read file from filestore")
return Block.new(cid, data) without var blk =? Block.new(cid, data), error:
return error.failure
method putBlock*( # TODO: add block to the cache
self: FSStore, return blk.some.success
blk: Block): Future[bool] {.async.} =
## Put a block to the blockstore method putBlock*(self: FSStore, blk: Block): Future[?!void] {.async.} =
## Write block contents to file with name based on blk.cid,
## save second copy to the cache
## ##
if blk.isEmpty: if blk.isEmpty:
trace "Empty block, ignoring" trace "Empty block, ignoring"
return true return success()
if blk.cid in self:
return true
# if directory exists it wont fail
if io2.createPath(self.blockPath(blk.cid).parentDir).isErr:
trace "Unable to create block prefix dir", dir = self.blockPath(blk.cid).parentDir
return false
let path = self.blockPath(blk.cid) let path = self.blockPath(blk.cid)
if ( if isFile(path):
let res = io2.writeFile(path, blk.data); return success()
res.isErr):
# If directory exists createPath wont fail
let dir = path.parentDir
if io2.createPath(dir).isErr:
trace "Unable to create block prefix dir", dir
return failure("Unable to create block prefix dir")
let res = io2.writeFile(path, blk.data)
if res.isErr:
let error = io2.ioErrorMsg(res.error) let error = io2.ioErrorMsg(res.error)
trace "Unable to store block", path, cid = blk.cid, error trace "Unable to store block", path, cid = blk.cid, error
return false return failure("Unable to store block")
if not (await self.cache.putBlock(blk)): if isErr (await self.cache.putBlock(blk)):
trace "Unable to store block in cache", cid = blk.cid trace "Unable to store block in cache", cid = blk.cid
return true return success()
method delBlock*( method delBlock*(self: FSStore, cid: Cid): Future[?!void] {.async.} =
self: FSStore,
cid: Cid): Future[?!void] {.async.} =
## Delete a block from the blockstore ## Delete a block from the blockstore
## ##
@ -111,33 +117,35 @@ method delBlock*(
res = io2.removeFile(path) res = io2.removeFile(path)
if res.isErr: if res.isErr:
let errmsg = io2.ioErrorMsg(res.error) let error = io2.ioErrorMsg(res.error)
trace "Unable to delete block", path, cid, errmsg trace "Unable to delete block", path, cid, error
return errmsg.failure return error.failure
return await self.cache.delBlock(cid) return await self.cache.delBlock(cid)
method hasBlock*(self: FSStore, cid: Cid): bool = method hasBlock*(self: FSStore, cid: Cid): Future[?!bool] {.async.} =
## Check if the block exists in the blockstore ## Check if the block exists in the blockstore
## ##
trace "Checking for block existence", cid trace "Checking filestore for block existence", cid
if cid.isEmpty: if cid.isEmpty:
trace "Empty block, ignoring" trace "Empty block, ignoring"
return true return true.success
self.blockPath(cid).isFile() return self.blockPath(cid).isFile().success
method listBlocks*(self: FSStore, onBlock: OnBlock) {.async.} = method listBlocks*(self: FSStore, onBlock: OnBlock): Future[?!void] {.async.} =
debug "Listing all blocks in store" ## Get the list of blocks in the BlockStore. This is an intensive operation
##
trace "Listing all blocks in filestore"
for (pkind, folderPath) in self.repoDir.walkDir(): for (pkind, folderPath) in self.repoDir.walkDir():
if pkind != pcDir: continue if pkind != pcDir: continue
let baseName = basename(folderPath) if len(folderPath.basename) != self.postfixLen: continue
if baseName.len != self.postfixLen: continue
for (fkind, filePath) in folderPath.walkDir(false): for (fkind, filename) in folderPath.walkDir(relative = true):
if fkind != pcFile: continue if fkind != pcFile: continue
let cid = Cid.init(basename(filePath)) let cid = Cid.init(filename)
if cid.isOk: if cid.isOk:
# getting a weird `Error: unhandled exception: index 1 not in 0 .. 0 [IndexError]` # getting a weird `Error: unhandled exception: index 1 not in 0 .. 0 [IndexError]`
# compilation error if using different syntax/construct bellow # compilation error if using different syntax/construct bellow
@ -149,6 +157,8 @@ method listBlocks*(self: FSStore, onBlock: OnBlock) {.async.} =
except CatchableError as exc: except CatchableError as exc:
trace "Couldn't get block", cid = $(cid.get()) trace "Couldn't get block", cid = $(cid.get())
return success()
proc new*( proc new*(
T: type FSStore, T: type FSStore,
repoDir: string, repoDir: string,

View File

@ -31,42 +31,43 @@ type
engine*: BlockExcEngine # blockexc decision engine engine*: BlockExcEngine # blockexc decision engine
localStore*: BlockStore # local block store localStore*: BlockStore # local block store
method getBlock*( method getBlock*(self: NetworkStore, cid: Cid): Future[?! (? bt.Block)] {.async.} =
self: NetworkStore,
cid: Cid): Future[?!bt.Block] {.async.} =
## Get a block from a remote peer ## Get a block from a remote peer
## ##
trace "Getting block", cid trace "Getting block from network store", cid
without var blk =? (await self.localStore.getBlock(cid)):
trace "Couldn't get from local store", cid let blk = await self.localStore.getBlock(cid)
if blk.isErr:
return blk
if blk.get.isSome:
trace "Retrieved block from local store", cid
return blk
trace "Block not found in local store", cid
try: try:
blk = await self.engine.requestBlock(cid) # TODO: What if block isn't available in the engine too?
let blk = await self.engine.requestBlock(cid)
# TODO: add block to the local store
return blk.some.success
except CatchableError as exc: except CatchableError as exc:
trace "Exception requesting block", cid, exc = exc.msg trace "Exception requesting block", cid, exc = exc.msg
return failure(exc.msg) return failure(exc)
trace "Retrieved block from local store", cid method putBlock*(self: NetworkStore, blk: bt.Block): Future[?!void] {.async.} =
return blk.success ## Store block locally and notify the network
method putBlock*(
self: NetworkStore,
blk: bt.Block): Future[bool] {.async.} =
## Store block locally and notify the
## network
## ##
trace "Puting block", cid = blk.cid trace "Puting block into network store", cid = blk.cid
if not (await self.localStore.putBlock(blk)): let res = await self.localStore.putBlock(blk)
return false if res.isErr:
return res
self.engine.resolveBlocks(@[blk]) await self.engine.resolveBlocks(@[blk])
return true return success()
method delBlock*( method delBlock*(self: NetworkStore, cid: Cid): Future[?!void] =
self: NetworkStore,
cid: Cid): Future[?!void] =
## Delete a block from the blockstore ## Delete a block from the blockstore
## ##
@ -75,13 +76,12 @@ method delBlock*(
{.pop.} {.pop.}
method hasBlock*( method hasBlock*(self: NetworkStore, cid: Cid): Future[?!bool] {.async.} =
self: NetworkStore,
cid: Cid): bool =
## Check if the block exists in the blockstore ## Check if the block exists in the blockstore
## ##
self.localStore.hasBlock(cid) trace "Checking network store for block existence", cid
return await self.localStore.hasBlock(cid)
proc new*( proc new*(
T: type NetworkStore, T: type NetworkStore,

View File

@ -7,6 +7,8 @@
## This file may not be copied, modified, or distributed except according to ## This file may not be copied, modified, or distributed except according to
## those terms. ## those terms.
import std/options
import pkg/upraises import pkg/upraises
push: {.upraises: [].} push: {.upraises: [].}
@ -52,36 +54,40 @@ proc `size=`*(self: StoreStream, size: int)
{.error: "Setting the size is forbidden".} = {.error: "Setting the size is forbidden".} =
discard discard
method atEof*(self: StoreStream): bool =
self.offset >= self.size
method readOnce*( method readOnce*(
self: StoreStream, self: StoreStream,
pbytes: pointer, pbytes: pointer,
nbytes: int): Future[int] {.async.} = nbytes: int): Future[int] {.async.} =
## Read `nbytes` from current position in the StoreStream into output buffer pointed by `pbytes`.
## Return how many bytes were actually read before EOF was encountered.
## Raise exception if we are already at EOF.
trace "Reading from manifest", cid = self.manifest.cid.get(), blocks = self.manifest.len
if self.atEof: if self.atEof:
raise newLPStreamEOFError() raise newLPStreamEOFError()
var # The loop iterates over blocks in the StoreStream,
read = 0 # reading them and copying their data into outbuf
var read = 0 # Bytes read so far, and thus write offset in the outbuf
trace "Reading from manifest", cid = self.manifest.cid.get(), blocks = self.manifest.len
while read < nbytes and not self.atEof: while read < nbytes and not self.atEof:
# Compute from the current stream position `self.offset` the block num/offset to read
# Compute how many bytes to read from this block
let let
pos = self.offset div self.manifest.blockSize blockNum = self.offset div self.manifest.blockSize
blk = (await self.store.getBlock(self.manifest[pos])).tryGet() blockOffset = self.offset mod self.manifest.blockSize
readBytes = min(nbytes - read, self.manifest.blockSize - blockOffset)
blockOffset = # Read contents of block `blockNum`
if self.offset >= self.manifest.blockSize: without blkOrNone =? await self.store.getBlock(self.manifest[blockNum]), error:
self.offset mod self.manifest.blockSize raise newLPStreamReadError(error)
else: without blk =? blkOrNone:
self.offset raise newLPStreamReadError("Block not found")
trace "Reading bytes from store stream", blockNum, cid = blk.cid, bytes = readBytes, blockOffset
readBytes = # Copy `readBytes` bytes starting at `blockOffset` from the block into the outbuf
if (nbytes - read) >= (self.manifest.blockSize - blockOffset):
self.manifest.blockSize - blockOffset
else:
min(nbytes - read, self.manifest.blockSize)
trace "Reading bytes from store stream", pos, cid = blk.cid, bytes = readBytes, blockOffset = blockOffset
copyMem( copyMem(
pbytes.offset(read), pbytes.offset(read),
if blk.isEmpty: if blk.isEmpty:
@ -90,14 +96,12 @@ method readOnce*(
blk.data[blockOffset].addr, blk.data[blockOffset].addr,
readBytes) readBytes)
# Update current positions in the stream and outbuf
self.offset += readBytes self.offset += readBytes
read += readBytes read += readBytes
return read return read
method atEof*(self: StoreStream): bool =
self.offset >= self.manifest.len * self.manifest.blockSize
method closeImpl*(self: StoreStream) {.async.} = method closeImpl*(self: StoreStream) {.async.} =
try: try:
trace "Closing StoreStream" trace "Closing StoreStream"

View File

@ -82,7 +82,7 @@ suite "Block Advertising and Discovery":
blockDiscovery.findBlockProvidersHandler = blockDiscovery.findBlockProvidersHandler =
proc(d: MockDiscovery, cid: Cid): Future[seq[SignedPeerRecord]] {.async.} = proc(d: MockDiscovery, cid: Cid): Future[seq[SignedPeerRecord]] {.async.} =
engine.resolveBlocks(blocks.filterIt( it.cid == cid )) await engine.resolveBlocks(blocks.filterIt( it.cid == cid ))
await allFuturesThrowing( await allFuturesThrowing(
allFinished(pendingBlocks)) allFinished(pendingBlocks))

View File

@ -106,7 +106,7 @@ suite "NetworkStore engine - 2 nodes":
test "Should send want-have for block": test "Should send want-have for block":
let blk = bt.Block.new("Block 1".toBytes).tryGet() let blk = bt.Block.new("Block 1".toBytes).tryGet()
check await nodeCmps2.localStore.putBlock(blk) (await nodeCmps2.localStore.putBlock(blk)).tryGet()
let entry = Entry( let entry = Entry(
`block`: blk.cid.data.buffer, `block`: blk.cid.data.buffer,
@ -121,12 +121,12 @@ suite "NetworkStore engine - 2 nodes":
.taskQueue .taskQueue
.pushOrUpdateNoWait(peerCtx1).isOk .pushOrUpdateNoWait(peerCtx1).isOk
check eventually nodeCmps1.localStore.hasBlock(blk.cid) check eventually (await nodeCmps1.localStore.hasBlock(blk.cid)).tryGet()
test "Should get blocks from remote": test "Should get blocks from remote":
let blocks = await allFinished( let blocks = await allFinished(
blocks2.mapIt( nodeCmps1.networkStore.getBlock(it.cid) )) blocks2.mapIt( nodeCmps1.networkStore.getBlock(it.cid) ))
check blocks.mapIt( !it.read ) == blocks2 check blocks.mapIt( it.read().tryGet().get() ) == blocks2
test "Remote should send blocks when available": test "Remote should send blocks when available":
let blk = bt.Block.new("Block 1".toBytes).tryGet() let blk = bt.Block.new("Block 1".toBytes).tryGet()
@ -137,7 +137,7 @@ suite "NetworkStore engine - 2 nodes":
# second trigger blockexc to resolve any pending requests # second trigger blockexc to resolve any pending requests
# for the block # for the block
check await nodeCmps2.networkStore.putBlock(blk) (await nodeCmps2.networkStore.putBlock(blk)).tryGet()
# should succeed retrieving block from remote # should succeed retrieving block from remote
check await nodeCmps1.networkStore.getBlock(blk.cid) check await nodeCmps1.networkStore.getBlock(blk.cid)
@ -196,14 +196,8 @@ suite "NetworkStore - multiple nodes":
pendingBlocks1 = blocks[0..3].mapIt( engine.pendingBlocks.getWantHandle( it.cid ) ) pendingBlocks1 = blocks[0..3].mapIt( engine.pendingBlocks.getWantHandle( it.cid ) )
pendingBlocks2 = blocks[12..15].mapIt( engine.pendingBlocks.getWantHandle( it.cid )) pendingBlocks2 = blocks[12..15].mapIt( engine.pendingBlocks.getWantHandle( it.cid ))
await allFutures( for i in 0..15:
blocks[0..3].mapIt( networkStore[0].engine.localStore.putBlock(it) )) (await networkStore[i div 4].engine.localStore.putBlock(blocks[i])).tryGet()
await allFutures(
blocks[4..7].mapIt( networkStore[1].engine.localStore.putBlock(it) ))
await allFutures(
blocks[8..11].mapIt( networkStore[2].engine.localStore.putBlock(it) ))
await allFutures(
blocks[12..15].mapIt( networkStore[3].engine.localStore.putBlock(it) ))
await connectNodes(switch) await connectNodes(switch)
await sleepAsync(1.seconds) await sleepAsync(1.seconds)
@ -232,14 +226,8 @@ suite "NetworkStore - multiple nodes":
pendingBlocks1 = blocks[0..3].mapIt( engine.pendingBlocks.getWantHandle( it.cid ) ) pendingBlocks1 = blocks[0..3].mapIt( engine.pendingBlocks.getWantHandle( it.cid ) )
pendingBlocks2 = blocks[12..15].mapIt( engine.pendingBlocks.getWantHandle( it.cid )) pendingBlocks2 = blocks[12..15].mapIt( engine.pendingBlocks.getWantHandle( it.cid ))
await allFutures( for i in 0..15:
blocks[0..3].mapIt( networkStore[0].engine.localStore.putBlock(it) )) (await networkStore[i div 4].engine.localStore.putBlock(blocks[i])).tryGet()
await allFutures(
blocks[4..7].mapIt( networkStore[1].engine.localStore.putBlock(it) ))
await allFutures(
blocks[8..11].mapIt( networkStore[2].engine.localStore.putBlock(it) ))
await allFutures(
blocks[12..15].mapIt( networkStore[3].engine.localStore.putBlock(it) ))
await connectNodes(switch) await connectNodes(switch)
await sleepAsync(1.seconds) await sleepAsync(1.seconds)

View File

@ -227,8 +227,8 @@ suite "NetworkStore engine handlers":
sendPresence: sendPresence sendPresence: sendPresence
)) ))
check await engine.localStore.putBlock(blocks[0]) (await engine.localStore.putBlock(blocks[0])).tryGet()
check await engine.localStore.putBlock(blocks[1]) (await engine.localStore.putBlock(blocks[1])).tryGet()
await engine.wantListHandler(peerId, wantList) await engine.wantListHandler(peerId, wantList)
await done await done
@ -242,7 +242,8 @@ suite "NetworkStore engine handlers":
let resolved = await allFinished(pending) let resolved = await allFinished(pending)
check resolved.mapIt( it.read ) == blocks check resolved.mapIt( it.read ) == blocks
for b in blocks: for b in blocks:
check engine.localStore.hasBlock(b.cid) let present = await engine.localStore.hasBlock(b.cid)
check present.tryGet()
test "Should send payments for received blocks": test "Should send payments for received blocks":
let account = Account(address: EthAddress.example) let account = Account(address: EthAddress.example)
@ -355,7 +356,7 @@ suite "Task Handler":
blks[0].cid == blocks[1].cid blks[0].cid == blocks[1].cid
for blk in blocks: for blk in blocks:
check await engine.localStore.putBlock(blk) (await engine.localStore.putBlock(blk)).tryGet()
engine.network.request.sendBlocks = sendBlocks engine.network.request.sendBlocks = sendBlocks
# second block to send by priority # second block to send by priority
@ -393,7 +394,7 @@ suite "Task Handler":
] ]
for blk in blocks: for blk in blocks:
check await engine.localStore.putBlock(blk) (await engine.localStore.putBlock(blk)).tryGet()
engine.network.request.sendPresence = sendPresence engine.network.request.sendPresence = sendPresence
# have block # have block

View File

@ -1,3 +1,5 @@
import std/options
import pkg/chronos import pkg/chronos
import pkg/libp2p import pkg/libp2p
import pkg/libp2p/varint import pkg/libp2p/varint
@ -45,7 +47,7 @@ proc corruptBlocks*(
pos.add(i) pos.add(i)
var var
blk = (await store.getBlock(manifest[i])).tryGet() blk = (await store.getBlock(manifest[i])).tryGet().get()
bytePos: seq[int] bytePos: seq[int]
while true: while true:

View File

@ -65,12 +65,9 @@ suite "Storage Proofs Network":
let chunk = await chunker.getBytes(); let chunk = await chunker.getBytes();
chunk.len > 0): chunk.len > 0):
let let blk = bt.Block.new(chunk).tryGet()
blk = bt.Block.new(chunk).tryGet()
manifest.add(blk.cid) manifest.add(blk.cid)
if not (await store.putBlock(blk)): (await store.putBlock(blk)).tryGet()
raise newException(CatchableError, "Unable to store block " & $blk.cid)
cid = manifest.cid.tryGet() cid = manifest.cid.tryGet()
por = await PoR.init( por = await PoR.init(

View File

@ -36,12 +36,9 @@ suite "BLS PoR":
let chunk = await chunker.getBytes(); let chunk = await chunker.getBytes();
chunk.len > 0): chunk.len > 0):
let let blk = bt.Block.new(chunk).tryGet()
blk = bt.Block.new(chunk).tryGet()
manifest.add(blk.cid) manifest.add(blk.cid)
if not (await store.putBlock(blk)): (await store.putBlock(blk)).tryGet()
raise newException(CatchableError, "Unable to store block " & $blk.cid)
test "Test PoR without corruption": test "Test PoR without corruption":
let let
@ -97,12 +94,9 @@ suite "Test Serialization":
let chunk = await chunker.getBytes(); let chunk = await chunker.getBytes();
chunk.len > 0): chunk.len > 0):
let let blk = bt.Block.new(chunk).tryGet()
blk = bt.Block.new(chunk).tryGet()
manifest.add(blk.cid) manifest.add(blk.cid)
if not (await store.putBlock(blk)): (await store.putBlock(blk)).tryGet()
raise newException(CatchableError, "Unable to store block " & $blk.cid)
(spk, ssk) = st.keyGen() (spk, ssk) = st.keyGen()
por = await PoR.init( por = await PoR.init(

View File

@ -44,12 +44,9 @@ suite "Test PoR store":
let chunk = await chunker.getBytes(); let chunk = await chunker.getBytes();
chunk.len > 0): chunk.len > 0):
let let blk = bt.Block.new(chunk).tryGet()
blk = bt.Block.new(chunk).tryGet()
manifest.add(blk.cid) manifest.add(blk.cid)
if not (await store.putBlock(blk)): (await store.putBlock(blk)).tryGet()
raise newException(CatchableError, "Unable to store block " & $blk.cid)
cid = manifest.cid.tryGet() cid = manifest.cid.tryGet()
por = await PoR.init( por = await PoR.init(

View File

@ -1,4 +1,5 @@
import std/strutils import std/strutils
import std/options
import pkg/chronos import pkg/chronos
import pkg/asynctest import pkg/asynctest
@ -48,13 +49,13 @@ suite "Cache Store tests":
test "putBlock": test "putBlock":
check: (await store.putBlock(newBlock1)).tryGet()
await store.putBlock(newBlock1) check (await store.hasBlock(newBlock1.cid)).tryGet()
newBlock1.cid in store
# block size bigger than entire cache # block size bigger than entire cache
store = CacheStore.new(cacheSize = 99, chunkSize = 98) store = CacheStore.new(cacheSize = 99, chunkSize = 98)
check not await store.putBlock(newBlock1) (await store.putBlock(newBlock1)).tryGet()
check not (await store.hasBlock(newBlock1.cid)).tryGet()
# block being added causes removal of LRU block # block being added causes removal of LRU block
store = CacheStore.new( store = CacheStore.new(
@ -62,60 +63,63 @@ suite "Cache Store tests":
cacheSize = 200, cacheSize = 200,
chunkSize = 1) chunkSize = 1)
check: check:
not store.hasBlock(newBlock1.cid) not (await store.hasBlock(newBlock1.cid)).tryGet()
store.hasBlock(newBlock2.cid) (await store.hasBlock(newBlock2.cid)).tryGet()
store.hasBlock(newBlock3.cid) (await store.hasBlock(newBlock2.cid)).tryGet()
store.currentSize == newBlock2.data.len + newBlock3.data.len # 200 store.currentSize == newBlock2.data.len + newBlock3.data.len # 200
test "getBlock": test "getBlock":
store = CacheStore.new(@[newBlock]) store = CacheStore.new(@[newBlock])
let blk = await store.getBlock(newBlock.cid) let blk = await store.getBlock(newBlock.cid)
check blk.tryGet().get() == newBlock
check:
blk.isOk
blk.get == newBlock
test "fail getBlock": test "fail getBlock":
let blk = await store.getBlock(newBlock.cid) let blk = await store.getBlock(newBlock.cid)
check blk.tryGet().isNone()
check:
blk.isErr
blk.error of system.KeyError
test "hasBlock": test "hasBlock":
let store = CacheStore.new(@[newBlock]) let store = CacheStore.new(@[newBlock])
check:
check store.hasBlock(newBlock.cid) (await store.hasBlock(newBlock.cid)).tryGet()
await newBlock.cid in store
test "fail hasBlock": test "fail hasBlock":
check not store.hasBlock(newBlock.cid) check:
not (await store.hasBlock(newBlock.cid)).tryGet()
not (await newBlock.cid in store)
test "delBlock": test "delBlock":
# empty cache # empty cache
(await store.delBlock(newBlock1.cid)).tryGet() (await store.delBlock(newBlock1.cid)).tryGet()
check not (await store.hasBlock(newBlock1.cid)).tryGet()
(await store.putBlock(newBlock1)).tryGet()
check (await store.hasBlock(newBlock1.cid)).tryGet()
# successfully deleted # successfully deleted
discard await store.putBlock(newBlock1)
(await store.delBlock(newBlock1.cid)).tryGet() (await store.delBlock(newBlock1.cid)).tryGet()
check not (await store.hasBlock(newBlock1.cid)).tryGet()
# deletes item should decrement size # deletes item should decrement size
store = CacheStore.new(@[newBlock1, newBlock2, newBlock3]) store = CacheStore.new(@[newBlock1, newBlock2, newBlock3])
check: check:
store.currentSize == 300 store.currentSize == 300
(await store.delBlock(newBlock2.cid)).tryGet() (await store.delBlock(newBlock2.cid)).tryGet()
check: check:
store.currentSize == 200 store.currentSize == 200
newBlock2.cid notin store not (await store.hasBlock(newBlock2.cid)).tryGet()
test "listBlocks": test "listBlocks":
discard await store.putBlock(newBlock1) (await store.putBlock(newBlock1)).tryGet()
var listed = false var listed = false
await store.listBlocks( (await store.listBlocks(
proc(cid: Cid) {.gcsafe, async.} = proc(cid: Cid) {.gcsafe, async.} =
check cid in store check (await store.hasBlock(cid)).tryGet()
listed = true listed = true
) )).tryGet()
check listed check listed

View File

@ -1,4 +1,5 @@
import std/os import std/os
import std/options
import pkg/questionable import pkg/questionable
import pkg/questionable/results import pkg/questionable/results
@ -33,36 +34,43 @@ suite "FS Store":
removeDir(repoDir) removeDir(repoDir)
test "putBlock": test "putBlock":
check await store.putBlock(newBlock) (await store.putBlock(newBlock)).tryGet()
check fileExists(store.blockPath(newBlock.cid)) check:
check newBlock.cid in store fileExists(store.blockPath(newBlock.cid))
(await store.hasBlock(newBlock.cid)).tryGet()
await newBlock.cid in store
test "getBlock": test "getBlock":
createDir(store.blockPath(newBlock.cid).parentDir) createDir(store.blockPath(newBlock.cid).parentDir)
writeFile(store.blockPath(newBlock.cid), newBlock.data) writeFile(store.blockPath(newBlock.cid), newBlock.data)
let blk = await store.getBlock(newBlock.cid) let blk = await store.getBlock(newBlock.cid)
check blk.option == newBlock.some check blk.tryGet().get() == newBlock
test "fail getBlock": test "fail getBlock":
let blk = await store.getBlock(newBlock.cid) let blk = await store.getBlock(newBlock.cid)
check blk.isErr check blk.tryGet().isNone
test "hasBlock": test "hasBlock":
createDir(store.blockPath(newBlock.cid).parentDir) createDir(store.blockPath(newBlock.cid).parentDir)
writeFile(store.blockPath(newBlock.cid), newBlock.data) writeFile(store.blockPath(newBlock.cid), newBlock.data)
check store.hasBlock(newBlock.cid) check:
(await store.hasBlock(newBlock.cid)).tryGet()
await newBlock.cid in store
test "fail hasBlock":
check:
not (await store.hasBlock(newBlock.cid)).tryGet()
not (await newBlock.cid in store)
test "listBlocks": test "listBlocks":
createDir(store.blockPath(newBlock.cid).parentDir) createDir(store.blockPath(newBlock.cid).parentDir)
writeFile(store.blockPath(newBlock.cid), newBlock.data) writeFile(store.blockPath(newBlock.cid), newBlock.data)
await store.listBlocks( (await store.listBlocks(
proc(cid: Cid) {.gcsafe, async.} = proc(cid: Cid) {.gcsafe, async.} =
check cid == newBlock.cid) check cid == newBlock.cid
)).tryGet()
test "fail hasBlock":
check not store.hasBlock(newBlock.cid)
test "delBlock": test "delBlock":
createDir(store.blockPath(newBlock.cid).parentDir) createDir(store.blockPath(newBlock.cid).parentDir)

View File

@ -37,7 +37,7 @@ suite "Erasure encode/decode":
let blk = bt.Block.new(chunk).tryGet() let blk = bt.Block.new(chunk).tryGet()
manifest.add(blk.cid) manifest.add(blk.cid)
check (await store.putBlock(blk)) (await store.putBlock(blk)).tryGet()
proc encode(buffers, parity: int): Future[Manifest] {.async.} = proc encode(buffers, parity: int): Future[Manifest] {.async.} =
let let
@ -78,7 +78,8 @@ suite "Erasure encode/decode":
decoded.len == encoded.originalLen decoded.len == encoded.originalLen
for d in dropped: for d in dropped:
check d in store let present = await store.hasBlock(d)
check present.tryGet()
test "Should not tolerate loosing more than M data blocks in a single random column": test "Should not tolerate loosing more than M data blocks in a single random column":
const const
@ -103,7 +104,8 @@ suite "Erasure encode/decode":
decoded = (await erasure.decode(encoded)).tryGet() decoded = (await erasure.decode(encoded)).tryGet()
for d in dropped: for d in dropped:
check d notin store let present = await store.hasBlock(d)
check not present.tryGet()
test "Should tolerate loosing M data blocks in M random columns": test "Should tolerate loosing M data blocks in M random columns":
const const
@ -130,7 +132,8 @@ suite "Erasure encode/decode":
discard (await erasure.decode(encoded)).tryGet() discard (await erasure.decode(encoded)).tryGet()
for d in manifest: for d in manifest:
check d in store let present = await store.hasBlock(d)
check present.tryGet()
test "Should not tolerate loosing more than M data blocks in M random columns": test "Should not tolerate loosing more than M data blocks in M random columns":
const const
@ -179,7 +182,8 @@ suite "Erasure encode/decode":
discard (await erasure.decode(encoded)).tryGet() discard (await erasure.decode(encoded)).tryGet()
for d in manifest: for d in manifest:
check d in store let present = await store.hasBlock(d)
check present.tryGet()
test "Should tolerate loosing M (a.k.a row) contiguous parity blocks": test "Should tolerate loosing M (a.k.a row) contiguous parity blocks":
const const
@ -194,7 +198,8 @@ suite "Erasure encode/decode":
discard (await erasure.decode(encoded)).tryGet() discard (await erasure.decode(encoded)).tryGet()
for d in manifest: for d in manifest:
check d in store let present = await store.hasBlock(d)
check present.tryGet()
test "handles edge case of 0 parity blocks": test "handles edge case of 0 parity blocks":
const const

View File

@ -85,10 +85,10 @@ suite "Test Node":
manifestCid = (await storeFut).tryGet() manifestCid = (await storeFut).tryGet()
check: check:
manifestCid in localStore (await localStore.hasBlock(manifestCid)).tryGet()
var var
manifestBlock = (await localStore.getBlock(manifestCid)).tryGet() manifestBlock = (await localStore.getBlock(manifestCid)).tryGet().get()
localManifest = Manifest.decode(manifestBlock.data).tryGet() localManifest = Manifest.decode(manifestBlock.data).tryGet()
check: check:
@ -104,20 +104,18 @@ suite "Test Node":
let chunk = await chunker.getBytes(); let chunk = await chunker.getBytes();
chunk.len > 0): chunk.len > 0):
let let blk = bt.Block.new(chunk).tryGet()
blk = bt.Block.new(chunk).tryGet()
original &= chunk original &= chunk
check await localStore.putBlock(blk) (await localStore.putBlock(blk)).tryGet()
manifest.add(blk.cid) manifest.add(blk.cid)
let let
manifestBlock = bt.Block.new( manifestBlock = bt.Block.new(
manifest.encode().tryGet(), manifest.encode().tryGet(),
codec = DagPBCodec) codec = DagPBCodec
.tryGet() ).tryGet()
check await localStore.putBlock(manifestBlock) (await localStore.putBlock(manifestBlock)).tryGet()
let stream = (await node.retrieve(manifestBlock.cid)).tryGet() let stream = (await node.retrieve(manifestBlock.cid)).tryGet()
var data: seq[byte] var data: seq[byte]
@ -125,9 +123,7 @@ suite "Test Node":
var var
buf = newSeq[byte](BlockSize) buf = newSeq[byte](BlockSize)
res = await stream.readOnce(addr buf[0], BlockSize div 2) res = await stream.readOnce(addr buf[0], BlockSize div 2)
buf.setLen(res) buf.setLen(res)
data &= buf data &= buf
check data == original check data == original
@ -137,7 +133,7 @@ suite "Test Node":
testString = "Block 1" testString = "Block 1"
blk = bt.Block.new(testString.toBytes).tryGet() blk = bt.Block.new(testString.toBytes).tryGet()
check (await localStore.putBlock(blk)) (await localStore.putBlock(blk)).tryGet()
let stream = (await node.retrieve(blk.cid)).tryGet() let stream = (await node.retrieve(blk.cid)).tryGet()
var data = newSeq[byte](testString.len) var data = newSeq[byte](testString.len)

View File

@ -16,6 +16,13 @@ suite "StoreStream":
store: BlockStore store: BlockStore
stream: StoreStream stream: StoreStream
# Check that `buf` contains `size` bytes with values start, start+1...
proc sequential_bytes(buf: seq[byte], size: int, start: int): bool =
for i in 0..<size:
if int(buf[i]) != start+i:
return false
return true
let let
data = [ data = [
[byte 0, 1, 2, 3, 4, 5, 6, 7, 8, 9], [byte 0, 1, 2, 3, 4, 5, 6, 7, 8, 9],
@ -36,59 +43,63 @@ suite "StoreStream":
stream = StoreStream.new(store, manifest) stream = StoreStream.new(store, manifest)
for d in data: for d in data:
let let blk = bt.Block.new(d).tryGet()
blk = bt.Block.new(d).tryGet()
manifest.add(blk.cid) manifest.add(blk.cid)
if not (await store.putBlock(blk)): (await store.putBlock(blk)).tryGet()
raise newException(CatchableError, "Unable to store block " & $blk.cid)
test "Read all blocks < blockSize": test "Read all blocks < blockSize":
var var
buf = newSeq[byte](8) buf = newSeq[byte](8)
n = 0
while not stream.atEof: while not stream.atEof:
let let read = (await stream.readOnce(addr buf[0], buf.len))
read = (await stream.readOnce(addr buf[0], buf.len))
if stream.atEof.not: if not stream.atEof:
check read == 8 check read == 8
else: else:
check read == 4 check read == 4
check sequential_bytes(buf,read,n)
n += read
test "Read all blocks == blockSize": test "Read all blocks == blockSize":
var var
buf = newSeq[byte](10) buf = newSeq[byte](10)
n = 0
while not stream.atEof: while not stream.atEof:
let let read = (await stream.readOnce(addr buf[0], buf.len))
read = (await stream.readOnce(addr buf[0], buf.len))
check read == 10 check read == 10
check sequential_bytes(buf,read,n)
n += read
test "Read all blocks > blockSize": test "Read all blocks > blockSize":
var var
buf = newSeq[byte](11) buf = newSeq[byte](11)
n = 0
while not stream.atEof: while not stream.atEof:
let let read = (await stream.readOnce(addr buf[0], buf.len))
read = (await stream.readOnce(addr buf[0], buf.len))
if stream.atEof.not: if not stream.atEof:
check read == 11 check read == 11
else: else:
check read == 1 check read == 1
check sequential_bytes(buf,read,n)
n += read
test "Read exact bytes within block boundary": test "Read exact bytes within block boundary":
var var
buf = newSeq[byte](5) buf = newSeq[byte](5)
await stream.readExactly(addr buf[0], 5) await stream.readExactly(addr buf[0], 5)
check buf == [byte 0, 1, 2, 3, 4] check sequential_bytes(buf,5,0)
test "Read exact bytes outside of block boundary": test "Read exact bytes outside of block boundary":
var var
buf = newSeq[byte](15) buf = newSeq[byte](15)
await stream.readExactly(addr buf[0], 15) await stream.readExactly(addr buf[0], 15)
check buf == [byte 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14] check sequential_bytes(buf,15,0)