Blockexchange uses merkle root and index to fetch blocks (#566)
* Blockexchange uses merkle root and index to fetch blocks * Links the network store getTree to the local store. * Update codex/stores/repostore.nim Co-authored-by: Dmitriy Ryajov <dryajov@gmail.com> Signed-off-by: Tomasz Bekas <tomasz.bekas@gmail.com> * Rework erasure.nim to include recent cleanup * Revert accidential changes to lib versions * Addressing review comments * Storing proofs instead of trees * Fix a comment * Fix broken tests * Fix for broken testerasure.nim * Addressing PR comments --------- Signed-off-by: Tomasz Bekas <tomasz.bekas@gmail.com> Co-authored-by: benbierens <thatbenbierens@gmail.com> Co-authored-by: Dmitriy Ryajov <dryajov@gmail.com>
This commit is contained in:
parent
778093d12e
commit
2396c4d76d
|
@ -65,7 +65,7 @@ type
|
||||||
|
|
||||||
proc discoveryQueueLoop(b: DiscoveryEngine) {.async.} =
|
proc discoveryQueueLoop(b: DiscoveryEngine) {.async.} =
|
||||||
while b.discEngineRunning:
|
while b.discEngineRunning:
|
||||||
for cid in toSeq(b.pendingBlocks.wantList):
|
for cid in toSeq(b.pendingBlocks.wantListBlockCids):
|
||||||
try:
|
try:
|
||||||
await b.discoveryQueue.put(cid)
|
await b.discoveryQueue.put(cid)
|
||||||
except CatchableError as exc:
|
except CatchableError as exc:
|
||||||
|
|
|
@ -11,16 +11,18 @@ import std/sequtils
|
||||||
import std/sets
|
import std/sets
|
||||||
import std/options
|
import std/options
|
||||||
import std/algorithm
|
import std/algorithm
|
||||||
|
import std/sugar
|
||||||
|
|
||||||
import pkg/chronos
|
import pkg/chronos
|
||||||
import pkg/chronicles
|
import pkg/chronicles
|
||||||
import pkg/libp2p/[cid, switch]
|
import pkg/libp2p/[cid, switch, multihash, multicodec]
|
||||||
import pkg/metrics
|
import pkg/metrics
|
||||||
import pkg/stint
|
import pkg/stint
|
||||||
|
|
||||||
import ../../stores/blockstore
|
import ../../stores/blockstore
|
||||||
import ../../blocktype as bt
|
import ../../blocktype
|
||||||
import ../../utils
|
import ../../utils
|
||||||
|
import ../../merkletree
|
||||||
|
|
||||||
import ../protobuf/blockexc
|
import ../protobuf/blockexc
|
||||||
import ../protobuf/presence
|
import ../protobuf/presence
|
||||||
|
@ -77,12 +79,6 @@ type
|
||||||
address*: EthAddress
|
address*: EthAddress
|
||||||
price*: UInt256
|
price*: UInt256
|
||||||
|
|
||||||
proc contains*(a: AsyncHeapQueue[Entry], b: Cid): bool =
|
|
||||||
## Convenience method to check for entry prepense
|
|
||||||
##
|
|
||||||
|
|
||||||
a.anyIt( it.cid == b )
|
|
||||||
|
|
||||||
# attach task scheduler to engine
|
# attach task scheduler to engine
|
||||||
proc scheduleTask(b: BlockExcEngine, task: BlockExcPeerCtx): bool {.gcsafe} =
|
proc scheduleTask(b: BlockExcEngine, task: BlockExcPeerCtx): bool {.gcsafe} =
|
||||||
b.taskQueue.pushOrUpdateNoWait(task).isOk()
|
b.taskQueue.pushOrUpdateNoWait(task).isOk()
|
||||||
|
@ -124,22 +120,30 @@ proc stop*(b: BlockExcEngine) {.async.} =
|
||||||
|
|
||||||
trace "NetworkStore stopped"
|
trace "NetworkStore stopped"
|
||||||
|
|
||||||
proc sendWantHave(b: BlockExcEngine, cid: Cid, selectedPeer: BlockExcPeerCtx, peers: seq[BlockExcPeerCtx]): Future[void] {.async.} =
|
|
||||||
trace "Sending wantHave request to peers", cid
|
proc sendWantHave(
|
||||||
|
b: BlockExcEngine,
|
||||||
|
address: BlockAddress,
|
||||||
|
selectedPeer: BlockExcPeerCtx,
|
||||||
|
peers: seq[BlockExcPeerCtx]): Future[void] {.async.} =
|
||||||
|
trace "Sending wantHave request to peers", address
|
||||||
for p in peers:
|
for p in peers:
|
||||||
if p != selectedPeer:
|
if p != selectedPeer:
|
||||||
if cid notin p.peerHave:
|
if address notin p.peerHave:
|
||||||
trace " wantHave > ", peer = p.id
|
trace " wantHave > ", peer = p.id
|
||||||
await b.network.request.sendWantList(
|
await b.network.request.sendWantList(
|
||||||
p.id,
|
p.id,
|
||||||
@[cid],
|
@[address],
|
||||||
wantType = WantType.WantHave) # we only want to know if the peer has the block
|
wantType = WantType.WantHave) # we only want to know if the peer has the block
|
||||||
|
|
||||||
proc sendWantBlock(b: BlockExcEngine, cid: Cid, blockPeer: BlockExcPeerCtx): Future[void] {.async.} =
|
proc sendWantBlock(
|
||||||
trace "Sending wantBlock request to", peer = blockPeer.id, cid
|
b: BlockExcEngine,
|
||||||
|
address: BlockAddress,
|
||||||
|
blockPeer: BlockExcPeerCtx): Future[void] {.async.} =
|
||||||
|
trace "Sending wantBlock request to", peer = blockPeer.id, address
|
||||||
await b.network.request.sendWantList(
|
await b.network.request.sendWantList(
|
||||||
blockPeer.id,
|
blockPeer.id,
|
||||||
@[cid],
|
@[address],
|
||||||
wantType = WantType.WantBlock) # we want this remote to send us a block
|
wantType = WantType.WantBlock) # we want this remote to send us a block
|
||||||
|
|
||||||
proc findCheapestPeerForBlock(b: BlockExcEngine, cheapestPeers: seq[BlockExcPeerCtx]): ?BlockExcPeerCtx =
|
proc findCheapestPeerForBlock(b: BlockExcEngine, cheapestPeers: seq[BlockExcPeerCtx]): ?BlockExcPeerCtx =
|
||||||
|
@ -152,64 +156,63 @@ proc findCheapestPeerForBlock(b: BlockExcEngine, cheapestPeers: seq[BlockExcPeer
|
||||||
return some(peers[0])
|
return some(peers[0])
|
||||||
return some(cheapestPeers[0]) # get cheapest
|
return some(cheapestPeers[0]) # get cheapest
|
||||||
|
|
||||||
|
proc monitorBlockHandle(b: BlockExcEngine, handle: Future[Block], address: BlockAddress, peerId: PeerId) {.async.} =
|
||||||
|
try:
|
||||||
|
trace "Monitoring block handle", address, peerId
|
||||||
|
discard await handle
|
||||||
|
trace "Block handle success", address, peerId
|
||||||
|
except CatchableError as exc:
|
||||||
|
trace "Error block handle, disconnecting peer", address, exc = exc.msg, peerId
|
||||||
|
|
||||||
|
# TODO: really, this is just a quick and dirty way of
|
||||||
|
# preventing hitting the same "bad" peer every time, however,
|
||||||
|
# we might as well discover this on or next iteration, so
|
||||||
|
# it doesn't mean that we're never talking to this peer again.
|
||||||
|
# TODO: we need a lot more work around peer selection and
|
||||||
|
# prioritization
|
||||||
|
|
||||||
|
# drop unresponsive peer
|
||||||
|
b.discovery.queueFindBlocksReq(@[address.cidOrTreeCid])
|
||||||
|
await b.network.switch.disconnect(peerId)
|
||||||
|
|
||||||
|
proc requestBlock*(
|
||||||
|
b: BlockExcEngine,
|
||||||
|
address: BlockAddress,
|
||||||
|
timeout = DefaultBlockTimeout
|
||||||
|
): Future[Block] {.async.} =
|
||||||
|
let blockFuture = b.pendingBlocks.getWantHandle(address, timeout)
|
||||||
|
|
||||||
|
if b.pendingBlocks.isInFlight(address):
|
||||||
|
return await blockFuture
|
||||||
|
|
||||||
|
let peers = b.peers.selectCheapest(address)
|
||||||
|
if peers.len == 0:
|
||||||
|
b.discovery.queueFindBlocksReq(@[address.cidOrTreeCid])
|
||||||
|
|
||||||
|
let maybePeer =
|
||||||
|
if peers.len > 0:
|
||||||
|
peers[hash(address) mod peers.len].some
|
||||||
|
elif b.peers.len > 0:
|
||||||
|
toSeq(b.peers)[hash(address) mod b.peers.len].some
|
||||||
|
else:
|
||||||
|
BlockExcPeerCtx.none
|
||||||
|
|
||||||
|
if peer =? maybePeer:
|
||||||
|
asyncSpawn b.monitorBlockHandle(blockFuture, address, peer.id)
|
||||||
|
b.pendingBlocks.setInFlight(address)
|
||||||
|
await b.sendWantBlock(address, peer)
|
||||||
|
codex_block_exchange_want_block_lists_sent.inc()
|
||||||
|
await b.sendWantHave(address, peer, toSeq(b.peers))
|
||||||
|
codex_block_exchange_want_have_lists_sent.inc()
|
||||||
|
|
||||||
|
return await blockFuture
|
||||||
|
|
||||||
proc requestBlock*(
|
proc requestBlock*(
|
||||||
b: BlockExcEngine,
|
b: BlockExcEngine,
|
||||||
cid: Cid,
|
cid: Cid,
|
||||||
timeout = DefaultBlockTimeout): Future[bt.Block] {.async.} =
|
timeout = DefaultBlockTimeout
|
||||||
trace "Begin block request", cid, peers = b.peers.len
|
): Future[Block] =
|
||||||
|
b.requestBlock(BlockAddress.init(cid))
|
||||||
if b.pendingBlocks.isInFlight(cid):
|
|
||||||
trace "Request handle already pending", cid
|
|
||||||
return await b.pendingBlocks.getWantHandle(cid, timeout)
|
|
||||||
|
|
||||||
let
|
|
||||||
blk = b.pendingBlocks.getWantHandle(cid, timeout)
|
|
||||||
|
|
||||||
trace "Selecting peers who have", cid
|
|
||||||
var
|
|
||||||
peers = b.peers.selectCheapest(cid)
|
|
||||||
|
|
||||||
without blockPeer =? b.findCheapestPeerForBlock(peers):
|
|
||||||
trace "No peers to request blocks from. Queue discovery...", cid
|
|
||||||
b.discovery.queueFindBlocksReq(@[cid])
|
|
||||||
return await blk
|
|
||||||
|
|
||||||
proc blockHandleMonitor() {.async.} =
|
|
||||||
try:
|
|
||||||
trace "Monitoring block handle", cid
|
|
||||||
b.pendingBlocks.setInFlight(cid, true)
|
|
||||||
discard await blk
|
|
||||||
trace "Block handle success", cid
|
|
||||||
except CatchableError as exc:
|
|
||||||
trace "Error block handle, disconnecting peer", cid, exc = exc.msg
|
|
||||||
|
|
||||||
# TODO: really, this is just a quick and dirty way of
|
|
||||||
# preventing hitting the same "bad" peer every time, however,
|
|
||||||
# we might as well discover this on or next iteration, so
|
|
||||||
# it doesn't mean that we're never talking to this peer again.
|
|
||||||
# TODO: we need a lot more work around peer selection and
|
|
||||||
# prioritization
|
|
||||||
|
|
||||||
# drop unresponsive peer
|
|
||||||
await b.network.switch.disconnect(blockPeer.id)
|
|
||||||
|
|
||||||
# monitor block handle
|
|
||||||
asyncSpawn blockHandleMonitor()
|
|
||||||
|
|
||||||
await b.sendWantBlock(cid, blockPeer)
|
|
||||||
|
|
||||||
codex_block_exchange_want_block_lists_sent.inc()
|
|
||||||
|
|
||||||
if (peers.len - 1) == 0:
|
|
||||||
trace "No peers to send want list to", cid
|
|
||||||
b.discovery.queueFindBlocksReq(@[cid])
|
|
||||||
return await blk
|
|
||||||
|
|
||||||
await b.sendWantHave(cid, blockPeer, toSeq(b.peers))
|
|
||||||
|
|
||||||
codex_block_exchange_want_have_lists_sent.inc()
|
|
||||||
|
|
||||||
return await blk
|
|
||||||
|
|
||||||
proc blockPresenceHandler*(
|
proc blockPresenceHandler*(
|
||||||
b: BlockExcEngine,
|
b: BlockExcEngine,
|
||||||
|
@ -226,7 +229,7 @@ proc blockPresenceHandler*(
|
||||||
for blk in blocks:
|
for blk in blocks:
|
||||||
if presence =? Presence.init(blk):
|
if presence =? Presence.init(blk):
|
||||||
logScope:
|
logScope:
|
||||||
cid = presence.cid
|
address = $presence.address
|
||||||
have = presence.have
|
have = presence.have
|
||||||
price = presence.price
|
price = presence.price
|
||||||
|
|
||||||
|
@ -255,22 +258,22 @@ proc blockPresenceHandler*(
|
||||||
# if none of the connected peers report our wants in their have list,
|
# if none of the connected peers report our wants in their have list,
|
||||||
# fire up discovery
|
# fire up discovery
|
||||||
b.discovery.queueFindBlocksReq(
|
b.discovery.queueFindBlocksReq(
|
||||||
toSeq(b.pendingBlocks.wantList)
|
toSeq(b.pendingBlocks.wantListCids)
|
||||||
.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.peerHaveCids ))
|
||||||
|
|
||||||
proc scheduleTasks(b: BlockExcEngine, blocks: seq[bt.Block]) {.async.} =
|
proc scheduleTasks(b: BlockExcEngine, blocksDelivery: seq[BlockDelivery]) {.async.} =
|
||||||
trace "Schedule a task for new blocks", items = blocks.len
|
trace "Schedule a task for new blocks", items = blocksDelivery.len
|
||||||
|
|
||||||
let
|
let
|
||||||
cids = blocks.mapIt( it.cid )
|
cids = blocksDelivery.mapIt( it.blk.cid )
|
||||||
|
|
||||||
# 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 cid
|
# schedule a peer if it wants at least one cid
|
||||||
# and we have it in our local store
|
# and we have it in our local store
|
||||||
if c in p.peerWants:
|
if c in p.peerWantsCids:
|
||||||
if await (c in b.localStore):
|
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
|
||||||
|
@ -279,50 +282,110 @@ proc scheduleTasks(b: BlockExcEngine, blocks: seq[bt.Block]) {.async.} =
|
||||||
|
|
||||||
break # do next peer
|
break # do next peer
|
||||||
|
|
||||||
proc resolveBlocks*(b: BlockExcEngine, blocks: seq[bt.Block]) {.async.} =
|
proc resolveBlocks*(b: BlockExcEngine, blocksDelivery: seq[BlockDelivery]) {.async.} =
|
||||||
trace "Resolving blocks", blocks = blocks.len
|
trace "Resolving blocks", blocks = blocksDelivery.len
|
||||||
|
|
||||||
b.pendingBlocks.resolve(blocks)
|
b.pendingBlocks.resolve(blocksDelivery)
|
||||||
await b.scheduleTasks(blocks)
|
await b.scheduleTasks(blocksDelivery)
|
||||||
b.discovery.queueProvideBlocksReq(blocks.mapIt( it.cid ))
|
var cids = initHashSet[Cid]()
|
||||||
|
for bd in blocksDelivery:
|
||||||
|
cids.incl(bd.blk.cid)
|
||||||
|
if bd.address.leaf:
|
||||||
|
cids.incl(bd.address.treeCid)
|
||||||
|
b.discovery.queueProvideBlocksReq(cids.toSeq)
|
||||||
|
|
||||||
|
proc resolveBlocks*(b: BlockExcEngine, blocks: seq[Block]) {.async.} =
|
||||||
|
await b.resolveBlocks(blocks.mapIt(BlockDelivery(blk: it, address: BlockAddress(leaf: false, cid: it.cid))))
|
||||||
|
|
||||||
proc payForBlocks(engine: BlockExcEngine,
|
proc payForBlocks(engine: BlockExcEngine,
|
||||||
peer: BlockExcPeerCtx,
|
peer: BlockExcPeerCtx,
|
||||||
blocks: seq[bt.Block]) {.async.} =
|
blocksDelivery: seq[BlockDelivery]) {.async.} =
|
||||||
trace "Paying for blocks", blocks = blocks.len
|
trace "Paying for blocks", len = blocksDelivery.len
|
||||||
|
|
||||||
let
|
let
|
||||||
sendPayment = engine.network.request.sendPayment
|
sendPayment = engine.network.request.sendPayment
|
||||||
price = peer.price(blocks.mapIt(it.cid))
|
price = peer.price(blocksDelivery.mapIt(it.address))
|
||||||
|
|
||||||
if payment =? engine.wallet.pay(peer, price):
|
if payment =? engine.wallet.pay(peer, price):
|
||||||
trace "Sending payment for blocks", price
|
trace "Sending payment for blocks", price
|
||||||
await sendPayment(peer.id, payment)
|
await sendPayment(peer.id, payment)
|
||||||
|
|
||||||
proc blocksHandler*(
|
proc validateBlockDelivery(
|
||||||
|
b: BlockExcEngine,
|
||||||
|
bd: BlockDelivery
|
||||||
|
): ?!void =
|
||||||
|
if bd.address notin b.pendingBlocks:
|
||||||
|
return failure("Received block is not currently a pending block")
|
||||||
|
|
||||||
|
if bd.address.leaf:
|
||||||
|
without proof =? bd.proof:
|
||||||
|
return failure("Missing proof")
|
||||||
|
|
||||||
|
if proof.index != bd.address.index:
|
||||||
|
return failure("Proof index " & $proof.index & " doesn't match leaf index " & $bd.address.index)
|
||||||
|
|
||||||
|
without leaf =? bd.blk.cid.mhash.mapFailure, err:
|
||||||
|
return failure("Unable to get mhash from cid for block, nested err: " & err.msg)
|
||||||
|
|
||||||
|
without treeRoot =? bd.address.treeCid.mhash.mapFailure, err:
|
||||||
|
return failure("Unable to get mhash from treeCid for block, nested err: " & err.msg)
|
||||||
|
|
||||||
|
without verifyOutcome =? proof.verifyLeaf(leaf, treeRoot), err:
|
||||||
|
return failure("Unable to verify proof for block, nested err: " & err.msg)
|
||||||
|
|
||||||
|
if not verifyOutcome:
|
||||||
|
return failure("Provided inclusion proof is invalid")
|
||||||
|
else: # not leaf
|
||||||
|
if bd.address.cid != bd.blk.cid:
|
||||||
|
return failure("Delivery cid " & $bd.address.cid & " doesn't match block cid " & $bd.blk.cid)
|
||||||
|
|
||||||
|
return success()
|
||||||
|
|
||||||
|
proc blocksDeliveryHandler*(
|
||||||
b: BlockExcEngine,
|
b: BlockExcEngine,
|
||||||
peer: PeerId,
|
peer: PeerId,
|
||||||
blocks: seq[bt.Block]) {.async.} =
|
blocksDelivery: seq[BlockDelivery]) {.async.} =
|
||||||
trace "Got blocks from peer", peer, len = blocks.len
|
trace "Got blocks from peer", peer, len = blocksDelivery.len
|
||||||
for blk in blocks:
|
|
||||||
if isErr (await b.localStore.putBlock(blk)):
|
|
||||||
trace "Unable to store block", cid = blk.cid
|
|
||||||
|
|
||||||
await b.resolveBlocks(blocks)
|
var validatedBlocksDelivery: seq[BlockDelivery]
|
||||||
codex_block_exchange_blocks_received.inc(blocks.len.int64)
|
for bd in blocksDelivery:
|
||||||
|
logScope:
|
||||||
|
peer = peer
|
||||||
|
address = bd.address
|
||||||
|
|
||||||
|
if err =? b.validateBlockDelivery(bd).errorOption:
|
||||||
|
warn "Block validation failed", msg = err.msg
|
||||||
|
continue
|
||||||
|
|
||||||
|
if err =? (await b.localStore.putBlock(bd.blk)).errorOption:
|
||||||
|
error "Unable to store block", err = err.msg
|
||||||
|
continue
|
||||||
|
|
||||||
|
if bd.address.leaf:
|
||||||
|
without proof =? bd.proof:
|
||||||
|
error "Proof expected for a leaf block delivery"
|
||||||
|
continue
|
||||||
|
if err =? (await b.localStore.putBlockCidAndProof(bd.address.treeCid, bd.address.index, bd.blk.cid, proof)).errorOption:
|
||||||
|
error "Unable to store proof and cid for a block"
|
||||||
|
continue
|
||||||
|
|
||||||
|
validatedBlocksDelivery.add(bd)
|
||||||
|
|
||||||
|
await b.resolveBlocks(validatedBlocksDelivery)
|
||||||
|
codex_block_exchange_blocks_received.inc(validatedBlocksDelivery.len.int64)
|
||||||
|
|
||||||
let
|
let
|
||||||
peerCtx = b.peers.get(peer)
|
peerCtx = b.peers.get(peer)
|
||||||
|
|
||||||
if peerCtx != nil:
|
if peerCtx != nil:
|
||||||
await b.payForBlocks(peerCtx, blocks)
|
await b.payForBlocks(peerCtx, blocksDelivery)
|
||||||
## shouldn't we remove them from the want-list instead of this:
|
## shouldn't we remove them from the want-list instead of this:
|
||||||
peerCtx.cleanPresence(blocks.mapIt( it.cid ))
|
peerCtx.cleanPresence(blocksDelivery.mapIt( it.address ))
|
||||||
|
|
||||||
proc wantListHandler*(
|
proc wantListHandler*(
|
||||||
b: BlockExcEngine,
|
b: BlockExcEngine,
|
||||||
peer: PeerId,
|
peer: PeerId,
|
||||||
wantList: Wantlist) {.async.} =
|
wantList: WantList) {.async.} =
|
||||||
trace "Got wantList for peer", peer, items = wantList.entries.len
|
trace "Got wantList for peer", peer, items = wantList.entries.len
|
||||||
let
|
let
|
||||||
peerCtx = b.peers.get(peer)
|
peerCtx = b.peers.get(peer)
|
||||||
|
@ -338,14 +401,14 @@ proc wantListHandler*(
|
||||||
|
|
||||||
logScope:
|
logScope:
|
||||||
peer = peerCtx.id
|
peer = peerCtx.id
|
||||||
cid = e.cid
|
address = e.address
|
||||||
wantType = $e.wantType
|
wantType = $e.wantType
|
||||||
|
|
||||||
if idx < 0: # updating entry
|
if idx < 0: # updating entry
|
||||||
trace "Processing new want list entry", cid = e.cid
|
trace "Processing new want list entry"
|
||||||
|
|
||||||
let
|
let
|
||||||
have = await e.cid in b.localStore
|
have = await e.address in b.localStore
|
||||||
price = @(
|
price = @(
|
||||||
b.pricing.get(Pricing(price: 0.u256))
|
b.pricing.get(Pricing(price: 0.u256))
|
||||||
.price.toBytesBE)
|
.price.toBytesBE)
|
||||||
|
@ -354,21 +417,21 @@ proc wantListHandler*(
|
||||||
codex_block_exchange_want_have_lists_received.inc()
|
codex_block_exchange_want_have_lists_received.inc()
|
||||||
|
|
||||||
if not have and e.sendDontHave:
|
if not have and e.sendDontHave:
|
||||||
trace "Adding dont have entry to presence response", cid = e.cid
|
trace "Adding dont have entry to presence response"
|
||||||
presence.add(
|
presence.add(
|
||||||
BlockPresence(
|
BlockPresence(
|
||||||
cid: e.cid.data.buffer,
|
address: e.address,
|
||||||
`type`: BlockPresenceType.DontHave,
|
`type`: BlockPresenceType.DontHave,
|
||||||
price: price))
|
price: price))
|
||||||
elif have and e.wantType == WantType.WantHave:
|
elif have and e.wantType == WantType.WantHave:
|
||||||
trace "Adding have entry to presence response", cid = e.cid
|
trace "Adding have entry to presence response"
|
||||||
presence.add(
|
presence.add(
|
||||||
BlockPresence(
|
BlockPresence(
|
||||||
cid: e.cid.data.buffer,
|
address: e.address,
|
||||||
`type`: BlockPresenceType.Have,
|
`type`: BlockPresenceType.Have,
|
||||||
price: price))
|
price: price))
|
||||||
elif e.wantType == WantType.WantBlock:
|
elif e.wantType == WantType.WantBlock:
|
||||||
trace "Added entry to peer's want blocks list", cid = e.cid
|
trace "Added entry to peer's want blocks list"
|
||||||
peerCtx.peerWants.add(e)
|
peerCtx.peerWants.add(e)
|
||||||
codex_block_exchange_want_block_lists_received.inc()
|
codex_block_exchange_want_block_lists_received.inc()
|
||||||
else:
|
else:
|
||||||
|
@ -424,6 +487,8 @@ proc setupPeer*(b: BlockExcEngine, peer: PeerId) {.async.} =
|
||||||
## list exchange
|
## list exchange
|
||||||
##
|
##
|
||||||
|
|
||||||
|
trace "Setting up peer", peer
|
||||||
|
|
||||||
if peer notin b.peers:
|
if peer notin b.peers:
|
||||||
trace "Setting up new peer", peer
|
trace "Setting up new peer", peer
|
||||||
b.peers.add(BlockExcPeerCtx(
|
b.peers.add(BlockExcPeerCtx(
|
||||||
|
@ -432,9 +497,11 @@ proc setupPeer*(b: BlockExcEngine, peer: PeerId) {.async.} =
|
||||||
trace "Added peer", peers = b.peers.len
|
trace "Added peer", peers = b.peers.len
|
||||||
|
|
||||||
# broadcast our want list, the other peer will do the same
|
# broadcast our want list, the other peer will do the same
|
||||||
if b.pendingBlocks.len > 0:
|
if b.pendingBlocks.wantListLen > 0:
|
||||||
|
trace "Sending our want list to a peer", peer
|
||||||
|
let cids = toSeq(b.pendingBlocks.wantList)
|
||||||
await b.network.request.sendWantList(
|
await b.network.request.sendWantList(
|
||||||
peer, toSeq(b.pendingBlocks.wantList), full = true)
|
peer, cids, full = true)
|
||||||
|
|
||||||
if address =? b.pricing.?address:
|
if address =? b.pricing.?address:
|
||||||
await b.network.request.sendAccount(peer, Account(address: address))
|
await b.network.request.sendAccount(peer, Account(address: address))
|
||||||
|
@ -468,30 +535,41 @@ proc taskHandler*(b: BlockExcEngine, task: BlockExcPeerCtx) {.gcsafe, async.} =
|
||||||
|
|
||||||
wantsBlocks.sort(SortOrder.Descending)
|
wantsBlocks.sort(SortOrder.Descending)
|
||||||
|
|
||||||
|
proc localLookup(e: WantListEntry): Future[?!BlockDelivery] {.async.} =
|
||||||
|
trace "Handling lookup for entry", address = e.address
|
||||||
|
if e.address.leaf:
|
||||||
|
(await b.localStore.getBlockAndProof(e.address.treeCid, e.address.index)).map(
|
||||||
|
(blkAndProof: (Block, MerkleProof)) =>
|
||||||
|
BlockDelivery(address: e.address, blk: blkAndProof[0], proof: blkAndProof[1].some)
|
||||||
|
)
|
||||||
|
else:
|
||||||
|
(await b.localStore.getBlock(e.address)).map(
|
||||||
|
(blk: Block) => BlockDelivery(address: e.address, blk: blk, proof: MerkleProof.none)
|
||||||
|
)
|
||||||
|
|
||||||
let
|
let
|
||||||
blockFuts = await allFinished(wantsBlocks.mapIt(
|
blocksDeliveryFut = await allFinished(wantsBlocks.map(localLookup))
|
||||||
b.localStore.getBlock(it.cid)
|
|
||||||
))
|
|
||||||
|
|
||||||
# Extract successfully received blocks
|
# Extract successfully received blocks
|
||||||
let
|
let
|
||||||
blocks = blockFuts
|
blocksDelivery = blocksDeliveryFut
|
||||||
.filterIt(it.completed and it.read.isOk)
|
.filterIt(it.completed and it.read.isOk)
|
||||||
.mapIt(it.read.get)
|
.mapIt(it.read.get)
|
||||||
|
|
||||||
if blocks.len > 0:
|
if blocksDelivery.len > 0:
|
||||||
trace "Sending blocks to peer", peer = task.id, blocks = blocks.len
|
trace "Sending blocks to peer", peer = task.id, blocks = blocksDelivery.len
|
||||||
await b.network.request.sendBlocks(
|
await b.network.request.sendBlocksDelivery(
|
||||||
task.id,
|
task.id,
|
||||||
blocks)
|
blocksDelivery
|
||||||
|
)
|
||||||
|
|
||||||
codex_block_exchange_blocks_sent.inc(blocks.len.int64)
|
codex_block_exchange_blocks_sent.inc(blocksDelivery.len.int64)
|
||||||
|
|
||||||
trace "About to remove entries from peerWants", blocks = blocks.len, items = task.peerWants.len
|
trace "About to remove entries from peerWants", blocks = blocksDelivery.len, items = task.peerWants.len
|
||||||
# Remove successfully sent blocks
|
# Remove successfully sent blocks
|
||||||
task.peerWants.keepIf(
|
task.peerWants.keepIf(
|
||||||
proc(e: Entry): bool =
|
proc(e: WantListEntry): bool =
|
||||||
not blocks.anyIt( it.cid == e.cid )
|
not blocksDelivery.anyIt( it.address == e.address )
|
||||||
)
|
)
|
||||||
trace "Removed entries from peerWants", items = task.peerWants.len
|
trace "Removed entries from peerWants", items = task.peerWants.len
|
||||||
|
|
||||||
|
@ -547,7 +625,7 @@ proc new*(
|
||||||
|
|
||||||
proc blockWantListHandler(
|
proc blockWantListHandler(
|
||||||
peer: PeerId,
|
peer: PeerId,
|
||||||
wantList: Wantlist): Future[void] {.gcsafe.} =
|
wantList: WantList): Future[void] {.gcsafe.} =
|
||||||
engine.wantListHandler(peer, wantList)
|
engine.wantListHandler(peer, wantList)
|
||||||
|
|
||||||
proc blockPresenceHandler(
|
proc blockPresenceHandler(
|
||||||
|
@ -555,10 +633,10 @@ proc new*(
|
||||||
presence: seq[BlockPresence]): Future[void] {.gcsafe.} =
|
presence: seq[BlockPresence]): Future[void] {.gcsafe.} =
|
||||||
engine.blockPresenceHandler(peer, presence)
|
engine.blockPresenceHandler(peer, presence)
|
||||||
|
|
||||||
proc blocksHandler(
|
proc blocksDeliveryHandler(
|
||||||
peer: PeerId,
|
peer: PeerId,
|
||||||
blocks: seq[bt.Block]): Future[void] {.gcsafe.} =
|
blocksDelivery: seq[BlockDelivery]): Future[void] {.gcsafe.} =
|
||||||
engine.blocksHandler(peer, blocks)
|
engine.blocksDeliveryHandler(peer, blocksDelivery)
|
||||||
|
|
||||||
proc accountHandler(peer: PeerId, account: Account): Future[void] {.gcsafe.} =
|
proc accountHandler(peer: PeerId, account: Account): Future[void] {.gcsafe.} =
|
||||||
engine.accountHandler(peer, account)
|
engine.accountHandler(peer, account)
|
||||||
|
@ -568,7 +646,7 @@ proc new*(
|
||||||
|
|
||||||
network.handlers = BlockExcHandlers(
|
network.handlers = BlockExcHandlers(
|
||||||
onWantList: blockWantListHandler,
|
onWantList: blockWantListHandler,
|
||||||
onBlocks: blocksHandler,
|
onBlocksDelivery: blocksDeliveryHandler,
|
||||||
onPresence: blockPresenceHandler,
|
onPresence: blockPresenceHandler,
|
||||||
onAccount: accountHandler,
|
onAccount: accountHandler,
|
||||||
onPayment: paymentHandler)
|
onPayment: paymentHandler)
|
||||||
|
|
|
@ -18,8 +18,11 @@ import pkg/chronicles
|
||||||
import pkg/chronos
|
import pkg/chronos
|
||||||
import pkg/libp2p
|
import pkg/libp2p
|
||||||
import pkg/metrics
|
import pkg/metrics
|
||||||
|
import pkg/questionable/results
|
||||||
|
|
||||||
|
import ../protobuf/blockexc
|
||||||
import ../../blocktype
|
import ../../blocktype
|
||||||
|
import ../../merkletree
|
||||||
|
|
||||||
logScope:
|
logScope:
|
||||||
topics = "codex pendingblocks"
|
topics = "codex pendingblocks"
|
||||||
|
@ -37,14 +40,14 @@ type
|
||||||
startTime*: int64
|
startTime*: int64
|
||||||
|
|
||||||
PendingBlocksManager* = ref object of RootObj
|
PendingBlocksManager* = ref object of RootObj
|
||||||
blocks*: Table[Cid, BlockReq] # pending Block requests
|
blocks*: Table[BlockAddress, BlockReq] # pending Block requests
|
||||||
|
|
||||||
proc updatePendingBlockGauge(p: PendingBlocksManager) =
|
proc updatePendingBlockGauge(p: PendingBlocksManager) =
|
||||||
codex_block_exchange_pending_block_requests.set(p.blocks.len.int64)
|
codex_block_exchange_pending_block_requests.set(p.blocks.len.int64)
|
||||||
|
|
||||||
proc getWantHandle*(
|
proc getWantHandle*(
|
||||||
p: PendingBlocksManager,
|
p: PendingBlocksManager,
|
||||||
cid: Cid,
|
address: BlockAddress,
|
||||||
timeout = DefaultBlockTimeout,
|
timeout = DefaultBlockTimeout,
|
||||||
inFlight = false
|
inFlight = false
|
||||||
): Future[Block] {.async.} =
|
): Future[Block] {.async.} =
|
||||||
|
@ -52,73 +55,106 @@ proc getWantHandle*(
|
||||||
##
|
##
|
||||||
|
|
||||||
try:
|
try:
|
||||||
if cid notin p.blocks:
|
if address notin p.blocks:
|
||||||
p.blocks[cid] = BlockReq(
|
p.blocks[address] = BlockReq(
|
||||||
handle: newFuture[Block]("pendingBlocks.getWantHandle"),
|
handle: newFuture[Block]("pendingBlocks.getWantHandle"),
|
||||||
inFlight: inFlight,
|
inFlight: inFlight,
|
||||||
startTime: getMonoTime().ticks)
|
startTime: getMonoTime().ticks)
|
||||||
|
|
||||||
trace "Adding pending future for block", cid, inFlight = p.blocks[cid].inFlight
|
trace "Adding pending future for block", address, inFlight = p.blocks[address].inFlight
|
||||||
|
|
||||||
p.updatePendingBlockGauge()
|
p.updatePendingBlockGauge()
|
||||||
return await p.blocks[cid].handle.wait(timeout)
|
return await p.blocks[address].handle.wait(timeout)
|
||||||
except CancelledError as exc:
|
except CancelledError as exc:
|
||||||
trace "Blocks cancelled", exc = exc.msg, cid
|
trace "Blocks cancelled", exc = exc.msg, address
|
||||||
raise exc
|
raise exc
|
||||||
except CatchableError as exc:
|
except CatchableError as exc:
|
||||||
trace "Pending WANT failed or expired", exc = exc.msg
|
trace "Pending WANT failed or expired", exc = exc.msg
|
||||||
# no need to cancel, it is already cancelled by wait()
|
# no need to cancel, it is already cancelled by wait()
|
||||||
raise exc
|
raise exc
|
||||||
finally:
|
finally:
|
||||||
p.blocks.del(cid)
|
p.blocks.del(address)
|
||||||
p.updatePendingBlockGauge()
|
p.updatePendingBlockGauge()
|
||||||
|
|
||||||
proc resolve*(p: PendingBlocksManager,
|
proc getWantHandle*(
|
||||||
blocks: seq[Block]) =
|
p: PendingBlocksManager,
|
||||||
|
cid: Cid,
|
||||||
|
timeout = DefaultBlockTimeout,
|
||||||
|
inFlight = false
|
||||||
|
): Future[Block] =
|
||||||
|
p.getWantHandle(BlockAddress.init(cid), timeout, inFlight)
|
||||||
|
|
||||||
|
proc resolve*(
|
||||||
|
p: PendingBlocksManager,
|
||||||
|
blocksDelivery: seq[BlockDelivery]
|
||||||
|
) {.gcsafe, raises: [].} =
|
||||||
## Resolve pending blocks
|
## Resolve pending blocks
|
||||||
##
|
##
|
||||||
|
|
||||||
for blk in blocks:
|
for bd in blocksDelivery:
|
||||||
# resolve any pending blocks
|
p.blocks.withValue(bd.address, blockReq):
|
||||||
p.blocks.withValue(blk.cid, pending):
|
trace "Resolving block", address = bd.address
|
||||||
if not pending[].handle.completed:
|
|
||||||
trace "Resolving block", cid = blk.cid
|
if not blockReq.handle.finished:
|
||||||
pending[].handle.complete(blk)
|
|
||||||
let
|
let
|
||||||
startTime = pending[].startTime
|
startTime = blockReq.startTime
|
||||||
stopTime = getMonoTime().ticks
|
stopTime = getMonoTime().ticks
|
||||||
retrievalDurationUs = (stopTime - startTime) div 1000
|
retrievalDurationUs = (stopTime - startTime) div 1000
|
||||||
|
|
||||||
|
blockReq.handle.complete(bd.blk)
|
||||||
|
|
||||||
codex_block_exchange_retrieval_time_us.set(retrievalDurationUs)
|
codex_block_exchange_retrieval_time_us.set(retrievalDurationUs)
|
||||||
trace "Block retrieval time", retrievalDurationUs
|
trace "Block retrieval time", retrievalDurationUs, address = bd.address
|
||||||
|
else:
|
||||||
|
trace "Block handle already finished", address = bd.address
|
||||||
|
do:
|
||||||
|
warn "Attempting to resolve block that's not currently a pending block", address = bd.address
|
||||||
|
|
||||||
proc setInFlight*(p: PendingBlocksManager,
|
proc setInFlight*(p: PendingBlocksManager,
|
||||||
cid: Cid,
|
address: BlockAddress,
|
||||||
inFlight = true) =
|
inFlight = true) =
|
||||||
p.blocks.withValue(cid, pending):
|
p.blocks.withValue(address, pending):
|
||||||
pending[].inFlight = inFlight
|
pending[].inFlight = inFlight
|
||||||
trace "Setting inflight", cid, inFlight = pending[].inFlight
|
trace "Setting inflight", address, inFlight = pending[].inFlight
|
||||||
|
|
||||||
proc isInFlight*(p: PendingBlocksManager,
|
proc isInFlight*(p: PendingBlocksManager,
|
||||||
cid: Cid
|
address: BlockAddress,
|
||||||
): bool =
|
): bool =
|
||||||
p.blocks.withValue(cid, pending):
|
p.blocks.withValue(address, pending):
|
||||||
result = pending[].inFlight
|
result = pending[].inFlight
|
||||||
trace "Getting inflight", cid, inFlight = result
|
trace "Getting inflight", address, inFlight = result
|
||||||
|
|
||||||
proc pending*(p: PendingBlocksManager, cid: Cid): bool =
|
|
||||||
cid in p.blocks
|
|
||||||
|
|
||||||
proc contains*(p: PendingBlocksManager, cid: Cid): bool =
|
proc contains*(p: PendingBlocksManager, cid: Cid): bool =
|
||||||
p.pending(cid)
|
BlockAddress.init(cid) in p.blocks
|
||||||
|
|
||||||
|
proc contains*(p: PendingBlocksManager, address: BlockAddress): bool =
|
||||||
|
address in p.blocks
|
||||||
|
|
||||||
|
iterator wantList*(p: PendingBlocksManager): BlockAddress =
|
||||||
|
for a in p.blocks.keys:
|
||||||
|
yield a
|
||||||
|
|
||||||
|
iterator wantListBlockCids*(p: PendingBlocksManager): Cid =
|
||||||
|
for a in p.blocks.keys:
|
||||||
|
if not a.leaf:
|
||||||
|
yield a.cid
|
||||||
|
|
||||||
|
iterator wantListCids*(p: PendingBlocksManager): Cid =
|
||||||
|
var yieldedCids = initHashSet[Cid]()
|
||||||
|
for a in p.blocks.keys:
|
||||||
|
let cid = a.cidOrTreeCid
|
||||||
|
if cid notin yieldedCids:
|
||||||
|
yieldedCids.incl(cid)
|
||||||
|
yield cid
|
||||||
|
|
||||||
iterator wantList*(p: PendingBlocksManager): Cid =
|
|
||||||
for k in p.blocks.keys:
|
|
||||||
yield k
|
|
||||||
|
|
||||||
iterator wantHandles*(p: PendingBlocksManager): Future[Block] =
|
iterator wantHandles*(p: PendingBlocksManager): Future[Block] =
|
||||||
for v in p.blocks.values:
|
for v in p.blocks.values:
|
||||||
yield v.handle
|
yield v.handle
|
||||||
|
|
||||||
|
proc wantListLen*(p: PendingBlocksManager): int =
|
||||||
|
p.blocks.len
|
||||||
|
|
||||||
func len*(p: PendingBlocksManager): int =
|
func len*(p: PendingBlocksManager): int =
|
||||||
p.blocks.len
|
p.blocks.len
|
||||||
|
|
||||||
|
|
|
@ -34,14 +34,14 @@ const
|
||||||
MaxInflight* = 100
|
MaxInflight* = 100
|
||||||
|
|
||||||
type
|
type
|
||||||
WantListHandler* = proc(peer: PeerId, wantList: Wantlist): Future[void] {.gcsafe.}
|
WantListHandler* = proc(peer: PeerId, wantList: WantList): Future[void] {.gcsafe.}
|
||||||
BlocksHandler* = proc(peer: PeerId, blocks: seq[bt.Block]): Future[void] {.gcsafe.}
|
BlocksDeliveryHandler* = proc(peer: PeerId, blocks: seq[BlockDelivery]): Future[void] {.gcsafe.}
|
||||||
BlockPresenceHandler* = proc(peer: PeerId, precense: seq[BlockPresence]): Future[void] {.gcsafe.}
|
BlockPresenceHandler* = proc(peer: PeerId, precense: seq[BlockPresence]): Future[void] {.gcsafe.}
|
||||||
AccountHandler* = proc(peer: PeerId, account: Account): Future[void] {.gcsafe.}
|
AccountHandler* = proc(peer: PeerId, account: Account): Future[void] {.gcsafe.}
|
||||||
PaymentHandler* = proc(peer: PeerId, payment: SignedState): Future[void] {.gcsafe.}
|
PaymentHandler* = proc(peer: PeerId, payment: SignedState): Future[void] {.gcsafe.}
|
||||||
WantListSender* = proc(
|
WantListSender* = proc(
|
||||||
id: PeerId,
|
id: PeerId,
|
||||||
cids: seq[Cid],
|
addresses: seq[BlockAddress],
|
||||||
priority: int32 = 0,
|
priority: int32 = 0,
|
||||||
cancel: bool = false,
|
cancel: bool = false,
|
||||||
wantType: WantType = WantType.WantHave,
|
wantType: WantType = WantType.WantHave,
|
||||||
|
@ -50,19 +50,19 @@ type
|
||||||
|
|
||||||
BlockExcHandlers* = object
|
BlockExcHandlers* = object
|
||||||
onWantList*: WantListHandler
|
onWantList*: WantListHandler
|
||||||
onBlocks*: BlocksHandler
|
onBlocksDelivery*: BlocksDeliveryHandler
|
||||||
onPresence*: BlockPresenceHandler
|
onPresence*: BlockPresenceHandler
|
||||||
onAccount*: AccountHandler
|
onAccount*: AccountHandler
|
||||||
onPayment*: PaymentHandler
|
onPayment*: PaymentHandler
|
||||||
|
|
||||||
BlocksSender* = proc(peer: PeerId, presence: seq[bt.Block]): Future[void] {.gcsafe.}
|
BlocksDeliverySender* = proc(peer: PeerId, blocksDelivery: seq[BlockDelivery]): Future[void] {.gcsafe.}
|
||||||
PresenceSender* = proc(peer: PeerId, presence: seq[BlockPresence]): Future[void] {.gcsafe.}
|
PresenceSender* = proc(peer: PeerId, presence: seq[BlockPresence]): Future[void] {.gcsafe.}
|
||||||
AccountSender* = proc(peer: PeerId, account: Account): Future[void] {.gcsafe.}
|
AccountSender* = proc(peer: PeerId, account: Account): Future[void] {.gcsafe.}
|
||||||
PaymentSender* = proc(peer: PeerId, payment: SignedState): Future[void] {.gcsafe.}
|
PaymentSender* = proc(peer: PeerId, payment: SignedState): Future[void] {.gcsafe.}
|
||||||
|
|
||||||
BlockExcRequest* = object
|
BlockExcRequest* = object
|
||||||
sendWantList*: WantListSender
|
sendWantList*: WantListSender
|
||||||
sendBlocks*: BlocksSender
|
sendBlocksDelivery*: BlocksDeliverySender
|
||||||
sendPresence*: PresenceSender
|
sendPresence*: PresenceSender
|
||||||
sendAccount*: AccountSender
|
sendAccount*: AccountSender
|
||||||
sendPayment*: PaymentSender
|
sendPayment*: PaymentSender
|
||||||
|
@ -94,7 +94,7 @@ proc send*(b: BlockExcNetwork, id: PeerId, msg: pb.Message) {.async.} =
|
||||||
proc handleWantList(
|
proc handleWantList(
|
||||||
b: BlockExcNetwork,
|
b: BlockExcNetwork,
|
||||||
peer: NetworkPeer,
|
peer: NetworkPeer,
|
||||||
list: Wantlist) {.async.} =
|
list: WantList) {.async.} =
|
||||||
## Handle incoming want list
|
## Handle incoming want list
|
||||||
##
|
##
|
||||||
|
|
||||||
|
@ -102,32 +102,10 @@ proc handleWantList(
|
||||||
trace "Handling want list for peer", peer = peer.id, items = list.entries.len
|
trace "Handling want list for peer", peer = peer.id, items = list.entries.len
|
||||||
await b.handlers.onWantList(peer.id, list)
|
await b.handlers.onWantList(peer.id, list)
|
||||||
|
|
||||||
# TODO: make into a template
|
|
||||||
proc makeWantList*(
|
|
||||||
cids: seq[Cid],
|
|
||||||
priority: int = 0,
|
|
||||||
cancel: bool = false,
|
|
||||||
wantType: WantType = WantType.WantHave,
|
|
||||||
full: bool = false,
|
|
||||||
sendDontHave: bool = false
|
|
||||||
): Wantlist =
|
|
||||||
## make list of wanted entries
|
|
||||||
##
|
|
||||||
|
|
||||||
Wantlist(
|
|
||||||
entries: cids.mapIt(
|
|
||||||
Entry(
|
|
||||||
`block`: it.data.buffer,
|
|
||||||
priority: priority.int32,
|
|
||||||
cancel: cancel,
|
|
||||||
wantType: wantType,
|
|
||||||
sendDontHave: sendDontHave) ),
|
|
||||||
full: full)
|
|
||||||
|
|
||||||
proc sendWantList*(
|
proc sendWantList*(
|
||||||
b: BlockExcNetwork,
|
b: BlockExcNetwork,
|
||||||
id: PeerId,
|
id: PeerId,
|
||||||
cids: seq[Cid],
|
addresses: seq[BlockAddress],
|
||||||
priority: int32 = 0,
|
priority: int32 = 0,
|
||||||
cancel: bool = false,
|
cancel: bool = false,
|
||||||
wantType: WantType = WantType.WantHave,
|
wantType: WantType = WantType.WantHave,
|
||||||
|
@ -137,58 +115,40 @@ proc sendWantList*(
|
||||||
## Send a want message to peer
|
## Send a want message to peer
|
||||||
##
|
##
|
||||||
|
|
||||||
trace "Sending want list to peer", peer = id, `type` = $wantType, items = cids.len
|
trace "Sending want list to peer", peer = id, `type` = $wantType, items = addresses.len
|
||||||
let msg = makeWantList(
|
let msg = WantList(
|
||||||
cids,
|
entries: addresses.mapIt(
|
||||||
priority,
|
WantListEntry(
|
||||||
cancel,
|
address: it,
|
||||||
wantType,
|
priority: priority,
|
||||||
full,
|
cancel: cancel,
|
||||||
sendDontHave)
|
wantType: wantType,
|
||||||
|
sendDontHave: sendDontHave) ),
|
||||||
|
full: full)
|
||||||
|
|
||||||
b.send(id, Message(wantlist: msg))
|
b.send(id, Message(wantlist: msg))
|
||||||
|
|
||||||
proc handleBlocks(
|
proc handleBlocksDelivery(
|
||||||
b: BlockExcNetwork,
|
b: BlockExcNetwork,
|
||||||
peer: NetworkPeer,
|
peer: NetworkPeer,
|
||||||
blocks: seq[pb.Block]
|
blocksDelivery: seq[BlockDelivery]
|
||||||
) {.async.} =
|
) {.async.} =
|
||||||
## Handle incoming blocks
|
## Handle incoming blocks
|
||||||
##
|
##
|
||||||
|
|
||||||
if not b.handlers.onBlocks.isNil:
|
if not b.handlers.onBlocksDelivery.isNil:
|
||||||
trace "Handling blocks for peer", peer = peer.id, items = blocks.len
|
trace "Handling blocks for peer", peer = peer.id, items = blocksDelivery.len
|
||||||
|
await b.handlers.onBlocksDelivery(peer.id, blocksDelivery)
|
||||||
|
|
||||||
var blks: seq[bt.Block]
|
|
||||||
for blob in blocks:
|
|
||||||
without cid =? Cid.init(blob.prefix):
|
|
||||||
trace "Unable to initialize Cid from protobuf message"
|
|
||||||
|
|
||||||
without blk =? bt.Block.new(cid, blob.data, verify = true):
|
proc sendBlocksDelivery*(
|
||||||
trace "Unable to initialize Block from data"
|
|
||||||
|
|
||||||
blks.add(blk)
|
|
||||||
|
|
||||||
await b.handlers.onBlocks(peer.id, blks)
|
|
||||||
|
|
||||||
template makeBlocks*(blocks: seq[bt.Block]): seq[pb.Block] =
|
|
||||||
var blks: seq[pb.Block]
|
|
||||||
for blk in blocks:
|
|
||||||
blks.add(pb.Block(
|
|
||||||
prefix: blk.cid.data.buffer,
|
|
||||||
data: blk.data
|
|
||||||
))
|
|
||||||
|
|
||||||
blks
|
|
||||||
|
|
||||||
proc sendBlocks*(
|
|
||||||
b: BlockExcNetwork,
|
b: BlockExcNetwork,
|
||||||
id: PeerId,
|
id: PeerId,
|
||||||
blocks: seq[bt.Block]): Future[void] =
|
blocksDelivery: seq[BlockDelivery]): Future[void] =
|
||||||
## Send blocks to remote
|
## Send blocks to remote
|
||||||
##
|
##
|
||||||
|
|
||||||
b.send(id, pb.Message(payload: makeBlocks(blocks)))
|
b.send(id, pb.Message(payload: blocksDelivery))
|
||||||
|
|
||||||
proc handleBlockPresence(
|
proc handleBlockPresence(
|
||||||
b: BlockExcNetwork,
|
b: BlockExcNetwork,
|
||||||
|
@ -260,11 +220,11 @@ proc rpcHandler(
|
||||||
## handle rpc messages
|
## handle rpc messages
|
||||||
##
|
##
|
||||||
try:
|
try:
|
||||||
if msg.wantlist.entries.len > 0:
|
if msg.wantList.entries.len > 0:
|
||||||
asyncSpawn b.handleWantList(peer, msg.wantlist)
|
asyncSpawn b.handleWantList(peer, msg.wantList)
|
||||||
|
|
||||||
if msg.payload.len > 0:
|
if msg.payload.len > 0:
|
||||||
asyncSpawn b.handleBlocks(peer, msg.payload)
|
asyncSpawn b.handleBlocksDelivery(peer, msg.payload)
|
||||||
|
|
||||||
if msg.blockPresences.len > 0:
|
if msg.blockPresences.len > 0:
|
||||||
asyncSpawn b.handleBlockPresence(peer, msg.blockPresences)
|
asyncSpawn b.handleBlockPresence(peer, msg.blockPresences)
|
||||||
|
@ -359,7 +319,7 @@ proc new*(
|
||||||
|
|
||||||
proc sendWantList(
|
proc sendWantList(
|
||||||
id: PeerId,
|
id: PeerId,
|
||||||
cids: seq[Cid],
|
cids: seq[BlockAddress],
|
||||||
priority: int32 = 0,
|
priority: int32 = 0,
|
||||||
cancel: bool = false,
|
cancel: bool = false,
|
||||||
wantType: WantType = WantType.WantHave,
|
wantType: WantType = WantType.WantHave,
|
||||||
|
@ -369,8 +329,8 @@ proc new*(
|
||||||
id, cids, priority, cancel,
|
id, cids, priority, cancel,
|
||||||
wantType, full, sendDontHave)
|
wantType, full, sendDontHave)
|
||||||
|
|
||||||
proc sendBlocks(id: PeerId, blocks: seq[bt.Block]): Future[void] {.gcsafe.} =
|
proc sendBlocksDelivery(id: PeerId, blocksDelivery: seq[BlockDelivery]): Future[void] {.gcsafe.} =
|
||||||
self.sendBlocks(id, blocks)
|
self.sendBlocksDelivery(id, blocksDelivery)
|
||||||
|
|
||||||
proc sendPresence(id: PeerId, presence: seq[BlockPresence]): Future[void] {.gcsafe.} =
|
proc sendPresence(id: PeerId, presence: seq[BlockPresence]): Future[void] {.gcsafe.} =
|
||||||
self.sendBlockPresence(id, presence)
|
self.sendBlockPresence(id, presence)
|
||||||
|
@ -383,7 +343,7 @@ proc new*(
|
||||||
|
|
||||||
self.request = BlockExcRequest(
|
self.request = BlockExcRequest(
|
||||||
sendWantList: sendWantList,
|
sendWantList: sendWantList,
|
||||||
sendBlocks: sendBlocks,
|
sendBlocksDelivery: sendBlocksDelivery,
|
||||||
sendPresence: sendPresence,
|
sendPresence: sendPresence,
|
||||||
sendAccount: sendAccount,
|
sendAccount: sendAccount,
|
||||||
sendPayment: sendPayment)
|
sendPayment: sendPayment)
|
||||||
|
|
|
@ -9,6 +9,8 @@
|
||||||
|
|
||||||
import std/sequtils
|
import std/sequtils
|
||||||
import std/tables
|
import std/tables
|
||||||
|
import std/sugar
|
||||||
|
import std/sets
|
||||||
|
|
||||||
import pkg/chronicles
|
import pkg/chronicles
|
||||||
import pkg/libp2p
|
import pkg/libp2p
|
||||||
|
@ -20,6 +22,8 @@ import ../protobuf/blockexc
|
||||||
import ../protobuf/payments
|
import ../protobuf/payments
|
||||||
import ../protobuf/presence
|
import ../protobuf/presence
|
||||||
|
|
||||||
|
import ../../blocktype
|
||||||
|
|
||||||
export payments, nitro
|
export payments, nitro
|
||||||
|
|
||||||
logScope:
|
logScope:
|
||||||
|
@ -28,33 +32,39 @@ logScope:
|
||||||
type
|
type
|
||||||
BlockExcPeerCtx* = ref object of RootObj
|
BlockExcPeerCtx* = ref object of RootObj
|
||||||
id*: PeerId
|
id*: PeerId
|
||||||
blocks*: Table[Cid, Presence] # remote peer have list including price
|
blocks*: Table[BlockAddress, Presence] # remote peer have list including price
|
||||||
peerWants*: seq[Entry] # remote peers want lists
|
peerWants*: seq[WantListEntry] # remote peers want lists
|
||||||
exchanged*: int # times peer has exchanged with us
|
exchanged*: int # times peer has exchanged with us
|
||||||
lastExchange*: Moment # last time peer has exchanged with us
|
lastExchange*: Moment # last time peer has exchanged with us
|
||||||
account*: ?Account # ethereum account of this peer
|
account*: ?Account # ethereum account of this peer
|
||||||
paymentChannel*: ?ChannelId # payment channel id
|
paymentChannel*: ?ChannelId # payment channel id
|
||||||
|
|
||||||
proc peerHave*(self: BlockExcPeerCtx): seq[Cid] =
|
proc peerHave*(self: BlockExcPeerCtx): seq[BlockAddress] =
|
||||||
toSeq(self.blocks.keys)
|
toSeq(self.blocks.keys)
|
||||||
|
|
||||||
proc contains*(self: BlockExcPeerCtx, cid: Cid): bool =
|
proc peerHaveCids*(self: BlockExcPeerCtx): HashSet[Cid] =
|
||||||
cid in self.blocks
|
self.blocks.keys.toSeq.mapIt(it.cidOrTreeCid).toHashSet
|
||||||
|
|
||||||
|
proc peerWantsCids*(self: BlockExcPeerCtx): HashSet[Cid] =
|
||||||
|
self.peerWants.mapIt(it.address.cidOrTreeCid).toHashSet
|
||||||
|
|
||||||
|
proc contains*(self: BlockExcPeerCtx, address: BlockAddress): bool =
|
||||||
|
address in self.blocks
|
||||||
|
|
||||||
func setPresence*(self: BlockExcPeerCtx, presence: Presence) =
|
func setPresence*(self: BlockExcPeerCtx, presence: Presence) =
|
||||||
self.blocks[presence.cid] = presence
|
self.blocks[presence.address] = presence
|
||||||
|
|
||||||
func cleanPresence*(self: BlockExcPeerCtx, cids: seq[Cid]) =
|
func cleanPresence*(self: BlockExcPeerCtx, addresses: seq[BlockAddress]) =
|
||||||
for cid in cids:
|
for a in addresses:
|
||||||
self.blocks.del(cid)
|
self.blocks.del(a)
|
||||||
|
|
||||||
func cleanPresence*(self: BlockExcPeerCtx, cid: Cid) =
|
func cleanPresence*(self: BlockExcPeerCtx, address: BlockAddress) =
|
||||||
self.cleanPresence(@[cid])
|
self.cleanPresence(@[address])
|
||||||
|
|
||||||
func price*(self: BlockExcPeerCtx, cids: seq[Cid]): UInt256 =
|
func price*(self: BlockExcPeerCtx, addresses: seq[BlockAddress]): UInt256 =
|
||||||
var price = 0.u256
|
var price = 0.u256
|
||||||
for cid in cids:
|
for a in addresses:
|
||||||
self.blocks.withValue(cid, precense):
|
self.blocks.withValue(a, precense):
|
||||||
price += precense[].price
|
price += precense[].price
|
||||||
|
|
||||||
trace "Blocks price", price
|
trace "Blocks price", price
|
||||||
|
|
|
@ -20,6 +20,7 @@ import pkg/chronicles
|
||||||
import pkg/libp2p
|
import pkg/libp2p
|
||||||
|
|
||||||
import ../protobuf/blockexc
|
import ../protobuf/blockexc
|
||||||
|
import ../../blocktype
|
||||||
|
|
||||||
import ./peercontext
|
import ./peercontext
|
||||||
export peercontext
|
export peercontext
|
||||||
|
@ -59,24 +60,32 @@ func get*(self: PeerCtxStore, peerId: PeerId): BlockExcPeerCtx =
|
||||||
func len*(self: PeerCtxStore): int =
|
func len*(self: PeerCtxStore): int =
|
||||||
self.peers.len
|
self.peers.len
|
||||||
|
|
||||||
|
func peersHave*(self: PeerCtxStore, address: BlockAddress): seq[BlockExcPeerCtx] =
|
||||||
|
toSeq(self.peers.values).filterIt( it.peerHave.anyIt( it == address ) )
|
||||||
|
|
||||||
func peersHave*(self: PeerCtxStore, cid: Cid): seq[BlockExcPeerCtx] =
|
func peersHave*(self: PeerCtxStore, cid: Cid): seq[BlockExcPeerCtx] =
|
||||||
toSeq(self.peers.values).filterIt( it.peerHave.anyIt( it == cid ) )
|
toSeq(self.peers.values).filterIt( it.peerHave.anyIt( it.cidOrTreeCid == cid ) )
|
||||||
|
|
||||||
|
func peersWant*(self: PeerCtxStore, address: BlockAddress): seq[BlockExcPeerCtx] =
|
||||||
|
toSeq(self.peers.values).filterIt( it.peerWants.anyIt( it == address ) )
|
||||||
|
|
||||||
func peersWant*(self: PeerCtxStore, cid: Cid): seq[BlockExcPeerCtx] =
|
func peersWant*(self: PeerCtxStore, cid: Cid): seq[BlockExcPeerCtx] =
|
||||||
toSeq(self.peers.values).filterIt( it.peerWants.anyIt( it.cid == cid ) )
|
toSeq(self.peers.values).filterIt( it.peerWants.anyIt( it.address.cidOrTreeCid == cid ) )
|
||||||
|
|
||||||
func selectCheapest*(self: PeerCtxStore, cid: Cid): seq[BlockExcPeerCtx] =
|
func selectCheapest*(self: PeerCtxStore, address: BlockAddress): seq[BlockExcPeerCtx] =
|
||||||
var peers = self.peersHave(cid)
|
# assume that the price for all leaves in a tree is the same
|
||||||
|
let rootAddress = BlockAddress(leaf: false, cid: address.cidOrTreeCid)
|
||||||
|
var peers = self.peersHave(rootAddress)
|
||||||
|
|
||||||
func cmp(a, b: BlockExcPeerCtx): int =
|
func cmp(a, b: BlockExcPeerCtx): int =
|
||||||
var
|
var
|
||||||
priceA = 0.u256
|
priceA = 0.u256
|
||||||
priceB = 0.u256
|
priceB = 0.u256
|
||||||
|
|
||||||
a.blocks.withValue(cid, precense):
|
a.blocks.withValue(rootAddress, precense):
|
||||||
priceA = precense[].price
|
priceA = precense[].price
|
||||||
|
|
||||||
b.blocks.withValue(cid, precense):
|
b.blocks.withValue(rootAddress, precense):
|
||||||
priceB = precense[].price
|
priceB = precense[].price
|
||||||
|
|
||||||
if priceA == priceB:
|
if priceA == priceB:
|
||||||
|
|
|
@ -10,46 +10,45 @@
|
||||||
import std/hashes
|
import std/hashes
|
||||||
import std/sequtils
|
import std/sequtils
|
||||||
import pkg/libp2p
|
import pkg/libp2p
|
||||||
|
import pkg/stew/endians2
|
||||||
|
|
||||||
import message
|
import message
|
||||||
|
|
||||||
|
import ../../blocktype
|
||||||
|
|
||||||
export Message, protobufEncode, protobufDecode
|
export Message, protobufEncode, protobufDecode
|
||||||
export Wantlist, WantType, Entry
|
export Wantlist, WantType, WantListEntry
|
||||||
export Block, BlockPresenceType, BlockPresence
|
export BlockDelivery, BlockPresenceType, BlockPresence
|
||||||
export AccountMessage, StateChannelUpdate
|
export AccountMessage, StateChannelUpdate
|
||||||
|
|
||||||
proc hash*(e: Entry): Hash =
|
proc hash*(a: BlockAddress): Hash =
|
||||||
hash(e.`block`)
|
if a.leaf:
|
||||||
|
let data = a.treeCid.data.buffer & @(a.index.uint64.toBytesBE)
|
||||||
|
hash(data)
|
||||||
|
else:
|
||||||
|
hash(a.cid.data.buffer)
|
||||||
|
|
||||||
proc cid*(e: Entry): Cid =
|
proc hash*(e: WantListEntry): Hash =
|
||||||
## Helper to convert raw bytes to Cid
|
hash(e.address)
|
||||||
##
|
|
||||||
|
|
||||||
Cid.init(e.`block`).get()
|
proc contains*(a: openArray[WantListEntry], b: BlockAddress): bool =
|
||||||
|
|
||||||
proc contains*(a: openArray[Entry], b: Cid): bool =
|
|
||||||
## Convenience method to check for peer precense
|
## Convenience method to check for peer precense
|
||||||
##
|
##
|
||||||
|
|
||||||
a.filterIt( it.cid == b ).len > 0
|
a.anyIt(it.address == b)
|
||||||
|
|
||||||
proc `==`*(a: Entry, cid: Cid): bool =
|
proc `==`*(a: WantListEntry, b: BlockAddress): bool =
|
||||||
return a.cid == cid
|
return a.address == b
|
||||||
|
|
||||||
proc `<`*(a, b: Entry): bool =
|
proc `<`*(a, b: WantListEntry): bool =
|
||||||
a.priority < b.priority
|
a.priority < b.priority
|
||||||
|
|
||||||
proc cid*(e: BlockPresence): Cid =
|
|
||||||
## Helper to convert raw bytes to Cid
|
|
||||||
##
|
|
||||||
|
|
||||||
Cid.init(e.cid).get()
|
proc `==`*(a: BlockPresence, b: BlockAddress): bool =
|
||||||
|
return a.address == b
|
||||||
|
|
||||||
proc `==`*(a: BlockPresence, cid: Cid): bool =
|
proc contains*(a: openArray[BlockPresence], b: BlockAddress): bool =
|
||||||
return cid(a) == cid
|
|
||||||
|
|
||||||
proc contains*(a: openArray[BlockPresence], b: Cid): bool =
|
|
||||||
## Convenience method to check for peer precense
|
## Convenience method to check for peer precense
|
||||||
##
|
##
|
||||||
|
|
||||||
a.filterIt( cid(it) == b ).len > 0
|
a.anyIt(it.address == b)
|
||||||
|
|
|
@ -2,11 +2,18 @@
|
||||||
# and Protobuf encoder/decoder for these messages.
|
# and Protobuf encoder/decoder for these messages.
|
||||||
#
|
#
|
||||||
# Eventually all this code should be auto-generated from message.proto.
|
# Eventually all this code should be auto-generated from message.proto.
|
||||||
|
import std/sugar
|
||||||
|
|
||||||
import pkg/libp2p/protobuf/minprotobuf
|
import pkg/libp2p/protobuf/minprotobuf
|
||||||
|
import pkg/libp2p/cid
|
||||||
|
|
||||||
|
import pkg/questionable
|
||||||
|
|
||||||
import ../../units
|
import ../../units
|
||||||
|
|
||||||
|
import ../../merkletree
|
||||||
|
import ../../blocktype
|
||||||
|
|
||||||
const
|
const
|
||||||
MaxBlockSize* = 100.MiBs.uint
|
MaxBlockSize* = 100.MiBs.uint
|
||||||
MaxMessageSize* = 100.MiBs.uint
|
MaxMessageSize* = 100.MiBs.uint
|
||||||
|
@ -16,27 +23,28 @@ type
|
||||||
WantBlock = 0,
|
WantBlock = 0,
|
||||||
WantHave = 1
|
WantHave = 1
|
||||||
|
|
||||||
Entry* = object
|
WantListEntry* = object
|
||||||
`block`*: seq[byte] # The block cid
|
address*: BlockAddress
|
||||||
priority*: int32 # The priority (normalized). default to 1
|
priority*: int32 # The priority (normalized). default to 1
|
||||||
cancel*: bool # Whether this revokes an entry
|
cancel*: bool # Whether this revokes an entry
|
||||||
wantType*: WantType # Note: defaults to enum 0, ie Block
|
wantType*: WantType # Note: defaults to enum 0, ie Block
|
||||||
sendDontHave*: bool # Note: defaults to false
|
sendDontHave*: bool # Note: defaults to false
|
||||||
|
|
||||||
Wantlist* = object
|
WantList* = object
|
||||||
entries*: seq[Entry] # A list of wantlist entries
|
entries*: seq[WantListEntry] # A list of wantList entries
|
||||||
full*: bool # Whether this is the full wantlist. default to false
|
full*: bool # Whether this is the full wantList. default to false
|
||||||
|
|
||||||
Block* = object
|
BlockDelivery* = object
|
||||||
prefix*: seq[byte] # CID prefix (cid version, multicodec and multihash prefix (type + length)
|
blk*: Block
|
||||||
data*: seq[byte]
|
address*: BlockAddress
|
||||||
|
proof*: ?MerkleProof # Present only if `address.leaf` is true
|
||||||
|
|
||||||
BlockPresenceType* = enum
|
BlockPresenceType* = enum
|
||||||
Have = 0,
|
Have = 0,
|
||||||
DontHave = 1
|
DontHave = 1
|
||||||
|
|
||||||
BlockPresence* = object
|
BlockPresence* = object
|
||||||
cid*: seq[byte] # The block cid
|
address*: BlockAddress
|
||||||
`type`*: BlockPresenceType
|
`type`*: BlockPresenceType
|
||||||
price*: seq[byte] # Amount of assets to pay for the block (UInt256)
|
price*: seq[byte] # Amount of assets to pay for the block (UInt256)
|
||||||
|
|
||||||
|
@ -47,8 +55,8 @@ type
|
||||||
update*: seq[byte] # Signed Nitro state, serialized as JSON
|
update*: seq[byte] # Signed Nitro state, serialized as JSON
|
||||||
|
|
||||||
Message* = object
|
Message* = object
|
||||||
wantlist*: Wantlist
|
wantList*: WantList
|
||||||
payload*: seq[Block]
|
payload*: seq[BlockDelivery]
|
||||||
blockPresences*: seq[BlockPresence]
|
blockPresences*: seq[BlockPresence]
|
||||||
pendingBytes*: uint
|
pendingBytes*: uint
|
||||||
account*: AccountMessage
|
account*: AccountMessage
|
||||||
|
@ -58,9 +66,20 @@ type
|
||||||
# Encoding Message into seq[byte] in Protobuf format
|
# Encoding Message into seq[byte] in Protobuf format
|
||||||
#
|
#
|
||||||
|
|
||||||
proc write*(pb: var ProtoBuffer, field: int, value: Entry) =
|
proc write*(pb: var ProtoBuffer, field: int, value: BlockAddress) =
|
||||||
var ipb = initProtoBuffer()
|
var ipb = initProtoBuffer()
|
||||||
ipb.write(1, value.`block`)
|
ipb.write(1, value.leaf.uint)
|
||||||
|
if value.leaf:
|
||||||
|
ipb.write(2, value.treeCid.data.buffer)
|
||||||
|
ipb.write(3, value.index.uint64)
|
||||||
|
else:
|
||||||
|
ipb.write(4, value.cid.data.buffer)
|
||||||
|
ipb.finish()
|
||||||
|
pb.write(field, ipb)
|
||||||
|
|
||||||
|
proc write*(pb: var ProtoBuffer, field: int, value: WantListEntry) =
|
||||||
|
var ipb = initProtoBuffer()
|
||||||
|
ipb.write(1, value.address)
|
||||||
ipb.write(2, value.priority.uint64)
|
ipb.write(2, value.priority.uint64)
|
||||||
ipb.write(3, value.cancel.uint)
|
ipb.write(3, value.cancel.uint)
|
||||||
ipb.write(4, value.wantType.uint)
|
ipb.write(4, value.wantType.uint)
|
||||||
|
@ -68,7 +87,7 @@ proc write*(pb: var ProtoBuffer, field: int, value: Entry) =
|
||||||
ipb.finish()
|
ipb.finish()
|
||||||
pb.write(field, ipb)
|
pb.write(field, ipb)
|
||||||
|
|
||||||
proc write*(pb: var ProtoBuffer, field: int, value: Wantlist) =
|
proc write*(pb: var ProtoBuffer, field: int, value: WantList) =
|
||||||
var ipb = initProtoBuffer()
|
var ipb = initProtoBuffer()
|
||||||
for v in value.entries:
|
for v in value.entries:
|
||||||
ipb.write(1, v)
|
ipb.write(1, v)
|
||||||
|
@ -76,16 +95,20 @@ proc write*(pb: var ProtoBuffer, field: int, value: Wantlist) =
|
||||||
ipb.finish()
|
ipb.finish()
|
||||||
pb.write(field, ipb)
|
pb.write(field, ipb)
|
||||||
|
|
||||||
proc write*(pb: var ProtoBuffer, field: int, value: Block) =
|
proc write*(pb: var ProtoBuffer, field: int, value: BlockDelivery) =
|
||||||
var ipb = initProtoBuffer(maxSize = MaxBlockSize)
|
var ipb = initProtoBuffer(maxSize = MaxBlockSize)
|
||||||
ipb.write(1, value.prefix)
|
ipb.write(1, value.blk.cid.data.buffer)
|
||||||
ipb.write(2, value.data)
|
ipb.write(2, value.blk.data)
|
||||||
|
ipb.write(3, value.address)
|
||||||
|
if value.address.leaf:
|
||||||
|
if proof =? value.proof:
|
||||||
|
ipb.write(4, proof.encode())
|
||||||
ipb.finish()
|
ipb.finish()
|
||||||
pb.write(field, ipb)
|
pb.write(field, ipb)
|
||||||
|
|
||||||
proc write*(pb: var ProtoBuffer, field: int, value: BlockPresence) =
|
proc write*(pb: var ProtoBuffer, field: int, value: BlockPresence) =
|
||||||
var ipb = initProtoBuffer()
|
var ipb = initProtoBuffer()
|
||||||
ipb.write(1, value.cid)
|
ipb.write(1, value.address)
|
||||||
ipb.write(2, value.`type`.uint)
|
ipb.write(2, value.`type`.uint)
|
||||||
ipb.write(3, value.price)
|
ipb.write(3, value.price)
|
||||||
ipb.finish()
|
ipb.finish()
|
||||||
|
@ -105,7 +128,7 @@ proc write*(pb: var ProtoBuffer, field: int, value: StateChannelUpdate) =
|
||||||
|
|
||||||
proc protobufEncode*(value: Message): seq[byte] =
|
proc protobufEncode*(value: Message): seq[byte] =
|
||||||
var ipb = initProtoBuffer(maxSize = MaxMessageSize)
|
var ipb = initProtoBuffer(maxSize = MaxMessageSize)
|
||||||
ipb.write(1, value.wantlist)
|
ipb.write(1, value.wantList)
|
||||||
for v in value.payload:
|
for v in value.payload:
|
||||||
ipb.write(3, v)
|
ipb.write(3, v)
|
||||||
for v in value.blockPresences:
|
for v in value.blockPresences:
|
||||||
|
@ -120,12 +143,41 @@ proc protobufEncode*(value: Message): seq[byte] =
|
||||||
#
|
#
|
||||||
# Decoding Message from seq[byte] in Protobuf format
|
# Decoding Message from seq[byte] in Protobuf format
|
||||||
#
|
#
|
||||||
|
proc decode*(_: type BlockAddress, pb: ProtoBuffer): ProtoResult[BlockAddress] =
|
||||||
proc decode*(_: type Entry, pb: ProtoBuffer): ProtoResult[Entry] =
|
|
||||||
var
|
var
|
||||||
value = Entry()
|
value: BlockAddress
|
||||||
|
leaf: bool
|
||||||
field: uint64
|
field: uint64
|
||||||
discard ? pb.getField(1, value.`block`)
|
cidBuf = newSeq[byte]()
|
||||||
|
|
||||||
|
if ? pb.getField(1, field):
|
||||||
|
leaf = bool(field)
|
||||||
|
|
||||||
|
if leaf:
|
||||||
|
var
|
||||||
|
treeCid: Cid
|
||||||
|
index: Natural
|
||||||
|
if ? pb.getField(2, cidBuf):
|
||||||
|
treeCid = ? Cid.init(cidBuf).mapErr(x => ProtoError.IncorrectBlob)
|
||||||
|
if ? pb.getField(3, field):
|
||||||
|
index = field
|
||||||
|
value = BlockAddress(leaf: true, treeCid: treeCid, index: index)
|
||||||
|
else:
|
||||||
|
var cid: Cid
|
||||||
|
if ? pb.getField(4, cidBuf):
|
||||||
|
cid = ? Cid.init(cidBuf).mapErr(x => ProtoError.IncorrectBlob)
|
||||||
|
value = BlockAddress(leaf: false, cid: cid)
|
||||||
|
|
||||||
|
ok(value)
|
||||||
|
|
||||||
|
proc decode*(_: type WantListEntry, pb: ProtoBuffer): ProtoResult[WantListEntry] =
|
||||||
|
var
|
||||||
|
value = WantListEntry()
|
||||||
|
field: uint64
|
||||||
|
ipb: ProtoBuffer
|
||||||
|
buf = newSeq[byte]()
|
||||||
|
if ? pb.getField(1, ipb):
|
||||||
|
value.address = ? BlockAddress.decode(ipb)
|
||||||
if ? pb.getField(2, field):
|
if ? pb.getField(2, field):
|
||||||
value.priority = int32(field)
|
value.priority = int32(field)
|
||||||
if ? pb.getField(3, field):
|
if ? pb.getField(3, field):
|
||||||
|
@ -136,30 +188,53 @@ proc decode*(_: type Entry, pb: ProtoBuffer): ProtoResult[Entry] =
|
||||||
value.sendDontHave = bool(field)
|
value.sendDontHave = bool(field)
|
||||||
ok(value)
|
ok(value)
|
||||||
|
|
||||||
proc decode*(_: type Wantlist, pb: ProtoBuffer): ProtoResult[Wantlist] =
|
proc decode*(_: type WantList, pb: ProtoBuffer): ProtoResult[WantList] =
|
||||||
var
|
var
|
||||||
value = Wantlist()
|
value = WantList()
|
||||||
field: uint64
|
field: uint64
|
||||||
sublist: seq[seq[byte]]
|
sublist: seq[seq[byte]]
|
||||||
if ? pb.getRepeatedField(1, sublist):
|
if ? pb.getRepeatedField(1, sublist):
|
||||||
for item in sublist:
|
for item in sublist:
|
||||||
value.entries.add(? Entry.decode(initProtoBuffer(item)))
|
value.entries.add(? WantListEntry.decode(initProtoBuffer(item)))
|
||||||
if ? pb.getField(2, field):
|
if ? pb.getField(2, field):
|
||||||
value.full = bool(field)
|
value.full = bool(field)
|
||||||
ok(value)
|
ok(value)
|
||||||
|
|
||||||
proc decode*(_: type Block, pb: ProtoBuffer): ProtoResult[Block] =
|
proc decode*(_: type BlockDelivery, pb: ProtoBuffer): ProtoResult[BlockDelivery] =
|
||||||
var
|
var
|
||||||
value = Block()
|
value = BlockDelivery()
|
||||||
discard ? pb.getField(1, value.prefix)
|
field: uint64
|
||||||
discard ? pb.getField(2, value.data)
|
dataBuf = newSeq[byte]()
|
||||||
|
cidBuf = newSeq[byte]()
|
||||||
|
cid: Cid
|
||||||
|
ipb: ProtoBuffer
|
||||||
|
|
||||||
|
if ? pb.getField(1, cidBuf):
|
||||||
|
cid = ? Cid.init(cidBuf).mapErr(x => ProtoError.IncorrectBlob)
|
||||||
|
if ? pb.getField(2, dataBuf):
|
||||||
|
value.blk = ? Block.new(cid, dataBuf, verify = true).mapErr(x => ProtoError.IncorrectBlob)
|
||||||
|
if ? pb.getField(3, ipb):
|
||||||
|
value.address = ? BlockAddress.decode(ipb)
|
||||||
|
|
||||||
|
if value.address.leaf:
|
||||||
|
var proofBuf = newSeq[byte]()
|
||||||
|
if ? pb.getField(4, proofBuf):
|
||||||
|
let proof = ? MerkleProof.decode(proofBuf).mapErr(x => ProtoError.IncorrectBlob)
|
||||||
|
value.proof = proof.some
|
||||||
|
else:
|
||||||
|
value.proof = MerkleProof.none
|
||||||
|
else:
|
||||||
|
value.proof = MerkleProof.none
|
||||||
|
|
||||||
ok(value)
|
ok(value)
|
||||||
|
|
||||||
proc decode*(_: type BlockPresence, pb: ProtoBuffer): ProtoResult[BlockPresence] =
|
proc decode*(_: type BlockPresence, pb: ProtoBuffer): ProtoResult[BlockPresence] =
|
||||||
var
|
var
|
||||||
value = BlockPresence()
|
value = BlockPresence()
|
||||||
field: uint64
|
field: uint64
|
||||||
discard ? pb.getField(1, value.cid)
|
ipb: ProtoBuffer
|
||||||
|
if ? pb.getField(1, ipb):
|
||||||
|
value.address = ? BlockAddress.decode(ipb)
|
||||||
if ? pb.getField(2, field):
|
if ? pb.getField(2, field):
|
||||||
value.`type` = BlockPresenceType(field)
|
value.`type` = BlockPresenceType(field)
|
||||||
discard ? pb.getField(3, value.price)
|
discard ? pb.getField(3, value.price)
|
||||||
|
@ -184,10 +259,10 @@ proc protobufDecode*(_: type Message, msg: seq[byte]): ProtoResult[Message] =
|
||||||
ipb: ProtoBuffer
|
ipb: ProtoBuffer
|
||||||
sublist: seq[seq[byte]]
|
sublist: seq[seq[byte]]
|
||||||
if ? pb.getField(1, ipb):
|
if ? pb.getField(1, ipb):
|
||||||
value.wantlist = ? Wantlist.decode(ipb)
|
value.wantList = ? WantList.decode(ipb)
|
||||||
if ? pb.getRepeatedField(3, sublist):
|
if ? pb.getRepeatedField(3, sublist):
|
||||||
for item in sublist:
|
for item in sublist:
|
||||||
value.payload.add(? Block.decode(initProtoBuffer(item, maxSize = MaxBlockSize)))
|
value.payload.add(? BlockDelivery.decode(initProtoBuffer(item, maxSize = MaxBlockSize)))
|
||||||
if ? pb.getRepeatedField(4, sublist):
|
if ? pb.getRepeatedField(4, sublist):
|
||||||
for item in sublist:
|
for item in sublist:
|
||||||
value.blockPresences.add(? BlockPresence.decode(initProtoBuffer(item)))
|
value.blockPresences.add(? BlockPresence.decode(initProtoBuffer(item)))
|
||||||
|
|
|
@ -5,6 +5,8 @@ import pkg/questionable/results
|
||||||
import pkg/upraises
|
import pkg/upraises
|
||||||
import ./blockexc
|
import ./blockexc
|
||||||
|
|
||||||
|
import ../../blocktype
|
||||||
|
|
||||||
export questionable
|
export questionable
|
||||||
export stint
|
export stint
|
||||||
export BlockPresenceType
|
export BlockPresenceType
|
||||||
|
@ -14,7 +16,7 @@ upraises.push: {.upraises: [].}
|
||||||
type
|
type
|
||||||
PresenceMessage* = blockexc.BlockPresence
|
PresenceMessage* = blockexc.BlockPresence
|
||||||
Presence* = object
|
Presence* = object
|
||||||
cid*: Cid
|
address*: BlockAddress
|
||||||
have*: bool
|
have*: bool
|
||||||
price*: UInt256
|
price*: UInt256
|
||||||
|
|
||||||
|
@ -24,19 +26,18 @@ func parse(_: type UInt256, bytes: seq[byte]): ?UInt256 =
|
||||||
UInt256.fromBytesBE(bytes).some
|
UInt256.fromBytesBE(bytes).some
|
||||||
|
|
||||||
func init*(_: type Presence, message: PresenceMessage): ?Presence =
|
func init*(_: type Presence, message: PresenceMessage): ?Presence =
|
||||||
without cid =? Cid.init(message.cid) and
|
without price =? UInt256.parse(message.price):
|
||||||
price =? UInt256.parse(message.price):
|
|
||||||
return none Presence
|
return none Presence
|
||||||
|
|
||||||
some Presence(
|
some Presence(
|
||||||
cid: cid,
|
address: message.address,
|
||||||
have: message.`type` == BlockPresenceType.Have,
|
have: message.`type` == BlockPresenceType.Have,
|
||||||
price: price
|
price: price
|
||||||
)
|
)
|
||||||
|
|
||||||
func init*(_: type PresenceMessage, presence: Presence): PresenceMessage =
|
func init*(_: type PresenceMessage, presence: Presence): PresenceMessage =
|
||||||
PresenceMessage(
|
PresenceMessage(
|
||||||
cid: presence.cid.data.buffer,
|
address: presence.address,
|
||||||
`type`: if presence.have:
|
`type`: if presence.have:
|
||||||
BlockPresenceType.Have
|
BlockPresenceType.Have
|
||||||
else:
|
else:
|
||||||
|
|
|
@ -8,17 +8,19 @@
|
||||||
## those terms.
|
## those terms.
|
||||||
|
|
||||||
import std/tables
|
import std/tables
|
||||||
|
import std/sugar
|
||||||
export tables
|
export tables
|
||||||
|
|
||||||
import pkg/upraises
|
import pkg/upraises
|
||||||
|
|
||||||
push: {.upraises: [].}
|
push: {.upraises: [].}
|
||||||
|
|
||||||
import pkg/libp2p/[cid, multicodec]
|
import pkg/libp2p/[cid, multicodec, multihash]
|
||||||
import pkg/stew/byteutils
|
import pkg/stew/byteutils
|
||||||
import pkg/questionable
|
import pkg/questionable
|
||||||
import pkg/questionable/results
|
import pkg/questionable/results
|
||||||
import pkg/chronicles
|
import pkg/chronicles
|
||||||
|
import pkg/json_serialization
|
||||||
|
|
||||||
import ./units
|
import ./units
|
||||||
import ./utils
|
import ./utils
|
||||||
|
@ -37,91 +39,50 @@ type
|
||||||
cid*: Cid
|
cid*: Cid
|
||||||
data*: seq[byte]
|
data*: seq[byte]
|
||||||
|
|
||||||
template EmptyCid*: untyped =
|
BlockAddress* = object
|
||||||
var
|
case leaf*: bool
|
||||||
EmptyCid {.global, threadvar.}:
|
of true:
|
||||||
array[CIDv0..CIDv1, Table[MultiCodec, Cid]]
|
treeCid*: Cid
|
||||||
|
index*: Natural
|
||||||
|
else:
|
||||||
|
cid*: Cid
|
||||||
|
|
||||||
once:
|
|
||||||
EmptyCid = [
|
|
||||||
CIDv0: {
|
|
||||||
multiCodec("sha2-256"): Cid
|
|
||||||
.init("QmdfTbBqBPQ7VNxZEYEj14VmRuZBkqFbiwReogJgS1zR1n")
|
|
||||||
.get()
|
|
||||||
}.toTable,
|
|
||||||
CIDv1: {
|
|
||||||
multiCodec("sha2-256"): Cid
|
|
||||||
.init("bafybeihdwdcefgh4dqkjv67uzcmw7ojee6xedzdetojuzjevtenxquvyku")
|
|
||||||
.get()
|
|
||||||
}.toTable,
|
|
||||||
]
|
|
||||||
|
|
||||||
EmptyCid
|
proc `==`*(a, b: BlockAddress): bool =
|
||||||
|
a.leaf == b.leaf and
|
||||||
|
(
|
||||||
|
if a.leaf:
|
||||||
|
a.treeCid == b.treeCid and a.index == b.index
|
||||||
|
else:
|
||||||
|
a.cid == b.cid
|
||||||
|
)
|
||||||
|
|
||||||
template EmptyDigests*: untyped =
|
proc `$`*(a: BlockAddress): string =
|
||||||
var
|
if a.leaf:
|
||||||
EmptyDigests {.global, threadvar.}:
|
"treeCid: " & $a.treeCid & ", index: " & $a.index
|
||||||
array[CIDv0..CIDv1, Table[MultiCodec, MultiHash]]
|
else:
|
||||||
|
"cid: " & $a.cid
|
||||||
|
|
||||||
once:
|
proc writeValue*(
|
||||||
EmptyDigests = [
|
writer: var JsonWriter,
|
||||||
CIDv0: {
|
value: Cid
|
||||||
multiCodec("sha2-256"): EmptyCid[CIDv0]
|
) {.upraises:[IOError].} =
|
||||||
.catch
|
writer.writeValue($value)
|
||||||
.get()[multiCodec("sha2-256")]
|
|
||||||
.catch
|
|
||||||
.get()
|
|
||||||
.mhash
|
|
||||||
.get()
|
|
||||||
}.toTable,
|
|
||||||
CIDv1: {
|
|
||||||
multiCodec("sha2-256"): EmptyCid[CIDv1]
|
|
||||||
.catch
|
|
||||||
.get()[multiCodec("sha2-256")]
|
|
||||||
.catch
|
|
||||||
.get()
|
|
||||||
.mhash
|
|
||||||
.get()
|
|
||||||
}.toTable,
|
|
||||||
]
|
|
||||||
|
|
||||||
EmptyDigests
|
proc cidOrTreeCid*(a: BlockAddress): Cid =
|
||||||
|
if a.leaf:
|
||||||
|
a.treeCid
|
||||||
|
else:
|
||||||
|
a.cid
|
||||||
|
|
||||||
template EmptyBlock*: untyped =
|
proc address*(b: Block): BlockAddress =
|
||||||
var
|
BlockAddress(leaf: false, cid: b.cid)
|
||||||
EmptyBlock {.global, threadvar.}:
|
|
||||||
array[CIDv0..CIDv1, Table[MultiCodec, Block]]
|
|
||||||
|
|
||||||
once:
|
proc init*(_: type BlockAddress, cid: Cid): BlockAddress =
|
||||||
EmptyBlock = [
|
BlockAddress(leaf: false, cid: cid)
|
||||||
CIDv0: {
|
|
||||||
multiCodec("sha2-256"): Block(
|
|
||||||
cid: EmptyCid[CIDv0][multiCodec("sha2-256")])
|
|
||||||
}.toTable,
|
|
||||||
CIDv1: {
|
|
||||||
multiCodec("sha2-256"): Block(
|
|
||||||
cid: EmptyCid[CIDv1][multiCodec("sha2-256")])
|
|
||||||
}.toTable,
|
|
||||||
]
|
|
||||||
|
|
||||||
EmptyBlock
|
proc init*(_: type BlockAddress, treeCid: Cid, index: Natural): BlockAddress =
|
||||||
|
BlockAddress(leaf: true, treeCid: treeCid, index: index)
|
||||||
proc isEmpty*(cid: Cid): bool =
|
|
||||||
cid == EmptyCid[cid.cidver]
|
|
||||||
.catch
|
|
||||||
.get()[cid.mhash.get().mcodec]
|
|
||||||
.catch
|
|
||||||
.get()
|
|
||||||
|
|
||||||
proc isEmpty*(blk: Block): bool =
|
|
||||||
blk.cid.isEmpty
|
|
||||||
|
|
||||||
proc emptyBlock*(cid: Cid): Block =
|
|
||||||
EmptyBlock[cid.cidver]
|
|
||||||
.catch
|
|
||||||
.get()[cid.mhash.get().mcodec]
|
|
||||||
.catch
|
|
||||||
.get()
|
|
||||||
|
|
||||||
proc `$`*(b: Block): string =
|
proc `$`*(b: Block): string =
|
||||||
result &= "cid: " & $b.cid
|
result &= "cid: " & $b.cid
|
||||||
|
@ -154,17 +115,58 @@ func new*(
|
||||||
verify: bool = true
|
verify: bool = true
|
||||||
): ?!Block =
|
): ?!Block =
|
||||||
## creates a new block for both storage and network IO
|
## creates a new block for both storage and network IO
|
||||||
|
##
|
||||||
|
|
||||||
|
if verify:
|
||||||
|
let
|
||||||
|
mhash = ? cid.mhash.mapFailure
|
||||||
|
computedMhash = ? MultiHash.digest($mhash.mcodec, data).mapFailure
|
||||||
|
computedCid = ? Cid.init(cid.cidver, cid.mcodec, computedMhash).mapFailure
|
||||||
|
if computedCid != cid:
|
||||||
|
return "Cid doesn't match the data".failure
|
||||||
|
|
||||||
|
return Block(
|
||||||
|
cid: cid,
|
||||||
|
data: @data
|
||||||
|
).success
|
||||||
|
|
||||||
|
proc emptyCid*(version: CidVersion, hcodec: MultiCodec, dcodec: MultiCodec): ?!Cid =
|
||||||
|
## Returns cid representing empty content, given cid version, hash codec and data codec
|
||||||
##
|
##
|
||||||
|
|
||||||
let
|
const
|
||||||
mhash = ? cid.mhash.mapFailure
|
Sha256 = multiCodec("sha2-256")
|
||||||
b = ? Block.new(
|
Raw = multiCodec("raw")
|
||||||
data = @data,
|
DagPB = multiCodec("dag-pb")
|
||||||
version = cid.cidver,
|
DagJson = multiCodec("dag-json")
|
||||||
codec = cid.mcodec,
|
|
||||||
mcodec = mhash.mcodec)
|
|
||||||
|
|
||||||
if verify and cid != b.cid:
|
var index {.global, threadvar.}: Table[(CidVersion, MultiCodec, MultiCodec), Cid]
|
||||||
return "Cid and content don't match!".failure
|
once:
|
||||||
|
index = {
|
||||||
|
# source https://ipld.io/specs/codecs/dag-pb/fixtures/cross-codec/#dagpb_empty
|
||||||
|
(CIDv0, Sha256, DagPB): ? Cid.init("QmdfTbBqBPQ7VNxZEYEj14VmRuZBkqFbiwReogJgS1zR1n").mapFailure,
|
||||||
|
(CIDv1, Sha256, DagPB): ? Cid.init("zdj7Wkkhxcu2rsiN6GUyHCLsSLL47kdUNfjbFqBUUhMFTZKBi").mapFailure, # base36: bafybeihdwdcefgh4dqkjv67uzcmw7ojee6xedzdetojuzjevtenxquvyku
|
||||||
|
(CIDv1, Sha256, DagJson): ? Cid.init("z4EBG9jGUWMVxX9deANWX7iPyExLswe2akyF7xkNAaYgugvnhmP").mapFailure, # base36: baguqeera6mfu3g6n722vx7dbitpnbiyqnwah4ddy4b5c3rwzxc5pntqcupta
|
||||||
|
(CIDv1, Sha256, Raw): ? Cid.init("zb2rhmy65F3REf8SZp7De11gxtECBGgUKaLdiDj7MCGCHxbDW").mapFailure,
|
||||||
|
}.toTable
|
||||||
|
|
||||||
success b
|
index[(version, hcodec, dcodec)].catch
|
||||||
|
|
||||||
|
proc emptyDigest*(version: CidVersion, hcodec: MultiCodec, dcodec: MultiCodec): ?!MultiHash =
|
||||||
|
emptyCid(version, hcodec, dcodec)
|
||||||
|
.flatMap((cid: Cid) => cid.mhash.mapFailure)
|
||||||
|
|
||||||
|
proc emptyBlock*(version: CidVersion, hcodec: MultiCodec): ?!Block =
|
||||||
|
emptyCid(version, hcodec, multiCodec("raw"))
|
||||||
|
.flatMap((cid: Cid) => Block.new(cid = cid, data = @[]))
|
||||||
|
|
||||||
|
proc emptyBlock*(cid: Cid): ?!Block =
|
||||||
|
cid.mhash.mapFailure.flatMap((mhash: MultiHash) =>
|
||||||
|
emptyBlock(cid.cidver, mhash.mcodec))
|
||||||
|
|
||||||
|
proc isEmpty*(cid: Cid): bool =
|
||||||
|
success(cid) == cid.mhash.mapFailure.flatMap((mhash: MultiHash) =>
|
||||||
|
emptyCid(cid.cidver, mhash.mcodec, cid.mcodec))
|
||||||
|
|
||||||
|
proc isEmpty*(blk: Block): bool =
|
||||||
|
blk.cid.isEmpty
|
||||||
|
|
|
@ -12,15 +12,21 @@ import pkg/upraises
|
||||||
push: {.upraises: [].}
|
push: {.upraises: [].}
|
||||||
|
|
||||||
import std/sequtils
|
import std/sequtils
|
||||||
import std/options
|
import std/sugar
|
||||||
|
|
||||||
import pkg/chronos
|
import pkg/chronos
|
||||||
import pkg/chronicles
|
import pkg/chronicles
|
||||||
import pkg/questionable
|
import pkg/libp2p/[multicodec, cid, multibase, multihash]
|
||||||
|
import pkg/libp2p/protobuf/minprotobuf
|
||||||
|
|
||||||
import ../manifest
|
import ../manifest
|
||||||
|
import ../merkletree
|
||||||
import ../stores
|
import ../stores
|
||||||
import ../blocktype as bt
|
import ../blocktype as bt
|
||||||
|
import ../utils
|
||||||
|
import ../utils/asynciter
|
||||||
|
|
||||||
|
import pkg/stew/byteutils
|
||||||
|
|
||||||
import ./backend
|
import ./backend
|
||||||
|
|
||||||
|
@ -64,12 +70,14 @@ type
|
||||||
decoderProvider*: DecoderProvider
|
decoderProvider*: DecoderProvider
|
||||||
store*: BlockStore
|
store*: BlockStore
|
||||||
|
|
||||||
GetNext = proc(): Future[?(bt.Block, int)] {.upraises: [], gcsafe, closure.}
|
EncodingParams = object
|
||||||
PendingBlocksIter* = ref object
|
ecK: int
|
||||||
finished*: bool
|
ecM: int
|
||||||
next*: GetNext
|
rounded: int
|
||||||
|
steps: int
|
||||||
|
blocksCount: int
|
||||||
|
|
||||||
func indexToPos(self: Erasure, encoded: Manifest, idx, step: int): int {.inline.} =
|
func indexToPos(steps, idx, step: int): int {.inline.} =
|
||||||
## Convert an index to a position in the encoded
|
## Convert an index to a position in the encoded
|
||||||
## dataset
|
## dataset
|
||||||
## `idx` - the index to convert
|
## `idx` - the index to convert
|
||||||
|
@ -77,93 +85,71 @@ func indexToPos(self: Erasure, encoded: Manifest, idx, step: int): int {.inline.
|
||||||
## `pos` - the position in the encoded dataset
|
## `pos` - the position in the encoded dataset
|
||||||
##
|
##
|
||||||
|
|
||||||
(idx - step) div encoded.steps
|
(idx - step) div steps
|
||||||
|
|
||||||
iterator items*(blocks: PendingBlocksIter): Future[?(bt.Block, int)] =
|
|
||||||
while not blocks.finished:
|
|
||||||
yield blocks.next()
|
|
||||||
|
|
||||||
proc getPendingBlocks(
|
proc getPendingBlocks(
|
||||||
self: Erasure,
|
self: Erasure,
|
||||||
manifest: Manifest,
|
manifest: Manifest,
|
||||||
start, stop, steps: int): ?!PendingBlocksIter =
|
indicies: seq[int]): AsyncIter[(?!bt.Block, int)] =
|
||||||
## Get pending blocks iterator
|
## Get pending blocks iterator
|
||||||
##
|
##
|
||||||
|
|
||||||
var
|
var
|
||||||
# calculate block indexes to retrieve
|
# request blocks from the store
|
||||||
blockIdx = toSeq(countup(start, stop, steps))
|
pendingBlocks = indicies.map( (i: int) =>
|
||||||
# request all blocks from the store
|
self.store.getBlock(BlockAddress.init(manifest.treeCid, i)).map((r: ?!bt.Block) => (r, i)) # Get the data blocks (first K)
|
||||||
pendingBlocks = blockIdx.mapIt(
|
|
||||||
self.store.getBlock(manifest[it]) # Get the data blocks (first K)
|
|
||||||
)
|
)
|
||||||
indices = pendingBlocks # needed so we can track the block indices
|
|
||||||
iter = PendingBlocksIter(finished: false)
|
|
||||||
|
|
||||||
trace "Requesting blocks", pendingBlocks = pendingBlocks.len
|
proc isFinished(): bool = pendingBlocks.len == 0
|
||||||
proc next(): Future[?(bt.Block, int)] {.async.} =
|
|
||||||
if iter.finished:
|
|
||||||
trace "No more blocks"
|
|
||||||
return none (bt.Block, int)
|
|
||||||
|
|
||||||
if pendingBlocks.len == 0:
|
proc genNext(): Future[(?!bt.Block, int)] {.async.} =
|
||||||
iter.finished = true
|
let completedFut = await one(pendingBlocks)
|
||||||
trace "No more blocks - finished"
|
if (let i = pendingBlocks.find(completedFut); i >= 0):
|
||||||
return none (bt.Block, int)
|
pendingBlocks.del(i)
|
||||||
|
return await completedFut
|
||||||
let
|
else:
|
||||||
done = await one(pendingBlocks)
|
let (_, index) = await completedFut
|
||||||
idx = indices.find(done)
|
raise newException(CatchableError, "Future for block id not found, tree cid: " & $manifest.treeCid & ", index: " & $index)
|
||||||
|
|
||||||
logScope:
|
|
||||||
idx = idx
|
|
||||||
blockIdx = blockIdx[idx]
|
|
||||||
manifest = manifest[blockIdx[idx]]
|
|
||||||
|
|
||||||
pendingBlocks.del(pendingBlocks.find(done))
|
|
||||||
without blk =? (await done), error:
|
|
||||||
trace "Failed retrieving block", err = $error.msg
|
|
||||||
return none (bt.Block, int)
|
|
||||||
|
|
||||||
trace "Retrieved block"
|
|
||||||
some (blk, blockIdx[idx])
|
|
||||||
|
|
||||||
iter.next = next
|
|
||||||
success iter
|
|
||||||
|
|
||||||
|
Iter.new(genNext, isFinished)
|
||||||
|
|
||||||
proc prepareEncodingData(
|
proc prepareEncodingData(
|
||||||
self: Erasure,
|
self: Erasure,
|
||||||
encoded: Manifest,
|
manifest: Manifest,
|
||||||
|
params: EncodingParams,
|
||||||
step: int,
|
step: int,
|
||||||
data: ref seq[seq[byte]],
|
data: ref seq[seq[byte]],
|
||||||
|
cids: ref seq[Cid],
|
||||||
emptyBlock: seq[byte]): Future[?!int] {.async.} =
|
emptyBlock: seq[byte]): Future[?!int] {.async.} =
|
||||||
## Prepare data for encoding
|
## Prepare data for encoding
|
||||||
##
|
##
|
||||||
|
|
||||||
without pendingBlocksIter =?
|
let
|
||||||
self.getPendingBlocks(
|
indicies = toSeq(countup(step, params.rounded - 1, params.steps))
|
||||||
encoded,
|
pendingBlocksIter = self.getPendingBlocks(manifest, indicies.filterIt(it < manifest.blocksCount))
|
||||||
step,
|
|
||||||
encoded.rounded - 1, encoded.steps), err:
|
|
||||||
trace "Unable to get pending blocks", error = err.msg
|
|
||||||
return failure(err)
|
|
||||||
|
|
||||||
var resolved = 0
|
var resolved = 0
|
||||||
for blkFut in pendingBlocksIter:
|
for fut in pendingBlocksIter:
|
||||||
if (blk, idx) =? (await blkFut):
|
let (blkOrErr, idx) = await fut
|
||||||
let
|
without blk =? blkOrErr, err:
|
||||||
pos = self.indexToPos(encoded, idx, step)
|
warn "Failed retreiving a block", treeCid = manifest.treeCid, idx, msg = err.msg
|
||||||
|
continue
|
||||||
|
|
||||||
|
let pos = indexToPos(params.steps, idx, step)
|
||||||
|
shallowCopy(data[pos], if blk.isEmpty: emptyBlock else: blk.data)
|
||||||
|
cids[idx] = blk.cid
|
||||||
|
|
||||||
if blk.isEmpty:
|
resolved.inc()
|
||||||
trace "Padding with empty block", idx
|
|
||||||
shallowCopy(data[pos], emptyBlock)
|
|
||||||
else:
|
|
||||||
trace "Encoding block", cid = blk.cid, idx
|
|
||||||
shallowCopy(data[pos], blk.data)
|
|
||||||
|
|
||||||
resolved.inc()
|
for idx in indicies.filterIt(it >= manifest.blocksCount):
|
||||||
|
let pos = indexToPos(params.steps, idx, step)
|
||||||
|
trace "Padding with empty block", idx
|
||||||
|
shallowCopy(data[pos], emptyBlock)
|
||||||
|
without emptyBlockCid =? emptyCid(manifest.version, manifest.hcodec, manifest.codec), err:
|
||||||
|
return failure(err)
|
||||||
|
cids[idx] = emptyBlockCid
|
||||||
|
|
||||||
success resolved
|
success(resolved)
|
||||||
|
|
||||||
proc prepareDecodingData(
|
proc prepareDecodingData(
|
||||||
self: Erasure,
|
self: Erasure,
|
||||||
|
@ -171,129 +157,116 @@ proc prepareDecodingData(
|
||||||
step: int,
|
step: int,
|
||||||
data: ref seq[seq[byte]],
|
data: ref seq[seq[byte]],
|
||||||
parityData: ref seq[seq[byte]],
|
parityData: ref seq[seq[byte]],
|
||||||
|
cids: ref seq[Cid],
|
||||||
emptyBlock: seq[byte]): Future[?!(int, int)] {.async.} =
|
emptyBlock: seq[byte]): Future[?!(int, int)] {.async.} =
|
||||||
## Prepare data for decoding
|
## Prepare data for decoding
|
||||||
## `encoded` - the encoded manifest
|
## `encoded` - the encoded manifest
|
||||||
## `step` - the current step
|
## `step` - the current step
|
||||||
## `data` - the data to be prepared
|
## `data` - the data to be prepared
|
||||||
## `parityData` - the parityData to be prepared
|
## `parityData` - the parityData to be prepared
|
||||||
|
## `cids` - cids of prepared data
|
||||||
## `emptyBlock` - the empty block to be used for padding
|
## `emptyBlock` - the empty block to be used for padding
|
||||||
##
|
##
|
||||||
|
|
||||||
without pendingBlocksIter =?
|
let
|
||||||
self.getPendingBlocks(
|
indicies = toSeq(countup(step, encoded.blocksCount - 1, encoded.steps))
|
||||||
encoded,
|
pendingBlocksIter = self.getPendingBlocks(encoded, indicies)
|
||||||
step,
|
|
||||||
encoded.len - 1, encoded.steps), err:
|
|
||||||
trace "Unable to get pending blocks", error = err.msg
|
|
||||||
return failure(err)
|
|
||||||
|
|
||||||
var
|
var
|
||||||
dataPieces = 0
|
dataPieces = 0
|
||||||
parityPieces = 0
|
parityPieces = 0
|
||||||
resolved = 0
|
resolved = 0
|
||||||
for blkFut in pendingBlocksIter:
|
for fut in pendingBlocksIter:
|
||||||
# Continue to receive blocks until we have just enough for decoding
|
# Continue to receive blocks until we have just enough for decoding
|
||||||
# or no more blocks can arrive
|
# or no more blocks can arrive
|
||||||
if resolved >= encoded.ecK:
|
if resolved >= encoded.ecK:
|
||||||
break
|
break
|
||||||
|
|
||||||
if (blk, idx) =? (await blkFut):
|
let (blkOrErr, idx) = await fut
|
||||||
let
|
without blk =? blkOrErr, err:
|
||||||
pos = self.indexToPos(encoded, idx, step)
|
trace "Failed retreiving a block", idx, treeCid = encoded.treeCid, msg = err.msg
|
||||||
|
continue
|
||||||
|
|
||||||
logScope:
|
let
|
||||||
cid = blk.cid
|
pos = indexToPos(encoded.steps, idx, step)
|
||||||
idx = idx
|
|
||||||
pos = pos
|
|
||||||
step = step
|
|
||||||
empty = blk.isEmpty
|
|
||||||
|
|
||||||
if idx >= encoded.rounded:
|
logScope:
|
||||||
trace "Retrieved parity block"
|
cid = blk.cid
|
||||||
shallowCopy(parityData[pos - encoded.ecK], if blk.isEmpty: emptyBlock else: blk.data)
|
idx = idx
|
||||||
parityPieces.inc
|
pos = pos
|
||||||
else:
|
step = step
|
||||||
trace "Retrieved data block"
|
empty = blk.isEmpty
|
||||||
shallowCopy(data[pos], if blk.isEmpty: emptyBlock else: blk.data)
|
|
||||||
dataPieces.inc
|
|
||||||
|
|
||||||
resolved.inc
|
cids[idx] = blk.cid
|
||||||
|
if idx >= encoded.rounded:
|
||||||
|
trace "Retrieved parity block"
|
||||||
|
shallowCopy(parityData[pos - encoded.ecK], if blk.isEmpty: emptyBlock else: blk.data)
|
||||||
|
parityPieces.inc
|
||||||
|
else:
|
||||||
|
trace "Retrieved data block"
|
||||||
|
shallowCopy(data[pos], if blk.isEmpty: emptyBlock else: blk.data)
|
||||||
|
dataPieces.inc
|
||||||
|
|
||||||
|
resolved.inc
|
||||||
|
|
||||||
return success (dataPieces, parityPieces)
|
return success (dataPieces, parityPieces)
|
||||||
|
|
||||||
proc prepareManifest(
|
proc init(_: type EncodingParams, manifest: Manifest, ecK: int, ecM: int): ?!EncodingParams =
|
||||||
self: Erasure,
|
if ecK > manifest.blocksCount:
|
||||||
manifest: Manifest,
|
return failure("Unable to encode manifest, not enough blocks, ecK = " & $ecK & ", blocksCount = " & $manifest.blocksCount)
|
||||||
blocks: int,
|
|
||||||
parity: int): ?!Manifest =
|
|
||||||
|
|
||||||
logScope:
|
let
|
||||||
original_cid = manifest.cid.get()
|
rounded = roundUp(manifest.blocksCount, ecK)
|
||||||
original_len = manifest.len
|
steps = divUp(manifest.blocksCount, ecK)
|
||||||
blocks = blocks
|
blocksCount = rounded + (steps * ecM)
|
||||||
parity = parity
|
|
||||||
|
|
||||||
if blocks > manifest.len:
|
EncodingParams(
|
||||||
trace "Unable to encode manifest, not enough blocks", blocks = blocks, len = manifest.len
|
ecK: ecK,
|
||||||
return failure("Not enough blocks to encode")
|
ecM: ecM,
|
||||||
|
rounded: rounded,
|
||||||
trace "Preparing erasure coded manifest", blocks, parity
|
steps: steps,
|
||||||
without var encoded =? Manifest.new(manifest, blocks, parity), error:
|
blocksCount: blocksCount
|
||||||
trace "Unable to create manifest", msg = error.msg
|
).success
|
||||||
return error.failure
|
|
||||||
|
|
||||||
logScope:
|
|
||||||
steps = encoded.steps
|
|
||||||
rounded_blocks = encoded.rounded
|
|
||||||
new_manifest = encoded.len
|
|
||||||
|
|
||||||
trace "Erasure coded manifest prepared"
|
|
||||||
|
|
||||||
success encoded
|
|
||||||
|
|
||||||
proc encodeData(
|
proc encodeData(
|
||||||
self: Erasure,
|
self: Erasure,
|
||||||
manifest: Manifest): Future[?!void] {.async.} =
|
manifest: Manifest,
|
||||||
|
params: EncodingParams
|
||||||
|
): Future[?!Manifest] {.async.} =
|
||||||
## Encode blocks pointed to by the protected manifest
|
## Encode blocks pointed to by the protected manifest
|
||||||
##
|
##
|
||||||
## `manifest` - the manifest to encode
|
## `manifest` - the manifest to encode
|
||||||
##
|
##
|
||||||
|
|
||||||
var
|
|
||||||
encoded = manifest
|
|
||||||
|
|
||||||
logScope:
|
logScope:
|
||||||
steps = encoded.steps
|
steps = params.steps
|
||||||
rounded_blocks = encoded.rounded
|
rounded_blocks = params.rounded
|
||||||
new_manifest = encoded.len
|
blocks_count = params.blocksCount
|
||||||
protected = encoded.protected
|
ecK = params.ecK
|
||||||
ecK = encoded.ecK
|
ecM = params.ecM
|
||||||
ecM = encoded.ecM
|
|
||||||
|
|
||||||
if not encoded.protected:
|
|
||||||
trace "Manifest is not erasure protected"
|
|
||||||
return failure("Manifest is not erasure protected")
|
|
||||||
|
|
||||||
var
|
var
|
||||||
encoder = self.encoderProvider(encoded.blockSize.int, encoded.ecK, encoded.ecM)
|
cids = seq[Cid].new()
|
||||||
emptyBlock = newSeq[byte](encoded.blockSize.int)
|
encoder = self.encoderProvider(manifest.blockSize.int, params.ecK, params.ecM)
|
||||||
|
emptyBlock = newSeq[byte](manifest.blockSize.int)
|
||||||
|
|
||||||
|
cids[].setLen(params.blocksCount)
|
||||||
|
|
||||||
try:
|
try:
|
||||||
for step in 0..<encoded.steps:
|
for step in 0..<params.steps:
|
||||||
# TODO: Don't allocate a new seq every time, allocate once and zero out
|
# TODO: Don't allocate a new seq every time, allocate once and zero out
|
||||||
var
|
var
|
||||||
data = seq[seq[byte]].new() # number of blocks to encode
|
data = seq[seq[byte]].new() # number of blocks to encode
|
||||||
parityData = newSeqWith[seq[byte]](encoded.ecM, newSeq[byte](encoded.blockSize.int))
|
parityData = newSeqWith[seq[byte]](params.ecM, newSeq[byte](manifest.blockSize.int))
|
||||||
|
|
||||||
data[].setLen(encoded.ecK)
|
data[].setLen(params.ecK)
|
||||||
# TODO: this is a tight blocking loop so we sleep here to allow
|
# TODO: this is a tight blocking loop so we sleep here to allow
|
||||||
# other events to be processed, this should be addressed
|
# other events to be processed, this should be addressed
|
||||||
# by threading
|
# by threading
|
||||||
await sleepAsync(10.millis)
|
await sleepAsync(10.millis)
|
||||||
|
|
||||||
without resolved =?
|
without resolved =?
|
||||||
(await self.prepareEncodingData(encoded, step, data, emptyBlock)), err:
|
(await self.prepareEncodingData(manifest, params, step, data, cids, emptyBlock)), err:
|
||||||
trace "Unable to prepare data", error = err.msg
|
trace "Unable to prepare data", error = err.msg
|
||||||
return failure(err)
|
return failure(err)
|
||||||
|
|
||||||
|
@ -303,20 +276,39 @@ proc encodeData(
|
||||||
let res = encoder.encode(data[], parityData);
|
let res = encoder.encode(data[], parityData);
|
||||||
res.isErr):
|
res.isErr):
|
||||||
trace "Unable to encode manifest!", error = $res.error
|
trace "Unable to encode manifest!", error = $res.error
|
||||||
return res.mapFailure
|
return failure($res.error)
|
||||||
|
|
||||||
var idx = encoded.rounded + step
|
var idx = params.rounded + step
|
||||||
for j in 0..<encoded.ecM:
|
for j in 0..<params.ecM:
|
||||||
without blk =? bt.Block.new(parityData[j]), error:
|
without blk =? bt.Block.new(parityData[j]), error:
|
||||||
trace "Unable to create parity block", err = error.msg
|
trace "Unable to create parity block", err = error.msg
|
||||||
return failure(error)
|
return failure(error)
|
||||||
|
|
||||||
trace "Adding parity block", cid = blk.cid, idx
|
trace "Adding parity block", cid = blk.cid, idx
|
||||||
encoded[idx] = blk.cid
|
cids[idx] = blk.cid
|
||||||
if isErr (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!")
|
||||||
idx.inc(encoded.steps)
|
idx.inc(params.steps)
|
||||||
|
|
||||||
|
without tree =? MerkleTree.init(cids[]), err:
|
||||||
|
return failure(err)
|
||||||
|
|
||||||
|
without treeCid =? tree.rootCid, err:
|
||||||
|
return failure(err)
|
||||||
|
|
||||||
|
if err =? (await self.store.putAllProofs(tree)).errorOption:
|
||||||
|
return failure(err)
|
||||||
|
|
||||||
|
let encodedManifest = Manifest.new(
|
||||||
|
manifest = manifest,
|
||||||
|
treeCid = treeCid,
|
||||||
|
datasetSize = (manifest.blockSize.int * params.blocksCount).NBytes,
|
||||||
|
ecK = params.ecK,
|
||||||
|
ecM = params.ecM
|
||||||
|
)
|
||||||
|
|
||||||
|
return encodedManifest.success
|
||||||
except CancelledError as exc:
|
except CancelledError as exc:
|
||||||
trace "Erasure coding encoding cancelled"
|
trace "Erasure coding encoding cancelled"
|
||||||
raise exc # cancellation needs to be propagated
|
raise exc # cancellation needs to be propagated
|
||||||
|
@ -326,8 +318,6 @@ proc encodeData(
|
||||||
finally:
|
finally:
|
||||||
encoder.release()
|
encoder.release()
|
||||||
|
|
||||||
return success()
|
|
||||||
|
|
||||||
proc encode*(
|
proc encode*(
|
||||||
self: Erasure,
|
self: Erasure,
|
||||||
manifest: Manifest,
|
manifest: Manifest,
|
||||||
|
@ -340,47 +330,37 @@ proc encode*(
|
||||||
## `parity` - the number of parity blocks to generate - M
|
## `parity` - the number of parity blocks to generate - M
|
||||||
##
|
##
|
||||||
|
|
||||||
without var encoded =? self.prepareManifest(manifest, blocks, parity), error:
|
without params =? EncodingParams.init(manifest, blocks, parity), err:
|
||||||
trace "Unable to prepare manifest", error = error.msg
|
return failure(err)
|
||||||
return failure error
|
|
||||||
|
|
||||||
if err =? (await self.encodeData(encoded)).errorOption:
|
without encodedManifest =? await self.encodeData(manifest, params), err:
|
||||||
trace "Unable to encode data", error = err.msg
|
return failure(err)
|
||||||
return failure err
|
|
||||||
|
|
||||||
return success encoded
|
return success encodedManifest
|
||||||
|
|
||||||
proc decode*(
|
proc decode*(
|
||||||
self: Erasure,
|
self: Erasure,
|
||||||
encoded: Manifest,
|
encoded: Manifest
|
||||||
all = true): Future[?!Manifest] {.async.} =
|
): Future[?!Manifest] {.async.} =
|
||||||
## Decode a protected manifest into it's original
|
## Decode a protected manifest into it's original
|
||||||
## manifest
|
## manifest
|
||||||
##
|
##
|
||||||
## `encoded` - the encoded (protected) manifest to
|
## `encoded` - the encoded (protected) manifest to
|
||||||
## be recovered
|
## be recovered
|
||||||
## `all` - if true, all blocks will be recovered,
|
|
||||||
## including parity
|
|
||||||
##
|
##
|
||||||
|
|
||||||
logScope:
|
logScope:
|
||||||
steps = encoded.steps
|
steps = encoded.steps
|
||||||
rounded_blocks = encoded.rounded
|
rounded_blocks = encoded.rounded
|
||||||
new_manifest = encoded.len
|
new_manifest = encoded.blocksCount
|
||||||
protected = encoded.protected
|
|
||||||
ecK = encoded.ecK
|
|
||||||
ecM = encoded.ecM
|
|
||||||
|
|
||||||
if not encoded.protected:
|
|
||||||
trace "Manifest is not erasure protected"
|
|
||||||
return failure "Manifest is not erasure protected"
|
|
||||||
|
|
||||||
var
|
var
|
||||||
|
cids = seq[Cid].new()
|
||||||
|
recoveredIndices = newSeq[int]()
|
||||||
decoder = self.decoderProvider(encoded.blockSize.int, encoded.ecK, encoded.ecM)
|
decoder = self.decoderProvider(encoded.blockSize.int, encoded.ecK, encoded.ecM)
|
||||||
emptyBlock = newSeq[byte](encoded.blockSize.int)
|
emptyBlock = newSeq[byte](encoded.blockSize.int)
|
||||||
hasParity = false
|
|
||||||
|
|
||||||
trace "Decoding erasure coded manifest"
|
cids[].setLen(encoded.blocksCount)
|
||||||
try:
|
try:
|
||||||
for step in 0..<encoded.steps:
|
for step in 0..<encoded.steps:
|
||||||
# TODO: this is a tight blocking loop so we sleep here to allow
|
# TODO: this is a tight blocking loop so we sleep here to allow
|
||||||
|
@ -390,16 +370,14 @@ proc decode*(
|
||||||
|
|
||||||
var
|
var
|
||||||
data = seq[seq[byte]].new()
|
data = seq[seq[byte]].new()
|
||||||
# newSeq[seq[byte]](encoded.ecK) # number of blocks to encode
|
|
||||||
parityData = seq[seq[byte]].new()
|
parityData = seq[seq[byte]].new()
|
||||||
recovered = newSeqWith[seq[byte]](encoded.ecK, newSeq[byte](encoded.blockSize.int))
|
recovered = newSeqWith[seq[byte]](encoded.ecK, newSeq[byte](encoded.blockSize.int))
|
||||||
resolved = 0
|
|
||||||
|
|
||||||
data[].setLen(encoded.ecK) # set len to K
|
data[].setLen(encoded.ecK) # set len to K
|
||||||
parityData[].setLen(encoded.ecM) # set len to M
|
parityData[].setLen(encoded.ecM) # set len to M
|
||||||
|
|
||||||
without (dataPieces, parityPieces) =?
|
without (dataPieces, parityPieces) =?
|
||||||
(await self.prepareDecodingData(encoded, step, data, parityData, emptyBlock)), err:
|
(await self.prepareDecodingData(encoded, step, data, parityData, cids, emptyBlock)), err:
|
||||||
trace "Unable to prepare data", error = err.msg
|
trace "Unable to prepare data", error = err.msg
|
||||||
return failure(err)
|
return failure(err)
|
||||||
|
|
||||||
|
@ -415,18 +393,19 @@ proc decode*(
|
||||||
return failure($err.error)
|
return failure($err.error)
|
||||||
|
|
||||||
for i in 0..<encoded.ecK:
|
for i in 0..<encoded.ecK:
|
||||||
if data[i].len <= 0 and not encoded.blocks[i].isEmpty:
|
let idx = i * encoded.steps + step
|
||||||
|
if data[i].len <= 0 and not cids[idx].isEmpty:
|
||||||
without blk =? bt.Block.new(recovered[i]), error:
|
without blk =? bt.Block.new(recovered[i]), error:
|
||||||
trace "Unable to create block!", exc = error.msg
|
trace "Unable to create block!", exc = error.msg
|
||||||
return failure(error)
|
return failure(error)
|
||||||
|
|
||||||
doAssert blk.cid in encoded.blocks,
|
|
||||||
"Recovered block not in original manifest"
|
|
||||||
|
|
||||||
trace "Recovered block", cid = blk.cid, index = i
|
trace "Recovered block", cid = blk.cid, index = i
|
||||||
if isErr (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!")
|
||||||
|
|
||||||
|
cids[idx] = blk.cid
|
||||||
|
recoveredIndices.add(idx)
|
||||||
except CancelledError as exc:
|
except CancelledError as exc:
|
||||||
trace "Erasure coding decoding cancelled"
|
trace "Erasure coding decoding cancelled"
|
||||||
raise exc # cancellation needs to be propagated
|
raise exc # cancellation needs to be propagated
|
||||||
|
@ -436,8 +415,23 @@ proc decode*(
|
||||||
finally:
|
finally:
|
||||||
decoder.release()
|
decoder.release()
|
||||||
|
|
||||||
without decoded =? Manifest.new(blocks = encoded.blocks[0..<encoded.originalLen]), error:
|
without tree =? MerkleTree.init(cids[0..<encoded.originalBlocksCount]), err:
|
||||||
return error.failure
|
return failure(err)
|
||||||
|
|
||||||
|
without treeCid =? tree.rootCid, err:
|
||||||
|
return failure(err)
|
||||||
|
|
||||||
|
if treeCid != encoded.originalTreeCid:
|
||||||
|
return failure("Original tree root differs from the tree root computed out of recovered data")
|
||||||
|
|
||||||
|
let idxIter = Iter
|
||||||
|
.fromItems(recoveredIndices)
|
||||||
|
.filter((i: int) => i < tree.leavesCount)
|
||||||
|
|
||||||
|
if err =? (await self.store.putSomeProofs(tree, idxIter)).errorOption:
|
||||||
|
return failure(err)
|
||||||
|
|
||||||
|
let decoded = Manifest.new(encoded)
|
||||||
|
|
||||||
return decoded.success
|
return decoded.success
|
||||||
|
|
||||||
|
|
|
@ -34,54 +34,45 @@ proc encode*(_: DagPBCoder, manifest: Manifest): ?!seq[byte] =
|
||||||
? manifest.verify()
|
? manifest.verify()
|
||||||
var pbNode = initProtoBuffer()
|
var pbNode = initProtoBuffer()
|
||||||
|
|
||||||
for c in manifest.blocks:
|
|
||||||
var pbLink = initProtoBuffer()
|
|
||||||
pbLink.write(1, c.data.buffer) # write Cid links
|
|
||||||
pbLink.finish()
|
|
||||||
pbNode.write(2, pbLink)
|
|
||||||
|
|
||||||
# NOTE: The `Data` field in the the `dag-pb`
|
# NOTE: The `Data` field in the the `dag-pb`
|
||||||
# contains the following protobuf `Message`
|
# contains the following protobuf `Message`
|
||||||
#
|
#
|
||||||
# ```protobuf
|
# ```protobuf
|
||||||
# Message ErasureInfo {
|
# Message ErasureInfo {
|
||||||
# optional uint32 K = 1; # number of encoded blocks
|
# optional uint32 ecK = 1; # number of encoded blocks
|
||||||
# optional uint32 M = 2; # number of parity blocks
|
# optional uint32 ecM = 2; # number of parity blocks
|
||||||
# optional bytes cid = 3; # cid of the original dataset
|
# optional bytes originalTreeCid = 3; # cid of the original dataset
|
||||||
# optional uint32 original = 4; # number of original blocks
|
# optional uint32 originalDatasetSize = 4; # size of the original dataset
|
||||||
# }
|
# }
|
||||||
# Message Header {
|
# Message Header {
|
||||||
# optional bytes rootHash = 1; # the root (tree) hash
|
# optional bytes treeCid = 1; # cid (root) of the tree
|
||||||
# optional uint32 blockSize = 2; # size of a single block
|
# optional uint32 blockSize = 2; # size of a single block
|
||||||
# optional uint32 blocksLen = 3; # total amount of blocks
|
# optional uint64 datasetSize = 3; # size of the dataset
|
||||||
# optional ErasureInfo erasure = 4; # erasure coding info
|
# optional ErasureInfo erasure = 4; # erasure coding info
|
||||||
# optional uint64 originalBytes = 5;# exact file size
|
|
||||||
# }
|
# }
|
||||||
# ```
|
# ```
|
||||||
#
|
#
|
||||||
|
# var treeRootVBuf = initVBuffer()
|
||||||
let cid = ? manifest.cid
|
|
||||||
var header = initProtoBuffer()
|
var header = initProtoBuffer()
|
||||||
header.write(1, cid.data.buffer)
|
header.write(1, manifest.treeCid.data.buffer)
|
||||||
header.write(2, manifest.blockSize.uint32)
|
header.write(2, manifest.blockSize.uint32)
|
||||||
header.write(3, manifest.len.uint32)
|
header.write(3, manifest.datasetSize.uint32)
|
||||||
header.write(5, manifest.originalBytes.uint64)
|
|
||||||
if manifest.protected:
|
if manifest.protected:
|
||||||
var erasureInfo = initProtoBuffer()
|
var erasureInfo = initProtoBuffer()
|
||||||
erasureInfo.write(1, manifest.ecK.uint32)
|
erasureInfo.write(1, manifest.ecK.uint32)
|
||||||
erasureInfo.write(2, manifest.ecM.uint32)
|
erasureInfo.write(2, manifest.ecM.uint32)
|
||||||
erasureInfo.write(3, manifest.originalCid.data.buffer)
|
erasureInfo.write(3, manifest.originalTreeCid.data.buffer)
|
||||||
erasureInfo.write(4, manifest.originalLen.uint32)
|
erasureInfo.write(4, manifest.originalDatasetSize.uint32)
|
||||||
erasureInfo.finish()
|
erasureInfo.finish()
|
||||||
|
|
||||||
header.write(4, erasureInfo)
|
header.write(4, erasureInfo)
|
||||||
|
|
||||||
pbNode.write(1, header) # set the rootHash Cid as the data field
|
pbNode.write(1, header) # set the treeCid as the data field
|
||||||
pbNode.finish()
|
pbNode.finish()
|
||||||
|
|
||||||
return pbNode.buffer.success
|
return pbNode.buffer.success
|
||||||
|
|
||||||
func decode*(_: DagPBCoder, data: openArray[byte]): ?!Manifest =
|
proc decode*(_: DagPBCoder, data: openArray[byte]): ?!Manifest =
|
||||||
## Decode a manifest from a data blob
|
## Decode a manifest from a data blob
|
||||||
##
|
##
|
||||||
|
|
||||||
|
@ -89,86 +80,70 @@ func decode*(_: DagPBCoder, data: openArray[byte]): ?!Manifest =
|
||||||
pbNode = initProtoBuffer(data)
|
pbNode = initProtoBuffer(data)
|
||||||
pbHeader: ProtoBuffer
|
pbHeader: ProtoBuffer
|
||||||
pbErasureInfo: ProtoBuffer
|
pbErasureInfo: ProtoBuffer
|
||||||
rootHash: seq[byte]
|
treeCidBuf: seq[byte]
|
||||||
originalCid: seq[byte]
|
originalTreeCid: seq[byte]
|
||||||
originalBytes: uint64
|
datasetSize: uint32
|
||||||
blockSize: uint32
|
blockSize: uint32
|
||||||
blocksLen: uint32
|
originalDatasetSize: uint32
|
||||||
originalLen: uint32
|
|
||||||
ecK, ecM: uint32
|
ecK, ecM: uint32
|
||||||
blocks: seq[Cid]
|
|
||||||
|
|
||||||
# Decode `Header` message
|
# Decode `Header` message
|
||||||
if pbNode.getField(1, pbHeader).isErr:
|
if pbNode.getField(1, pbHeader).isErr:
|
||||||
return failure("Unable to decode `Header` from dag-pb manifest!")
|
return failure("Unable to decode `Header` from dag-pb manifest!")
|
||||||
|
|
||||||
# Decode `Header` contents
|
# Decode `Header` contents
|
||||||
if pbHeader.getField(1, rootHash).isErr:
|
if pbHeader.getField(1, treeCidBuf).isErr:
|
||||||
return failure("Unable to decode `rootHash` from manifest!")
|
return failure("Unable to decode `treeCid` from manifest!")
|
||||||
|
|
||||||
if pbHeader.getField(2, blockSize).isErr:
|
if pbHeader.getField(2, blockSize).isErr:
|
||||||
return failure("Unable to decode `blockSize` from manifest!")
|
return failure("Unable to decode `blockSize` from manifest!")
|
||||||
|
|
||||||
if pbHeader.getField(3, blocksLen).isErr:
|
if pbHeader.getField(3, datasetSize).isErr:
|
||||||
return failure("Unable to decode `blocksLen` from manifest!")
|
return failure("Unable to decode `datasetSize` from manifest!")
|
||||||
|
|
||||||
if pbHeader.getField(5, originalBytes).isErr:
|
|
||||||
return failure("Unable to decode `originalBytes` from manifest!")
|
|
||||||
|
|
||||||
if pbHeader.getField(4, pbErasureInfo).isErr:
|
if pbHeader.getField(4, pbErasureInfo).isErr:
|
||||||
return failure("Unable to decode `erasureInfo` from manifest!")
|
return failure("Unable to decode `erasureInfo` from manifest!")
|
||||||
|
|
||||||
if pbErasureInfo.buffer.len > 0:
|
let protected = pbErasureInfo.buffer.len > 0
|
||||||
|
if protected:
|
||||||
if pbErasureInfo.getField(1, ecK).isErr:
|
if pbErasureInfo.getField(1, ecK).isErr:
|
||||||
return failure("Unable to decode `K` from manifest!")
|
return failure("Unable to decode `K` from manifest!")
|
||||||
|
|
||||||
if pbErasureInfo.getField(2, ecM).isErr:
|
if pbErasureInfo.getField(2, ecM).isErr:
|
||||||
return failure("Unable to decode `M` from manifest!")
|
return failure("Unable to decode `M` from manifest!")
|
||||||
|
|
||||||
if pbErasureInfo.getField(3, originalCid).isErr:
|
if pbErasureInfo.getField(3, originalTreeCid).isErr:
|
||||||
return failure("Unable to decode `originalCid` from manifest!")
|
return failure("Unable to decode `originalTreeCid` from manifest!")
|
||||||
|
|
||||||
if pbErasureInfo.getField(4, originalLen).isErr:
|
if pbErasureInfo.getField(4, originalDatasetSize).isErr:
|
||||||
return failure("Unable to decode `originalLen` from manifest!")
|
return failure("Unable to decode `originalDatasetSize` from manifest!")
|
||||||
|
|
||||||
let rootHashCid = ? Cid.init(rootHash).mapFailure
|
|
||||||
var linksBuf: seq[seq[byte]]
|
|
||||||
if pbNode.getRepeatedField(2, linksBuf).isOk:
|
|
||||||
for pbLinkBuf in linksBuf:
|
|
||||||
var
|
|
||||||
blockBuf: seq[byte]
|
|
||||||
pbLink = initProtoBuffer(pbLinkBuf)
|
|
||||||
|
|
||||||
if pbLink.getField(1, blockBuf).isOk:
|
let
|
||||||
blocks.add(? Cid.init(blockBuf).mapFailure)
|
treeCid = ? Cid.init(treeCidBuf).mapFailure
|
||||||
|
|
||||||
if blocksLen.int != blocks.len:
|
|
||||||
return failure("Total blocks and length of blocks in header don't match!")
|
|
||||||
|
|
||||||
let
|
let
|
||||||
self = if pbErasureInfo.buffer.len > 0:
|
self = if protected:
|
||||||
Manifest.new(
|
Manifest.new(
|
||||||
rootHash = rootHashCid,
|
treeCid = treeCid,
|
||||||
originalBytes = originalBytes.NBytes,
|
datasetSize = datasetSize.NBytes,
|
||||||
blockSize = blockSize.NBytes,
|
blockSize = blockSize.NBytes,
|
||||||
blocks = blocks,
|
version = treeCid.cidver,
|
||||||
version = rootHashCid.cidver,
|
hcodec = (? treeCid.mhash.mapFailure).mcodec,
|
||||||
hcodec = (? rootHashCid.mhash.mapFailure).mcodec,
|
codec = treeCid.mcodec,
|
||||||
codec = rootHashCid.mcodec,
|
|
||||||
ecK = ecK.int,
|
ecK = ecK.int,
|
||||||
ecM = ecM.int,
|
ecM = ecM.int,
|
||||||
originalCid = ? Cid.init(originalCid).mapFailure,
|
originalTreeCid = ? Cid.init(originalTreeCid).mapFailure,
|
||||||
originalLen = originalLen.int
|
originalDatasetSize = originalDatasetSize.NBytes
|
||||||
)
|
)
|
||||||
else:
|
else:
|
||||||
Manifest.new(
|
Manifest.new(
|
||||||
rootHash = rootHashCid,
|
treeCid = treeCid,
|
||||||
originalBytes = originalBytes.NBytes,
|
datasetSize = datasetSize.NBytes,
|
||||||
blockSize = blockSize.NBytes,
|
blockSize = blockSize.NBytes,
|
||||||
blocks = blocks,
|
version = treeCid.cidver,
|
||||||
version = rootHashCid.cidver,
|
hcodec = (? treeCid.mhash.mapFailure).mcodec,
|
||||||
hcodec = (? rootHashCid.mhash.mapFailure).mcodec,
|
codec = treeCid.mcodec
|
||||||
codec = rootHashCid.mcodec
|
|
||||||
)
|
)
|
||||||
|
|
||||||
? self.verify()
|
? self.verify()
|
||||||
|
|
|
@ -30,19 +30,18 @@ export types
|
||||||
|
|
||||||
type
|
type
|
||||||
Manifest* = ref object of RootObj
|
Manifest* = ref object of RootObj
|
||||||
rootHash {.serialize.}: ?Cid # Root (tree) hash of the contained data set
|
treeCid {.serialize.}: Cid # Root of the merkle tree
|
||||||
originalBytes* {.serialize.}: NBytes # Exact size of the original (uploaded) file
|
datasetSize {.serialize.}: NBytes # Total size of all blocks
|
||||||
blockSize {.serialize.}: NBytes # Size of each contained block (might not be needed if blocks are len-prefixed)
|
blockSize {.serialize.}: NBytes # Size of each contained block (might not be needed if blocks are len-prefixed)
|
||||||
blocks: seq[Cid] # Block Cid
|
|
||||||
version: CidVersion # Cid version
|
version: CidVersion # Cid version
|
||||||
hcodec: MultiCodec # Multihash codec
|
hcodec: MultiCodec # Multihash codec
|
||||||
codec: MultiCodec # Data set codec
|
codec: MultiCodec # Data set codec
|
||||||
case protected {.serialize.}: bool # Protected datasets have erasure coded info
|
case protected {.serialize.}: bool # Protected datasets have erasure coded info
|
||||||
of true:
|
of true:
|
||||||
ecK: int # Number of blocks to encode
|
ecK: int # Number of blocks to encode
|
||||||
ecM: int # Number of resulting parity blocks
|
ecM: int # Number of resulting parity blocks
|
||||||
originalCid: Cid # The original Cid of the dataset being erasure coded
|
originalTreeCid: Cid # The original root of the dataset being erasure coded
|
||||||
originalLen: int # The length of the original manifest
|
originalDatasetSize: NBytes
|
||||||
else:
|
else:
|
||||||
discard
|
discard
|
||||||
|
|
||||||
|
@ -53,8 +52,8 @@ type
|
||||||
proc blockSize*(self: Manifest): NBytes =
|
proc blockSize*(self: Manifest): NBytes =
|
||||||
self.blockSize
|
self.blockSize
|
||||||
|
|
||||||
proc blocks*(self: Manifest): seq[Cid] =
|
proc datasetSize*(self: Manifest): NBytes =
|
||||||
self.blocks
|
self.datasetSize
|
||||||
|
|
||||||
proc version*(self: Manifest): CidVersion =
|
proc version*(self: Manifest): CidVersion =
|
||||||
self.version
|
self.version
|
||||||
|
@ -74,33 +73,25 @@ proc ecK*(self: Manifest): int =
|
||||||
proc ecM*(self: Manifest): int =
|
proc ecM*(self: Manifest): int =
|
||||||
self.ecM
|
self.ecM
|
||||||
|
|
||||||
proc originalCid*(self: Manifest): Cid =
|
proc originalTreeCid*(self: Manifest): Cid =
|
||||||
self.originalCid
|
self.originalTreeCid
|
||||||
|
|
||||||
proc originalLen*(self: Manifest): int =
|
proc originalBlocksCount*(self: Manifest): int =
|
||||||
self.originalLen
|
divUp(self.originalDatasetSize.int, self.blockSize.int)
|
||||||
|
|
||||||
|
proc originalDatasetSize*(self: Manifest): NBytes =
|
||||||
|
self.originalDatasetSize
|
||||||
|
|
||||||
|
proc treeCid*(self: Manifest): Cid =
|
||||||
|
self.treeCid
|
||||||
|
|
||||||
|
proc blocksCount*(self: Manifest): int =
|
||||||
|
divUp(self.datasetSize.int, self.blockSize.int)
|
||||||
|
|
||||||
############################################################
|
############################################################
|
||||||
# Operations on block list
|
# Operations on block list
|
||||||
############################################################
|
############################################################
|
||||||
|
|
||||||
func len*(self: Manifest): int =
|
|
||||||
self.blocks.len
|
|
||||||
|
|
||||||
func `[]`*(self: Manifest, i: Natural): Cid =
|
|
||||||
self.blocks[i]
|
|
||||||
|
|
||||||
func `[]=`*(self: var Manifest, i: Natural, item: Cid) =
|
|
||||||
self.rootHash = Cid.none
|
|
||||||
self.blocks[i] = item
|
|
||||||
|
|
||||||
func `[]`*(self: Manifest, i: BackwardsIndex): Cid =
|
|
||||||
self.blocks[self.len - i.int]
|
|
||||||
|
|
||||||
func `[]=`*(self: Manifest, i: BackwardsIndex, item: Cid) =
|
|
||||||
self.rootHash = Cid.none
|
|
||||||
self.blocks[self.len - i.int] = item
|
|
||||||
|
|
||||||
func isManifest*(cid: Cid): ?!bool =
|
func isManifest*(cid: Cid): ?!bool =
|
||||||
let res = ?cid.contentType().mapFailure(CodexError)
|
let res = ?cid.contentType().mapFailure(CodexError)
|
||||||
($(res) in ManifestContainers).success
|
($(res) in ManifestContainers).success
|
||||||
|
@ -108,25 +99,6 @@ func isManifest*(cid: Cid): ?!bool =
|
||||||
func isManifest*(mc: MultiCodec): ?!bool =
|
func isManifest*(mc: MultiCodec): ?!bool =
|
||||||
($mc in ManifestContainers).success
|
($mc in ManifestContainers).success
|
||||||
|
|
||||||
proc add*(self: Manifest, cid: Cid) =
|
|
||||||
assert not self.protected # we expect that protected manifests are created with properly-sized self.blocks
|
|
||||||
self.rootHash = Cid.none
|
|
||||||
trace "Adding cid to manifest", cid
|
|
||||||
self.blocks.add(cid)
|
|
||||||
self.originalBytes = self.blocks.len.NBytes * self.blockSize
|
|
||||||
|
|
||||||
iterator items*(self: Manifest): Cid =
|
|
||||||
for b in self.blocks:
|
|
||||||
yield b
|
|
||||||
|
|
||||||
iterator pairs*(self: Manifest): tuple[key: int, val: Cid] =
|
|
||||||
for pair in self.blocks.pairs():
|
|
||||||
yield pair
|
|
||||||
|
|
||||||
func contains*(self: Manifest, cid: Cid): bool =
|
|
||||||
cid in self.blocks
|
|
||||||
|
|
||||||
|
|
||||||
############################################################
|
############################################################
|
||||||
# Various sizes and verification
|
# Various sizes and verification
|
||||||
############################################################
|
############################################################
|
||||||
|
@ -134,79 +106,61 @@ func contains*(self: Manifest, cid: Cid): bool =
|
||||||
func bytes*(self: Manifest, pad = true): NBytes =
|
func bytes*(self: Manifest, pad = true): NBytes =
|
||||||
## Compute how many bytes corresponding StoreStream(Manifest, pad) will return
|
## Compute how many bytes corresponding StoreStream(Manifest, pad) will return
|
||||||
if pad or self.protected:
|
if pad or self.protected:
|
||||||
self.len.NBytes * self.blockSize
|
self.blocksCount.NBytes * self.blockSize
|
||||||
else:
|
else:
|
||||||
self.originalBytes
|
self.datasetSize
|
||||||
|
|
||||||
func rounded*(self: Manifest): int =
|
func rounded*(self: Manifest): int =
|
||||||
## Number of data blocks in *protected* manifest including padding at the end
|
## Number of data blocks in *protected* manifest including padding at the end
|
||||||
roundUp(self.originalLen, self.ecK)
|
roundUp(self.originalBlocksCount, self.ecK)
|
||||||
|
|
||||||
func steps*(self: Manifest): int =
|
func steps*(self: Manifest): int =
|
||||||
## Number of EC groups in *protected* manifest
|
## Number of EC groups in *protected* manifest
|
||||||
divUp(self.originalLen, self.ecK)
|
divUp(self.originalBlocksCount, self.ecK)
|
||||||
|
|
||||||
func verify*(self: Manifest): ?!void =
|
func verify*(self: Manifest): ?!void =
|
||||||
## Check manifest correctness
|
## Check manifest correctness
|
||||||
##
|
##
|
||||||
let originalLen = (if self.protected: self.originalLen else: self.len)
|
|
||||||
|
|
||||||
if divUp(self.originalBytes, self.blockSize) != originalLen:
|
if self.protected and (self.blocksCount != self.steps * (self.ecK + self.ecM)):
|
||||||
return failure newException(CodexError, "Broken manifest: wrong originalBytes")
|
return failure newException(CodexError, "Broken manifest: wrong originalBlocksCount")
|
||||||
|
|
||||||
if self.protected and (self.len != self.steps * (self.ecK + self.ecM)):
|
|
||||||
return failure newException(CodexError, "Broken manifest: wrong originalLen")
|
|
||||||
|
|
||||||
return success()
|
return success()
|
||||||
|
|
||||||
|
proc cid*(self: Manifest): ?!Cid {.deprecated: "use treeCid instead".} =
|
||||||
|
self.treeCid.success
|
||||||
|
|
||||||
############################################################
|
proc `==`*(a, b: Manifest): bool =
|
||||||
# Cid computation
|
(a.treeCid == b.treeCid) and
|
||||||
############################################################
|
(a.datasetSize == b.datasetSize) and
|
||||||
|
(a.blockSize == b.blockSize) and
|
||||||
template hashBytes(mh: MultiHash): seq[byte] =
|
(a.version == b.version) and
|
||||||
## get the hash bytes of a multihash object
|
(a.hcodec == b.hcodec) and
|
||||||
##
|
(a.codec == b.codec) and
|
||||||
|
(a.protected == b.protected) and
|
||||||
mh.data.buffer[mh.dpos..(mh.dpos + mh.size - 1)]
|
(if a.protected:
|
||||||
|
(a.ecK == b.ecK) and
|
||||||
proc makeRoot*(self: Manifest): ?!void =
|
(a.ecM == b.ecM) and
|
||||||
## Create a tree hash root of the contained
|
(a.originalTreeCid == b.originalTreeCid) and
|
||||||
## block hashes
|
(a.originalDatasetSize == b.originalDatasetSize)
|
||||||
##
|
else:
|
||||||
|
true)
|
||||||
var
|
|
||||||
stack: seq[MultiHash]
|
|
||||||
|
|
||||||
for cid in self:
|
|
||||||
stack.add(? cid.mhash.mapFailure)
|
|
||||||
|
|
||||||
while stack.len > 1:
|
|
||||||
let
|
|
||||||
(b1, b2) = (stack.pop(), stack.pop())
|
|
||||||
mh = ? MultiHash.digest(
|
|
||||||
$self.hcodec,
|
|
||||||
(b1.hashBytes() & b2.hashBytes()))
|
|
||||||
.mapFailure
|
|
||||||
stack.add(mh)
|
|
||||||
|
|
||||||
if stack.len == 1:
|
|
||||||
let digest = ? EmptyDigests[self.version][self.hcodec].catch
|
|
||||||
let cid = ? Cid.init(self.version, self.codec, digest).mapFailure
|
|
||||||
|
|
||||||
self.rootHash = cid.some
|
|
||||||
|
|
||||||
success()
|
|
||||||
|
|
||||||
proc cid*(self: Manifest): ?!Cid =
|
|
||||||
## Generate a root hash using the treehash algorithm
|
|
||||||
##
|
|
||||||
|
|
||||||
if self.rootHash.isNone:
|
|
||||||
? self.makeRoot()
|
|
||||||
|
|
||||||
(!self.rootHash).success
|
|
||||||
|
|
||||||
|
proc `$`*(self: Manifest): string =
|
||||||
|
"treeCid: " & $self.treeCid &
|
||||||
|
", datasetSize: " & $self.datasetSize &
|
||||||
|
", blockSize: " & $self.blockSize &
|
||||||
|
", version: " & $self.version &
|
||||||
|
", hcodec: " & $self.hcodec &
|
||||||
|
", codec: " & $self.codec &
|
||||||
|
", protected: " & $self.protected &
|
||||||
|
(if self.protected:
|
||||||
|
", ecK: " & $self.ecK &
|
||||||
|
", ecM: " & $self.ecM &
|
||||||
|
", originalTreeCid: " & $self.originalTreeCid &
|
||||||
|
", originalDatasetSize: " & $self.originalDatasetSize
|
||||||
|
else:
|
||||||
|
"")
|
||||||
|
|
||||||
############################################################
|
############################################################
|
||||||
# Constructors
|
# Constructors
|
||||||
|
@ -214,67 +168,61 @@ proc cid*(self: Manifest): ?!Cid =
|
||||||
|
|
||||||
proc new*(
|
proc new*(
|
||||||
T: type Manifest,
|
T: type Manifest,
|
||||||
blocks: openArray[Cid] = [],
|
treeCid: Cid,
|
||||||
protected = false,
|
blockSize: NBytes,
|
||||||
version = CIDv1,
|
datasetSize: NBytes,
|
||||||
|
version: CidVersion = CIDv1,
|
||||||
hcodec = multiCodec("sha2-256"),
|
hcodec = multiCodec("sha2-256"),
|
||||||
codec = multiCodec("raw"),
|
codec = multiCodec("raw"),
|
||||||
blockSize = DefaultBlockSize
|
protected = false,
|
||||||
): ?!Manifest =
|
): Manifest =
|
||||||
## Create a manifest using an array of `Cid`s
|
|
||||||
##
|
|
||||||
|
|
||||||
if hcodec notin EmptyDigests[version]:
|
|
||||||
return failure("Unsupported manifest hash codec!")
|
|
||||||
|
|
||||||
T(
|
T(
|
||||||
blocks: @blocks,
|
treeCid: treeCid,
|
||||||
|
blockSize: blockSize,
|
||||||
|
datasetSize: datasetSize,
|
||||||
version: version,
|
version: version,
|
||||||
codec: codec,
|
codec: codec,
|
||||||
hcodec: hcodec,
|
hcodec: hcodec,
|
||||||
blockSize: blockSize,
|
protected: protected)
|
||||||
originalBytes: blocks.len.NBytes * blockSize,
|
|
||||||
protected: protected).success
|
|
||||||
|
|
||||||
proc new*(
|
proc new*(
|
||||||
T: type Manifest,
|
T: type Manifest,
|
||||||
manifest: Manifest,
|
manifest: Manifest,
|
||||||
|
treeCid: Cid,
|
||||||
|
datasetSize: NBytes,
|
||||||
ecK, ecM: int
|
ecK, ecM: int
|
||||||
): ?!Manifest =
|
): Manifest =
|
||||||
## Create an erasure protected dataset from an
|
## Create an erasure protected dataset from an
|
||||||
## un-protected one
|
## unprotected one
|
||||||
##
|
##
|
||||||
|
Manifest(
|
||||||
|
treeCid: treeCid,
|
||||||
|
datasetSize: datasetSize,
|
||||||
|
version: manifest.version,
|
||||||
|
codec: manifest.codec,
|
||||||
|
hcodec: manifest.hcodec,
|
||||||
|
blockSize: manifest.blockSize,
|
||||||
|
protected: true,
|
||||||
|
ecK: ecK, ecM: ecM,
|
||||||
|
originalTreeCid: manifest.treeCid,
|
||||||
|
originalDatasetSize: manifest.datasetSize)
|
||||||
|
|
||||||
var
|
proc new*(
|
||||||
self = Manifest(
|
T: type Manifest,
|
||||||
version: manifest.version,
|
manifest: Manifest
|
||||||
codec: manifest.codec,
|
): Manifest =
|
||||||
hcodec: manifest.hcodec,
|
## Create an unprotected dataset from an
|
||||||
originalBytes: manifest.originalBytes,
|
## erasure protected one
|
||||||
blockSize: manifest.blockSize,
|
##
|
||||||
protected: true,
|
Manifest(
|
||||||
ecK: ecK, ecM: ecM,
|
treeCid: manifest.originalTreeCid,
|
||||||
originalCid: ? manifest.cid,
|
datasetSize: manifest.originalDatasetSize,
|
||||||
originalLen: manifest.len)
|
version: manifest.version,
|
||||||
|
codec: manifest.codec,
|
||||||
let
|
hcodec: manifest.hcodec,
|
||||||
encodedLen = self.rounded + (self.steps * ecM)
|
blockSize: manifest.blockSize,
|
||||||
|
protected: false)
|
||||||
self.blocks = newSeq[Cid](encodedLen)
|
|
||||||
|
|
||||||
# copy original manifest blocks
|
|
||||||
for i in 0..<self.rounded:
|
|
||||||
if i < manifest.len:
|
|
||||||
self.blocks[i] = manifest[i]
|
|
||||||
else:
|
|
||||||
self.blocks[i] = EmptyCid[manifest.version]
|
|
||||||
.catch
|
|
||||||
.get()[manifest.hcodec]
|
|
||||||
.catch
|
|
||||||
.get()
|
|
||||||
|
|
||||||
? self.verify()
|
|
||||||
self.success
|
|
||||||
|
|
||||||
proc new*(
|
proc new*(
|
||||||
T: type Manifest,
|
T: type Manifest,
|
||||||
|
@ -287,50 +235,27 @@ proc new*(
|
||||||
|
|
||||||
proc new*(
|
proc new*(
|
||||||
T: type Manifest,
|
T: type Manifest,
|
||||||
rootHash: Cid,
|
treeCid: Cid,
|
||||||
originalBytes: NBytes,
|
datasetSize: NBytes,
|
||||||
blockSize: NBytes,
|
blockSize: NBytes,
|
||||||
blocks: seq[Cid],
|
|
||||||
version: CidVersion,
|
version: CidVersion,
|
||||||
hcodec: MultiCodec,
|
hcodec: MultiCodec,
|
||||||
codec: MultiCodec,
|
codec: MultiCodec,
|
||||||
ecK: int,
|
ecK: int,
|
||||||
ecM: int,
|
ecM: int,
|
||||||
originalCid: Cid,
|
originalTreeCid: Cid,
|
||||||
originalLen: int
|
originalDatasetSize: NBytes
|
||||||
): Manifest =
|
): Manifest =
|
||||||
Manifest(
|
Manifest(
|
||||||
rootHash: rootHash.some,
|
treeCid: treeCid,
|
||||||
originalBytes: originalBytes,
|
datasetSize: datasetSize,
|
||||||
blockSize: blockSize,
|
blockSize: blockSize,
|
||||||
blocks: blocks,
|
|
||||||
version: version,
|
version: version,
|
||||||
hcodec: hcodec,
|
hcodec: hcodec,
|
||||||
codec: codec,
|
codec: codec,
|
||||||
protected: true,
|
protected: true,
|
||||||
ecK: ecK,
|
ecK: ecK,
|
||||||
ecM: ecM,
|
ecM: ecM,
|
||||||
originalCid: originalCid,
|
originalTreeCid: originalTreeCid,
|
||||||
originalLen: originalLen
|
originalDatasetSize: originalDatasetSize
|
||||||
)
|
|
||||||
|
|
||||||
proc new*(
|
|
||||||
T: type Manifest,
|
|
||||||
rootHash: Cid,
|
|
||||||
originalBytes: NBytes,
|
|
||||||
blockSize: NBytes,
|
|
||||||
blocks: seq[Cid],
|
|
||||||
version: CidVersion,
|
|
||||||
hcodec: MultiCodec,
|
|
||||||
codec: MultiCodec
|
|
||||||
): Manifest =
|
|
||||||
Manifest(
|
|
||||||
rootHash: rootHash.some,
|
|
||||||
originalBytes: originalBytes,
|
|
||||||
blockSize: blockSize,
|
|
||||||
blocks: blocks,
|
|
||||||
version: version,
|
|
||||||
hcodec: hcodec,
|
|
||||||
codec: codec,
|
|
||||||
protected: false,
|
|
||||||
)
|
)
|
||||||
|
|
|
@ -0,0 +1,4 @@
|
||||||
|
import ./merkletree/merkletree
|
||||||
|
import ./merkletree/coders
|
||||||
|
|
||||||
|
export merkletree, coders
|
|
@ -0,0 +1,75 @@
|
||||||
|
## Nim-Codex
|
||||||
|
## Copyright (c) 2023 Status Research & Development GmbH
|
||||||
|
## Licensed under either of
|
||||||
|
## * Apache License, version 2.0, ([LICENSE-APACHE](LICENSE-APACHE))
|
||||||
|
## * MIT license ([LICENSE-MIT](LICENSE-MIT))
|
||||||
|
## at your option.
|
||||||
|
## This file may not be copied, modified, or distributed except according to
|
||||||
|
## those terms.
|
||||||
|
|
||||||
|
import pkg/libp2p
|
||||||
|
import pkg/questionable
|
||||||
|
import pkg/questionable/results
|
||||||
|
|
||||||
|
import ./merkletree
|
||||||
|
import ../units
|
||||||
|
import ../errors
|
||||||
|
|
||||||
|
const MaxMerkleTreeSize = 100.MiBs.uint
|
||||||
|
const MaxMerkleProofSize = 1.MiBs.uint
|
||||||
|
|
||||||
|
proc encode*(self: MerkleTree): seq[byte] =
|
||||||
|
var pb = initProtoBuffer(maxSize = MaxMerkleTreeSize)
|
||||||
|
pb.write(1, self.mcodec.uint64)
|
||||||
|
pb.write(2, self.digestSize.uint64)
|
||||||
|
pb.write(3, self.leavesCount.uint64)
|
||||||
|
pb.write(4, self.nodesBuffer)
|
||||||
|
pb.finish
|
||||||
|
pb.buffer
|
||||||
|
|
||||||
|
proc decode*(_: type MerkleTree, data: seq[byte]): ?!MerkleTree =
|
||||||
|
var pb = initProtoBuffer(data, maxSize = MaxMerkleTreeSize)
|
||||||
|
var mcodecCode: uint64
|
||||||
|
var digestSize: uint64
|
||||||
|
var leavesCount: uint64
|
||||||
|
discard ? pb.getField(1, mcodecCode).mapFailure
|
||||||
|
discard ? pb.getField(2, digestSize).mapFailure
|
||||||
|
discard ? pb.getField(3, leavesCount).mapFailure
|
||||||
|
|
||||||
|
let mcodec = MultiCodec.codec(cast[int](mcodecCode))
|
||||||
|
if mcodec == InvalidMultiCodec:
|
||||||
|
return failure("Invalid MultiCodec code " & $cast[int](mcodec))
|
||||||
|
|
||||||
|
var nodesBuffer = newSeq[byte]()
|
||||||
|
discard ? pb.getField(4, nodesBuffer).mapFailure
|
||||||
|
|
||||||
|
let tree = ? MerkleTree.init(mcodec, digestSize, leavesCount, nodesBuffer)
|
||||||
|
success(tree)
|
||||||
|
|
||||||
|
proc encode*(self: MerkleProof): seq[byte] =
|
||||||
|
var pb = initProtoBuffer(maxSize = MaxMerkleProofSize)
|
||||||
|
pb.write(1, self.mcodec.uint64)
|
||||||
|
pb.write(2, self.digestSize.uint64)
|
||||||
|
pb.write(3, self.index.uint64)
|
||||||
|
pb.write(4, self.nodesBuffer)
|
||||||
|
pb.finish
|
||||||
|
pb.buffer
|
||||||
|
|
||||||
|
proc decode*(_: type MerkleProof, data: seq[byte]): ?!MerkleProof =
|
||||||
|
var pb = initProtoBuffer(data, maxSize = MaxMerkleProofSize)
|
||||||
|
var mcodecCode: uint64
|
||||||
|
var digestSize: uint64
|
||||||
|
var index: uint64
|
||||||
|
discard ? pb.getField(1, mcodecCode).mapFailure
|
||||||
|
discard ? pb.getField(2, digestSize).mapFailure
|
||||||
|
discard ? pb.getField(3, index).mapFailure
|
||||||
|
|
||||||
|
let mcodec = MultiCodec.codec(cast[int](mcodecCode))
|
||||||
|
if mcodec == InvalidMultiCodec:
|
||||||
|
return failure("Invalid MultiCodec code " & $cast[int](mcodec))
|
||||||
|
|
||||||
|
var nodesBuffer = newSeq[byte]()
|
||||||
|
discard ? pb.getField(4, nodesBuffer).mapFailure
|
||||||
|
|
||||||
|
let proof = ? MerkleProof.init(mcodec, digestSize, index, nodesBuffer)
|
||||||
|
success(proof)
|
|
@ -1,5 +1,5 @@
|
||||||
## Nim-Codex
|
## Nim-Codex
|
||||||
## Copyright (c) 2022 Status Research & Development GmbH
|
## Copyright (c) 2023 Status Research & Development GmbH
|
||||||
## Licensed under either of
|
## Licensed under either of
|
||||||
## * Apache License, version 2.0, ([LICENSE-APACHE](LICENSE-APACHE))
|
## * Apache License, version 2.0, ([LICENSE-APACHE](LICENSE-APACHE))
|
||||||
## * MIT license ([LICENSE-MIT](LICENSE-MIT))
|
## * MIT license ([LICENSE-MIT](LICENSE-MIT))
|
||||||
|
@ -7,41 +7,43 @@
|
||||||
## 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/sequtils
|
|
||||||
import std/math
|
import std/math
|
||||||
import std/bitops
|
import std/bitops
|
||||||
|
import std/sequtils
|
||||||
import std/sugar
|
import std/sugar
|
||||||
|
import std/algorithm
|
||||||
|
|
||||||
import pkg/libp2p
|
import pkg/chronicles
|
||||||
import pkg/stew/byteutils
|
|
||||||
import pkg/questionable
|
import pkg/questionable
|
||||||
import pkg/questionable/results
|
import pkg/questionable/results
|
||||||
|
import pkg/nimcrypto/sha2
|
||||||
|
import pkg/libp2p/[cid, multicodec, multihash, vbuffer]
|
||||||
|
import pkg/stew/byteutils
|
||||||
|
|
||||||
|
import ../errors
|
||||||
|
|
||||||
|
logScope:
|
||||||
|
topics = "codex merkletree"
|
||||||
|
|
||||||
type
|
type
|
||||||
MerkleHash* = MultiHash
|
|
||||||
MerkleTree* = object
|
MerkleTree* = object
|
||||||
leavesCount: int
|
mcodec: MultiCodec
|
||||||
nodes: seq[MerkleHash]
|
digestSize: Natural
|
||||||
|
leavesCount: Natural
|
||||||
|
nodesBuffer*: seq[byte]
|
||||||
MerkleProof* = object
|
MerkleProof* = object
|
||||||
index: int
|
mcodec: MultiCodec
|
||||||
path: seq[MerkleHash]
|
digestSize: Natural
|
||||||
|
index: Natural
|
||||||
# Tree constructed from leaves H0..H2 is
|
nodesBuffer*: seq[byte]
|
||||||
# H5=H(H3 & H4)
|
MerkleTreeBuilder* = object
|
||||||
# / \
|
mcodec: MultiCodec
|
||||||
# H3=H(H0 & H1) H4=H(H2 & H2)
|
digestSize: Natural
|
||||||
# / \ /
|
buffer: seq[byte]
|
||||||
# H0=H(A) H1=H(B) H2=H(C)
|
|
||||||
# | | |
|
|
||||||
# A B C
|
|
||||||
#
|
|
||||||
# Memory layout is [H0, H1, H2, H3, H4, H5]
|
|
||||||
#
|
|
||||||
# Proofs of inclusion are
|
|
||||||
# - [H1, H4] for A
|
|
||||||
# - [H0, H4] for B
|
|
||||||
# - [H2, H3] for C
|
|
||||||
|
|
||||||
|
###########################################################
|
||||||
|
# Helper functions
|
||||||
|
###########################################################
|
||||||
|
|
||||||
func computeTreeHeight(leavesCount: int): int =
|
func computeTreeHeight(leavesCount: int): int =
|
||||||
if isPowerOfTwo(leavesCount):
|
if isPowerOfTwo(leavesCount):
|
||||||
|
@ -49,141 +51,368 @@ func computeTreeHeight(leavesCount: int): int =
|
||||||
else:
|
else:
|
||||||
fastLog2(leavesCount) + 2
|
fastLog2(leavesCount) + 2
|
||||||
|
|
||||||
func getLowHigh(leavesCount, level: int): (int, int) =
|
func computeLevels(leavesCount: int): seq[tuple[offset: int, width: int, index: int]] =
|
||||||
var width = leavesCount
|
|
||||||
var low = 0
|
|
||||||
for _ in 0..<level:
|
|
||||||
low += width
|
|
||||||
width = (width + 1) div 2
|
|
||||||
|
|
||||||
(low, low + width - 1)
|
|
||||||
|
|
||||||
func getLowHigh(self: MerkleTree, level: int): (int, int) =
|
|
||||||
getLowHigh(self.leavesCount, level)
|
|
||||||
|
|
||||||
func computeTotalSize(leavesCount: int): int =
|
|
||||||
let height = computeTreeHeight(leavesCount)
|
let height = computeTreeHeight(leavesCount)
|
||||||
getLowHigh(leavesCount, height - 1)[1] + 1
|
var levels = newSeq[tuple[offset: int, width: int, index: int]](height)
|
||||||
|
|
||||||
proc getWidth(self: MerkleTree, level: int): int =
|
levels[0].offset = 0
|
||||||
let (low, high) = self.getLowHigh(level)
|
levels[0].width = leavesCount
|
||||||
high - low + 1
|
levels[0].index = 0
|
||||||
|
for i in 1..<height:
|
||||||
|
levels[i].offset = levels[i - 1].offset + levels[i - 1].width
|
||||||
|
levels[i].width = (levels[i - 1].width + 1) div 2
|
||||||
|
levels[i].index = i
|
||||||
|
levels
|
||||||
|
|
||||||
func getChildren(self: MerkleTree, level, i: int): (MerkleHash, MerkleHash) =
|
proc digestFn(mcodec: MultiCodec, dst: var openArray[byte], dstPos: int, data: openArray[byte]): ?!void =
|
||||||
let (low, high) = self.getLowHigh(level - 1)
|
var mhash = ? MultiHash.digest($mcodec, data).mapFailure
|
||||||
let leftIdx = low + 2 * i
|
if (dstPos + mhash.size) > dst.len:
|
||||||
let rightIdx = min(leftIdx + 1, high)
|
return failure("Not enough space in a destination buffer")
|
||||||
|
dst[dstPos..<dstPos + mhash.size] = mhash.data.buffer[mhash.dpos..<mhash.dpos + mhash.size]
|
||||||
|
success()
|
||||||
|
|
||||||
(self.nodes[leftIdx], self.nodes[rightIdx])
|
###########################################################
|
||||||
|
# MerkleTreeBuilder
|
||||||
|
###########################################################
|
||||||
|
|
||||||
func getSibling(self: MerkleTree, level, i: int): MerkleHash =
|
proc init*(
|
||||||
let (low, high) = self.getLowHigh(level)
|
T: type MerkleTreeBuilder,
|
||||||
if i mod 2 == 0:
|
mcodec: MultiCodec = multiCodec("sha2-256")
|
||||||
self.nodes[min(low + i + 1, high)]
|
): ?!MerkleTreeBuilder =
|
||||||
else:
|
let mhash = ? MultiHash.digest($mcodec, "".toBytes).mapFailure
|
||||||
self.nodes[low + i - 1]
|
success(MerkleTreeBuilder(mcodec: mcodec, digestSize: mhash.size, buffer: newSeq[byte]()))
|
||||||
|
|
||||||
proc setNode(self: var MerkleTree, level, i: int, value: MerkleHash): void =
|
proc addDataBlock*(self: var MerkleTreeBuilder, dataBlock: openArray[byte]): ?!void =
|
||||||
let (low, _) = self.getLowHigh(level)
|
## Hashes the data block and adds the result of hashing to a buffer
|
||||||
self.nodes[low + i] = value
|
##
|
||||||
|
let oldLen = self.buffer.len
|
||||||
|
self.buffer.setLen(oldLen + self.digestSize)
|
||||||
|
digestFn(self.mcodec, self.buffer, oldLen, dataBlock)
|
||||||
|
|
||||||
proc root*(self: MerkleTree): MerkleHash =
|
proc addLeaf*(self: var MerkleTreeBuilder, leaf: MultiHash): ?!void =
|
||||||
self.nodes[^1]
|
if leaf.mcodec != self.mcodec or leaf.size != self.digestSize:
|
||||||
|
return failure("Expected mcodec to be " & $self.mcodec & " and digest size to be " &
|
||||||
|
$self.digestSize & " but was " & $leaf.mcodec & " and " & $leaf.size)
|
||||||
|
|
||||||
|
let oldLen = self.buffer.len
|
||||||
|
self.buffer.setLen(oldLen + self.digestSize)
|
||||||
|
self.buffer[oldLen..<oldLen + self.digestSize] = leaf.data.buffer[leaf.dpos..<leaf.dpos + self.digestSize]
|
||||||
|
success()
|
||||||
|
|
||||||
proc len*(self: MerkleTree): int =
|
proc build*(self: MerkleTreeBuilder): ?!MerkleTree =
|
||||||
self.nodes.len
|
## Builds a tree from previously added data blocks
|
||||||
|
##
|
||||||
|
## Tree built from data blocks A, B and C is
|
||||||
|
## H5=H(H3 & H4)
|
||||||
|
## / \
|
||||||
|
## H3=H(H0 & H1) H4=H(H2 & 0x00)
|
||||||
|
## / \ /
|
||||||
|
## H0=H(A) H1=H(B) H2=H(C)
|
||||||
|
## | | |
|
||||||
|
## A B C
|
||||||
|
##
|
||||||
|
## Memory layout is [H0, H1, H2, H3, H4, H5]
|
||||||
|
##
|
||||||
|
let
|
||||||
|
mcodec = self.mcodec
|
||||||
|
digestSize = self.digestSize
|
||||||
|
leavesCount = self.buffer.len div self.digestSize
|
||||||
|
|
||||||
proc leaves*(self: MerkleTree): seq[MerkleHash] =
|
if leavesCount == 0:
|
||||||
self.nodes[0..<self.leavesCount]
|
return failure("At least one data block is required")
|
||||||
|
|
||||||
proc nodes*(self: MerkleTree): seq[MerkleHash] =
|
let levels = computeLevels(leavesCount)
|
||||||
self.nodes
|
let totalNodes = levels[^1].offset + 1
|
||||||
|
|
||||||
proc height*(self: MerkleTree): int =
|
var tree = MerkleTree(mcodec: mcodec, digestSize: digestSize, leavesCount: leavesCount, nodesBuffer: newSeq[byte](totalNodes * digestSize))
|
||||||
computeTreeHeight(self.leavesCount)
|
|
||||||
|
|
||||||
proc `$`*(self: MerkleTree): string =
|
|
||||||
result &= "leavesCount: " & $self.leavesCount
|
|
||||||
result &= "\nnodes: " & $self.nodes
|
|
||||||
|
|
||||||
proc getProof*(self: MerkleTree, index: int): ?!MerkleProof =
|
|
||||||
if index >= self.leavesCount or index < 0:
|
|
||||||
return failure("Index " & $index & " out of range [0.." & $self.leaves.high & "]" )
|
|
||||||
|
|
||||||
var path = newSeq[MerkleHash](self.height - 1)
|
|
||||||
for level in 0..<path.len:
|
|
||||||
let i = index div (1 shl level)
|
|
||||||
path[level] = self.getSibling(level, i)
|
|
||||||
|
|
||||||
success(MerkleProof(index: index, path: path))
|
|
||||||
|
|
||||||
proc initTreeFromLeaves(leaves: openArray[MerkleHash]): ?!MerkleTree =
|
|
||||||
without mcodec =? leaves.?[0].?mcodec and
|
|
||||||
digestSize =? leaves.?[0].?size:
|
|
||||||
return failure("At least one leaf is required")
|
|
||||||
|
|
||||||
if not leaves.allIt(it.mcodec == mcodec):
|
|
||||||
return failure("All leaves must use the same codec")
|
|
||||||
|
|
||||||
let totalSize = computeTotalSize(leaves.len)
|
|
||||||
var tree = MerkleTree(leavesCount: leaves.len, nodes: newSeq[MerkleHash](totalSize))
|
|
||||||
|
|
||||||
var buf = newSeq[byte](digestSize * 2)
|
|
||||||
proc combine(l, r: MerkleHash): ?!MerkleHash =
|
|
||||||
copyMem(addr buf[0], unsafeAddr l.data.buffer[0], digestSize)
|
|
||||||
copyMem(addr buf[digestSize], unsafeAddr r.data.buffer[0], digestSize)
|
|
||||||
|
|
||||||
MultiHash.digest($mcodec, buf).mapErr(
|
|
||||||
c => newException(CatchableError, "Error calculating hash using codec " & $mcodec & ": " & $c)
|
|
||||||
)
|
|
||||||
|
|
||||||
# copy leaves
|
# copy leaves
|
||||||
for i in 0..<tree.getWidth(0):
|
tree.nodesBuffer[0..<leavesCount * digestSize] = self.buffer[0..<leavesCount * digestSize]
|
||||||
tree.setNode(0, i, leaves[i])
|
|
||||||
|
|
||||||
# calculate intermediate nodes
|
# calculate intermediate nodes
|
||||||
for level in 1..<tree.height:
|
var zero = newSeq[byte](digestSize)
|
||||||
for i in 0..<tree.getWidth(level):
|
var one = newSeq[byte](digestSize)
|
||||||
let (left, right) = tree.getChildren(level, i)
|
one[^1] = 0x01
|
||||||
|
|
||||||
without mhash =? combine(left, right), error:
|
|
||||||
return failure(error)
|
|
||||||
tree.setNode(level, i, mhash)
|
|
||||||
|
|
||||||
success(tree)
|
var
|
||||||
|
concatBuf = newSeq[byte](2 * digestSize)
|
||||||
|
prevLevel = levels[0]
|
||||||
|
for level in levels[1..^1]:
|
||||||
|
for i in 0..<level.width:
|
||||||
|
let parentIndex = level.offset + i
|
||||||
|
let leftChildIndex = prevLevel.offset + 2 * i
|
||||||
|
let rightChildIndex = leftChildIndex + 1
|
||||||
|
|
||||||
func init*(
|
concatBuf[0..<digestSize] = tree.nodesBuffer[leftChildIndex * digestSize..<(leftChildIndex + 1) * digestSize]
|
||||||
T: type MerkleTree,
|
|
||||||
root: MerkleHash,
|
var dummyValue = if prevLevel.index == 0: zero else: one
|
||||||
leavesCount: int
|
|
||||||
): MerkleTree =
|
if rightChildIndex < prevLevel.offset + prevLevel.width:
|
||||||
let totalSize = computeTotalSize(leavesCount)
|
concatBuf[digestSize..^1] = tree.nodesBuffer[rightChildIndex * digestSize..<(rightChildIndex + 1) * digestSize]
|
||||||
var nodes = newSeq[MerkleHash](totalSize)
|
else:
|
||||||
nodes[^1] = root
|
concatBuf[digestSize..^1] = dummyValue
|
||||||
MerkleTree(nodes: nodes, leavesCount: leavesCount)
|
|
||||||
|
? digestFn(mcodec, tree.nodesBuffer, parentIndex * digestSize, concatBuf)
|
||||||
|
prevLevel = level
|
||||||
|
|
||||||
|
return success(tree)
|
||||||
|
|
||||||
|
###########################################################
|
||||||
|
# MerkleTree
|
||||||
|
###########################################################
|
||||||
|
|
||||||
|
proc nodeBufferToMultiHash(self: (MerkleTree | MerkleProof), index: int): MultiHash =
|
||||||
|
var buf = newSeq[byte](self.digestSize)
|
||||||
|
let offset = index * self.digestSize
|
||||||
|
buf[0..^1] = self.nodesBuffer[offset..<(offset + self.digestSize)]
|
||||||
|
|
||||||
|
{.noSideEffect.}:
|
||||||
|
without mhash =? MultiHash.init($self.mcodec, buf).mapFailure, errx:
|
||||||
|
error "Error converting bytes to hash", msg = errx.msg
|
||||||
|
mhash
|
||||||
|
|
||||||
|
proc len*(self: (MerkleTree | MerkleProof)): Natural =
|
||||||
|
self.nodesBuffer.len div self.digestSize
|
||||||
|
|
||||||
|
proc nodes*(self: (MerkleTree | MerkleProof)): seq[MultiHash] {.noSideEffect.} =
|
||||||
|
toSeq(0..<self.len).map(i => self.nodeBufferToMultiHash(i))
|
||||||
|
|
||||||
|
proc mcodec*(self: (MerkleTree | MerkleProof)): MultiCodec =
|
||||||
|
self.mcodec
|
||||||
|
|
||||||
|
proc digestSize*(self: (MerkleTree | MerkleProof)): Natural =
|
||||||
|
self.digestSize
|
||||||
|
|
||||||
|
proc root*(self: MerkleTree): MultiHash =
|
||||||
|
let rootIndex = self.len - 1
|
||||||
|
self.nodeBufferToMultiHash(rootIndex)
|
||||||
|
|
||||||
|
proc rootCid*(self: MerkleTree, version = CIDv1, dataCodec = multiCodec("raw")): ?!Cid =
|
||||||
|
Cid.init(version, dataCodec, self.root).mapFailure
|
||||||
|
|
||||||
|
iterator leaves*(self: MerkleTree): MultiHash =
|
||||||
|
for i in 0..<self.leavesCount:
|
||||||
|
yield self.nodeBufferToMultiHash(i)
|
||||||
|
|
||||||
|
iterator leavesCids*(self: MerkleTree, version = CIDv1, dataCodec = multiCodec("raw")): ?!Cid =
|
||||||
|
for leaf in self.leaves:
|
||||||
|
yield Cid.init(version, dataCodec, leaf).mapFailure
|
||||||
|
|
||||||
|
proc leavesCount*(self: MerkleTree): Natural =
|
||||||
|
self.leavesCount
|
||||||
|
|
||||||
|
proc getLeaf*(self: MerkleTree, index: Natural): ?!MultiHash =
|
||||||
|
if index >= self.leavesCount:
|
||||||
|
return failure("Index " & $index & " out of range [0.." & $(self.leavesCount - 1) & "]" )
|
||||||
|
|
||||||
|
success(self.nodeBufferToMultiHash(index))
|
||||||
|
|
||||||
|
proc getLeafCid*(self: MerkleTree, index: Natural, version = CIDv1, dataCodec = multiCodec("raw")): ?!Cid =
|
||||||
|
let leaf = ? self.getLeaf(index)
|
||||||
|
Cid.init(version, dataCodec, leaf).mapFailure
|
||||||
|
|
||||||
|
proc height*(self: MerkleTree): Natural =
|
||||||
|
computeTreeHeight(self.leavesCount)
|
||||||
|
|
||||||
|
proc getProof*(self: MerkleTree, index: Natural): ?!MerkleProof =
|
||||||
|
## Extracts proof from a tree for a given index
|
||||||
|
##
|
||||||
|
## Given a tree built from data blocks A, B and C
|
||||||
|
## H5
|
||||||
|
## / \
|
||||||
|
## H3 H4
|
||||||
|
## / \ /
|
||||||
|
## H0 H1 H2
|
||||||
|
## | | |
|
||||||
|
## A B C
|
||||||
|
##
|
||||||
|
## Proofs of inclusion (index and path) are
|
||||||
|
## - 0,[H1, H4] for data block A
|
||||||
|
## - 1,[H0, H4] for data block B
|
||||||
|
## - 2,[0x00, H3] for data block C
|
||||||
|
##
|
||||||
|
if index >= self.leavesCount:
|
||||||
|
return failure("Index " & $index & " out of range [0.." & $(self.leavesCount - 1) & "]" )
|
||||||
|
|
||||||
|
var zero = newSeq[byte](self.digestSize)
|
||||||
|
var one = newSeq[byte](self.digestSize)
|
||||||
|
one[^1] = 0x01
|
||||||
|
|
||||||
|
let levels = computeLevels(self.leavesCount)
|
||||||
|
var proofNodesBuffer = newSeq[byte]((levels.len - 1) * self.digestSize)
|
||||||
|
for level in levels[0..^2]:
|
||||||
|
let lr = index shr level.index
|
||||||
|
let siblingIndex = if lr mod 2 == 0:
|
||||||
|
level.offset + lr + 1
|
||||||
|
else:
|
||||||
|
level.offset + lr - 1
|
||||||
|
|
||||||
|
var dummyValue = if level.index == 0: zero else: one
|
||||||
|
|
||||||
|
if siblingIndex < level.offset + level.width:
|
||||||
|
proofNodesBuffer[level.index * self.digestSize..<(level.index + 1) * self.digestSize] =
|
||||||
|
self.nodesBuffer[siblingIndex * self.digestSize..<(siblingIndex + 1) * self.digestSize]
|
||||||
|
else:
|
||||||
|
proofNodesBuffer[level.index * self.digestSize..<(level.index + 1) * self.digestSize] = dummyValue
|
||||||
|
|
||||||
|
success(MerkleProof(mcodec: self.mcodec, digestSize: self.digestSize, index: index, nodesBuffer: proofNodesBuffer))
|
||||||
|
|
||||||
|
proc `$`*(self: MerkleTree): string {.noSideEffect.} =
|
||||||
|
"mcodec:" & $self.mcodec &
|
||||||
|
", digestSize: " & $self.digestSize &
|
||||||
|
", leavesCount: " & $self.leavesCount &
|
||||||
|
", nodes: " & $self.nodes
|
||||||
|
|
||||||
|
proc `==`*(a, b: MerkleTree): bool =
|
||||||
|
(a.mcodec == b.mcodec) and
|
||||||
|
(a.digestSize == b.digestSize) and
|
||||||
|
(a.leavesCount == b.leavesCount) and
|
||||||
|
(a.nodesBuffer == b.nodesBuffer)
|
||||||
|
|
||||||
proc init*(
|
proc init*(
|
||||||
T: type MerkleTree,
|
T: type MerkleTree,
|
||||||
leaves: openArray[MerkleHash]
|
mcodec: MultiCodec,
|
||||||
|
digestSize: Natural,
|
||||||
|
leavesCount: Natural,
|
||||||
|
nodesBuffer: seq[byte]
|
||||||
): ?!MerkleTree =
|
): ?!MerkleTree =
|
||||||
initTreeFromLeaves(leaves)
|
let levels = computeLevels(leavesCount)
|
||||||
|
let totalNodes = levels[^1].offset + 1
|
||||||
|
if totalNodes * digestSize == nodesBuffer.len:
|
||||||
|
success(
|
||||||
|
MerkleTree(
|
||||||
|
mcodec: mcodec,
|
||||||
|
digestSize: digestSize,
|
||||||
|
leavesCount: leavesCount,
|
||||||
|
nodesBuffer: nodesBuffer
|
||||||
|
)
|
||||||
|
)
|
||||||
|
else:
|
||||||
|
failure("Expected nodesBuffer len to be " & $(totalNodes * digestSize) & " but was " & $nodesBuffer.len)
|
||||||
|
|
||||||
proc index*(self: MerkleProof): int =
|
proc init*(
|
||||||
|
T: type MerkleTree,
|
||||||
|
leaves: openArray[MultiHash]
|
||||||
|
): ?!MerkleTree =
|
||||||
|
without leaf =? leaves.?[0]:
|
||||||
|
return failure("At least one leaf is required")
|
||||||
|
|
||||||
|
var builder = ? MerkleTreeBuilder.init(mcodec = leaf.mcodec)
|
||||||
|
|
||||||
|
for l in leaves:
|
||||||
|
let res = builder.addLeaf(l)
|
||||||
|
if res.isErr:
|
||||||
|
return failure(res.error)
|
||||||
|
|
||||||
|
builder.build()
|
||||||
|
|
||||||
|
proc init*(
|
||||||
|
T: type MerkleTree,
|
||||||
|
cids: openArray[Cid]
|
||||||
|
): ?!MerkleTree =
|
||||||
|
var leaves = newSeq[MultiHash]()
|
||||||
|
|
||||||
|
for cid in cids:
|
||||||
|
let res = cid.mhash.mapFailure
|
||||||
|
if res.isErr:
|
||||||
|
return failure(res.error)
|
||||||
|
else:
|
||||||
|
leaves.add(res.value)
|
||||||
|
|
||||||
|
MerkleTree.init(leaves)
|
||||||
|
|
||||||
|
###########################################################
|
||||||
|
# MerkleProof
|
||||||
|
###########################################################
|
||||||
|
|
||||||
|
proc verifyLeaf*(self: MerkleProof, leaf: MultiHash, treeRoot: MultiHash): ?!bool =
|
||||||
|
if leaf.mcodec != self.mcodec:
|
||||||
|
return failure("Leaf mcodec was " & $leaf.mcodec & ", but " & $self.mcodec & " expected")
|
||||||
|
|
||||||
|
if leaf.mcodec != self.mcodec:
|
||||||
|
return failure("Tree root mcodec was " & $treeRoot.mcodec & ", but " & $treeRoot.mcodec & " expected")
|
||||||
|
|
||||||
|
var digestBuf = newSeq[byte](self.digestSize)
|
||||||
|
digestBuf[0..^1] = leaf.data.buffer[leaf.dpos..<(leaf.dpos + self.digestSize)]
|
||||||
|
|
||||||
|
let proofLen = self.nodesBuffer.len div self.digestSize
|
||||||
|
var concatBuf = newSeq[byte](2 * self.digestSize)
|
||||||
|
for i in 0..<proofLen:
|
||||||
|
let offset = i * self.digestSize
|
||||||
|
let lr = self.index shr i
|
||||||
|
if lr mod 2 == 0:
|
||||||
|
concatBuf[0..^1] = digestBuf & self.nodesBuffer[offset..<(offset + self.digestSize)]
|
||||||
|
else:
|
||||||
|
concatBuf[0..^1] = self.nodesBuffer[offset..<(offset + self.digestSize)] & digestBuf
|
||||||
|
? digestFn(self.mcodec, digestBuf, 0, concatBuf)
|
||||||
|
|
||||||
|
let computedRoot = ? MultiHash.init(self.mcodec, digestBuf).mapFailure
|
||||||
|
|
||||||
|
success(computedRoot == treeRoot)
|
||||||
|
|
||||||
|
|
||||||
|
proc verifyDataBlock*(self: MerkleProof, dataBlock: openArray[byte], treeRoot: MultiHash): ?!bool =
|
||||||
|
var digestBuf = newSeq[byte](self.digestSize)
|
||||||
|
? digestFn(self.mcodec, digestBuf, 0, dataBlock)
|
||||||
|
|
||||||
|
let leaf = ? MultiHash.init(self.mcodec, digestBuf).mapFailure
|
||||||
|
|
||||||
|
self.verifyLeaf(leaf, treeRoot)
|
||||||
|
|
||||||
|
proc index*(self: MerkleProof): Natural =
|
||||||
self.index
|
self.index
|
||||||
|
|
||||||
proc path*(self: MerkleProof): seq[MerkleHash] =
|
|
||||||
self.path
|
|
||||||
|
|
||||||
proc `$`*(self: MerkleProof): string =
|
proc `$`*(self: MerkleProof): string =
|
||||||
result &= "index: " & $self.index
|
"mcodec:" & $self.mcodec &
|
||||||
result &= "\npath: " & $self.path
|
", digestSize: " & $self.digestSize &
|
||||||
|
", index: " & $self.index &
|
||||||
|
", nodes: " & $self.nodes
|
||||||
|
|
||||||
func `==`*(a, b: MerkleProof): bool =
|
func `==`*(a, b: MerkleProof): bool =
|
||||||
(a.index == b.index) and (a.path == b.path)
|
(a.index == b.index) and
|
||||||
|
(a.mcodec == b.mcodec) and
|
||||||
|
(a.digestSize == b.digestSize) and
|
||||||
|
(a.nodesBuffer == b.nodesBuffer)
|
||||||
|
|
||||||
proc init*(
|
proc init*(
|
||||||
T: type MerkleProof,
|
T: type MerkleProof,
|
||||||
index: int,
|
index: Natural,
|
||||||
path: seq[MerkleHash]
|
nodes: seq[MultiHash]
|
||||||
): MerkleProof =
|
): ?!MerkleProof =
|
||||||
MerkleProof(index: index, path: path)
|
if nodes.len == 0:
|
||||||
|
return failure("At least one node is required")
|
||||||
|
|
||||||
|
let
|
||||||
|
mcodec = nodes[0].mcodec
|
||||||
|
digestSize = nodes[0].size
|
||||||
|
|
||||||
|
var nodesBuffer = newSeq[byte](nodes.len * digestSize)
|
||||||
|
for nodeIndex, node in nodes:
|
||||||
|
nodesBuffer[nodeIndex * digestSize..<(nodeIndex + 1) * digestSize] = node.data.buffer[node.dpos..<node.dpos + digestSize]
|
||||||
|
|
||||||
|
success(MerkleProof(mcodec: mcodec, digestSize: digestSize, index: index, nodesBuffer: nodesBuffer))
|
||||||
|
|
||||||
|
func init*(
|
||||||
|
T: type MerkleProof,
|
||||||
|
mcodec: MultiCodec,
|
||||||
|
digestSize: Natural,
|
||||||
|
index: Natural,
|
||||||
|
nodesBuffer: seq[byte]
|
||||||
|
): ?!MerkleProof =
|
||||||
|
|
||||||
|
if nodesBuffer.len mod digestSize != 0:
|
||||||
|
return failure("nodesBuffer len is not a multiple of digestSize")
|
||||||
|
|
||||||
|
let treeHeight = (nodesBuffer.len div digestSize) + 1
|
||||||
|
let maxLeavesCount = 1 shl treeHeight
|
||||||
|
if index < maxLeavesCount:
|
||||||
|
return success(
|
||||||
|
MerkleProof(
|
||||||
|
mcodec: mcodec,
|
||||||
|
digestSize: digestSize,
|
||||||
|
index: index,
|
||||||
|
nodesBuffer: nodesBuffer
|
||||||
|
)
|
||||||
|
)
|
||||||
|
else:
|
||||||
|
return failure("index higher than max leaves count")
|
||||||
|
|
|
@ -16,6 +16,8 @@ const
|
||||||
CodexManifestNamespace* = CodexRepoNamespace & "/manifests" # manifest namespace
|
CodexManifestNamespace* = CodexRepoNamespace & "/manifests" # manifest namespace
|
||||||
CodexBlocksTtlNamespace* = # Cid TTL
|
CodexBlocksTtlNamespace* = # Cid TTL
|
||||||
CodexMetaNamespace & "/ttl"
|
CodexMetaNamespace & "/ttl"
|
||||||
|
CodexBlockProofNamespace* = # Cid and Proof
|
||||||
|
CodexMetaNamespace & "/proof"
|
||||||
CodexDhtNamespace* = "dht" # Dht namespace
|
CodexDhtNamespace* = "dht" # Dht namespace
|
||||||
CodexDhtProvidersNamespace* = # Dht providers namespace
|
CodexDhtProvidersNamespace* = # Dht providers namespace
|
||||||
CodexDhtNamespace & "/providers"
|
CodexDhtNamespace & "/providers"
|
||||||
|
|
108
codex/node.nim
108
codex/node.nim
|
@ -11,13 +11,14 @@ import std/options
|
||||||
import std/tables
|
import std/tables
|
||||||
import std/sequtils
|
import std/sequtils
|
||||||
import std/strformat
|
import std/strformat
|
||||||
|
import std/sugar
|
||||||
|
|
||||||
import pkg/questionable
|
import pkg/questionable
|
||||||
import pkg/questionable/results
|
import pkg/questionable/results
|
||||||
import pkg/chronicles
|
import pkg/chronicles
|
||||||
import pkg/chronos
|
import pkg/chronos
|
||||||
|
|
||||||
import pkg/libp2p/switch
|
import pkg/libp2p/[switch, multicodec, multihash]
|
||||||
import pkg/libp2p/stream/bufferstream
|
import pkg/libp2p/stream/bufferstream
|
||||||
|
|
||||||
# TODO: remove once exported by libp2p
|
# TODO: remove once exported by libp2p
|
||||||
|
@ -27,6 +28,7 @@ import pkg/libp2p/signed_envelope
|
||||||
import ./chunker
|
import ./chunker
|
||||||
import ./blocktype as bt
|
import ./blocktype as bt
|
||||||
import ./manifest
|
import ./manifest
|
||||||
|
import ./merkletree
|
||||||
import ./stores/blockstore
|
import ./stores/blockstore
|
||||||
import ./blockexchange
|
import ./blockexchange
|
||||||
import ./streams
|
import ./streams
|
||||||
|
@ -34,6 +36,7 @@ import ./erasure
|
||||||
import ./discovery
|
import ./discovery
|
||||||
import ./contracts
|
import ./contracts
|
||||||
import ./node/batch
|
import ./node/batch
|
||||||
|
import ./utils
|
||||||
|
|
||||||
export batch
|
export batch
|
||||||
|
|
||||||
|
@ -87,7 +90,7 @@ proc fetchManifest*(
|
||||||
|
|
||||||
trace "Retrieving manifest for cid", cid
|
trace "Retrieving manifest for cid", cid
|
||||||
|
|
||||||
without blk =? await node.blockStore.getBlock(cid), err:
|
without blk =? await node.blockStore.getBlock(BlockAddress.init(cid)), err:
|
||||||
trace "Error retrieve manifest block", cid, err = err.msg
|
trace "Error retrieve manifest block", cid, err = err.msg
|
||||||
return failure err
|
return failure err
|
||||||
|
|
||||||
|
@ -108,18 +111,21 @@ proc fetchBatched*(
|
||||||
onBatch: BatchProc = nil): Future[?!void] {.async, gcsafe.} =
|
onBatch: BatchProc = nil): Future[?!void] {.async, gcsafe.} =
|
||||||
## Fetch manifest in batches of `batchSize`
|
## Fetch manifest in batches of `batchSize`
|
||||||
##
|
##
|
||||||
|
|
||||||
let
|
let batchCount = divUp(manifest.blocksCount, batchSize)
|
||||||
batches =
|
|
||||||
(manifest.blocks.len div batchSize) +
|
|
||||||
(manifest.blocks.len mod batchSize)
|
|
||||||
|
|
||||||
trace "Fetching blocks in batches of", size = batchSize
|
trace "Fetching blocks in batches of", size = batchSize
|
||||||
for blks in manifest.blocks.distribute(max(1, batches), true):
|
|
||||||
try:
|
|
||||||
let
|
|
||||||
blocks = blks.mapIt(node.blockStore.getBlock( it ))
|
|
||||||
|
|
||||||
|
let iter = Iter.fromSlice(0..<manifest.blocksCount)
|
||||||
|
.map((i: int) => node.blockStore.getBlock(BlockAddress.init(manifest.treeCid, i)))
|
||||||
|
|
||||||
|
for batchNum in 0..<batchCount:
|
||||||
|
let blocks = collect:
|
||||||
|
for i in 0..<batchSize:
|
||||||
|
if not iter.finished:
|
||||||
|
iter.next()
|
||||||
|
|
||||||
|
try:
|
||||||
await allFuturesThrowing(allFinished(blocks))
|
await allFuturesThrowing(allFinished(blocks))
|
||||||
if not onBatch.isNil:
|
if not onBatch.isNil:
|
||||||
await onBatch(blocks.mapIt( it.read.get ))
|
await onBatch(blocks.mapIt( it.read.get ))
|
||||||
|
@ -157,7 +163,7 @@ proc retrieve*(
|
||||||
let
|
let
|
||||||
stream = BufferStream.new()
|
stream = BufferStream.new()
|
||||||
|
|
||||||
without blk =? (await node.blockStore.getBlock(cid)), err:
|
without blk =? (await node.blockStore.getBlock(BlockAddress.init(cid))), err:
|
||||||
return failure(err)
|
return failure(err)
|
||||||
|
|
||||||
proc streamOneBlock(): Future[void] {.async.} =
|
proc streamOneBlock(): Future[void] {.async.} =
|
||||||
|
@ -181,11 +187,12 @@ proc store*(
|
||||||
##
|
##
|
||||||
trace "Storing data"
|
trace "Storing data"
|
||||||
|
|
||||||
without var blockManifest =? Manifest.new(blockSize = blockSize):
|
let
|
||||||
return failure("Unable to create Block Set")
|
hcodec = multiCodec("sha2-256")
|
||||||
|
dataCodec = multiCodec("raw")
|
||||||
|
chunker = LPStreamChunker.new(stream, chunkSize = blockSize)
|
||||||
|
|
||||||
# Manifest and chunker should use the same blockSize
|
var cids: seq[Cid]
|
||||||
let chunker = LPStreamChunker.new(stream, chunkSize = blockSize)
|
|
||||||
|
|
||||||
try:
|
try:
|
||||||
while (
|
while (
|
||||||
|
@ -193,10 +200,18 @@ proc store*(
|
||||||
chunk.len > 0):
|
chunk.len > 0):
|
||||||
|
|
||||||
trace "Got data from stream", len = chunk.len
|
trace "Got data from stream", len = chunk.len
|
||||||
without blk =? bt.Block.new(chunk):
|
|
||||||
return failure("Unable to init block from chunk!")
|
|
||||||
|
|
||||||
blockManifest.add(blk.cid)
|
without mhash =? MultiHash.digest($hcodec, chunk).mapFailure, err:
|
||||||
|
return failure(err)
|
||||||
|
|
||||||
|
without cid =? Cid.init(CIDv1, dataCodec, mhash).mapFailure, err:
|
||||||
|
return failure(err)
|
||||||
|
|
||||||
|
without blk =? bt.Block.new(cid, chunk, verify = false):
|
||||||
|
return failure("Unable to init block from chunk!")
|
||||||
|
|
||||||
|
cids.add(cid)
|
||||||
|
|
||||||
if err =? (await self.blockStore.putBlock(blk)).errorOption:
|
if err =? (await self.blockStore.putBlock(blk)).errorOption:
|
||||||
trace "Unable to store block", cid = blk.cid, err = err.msg
|
trace "Unable to store block", cid = blk.cid, err = err.msg
|
||||||
return failure(&"Unable to store block {blk.cid}")
|
return failure(&"Unable to store block {blk.cid}")
|
||||||
|
@ -208,34 +223,51 @@ proc store*(
|
||||||
finally:
|
finally:
|
||||||
await stream.close()
|
await stream.close()
|
||||||
|
|
||||||
|
without tree =? MerkleTree.init(cids), err:
|
||||||
|
return failure(err)
|
||||||
|
|
||||||
|
without treeCid =? tree.rootCid(CIDv1, dataCodec), err:
|
||||||
|
return failure(err)
|
||||||
|
|
||||||
|
for index, cid in cids:
|
||||||
|
without proof =? tree.getProof(index), err:
|
||||||
|
return failure(err)
|
||||||
|
if err =? (await self.blockStore.putBlockCidAndProof(treeCid, index, cid, proof)).errorOption:
|
||||||
|
# TODO add log here
|
||||||
|
return failure(err)
|
||||||
|
|
||||||
|
let manifest = Manifest.new(
|
||||||
|
treeCid = treeCid,
|
||||||
|
blockSize = blockSize,
|
||||||
|
datasetSize = NBytes(chunker.offset),
|
||||||
|
version = CIDv1,
|
||||||
|
hcodec = hcodec,
|
||||||
|
codec = dataCodec
|
||||||
|
)
|
||||||
# Generate manifest
|
# Generate manifest
|
||||||
blockManifest.originalBytes = NBytes(chunker.offset) # store the exact file size
|
without data =? manifest.encode(), err:
|
||||||
without data =? blockManifest.encode():
|
|
||||||
return failure(
|
return failure(
|
||||||
newException(CodexError, "Could not generate dataset manifest!"))
|
newException(CodexError, "Error encoding manifest: " & err.msg))
|
||||||
|
|
||||||
# Store as a dag-pb block
|
# Store as a dag-pb block
|
||||||
without manifest =? bt.Block.new(data = data, codec = DagPBCodec):
|
without manifestBlk =? bt.Block.new(data = data, codec = DagPBCodec):
|
||||||
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 isErr (await self.blockStore.putBlock(manifest)):
|
if isErr (await self.blockStore.putBlock(manifestBlk)):
|
||||||
trace "Unable to store manifest", cid = manifest.cid
|
trace "Unable to store manifest", cid = manifestBlk.cid
|
||||||
return failure("Unable to store manifest " & $manifest.cid)
|
return failure("Unable to store manifest " & $manifestBlk.cid)
|
||||||
|
|
||||||
without cid =? blockManifest.cid, error:
|
info "Stored data", manifestCid = manifestBlk.cid,
|
||||||
trace "Unable to generate manifest Cid!", exc = error.msg
|
treeCid = treeCid,
|
||||||
return failure(error.msg)
|
blocks = manifest.blocksCount,
|
||||||
|
datasetSize = manifest.datasetSize
|
||||||
trace "Stored data", manifestCid = manifest.cid,
|
|
||||||
contentCid = cid,
|
|
||||||
blocks = blockManifest.len,
|
|
||||||
size=blockManifest.originalBytes
|
|
||||||
|
|
||||||
# Announce manifest
|
# Announce manifest
|
||||||
await self.discovery.provide(manifest.cid)
|
await self.discovery.provide(manifestBlk.cid)
|
||||||
|
await self.discovery.provide(treeCid)
|
||||||
|
|
||||||
return manifest.cid.success
|
return manifestBlk.cid.success
|
||||||
|
|
||||||
proc iterateManifests*(node: CodexNodeRef, onManifest: OnManifest) {.async.} =
|
proc iterateManifests*(node: CodexNodeRef, onManifest: OnManifest) {.async.} =
|
||||||
without cids =? await node.blockStore.listBlocks(BlockType.Manifest):
|
without cids =? await node.blockStore.listBlocks(BlockType.Manifest):
|
||||||
|
@ -309,7 +341,7 @@ proc requestStorage*(
|
||||||
# because the slotSize is used to determine the amount of bytes to reserve
|
# because the slotSize is used to determine the amount of bytes to reserve
|
||||||
# in a Reservations
|
# in a Reservations
|
||||||
# TODO: slotSize: (encoded.blockSize.int * encoded.steps).u256,
|
# TODO: slotSize: (encoded.blockSize.int * encoded.steps).u256,
|
||||||
slotSize: (encoded.blockSize.int * encoded.blocks.len).u256,
|
slotSize: (encoded.blockSize.int * encoded.blocksCount).u256,
|
||||||
duration: duration,
|
duration: duration,
|
||||||
proofProbability: proofProbability,
|
proofProbability: proofProbability,
|
||||||
reward: reward,
|
reward: reward,
|
||||||
|
@ -319,7 +351,7 @@ proc requestStorage*(
|
||||||
content: StorageContent(
|
content: StorageContent(
|
||||||
cid: $encodedBlk.cid,
|
cid: $encodedBlk.cid,
|
||||||
erasure: StorageErasure(
|
erasure: StorageErasure(
|
||||||
totalChunks: encoded.len.uint64,
|
totalChunks: encoded.blocksCount.uint64,
|
||||||
),
|
),
|
||||||
por: StoragePoR(
|
por: StoragePoR(
|
||||||
u: @[], # TODO: PoR setup
|
u: @[], # TODO: PoR setup
|
||||||
|
|
|
@ -4,5 +4,6 @@ import ./stores/networkstore
|
||||||
import ./stores/repostore
|
import ./stores/repostore
|
||||||
import ./stores/maintenance
|
import ./stores/maintenance
|
||||||
import ./stores/keyutils
|
import ./stores/keyutils
|
||||||
|
import ./stores/treehelper
|
||||||
|
|
||||||
export cachestore, blockstore, networkstore, repostore, maintenance, keyutils
|
export cachestore, blockstore, networkstore, repostore, maintenance, keyutils, treehelper
|
||||||
|
|
|
@ -18,6 +18,8 @@ import pkg/questionable/results
|
||||||
|
|
||||||
import ../clock
|
import ../clock
|
||||||
import ../blocktype
|
import ../blocktype
|
||||||
|
import ../merkletree
|
||||||
|
import ../utils
|
||||||
|
|
||||||
export blocktype
|
export blocktype
|
||||||
|
|
||||||
|
@ -27,23 +29,31 @@ type
|
||||||
BlockType* {.pure.} = enum
|
BlockType* {.pure.} = enum
|
||||||
Manifest, Block, Both
|
Manifest, Block, Both
|
||||||
|
|
||||||
GetNext* = proc(): Future[?Cid] {.upraises: [], gcsafe, closure.}
|
|
||||||
|
|
||||||
BlocksIter* = ref object
|
|
||||||
finished*: bool
|
|
||||||
next*: GetNext
|
|
||||||
|
|
||||||
BlockStore* = ref object of RootObj
|
BlockStore* = ref object of RootObj
|
||||||
|
|
||||||
iterator items*(self: BlocksIter): Future[?Cid] =
|
|
||||||
while not self.finished:
|
|
||||||
yield self.next()
|
|
||||||
|
|
||||||
method getBlock*(self: BlockStore, cid: Cid): Future[?!Block] {.base.} =
|
method getBlock*(self: BlockStore, cid: Cid): Future[?!Block] {.base.} =
|
||||||
## Get a block from the blockstore
|
## Get a block from the blockstore
|
||||||
##
|
##
|
||||||
|
|
||||||
raiseAssert("Not implemented!")
|
raiseAssert("getBlock by cid not implemented!")
|
||||||
|
|
||||||
|
method getBlock*(self: BlockStore, treeCid: Cid, index: Natural): Future[?!Block] {.base.} =
|
||||||
|
## Get a block from the blockstore
|
||||||
|
##
|
||||||
|
|
||||||
|
raiseAssert("getBlock by treecid not implemented!")
|
||||||
|
|
||||||
|
method getBlock*(self: BlockStore, address: BlockAddress): Future[?!Block] {.base.} =
|
||||||
|
## Get a block from the blockstore
|
||||||
|
##
|
||||||
|
|
||||||
|
raiseAssert("getBlock by addr not implemented!")
|
||||||
|
|
||||||
|
method getBlockAndProof*(self: BlockStore, treeCid: Cid, index: Natural): Future[?!(Block, MerkleProof)] {.base.} =
|
||||||
|
## Get a block and associated inclusion proof by Cid of a merkle tree and an index of a leaf in a tree
|
||||||
|
##
|
||||||
|
|
||||||
|
raiseAssert("getBlockAndProof not implemented!")
|
||||||
|
|
||||||
method putBlock*(
|
method putBlock*(
|
||||||
self: BlockStore,
|
self: BlockStore,
|
||||||
|
@ -53,7 +63,19 @@ method putBlock*(
|
||||||
## Put a block to the blockstore
|
## Put a block to the blockstore
|
||||||
##
|
##
|
||||||
|
|
||||||
raiseAssert("Not implemented!")
|
raiseAssert("putBlock not implemented!")
|
||||||
|
|
||||||
|
method putBlockCidAndProof*(
|
||||||
|
self: BlockStore,
|
||||||
|
treeCid: Cid,
|
||||||
|
index: Natural,
|
||||||
|
blockCid: Cid,
|
||||||
|
proof: MerkleProof
|
||||||
|
): Future[?!void] {.base.} =
|
||||||
|
## Put a block to the blockstore
|
||||||
|
##
|
||||||
|
|
||||||
|
raiseAssert("putBlockCidAndProof not implemented!")
|
||||||
|
|
||||||
method ensureExpiry*(
|
method ensureExpiry*(
|
||||||
self: BlockStore,
|
self: BlockStore,
|
||||||
|
@ -70,28 +92,40 @@ method delBlock*(self: BlockStore, cid: Cid): Future[?!void] {.base.} =
|
||||||
## Delete a block from the blockstore
|
## Delete a block from the blockstore
|
||||||
##
|
##
|
||||||
|
|
||||||
raiseAssert("Not implemented!")
|
raiseAssert("delBlock not implemented!")
|
||||||
|
|
||||||
|
method delBlock*(self: BlockStore, treeCid: Cid, index: Natural): Future[?!void] {.base.} =
|
||||||
|
## Delete a block from the blockstore
|
||||||
|
##
|
||||||
|
|
||||||
|
raiseAssert("delBlock not implemented!")
|
||||||
|
|
||||||
method hasBlock*(self: BlockStore, cid: Cid): Future[?!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
|
||||||
##
|
##
|
||||||
|
|
||||||
raiseAssert("Not implemented!")
|
raiseAssert("hasBlock not implemented!")
|
||||||
|
|
||||||
|
method hasBlock*(self: BlockStore, tree: Cid, index: Natural): Future[?!bool] {.base.} =
|
||||||
|
## Check if the block exists in the blockstore
|
||||||
|
##
|
||||||
|
|
||||||
|
raiseAssert("hasBlock not implemented!")
|
||||||
|
|
||||||
method listBlocks*(
|
method listBlocks*(
|
||||||
self: BlockStore,
|
self: BlockStore,
|
||||||
blockType = BlockType.Manifest): Future[?!BlocksIter] {.base.} =
|
blockType = BlockType.Manifest): Future[?!AsyncIter[?Cid]] {.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("listBlocks not implemented!")
|
||||||
|
|
||||||
method close*(self: BlockStore): Future[void] {.base.} =
|
method close*(self: BlockStore): Future[void] {.base.} =
|
||||||
## Close the blockstore, cleaning up resources managed by it.
|
## Close the blockstore, cleaning up resources managed by it.
|
||||||
## For some implementations this may be a no-op
|
## For some implementations this may be a no-op
|
||||||
##
|
##
|
||||||
|
|
||||||
raiseAssert("Not implemented!")
|
raiseAssert("close not implemented!")
|
||||||
|
|
||||||
proc contains*(self: BlockStore, blk: Cid): Future[bool] {.async.} =
|
proc contains*(self: BlockStore, blk: Cid): Future[bool] {.async.} =
|
||||||
## Check if the block exists in the blockstore.
|
## Check if the block exists in the blockstore.
|
||||||
|
@ -99,3 +133,9 @@ proc contains*(self: BlockStore, blk: Cid): Future[bool] {.async.} =
|
||||||
##
|
##
|
||||||
|
|
||||||
return (await self.hasBlock(blk)) |? false
|
return (await self.hasBlock(blk)) |? false
|
||||||
|
|
||||||
|
proc contains*(self: BlockStore, address: BlockAddress): Future[bool] {.async.} =
|
||||||
|
return if address.leaf:
|
||||||
|
(await self.hasBlock(address.treeCid, address.index)) |? false
|
||||||
|
else:
|
||||||
|
(await self.hasBlock(address.cid)) |? false
|
||||||
|
|
|
@ -25,6 +25,8 @@ import ../units
|
||||||
import ../chunker
|
import ../chunker
|
||||||
import ../errors
|
import ../errors
|
||||||
import ../manifest
|
import ../manifest
|
||||||
|
import ../merkletree
|
||||||
|
import ../utils
|
||||||
import ../clock
|
import ../clock
|
||||||
|
|
||||||
export blockstore
|
export blockstore
|
||||||
|
@ -37,6 +39,7 @@ type
|
||||||
currentSize*: NBytes
|
currentSize*: NBytes
|
||||||
size*: NBytes
|
size*: NBytes
|
||||||
cache: LruCache[Cid, Block]
|
cache: LruCache[Cid, Block]
|
||||||
|
cidAndProofCache: LruCache[(Cid, Natural), (Cid, MerkleProof)]
|
||||||
|
|
||||||
InvalidBlockSize* = object of CodexError
|
InvalidBlockSize* = object of CodexError
|
||||||
|
|
||||||
|
@ -51,10 +54,10 @@ method getBlock*(self: CacheStore, cid: Cid): Future[?!Block] {.async.} =
|
||||||
|
|
||||||
if cid.isEmpty:
|
if cid.isEmpty:
|
||||||
trace "Empty block, ignoring"
|
trace "Empty block, ignoring"
|
||||||
return success cid.emptyBlock
|
return cid.emptyBlock
|
||||||
|
|
||||||
if cid notin self.cache:
|
if cid notin self.cache:
|
||||||
return failure (ref BlockNotFoundError)(msg: "Block not in cache")
|
return failure (ref BlockNotFoundError)(msg: "Block not in cache " & $cid)
|
||||||
|
|
||||||
try:
|
try:
|
||||||
return success self.cache[cid]
|
return success self.cache[cid]
|
||||||
|
@ -62,6 +65,35 @@ method getBlock*(self: CacheStore, cid: Cid): Future[?!Block] {.async.} =
|
||||||
trace "Error requesting block from cache", cid, error = exc.msg
|
trace "Error requesting block from cache", cid, error = exc.msg
|
||||||
return failure exc
|
return failure exc
|
||||||
|
|
||||||
|
proc getCidAndProof(self: CacheStore, treeCid: Cid, index: Natural): ?!(Cid, MerkleProof) =
|
||||||
|
if cidAndProof =? self.cidAndProofCache.getOption((treeCid, index)):
|
||||||
|
success(cidAndProof)
|
||||||
|
else:
|
||||||
|
failure(newException(BlockNotFoundError, "Block not in cache: " & $BlockAddress.init(treeCid, index)))
|
||||||
|
|
||||||
|
method getBlock*(self: CacheStore, treeCid: Cid, index: Natural): Future[?!Block] {.async.} =
|
||||||
|
without cidAndProof =? self.getCidAndProof(treeCid, index), err:
|
||||||
|
return failure(err)
|
||||||
|
|
||||||
|
await self.getBlock(cidAndProof[0])
|
||||||
|
|
||||||
|
method getBlockAndProof*(self: CacheStore, treeCid: Cid, index: Natural): Future[?!(Block, MerkleProof)] {.async.} =
|
||||||
|
without cidAndProof =? self.getCidAndProof(treeCid, index), err:
|
||||||
|
return failure(err)
|
||||||
|
|
||||||
|
let (cid, proof) = cidAndProof
|
||||||
|
|
||||||
|
without blk =? await self.getBlock(cid), err:
|
||||||
|
return failure(err)
|
||||||
|
|
||||||
|
success((blk, proof))
|
||||||
|
|
||||||
|
method getBlock*(self: CacheStore, address: BlockAddress): Future[?!Block] =
|
||||||
|
if address.leaf:
|
||||||
|
self.getBlock(address.treeCid, address.index)
|
||||||
|
else:
|
||||||
|
self.getBlock(address.cid)
|
||||||
|
|
||||||
method hasBlock*(self: CacheStore, cid: Cid): Future[?!bool] {.async.} =
|
method hasBlock*(self: CacheStore, cid: Cid): Future[?!bool] {.async.} =
|
||||||
## Check if the block exists in the blockstore
|
## Check if the block exists in the blockstore
|
||||||
##
|
##
|
||||||
|
@ -73,6 +105,16 @@ method hasBlock*(self: CacheStore, cid: Cid): Future[?!bool] {.async.} =
|
||||||
|
|
||||||
return (cid in self.cache).success
|
return (cid in self.cache).success
|
||||||
|
|
||||||
|
method hasBlock*(self: CacheStore, treeCid: Cid, index: Natural): Future[?!bool] {.async.} =
|
||||||
|
without cidAndProof =? self.getCidAndProof(treeCid, index), err:
|
||||||
|
if err of BlockNotFoundError:
|
||||||
|
return success(false)
|
||||||
|
else:
|
||||||
|
return failure(err)
|
||||||
|
|
||||||
|
await self.hasBlock(cidAndProof[0])
|
||||||
|
|
||||||
|
|
||||||
func cids(self: CacheStore): (iterator: Cid {.gcsafe.}) =
|
func cids(self: CacheStore): (iterator: Cid {.gcsafe.}) =
|
||||||
return iterator(): Cid =
|
return iterator(): Cid =
|
||||||
for cid in self.cache.keys:
|
for cid in self.cache.keys:
|
||||||
|
@ -81,12 +123,12 @@ func cids(self: CacheStore): (iterator: Cid {.gcsafe.}) =
|
||||||
method listBlocks*(
|
method listBlocks*(
|
||||||
self: CacheStore,
|
self: CacheStore,
|
||||||
blockType = BlockType.Manifest
|
blockType = BlockType.Manifest
|
||||||
): Future[?!BlocksIter] {.async.} =
|
): Future[?!AsyncIter[?Cid]] {.async.} =
|
||||||
## 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
|
||||||
##
|
##
|
||||||
|
|
||||||
var
|
var
|
||||||
iter = BlocksIter()
|
iter = AsyncIter[?Cid]()
|
||||||
|
|
||||||
let
|
let
|
||||||
cids = self.cids()
|
cids = self.cids()
|
||||||
|
@ -102,7 +144,7 @@ method listBlocks*(
|
||||||
cid = cids()
|
cid = cids()
|
||||||
|
|
||||||
if finished(cids):
|
if finished(cids):
|
||||||
iter.finished = true
|
iter.finish
|
||||||
return Cid.none
|
return Cid.none
|
||||||
|
|
||||||
without isManifest =? cid.isManifest, err:
|
without isManifest =? cid.isManifest, err:
|
||||||
|
@ -168,6 +210,16 @@ method putBlock*(
|
||||||
discard self.putBlockSync(blk)
|
discard self.putBlockSync(blk)
|
||||||
return success()
|
return success()
|
||||||
|
|
||||||
|
method putBlockCidAndProof*(
|
||||||
|
self: CacheStore,
|
||||||
|
treeCid: Cid,
|
||||||
|
index: Natural,
|
||||||
|
blockCid: Cid,
|
||||||
|
proof: MerkleProof
|
||||||
|
): Future[?!void] {.async.} =
|
||||||
|
self.cidAndProofCache[(treeCid, index)] = (blockCid, proof)
|
||||||
|
success()
|
||||||
|
|
||||||
method ensureExpiry*(
|
method ensureExpiry*(
|
||||||
self: CacheStore,
|
self: CacheStore,
|
||||||
cid: Cid,
|
cid: Cid,
|
||||||
|
@ -193,6 +245,14 @@ method delBlock*(self: CacheStore, cid: Cid): Future[?!void] {.async.} =
|
||||||
|
|
||||||
return success()
|
return success()
|
||||||
|
|
||||||
|
method delBlock*(self: CacheStore, treeCid: Cid, index: Natural): Future[?!void] {.async.} =
|
||||||
|
let maybeRemoved = self.cidAndProofCache.del((treeCid, index))
|
||||||
|
|
||||||
|
if removed =? maybeRemoved:
|
||||||
|
return await self.delBlock(removed[0])
|
||||||
|
|
||||||
|
return success()
|
||||||
|
|
||||||
method close*(self: CacheStore): Future[void] {.async.} =
|
method close*(self: CacheStore): Future[void] {.async.} =
|
||||||
## Close the blockstore, a no-op for this implementation
|
## Close the blockstore, a no-op for this implementation
|
||||||
##
|
##
|
||||||
|
@ -217,8 +277,10 @@ proc new*(
|
||||||
currentSize = 0'nb
|
currentSize = 0'nb
|
||||||
size = int(cacheSize div chunkSize)
|
size = int(cacheSize div chunkSize)
|
||||||
cache = newLruCache[Cid, Block](size)
|
cache = newLruCache[Cid, Block](size)
|
||||||
|
cidAndProofCache = newLruCache[(Cid, Natural), (Cid, MerkleProof)](size)
|
||||||
store = CacheStore(
|
store = CacheStore(
|
||||||
cache: cache,
|
cache: cache,
|
||||||
|
cidAndProofCache: cidAndProofCache,
|
||||||
currentSize: currentSize,
|
currentSize: currentSize,
|
||||||
size: cacheSize)
|
size: cacheSize)
|
||||||
|
|
||||||
|
|
|
@ -10,6 +10,7 @@
|
||||||
import pkg/upraises
|
import pkg/upraises
|
||||||
push: {.upraises: [].}
|
push: {.upraises: [].}
|
||||||
|
|
||||||
|
import std/sugar
|
||||||
import pkg/questionable/results
|
import pkg/questionable/results
|
||||||
import pkg/datastore
|
import pkg/datastore
|
||||||
import pkg/libp2p
|
import pkg/libp2p
|
||||||
|
@ -23,6 +24,7 @@ const
|
||||||
CodexTotalBlocksKey* = Key.init(CodexBlockTotalNamespace).tryGet
|
CodexTotalBlocksKey* = Key.init(CodexBlockTotalNamespace).tryGet
|
||||||
CodexManifestKey* = Key.init(CodexManifestNamespace).tryGet
|
CodexManifestKey* = Key.init(CodexManifestNamespace).tryGet
|
||||||
BlocksTtlKey* = Key.init(CodexBlocksTtlNamespace).tryGet
|
BlocksTtlKey* = Key.init(CodexBlocksTtlNamespace).tryGet
|
||||||
|
BlockProofKey* = Key.init(CodexBlockProofNamespace).tryGet
|
||||||
QuotaKey* = Key.init(CodexQuotaNamespace).tryGet
|
QuotaKey* = Key.init(CodexQuotaNamespace).tryGet
|
||||||
QuotaUsedKey* = (QuotaKey / "used").tryGet
|
QuotaUsedKey* = (QuotaKey / "used").tryGet
|
||||||
QuotaReservedKey* = (QuotaKey / "reserved").tryGet
|
QuotaReservedKey* = (QuotaKey / "reserved").tryGet
|
||||||
|
@ -42,3 +44,7 @@ proc createBlockExpirationMetadataKey*(cid: Cid): ?!Key =
|
||||||
proc createBlockExpirationMetadataQueryKey*(): ?!Key =
|
proc createBlockExpirationMetadataQueryKey*(): ?!Key =
|
||||||
let queryString = ? (BlocksTtlKey / "*")
|
let queryString = ? (BlocksTtlKey / "*")
|
||||||
Key.init(queryString)
|
Key.init(queryString)
|
||||||
|
|
||||||
|
proc createBlockCidAndProofMetadataKey*(treeCid: Cid, index: Natural): ?!Key =
|
||||||
|
(BlockProofKey / $treeCid).flatMap((k: Key) => k / $index)
|
||||||
|
|
|
@ -17,6 +17,7 @@ import pkg/questionable/results
|
||||||
|
|
||||||
import ./repostore
|
import ./repostore
|
||||||
import ../utils/timer
|
import ../utils/timer
|
||||||
|
import ../utils/asynciter
|
||||||
import ../clock
|
import ../clock
|
||||||
import ../systemclock
|
import ../systemclock
|
||||||
|
|
||||||
|
|
|
@ -11,45 +11,64 @@ import pkg/upraises
|
||||||
|
|
||||||
push: {.upraises: [].}
|
push: {.upraises: [].}
|
||||||
|
|
||||||
|
import std/sugar
|
||||||
|
|
||||||
import pkg/chronicles
|
import pkg/chronicles
|
||||||
import pkg/chronos
|
import pkg/chronos
|
||||||
import pkg/libp2p
|
import pkg/libp2p
|
||||||
|
|
||||||
import ../blocktype as bt
|
import ../blocktype
|
||||||
import ../utils/asyncheapqueue
|
import ../utils/asyncheapqueue
|
||||||
|
import ../utils/asynciter
|
||||||
import ../clock
|
import ../clock
|
||||||
|
|
||||||
import ./blockstore
|
import ./blockstore
|
||||||
import ../blockexchange
|
import ../blockexchange
|
||||||
|
import ../merkletree
|
||||||
|
import ../blocktype
|
||||||
|
|
||||||
export blockstore, blockexchange, asyncheapqueue
|
export blockstore, blockexchange, asyncheapqueue
|
||||||
|
|
||||||
logScope:
|
logScope:
|
||||||
topics = "codex networkstore"
|
topics = "codex networkstore"
|
||||||
|
|
||||||
|
const BlockPrefetchAmount = 5
|
||||||
|
|
||||||
type
|
type
|
||||||
NetworkStore* = ref object of BlockStore
|
NetworkStore* = ref object of BlockStore
|
||||||
engine*: BlockExcEngine # blockexc decision engine
|
engine*: BlockExcEngine # blockexc decision engine
|
||||||
localStore*: BlockStore # local block store
|
localStore*: BlockStore # local block store
|
||||||
|
|
||||||
method getBlock*(self: NetworkStore, cid: Cid): Future[?!bt.Block] {.async.} =
|
method getBlock*(self: NetworkStore, address: BlockAddress): Future[?!Block] {.async.} =
|
||||||
trace "Getting block from local store or network", cid
|
trace "Getting block from local store or network", address
|
||||||
|
|
||||||
without blk =? await self.localStore.getBlock(cid), error:
|
without blk =? await self.localStore.getBlock(address), error:
|
||||||
if not (error of BlockNotFoundError): return failure error
|
if not (error of BlockNotFoundError): return failure error
|
||||||
trace "Block not in local store", cid
|
trace "Block not in local store", address
|
||||||
|
|
||||||
without newBlock =? (await self.engine.requestBlock(cid)).catch, error:
|
without newBlock =? (await self.engine.requestBlock(address)).catch, error:
|
||||||
trace "Unable to get block from exchange engine", cid
|
trace "Unable to get block from exchange engine", address
|
||||||
return failure error
|
return failure error
|
||||||
|
|
||||||
return success newBlock
|
return success newBlock
|
||||||
|
|
||||||
return success blk
|
return success blk
|
||||||
|
|
||||||
|
method getBlock*(self: NetworkStore, cid: Cid): Future[?!Block] =
|
||||||
|
## Get a block from the blockstore
|
||||||
|
##
|
||||||
|
|
||||||
|
self.getBlock(BlockAddress.init(cid))
|
||||||
|
|
||||||
|
method getBlock*(self: NetworkStore, treeCid: Cid, index: Natural): Future[?!Block] =
|
||||||
|
## Get a block from the blockstore
|
||||||
|
##
|
||||||
|
|
||||||
|
self.getBlock(BlockAddress.init(treeCid, index))
|
||||||
|
|
||||||
method putBlock*(
|
method putBlock*(
|
||||||
self: NetworkStore,
|
self: NetworkStore,
|
||||||
blk: bt.Block,
|
blk: Block,
|
||||||
ttl = Duration.none
|
ttl = Duration.none
|
||||||
): Future[?!void] {.async.} =
|
): Future[?!void] {.async.} =
|
||||||
## Store block locally and notify the network
|
## Store block locally and notify the network
|
||||||
|
@ -64,6 +83,15 @@ method putBlock*(
|
||||||
await self.engine.resolveBlocks(@[blk])
|
await self.engine.resolveBlocks(@[blk])
|
||||||
return success()
|
return success()
|
||||||
|
|
||||||
|
method putBlockCidAndProof*(
|
||||||
|
self: NetworkStore,
|
||||||
|
treeCid: Cid,
|
||||||
|
index: Natural,
|
||||||
|
blockCid: Cid,
|
||||||
|
proof: MerkleProof
|
||||||
|
): Future[?!void] =
|
||||||
|
self.localStore.putBlockCidAndProof(treeCid, index, blockCid, proof)
|
||||||
|
|
||||||
method ensureExpiry*(
|
method ensureExpiry*(
|
||||||
self: NetworkStore,
|
self: NetworkStore,
|
||||||
cid: Cid,
|
cid: Cid,
|
||||||
|
@ -82,7 +110,7 @@ method ensureExpiry*(
|
||||||
|
|
||||||
method listBlocks*(
|
method listBlocks*(
|
||||||
self: NetworkStore,
|
self: NetworkStore,
|
||||||
blockType = BlockType.Manifest): Future[?!BlocksIter] =
|
blockType = BlockType.Manifest): Future[?!AsyncIter[?Cid]] =
|
||||||
self.localStore.listBlocks(blockType)
|
self.localStore.listBlocks(blockType)
|
||||||
|
|
||||||
method delBlock*(self: NetworkStore, cid: Cid): Future[?!void] =
|
method delBlock*(self: NetworkStore, cid: Cid): Future[?!void] =
|
||||||
|
|
|
@ -12,8 +12,10 @@ import pkg/upraises
|
||||||
push: {.upraises: [].}
|
push: {.upraises: [].}
|
||||||
|
|
||||||
import pkg/chronos
|
import pkg/chronos
|
||||||
|
import pkg/chronos/futures
|
||||||
import pkg/chronicles
|
import pkg/chronicles
|
||||||
import pkg/libp2p/cid
|
import pkg/libp2p/[cid, multicodec, multihash]
|
||||||
|
import pkg/lrucache
|
||||||
import pkg/metrics
|
import pkg/metrics
|
||||||
import pkg/questionable
|
import pkg/questionable
|
||||||
import pkg/questionable/results
|
import pkg/questionable/results
|
||||||
|
@ -25,6 +27,8 @@ import ./keyutils
|
||||||
import ../blocktype
|
import ../blocktype
|
||||||
import ../clock
|
import ../clock
|
||||||
import ../systemclock
|
import ../systemclock
|
||||||
|
import ../merkletree
|
||||||
|
import ../utils
|
||||||
|
|
||||||
export blocktype, cid
|
export blocktype, cid
|
||||||
|
|
||||||
|
@ -58,16 +62,7 @@ type
|
||||||
BlockExpiration* = object
|
BlockExpiration* = object
|
||||||
cid*: Cid
|
cid*: Cid
|
||||||
expiration*: SecondsSince1970
|
expiration*: SecondsSince1970
|
||||||
|
|
||||||
GetNext = proc(): Future[?BlockExpiration] {.upraises: [], gcsafe, closure.}
|
|
||||||
BlockExpirationIter* = ref object
|
|
||||||
finished*: bool
|
|
||||||
next*: GetNext
|
|
||||||
|
|
||||||
iterator items*(q: BlockExpirationIter): Future[?BlockExpiration] =
|
|
||||||
while not q.finished:
|
|
||||||
yield q.next()
|
|
||||||
|
|
||||||
proc updateMetrics(self: RepoStore) =
|
proc updateMetrics(self: RepoStore) =
|
||||||
codex_repostore_blocks.set(self.totalBlocks.int64)
|
codex_repostore_blocks.set(self.totalBlocks.int64)
|
||||||
codex_repostore_bytes_used.set(self.quotaUsedBytes.int64)
|
codex_repostore_bytes_used.set(self.quotaUsedBytes.int64)
|
||||||
|
@ -82,6 +77,63 @@ func available*(self: RepoStore): uint =
|
||||||
func available*(self: RepoStore, bytes: uint): bool =
|
func available*(self: RepoStore, bytes: uint): bool =
|
||||||
return bytes < self.available()
|
return bytes < self.available()
|
||||||
|
|
||||||
|
proc encode(cidAndProof: (Cid, MerkleProof)): seq[byte] =
|
||||||
|
## Encodes a tuple of cid and merkle proof in a following format:
|
||||||
|
## | 8-bytes | n-bytes | remaining bytes |
|
||||||
|
## | n | cid | proof |
|
||||||
|
##
|
||||||
|
## where n is a size of cid
|
||||||
|
##
|
||||||
|
let
|
||||||
|
(cid, proof) = cidAndProof
|
||||||
|
cidBytes = cid.data.buffer
|
||||||
|
proofBytes = proof.encode
|
||||||
|
n = cidBytes.len
|
||||||
|
nBytes = n.uint64.toBytesBE
|
||||||
|
|
||||||
|
@nBytes & cidBytes & proofBytes
|
||||||
|
|
||||||
|
proc decode(_: type (Cid, MerkleProof), data: seq[byte]): ?!(Cid, MerkleProof) =
|
||||||
|
let
|
||||||
|
n = uint64.fromBytesBE(data[0..<sizeof(uint64)]).int
|
||||||
|
let
|
||||||
|
cid = ? Cid.init(data[sizeof(uint64)..<sizeof(uint64) + n]).mapFailure
|
||||||
|
proof = ? MerkleProof.decode(data[sizeof(uint64) + n..^1])
|
||||||
|
success((cid, proof))
|
||||||
|
|
||||||
|
method putBlockCidAndProof*(
|
||||||
|
self: RepoStore,
|
||||||
|
treeCid: Cid,
|
||||||
|
index: Natural,
|
||||||
|
blockCid: Cid,
|
||||||
|
proof: MerkleProof
|
||||||
|
): Future[?!void] {.async.} =
|
||||||
|
## Put a block to the blockstore
|
||||||
|
##
|
||||||
|
|
||||||
|
without key =? createBlockCidAndProofMetadataKey(treeCid, index), err:
|
||||||
|
return failure(err)
|
||||||
|
|
||||||
|
let value = (blockCid, proof).encode()
|
||||||
|
|
||||||
|
await self.metaDs.put(key, value)
|
||||||
|
|
||||||
|
proc getCidAndProof(
|
||||||
|
self: RepoStore,
|
||||||
|
treeCid: Cid,
|
||||||
|
index: Natural
|
||||||
|
): Future[?!(Cid, MerkleProof)] {.async.} =
|
||||||
|
without key =? createBlockCidAndProofMetadataKey(treeCid, index), err:
|
||||||
|
return failure(err)
|
||||||
|
|
||||||
|
without value =? await self.metaDs.get(key), err:
|
||||||
|
if err of DatastoreKeyNotFound:
|
||||||
|
return failure(newException(BlockNotFoundError, err.msg))
|
||||||
|
else:
|
||||||
|
return failure(err)
|
||||||
|
|
||||||
|
return (Cid, MerkleProof).decode(value)
|
||||||
|
|
||||||
method getBlock*(self: RepoStore, cid: Cid): Future[?!Block] {.async.} =
|
method getBlock*(self: RepoStore, cid: Cid): Future[?!Block] {.async.} =
|
||||||
## Get a block from the blockstore
|
## Get a block from the blockstore
|
||||||
##
|
##
|
||||||
|
@ -91,7 +143,7 @@ method getBlock*(self: RepoStore, cid: Cid): Future[?!Block] {.async.} =
|
||||||
|
|
||||||
if cid.isEmpty:
|
if cid.isEmpty:
|
||||||
trace "Empty block, ignoring"
|
trace "Empty block, ignoring"
|
||||||
return success cid.emptyBlock
|
return cid.emptyBlock
|
||||||
|
|
||||||
without key =? makePrefixKey(self.postFixLen, cid), err:
|
without key =? makePrefixKey(self.postFixLen, cid), err:
|
||||||
trace "Error getting key from provider", err = err.msg
|
trace "Error getting key from provider", err = err.msg
|
||||||
|
@ -105,7 +157,34 @@ method getBlock*(self: RepoStore, cid: Cid): Future[?!Block] {.async.} =
|
||||||
return failure(newException(BlockNotFoundError, err.msg))
|
return failure(newException(BlockNotFoundError, err.msg))
|
||||||
|
|
||||||
trace "Got block for cid", cid
|
trace "Got block for cid", cid
|
||||||
return Block.new(cid, data)
|
return Block.new(cid, data, verify = true)
|
||||||
|
|
||||||
|
|
||||||
|
method getBlockAndProof*(self: RepoStore, treeCid: Cid, index: Natural): Future[?!(Block, MerkleProof)] {.async.} =
|
||||||
|
without cidAndProof =? await self.getCidAndProof(treeCid, index), err:
|
||||||
|
return failure(err)
|
||||||
|
|
||||||
|
let (cid, proof) = cidAndProof
|
||||||
|
|
||||||
|
without blk =? await self.getBlock(cid), err:
|
||||||
|
return failure(err)
|
||||||
|
|
||||||
|
success((blk, proof))
|
||||||
|
|
||||||
|
method getBlock*(self: RepoStore, treeCid: Cid, index: Natural): Future[?!Block] {.async.} =
|
||||||
|
without cidAndProof =? await self.getCidAndProof(treeCid, index), err:
|
||||||
|
return failure(err)
|
||||||
|
|
||||||
|
await self.getBlock(cidAndProof[0])
|
||||||
|
|
||||||
|
method getBlock*(self: RepoStore, address: BlockAddress): Future[?!Block] =
|
||||||
|
## Get a block from the blockstore
|
||||||
|
##
|
||||||
|
|
||||||
|
if address.leaf:
|
||||||
|
self.getBlock(address.treeCid, address.index)
|
||||||
|
else:
|
||||||
|
self.getBlock(address.cid)
|
||||||
|
|
||||||
proc getBlockExpirationEntry(
|
proc getBlockExpirationEntry(
|
||||||
self: RepoStore,
|
self: RepoStore,
|
||||||
|
@ -291,6 +370,23 @@ method delBlock*(self: RepoStore, cid: Cid): Future[?!void] {.async.} =
|
||||||
self.updateMetrics()
|
self.updateMetrics()
|
||||||
return success()
|
return success()
|
||||||
|
|
||||||
|
method delBlock*(self: RepoStore, treeCid: Cid, index: Natural): Future[?!void] {.async.} =
|
||||||
|
without key =? createBlockCidAndProofMetadataKey(treeCid, index), err:
|
||||||
|
return failure(err)
|
||||||
|
|
||||||
|
without value =? await self.metaDs.get(key), err:
|
||||||
|
if err of DatastoreKeyNotFound:
|
||||||
|
return success()
|
||||||
|
else:
|
||||||
|
return failure(err)
|
||||||
|
|
||||||
|
without cidAndProof =? (Cid, MerkleProof).decode(value), err:
|
||||||
|
return failure(err)
|
||||||
|
|
||||||
|
self.delBlock(cidAndProof[0])
|
||||||
|
|
||||||
|
await self.metaDs.delete(key)
|
||||||
|
|
||||||
method hasBlock*(self: RepoStore, cid: Cid): Future[?!bool] {.async.} =
|
method hasBlock*(self: RepoStore, cid: Cid): Future[?!bool] {.async.} =
|
||||||
## Check if the block exists in the blockstore
|
## Check if the block exists in the blockstore
|
||||||
##
|
##
|
||||||
|
@ -308,15 +404,25 @@ method hasBlock*(self: RepoStore, cid: Cid): Future[?!bool] {.async.} =
|
||||||
|
|
||||||
return await self.repoDs.has(key)
|
return await self.repoDs.has(key)
|
||||||
|
|
||||||
|
method hasBlock*(self: RepoStore, treeCid: Cid, index: Natural): Future[?!bool] {.async.} =
|
||||||
|
without cidAndProof =? await self.getCidAndProof(treeCid, index), err:
|
||||||
|
if err of BlockNotFoundError:
|
||||||
|
return success(false)
|
||||||
|
else:
|
||||||
|
return failure(err)
|
||||||
|
|
||||||
|
await self.hasBlock(cidAndProof[0])
|
||||||
|
|
||||||
method listBlocks*(
|
method listBlocks*(
|
||||||
self: RepoStore,
|
self: RepoStore,
|
||||||
blockType = BlockType.Manifest): Future[?!BlocksIter] {.async.} =
|
blockType = BlockType.Manifest
|
||||||
|
): Future[?!AsyncIter[?Cid]] {.async.} =
|
||||||
## Get the list of blocks in the RepoStore.
|
## Get the list of blocks in the RepoStore.
|
||||||
## This is an intensive operation
|
## This is an intensive operation
|
||||||
##
|
##
|
||||||
|
|
||||||
var
|
var
|
||||||
iter = BlocksIter()
|
iter = AsyncIter[?Cid]()
|
||||||
|
|
||||||
let key =
|
let key =
|
||||||
case blockType:
|
case blockType:
|
||||||
|
@ -331,14 +437,15 @@ method listBlocks*(
|
||||||
|
|
||||||
proc next(): Future[?Cid] {.async.} =
|
proc next(): Future[?Cid] {.async.} =
|
||||||
await idleAsync()
|
await idleAsync()
|
||||||
iter.finished = queryIter.finished
|
if queryIter.finished:
|
||||||
if not queryIter.finished:
|
iter.finish
|
||||||
|
else:
|
||||||
if pair =? (await queryIter.next()) and cid =? pair.key:
|
if pair =? (await queryIter.next()) and cid =? pair.key:
|
||||||
doAssert pair.data.len == 0
|
doAssert pair.data.len == 0
|
||||||
trace "Retrieved record from repo", cid
|
trace "Retrieved record from repo", cid
|
||||||
return Cid.init(cid.value).option
|
return Cid.init(cid.value).option
|
||||||
|
else:
|
||||||
return Cid.none
|
return Cid.none
|
||||||
|
|
||||||
iter.next = next
|
iter.next = next
|
||||||
return success iter
|
return success iter
|
||||||
|
@ -350,7 +457,7 @@ proc createBlockExpirationQuery(maxNumber: int, offset: int): ?!Query =
|
||||||
method getBlockExpirations*(
|
method getBlockExpirations*(
|
||||||
self: RepoStore,
|
self: RepoStore,
|
||||||
maxNumber: int,
|
maxNumber: int,
|
||||||
offset: int): Future[?!BlockExpirationIter] {.async, base.} =
|
offset: int): Future[?!AsyncIter[?BlockExpiration]] {.async, base.} =
|
||||||
## Get block expirations from the given RepoStore
|
## Get block expirations from the given RepoStore
|
||||||
##
|
##
|
||||||
|
|
||||||
|
@ -362,7 +469,7 @@ method getBlockExpirations*(
|
||||||
trace "Unable to execute block expirations query"
|
trace "Unable to execute block expirations query"
|
||||||
return failure(err)
|
return failure(err)
|
||||||
|
|
||||||
var iter = BlockExpirationIter()
|
var iter = AsyncIter[?BlockExpiration]()
|
||||||
|
|
||||||
proc next(): Future[?BlockExpiration] {.async.} =
|
proc next(): Future[?BlockExpiration] {.async.} =
|
||||||
if not queryIter.finished:
|
if not queryIter.finished:
|
||||||
|
@ -377,7 +484,7 @@ method getBlockExpirations*(
|
||||||
).some
|
).some
|
||||||
else:
|
else:
|
||||||
discard await queryIter.dispose()
|
discard await queryIter.dispose()
|
||||||
iter.finished = true
|
iter.finish
|
||||||
return BlockExpiration.none
|
return BlockExpiration.none
|
||||||
|
|
||||||
iter.next = next
|
iter.next = next
|
||||||
|
@ -519,4 +626,5 @@ func new*(
|
||||||
clock: clock,
|
clock: clock,
|
||||||
postFixLen: postFixLen,
|
postFixLen: postFixLen,
|
||||||
quotaMaxBytes: quotaMaxBytes,
|
quotaMaxBytes: quotaMaxBytes,
|
||||||
blockTtl: blockTtl)
|
blockTtl: blockTtl
|
||||||
|
)
|
||||||
|
|
|
@ -0,0 +1,52 @@
|
||||||
|
## Nim-Codex
|
||||||
|
## Copyright (c) 2023 Status Research & Development GmbH
|
||||||
|
## Licensed under either of
|
||||||
|
## * Apache License, version 2.0, ([LICENSE-APACHE](LICENSE-APACHE))
|
||||||
|
## * MIT license ([LICENSE-MIT](LICENSE-MIT))
|
||||||
|
## at your option.
|
||||||
|
## This file may not be copied, modified, or distributed except according to
|
||||||
|
## those terms.
|
||||||
|
|
||||||
|
import pkg/upraises
|
||||||
|
|
||||||
|
push: {.upraises: [].}
|
||||||
|
|
||||||
|
import std/sugar
|
||||||
|
import pkg/chronos
|
||||||
|
import pkg/chronos/futures
|
||||||
|
import pkg/metrics
|
||||||
|
import pkg/questionable
|
||||||
|
import pkg/questionable/results
|
||||||
|
|
||||||
|
import ./blockstore
|
||||||
|
import ../utils/asynciter
|
||||||
|
import ../merkletree
|
||||||
|
|
||||||
|
proc putSomeProofs*(store: BlockStore, tree: MerkleTree, iter: Iter[int]): Future[?!void] {.async.} =
|
||||||
|
without treeCid =? tree.rootCid, err:
|
||||||
|
return failure(err)
|
||||||
|
|
||||||
|
for i in iter:
|
||||||
|
if i notin 0..<tree.leavesCount:
|
||||||
|
return failure("Invalid leaf index " & $i & ", tree with cid " & $treeCid & " has " & $tree.leavesCount & " leaves")
|
||||||
|
|
||||||
|
without blkCid =? tree.getLeafCid(i), err:
|
||||||
|
return failure(err)
|
||||||
|
|
||||||
|
without proof =? tree.getProof(i), err:
|
||||||
|
return failure(err)
|
||||||
|
|
||||||
|
let res = await store.putBlockCidAndProof(treeCid, i, blkCid, proof)
|
||||||
|
|
||||||
|
if err =? res.errorOption:
|
||||||
|
return failure(err)
|
||||||
|
|
||||||
|
success()
|
||||||
|
|
||||||
|
proc putSomeProofs*(store: BlockStore, tree: MerkleTree, iter: Iter[Natural]): Future[?!void] =
|
||||||
|
store.putSomeProofs(tree, iter.map((i: Natural) => i.ord))
|
||||||
|
|
||||||
|
proc putAllProofs*(store: BlockStore, tree: MerkleTree): Future[?!void] =
|
||||||
|
store.putSomeProofs(tree, Iter.fromSlice(0..<tree.leavesCount))
|
||||||
|
|
||||||
|
|
|
@ -1,5 +1,5 @@
|
||||||
## Nim-Dagger
|
## Nim-Codex
|
||||||
## Copyright (c) 2022 Status Research & Development GmbH
|
## Copyright (c) 2023 Status Research & Development GmbH
|
||||||
## Licensed under either of
|
## Licensed under either of
|
||||||
## * Apache License, version 2.0, ([LICENSE-APACHE](LICENSE-APACHE))
|
## * Apache License, version 2.0, ([LICENSE-APACHE](LICENSE-APACHE))
|
||||||
## * MIT license ([LICENSE-MIT](LICENSE-MIT))
|
## * MIT license ([LICENSE-MIT](LICENSE-MIT))
|
||||||
|
@ -20,6 +20,7 @@ import pkg/stew/ptrops
|
||||||
import ../stores
|
import ../stores
|
||||||
import ../manifest
|
import ../manifest
|
||||||
import ../blocktype
|
import ../blocktype
|
||||||
|
import ../utils
|
||||||
|
|
||||||
import ./seekablestream
|
import ./seekablestream
|
||||||
|
|
||||||
|
@ -52,7 +53,7 @@ proc new*(
|
||||||
pad = true
|
pad = true
|
||||||
): StoreStream =
|
): StoreStream =
|
||||||
## Create a new StoreStream instance for a given store and manifest
|
## Create a new StoreStream instance for a given store and manifest
|
||||||
##
|
##
|
||||||
result = StoreStream(
|
result = StoreStream(
|
||||||
store: store,
|
store: store,
|
||||||
manifest: manifest,
|
manifest: manifest,
|
||||||
|
@ -79,9 +80,9 @@ method readOnce*(
|
||||||
## Read `nbytes` from current position in the StoreStream into output buffer pointed by `pbytes`.
|
## 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.
|
## Return how many bytes were actually read before EOF was encountered.
|
||||||
## Raise exception if we are already at EOF.
|
## Raise exception if we are already at EOF.
|
||||||
##
|
##
|
||||||
|
|
||||||
trace "Reading from manifest", cid = self.manifest.cid.get(), blocks = self.manifest.len
|
trace "Reading from manifest", cid = self.manifest.cid.get(), blocks = self.manifest.blocksCount
|
||||||
if self.atEof:
|
if self.atEof:
|
||||||
raise newLPStreamEOFError()
|
raise newLPStreamEOFError()
|
||||||
|
|
||||||
|
@ -97,9 +98,10 @@ method readOnce*(
|
||||||
readBytes = min([self.size - self.offset,
|
readBytes = min([self.size - self.offset,
|
||||||
nbytes - read,
|
nbytes - read,
|
||||||
self.manifest.blockSize.int - blockOffset])
|
self.manifest.blockSize.int - blockOffset])
|
||||||
|
address = BlockAddress(leaf: true, treeCid: self.manifest.treeCid, index: blockNum)
|
||||||
|
|
||||||
# Read contents of block `blockNum`
|
# Read contents of block `blockNum`
|
||||||
without blk =? await self.store.getBlock(self.manifest[blockNum]), error:
|
without blk =? await self.store.getBlock(address), error:
|
||||||
raise newLPStreamReadError(error)
|
raise newLPStreamReadError(error)
|
||||||
|
|
||||||
trace "Reading bytes from store stream", blockNum, cid = blk.cid, bytes = readBytes, blockOffset
|
trace "Reading bytes from store stream", blockNum, cid = blk.cid, bytes = readBytes, blockOffset
|
||||||
|
|
|
@ -9,13 +9,15 @@
|
||||||
##
|
##
|
||||||
|
|
||||||
import std/parseutils
|
import std/parseutils
|
||||||
|
import std/options
|
||||||
|
|
||||||
import pkg/chronos
|
import pkg/chronos
|
||||||
|
|
||||||
import ./utils/asyncheapqueue
|
import ./utils/asyncheapqueue
|
||||||
import ./utils/fileutils
|
import ./utils/fileutils
|
||||||
|
import ./utils/asynciter
|
||||||
|
|
||||||
export asyncheapqueue, fileutils
|
export asyncheapqueue, fileutils, asynciter
|
||||||
|
|
||||||
|
|
||||||
func divUp*[T: SomeInteger](a, b : T): T =
|
func divUp*[T: SomeInteger](a, b : T): T =
|
||||||
|
@ -27,6 +29,13 @@ func roundUp*[T](a, b : T): T =
|
||||||
## Round up 'a' to the next value divisible by 'b'
|
## Round up 'a' to the next value divisible by 'b'
|
||||||
divUp(a,b) * b
|
divUp(a,b) * b
|
||||||
|
|
||||||
|
proc orElse*[A](a, b: Option[A]): Option[A] =
|
||||||
|
if (a.isSome()):
|
||||||
|
a
|
||||||
|
else:
|
||||||
|
b
|
||||||
|
|
||||||
|
|
||||||
when not declared(parseDuration): # Odd code formatting to minimize diff v. mainLine
|
when not declared(parseDuration): # Odd code formatting to minimize diff v. mainLine
|
||||||
const Whitespace = {' ', '\t', '\v', '\r', '\l', '\f'}
|
const Whitespace = {' ', '\t', '\v', '\r', '\l', '\f'}
|
||||||
|
|
||||||
|
|
|
@ -0,0 +1,148 @@
|
||||||
|
import std/sugar
|
||||||
|
|
||||||
|
import pkg/questionable
|
||||||
|
import pkg/chronos
|
||||||
|
import pkg/upraises
|
||||||
|
|
||||||
|
type
|
||||||
|
Function*[T, U] = proc(fut: T): U {.upraises: [CatchableError], gcsafe, closure.}
|
||||||
|
IsFinished* = proc(): bool {.upraises: [], gcsafe, closure.}
|
||||||
|
GenNext*[T] = proc(): T {.upraises: [CatchableError], gcsafe, closure.}
|
||||||
|
Iter*[T] = ref object
|
||||||
|
finished: bool
|
||||||
|
next*: GenNext[T]
|
||||||
|
AsyncIter*[T] = Iter[Future[T]]
|
||||||
|
|
||||||
|
proc finish*[T](self: Iter[T]): void =
|
||||||
|
self.finished = true
|
||||||
|
|
||||||
|
proc finished*[T](self: Iter[T]): bool =
|
||||||
|
self.finished
|
||||||
|
|
||||||
|
iterator items*[T](self: Iter[T]): T =
|
||||||
|
while not self.finished:
|
||||||
|
yield self.next()
|
||||||
|
|
||||||
|
iterator pairs*[T](self: Iter[T]): tuple[key: int, val: T] {.inline.} =
|
||||||
|
var i = 0
|
||||||
|
while not self.finished:
|
||||||
|
yield (i, self.next())
|
||||||
|
inc(i)
|
||||||
|
|
||||||
|
proc map*[T, U](fut: Future[T], fn: Function[T, U]): Future[U] {.async.} =
|
||||||
|
let t = await fut
|
||||||
|
fn(t)
|
||||||
|
|
||||||
|
proc new*[T](_: type Iter, genNext: GenNext[T], isFinished: IsFinished, finishOnErr: bool = true): Iter[T] =
|
||||||
|
var iter = Iter[T]()
|
||||||
|
|
||||||
|
proc next(): T {.upraises: [CatchableError].} =
|
||||||
|
if not iter.finished:
|
||||||
|
var item: T
|
||||||
|
try:
|
||||||
|
item = genNext()
|
||||||
|
except CatchableError as err:
|
||||||
|
if finishOnErr or isFinished():
|
||||||
|
iter.finish
|
||||||
|
raise err
|
||||||
|
|
||||||
|
if isFinished():
|
||||||
|
iter.finish
|
||||||
|
return item
|
||||||
|
else:
|
||||||
|
raise newException(CatchableError, "Iterator is finished but next item was requested")
|
||||||
|
|
||||||
|
if isFinished():
|
||||||
|
iter.finish
|
||||||
|
|
||||||
|
iter.next = next
|
||||||
|
return iter
|
||||||
|
|
||||||
|
proc fromItems*[T](_: type Iter, items: seq[T]): Iter[T] =
|
||||||
|
## Create new iterator from items
|
||||||
|
##
|
||||||
|
|
||||||
|
Iter.fromSlice(0..<items.len)
|
||||||
|
.map((i: int) => items[i])
|
||||||
|
|
||||||
|
proc fromSlice*[U, V: Ordinal](_: type Iter, slice: HSlice[U, V]): Iter[U] =
|
||||||
|
## Creates new iterator from slice
|
||||||
|
##
|
||||||
|
|
||||||
|
Iter.fromRange(slice.a.int, slice.b.int, 1)
|
||||||
|
|
||||||
|
proc fromRange*[U, V, S: Ordinal](_: type Iter, a: U, b: V, step: S = 1): Iter[U] =
|
||||||
|
## Creates new iterator in range a..b with specified step (default 1)
|
||||||
|
##
|
||||||
|
|
||||||
|
var i = a
|
||||||
|
|
||||||
|
proc genNext(): U =
|
||||||
|
let u = i
|
||||||
|
inc(i, step)
|
||||||
|
u
|
||||||
|
|
||||||
|
proc isFinished(): bool =
|
||||||
|
(step > 0 and i > b) or
|
||||||
|
(step < 0 and i < b)
|
||||||
|
|
||||||
|
Iter.new(genNext, isFinished)
|
||||||
|
|
||||||
|
proc map*[T, U](iter: Iter[T], fn: Function[T, U]): Iter[U] =
|
||||||
|
Iter.new(
|
||||||
|
genNext = () => fn(iter.next()),
|
||||||
|
isFinished = () => iter.finished
|
||||||
|
)
|
||||||
|
|
||||||
|
proc filter*[T](iter: Iter[T], predicate: Function[T, bool]): Iter[T] =
|
||||||
|
var nextT: Option[T]
|
||||||
|
|
||||||
|
proc tryFetch(): void =
|
||||||
|
nextT = T.none
|
||||||
|
while not iter.finished:
|
||||||
|
let t = iter.next()
|
||||||
|
if predicate(t):
|
||||||
|
nextT = some(t)
|
||||||
|
break
|
||||||
|
|
||||||
|
proc genNext(): T =
|
||||||
|
let t = nextT.unsafeGet
|
||||||
|
tryFetch()
|
||||||
|
return t
|
||||||
|
|
||||||
|
proc isFinished(): bool =
|
||||||
|
nextT.isNone
|
||||||
|
|
||||||
|
tryFetch()
|
||||||
|
Iter.new(genNext, isFinished)
|
||||||
|
|
||||||
|
proc prefetch*[T](iter: Iter[T], n: Positive): Iter[T] =
|
||||||
|
var ringBuf = newSeq[T](n)
|
||||||
|
var iterLen = int.high
|
||||||
|
var i = 0
|
||||||
|
|
||||||
|
proc tryFetch(j: int): void =
|
||||||
|
if not iter.finished:
|
||||||
|
let item = iter.next()
|
||||||
|
ringBuf[j mod n] = item
|
||||||
|
if iter.finished:
|
||||||
|
iterLen = min(j + 1, iterLen)
|
||||||
|
else:
|
||||||
|
if j == 0:
|
||||||
|
iterLen = 0
|
||||||
|
|
||||||
|
proc genNext(): T =
|
||||||
|
let item = ringBuf[i mod n]
|
||||||
|
tryFetch(i + n)
|
||||||
|
inc i
|
||||||
|
return item
|
||||||
|
|
||||||
|
proc isFinished(): bool =
|
||||||
|
i >= iterLen
|
||||||
|
|
||||||
|
# initialize ringBuf with n prefetched values
|
||||||
|
for j in 0..<n:
|
||||||
|
tryFetch(j)
|
||||||
|
|
||||||
|
Iter.new(genNext, isFinished)
|
||||||
|
|
|
@ -12,6 +12,7 @@ import pkg/codex/stores
|
||||||
import pkg/codex/blockexchange
|
import pkg/codex/blockexchange
|
||||||
import pkg/codex/chunker
|
import pkg/codex/chunker
|
||||||
import pkg/codex/manifest
|
import pkg/codex/manifest
|
||||||
|
import pkg/codex/merkletree
|
||||||
import pkg/codex/blocktype as bt
|
import pkg/codex/blocktype as bt
|
||||||
|
|
||||||
import ../../helpers/mockdiscovery
|
import ../../helpers/mockdiscovery
|
||||||
|
@ -25,6 +26,7 @@ asyncchecksuite "Block Advertising and Discovery":
|
||||||
var
|
var
|
||||||
blocks: seq[bt.Block]
|
blocks: seq[bt.Block]
|
||||||
manifest: Manifest
|
manifest: Manifest
|
||||||
|
tree: MerkleTree
|
||||||
manifestBlock: bt.Block
|
manifestBlock: bt.Block
|
||||||
switch: Switch
|
switch: Switch
|
||||||
peerStore: PeerCtxStore
|
peerStore: PeerCtxStore
|
||||||
|
@ -52,7 +54,7 @@ asyncchecksuite "Block Advertising and Discovery":
|
||||||
peerStore = PeerCtxStore.new()
|
peerStore = PeerCtxStore.new()
|
||||||
pendingBlocks = PendingBlocksManager.new()
|
pendingBlocks = PendingBlocksManager.new()
|
||||||
|
|
||||||
manifest = Manifest.new( blocks.mapIt( it.cid ) ).tryGet()
|
(manifest, tree) = makeManifestAndTree(blocks).tryGet()
|
||||||
manifestBlock = bt.Block.new(
|
manifestBlock = bt.Block.new(
|
||||||
manifest.encode().tryGet(), codec = DagPBCodec).tryGet()
|
manifest.encode().tryGet(), codec = DagPBCodec).tryGet()
|
||||||
|
|
||||||
|
@ -151,7 +153,7 @@ asyncchecksuite "Block Advertising and Discovery":
|
||||||
peerId = PeerId.example
|
peerId = PeerId.example
|
||||||
haves = collect(initTable()):
|
haves = collect(initTable()):
|
||||||
for blk in blocks:
|
for blk in blocks:
|
||||||
{ blk.cid: Presence(cid: blk.cid, price: 0.u256) }
|
{ blk.address: Presence(address: blk.address, price: 0.u256) }
|
||||||
|
|
||||||
engine.peers.add(
|
engine.peers.add(
|
||||||
BlockExcPeerCtx(
|
BlockExcPeerCtx(
|
||||||
|
@ -164,7 +166,7 @@ asyncchecksuite "Block Advertising and Discovery":
|
||||||
check false
|
check false
|
||||||
|
|
||||||
await engine.start() # fire up discovery loop
|
await engine.start() # fire up discovery loop
|
||||||
engine.pendingBlocks.resolve(blocks)
|
engine.pendingBlocks.resolve(blocks.mapIt(BlockDelivery(blk: it, address: it.address)))
|
||||||
|
|
||||||
await allFuturesThrowing(
|
await allFuturesThrowing(
|
||||||
allFinished(pendingBlocks))
|
allFinished(pendingBlocks))
|
||||||
|
@ -240,9 +242,14 @@ asyncchecksuite "E2E - Multiple Nodes Discovery":
|
||||||
.publishBlockProvideHandler = proc(d: MockDiscovery, cid: Cid): Future[void] {.async.} =
|
.publishBlockProvideHandler = proc(d: MockDiscovery, cid: Cid): Future[void] {.async.} =
|
||||||
advertised[cid] = switch[3].peerInfo.signedPeerRecord
|
advertised[cid] = switch[3].peerInfo.signedPeerRecord
|
||||||
|
|
||||||
await blockexc[1].engine.blocksHandler(switch[0].peerInfo.peerId, blocks[0..5])
|
discard blocks[0..5].mapIt(blockexc[1].engine.pendingBlocks.getWantHandle(it.address))
|
||||||
await blockexc[2].engine.blocksHandler(switch[0].peerInfo.peerId, blocks[4..10])
|
await blockexc[1].engine.blocksDeliveryHandler(switch[0].peerInfo.peerId, blocks[0..5].mapIt(BlockDelivery(blk: it, address: it.address)))
|
||||||
await blockexc[3].engine.blocksHandler(switch[0].peerInfo.peerId, blocks[10..15])
|
|
||||||
|
discard blocks[4..10].mapIt(blockexc[2].engine.pendingBlocks.getWantHandle(it.address))
|
||||||
|
await blockexc[2].engine.blocksDeliveryHandler(switch[0].peerInfo.peerId, blocks[4..10].mapIt(BlockDelivery(blk: it, address: it.address)))
|
||||||
|
|
||||||
|
discard blocks[10..15].mapIt(blockexc[3].engine.pendingBlocks.getWantHandle(it.address))
|
||||||
|
await blockexc[3].engine.blocksDeliveryHandler(switch[0].peerInfo.peerId, blocks[10..15].mapIt(BlockDelivery(blk: it, address: it.address)))
|
||||||
|
|
||||||
MockDiscovery(blockexc[0].engine.discovery.discovery)
|
MockDiscovery(blockexc[0].engine.discovery.discovery)
|
||||||
.findBlockProvidersHandler = proc(d: MockDiscovery, cid: Cid):
|
.findBlockProvidersHandler = proc(d: MockDiscovery, cid: Cid):
|
||||||
|
@ -256,13 +263,13 @@ asyncchecksuite "E2E - Multiple Nodes Discovery":
|
||||||
|
|
||||||
await allFuturesThrowing(
|
await allFuturesThrowing(
|
||||||
switch.mapIt( it.start() ) &
|
switch.mapIt( it.start() ) &
|
||||||
blockexc.mapIt( it.engine.start() ))
|
blockexc.mapIt( it.engine.start() )).wait(10.seconds)
|
||||||
|
|
||||||
await allFutures(futs)
|
await allFutures(futs).wait(10.seconds)
|
||||||
|
|
||||||
await allFuturesThrowing(
|
await allFuturesThrowing(
|
||||||
blockexc.mapIt( it.engine.stop() ) &
|
blockexc.mapIt( it.engine.stop() ) &
|
||||||
switch.mapIt( it.stop() ))
|
switch.mapIt( it.stop() )).wait(10.seconds)
|
||||||
|
|
||||||
test "E2E - Should advertise and discover blocks with peers already connected":
|
test "E2E - Should advertise and discover blocks with peers already connected":
|
||||||
# Distribute the blocks amongst 1..3
|
# Distribute the blocks amongst 1..3
|
||||||
|
@ -282,9 +289,14 @@ asyncchecksuite "E2E - Multiple Nodes Discovery":
|
||||||
.publishBlockProvideHandler = proc(d: MockDiscovery, cid: Cid): Future[void] {.async.} =
|
.publishBlockProvideHandler = proc(d: MockDiscovery, cid: Cid): Future[void] {.async.} =
|
||||||
advertised[cid] = switch[3].peerInfo.signedPeerRecord
|
advertised[cid] = switch[3].peerInfo.signedPeerRecord
|
||||||
|
|
||||||
await blockexc[1].engine.blocksHandler(switch[0].peerInfo.peerId, blocks[0..5])
|
discard blocks[0..5].mapIt(blockexc[1].engine.pendingBlocks.getWantHandle(it.address))
|
||||||
await blockexc[2].engine.blocksHandler(switch[0].peerInfo.peerId, blocks[4..10])
|
await blockexc[1].engine.blocksDeliveryHandler(switch[0].peerInfo.peerId, blocks[0..5].mapIt(BlockDelivery(blk: it, address: it.address)))
|
||||||
await blockexc[3].engine.blocksHandler(switch[0].peerInfo.peerId, blocks[10..15])
|
|
||||||
|
discard blocks[4..10].mapIt(blockexc[2].engine.pendingBlocks.getWantHandle(it.address))
|
||||||
|
await blockexc[2].engine.blocksDeliveryHandler(switch[0].peerInfo.peerId, blocks[4..10].mapIt(BlockDelivery(blk: it, address: it.address)))
|
||||||
|
|
||||||
|
discard blocks[10..15].mapIt(blockexc[3].engine.pendingBlocks.getWantHandle(it.address))
|
||||||
|
await blockexc[3].engine.blocksDeliveryHandler(switch[0].peerInfo.peerId, blocks[10..15].mapIt(BlockDelivery(blk: it, address: it.address)))
|
||||||
|
|
||||||
MockDiscovery(blockexc[0].engine.discovery.discovery)
|
MockDiscovery(blockexc[0].engine.discovery.discovery)
|
||||||
.findBlockProvidersHandler = proc(d: MockDiscovery, cid: Cid):
|
.findBlockProvidersHandler = proc(d: MockDiscovery, cid: Cid):
|
||||||
|
@ -297,10 +309,10 @@ asyncchecksuite "E2E - Multiple Nodes Discovery":
|
||||||
|
|
||||||
await allFuturesThrowing(
|
await allFuturesThrowing(
|
||||||
switch.mapIt( it.start() ) &
|
switch.mapIt( it.start() ) &
|
||||||
blockexc.mapIt( it.engine.start() ))
|
blockexc.mapIt( it.engine.start() )).wait(10.seconds)
|
||||||
|
|
||||||
await allFutures(futs).wait(10.seconds)
|
await allFutures(futs).wait(10.seconds)
|
||||||
|
|
||||||
await allFuturesThrowing(
|
await allFuturesThrowing(
|
||||||
blockexc.mapIt( it.engine.stop() ) &
|
blockexc.mapIt( it.engine.stop() ) &
|
||||||
switch.mapIt( it.stop() ))
|
switch.mapIt( it.stop() )).wait(10.seconds)
|
||||||
|
|
|
@ -57,7 +57,7 @@ asyncchecksuite "Test Discovery Engine":
|
||||||
|
|
||||||
blockDiscovery.findBlockProvidersHandler =
|
blockDiscovery.findBlockProvidersHandler =
|
||||||
proc(d: MockDiscovery, cid: Cid): Future[seq[SignedPeerRecord]] {.async, gcsafe.} =
|
proc(d: MockDiscovery, cid: Cid): Future[seq[SignedPeerRecord]] {.async, gcsafe.} =
|
||||||
pendingBlocks.resolve(blocks.filterIt( it.cid == cid))
|
pendingBlocks.resolve(blocks.filterIt( it.cid == cid).mapIt(BlockDelivery(blk: it, address: it.address)))
|
||||||
|
|
||||||
await discoveryEngine.start()
|
await discoveryEngine.start()
|
||||||
await allFuturesThrowing(allFinished(wants)).wait(1.seconds)
|
await allFuturesThrowing(allFinished(wants)).wait(1.seconds)
|
||||||
|
@ -154,7 +154,9 @@ asyncchecksuite "Test Discovery Engine":
|
||||||
var
|
var
|
||||||
peerCtx = BlockExcPeerCtx(id: PeerId.example)
|
peerCtx = BlockExcPeerCtx(id: PeerId.example)
|
||||||
|
|
||||||
peerCtx.blocks[cid] = Presence(cid: cid, price: 0.u256)
|
let address = BlockAddress(leaf: false, cid: cid)
|
||||||
|
|
||||||
|
peerCtx.blocks[address] = Presence(address: address, price: 0.u256)
|
||||||
peerStore.add(peerCtx)
|
peerStore.add(peerCtx)
|
||||||
want.fire()
|
want.fire()
|
||||||
|
|
||||||
|
|
|
@ -120,10 +120,11 @@ asyncchecksuite "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()
|
||||||
|
let blkFut = nodeCmps1.pendingBlocks.getWantHandle( blk.cid )
|
||||||
(await nodeCmps2.localStore.putBlock(blk)).tryGet()
|
(await nodeCmps2.localStore.putBlock(blk)).tryGet()
|
||||||
|
|
||||||
let entry = Entry(
|
let entry = WantListEntry(
|
||||||
`block`: blk.cid.data.buffer,
|
address: blk.address,
|
||||||
priority: 1,
|
priority: 1,
|
||||||
cancel: false,
|
cancel: false,
|
||||||
wantType: WantType.WantBlock,
|
wantType: WantType.WantBlock,
|
||||||
|
@ -136,6 +137,7 @@ asyncchecksuite "NetworkStore engine - 2 nodes":
|
||||||
.pushOrUpdateNoWait(peerCtx1).isOk
|
.pushOrUpdateNoWait(peerCtx1).isOk
|
||||||
|
|
||||||
check eventually (await nodeCmps1.localStore.hasBlock(blk.cid)).tryGet()
|
check eventually (await nodeCmps1.localStore.hasBlock(blk.cid)).tryGet()
|
||||||
|
check eventually (await blkFut) == blk
|
||||||
|
|
||||||
test "Should get blocks from remote":
|
test "Should get blocks from remote":
|
||||||
let
|
let
|
||||||
|
|
|
@ -14,7 +14,7 @@ import pkg/codex/blockexchange
|
||||||
import pkg/codex/stores
|
import pkg/codex/stores
|
||||||
import pkg/codex/chunker
|
import pkg/codex/chunker
|
||||||
import pkg/codex/discovery
|
import pkg/codex/discovery
|
||||||
import pkg/codex/blocktype as bt
|
import pkg/codex/blocktype
|
||||||
import pkg/codex/utils/asyncheapqueue
|
import pkg/codex/utils/asyncheapqueue
|
||||||
|
|
||||||
import ../../helpers
|
import ../../helpers
|
||||||
|
@ -30,7 +30,7 @@ asyncchecksuite "NetworkStore engine basic":
|
||||||
blockDiscovery: Discovery
|
blockDiscovery: Discovery
|
||||||
peerStore: PeerCtxStore
|
peerStore: PeerCtxStore
|
||||||
pendingBlocks: PendingBlocksManager
|
pendingBlocks: PendingBlocksManager
|
||||||
blocks: seq[bt.Block]
|
blocks: seq[Block]
|
||||||
done: Future[void]
|
done: Future[void]
|
||||||
|
|
||||||
setup:
|
setup:
|
||||||
|
@ -48,20 +48,20 @@ asyncchecksuite "NetworkStore engine basic":
|
||||||
if chunk.len <= 0:
|
if chunk.len <= 0:
|
||||||
break
|
break
|
||||||
|
|
||||||
blocks.add(bt.Block.new(chunk).tryGet())
|
blocks.add(Block.new(chunk).tryGet())
|
||||||
|
|
||||||
done = newFuture[void]()
|
done = newFuture[void]()
|
||||||
|
|
||||||
test "Should send want list to new peers":
|
test "Should send want list to new peers":
|
||||||
proc sendWantList(
|
proc sendWantList(
|
||||||
id: PeerId,
|
id: PeerId,
|
||||||
cids: seq[Cid],
|
addresses: seq[BlockAddress],
|
||||||
priority: int32 = 0,
|
priority: int32 = 0,
|
||||||
cancel: bool = false,
|
cancel: bool = false,
|
||||||
wantType: WantType = WantType.WantHave,
|
wantType: WantType = WantType.WantHave,
|
||||||
full: bool = false,
|
full: bool = false,
|
||||||
sendDontHave: bool = false) {.gcsafe, async.} =
|
sendDontHave: bool = false) {.gcsafe, async.} =
|
||||||
check cids.mapIt($it).sorted == blocks.mapIt( $it.cid ).sorted
|
check addresses.mapIt($it.cidOrTreeCid).sorted == blocks.mapIt( $it.cid ).sorted
|
||||||
done.complete()
|
done.complete()
|
||||||
|
|
||||||
let
|
let
|
||||||
|
@ -140,7 +140,7 @@ asyncchecksuite "NetworkStore engine handlers":
|
||||||
discovery: DiscoveryEngine
|
discovery: DiscoveryEngine
|
||||||
peerCtx: BlockExcPeerCtx
|
peerCtx: BlockExcPeerCtx
|
||||||
localStore: BlockStore
|
localStore: BlockStore
|
||||||
blocks: seq[bt.Block]
|
blocks: seq[Block]
|
||||||
|
|
||||||
setup:
|
setup:
|
||||||
rng = Rng.instance()
|
rng = Rng.instance()
|
||||||
|
@ -151,7 +151,7 @@ asyncchecksuite "NetworkStore engine handlers":
|
||||||
if chunk.len <= 0:
|
if chunk.len <= 0:
|
||||||
break
|
break
|
||||||
|
|
||||||
blocks.add(bt.Block.new(chunk).tryGet())
|
blocks.add(Block.new(chunk).tryGet())
|
||||||
|
|
||||||
seckey = PrivateKey.random(rng[]).tryGet()
|
seckey = PrivateKey.random(rng[]).tryGet()
|
||||||
peerId = PeerId.init(seckey.getPublicKey().tryGet()).tryGet()
|
peerId = PeerId.init(seckey.getPublicKey().tryGet()).tryGet()
|
||||||
|
@ -193,7 +193,7 @@ asyncchecksuite "NetworkStore engine handlers":
|
||||||
let ctx = await engine.taskQueue.pop()
|
let ctx = await engine.taskQueue.pop()
|
||||||
check ctx.id == peerId
|
check ctx.id == peerId
|
||||||
# only `wantBlock` scheduled
|
# only `wantBlock` scheduled
|
||||||
check ctx.peerWants.mapIt( it.cid ) == blocks.mapIt( it.cid )
|
check ctx.peerWants.mapIt( it.address.cidOrTreeCid ) == blocks.mapIt( it.cid )
|
||||||
|
|
||||||
let done = handler()
|
let done = handler()
|
||||||
await engine.wantListHandler(peerId, wantList)
|
await engine.wantListHandler(peerId, wantList)
|
||||||
|
@ -205,7 +205,7 @@ asyncchecksuite "NetworkStore engine handlers":
|
||||||
wantList = makeWantList(blocks.mapIt( it.cid ))
|
wantList = makeWantList(blocks.mapIt( it.cid ))
|
||||||
|
|
||||||
proc sendPresence(peerId: PeerId, presence: seq[BlockPresence]) {.gcsafe, async.} =
|
proc sendPresence(peerId: PeerId, presence: seq[BlockPresence]) {.gcsafe, async.} =
|
||||||
check presence.mapIt( it.cid ) == wantList.entries.mapIt( it.`block` )
|
check presence.mapIt( it.address ) == wantList.entries.mapIt( it.address )
|
||||||
done.complete()
|
done.complete()
|
||||||
|
|
||||||
engine.network = BlockExcNetwork(
|
engine.network = BlockExcNetwork(
|
||||||
|
@ -227,7 +227,7 @@ asyncchecksuite "NetworkStore engine handlers":
|
||||||
sendDontHave = true)
|
sendDontHave = true)
|
||||||
|
|
||||||
proc sendPresence(peerId: PeerId, presence: seq[BlockPresence]) {.gcsafe, async.} =
|
proc sendPresence(peerId: PeerId, presence: seq[BlockPresence]) {.gcsafe, async.} =
|
||||||
check presence.mapIt( it.cid ) == wantList.entries.mapIt( it.`block` )
|
check presence.mapIt( it.address ) == wantList.entries.mapIt( it.address )
|
||||||
for p in presence:
|
for p in presence:
|
||||||
check:
|
check:
|
||||||
p.`type` == BlockPresenceType.DontHave
|
p.`type` == BlockPresenceType.DontHave
|
||||||
|
@ -249,12 +249,8 @@ asyncchecksuite "NetworkStore engine handlers":
|
||||||
sendDontHave = true)
|
sendDontHave = true)
|
||||||
|
|
||||||
proc sendPresence(peerId: PeerId, presence: seq[BlockPresence]) {.gcsafe, async.} =
|
proc sendPresence(peerId: PeerId, presence: seq[BlockPresence]) {.gcsafe, async.} =
|
||||||
let
|
|
||||||
cid1Buf = blocks[0].cid.data.buffer
|
|
||||||
cid2Buf = blocks[1].cid.data.buffer
|
|
||||||
|
|
||||||
for p in presence:
|
for p in presence:
|
||||||
if p.cid != cid1Buf and p.cid != cid2Buf:
|
if p.address.cidOrTreeCid != blocks[0].cid and p.address.cidOrTreeCid != blocks[1].cid:
|
||||||
check p.`type` == BlockPresenceType.DontHave
|
check p.`type` == BlockPresenceType.DontHave
|
||||||
else:
|
else:
|
||||||
check p.`type` == BlockPresenceType.Have
|
check p.`type` == BlockPresenceType.Have
|
||||||
|
@ -277,7 +273,9 @@ asyncchecksuite "NetworkStore engine handlers":
|
||||||
engine.pendingBlocks.getWantHandle( it.cid )
|
engine.pendingBlocks.getWantHandle( it.cid )
|
||||||
)
|
)
|
||||||
|
|
||||||
await engine.blocksHandler(peerId, blocks)
|
let blocksDelivery = blocks.mapIt(BlockDelivery(blk: it, address: it.address))
|
||||||
|
|
||||||
|
await engine.blocksDeliveryHandler(peerId, blocksDelivery)
|
||||||
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:
|
||||||
|
@ -292,7 +290,7 @@ asyncchecksuite "NetworkStore engine handlers":
|
||||||
|
|
||||||
peerContext.account = account.some
|
peerContext.account = account.some
|
||||||
peerContext.blocks = blocks.mapIt(
|
peerContext.blocks = blocks.mapIt(
|
||||||
(it.cid, Presence(cid: it.cid, price: rand(uint16).u256))
|
(it.address, Presence(address: it.address, price: rand(uint16).u256))
|
||||||
).toTable
|
).toTable
|
||||||
|
|
||||||
engine.network = BlockExcNetwork(
|
engine.network = BlockExcNetwork(
|
||||||
|
@ -301,7 +299,7 @@ asyncchecksuite "NetworkStore engine handlers":
|
||||||
let
|
let
|
||||||
amount =
|
amount =
|
||||||
blocks.mapIt(
|
blocks.mapIt(
|
||||||
peerContext.blocks[it.cid].price
|
peerContext.blocks[it.address].price
|
||||||
).foldl(a + b)
|
).foldl(a + b)
|
||||||
|
|
||||||
balances = !payment.state.outcome.balances(Asset)
|
balances = !payment.state.outcome.balances(Asset)
|
||||||
|
@ -311,22 +309,24 @@ asyncchecksuite "NetworkStore engine handlers":
|
||||||
done.complete()
|
done.complete()
|
||||||
))
|
))
|
||||||
|
|
||||||
await engine.blocksHandler(peerId, blocks)
|
await engine.blocksDeliveryHandler(peerId, blocks.mapIt(BlockDelivery(blk: it, address: it.address)))
|
||||||
await done.wait(100.millis)
|
await done.wait(100.millis)
|
||||||
|
|
||||||
test "Should handle block presence":
|
test "Should handle block presence":
|
||||||
var
|
var
|
||||||
handles: Table[Cid, Future[bt.Block]]
|
handles: Table[Cid, Future[Block]]
|
||||||
|
|
||||||
proc sendWantList(
|
proc sendWantList(
|
||||||
id: PeerId,
|
id: PeerId,
|
||||||
cids: seq[Cid],
|
addresses: seq[BlockAddress],
|
||||||
priority: int32 = 0,
|
priority: int32 = 0,
|
||||||
cancel: bool = false,
|
cancel: bool = false,
|
||||||
wantType: WantType = WantType.WantHave,
|
wantType: WantType = WantType.WantHave,
|
||||||
full: bool = false,
|
full: bool = false,
|
||||||
sendDontHave: bool = false) {.gcsafe, async.} =
|
sendDontHave: bool = false) {.gcsafe, async.} =
|
||||||
engine.pendingBlocks.resolve(blocks.filterIt( it.cid in cids ))
|
engine.pendingBlocks.resolve(blocks
|
||||||
|
.filterIt( it.address in addresses )
|
||||||
|
.mapIt(BlockDelivery(blk: it, address: it.address)))
|
||||||
|
|
||||||
engine.network = BlockExcNetwork(
|
engine.network = BlockExcNetwork(
|
||||||
request: BlockExcRequest(
|
request: BlockExcRequest(
|
||||||
|
@ -343,14 +343,14 @@ asyncchecksuite "NetworkStore engine handlers":
|
||||||
blocks.mapIt(
|
blocks.mapIt(
|
||||||
PresenceMessage.init(
|
PresenceMessage.init(
|
||||||
Presence(
|
Presence(
|
||||||
cid: it.cid,
|
address: it.address,
|
||||||
have: true,
|
have: true,
|
||||||
price: price
|
price: price
|
||||||
))))
|
))))
|
||||||
|
|
||||||
for cid in blocks.mapIt(it.cid):
|
for a in blocks.mapIt(it.address):
|
||||||
check cid in peerCtx.peerHave
|
check a in peerCtx.peerHave
|
||||||
check peerCtx.blocks[cid].price == price
|
check peerCtx.blocks[a].price == price
|
||||||
|
|
||||||
asyncchecksuite "Task Handler":
|
asyncchecksuite "Task Handler":
|
||||||
var
|
var
|
||||||
|
@ -369,7 +369,7 @@ asyncchecksuite "Task Handler":
|
||||||
|
|
||||||
peersCtx: seq[BlockExcPeerCtx]
|
peersCtx: seq[BlockExcPeerCtx]
|
||||||
peers: seq[PeerId]
|
peers: seq[PeerId]
|
||||||
blocks: seq[bt.Block]
|
blocks: seq[Block]
|
||||||
|
|
||||||
setup:
|
setup:
|
||||||
rng = Rng.instance()
|
rng = Rng.instance()
|
||||||
|
@ -379,7 +379,7 @@ asyncchecksuite "Task Handler":
|
||||||
if chunk.len <= 0:
|
if chunk.len <= 0:
|
||||||
break
|
break
|
||||||
|
|
||||||
blocks.add(bt.Block.new(chunk).tryGet())
|
blocks.add(Block.new(chunk).tryGet())
|
||||||
|
|
||||||
seckey = PrivateKey.random(rng[]).tryGet()
|
seckey = PrivateKey.random(rng[]).tryGet()
|
||||||
peerId = PeerId.init(seckey.getPublicKey().tryGet()).tryGet()
|
peerId = PeerId.init(seckey.getPublicKey().tryGet()).tryGet()
|
||||||
|
@ -419,22 +419,22 @@ asyncchecksuite "Task Handler":
|
||||||
engine.pricing = Pricing.example.some
|
engine.pricing = Pricing.example.some
|
||||||
|
|
||||||
test "Should send want-blocks in priority order":
|
test "Should send want-blocks in priority order":
|
||||||
proc sendBlocks(
|
proc sendBlocksDelivery(
|
||||||
id: PeerId,
|
id: PeerId,
|
||||||
blks: seq[bt.Block]) {.gcsafe, async.} =
|
blocksDelivery: seq[BlockDelivery]) {.gcsafe, async.} =
|
||||||
check blks.len == 2
|
check blocksDelivery.len == 2
|
||||||
check:
|
check:
|
||||||
blks[1].cid == blocks[0].cid
|
blocksDelivery[1].address == blocks[0].address
|
||||||
blks[0].cid == blocks[1].cid
|
blocksDelivery[0].address == blocks[1].address
|
||||||
|
|
||||||
for blk in blocks:
|
for blk in blocks:
|
||||||
(await engine.localStore.putBlock(blk)).tryGet()
|
(await engine.localStore.putBlock(blk)).tryGet()
|
||||||
engine.network.request.sendBlocks = sendBlocks
|
engine.network.request.sendBlocksDelivery = sendBlocksDelivery
|
||||||
|
|
||||||
# second block to send by priority
|
# second block to send by priority
|
||||||
peersCtx[0].peerWants.add(
|
peersCtx[0].peerWants.add(
|
||||||
Entry(
|
WantListEntry(
|
||||||
`block`: blocks[0].cid.data.buffer,
|
address: blocks[0].address,
|
||||||
priority: 49,
|
priority: 49,
|
||||||
cancel: false,
|
cancel: false,
|
||||||
wantType: WantType.WantBlock,
|
wantType: WantType.WantBlock,
|
||||||
|
@ -443,8 +443,8 @@ asyncchecksuite "Task Handler":
|
||||||
|
|
||||||
# first block to send by priority
|
# first block to send by priority
|
||||||
peersCtx[0].peerWants.add(
|
peersCtx[0].peerWants.add(
|
||||||
Entry(
|
WantListEntry(
|
||||||
`block`: blocks[1].cid.data.buffer,
|
address: blocks[1].address,
|
||||||
priority: 50,
|
priority: 50,
|
||||||
cancel: false,
|
cancel: false,
|
||||||
wantType: WantType.WantBlock,
|
wantType: WantType.WantBlock,
|
||||||
|
@ -455,14 +455,14 @@ asyncchecksuite "Task Handler":
|
||||||
|
|
||||||
test "Should send presence":
|
test "Should send presence":
|
||||||
let present = blocks
|
let present = blocks
|
||||||
let missing = @[bt.Block.new("missing".toBytes).tryGet()]
|
let missing = @[Block.new("missing".toBytes).tryGet()]
|
||||||
let price = (!engine.pricing).price
|
let price = (!engine.pricing).price
|
||||||
|
|
||||||
proc sendPresence(id: PeerId, presence: seq[BlockPresence]) {.gcsafe, async.} =
|
proc sendPresence(id: PeerId, presence: seq[BlockPresence]) {.gcsafe, async.} =
|
||||||
check presence.mapIt(!Presence.init(it)) == @[
|
check presence.mapIt(!Presence.init(it)) == @[
|
||||||
Presence(cid: present[0].cid, have: true, price: price),
|
Presence(address: present[0].address, have: true, price: price),
|
||||||
Presence(cid: present[1].cid, have: true, price: price),
|
Presence(address: present[1].address, have: true, price: price),
|
||||||
Presence(cid: missing[0].cid, have: false)
|
Presence(address: missing[0].address, have: false)
|
||||||
]
|
]
|
||||||
|
|
||||||
for blk in blocks:
|
for blk in blocks:
|
||||||
|
@ -471,8 +471,8 @@ asyncchecksuite "Task Handler":
|
||||||
|
|
||||||
# have block
|
# have block
|
||||||
peersCtx[0].peerWants.add(
|
peersCtx[0].peerWants.add(
|
||||||
Entry(
|
WantListEntry(
|
||||||
`block`: present[0].cid.data.buffer,
|
address: present[0].address,
|
||||||
priority: 1,
|
priority: 1,
|
||||||
cancel: false,
|
cancel: false,
|
||||||
wantType: WantType.WantHave,
|
wantType: WantType.WantHave,
|
||||||
|
@ -481,8 +481,8 @@ asyncchecksuite "Task Handler":
|
||||||
|
|
||||||
# have block
|
# have block
|
||||||
peersCtx[0].peerWants.add(
|
peersCtx[0].peerWants.add(
|
||||||
Entry(
|
WantListEntry(
|
||||||
`block`: present[1].cid.data.buffer,
|
address: present[1].address,
|
||||||
priority: 1,
|
priority: 1,
|
||||||
cancel: false,
|
cancel: false,
|
||||||
wantType: WantType.WantHave,
|
wantType: WantType.WantHave,
|
||||||
|
@ -491,8 +491,8 @@ asyncchecksuite "Task Handler":
|
||||||
|
|
||||||
# don't have block
|
# don't have block
|
||||||
peersCtx[0].peerWants.add(
|
peersCtx[0].peerWants.add(
|
||||||
Entry(
|
WantListEntry(
|
||||||
`block`: missing[0].cid.data.buffer,
|
address: missing[0].address,
|
||||||
priority: 1,
|
priority: 1,
|
||||||
cancel: false,
|
cancel: false,
|
||||||
wantType: WantType.WantHave,
|
wantType: WantType.WantHave,
|
||||||
|
|
|
@ -7,13 +7,12 @@ import ../../helpers
|
||||||
|
|
||||||
checksuite "block presence protobuf messages":
|
checksuite "block presence protobuf messages":
|
||||||
|
|
||||||
let cid = Cid.example
|
let
|
||||||
let price = UInt256.example
|
cid = Cid.example
|
||||||
let presence = Presence(cid: cid, have: true, price: price)
|
address = BlockAddress(leaf: false, cid: cid)
|
||||||
let message = PresenceMessage.init(presence)
|
price = UInt256.example
|
||||||
|
presence = Presence(address: address, have: true, price: price)
|
||||||
test "encodes CID":
|
message = PresenceMessage.init(presence)
|
||||||
check message.cid == cid.data.buffer
|
|
||||||
|
|
||||||
test "encodes have/donthave":
|
test "encodes have/donthave":
|
||||||
var presence = presence
|
var presence = presence
|
||||||
|
@ -26,12 +25,7 @@ checksuite "block presence protobuf messages":
|
||||||
check message.price == @(price.toBytesBE)
|
check message.price == @(price.toBytesBE)
|
||||||
|
|
||||||
test "decodes CID":
|
test "decodes CID":
|
||||||
check Presence.init(message).?cid == cid.some
|
check Presence.init(message).?address == address.some
|
||||||
|
|
||||||
test "fails to decode when CID is invalid":
|
|
||||||
var incorrect = message
|
|
||||||
incorrect.cid.del(0)
|
|
||||||
check Presence.init(incorrect).isNone
|
|
||||||
|
|
||||||
test "decodes have/donthave":
|
test "decodes have/donthave":
|
||||||
var message = message
|
var message = message
|
||||||
|
|
|
@ -47,13 +47,13 @@ asyncchecksuite "Network - Handlers":
|
||||||
discard await networkPeer.connect()
|
discard await networkPeer.connect()
|
||||||
|
|
||||||
test "Want List handler":
|
test "Want List handler":
|
||||||
proc wantListHandler(peer: PeerId, wantList: Wantlist) {.gcsafe, async.} =
|
proc wantListHandler(peer: PeerId, wantList: WantList) {.gcsafe, async.} =
|
||||||
# check that we got the correct amount of entries
|
# check that we got the correct amount of entries
|
||||||
check wantList.entries.len == 4
|
check wantList.entries.len == 4
|
||||||
|
|
||||||
for b in blocks:
|
for b in blocks:
|
||||||
check b.cid in wantList.entries
|
check b.address in wantList.entries
|
||||||
let entry = wantList.entries[wantList.entries.find(b.cid)]
|
let entry = wantList.entries[wantList.entries.find(b.address)]
|
||||||
check entry.wantType == WantType.WantHave
|
check entry.wantType == WantType.WantHave
|
||||||
check entry.priority == 1
|
check entry.priority == 1
|
||||||
check entry.cancel == true
|
check entry.cancel == true
|
||||||
|
@ -74,13 +74,13 @@ asyncchecksuite "Network - Handlers":
|
||||||
await done.wait(500.millis)
|
await done.wait(500.millis)
|
||||||
|
|
||||||
test "Blocks Handler":
|
test "Blocks Handler":
|
||||||
proc blocksHandler(peer: PeerId, blks: seq[bt.Block]) {.gcsafe, async.} =
|
proc blocksDeliveryHandler(peer: PeerId, blocksDelivery: seq[BlockDelivery]) {.gcsafe, async.} =
|
||||||
check blks == blocks
|
check blocks == blocksDelivery.mapIt(it.blk)
|
||||||
done.complete()
|
done.complete()
|
||||||
|
|
||||||
network.handlers.onBlocks = blocksHandler
|
network.handlers.onBlocksDelivery = blocksDeliveryHandler
|
||||||
|
|
||||||
let msg = Message(payload: makeBlocks(blocks))
|
let msg = Message(payload: blocks.mapIt(BlockDelivery(blk: it, address: it.address)))
|
||||||
await buffer.pushData(lenPrefix(protobufEncode(msg)))
|
await buffer.pushData(lenPrefix(protobufEncode(msg)))
|
||||||
|
|
||||||
await done.wait(500.millis)
|
await done.wait(500.millis)
|
||||||
|
@ -88,10 +88,10 @@ asyncchecksuite "Network - Handlers":
|
||||||
test "Presence Handler":
|
test "Presence Handler":
|
||||||
proc presenceHandler(
|
proc presenceHandler(
|
||||||
peer: PeerId,
|
peer: PeerId,
|
||||||
precense: seq[BlockPresence]) {.gcsafe, async.} =
|
presence: seq[BlockPresence]) {.gcsafe, async.} =
|
||||||
for b in blocks:
|
for b in blocks:
|
||||||
check:
|
check:
|
||||||
b.cid in precense
|
b.address in presence
|
||||||
|
|
||||||
done.complete()
|
done.complete()
|
||||||
|
|
||||||
|
@ -100,7 +100,7 @@ asyncchecksuite "Network - Handlers":
|
||||||
let msg = Message(
|
let msg = Message(
|
||||||
blockPresences: blocks.mapIt(
|
blockPresences: blocks.mapIt(
|
||||||
BlockPresence(
|
BlockPresence(
|
||||||
cid: it.cid.data.buffer,
|
address: it.address,
|
||||||
type: BlockPresenceType.Have
|
type: BlockPresenceType.Have
|
||||||
)))
|
)))
|
||||||
await buffer.pushData(lenPrefix(protobufEncode(msg)))
|
await buffer.pushData(lenPrefix(protobufEncode(msg)))
|
||||||
|
@ -177,13 +177,13 @@ asyncchecksuite "Network - Senders":
|
||||||
switch2.stop())
|
switch2.stop())
|
||||||
|
|
||||||
test "Send want list":
|
test "Send want list":
|
||||||
proc wantListHandler(peer: PeerId, wantList: Wantlist) {.gcsafe, async.} =
|
proc wantListHandler(peer: PeerId, wantList: WantList) {.gcsafe, async.} =
|
||||||
# check that we got the correct amount of entries
|
# check that we got the correct amount of entries
|
||||||
check wantList.entries.len == 4
|
check wantList.entries.len == 4
|
||||||
|
|
||||||
for b in blocks:
|
for b in blocks:
|
||||||
check b.cid in wantList.entries
|
check b.address in wantList.entries
|
||||||
let entry = wantList.entries[wantList.entries.find(b.cid)]
|
let entry = wantList.entries[wantList.entries.find(b.address)]
|
||||||
check entry.wantType == WantType.WantHave
|
check entry.wantType == WantType.WantHave
|
||||||
check entry.priority == 1
|
check entry.priority == 1
|
||||||
check entry.cancel == true
|
check entry.cancel == true
|
||||||
|
@ -194,21 +194,21 @@ asyncchecksuite "Network - Senders":
|
||||||
network2.handlers.onWantList = wantListHandler
|
network2.handlers.onWantList = wantListHandler
|
||||||
await network1.sendWantList(
|
await network1.sendWantList(
|
||||||
switch2.peerInfo.peerId,
|
switch2.peerInfo.peerId,
|
||||||
blocks.mapIt( it.cid ),
|
blocks.mapIt( it.address ),
|
||||||
1, true, WantType.WantHave,
|
1, true, WantType.WantHave,
|
||||||
true, true)
|
true, true)
|
||||||
|
|
||||||
await done.wait(500.millis)
|
await done.wait(500.millis)
|
||||||
|
|
||||||
test "send blocks":
|
test "send blocks":
|
||||||
proc blocksHandler(peer: PeerId, blks: seq[bt.Block]) {.gcsafe, async.} =
|
proc blocksDeliveryHandler(peer: PeerId, blocksDelivery: seq[BlockDelivery]) {.gcsafe, async.} =
|
||||||
check blks == blocks
|
check blocks == blocksDelivery.mapIt(it.blk)
|
||||||
done.complete()
|
done.complete()
|
||||||
|
|
||||||
network2.handlers.onBlocks = blocksHandler
|
network2.handlers.onBlocksDelivery = blocksDeliveryHandler
|
||||||
await network1.sendBlocks(
|
await network1.sendBlocksDelivery(
|
||||||
switch2.peerInfo.peerId,
|
switch2.peerInfo.peerId,
|
||||||
blocks)
|
blocks.mapIt(BlockDelivery(blk: it, address: it.address)))
|
||||||
|
|
||||||
await done.wait(500.millis)
|
await done.wait(500.millis)
|
||||||
|
|
||||||
|
@ -218,7 +218,7 @@ asyncchecksuite "Network - Senders":
|
||||||
precense: seq[BlockPresence]) {.gcsafe, async.} =
|
precense: seq[BlockPresence]) {.gcsafe, async.} =
|
||||||
for b in blocks:
|
for b in blocks:
|
||||||
check:
|
check:
|
||||||
b.cid in precense
|
b.address in precense
|
||||||
|
|
||||||
done.complete()
|
done.complete()
|
||||||
|
|
||||||
|
@ -228,7 +228,7 @@ asyncchecksuite "Network - Senders":
|
||||||
switch2.peerInfo.peerId,
|
switch2.peerInfo.peerId,
|
||||||
blocks.mapIt(
|
blocks.mapIt(
|
||||||
BlockPresence(
|
BlockPresence(
|
||||||
cid: it.cid.data.buffer,
|
address: it.address,
|
||||||
type: BlockPresenceType.Have
|
type: BlockPresenceType.Have
|
||||||
)))
|
)))
|
||||||
|
|
||||||
|
|
|
@ -35,12 +35,12 @@ checksuite "Peer Context Store Peer Selection":
|
||||||
var
|
var
|
||||||
store: PeerCtxStore
|
store: PeerCtxStore
|
||||||
peerCtxs: seq[BlockExcPeerCtx]
|
peerCtxs: seq[BlockExcPeerCtx]
|
||||||
cids: seq[Cid]
|
addresses: seq[BlockAddress]
|
||||||
|
|
||||||
setup:
|
setup:
|
||||||
store = PeerCtxStore.new()
|
store = PeerCtxStore.new()
|
||||||
cids = collect(newSeq):
|
addresses = collect(newSeq):
|
||||||
for i in 0..<10: Cid.example
|
for i in 0..<10: BlockAddress(leaf: false, cid: Cid.example)
|
||||||
|
|
||||||
peerCtxs = collect(newSeq):
|
peerCtxs = collect(newSeq):
|
||||||
for i in 0..<10: BlockExcPeerCtx.example
|
for i in 0..<10: BlockExcPeerCtx.example
|
||||||
|
@ -50,20 +50,20 @@ checksuite "Peer Context Store Peer Selection":
|
||||||
|
|
||||||
teardown:
|
teardown:
|
||||||
store = nil
|
store = nil
|
||||||
cids = @[]
|
addresses = @[]
|
||||||
peerCtxs = @[]
|
peerCtxs = @[]
|
||||||
|
|
||||||
test "Should select peers that have Cid":
|
test "Should select peers that have Cid":
|
||||||
peerCtxs[0].blocks = collect(initTable):
|
peerCtxs[0].blocks = collect(initTable):
|
||||||
for i, c in cids:
|
for i, a in addresses:
|
||||||
{ c: Presence(cid: c, price: i.u256) }
|
{ a: Presence(address: a, price: i.u256) }
|
||||||
|
|
||||||
peerCtxs[5].blocks = collect(initTable):
|
peerCtxs[5].blocks = collect(initTable):
|
||||||
for i, c in cids:
|
for i, a in addresses:
|
||||||
{ c: Presence(cid: c, price: i.u256) }
|
{ a: Presence(address: a, price: i.u256) }
|
||||||
|
|
||||||
let
|
let
|
||||||
peers = store.peersHave(cids[0])
|
peers = store.peersHave(addresses[0])
|
||||||
|
|
||||||
check peers.len == 2
|
check peers.len == 2
|
||||||
check peerCtxs[0] in peers
|
check peerCtxs[0] in peers
|
||||||
|
@ -71,19 +71,19 @@ checksuite "Peer Context Store Peer Selection":
|
||||||
|
|
||||||
test "Should select cheapest peers for Cid":
|
test "Should select cheapest peers for Cid":
|
||||||
peerCtxs[0].blocks = collect(initTable):
|
peerCtxs[0].blocks = collect(initTable):
|
||||||
for i, c in cids:
|
for i, a in addresses:
|
||||||
{ c: Presence(cid: c, price: (5 + i).u256) }
|
{ a: Presence(address: a, price: (5 + i).u256) }
|
||||||
|
|
||||||
peerCtxs[5].blocks = collect(initTable):
|
peerCtxs[5].blocks = collect(initTable):
|
||||||
for i, c in cids:
|
for i, a in addresses:
|
||||||
{ c: Presence(cid: c, price: (2 + i).u256) }
|
{ a: Presence(address: a, price: (2 + i).u256) }
|
||||||
|
|
||||||
peerCtxs[9].blocks = collect(initTable):
|
peerCtxs[9].blocks = collect(initTable):
|
||||||
for i, c in cids:
|
for i, a in addresses:
|
||||||
{ c: Presence(cid: c, price: i.u256) }
|
{ a: Presence(address: a, price: i.u256) }
|
||||||
|
|
||||||
let
|
let
|
||||||
peers = store.selectCheapest(cids[0])
|
peers = store.selectCheapest(addresses[0])
|
||||||
|
|
||||||
check peers.len == 3
|
check peers.len == 3
|
||||||
check peers[0] == peerCtxs[9]
|
check peers[0] == peerCtxs[9]
|
||||||
|
@ -92,9 +92,9 @@ checksuite "Peer Context Store Peer Selection":
|
||||||
|
|
||||||
test "Should select peers that want Cid":
|
test "Should select peers that want Cid":
|
||||||
let
|
let
|
||||||
entries = cids.mapIt(
|
entries = addresses.mapIt(
|
||||||
Entry(
|
WantListEntry(
|
||||||
`block`: it.data.buffer,
|
address: it,
|
||||||
priority: 1,
|
priority: 1,
|
||||||
cancel: false,
|
cancel: false,
|
||||||
wantType: WantType.WantBlock,
|
wantType: WantType.WantBlock,
|
||||||
|
@ -104,7 +104,7 @@ checksuite "Peer Context Store Peer Selection":
|
||||||
peerCtxs[5].peerWants = entries
|
peerCtxs[5].peerWants = entries
|
||||||
|
|
||||||
let
|
let
|
||||||
peers = store.peersWant(cids[4])
|
peers = store.peersWant(addresses[4])
|
||||||
|
|
||||||
check peers.len == 2
|
check peers.len == 2
|
||||||
check peerCtxs[0] in peers
|
check peerCtxs[0] in peers
|
||||||
|
|
|
@ -18,7 +18,7 @@ checksuite "Pending Blocks":
|
||||||
|
|
||||||
discard pendingBlocks.getWantHandle(blk.cid)
|
discard pendingBlocks.getWantHandle(blk.cid)
|
||||||
|
|
||||||
check pendingBlocks.pending(blk.cid)
|
check blk.cid in pendingBlocks
|
||||||
|
|
||||||
test "Should resolve want handle":
|
test "Should resolve want handle":
|
||||||
let
|
let
|
||||||
|
@ -27,7 +27,7 @@ checksuite "Pending Blocks":
|
||||||
handle = pendingBlocks.getWantHandle(blk.cid)
|
handle = pendingBlocks.getWantHandle(blk.cid)
|
||||||
|
|
||||||
check blk.cid in pendingBlocks
|
check blk.cid in pendingBlocks
|
||||||
pendingBlocks.resolve(@[blk])
|
pendingBlocks.resolve(@[blk].mapIt(BlockDelivery(blk: it, address: it.address)))
|
||||||
check (await handle) == blk
|
check (await handle) == blk
|
||||||
check blk.cid notin pendingBlocks
|
check blk.cid notin pendingBlocks
|
||||||
|
|
||||||
|
@ -64,7 +64,7 @@ checksuite "Pending Blocks":
|
||||||
|
|
||||||
check:
|
check:
|
||||||
blks.mapIt( $it.cid ).sorted(cmp[string]) ==
|
blks.mapIt( $it.cid ).sorted(cmp[string]) ==
|
||||||
toSeq(pendingBlocks.wantList).mapIt( $it ).sorted(cmp[string])
|
toSeq(pendingBlocks.wantListBlockCids).mapIt( $it ).sorted(cmp[string])
|
||||||
|
|
||||||
test "Should get want handles list":
|
test "Should get want handles list":
|
||||||
let
|
let
|
||||||
|
@ -74,7 +74,7 @@ checksuite "Pending Blocks":
|
||||||
wantHandles = toSeq(pendingBlocks.wantHandles)
|
wantHandles = toSeq(pendingBlocks.wantHandles)
|
||||||
|
|
||||||
check wantHandles.len == handles.len
|
check wantHandles.len == handles.len
|
||||||
pendingBlocks.resolve(blks)
|
pendingBlocks.resolve(blks.mapIt(BlockDelivery(blk: it, address: it.address)))
|
||||||
|
|
||||||
check:
|
check:
|
||||||
(await allFinished(wantHandles)).mapIt( $it.read.cid ).sorted(cmp[string]) ==
|
(await allFinished(wantHandles)).mapIt( $it.read.cid ).sorted(cmp[string]) ==
|
||||||
|
|
|
@ -53,6 +53,10 @@ proc example*(_: type BlockExcPeerCtx): BlockExcPeerCtx =
|
||||||
proc example*(_: type Cid): Cid =
|
proc example*(_: type Cid): Cid =
|
||||||
bt.Block.example.cid
|
bt.Block.example.cid
|
||||||
|
|
||||||
|
proc example*(_: type MultiHash, mcodec = multiCodec("sha2-256")): MultiHash =
|
||||||
|
let bytes = newSeqWith(256, rand(uint8))
|
||||||
|
MultiHash.digest($mcodec, bytes).tryGet()
|
||||||
|
|
||||||
proc example*(_: type Availability): Availability =
|
proc example*(_: type Availability): Availability =
|
||||||
Availability.init(
|
Availability.init(
|
||||||
size = uint16.example.u256,
|
size = uint16.example.u256,
|
||||||
|
|
|
@ -1,25 +1,30 @@
|
||||||
|
import std/sequtils
|
||||||
|
|
||||||
import pkg/chronos
|
import pkg/chronos
|
||||||
import pkg/libp2p
|
import pkg/libp2p
|
||||||
import pkg/libp2p/varint
|
import pkg/libp2p/varint
|
||||||
import pkg/codex/blocktype as bt
|
import pkg/codex/blocktype
|
||||||
import pkg/codex/stores
|
import pkg/codex/stores
|
||||||
import pkg/codex/manifest
|
import pkg/codex/manifest
|
||||||
|
import pkg/codex/merkletree
|
||||||
|
import pkg/codex/blockexchange
|
||||||
import pkg/codex/rng
|
import pkg/codex/rng
|
||||||
|
|
||||||
import ./helpers/nodeutils
|
import ./helpers/nodeutils
|
||||||
import ./helpers/randomchunker
|
import ./helpers/randomchunker
|
||||||
|
import ./helpers/mockchunker
|
||||||
import ./helpers/mockdiscovery
|
import ./helpers/mockdiscovery
|
||||||
import ./helpers/always
|
import ./helpers/always
|
||||||
import ../checktest
|
import ../checktest
|
||||||
|
|
||||||
export randomchunker, nodeutils, mockdiscovery, always, checktest, manifest
|
export randomchunker, nodeutils, mockdiscovery, mockchunker, always, checktest, manifest
|
||||||
|
|
||||||
export libp2p except setup, eventually
|
export libp2p except setup, eventually
|
||||||
|
|
||||||
# NOTE: The meaning of equality for blocks
|
# NOTE: The meaning of equality for blocks
|
||||||
# is changed here, because blocks are now `ref`
|
# is changed here, because blocks are now `ref`
|
||||||
# types. This is only in tests!!!
|
# types. This is only in tests!!!
|
||||||
func `==`*(a, b: bt.Block): bool =
|
func `==`*(a, b: Block): bool =
|
||||||
(a.cid == b.cid) and (a.data == b.data)
|
(a.cid == b.cid) and (a.data == b.data)
|
||||||
|
|
||||||
proc lenPrefix*(msg: openArray[byte]): seq[byte] =
|
proc lenPrefix*(msg: openArray[byte]): seq[byte] =
|
||||||
|
@ -33,21 +38,85 @@ proc lenPrefix*(msg: openArray[byte]): seq[byte] =
|
||||||
|
|
||||||
return buf
|
return buf
|
||||||
|
|
||||||
|
proc makeManifestAndTree*(blocks: seq[Block]): ?!(Manifest, MerkleTree) =
|
||||||
|
|
||||||
|
if blocks.len == 0:
|
||||||
|
return failure("Blocks list was empty")
|
||||||
|
|
||||||
|
let
|
||||||
|
datasetSize = blocks.mapIt(it.data.len).foldl(a + b)
|
||||||
|
blockSize = blocks.mapIt(it.data.len).foldl(max(a, b))
|
||||||
|
tree = ? MerkleTree.init(blocks.mapIt(it.cid))
|
||||||
|
treeCid = ? tree.rootCid
|
||||||
|
manifest = Manifest.new(
|
||||||
|
treeCid = treeCid,
|
||||||
|
blockSize = NBytes(blockSize),
|
||||||
|
datasetSize = NBytes(datasetSize),
|
||||||
|
version = CIDv1,
|
||||||
|
hcodec = tree.mcodec
|
||||||
|
)
|
||||||
|
|
||||||
|
return success((manifest, tree))
|
||||||
|
|
||||||
|
proc makeWantList*(
|
||||||
|
cids: seq[Cid],
|
||||||
|
priority: int = 0,
|
||||||
|
cancel: bool = false,
|
||||||
|
wantType: WantType = WantType.WantHave,
|
||||||
|
full: bool = false,
|
||||||
|
sendDontHave: bool = false
|
||||||
|
): WantList =
|
||||||
|
WantList(
|
||||||
|
entries: cids.mapIt(
|
||||||
|
WantListEntry(
|
||||||
|
address: BlockAddress(leaf: false, cid: it),
|
||||||
|
priority: priority.int32,
|
||||||
|
cancel: cancel,
|
||||||
|
wantType: wantType,
|
||||||
|
sendDontHave: sendDontHave) ),
|
||||||
|
full: full)
|
||||||
|
|
||||||
|
proc storeDataGetManifest*(store: BlockStore, chunker: Chunker): Future[Manifest] {.async.} =
|
||||||
|
var cids = newSeq[Cid]()
|
||||||
|
|
||||||
|
while (
|
||||||
|
let chunk = await chunker.getBytes();
|
||||||
|
chunk.len > 0):
|
||||||
|
|
||||||
|
let blk = Block.new(chunk).tryGet()
|
||||||
|
cids.add(blk.cid)
|
||||||
|
(await store.putBlock(blk)).tryGet()
|
||||||
|
|
||||||
|
let
|
||||||
|
tree = MerkleTree.init(cids).tryGet()
|
||||||
|
treeCid = tree.rootCid.tryGet()
|
||||||
|
manifest = Manifest.new(
|
||||||
|
treeCid = treeCid,
|
||||||
|
blockSize = NBytes(chunker.chunkSize),
|
||||||
|
datasetSize = NBytes(chunker.offset),
|
||||||
|
)
|
||||||
|
|
||||||
|
for i in 0..<tree.leavesCount:
|
||||||
|
let proof = tree.getProof(i).tryGet()
|
||||||
|
(await store.putBlockCidAndProof(treeCid, i, cids[i], proof)).tryGet()
|
||||||
|
|
||||||
|
return manifest
|
||||||
|
|
||||||
proc corruptBlocks*(
|
proc corruptBlocks*(
|
||||||
store: BlockStore,
|
store: BlockStore,
|
||||||
manifest: Manifest,
|
manifest: Manifest,
|
||||||
blks, bytes: int): Future[seq[int]] {.async.} =
|
blks, bytes: int): Future[seq[int]] {.async.} =
|
||||||
var pos: seq[int]
|
var pos: seq[int]
|
||||||
|
|
||||||
doAssert blks < manifest.len
|
doAssert blks < manifest.blocksCount
|
||||||
while pos.len < blks:
|
while pos.len < blks:
|
||||||
let i = Rng.instance.rand(manifest.len - 1)
|
let i = Rng.instance.rand(manifest.blocksCount - 1)
|
||||||
if pos.find(i) >= 0:
|
if pos.find(i) >= 0:
|
||||||
continue
|
continue
|
||||||
|
|
||||||
pos.add(i)
|
pos.add(i)
|
||||||
var
|
var
|
||||||
blk = (await store.getBlock(manifest[i])).tryGet()
|
blk = (await store.getBlock(manifest.treeCid, i)).tryGet()
|
||||||
bytePos: seq[int]
|
bytePos: seq[int]
|
||||||
|
|
||||||
doAssert bytes < blk.data.len
|
doAssert bytes < blk.data.len
|
||||||
|
|
|
@ -0,0 +1,45 @@
|
||||||
|
import std/sequtils
|
||||||
|
|
||||||
|
import pkg/chronos
|
||||||
|
|
||||||
|
import pkg/codex/chunker
|
||||||
|
import pkg/codex/rng
|
||||||
|
|
||||||
|
export chunker
|
||||||
|
|
||||||
|
type
|
||||||
|
MockChunker* = Chunker
|
||||||
|
|
||||||
|
proc new*(
|
||||||
|
T: type MockChunker,
|
||||||
|
dataset: openArray[byte],
|
||||||
|
chunkSize: int | NBytes,
|
||||||
|
pad: bool = false
|
||||||
|
): MockChunker =
|
||||||
|
## Create a chunker that produces data
|
||||||
|
##
|
||||||
|
|
||||||
|
let
|
||||||
|
chunkSize = chunkSize.NBytes
|
||||||
|
dataset = @dataset
|
||||||
|
|
||||||
|
var consumed = 0
|
||||||
|
proc reader(data: ChunkBuffer, len: int): Future[int] {.async, gcsafe, raises: [Defect].} =
|
||||||
|
|
||||||
|
if consumed >= dataset.len:
|
||||||
|
return 0
|
||||||
|
|
||||||
|
var read = 0
|
||||||
|
while read < len and
|
||||||
|
read < chunkSize.int and
|
||||||
|
(consumed + read) < dataset.len:
|
||||||
|
data[read] = dataset[consumed + read]
|
||||||
|
read.inc
|
||||||
|
|
||||||
|
consumed += read
|
||||||
|
return read
|
||||||
|
|
||||||
|
Chunker.new(
|
||||||
|
reader = reader,
|
||||||
|
pad = pad,
|
||||||
|
chunkSize = chunkSize)
|
|
@ -13,7 +13,8 @@ import pkg/libp2p
|
||||||
import pkg/questionable
|
import pkg/questionable
|
||||||
import pkg/questionable/results
|
import pkg/questionable/results
|
||||||
|
|
||||||
import codex/stores/repostore
|
import pkg/codex/stores/repostore
|
||||||
|
import pkg/codex/utils/asynciter
|
||||||
|
|
||||||
type
|
type
|
||||||
MockRepoStore* = ref object of RepoStore
|
MockRepoStore* = ref object of RepoStore
|
||||||
|
@ -31,15 +32,14 @@ method delBlock*(self: MockRepoStore, cid: Cid): Future[?!void] {.async.} =
|
||||||
dec self.iteratorIndex
|
dec self.iteratorIndex
|
||||||
return success()
|
return success()
|
||||||
|
|
||||||
method getBlockExpirations*(self: MockRepoStore, maxNumber: int, offset: int): Future[?!BlockExpirationIter] {.async.} =
|
method getBlockExpirations*(self: MockRepoStore, maxNumber: int, offset: int): Future[?!AsyncIter[?BlockExpiration]] {.async.} =
|
||||||
if self.getBlockExpirationsThrows:
|
if self.getBlockExpirationsThrows:
|
||||||
raise new CatchableError
|
raise new CatchableError
|
||||||
|
|
||||||
self.getBeMaxNumber = maxNumber
|
self.getBeMaxNumber = maxNumber
|
||||||
self.getBeOffset = offset
|
self.getBeOffset = offset
|
||||||
|
|
||||||
var iter = BlockExpirationIter()
|
var iter = AsyncIter[?BlockExpiration]()
|
||||||
iter.finished = false
|
|
||||||
|
|
||||||
self.iteratorIndex = offset
|
self.iteratorIndex = offset
|
||||||
var numberLeft = maxNumber
|
var numberLeft = maxNumber
|
||||||
|
@ -49,7 +49,7 @@ method getBlockExpirations*(self: MockRepoStore, maxNumber: int, offset: int): F
|
||||||
let selectedBlock = self.testBlockExpirations[self.iteratorIndex]
|
let selectedBlock = self.testBlockExpirations[self.iteratorIndex]
|
||||||
inc self.iteratorIndex
|
inc self.iteratorIndex
|
||||||
return selectedBlock.some
|
return selectedBlock.some
|
||||||
iter.finished = true
|
iter.finish
|
||||||
return BlockExpiration.none
|
return BlockExpiration.none
|
||||||
|
|
||||||
iter.next = next
|
iter.next = next
|
||||||
|
|
|
@ -0,0 +1,42 @@
|
||||||
|
import std/unittest
|
||||||
|
|
||||||
|
import pkg/questionable/results
|
||||||
|
import pkg/stew/byteutils
|
||||||
|
|
||||||
|
import pkg/codex/merkletree
|
||||||
|
import ../helpers
|
||||||
|
|
||||||
|
checksuite "merkletree - coders":
|
||||||
|
const data =
|
||||||
|
[
|
||||||
|
"0123456789012345678901234567890123456789".toBytes,
|
||||||
|
"1234567890123456789012345678901234567890".toBytes,
|
||||||
|
"2345678901234567890123456789012345678901".toBytes,
|
||||||
|
"3456789012345678901234567890123456789012".toBytes,
|
||||||
|
"4567890123456789012345678901234567890123".toBytes,
|
||||||
|
"5678901234567890123456789012345678901234".toBytes,
|
||||||
|
"6789012345678901234567890123456789012345".toBytes,
|
||||||
|
"7890123456789012345678901234567890123456".toBytes,
|
||||||
|
"8901234567890123456789012345678901234567".toBytes,
|
||||||
|
"9012345678901234567890123456789012345678".toBytes,
|
||||||
|
]
|
||||||
|
|
||||||
|
test "encoding and decoding a tree yields the same tree":
|
||||||
|
var builder = MerkleTreeBuilder.init(multiCodec("sha2-256")).tryGet()
|
||||||
|
builder.addDataBlock(data[0]).tryGet()
|
||||||
|
builder.addDataBlock(data[1]).tryGet()
|
||||||
|
builder.addDataBlock(data[2]).tryGet()
|
||||||
|
builder.addDataBlock(data[3]).tryGet()
|
||||||
|
builder.addDataBlock(data[4]).tryGet()
|
||||||
|
builder.addDataBlock(data[5]).tryGet()
|
||||||
|
builder.addDataBlock(data[6]).tryGet()
|
||||||
|
builder.addDataBlock(data[7]).tryGet()
|
||||||
|
builder.addDataBlock(data[8]).tryGet()
|
||||||
|
builder.addDataBlock(data[9]).tryGet()
|
||||||
|
|
||||||
|
let tree = builder.build().tryGet()
|
||||||
|
let encodedBytes = tree.encode()
|
||||||
|
let decodedTree = MerkleTree.decode(encodedBytes).tryGet()
|
||||||
|
|
||||||
|
check:
|
||||||
|
tree == decodedTree
|
|
@ -1,86 +1,165 @@
|
||||||
import std/unittest
|
import std/unittest
|
||||||
import std/bitops
|
|
||||||
import std/random
|
|
||||||
import std/sequtils
|
import std/sequtils
|
||||||
import pkg/libp2p
|
import std/tables
|
||||||
import codex/merkletree/merkletree
|
|
||||||
import ../helpers
|
|
||||||
import pkg/questionable/results
|
import pkg/questionable/results
|
||||||
|
import pkg/stew/byteutils
|
||||||
|
import pkg/nimcrypto/sha2
|
||||||
|
|
||||||
|
import pkg/codex/merkletree
|
||||||
|
import ../helpers
|
||||||
|
|
||||||
checksuite "merkletree":
|
checksuite "merkletree":
|
||||||
|
const data =
|
||||||
|
[
|
||||||
|
"0123456789012345678901234567890123456789".toBytes,
|
||||||
|
"1234567890123456789012345678901234567890".toBytes,
|
||||||
|
"2345678901234567890123456789012345678901".toBytes,
|
||||||
|
"3456789012345678901234567890123456789012".toBytes,
|
||||||
|
"4567890123456789012345678901234567890123".toBytes,
|
||||||
|
"5678901234567890123456789012345678901234".toBytes,
|
||||||
|
"6789012345678901234567890123456789012345".toBytes,
|
||||||
|
"7890123456789012345678901234567890123456".toBytes,
|
||||||
|
"8901234567890123456789012345678901234567".toBytes,
|
||||||
|
"9012345678901234567890123456789012345678".toBytes,
|
||||||
|
]
|
||||||
|
|
||||||
const sha256 = multiCodec("sha2-256")
|
const sha256 = multiCodec("sha2-256")
|
||||||
const sha512 = multiCodec("sha2-512")
|
const sha512 = multiCodec("sha2-512")
|
||||||
|
|
||||||
proc randomHash(codec: MultiCodec = sha256): MerkleHash =
|
proc combine(a, b: MultiHash, codec: MultiCodec = sha256): MultiHash =
|
||||||
var data: array[0..31, byte]
|
|
||||||
for i in 0..31:
|
|
||||||
data[i] = rand(uint8)
|
|
||||||
return MultiHash.digest($codec, data).tryGet()
|
|
||||||
|
|
||||||
proc combine(a, b: MerkleHash, codec: MultiCodec = sha256): MerkleHash =
|
|
||||||
var buf = newSeq[byte](a.size + b.size)
|
var buf = newSeq[byte](a.size + b.size)
|
||||||
for i in 0..<a.size:
|
copyMem(addr buf[0], unsafeAddr a.data.buffer[a.dpos], a.size)
|
||||||
buf[i] = a.data.buffer[i]
|
copyMem(addr buf[a.size], unsafeAddr b.data.buffer[b.dpos], b.size)
|
||||||
for i in 0..<b.size:
|
|
||||||
buf[i + a.size] = b.data.buffer[i]
|
|
||||||
return MultiHash.digest($codec, buf).tryGet()
|
return MultiHash.digest($codec, buf).tryGet()
|
||||||
|
|
||||||
var
|
var zeroHash: MultiHash
|
||||||
leaves: array[0..10, MerkleHash]
|
var oneHash: MultiHash
|
||||||
|
|
||||||
|
var expectedLeaves: array[data.len, MultiHash]
|
||||||
|
var builder: MerkleTreeBuilder
|
||||||
|
|
||||||
setup:
|
setup:
|
||||||
for i in 0..leaves.high:
|
for i in 0..<data.len:
|
||||||
leaves[i] = randomHash()
|
expectedLeaves[i] = MultiHash.digest($sha256, data[i]).tryGet()
|
||||||
|
|
||||||
|
builder = MerkleTreeBuilder.init(sha256).tryGet()
|
||||||
|
var zero: array[32, byte]
|
||||||
|
var one: array[32, byte]
|
||||||
|
one[^1] = 0x01
|
||||||
|
zeroHash = MultiHash.init($sha256, zero).tryGet()
|
||||||
|
oneHash = MultiHash.init($sha256, one).tryGet()
|
||||||
|
|
||||||
test "tree with one leaf has expected root":
|
test "tree with one leaf has expected structure":
|
||||||
let tree = MerkleTree.init(leaves[0..0]).tryGet()
|
builder.addDataBlock(data[0]).tryGet()
|
||||||
|
|
||||||
|
let tree = builder.build().tryGet()
|
||||||
|
|
||||||
check:
|
check:
|
||||||
tree.leaves == leaves[0..0]
|
tree.leaves.toSeq == expectedLeaves[0..0]
|
||||||
tree.root == leaves[0]
|
tree.root == expectedLeaves[0]
|
||||||
tree.len == 1
|
tree.len == 1
|
||||||
|
|
||||||
test "tree with two leaves has expected root":
|
test "tree with two leaves has expected structure":
|
||||||
let
|
builder.addDataBlock(data[0]).tryGet()
|
||||||
expectedRoot = combine(leaves[0], leaves[1])
|
builder.addDataBlock(data[1]).tryGet()
|
||||||
|
|
||||||
let tree = MerkleTree.init(leaves[0..1]).tryGet()
|
let tree = builder.build().tryGet()
|
||||||
|
|
||||||
|
let expectedRoot = combine(expectedLeaves[0], expectedLeaves[1])
|
||||||
|
|
||||||
check:
|
check:
|
||||||
tree.leaves == leaves[0..1]
|
tree.leaves.toSeq == expectedLeaves[0..1]
|
||||||
tree.len == 3
|
tree.len == 3
|
||||||
tree.root == expectedRoot
|
tree.root == expectedRoot
|
||||||
|
|
||||||
test "tree with three leaves has expected root":
|
test "tree with three leaves has expected structure":
|
||||||
let
|
builder.addDataBlock(data[0]).tryGet()
|
||||||
expectedRoot = combine(combine(leaves[0], leaves[1]), combine(leaves[2], leaves[2]))
|
builder.addDataBlock(data[1]).tryGet()
|
||||||
|
builder.addDataBlock(data[2]).tryGet()
|
||||||
|
|
||||||
let tree = MerkleTree.init(leaves[0..2]).tryGet()
|
let tree = builder.build().tryGet()
|
||||||
|
|
||||||
|
let
|
||||||
|
expectedRoot = combine(
|
||||||
|
combine(expectedLeaves[0], expectedLeaves[1]),
|
||||||
|
combine(expectedLeaves[2], zeroHash)
|
||||||
|
)
|
||||||
|
|
||||||
check:
|
check:
|
||||||
tree.leaves == leaves[0..2]
|
tree.leaves.toSeq == expectedLeaves[0..2]
|
||||||
tree.len == 6
|
tree.len == 6
|
||||||
tree.root == expectedRoot
|
tree.root == expectedRoot
|
||||||
|
|
||||||
test "tree with two leaves provides expected proofs":
|
test "tree with nine leaves has expected structure":
|
||||||
let tree = MerkleTree.init(leaves[0..1]).tryGet()
|
builder.addDataBlock(data[0]).tryGet()
|
||||||
|
builder.addDataBlock(data[1]).tryGet()
|
||||||
|
builder.addDataBlock(data[2]).tryGet()
|
||||||
|
builder.addDataBlock(data[3]).tryGet()
|
||||||
|
builder.addDataBlock(data[4]).tryGet()
|
||||||
|
builder.addDataBlock(data[5]).tryGet()
|
||||||
|
builder.addDataBlock(data[6]).tryGet()
|
||||||
|
builder.addDataBlock(data[7]).tryGet()
|
||||||
|
builder.addDataBlock(data[8]).tryGet()
|
||||||
|
|
||||||
|
let tree = builder.build().tryGet()
|
||||||
|
|
||||||
|
let
|
||||||
|
expectedRoot = combine(
|
||||||
|
combine(
|
||||||
|
combine(
|
||||||
|
combine(expectedLeaves[0], expectedLeaves[1]),
|
||||||
|
combine(expectedLeaves[2], expectedLeaves[3]),
|
||||||
|
),
|
||||||
|
combine(
|
||||||
|
combine(expectedLeaves[4], expectedLeaves[5]),
|
||||||
|
combine(expectedLeaves[6], expectedLeaves[7])
|
||||||
|
)
|
||||||
|
),
|
||||||
|
combine(
|
||||||
|
combine(
|
||||||
|
combine(expectedLeaves[8], zeroHash),
|
||||||
|
oneHash
|
||||||
|
),
|
||||||
|
oneHash
|
||||||
|
)
|
||||||
|
)
|
||||||
|
|
||||||
|
check:
|
||||||
|
tree.leaves.toSeq == expectedLeaves[0..8]
|
||||||
|
tree.len == 20
|
||||||
|
tree.root == expectedRoot
|
||||||
|
|
||||||
|
test "tree with two leaves provides expected and valid proofs":
|
||||||
|
builder.addDataBlock(data[0]).tryGet()
|
||||||
|
builder.addDataBlock(data[1]).tryGet()
|
||||||
|
|
||||||
|
let tree = builder.build().tryGet()
|
||||||
|
|
||||||
let expectedProofs = [
|
let expectedProofs = [
|
||||||
MerkleProof.init(0, @[leaves[1]]),
|
MerkleProof.init(0, @[expectedLeaves[1]]).tryGet(),
|
||||||
MerkleProof.init(1, @[leaves[0]]),
|
MerkleProof.init(1, @[expectedLeaves[0]]).tryGet(),
|
||||||
]
|
]
|
||||||
|
|
||||||
check:
|
check:
|
||||||
tree.getProof(0).tryGet() == expectedProofs[0]
|
tree.getProof(0).tryGet() == expectedProofs[0]
|
||||||
tree.getProof(1).tryGet() == expectedProofs[1]
|
tree.getProof(1).tryGet() == expectedProofs[1]
|
||||||
|
|
||||||
|
check:
|
||||||
|
tree.getProof(0).tryGet().verifyDataBlock(data[0], tree.root).tryGet()
|
||||||
|
tree.getProof(1).tryGet().verifyDataBlock(data[1], tree.root).tryGet()
|
||||||
|
|
||||||
test "tree with three leaves provides expected proofs":
|
test "tree with three leaves provides expected proofs":
|
||||||
let tree = MerkleTree.init(leaves[0..2]).tryGet()
|
builder.addDataBlock(data[0]).tryGet()
|
||||||
|
builder.addDataBlock(data[1]).tryGet()
|
||||||
|
builder.addDataBlock(data[2]).tryGet()
|
||||||
|
|
||||||
|
let tree = builder.build().tryGet()
|
||||||
|
|
||||||
let expectedProofs = [
|
let expectedProofs = [
|
||||||
MerkleProof.init(0, @[leaves[1], combine(leaves[2], leaves[2])]),
|
MerkleProof.init(0, @[expectedLeaves[1], combine(expectedLeaves[2], zeroHash)]).tryGet(),
|
||||||
MerkleProof.init(1, @[leaves[0], combine(leaves[2], leaves[2])]),
|
MerkleProof.init(1, @[expectedLeaves[0], combine(expectedLeaves[2], zeroHash)]).tryGet(),
|
||||||
MerkleProof.init(2, @[leaves[2], combine(leaves[0], leaves[1])]),
|
MerkleProof.init(2, @[zeroHash, combine(expectedLeaves[0], expectedLeaves[1])]).tryGet(),
|
||||||
]
|
]
|
||||||
|
|
||||||
check:
|
check:
|
||||||
|
@ -88,21 +167,73 @@ checksuite "merkletree":
|
||||||
tree.getProof(1).tryGet() == expectedProofs[1]
|
tree.getProof(1).tryGet() == expectedProofs[1]
|
||||||
tree.getProof(2).tryGet() == expectedProofs[2]
|
tree.getProof(2).tryGet() == expectedProofs[2]
|
||||||
|
|
||||||
|
check:
|
||||||
|
tree.getProof(0).tryGet().verifyDataBlock(data[0], tree.root).tryGet()
|
||||||
|
tree.getProof(1).tryGet().verifyDataBlock(data[1], tree.root).tryGet()
|
||||||
|
tree.getProof(2).tryGet().verifyDataBlock(data[2], tree.root).tryGet()
|
||||||
|
|
||||||
|
test "tree with nine leaves provides expected proofs":
|
||||||
|
builder.addDataBlock(data[0]).tryGet()
|
||||||
|
builder.addDataBlock(data[1]).tryGet()
|
||||||
|
builder.addDataBlock(data[2]).tryGet()
|
||||||
|
builder.addDataBlock(data[3]).tryGet()
|
||||||
|
builder.addDataBlock(data[4]).tryGet()
|
||||||
|
builder.addDataBlock(data[5]).tryGet()
|
||||||
|
builder.addDataBlock(data[6]).tryGet()
|
||||||
|
builder.addDataBlock(data[7]).tryGet()
|
||||||
|
builder.addDataBlock(data[8]).tryGet()
|
||||||
|
|
||||||
|
let tree = builder.build().tryGet()
|
||||||
|
|
||||||
|
let expectedProofs = {
|
||||||
|
4:
|
||||||
|
MerkleProof.init(4, @[
|
||||||
|
expectedLeaves[5],
|
||||||
|
combine(expectedLeaves[6], expectedLeaves[7]),
|
||||||
|
combine(
|
||||||
|
combine(expectedLeaves[0], expectedLeaves[1]),
|
||||||
|
combine(expectedLeaves[2], expectedLeaves[3]),
|
||||||
|
),
|
||||||
|
combine(
|
||||||
|
combine(
|
||||||
|
combine(expectedLeaves[8], zeroHash),
|
||||||
|
oneHash
|
||||||
|
),
|
||||||
|
oneHash
|
||||||
|
)
|
||||||
|
]).tryGet(),
|
||||||
|
8:
|
||||||
|
MerkleProof.init(8, @[
|
||||||
|
zeroHash,
|
||||||
|
oneHash,
|
||||||
|
oneHash,
|
||||||
|
combine(
|
||||||
|
combine(
|
||||||
|
combine(expectedLeaves[0], expectedLeaves[1]),
|
||||||
|
combine(expectedLeaves[2], expectedLeaves[3]),
|
||||||
|
),
|
||||||
|
combine(
|
||||||
|
combine(expectedLeaves[4], expectedLeaves[5]),
|
||||||
|
combine(expectedLeaves[6], expectedLeaves[7])
|
||||||
|
)
|
||||||
|
)
|
||||||
|
]).tryGet(),
|
||||||
|
}.newTable
|
||||||
|
|
||||||
|
check:
|
||||||
|
tree.getProof(4).tryGet() == expectedProofs[4]
|
||||||
|
tree.getProof(8).tryGet() == expectedProofs[8]
|
||||||
|
|
||||||
|
check:
|
||||||
|
tree.getProof(4).tryGet().verifyDataBlock(data[4], tree.root).tryGet()
|
||||||
|
tree.getProof(8).tryGet().verifyDataBlock(data[8], tree.root).tryGet()
|
||||||
|
|
||||||
test "getProof fails for index out of bounds":
|
test "getProof fails for index out of bounds":
|
||||||
let tree = MerkleTree.init(leaves[0..3]).tryGet()
|
builder.addDataBlock(data[0]).tryGet()
|
||||||
|
builder.addDataBlock(data[1]).tryGet()
|
||||||
|
builder.addDataBlock(data[2]).tryGet()
|
||||||
|
|
||||||
|
let tree = builder.build().tryGet()
|
||||||
|
|
||||||
check:
|
check:
|
||||||
isErr(tree.getProof(-1))
|
|
||||||
isErr(tree.getProof(4))
|
isErr(tree.getProof(4))
|
||||||
|
|
||||||
test "can create MerkleTree directly from root hash":
|
|
||||||
let tree = MerkleTree.init(leaves[0], 1)
|
|
||||||
|
|
||||||
check:
|
|
||||||
tree.root == leaves[0]
|
|
||||||
|
|
||||||
test "cannot create MerkleTree from leaves with different codec":
|
|
||||||
let res = MerkleTree.init(@[randomHash(sha256), randomHash(sha512)])
|
|
||||||
|
|
||||||
check:
|
|
||||||
isErr(res)
|
|
||||||
|
|
|
@ -11,6 +11,8 @@ import pkg/questionable/results
|
||||||
import pkg/codex/stores/cachestore
|
import pkg/codex/stores/cachestore
|
||||||
import pkg/codex/chunker
|
import pkg/codex/chunker
|
||||||
import pkg/codex/manifest
|
import pkg/codex/manifest
|
||||||
|
import pkg/codex/merkletree
|
||||||
|
import pkg/codex/utils
|
||||||
|
|
||||||
import ../helpers
|
import ../helpers
|
||||||
|
|
||||||
|
@ -27,6 +29,8 @@ proc commonBlockStoreTests*(name: string,
|
||||||
asyncchecksuite name & " Store Common":
|
asyncchecksuite name & " Store Common":
|
||||||
var
|
var
|
||||||
newBlock, newBlock1, newBlock2, newBlock3: Block
|
newBlock, newBlock1, newBlock2, newBlock3: Block
|
||||||
|
manifest: Manifest
|
||||||
|
tree: MerkleTree
|
||||||
store: BlockStore
|
store: BlockStore
|
||||||
|
|
||||||
setup:
|
setup:
|
||||||
|
@ -35,6 +39,8 @@ proc commonBlockStoreTests*(name: string,
|
||||||
newBlock2 = Block.new("2".repeat(100).toBytes()).tryGet()
|
newBlock2 = Block.new("2".repeat(100).toBytes()).tryGet()
|
||||||
newBlock3 = Block.new("3".repeat(100).toBytes()).tryGet()
|
newBlock3 = Block.new("3".repeat(100).toBytes()).tryGet()
|
||||||
|
|
||||||
|
(manifest, tree) = makeManifestAndTree(@[newBlock, newBlock1, newBlock2, newBlock3]).tryGet()
|
||||||
|
|
||||||
if not isNil(before):
|
if not isNil(before):
|
||||||
await before()
|
await before()
|
||||||
|
|
||||||
|
@ -104,10 +110,10 @@ proc commonBlockStoreTests*(name: string,
|
||||||
test "listBlocks Manifest":
|
test "listBlocks Manifest":
|
||||||
let
|
let
|
||||||
blocks = @[newBlock1, newBlock2, newBlock3]
|
blocks = @[newBlock1, newBlock2, newBlock3]
|
||||||
manifest = Manifest.new(blocks = blocks.mapIt( it.cid )).tryGet()
|
|
||||||
manifestBlock = Block.new(manifest.encode().tryGet(), codec = DagPBCodec).tryGet()
|
manifestBlock = Block.new(manifest.encode().tryGet(), codec = DagPBCodec).tryGet()
|
||||||
|
treeBlock = Block.new(tree.encode()).tryGet()
|
||||||
putHandles = await allFinished(
|
putHandles = await allFinished(
|
||||||
(manifestBlock & blocks).mapIt( store.putBlock( it ) ))
|
(@[treeBlock, manifestBlock] & blocks).mapIt( store.putBlock( it ) ))
|
||||||
|
|
||||||
for handle in putHandles:
|
for handle in putHandles:
|
||||||
check not handle.failed
|
check not handle.failed
|
||||||
|
@ -128,10 +134,10 @@ proc commonBlockStoreTests*(name: string,
|
||||||
test "listBlocks Both":
|
test "listBlocks Both":
|
||||||
let
|
let
|
||||||
blocks = @[newBlock1, newBlock2, newBlock3]
|
blocks = @[newBlock1, newBlock2, newBlock3]
|
||||||
manifest = Manifest.new(blocks = blocks.mapIt( it.cid )).tryGet()
|
|
||||||
manifestBlock = Block.new(manifest.encode().tryGet(), codec = DagPBCodec).tryGet()
|
manifestBlock = Block.new(manifest.encode().tryGet(), codec = DagPBCodec).tryGet()
|
||||||
|
treeBlock = Block.new(tree.encode()).tryGet()
|
||||||
putHandles = await allFinished(
|
putHandles = await allFinished(
|
||||||
(manifestBlock & blocks).mapIt( store.putBlock( it ) ))
|
(@[treeBlock, manifestBlock] & blocks).mapIt( store.putBlock( it ) ))
|
||||||
|
|
||||||
for handle in putHandles:
|
for handle in putHandles:
|
||||||
check not handle.failed
|
check not handle.failed
|
||||||
|
@ -146,4 +152,4 @@ proc commonBlockStoreTests*(name: string,
|
||||||
check (await store.hasBlock(cid)).tryGet()
|
check (await store.hasBlock(cid)).tryGet()
|
||||||
count.inc
|
count.inc
|
||||||
|
|
||||||
check count == 4
|
check count == 5
|
||||||
|
|
|
@ -70,4 +70,4 @@ checksuite "Cache Store":
|
||||||
|
|
||||||
commonBlockStoreTests(
|
commonBlockStoreTests(
|
||||||
"Cache", proc: BlockStore =
|
"Cache", proc: BlockStore =
|
||||||
BlockStore(CacheStore.new(cacheSize = 500, chunkSize = 1)))
|
BlockStore(CacheStore.new(cacheSize = 1000, chunkSize = 1)))
|
||||||
|
|
|
@ -16,6 +16,7 @@ import pkg/codex/chunker
|
||||||
import pkg/codex/stores
|
import pkg/codex/stores
|
||||||
import pkg/codex/blocktype as bt
|
import pkg/codex/blocktype as bt
|
||||||
import pkg/codex/clock
|
import pkg/codex/clock
|
||||||
|
import pkg/codex/utils/asynciter
|
||||||
|
|
||||||
import ../helpers
|
import ../helpers
|
||||||
import ../helpers/mockclock
|
import ../helpers/mockclock
|
||||||
|
@ -72,7 +73,7 @@ asyncchecksuite "RepoStore":
|
||||||
mockClock = MockClock.new()
|
mockClock = MockClock.new()
|
||||||
mockClock.set(now)
|
mockClock.set(now)
|
||||||
|
|
||||||
repo = RepoStore.new(repoDs, metaDs, mockClock, quotaMaxBytes = 200)
|
repo = RepoStore.new(repoDs, metaDs, clock = mockClock, quotaMaxBytes = 200)
|
||||||
|
|
||||||
teardown:
|
teardown:
|
||||||
(await repoDs.close()).tryGet
|
(await repoDs.close()).tryGet
|
||||||
|
@ -326,7 +327,7 @@ asyncchecksuite "RepoStore":
|
||||||
response.len == 0
|
response.len == 0
|
||||||
|
|
||||||
test "Should retrieve block expiration information":
|
test "Should retrieve block expiration information":
|
||||||
proc unpack(beIter: Future[?!BlockExpirationIter]): Future[seq[BlockExpiration]] {.async.} =
|
proc unpack(beIter: Future[?!AsyncIter[?BlockExpiration]]): Future[seq[BlockExpiration]] {.async.} =
|
||||||
var expirations = newSeq[BlockExpiration](0)
|
var expirations = newSeq[BlockExpiration](0)
|
||||||
without iter =? (await beIter), err:
|
without iter =? (await beIter), err:
|
||||||
return expirations
|
return expirations
|
||||||
|
@ -366,22 +367,22 @@ asyncchecksuite "RepoStore":
|
||||||
assertExpiration(blockExpirations2[0], blk3)
|
assertExpiration(blockExpirations2[0], blk3)
|
||||||
|
|
||||||
test "should put empty blocks":
|
test "should put empty blocks":
|
||||||
let blk = Cid.example.emptyBlock
|
let blk = Cid.example.emptyBlock.tryGet()
|
||||||
check (await repo.putBlock(blk)).isOk
|
check (await repo.putBlock(blk)).isOk
|
||||||
|
|
||||||
test "should get empty blocks":
|
test "should get empty blocks":
|
||||||
let blk = Cid.example.emptyBlock
|
let blk = Cid.example.emptyBlock.tryGet()
|
||||||
|
|
||||||
let got = await repo.getBlock(blk.cid)
|
let got = await repo.getBlock(blk.cid)
|
||||||
check got.isOk
|
check got.isOk
|
||||||
check got.get.cid == blk.cid
|
check got.get.cid == blk.cid
|
||||||
|
|
||||||
test "should delete empty blocks":
|
test "should delete empty blocks":
|
||||||
let blk = Cid.example.emptyBlock
|
let blk = Cid.example.emptyBlock.tryGet()
|
||||||
check (await repo.delBlock(blk.cid)).isOk
|
check (await repo.delBlock(blk.cid)).isOk
|
||||||
|
|
||||||
test "should have empty block":
|
test "should have empty block":
|
||||||
let blk = Cid.example.emptyBlock
|
let blk = Cid.example.emptyBlock.tryGet()
|
||||||
|
|
||||||
let has = await repo.hasBlock(blk.cid)
|
let has = await repo.hasBlock(blk.cid)
|
||||||
check has.isOk
|
check has.isOk
|
||||||
|
@ -393,7 +394,7 @@ commonBlockStoreTests(
|
||||||
RepoStore.new(
|
RepoStore.new(
|
||||||
SQLiteDatastore.new(Memory).tryGet(),
|
SQLiteDatastore.new(Memory).tryGet(),
|
||||||
SQLiteDatastore.new(Memory).tryGet(),
|
SQLiteDatastore.new(Memory).tryGet(),
|
||||||
MockClock.new())))
|
clock = MockClock.new())))
|
||||||
|
|
||||||
const
|
const
|
||||||
path = currentSourcePath().parentDir / "test"
|
path = currentSourcePath().parentDir / "test"
|
||||||
|
@ -413,6 +414,6 @@ commonBlockStoreTests(
|
||||||
RepoStore.new(
|
RepoStore.new(
|
||||||
FSDatastore.new(path, depth).tryGet(),
|
FSDatastore.new(path, depth).tryGet(),
|
||||||
SQLiteDatastore.new(Memory).tryGet(),
|
SQLiteDatastore.new(Memory).tryGet(),
|
||||||
MockClock.new())),
|
clock = MockClock.new())),
|
||||||
before = before,
|
before = before,
|
||||||
after = after)
|
after = after)
|
||||||
|
|
|
@ -22,25 +22,16 @@ asyncchecksuite "Erasure encode/decode":
|
||||||
var manifest: Manifest
|
var manifest: Manifest
|
||||||
var store: BlockStore
|
var store: BlockStore
|
||||||
var erasure: Erasure
|
var erasure: Erasure
|
||||||
var repoDs: Datastore
|
|
||||||
var metaDs: SQLiteDatastore
|
|
||||||
|
|
||||||
setup:
|
setup:
|
||||||
|
let
|
||||||
|
repoDs = SQLiteDatastore.new(Memory).tryGet()
|
||||||
|
metaDs = SQLiteDatastore.new(Memory).tryGet()
|
||||||
rng = Rng.instance()
|
rng = Rng.instance()
|
||||||
chunker = RandomChunker.new(rng, size = dataSetSize, chunkSize = BlockSize)
|
chunker = RandomChunker.new(rng, size = dataSetSize, chunkSize = BlockSize)
|
||||||
manifest = !Manifest.new(blockSize = BlockSize)
|
|
||||||
repoDs = SQLiteDatastore.new(Memory).tryGet()
|
|
||||||
metaDs = SQLiteDatastore.new(Memory).tryGet()
|
|
||||||
store = RepoStore.new(repoDs, metaDs)
|
store = RepoStore.new(repoDs, metaDs)
|
||||||
erasure = Erasure.new(store, leoEncoderProvider, leoDecoderProvider)
|
erasure = Erasure.new(store, leoEncoderProvider, leoDecoderProvider)
|
||||||
|
manifest = await storeDataGetManifest(store, chunker)
|
||||||
while (
|
|
||||||
let chunk = await chunker.getBytes();
|
|
||||||
chunk.len > 0):
|
|
||||||
|
|
||||||
let blk = bt.Block.new(chunk).tryGet()
|
|
||||||
manifest.add(blk.cid)
|
|
||||||
(await store.putBlock(blk)).tryGet()
|
|
||||||
|
|
||||||
proc encode(buffers, parity: int): Future[Manifest] {.async.} =
|
proc encode(buffers, parity: int): Future[Manifest] {.async.} =
|
||||||
let
|
let
|
||||||
|
@ -50,8 +41,8 @@ asyncchecksuite "Erasure encode/decode":
|
||||||
parity)).tryGet()
|
parity)).tryGet()
|
||||||
|
|
||||||
check:
|
check:
|
||||||
encoded.len mod (buffers + parity) == 0
|
encoded.blocksCount mod (buffers + parity) == 0
|
||||||
encoded.rounded == (manifest.len + (buffers - (manifest.len mod buffers)))
|
encoded.rounded == (manifest.blocksCount + (buffers - (manifest.blocksCount mod buffers)))
|
||||||
encoded.steps == encoded.rounded div buffers
|
encoded.steps == encoded.rounded div buffers
|
||||||
|
|
||||||
return encoded
|
return encoded
|
||||||
|
@ -64,24 +55,25 @@ asyncchecksuite "Erasure encode/decode":
|
||||||
let encoded = await encode(buffers, parity)
|
let encoded = await encode(buffers, parity)
|
||||||
|
|
||||||
var
|
var
|
||||||
column = rng.rand((encoded.len - 1) div encoded.steps) # random column
|
column = rng.rand((encoded.blocksCount - 1) div encoded.steps) # random column
|
||||||
dropped: seq[Cid]
|
dropped: seq[int]
|
||||||
|
|
||||||
for _ in 0..<encoded.ecM:
|
for _ in 0..<encoded.ecM:
|
||||||
dropped.add(encoded[column])
|
dropped.add(column)
|
||||||
(await store.delBlock(encoded[column])).tryGet()
|
(await store.delBlock(encoded.treeCid, column)).tryGet()
|
||||||
|
(await store.delBlock(manifest.treeCid, column)).tryGet()
|
||||||
column.inc(encoded.steps - 1)
|
column.inc(encoded.steps - 1)
|
||||||
|
|
||||||
var
|
var
|
||||||
decoded = (await erasure.decode(encoded)).tryGet()
|
decoded = (await erasure.decode(encoded)).tryGet()
|
||||||
|
|
||||||
check:
|
check:
|
||||||
decoded.cid.tryGet() == manifest.cid.tryGet()
|
decoded.treeCid == manifest.treeCid
|
||||||
decoded.cid.tryGet() == encoded.originalCid
|
decoded.treeCid == encoded.originalTreeCid
|
||||||
decoded.len == encoded.originalLen
|
decoded.blocksCount == encoded.originalBlocksCount
|
||||||
|
|
||||||
for d in dropped:
|
for d in dropped:
|
||||||
let present = await store.hasBlock(d)
|
let present = await store.hasBlock(manifest.treeCid, d)
|
||||||
check present.tryGet()
|
check present.tryGet()
|
||||||
|
|
||||||
test "Should not tolerate losing more than M data blocks in a single random column":
|
test "Should not tolerate losing more than M data blocks in a single random column":
|
||||||
|
@ -92,12 +84,13 @@ asyncchecksuite "Erasure encode/decode":
|
||||||
let encoded = await encode(buffers, parity)
|
let encoded = await encode(buffers, parity)
|
||||||
|
|
||||||
var
|
var
|
||||||
column = rng.rand((encoded.len - 1) div encoded.steps) # random column
|
column = rng.rand((encoded.blocksCount - 1) div encoded.steps) # random column
|
||||||
dropped: seq[Cid]
|
dropped: seq[int]
|
||||||
|
|
||||||
for _ in 0..<encoded.ecM + 1:
|
for _ in 0..<encoded.ecM + 1:
|
||||||
dropped.add(encoded[column])
|
dropped.add(column)
|
||||||
(await store.delBlock(encoded[column])).tryGet()
|
(await store.delBlock(encoded.treeCid, column)).tryGet()
|
||||||
|
(await store.delBlock(manifest.treeCid, column)).tryGet()
|
||||||
column.inc(encoded.steps)
|
column.inc(encoded.steps)
|
||||||
|
|
||||||
var
|
var
|
||||||
|
@ -107,7 +100,7 @@ asyncchecksuite "Erasure encode/decode":
|
||||||
decoded = (await erasure.decode(encoded)).tryGet()
|
decoded = (await erasure.decode(encoded)).tryGet()
|
||||||
|
|
||||||
for d in dropped:
|
for d in dropped:
|
||||||
let present = await store.hasBlock(d)
|
let present = await store.hasBlock(manifest.treeCid, d)
|
||||||
check not present.tryGet()
|
check not present.tryGet()
|
||||||
|
|
||||||
test "Should tolerate losing M data blocks in M random columns":
|
test "Should tolerate losing M data blocks in M random columns":
|
||||||
|
@ -123,19 +116,21 @@ asyncchecksuite "Erasure encode/decode":
|
||||||
|
|
||||||
while offset < encoded.steps - 1:
|
while offset < encoded.steps - 1:
|
||||||
let
|
let
|
||||||
blockIdx = toSeq(countup(offset, encoded.len - 1, encoded.steps))
|
blockIdx = toSeq(countup(offset, encoded.blocksCount - 1, encoded.steps))
|
||||||
|
|
||||||
for _ in 0..<encoded.ecM:
|
for _ in 0..<encoded.ecM:
|
||||||
blocks.add(rng.sample(blockIdx, blocks))
|
blocks.add(rng.sample(blockIdx, blocks))
|
||||||
offset.inc
|
offset.inc
|
||||||
|
|
||||||
for idx in blocks:
|
for idx in blocks:
|
||||||
(await store.delBlock(encoded[idx])).tryGet()
|
(await store.delBlock(encoded.treeCid, idx)).tryGet()
|
||||||
|
(await store.delBlock(manifest.treeCid, idx)).tryGet()
|
||||||
|
discard
|
||||||
|
|
||||||
discard (await erasure.decode(encoded)).tryGet()
|
discard (await erasure.decode(encoded)).tryGet()
|
||||||
|
|
||||||
for d in manifest:
|
for d in 0..<manifest.blocksCount:
|
||||||
let present = await store.hasBlock(d)
|
let present = await store.hasBlock(manifest.treeCid, d)
|
||||||
check present.tryGet()
|
check present.tryGet()
|
||||||
|
|
||||||
test "Should not tolerate losing more than M data blocks in M random columns":
|
test "Should not tolerate losing more than M data blocks in M random columns":
|
||||||
|
@ -151,20 +146,23 @@ asyncchecksuite "Erasure encode/decode":
|
||||||
|
|
||||||
while offset < encoded.steps - 1:
|
while offset < encoded.steps - 1:
|
||||||
let
|
let
|
||||||
blockIdx = toSeq(countup(offset, encoded.len - 1, encoded.steps))
|
blockIdx = toSeq(countup(offset, encoded.blocksCount - 1, encoded.steps))
|
||||||
|
|
||||||
for _ in 0..<encoded.ecM + 1: # NOTE: the +1
|
for _ in 0..<encoded.ecM + 1: # NOTE: the +1
|
||||||
var idx: int
|
var idx: int
|
||||||
while true:
|
while true:
|
||||||
idx = rng.sample(blockIdx, blocks)
|
idx = rng.sample(blockIdx, blocks)
|
||||||
if not encoded[idx].isEmpty:
|
let blk = (await store.getBlock(encoded.treeCid, idx)).tryGet()
|
||||||
|
if not blk.isEmpty:
|
||||||
break
|
break
|
||||||
|
|
||||||
blocks.add(idx)
|
blocks.add(idx)
|
||||||
offset.inc
|
offset.inc
|
||||||
|
|
||||||
for idx in blocks:
|
for idx in blocks:
|
||||||
(await store.delBlock(encoded[idx])).tryGet()
|
(await store.delBlock(encoded.treeCid, idx)).tryGet()
|
||||||
|
(await store.delBlock(manifest.treeCid, idx)).tryGet()
|
||||||
|
discard
|
||||||
|
|
||||||
var
|
var
|
||||||
decoded: Manifest
|
decoded: Manifest
|
||||||
|
@ -179,13 +177,14 @@ asyncchecksuite "Erasure encode/decode":
|
||||||
|
|
||||||
let encoded = await encode(buffers, parity)
|
let encoded = await encode(buffers, parity)
|
||||||
|
|
||||||
for b in encoded.blocks[0..<encoded.steps * encoded.ecM]:
|
for b in 0..<encoded.steps * encoded.ecM:
|
||||||
(await store.delBlock(b)).tryGet()
|
(await store.delBlock(encoded.treeCid, b)).tryGet()
|
||||||
|
(await store.delBlock(manifest.treeCid, b)).tryGet()
|
||||||
|
|
||||||
discard (await erasure.decode(encoded)).tryGet()
|
discard (await erasure.decode(encoded)).tryGet()
|
||||||
|
|
||||||
for d in manifest:
|
for d in 0..<manifest.blocksCount:
|
||||||
let present = await store.hasBlock(d)
|
let present = await store.hasBlock(manifest.treeCid, d)
|
||||||
check present.tryGet()
|
check present.tryGet()
|
||||||
|
|
||||||
test "Should tolerate losing M (a.k.a row) contiguous parity blocks":
|
test "Should tolerate losing M (a.k.a row) contiguous parity blocks":
|
||||||
|
@ -195,13 +194,14 @@ asyncchecksuite "Erasure encode/decode":
|
||||||
|
|
||||||
let encoded = await encode(buffers, parity)
|
let encoded = await encode(buffers, parity)
|
||||||
|
|
||||||
for b in encoded.blocks[^(encoded.steps * encoded.ecM)..^1]:
|
for b in (encoded.blocksCount - encoded.steps * encoded.ecM)..<encoded.blocksCount:
|
||||||
(await store.delBlock(b)).tryGet()
|
(await store.delBlock(encoded.treeCid, b)).tryGet()
|
||||||
|
(await store.delBlock(manifest.treeCid, b)).tryGet()
|
||||||
|
|
||||||
discard (await erasure.decode(encoded)).tryGet()
|
discard (await erasure.decode(encoded)).tryGet()
|
||||||
|
|
||||||
for d in manifest:
|
for d in 0..<manifest.blocksCount:
|
||||||
let present = await store.hasBlock(d)
|
let present = await store.hasBlock(manifest.treeCid, d)
|
||||||
check present.tryGet()
|
check present.tryGet()
|
||||||
|
|
||||||
test "handles edge case of 0 parity blocks":
|
test "handles edge case of 0 parity blocks":
|
||||||
|
|
|
@ -10,88 +10,20 @@ import pkg/codex/blocktype as bt
|
||||||
import pkg/codex/manifest
|
import pkg/codex/manifest
|
||||||
|
|
||||||
import ./helpers
|
import ./helpers
|
||||||
|
import ./examples
|
||||||
|
|
||||||
checksuite "Manifest":
|
checksuite "Manifest":
|
||||||
test "Should produce valid tree hash checksum":
|
|
||||||
var manifest = Manifest.new(
|
|
||||||
blocks = @[
|
|
||||||
Block.new("Block 1".toBytes).tryGet().cid,
|
|
||||||
Block.new("Block 2".toBytes).tryGet().cid,
|
|
||||||
Block.new("Block 3".toBytes).tryGet().cid,
|
|
||||||
Block.new("Block 4".toBytes).tryGet().cid,
|
|
||||||
Block.new("Block 5".toBytes).tryGet().cid,
|
|
||||||
Block.new("Block 6".toBytes).tryGet().cid,
|
|
||||||
Block.new("Block 7".toBytes).tryGet().cid,
|
|
||||||
]).tryGet()
|
|
||||||
|
|
||||||
let
|
|
||||||
encoded = @[byte 18, 32, 227, 176, 196, 66, 152,
|
|
||||||
252, 28, 20, 154, 251, 244, 200, 153,
|
|
||||||
111, 185, 36, 39, 174, 65, 228, 100,
|
|
||||||
155, 147, 76, 164, 149, 153, 27, 120,
|
|
||||||
82, 184, 85]
|
|
||||||
|
|
||||||
var mh: MultiHash
|
|
||||||
check MultiHash.decode(encoded, mh).tryGet() > 0
|
|
||||||
|
|
||||||
let encodedCid = Cid.init(manifest.version, manifest.codec, mh).tryGet()
|
|
||||||
check:
|
|
||||||
encodedCid == manifest.cid.tryGet()
|
|
||||||
|
|
||||||
test "Should encode/decode to/from manifest":
|
test "Should encode/decode to/from manifest":
|
||||||
let
|
|
||||||
blocks = (0..<1000).mapIt(
|
|
||||||
Block.new(("Block " & $it).toBytes).tryGet().cid
|
|
||||||
)
|
|
||||||
|
|
||||||
var
|
var
|
||||||
manifest = Manifest.new(blocks).tryGet()
|
manifest = Manifest.new(
|
||||||
|
treeCid = Cid.example,
|
||||||
|
blockSize = 1.MiBs,
|
||||||
|
datasetSize = 100.MiBs)
|
||||||
|
|
||||||
let
|
let
|
||||||
e = manifest.encode().tryGet()
|
e = manifest.encode().tryGet()
|
||||||
decoded = Manifest.decode(e).tryGet()
|
decoded = Manifest.decode(e).tryGet()
|
||||||
|
|
||||||
check:
|
check:
|
||||||
decoded.blocks == blocks
|
decoded == manifest
|
||||||
decoded.protected == false
|
|
||||||
|
|
||||||
test "Should produce a protected manifest":
|
|
||||||
let
|
|
||||||
blocks = (0..<333).mapIt(
|
|
||||||
Block.new(("Block " & $it).toBytes).tryGet().cid
|
|
||||||
)
|
|
||||||
manifest = Manifest.new(blocks).tryGet()
|
|
||||||
|
|
||||||
var
|
|
||||||
protected = Manifest.new(manifest, 2, 2).tryGet()
|
|
||||||
|
|
||||||
check:
|
|
||||||
protected.originalCid == manifest.cid.tryGet()
|
|
||||||
protected.blocks[0..<333] == manifest.blocks
|
|
||||||
protected.protected == true
|
|
||||||
protected.originalLen == manifest.len
|
|
||||||
|
|
||||||
# fill up with empty Cid's
|
|
||||||
for i in protected.rounded..<protected.len:
|
|
||||||
protected[i] = EmptyCid[manifest.version]
|
|
||||||
.catch
|
|
||||||
.get()[manifest.hcodec]
|
|
||||||
.catch
|
|
||||||
.get()
|
|
||||||
|
|
||||||
var
|
|
||||||
encoded = protected.encode().tryGet()
|
|
||||||
decoded = Manifest.decode(encoded).tryGet()
|
|
||||||
|
|
||||||
check:
|
|
||||||
decoded.protected == true
|
|
||||||
decoded.originalLen == manifest.len
|
|
||||||
|
|
||||||
decoded.ecK == protected.ecK
|
|
||||||
decoded.ecM == protected.ecM
|
|
||||||
|
|
||||||
decoded.originalCid == protected.originalCid
|
|
||||||
decoded.originalCid == manifest.cid.tryGet()
|
|
||||||
|
|
||||||
decoded.blocks == protected.blocks
|
|
||||||
decoded.blocks[0..<333] == manifest.blocks
|
|
||||||
|
|
|
@ -1,3 +1,4 @@
|
||||||
import ./merkletree/testmerkletree
|
import ./merkletree/testmerkletree
|
||||||
|
import ./merkletree/testcoders
|
||||||
|
|
||||||
{.warning[UnusedImport]: off.}
|
{.warning[UnusedImport]: off.}
|
||||||
|
|
|
@ -41,18 +41,7 @@ asyncchecksuite "Test Node":
|
||||||
|
|
||||||
proc fetch(T: type Manifest, chunker: Chunker): Future[Manifest] {.async.} =
|
proc fetch(T: type Manifest, chunker: Chunker): Future[Manifest] {.async.} =
|
||||||
# Collect blocks from Chunker into Manifest
|
# Collect blocks from Chunker into Manifest
|
||||||
var
|
await storeDataGetManifest(localStore, chunker)
|
||||||
manifest = Manifest.new().tryGet()
|
|
||||||
|
|
||||||
while (
|
|
||||||
let chunk = await chunker.getBytes();
|
|
||||||
chunk.len > 0):
|
|
||||||
|
|
||||||
let blk = bt.Block.new(chunk).tryGet()
|
|
||||||
(await localStore.putBlock(blk)).tryGet()
|
|
||||||
manifest.add(blk.cid)
|
|
||||||
|
|
||||||
return manifest
|
|
||||||
|
|
||||||
proc retrieve(cid: Cid): Future[seq[byte]] {.async.} =
|
proc retrieve(cid: Cid): Future[seq[byte]] {.async.} =
|
||||||
# Retrieve an entire file contents by file Cid
|
# Retrieve an entire file contents by file Cid
|
||||||
|
@ -113,8 +102,7 @@ asyncchecksuite "Test Node":
|
||||||
fetched = (await node.fetchManifest(manifestBlock.cid)).tryGet()
|
fetched = (await node.fetchManifest(manifestBlock.cid)).tryGet()
|
||||||
|
|
||||||
check:
|
check:
|
||||||
fetched.cid == manifest.cid
|
fetched == manifest
|
||||||
fetched.blocks == manifest.blocks
|
|
||||||
|
|
||||||
test "Block Batching":
|
test "Block Batching":
|
||||||
let
|
let
|
||||||
|
@ -159,7 +147,7 @@ asyncchecksuite "Test Node":
|
||||||
let data = await retrieve(manifestCid)
|
let data = await retrieve(manifestCid)
|
||||||
|
|
||||||
check:
|
check:
|
||||||
data.len == localManifest.originalBytes.int
|
data.len == localManifest.datasetSize.int
|
||||||
data.len == original.len
|
data.len == original.len
|
||||||
sha256.digest(data) == sha256.digest(original)
|
sha256.digest(data) == sha256.digest(original)
|
||||||
|
|
||||||
|
|
|
@ -23,32 +23,26 @@ asyncchecksuite "StoreStream":
|
||||||
return true
|
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],
|
10, 11, 12, 13, 14, 15, 16, 17, 18, 19,
|
||||||
[byte 10, 11, 12, 13, 14, 15, 16, 17, 18, 19],
|
20, 21, 22, 23, 24, 25, 26, 27, 28, 29,
|
||||||
[byte 20, 21, 22, 23, 24, 25, 26, 27, 28, 29],
|
30, 31, 32, 33, 34, 35, 36, 37, 38, 39,
|
||||||
[byte 30, 31, 32, 33, 34, 35, 36, 37, 38, 39],
|
40, 41, 42, 43, 44, 45, 46, 47, 48, 49,
|
||||||
[byte 40, 41, 42, 43, 44, 45, 46, 47, 48, 49],
|
50, 51, 52, 53, 54, 55, 56, 57, 58, 59,
|
||||||
[byte 50, 51, 52, 53, 54, 55, 56, 57, 58, 59],
|
60, 61, 62, 63, 64, 65, 66, 67, 68, 69,
|
||||||
[byte 60, 61, 62, 63, 64, 65, 66, 67, 68, 69],
|
70, 71, 72, 73, 74, 75, 76, 77, 78, 79,
|
||||||
[byte 70, 71, 72, 73, 74, 75, 76, 77, 78, 79],
|
80, 81, 82, 83, 84, 85, 86, 87, 88, 89,
|
||||||
[byte 80, 81, 82, 83, 84, 85, 86, 87, 88, 89],
|
90, 91, 92, 93, 94, 95, 96, 97, 98, 99]
|
||||||
[byte 90, 91, 92, 93, 94, 95, 96, 97, 98, 99],
|
chunkSize = 10
|
||||||
]
|
|
||||||
|
|
||||||
teardown:
|
teardown:
|
||||||
await stream.close()
|
await stream.close()
|
||||||
|
|
||||||
setup:
|
setup:
|
||||||
store = CacheStore.new()
|
store = CacheStore.new()
|
||||||
manifest = Manifest.new(blockSize = 10'nb).tryGet()
|
manifest = await storeDataGetManifest(store, MockChunker.new(dataset = data, chunkSize = chunkSize))
|
||||||
stream = StoreStream.new(store, manifest)
|
stream = StoreStream.new(store, manifest)
|
||||||
|
|
||||||
for d in data:
|
|
||||||
let blk = bt.Block.new(d).tryGet()
|
|
||||||
manifest.add(blk.cid)
|
|
||||||
(await store.putBlock(blk)).tryGet()
|
|
||||||
|
|
||||||
test "Read all blocks < blockSize":
|
test "Read all blocks < blockSize":
|
||||||
var
|
var
|
||||||
buf = newSeq[byte](8)
|
buf = newSeq[byte](8)
|
||||||
|
|
Loading…
Reference in New Issue