Build slots (#668)

Wiring in slots builder functionality into `requestStorage`
This commit is contained in:
Dmitriy Ryajov 2024-01-11 10:45:23 -06:00 committed by GitHub
parent 469d594463
commit 8b12934fe2
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
25 changed files with 680 additions and 359 deletions

View File

@ -44,7 +44,6 @@ type
else: else:
cid*: Cid cid*: Cid
proc `==`*(a, b: BlockAddress): bool = proc `==`*(a, b: BlockAddress): bool =
a.leaf == b.leaf and a.leaf == b.leaf and
( (

View File

@ -39,7 +39,7 @@ const
BlockCodec* = multiCodec("codex-block") BlockCodec* = multiCodec("codex-block")
SlotRootCodec* = multiCodec("codex-slot-root") SlotRootCodec* = multiCodec("codex-slot-root")
SlotProvingRootCodec* = multiCodec("codex-proving-root") SlotProvingRootCodec* = multiCodec("codex-proving-root")
CodexSlotCell* = multiCodec("codex-slot-cell") CodexSlotCellCodec* = multiCodec("codex-slot-cell")
CodexHashesCodecs* = [ CodexHashesCodecs* = [
Sha256HashCodec, Sha256HashCodec,
@ -53,7 +53,7 @@ const
BlockCodec, BlockCodec,
SlotRootCodec, SlotRootCodec,
SlotProvingRootCodec, SlotProvingRootCodec,
CodexSlotCell, CodexSlotCellCodec,
] ]
proc initEmptyCidTable(): ?!Table[(CidVersion, MultiCodec, MultiCodec), Cid] = proc initEmptyCidTable(): ?!Table[(CidVersion, MultiCodec, MultiCodec), Cid] =

View File

@ -72,11 +72,11 @@ type
store*: BlockStore store*: BlockStore
EncodingParams = object EncodingParams = object
ecK: int ecK: Natural
ecM: int ecM: Natural
rounded: int rounded: Natural
steps: int steps: Natural
blocksCount: int blocksCount: Natural
func indexToPos(steps, 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
@ -112,7 +112,8 @@ proc getPendingBlocks(
return await completedFut return await completedFut
else: else:
let (_, index) = await completedFut let (_, index) = await completedFut
raise newException(CatchableError, "Future for block id not found, tree cid: " & $manifest.treeCid & ", index: " & $index) raise newException(CatchableError,
"Future for block id not found, tree cid: " & $manifest.treeCid & ", index: " & $index)
Iter.new(genNext, isFinished) Iter.new(genNext, isFinished)
@ -120,10 +121,10 @@ proc prepareEncodingData(
self: Erasure, self: Erasure,
manifest: Manifest, manifest: Manifest,
params: EncodingParams, params: EncodingParams,
step: int, step: Natural,
data: ref seq[seq[byte]], data: ref seq[seq[byte]],
cids: ref seq[Cid], cids: ref seq[Cid],
emptyBlock: seq[byte]): Future[?!int] {.async.} = emptyBlock: seq[byte]): Future[?!Natural] {.async.} =
## Prepare data for encoding ## Prepare data for encoding
## ##
@ -157,16 +158,16 @@ proc prepareEncodingData(
return failure(err) return failure(err)
cids[idx] = emptyBlockCid cids[idx] = emptyBlockCid
success(resolved) success(resolved.Natural)
proc prepareDecodingData( proc prepareDecodingData(
self: Erasure, self: Erasure,
encoded: Manifest, encoded: Manifest,
step: int, step: Natural,
data: ref seq[seq[byte]], data: ref seq[seq[byte]],
parityData: ref seq[seq[byte]], parityData: ref seq[seq[byte]],
cids: ref seq[Cid], cids: ref seq[Cid],
emptyBlock: seq[byte]): Future[?!(int, int)] {.async.} = emptyBlock: seq[byte]): Future[?!(Natural, Natural)] {.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
@ -222,14 +223,18 @@ proc prepareDecodingData(
resolved.inc resolved.inc
return success (dataPieces, parityPieces) return success (dataPieces.Natural, parityPieces.Natural)
proc init*( proc init*(
_: type EncodingParams, _: type EncodingParams,
manifest: Manifest, manifest: Manifest,
ecK: int, ecM: int): ?!EncodingParams = ecK: Natural, ecM: Natural): ?!EncodingParams =
if ecK > manifest.blocksCount: if ecK > manifest.blocksCount:
return failure("Unable to encode manifest, not enough blocks, ecK = " & $ecK & ", blocksCount = " & $manifest.blocksCount) return failure(
"Unable to encode manifest, not enough blocks, ecK = " &
$ecK &
", blocksCount = " &
$manifest.blocksCount)
let let
rounded = roundUp(manifest.blocksCount, ecK) rounded = roundUp(manifest.blocksCount, ecK)
@ -338,8 +343,8 @@ proc encodeData(
proc encode*( proc encode*(
self: Erasure, self: Erasure,
manifest: Manifest, manifest: Manifest,
blocks: int, blocks: Natural,
parity: int): Future[?!Manifest] {.async.} = parity: Natural): Future[?!Manifest] {.async.} =
## Encode a manifest into one that is erasure protected. ## Encode a manifest into one that is erasure protected.
## ##
## `manifest` - the original manifest to be encoded ## `manifest` - the original manifest to be encoded
@ -347,7 +352,7 @@ proc encode*(
## `parity` - the number of parity blocks to generate - M ## `parity` - the number of parity blocks to generate - M
## ##
without params =? EncodingParams.init(manifest, blocks, parity), err: without params =? EncodingParams.init(manifest, blocks.int, parity.int), err:
return failure(err) return failure(err)
without encodedManifest =? await self.encodeData(manifest, params), err: without encodedManifest =? await self.encodeData(manifest, params), err:
@ -373,7 +378,7 @@ proc decode*(
var var
cids = seq[Cid].new() cids = seq[Cid].new()
recoveredIndices = newSeq[int]() recoveredIndices = newSeq[Natural]()
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)
@ -443,7 +448,7 @@ proc decode*(
let idxIter = Iter let idxIter = Iter
.fromItems(recoveredIndices) .fromItems(recoveredIndices)
.filter((i: int) => i < tree.leavesCount) .filter((i: Natural) => i < tree.leavesCount)
if err =? (await self.store.putSomeProofs(tree, idxIter)).errorOption: if err =? (await self.store.putSomeProofs(tree, idxIter)).errorOption:
return failure(err) return failure(err)

View File

@ -31,7 +31,8 @@ template mapFailure*[T, V, E](
template mapFailure*[T, V](exp: Result[T, V]): Result[T, ref CatchableError] = template mapFailure*[T, V](exp: Result[T, V]): Result[T, ref CatchableError] =
mapFailure(exp, CodexError) mapFailure(exp, CodexError)
template toResult*[T](exp: Option[T]): Result[T, ref CatchableError] = # TODO: using a template here, causes bad codegen
func toFailure*[T](exp: Option[T]): Result[T, ref CatchableError] {.inline.} =
if exp.isSome: if exp.isSome:
success exp.get success exp.get
else: else:

View File

@ -39,7 +39,7 @@ proc encode*(manifest: Manifest): ?!seq[byte] =
# #
# ```protobuf # ```protobuf
# Message VerificationInfo { # Message VerificationInfo {
# bytes verificationRoot = 1; # Decimal encoded field-element # bytes verifyRoot = 1; # Decimal encoded field-element
# repeated bytes slotRoots = 2; # Decimal encoded field-elements # repeated bytes slotRoots = 2; # Decimal encoded field-elements
# } # }
# Message ErasureInfo { # Message ErasureInfo {
@ -78,7 +78,7 @@ proc encode*(manifest: Manifest): ?!seq[byte] =
if manifest.verifiable: if manifest.verifiable:
var verificationInfo = initProtoBuffer() var verificationInfo = initProtoBuffer()
verificationInfo.write(1, manifest.verificationRoot.data.buffer) verificationInfo.write(1, manifest.verifyRoot.data.buffer)
for slotRoot in manifest.slotRoots: for slotRoot in manifest.slotRoots:
verificationInfo.write(2, slotRoot.data.buffer) verificationInfo.write(2, slotRoot.data.buffer)
erasureInfo.write(5, verificationInfo) erasureInfo.write(5, verificationInfo)
@ -109,7 +109,7 @@ proc decode*(_: type Manifest, data: openArray[byte]): ?!Manifest =
blockSize: uint32 blockSize: uint32
originalDatasetSize: uint32 originalDatasetSize: uint32
ecK, ecM: uint32 ecK, ecM: uint32
verificationRoot: seq[byte] verifyRoot: seq[byte]
slotRoots: seq[seq[byte]] slotRoots: seq[seq[byte]]
# Decode `Header` message # Decode `Header` message
@ -158,8 +158,8 @@ proc decode*(_: type Manifest, data: openArray[byte]): ?!Manifest =
verifiable = pbVerificationInfo.buffer.len > 0 verifiable = pbVerificationInfo.buffer.len > 0
if verifiable: if verifiable:
if pbVerificationInfo.getField(1, verificationRoot).isErr: if pbVerificationInfo.getField(1, verifyRoot).isErr:
return failure("Unable to decode `verificationRoot` from manifest!") return failure("Unable to decode `verifyRoot` from manifest!")
if pbVerificationInfo.getRequiredRepeatedField(2, slotRoots).isErr: if pbVerificationInfo.getRequiredRepeatedField(2, slotRoots).isErr:
return failure("Unable to decode `slotRoots` from manifest!") return failure("Unable to decode `slotRoots` from manifest!")
@ -193,12 +193,12 @@ proc decode*(_: type Manifest, data: openArray[byte]): ?!Manifest =
if verifiable: if verifiable:
let let
verificationRootCid = ? Cid.init(verificationRoot).mapFailure verifyRootCid = ? Cid.init(verifyRoot).mapFailure
slotRootCids = slotRoots.mapIt(? Cid.init(it).mapFailure) slotRootCids = slotRoots.mapIt(? Cid.init(it).mapFailure)
return Manifest.new( return Manifest.new(
manifest = self, manifest = self,
verificationRoot = verificationRootCid, verifyRoot = verifyRootCid,
slotRoots = slotRootCids slotRoots = slotRootCids
) )

View File

@ -39,8 +39,8 @@ type
originalDatasetSize: NBytes originalDatasetSize: NBytes
case verifiable {.serialize.}: bool # Verifiable datasets can be used to generate storage proofs case verifiable {.serialize.}: bool # Verifiable datasets can be used to generate storage proofs
of true: of true:
verificationRoot: Cid verifyRoot: Cid # Root of the top level merkle tree built from slot roots
slotRoots: seq[Cid] slotRoots: seq[Cid] # Individual slot root built from the original dataset blocks
else: else:
discard discard
else: else:
@ -92,8 +92,8 @@ proc blocksCount*(self: Manifest): int =
proc verifiable*(self: Manifest): bool = proc verifiable*(self: Manifest): bool =
self.verifiable self.verifiable
proc verificationRoot*(self: Manifest): Cid = proc verifyRoot*(self: Manifest): Cid =
self.verificationRoot self.verifyRoot
proc slotRoots*(self: Manifest): seq[Cid] = proc slotRoots*(self: Manifest): seq[Cid] =
self.slotRoots self.slotRoots
@ -103,6 +103,7 @@ proc numSlots*(self: Manifest): int =
0 0
else: else:
self.ecK + self.ecM self.ecK + self.ecM
############################################################ ############################################################
# Operations on block list # Operations on block list
############################################################ ############################################################
@ -159,7 +160,7 @@ proc `==`*(a, b: Manifest): bool =
(a.originalDatasetSize == b.originalDatasetSize) and (a.originalDatasetSize == b.originalDatasetSize) and
(a.verifiable == b.verifiable) and (a.verifiable == b.verifiable) and
(if a.verifiable: (if a.verifiable:
(a.verificationRoot == b.verificationRoot) and (a.verifyRoot == b.verifyRoot) and
(a.slotRoots == b.slotRoots) (a.slotRoots == b.slotRoots)
else: else:
true) true)
@ -181,7 +182,7 @@ proc `$`*(self: Manifest): string =
", originalDatasetSize: " & $self.originalDatasetSize & ", originalDatasetSize: " & $self.originalDatasetSize &
", verifiable: " & $self.verifiable & ", verifiable: " & $self.verifiable &
(if self.verifiable: (if self.verifiable:
", verificationRoot: " & $self.verificationRoot & ", verifyRoot: " & $self.verifyRoot &
", slotRoots: " & $self.slotRoots ", slotRoots: " & $self.slotRoots
else: else:
"") "")
@ -287,15 +288,21 @@ proc new*(
proc new*( proc new*(
T: type Manifest, T: type Manifest,
manifest: Manifest, manifest: Manifest,
verificationRoot: Cid, verifyRoot: Cid,
slotRoots: seq[Cid]): ?!Manifest = slotRoots: openArray[Cid]): ?!Manifest =
## Create a verifiable dataset from an ## Create a verifiable dataset from an
## protected one ## protected one
## ##
if not manifest.protected:
return failure newException(CodexError, "Can create verifiable manifest only from protected manifest.")
success(Manifest( if not manifest.protected:
return failure newException(
CodexError, "Can create verifiable manifest only from protected manifest.")
if slotRoots.len != manifest.numSlots:
return failure newException(
CodexError, "Wrong number of slot roots.")
success Manifest(
treeCid: manifest.treeCid, treeCid: manifest.treeCid,
datasetSize: manifest.datasetSize, datasetSize: manifest.datasetSize,
version: manifest.version, version: manifest.version,
@ -308,5 +315,5 @@ proc new*(
originalTreeCid: manifest.treeCid, originalTreeCid: manifest.treeCid,
originalDatasetSize: manifest.originalDatasetSize, originalDatasetSize: manifest.originalDatasetSize,
verifiable: true, verifiable: true,
verificationRoot: verificationRoot, verifyRoot: verifyRoot,
slotRoots: slotRoots)) slotRoots: @slotRoots)

View File

@ -3,3 +3,8 @@ import ./merkletree/codex
import ./merkletree/poseidon2 import ./merkletree/poseidon2
export codex, poseidon2, merkletree export codex, poseidon2, merkletree
type
SomeMerkleTree* = ByteTree | CodexTree | Poseidon2Tree
SomeMerkleProof* = ByteProof | CodexProof | Poseidon2Proof
SomeMerkleHash* = ByteHash | Poseidon2Hash

View File

@ -23,6 +23,8 @@ import ../../rng
import ../../errors import ../../errors
import ../../blocktype import ../../blocktype
from ../../utils/digest import digestBytes
import ../merkletree import ../merkletree
export merkletree export merkletree
@ -62,12 +64,6 @@ func digestSize*(self: (CodexTree or CodexProof)): int =
self.mhash.size self.mhash.size
func digestBytes*(mhash: MultiHash): seq[byte] =
## Extract hash digestBytes
##
mhash.data.buffer[mhash.dpos..<mhash.dpos + mhash.size]
func getProof*(self: CodexTree, index: int): ?!CodexProof = func getProof*(self: CodexTree, index: int): ?!CodexProof =
var var
proof = CodexProof(mcodec: self.mcodec) proof = CodexProof(mcodec: self.mcodec)

View File

@ -16,6 +16,7 @@ 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/poseidon2
import pkg/libp2p/[switch, multicodec, multihash] import pkg/libp2p/[switch, multicodec, multihash]
import pkg/libp2p/stream/bufferstream import pkg/libp2p/stream/bufferstream
@ -25,6 +26,7 @@ import pkg/libp2p/routing_record
import pkg/libp2p/signed_envelope import pkg/libp2p/signed_envelope
import ./chunker import ./chunker
import ./slots
import ./clock import ./clock
import ./blocktype as bt import ./blocktype as bt
import ./manifest import ./manifest
@ -38,6 +40,7 @@ import ./contracts
import ./node/batch import ./node/batch
import ./utils import ./utils
import ./errors import ./errors
import ./merkletree
export batch export batch
@ -56,17 +59,47 @@ type
validator: ?ValidatorInteractions validator: ?ValidatorInteractions
CodexNodeRef* = ref object CodexNodeRef* = ref object
switch*: Switch switch: Switch
networkId*: PeerId networkId: PeerId
blockStore*: BlockStore blockStore: BlockStore
engine*: BlockExcEngine engine: BlockExcEngine
erasure*: Erasure erasure: Erasure
discovery*: Discovery discovery: Discovery
contracts*: Contracts contracts*: Contracts
clock*: Clock clock*: Clock
OnManifest* = proc(cid: Cid, manifest: Manifest): void {.gcsafe, closure.} OnManifest* = proc(cid: Cid, manifest: Manifest): void {.gcsafe, closure.}
func switch*(self: CodexNodeRef): Switch =
return self.switch
func blockStore*(self: CodexNodeRef): BlockStore =
return self.blockStore
func engine*(self: CodexNodeRef): BlockExcEngine =
return self.engine
func erasure*(self: CodexNodeRef): Erasure =
return self.erasure
func discovery*(self: CodexNodeRef): Discovery =
return self.discovery
proc storeManifest*(self: CodexNodeRef, manifest: Manifest): Future[?!bt.Block] {.async.} =
without encodedVerifiable =? manifest.encode(), err:
trace "Unable to encode manifest"
return failure(err)
without blk =? bt.Block.new(data = encodedVerifiable, codec = ManifestCodec), error:
trace "Unable to create block from manifest"
return failure(error)
if err =? (await self.blockStore.putBlock(blk)).errorOption:
trace "Unable to store manifest block", cid = blk.cid, err = err.msg
return failure(err)
success blk
proc findPeer*( proc findPeer*(
node: CodexNodeRef, node: CodexNodeRef,
peerId: PeerId): Future[?PeerRecord] {.async.} = peerId: PeerId): Future[?PeerRecord] {.async.} =
@ -234,7 +267,6 @@ proc store*(
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}")
except CancelledError as exc: except CancelledError as exc:
raise exc raise exc
except CatchableError as exc: except CatchableError as exc:
@ -261,21 +293,11 @@ proc store*(
datasetSize = NBytes(chunker.offset), datasetSize = NBytes(chunker.offset),
version = CIDv1, version = CIDv1,
hcodec = hcodec, hcodec = hcodec,
codec = dataCodec codec = dataCodec)
)
# Generate manifest
without data =? manifest.encode(), err:
return failure(
newException(CodexError, "Error encoding manifest: " & err.msg))
# Store as a dag-pb block without manifestBlk =? await self.storeManifest(manifest), err:
without manifestBlk =? bt.Block.new(data = data, codec = ManifestCodec): trace "Unable to store manifest"
trace "Unable to init block from manifest data!" return failure(err)
return failure("Unable to init block from manifest data!")
if isErr (await self.blockStore.putBlock(manifestBlk)):
trace "Unable to store manifest", cid = manifestBlk.cid
return failure("Unable to store manifest " & $manifestBlk.cid)
info "Stored data", manifestCid = manifestBlk.cid, info "Stored data", manifestCid = manifestBlk.cid,
treeCid = treeCid, treeCid = treeCid,
@ -305,6 +327,91 @@ proc iterateManifests*(node: CodexNodeRef, onManifest: OnManifest) {.async.} =
onManifest(cid, manifest) onManifest(cid, manifest)
proc setupRequest(
self: CodexNodeRef,
cid: Cid,
duration: UInt256,
proofProbability: UInt256,
nodes: uint,
tolerance: uint,
reward: UInt256,
collateral: UInt256,
expiry: UInt256): Future[?!StorageRequest] {.async.} =
## Setup slots for a given dataset
##
let
ecK = nodes - tolerance
ecM = tolerance
logScope:
cid = cid
duration = duration
nodes = nodes
tolerance = tolerance
reward = reward
proofProbability = proofProbability
collateral = collateral
expiry = expiry
ecK = ecK
ecM = ecM
trace "Setting up slots"
without manifest =? await self.fetchManifest(cid), error:
trace "Unable to fetch manifest for cid"
return failure error
# Erasure code the dataset according to provided parameters
without encoded =? (await self.erasure.encode(manifest, ecK, ecM)), error:
trace "Unable to erasure code dataset"
return failure(error)
without builder =? SlotsBuilder.new(self.blockStore, encoded), err:
trace "Unable to create slot builder"
return failure(err)
without verifiable =? (await builder.buildManifest()), err:
trace "Unable to build verifiable manifest"
return failure(err)
without manifestBlk =? await self.storeManifest(verifiable), err:
trace "Unable to store verifiable manifest"
return failure(err)
let
verifyRoot =
if builder.verifyRoot.isNone:
return failure("No slots root")
else:
builder.verifyRoot.get.toBytes
slotRoots =
if builder.slotRoots.len <= 0:
return failure("Slots are empty")
else:
builder.slotRoots.mapIt( it.toBytes )
request = StorageRequest(
ask: StorageAsk(
slots: verifiable.numSlots.uint64,
slotSize: builder.slotBytes.uint.u256,
duration: duration,
proofProbability: proofProbability,
reward: reward,
collateral: collateral,
maxSlotLoss: tolerance
),
content: StorageContent(
cid: $manifestBlk.cid, # TODO: why string?
merkleRoot: verifyRoot
),
expiry: expiry
)
trace "Request created", request = $request
success request
proc requestStorage*( proc requestStorage*(
self: CodexNodeRef, self: CodexNodeRef,
cid: Cid, cid: Cid,
@ -318,64 +425,38 @@ proc requestStorage*(
## Initiate a request for storage sequence, this might ## Initiate a request for storage sequence, this might
## be a multistep procedure. ## be a multistep procedure.
## ##
## Roughly the flow is as follows:
## - Get the original cid from the store (should have already been uploaded) logScope:
## - Erasure code it according to the nodes and tolerance parameters cid = cid
## - Run the PoR setup on the erasure dataset duration = duration
## - Call into the marketplace and purchasing contracts nodes = nodes
## tolerance = tolerance
trace "Received a request for storage!", cid, duration, nodes, tolerance, reward, proofProbability, collateral, expiry reward = reward
proofProbability = proofProbability
collateral = collateral
expiry = expiry
trace "Received a request for storage!"
without contracts =? self.contracts.client: without contracts =? self.contracts.client:
trace "Purchasing not available" trace "Purchasing not available"
return failure "Purchasing not available" return failure "Purchasing not available"
without manifest =? await self.fetchManifest(cid), error: without request =?
trace "Unable to fetch manifest for cid", cid (await self.setupRequest(
raise error cid,
duration,
# Erasure code the dataset according to provided parameters proofProbability,
without encoded =? (await self.erasure.encode(manifest, nodes.int, tolerance.int)), error: nodes,
trace "Unable to erasure code dataset", cid tolerance,
return failure(error) reward,
collateral,
without encodedData =? encoded.encode(), error: expiry)), err:
trace "Unable to encode protected manifest" trace "Unable to setup request"
return failure(error) return failure err
without encodedBlk =? bt.Block.new(data = encodedData, codec = ManifestCodec), error:
trace "Unable to create block from encoded manifest"
return failure(error)
if isErr (await self.blockStore.putBlock(encodedBlk)):
trace "Unable to store encoded manifest block", cid = encodedBlk.cid
return failure("Unable to store encoded manifest block")
let request = StorageRequest(
ask: StorageAsk(
slots: nodes + tolerance,
# TODO: Specify slot-specific size (as below) once dispersal is
# implemented. The current implementation downloads the entire dataset, so
# it is currently set to be the size of the entire dataset. This is
# because the slotSize is used to determine the amount of bytes to reserve
# in a Reservations
# TODO: slotSize: (encoded.blockSize.int * encoded.steps).u256,
slotSize: (encoded.blockSize.int * encoded.blocksCount).u256,
duration: duration,
proofProbability: proofProbability,
reward: reward,
collateral: collateral,
maxSlotLoss: tolerance
),
content: StorageContent(
cid: $encodedBlk.cid,
merkleRoot: array[32, byte].default # TODO: add merkle root for storage proofs
),
expiry: expiry
)
let purchase = await contracts.purchasing.purchase(request) let purchase = await contracts.purchasing.purchase(request)
return success purchase.id success purchase.id
proc new*( proc new*(
T: type CodexNodeRef, T: type CodexNodeRef,

View File

@ -33,7 +33,8 @@ func new*(
## create a new instance of a Purchase ## create a new instance of a Purchase
## ##
var purchase = Purchase.new() var purchase = Purchase.new()
purchase.future = Future[void].new() {.cast(noSideEffect).}:
purchase.future = newFuture[void]()
purchase.requestId = requestId purchase.requestId = requestId
purchase.market = market purchase.market = market
purchase.clock = clock purchase.clock = clock

View File

@ -300,8 +300,8 @@ proc initPurchasingApi(node: CodexNodeRef, router: var RestRouter) =
## proofProbability - how often storage proofs are required ## proofProbability - how often storage proofs are required
## reward - the maximum amount of tokens paid per second per slot to hosts the client is willing to pay ## reward - the maximum amount of tokens paid per second per slot to hosts the client is willing to pay
## expiry - timestamp, in seconds, when the request expires if the Request does not find requested amount of nodes to host the data ## expiry - timestamp, in seconds, when the request expires if the Request does not find requested amount of nodes to host the data
## nodes - minimal number of nodes the content should be stored on ## nodes - number of nodes the content should be stored on
## tolerance - allowed number of nodes that can be lost before pronouncing the content lost ## tolerance - allowed number of nodes that can be lost before content is lost
## colateral - requested collateral from hosts when they fill slot ## colateral - requested collateral from hosts when they fill slot
try: try:
@ -319,6 +319,9 @@ proc initPurchasingApi(node: CodexNodeRef, router: var RestRouter) =
let nodes = params.nodes |? 1 let nodes = params.nodes |? 1
let tolerance = params.tolerance |? 0 let tolerance = params.tolerance |? 0
if (nodes - tolerance) < 1:
return RestApiResponse.error(Http400, "Tolerance cannot be greater or equal than nodes (nodes - tolerance)")
without expiry =? params.expiry: without expiry =? params.expiry:
return RestApiResponse.error(Http400, "Expiry required") return RestApiResponse.error(Http400, "Expiry required")

3
codex/slots.nim Normal file
View File

@ -0,0 +1,3 @@
import ./slots/builder
export builder

View File

@ -19,6 +19,7 @@ import pkg/chronicles
import pkg/questionable/results import pkg/questionable/results
import pkg/poseidon2 import pkg/poseidon2
import pkg/poseidon2/io import pkg/poseidon2/io
import pkg/constantine/math/arithmetic/finite_fields
import ../indexingstrategy import ../indexingstrategy
import ../merkletree import ../merkletree
@ -26,51 +27,90 @@ import ../stores
import ../manifest import ../manifest
import ../utils import ../utils
import ../utils/digest import ../utils/digest
import ../utils/poseidon2digest
const const
# TODO: Unified with the CellSize specified in branch "data-sampler" # TODO: Unified with the CellSize specified in branch "data-sampler"
# Number of bytes in a cell. A cell is the smallest unit of data used
# in the proving circuit. # in the proving circuit.
CellSize* = 2048 CellSize* = 2048
DefaultEmptyBlock* = newSeq[byte](DefaultBlockSize.int)
DefaultEmptyCell* = newSeq[byte](DefaultCellSize.int)
type type
SlotBuilder* = object of RootObj # TODO: should be a generic type that
# supports all merkle trees
SlotsBuilder* = ref object of RootObj
store: BlockStore store: BlockStore
manifest: Manifest manifest: Manifest
strategy: IndexingStrategy strategy: IndexingStrategy
cellSize: int cellSize: int
blockEmptyDigest: Poseidon2Hash
blockPadBytes: seq[byte] blockPadBytes: seq[byte]
slotsPadLeafs: seq[Poseidon2Hash] slotsPadLeafs: seq[Poseidon2Hash]
rootsPadLeafs: seq[Poseidon2Hash] rootsPadLeafs: seq[Poseidon2Hash]
slotRoots: seq[Poseidon2Hash]
verifyRoot: ?Poseidon2Hash
func numBlockPadBytes*(self: SlotBuilder): Natural = func slotRoots*(self: SlotsBuilder): seq[Poseidon2Hash] {.inline.} =
## Returns the slot roots.
##
self.slotRoots
func verifyRoot*(self: SlotsBuilder): ?Poseidon2Hash {.inline.} =
## Returns the slots root (verification root).
##
self.verifyRoot
func nextPowerOfTwoPad*(a: int): int =
## Returns the difference between the original
## value and the next power of two.
##
nextPowerOfTwo(a) - a
func numBlockPadBytes*(self: SlotsBuilder): Natural =
## Number of padding bytes required for a pow2 ## Number of padding bytes required for a pow2
## merkle tree for each block. ## merkle tree for each block.
## ##
self.blockPadBytes.len self.blockPadBytes.len
func numSlotsPadLeafs*(self: SlotBuilder): Natural = func numSlotsPadLeafs*(self: SlotsBuilder): Natural =
## Number of padding field elements required for a pow2 ## Number of padding field elements required for a pow2
## merkle tree for each slot. ## merkle tree for each slot.
## ##
self.slotsPadLeafs.len self.slotsPadLeafs.len
func numRootsPadLeafs*(self: SlotBuilder): Natural = func numRootsPadLeafs*(self: SlotsBuilder): Natural =
## Number of padding field elements required for a pow2 ## Number of padding field elements required for a pow2
## merkle tree for the slot roots. ## merkle tree for the slot roots.
## ##
self.rootsPadLeafs.len self.rootsPadLeafs.len
func numSlotBlocks*(self: SlotBuilder): Natural = func numSlots*(self: SlotsBuilder): Natural =
## Number of slots.
##
self.manifest.numSlots
func numSlotBlocks*(self: SlotsBuilder): Natural =
## Number of blocks per slot. ## Number of blocks per slot.
## ##
self.manifest.blocksCount div self.manifest.numSlots self.manifest.blocksCount div self.manifest.numSlots
func numBlockRoots*(self: SlotBuilder): Natural = func slotBytes*(self: SlotsBuilder): NBytes =
## Number of bytes per slot.
##
(self.manifest.blockSize.int * self.numSlotBlocks).NBytes
func numBlockCells*(self: SlotsBuilder): Natural =
## Number of cells per block. ## Number of cells per block.
## ##
@ -79,7 +119,7 @@ func numBlockRoots*(self: SlotBuilder): Natural =
func toCellCid*(cell: Poseidon2Hash): ?!Cid = func toCellCid*(cell: Poseidon2Hash): ?!Cid =
let let
cellMhash = ? MultiHash.init(Pos2Bn128MrklCodec, cell.toBytes).mapFailure cellMhash = ? MultiHash.init(Pos2Bn128MrklCodec, cell.toBytes).mapFailure
cellCid = ? Cid.init(CIDv1, CodexSlotCell, cellMhash).mapFailure cellCid = ? Cid.init(CIDv1, CodexSlotCellCodec, cellMhash).mapFailure
success cellCid success cellCid
@ -90,16 +130,16 @@ func toSlotCid*(root: Poseidon2Hash): ?!Cid =
success treeCid success treeCid
func toProvingCid*(root: Poseidon2Hash): ?!Cid = func toSlotCids*(slotRoots: openArray[Poseidon2Hash]): ?!seq[Cid] =
success slotRoots.mapIt( ? it.toSlotCid )
func toSlotsRootsCid*(root: Poseidon2Hash): ?!Cid =
let let
mhash = ? MultiHash.init($multiCodec("identity"), root.toBytes).mapFailure mhash = ? MultiHash.init($multiCodec("identity"), root.toBytes).mapFailure
treeCid = ? Cid.init(CIDv1, SlotProvingRootCodec, mhash).mapFailure treeCid = ? Cid.init(CIDv1, SlotProvingRootCodec, mhash).mapFailure
success treeCid success treeCid
func mapToSlotCids*(slotRoots: seq[Poseidon2Hash]): ?!seq[Cid] =
success slotRoots.mapIt( ? it.toSlotCid )
func toEncodableProof*( func toEncodableProof*(
proof: Poseidon2Proof): ?!CodexProof = proof: Poseidon2Proof): ?!CodexProof =
@ -120,13 +160,13 @@ func toVerifiableProof*(
index: proof.index, index: proof.index,
nleaves: proof.nleaves, nleaves: proof.nleaves,
path: proof.path.mapIt( path: proof.path.mapIt(
? Poseidon2Hash.fromBytes(it.toArray32).toResult ? Poseidon2Hash.fromBytes(it.toArray32).toFailure
)) ))
success verifiableProof success verifiableProof
proc getCellHashes*( proc getCellHashes*(
self: SlotBuilder, self: SlotsBuilder,
slotIndex: int): Future[?!seq[Poseidon2Hash]] {.async.} = slotIndex: int): Future[?!seq[Poseidon2Hash]] {.async.} =
let let
@ -150,16 +190,19 @@ proc getCellHashes*(
error "Failed to get block CID for tree at index" error "Failed to get block CID for tree at index"
return failure(err) return failure(err)
without digest =? Poseidon2Tree.digest(blk.data & self.blockPadBytes, self.cellSize), err: if blk.isEmpty:
error "Failed to create digest for block" self.blockEmptyDigest
return failure(err) else:
without digest =? Poseidon2Tree.digest(blk.data & self.blockPadBytes, self.cellSize), err:
error "Failed to create digest for block"
return failure(err)
digest digest
success hashes success hashes
proc buildSlotTree*( proc buildSlotTree*(
self: SlotBuilder, self: SlotsBuilder,
slotIndex: int): Future[?!Poseidon2Tree] {.async.} = slotIndex: int): Future[?!Poseidon2Tree] {.async.} =
without cellHashes =? (await self.getCellHashes(slotIndex)), err: without cellHashes =? (await self.getCellHashes(slotIndex)), err:
error "Failed to select slot blocks", err = err.msg error "Failed to select slot blocks", err = err.msg
@ -168,11 +211,17 @@ proc buildSlotTree*(
Poseidon2Tree.init(cellHashes & self.slotsPadLeafs) Poseidon2Tree.init(cellHashes & self.slotsPadLeafs)
proc buildSlot*( proc buildSlot*(
self: SlotBuilder, self: SlotsBuilder,
slotIndex: int): Future[?!Poseidon2Hash] {.async.} = slotIndex: int): Future[?!Poseidon2Hash] {.async.} =
## Build a slot tree and store it in the block store. ## Build a slot tree and store it in the block store.
## ##
logScope:
cid = self.manifest.treeCid
slotIndex = slotIndex
trace "Building slot tree"
without tree =? (await self.buildSlotTree(slotIndex)) and without tree =? (await self.buildSlotTree(slotIndex)) and
treeCid =? tree.root.?toSlotCid, err: treeCid =? tree.root.?toSlotCid, err:
error "Failed to build slot tree", err = err.msg error "Failed to build slot tree", err = err.msg
@ -196,54 +245,65 @@ proc buildSlot*(
tree.root() tree.root()
proc buildSlots*(self: SlotBuilder): Future[?!seq[Poseidon2Hash]] {.async.} = func buildRootsTree*(
let self: SlotsBuilder,
slotRoots: seq[Poseidon2Hash] = collect(newSeq): slotRoots: openArray[Poseidon2Hash]): ?!Poseidon2Tree =
Poseidon2Tree.init(@slotRoots & self.rootsPadLeafs)
proc buildSlots*(self: SlotsBuilder): Future[?!void] {.async.} =
## Build all slot trees and store them in the block store.
##
logScope:
cid = self.manifest.treeCid
blockCount = self.manifest.blocksCount
trace "Building slots"
if self.slotRoots.len == 0:
self.slotRoots = collect(newSeq):
for i in 0..<self.manifest.numSlots: for i in 0..<self.manifest.numSlots:
without root =? (await self.buildSlot(i)), err: without slotRoot =? (await self.buildSlot(i)), err:
error "Failed to build slot", err = err.msg, index = i error "Failed to build slot", err = err.msg, index = i
return failure(err) return failure(err)
root slotRoot
success slotRoots without root =? self.buildRootsTree(self.slotRoots).?root(), err:
error "Failed to build slot roots tree", err = err.msg
return failure(err)
func buildRootsTree*( if self.verifyRoot.isSome and self.verifyRoot.get != root: # TODO: `!=` doesn't work for SecretBool
self: SlotBuilder, return failure "Existing slots root doesn't match reconstructed root."
slotRoots: seq[Poseidon2Hash]): ?!Poseidon2Tree = else:
Poseidon2Tree.init(slotRoots & self.rootsPadLeafs) self.verifyRoot = some root
proc buildManifest*(self: SlotBuilder): Future[?!Manifest] {.async.} = success()
without slotRoots =? await self.buildSlots(), err: proc buildManifest*(self: SlotsBuilder): Future[?!Manifest] {.async.} =
if err =? (await self.buildSlots()).errorOption:
error "Failed to build slot roots", err = err.msg error "Failed to build slot roots", err = err.msg
return failure(err) return failure(err)
without provingRootCid =? self.buildRootsTree(slotRoots).?root.?toProvingCid, err: without rootCids =? self.slotRoots.toSlotCids(), err:
error "Failed to build proving tree", err = err.msg
return failure(err)
without rootCids =? slotRoots.mapToSlotCids(), err:
error "Failed to map slot roots to CIDs", err = err.msg error "Failed to map slot roots to CIDs", err = err.msg
return failure(err) return failure(err)
Manifest.new(self.manifest, provingRootCid, rootCids) without rootProvingCidRes =? self.verifyRoot.?toSlotsRootsCid() and
rootProvingCid =? rootProvingCidRes, err: # TODO: why doesn't `.?` unpack the result?
error "Failed to map slot roots to CIDs", err = err.msg
return failure(err)
func nextPowerOfTwoPad*(a: int): int = Manifest.new(self.manifest, rootProvingCid, rootCids)
## Returns the difference between the original
## value and the next power of two.
##
nextPowerOfTwo(a) - a
proc new*( proc new*(
T: type SlotBuilder, T: type SlotsBuilder,
store: BlockStore, store: BlockStore,
manifest: Manifest, manifest: Manifest,
strategy: IndexingStrategy = nil, strategy: ?IndexingStrategy = none IndexingStrategy,
cellSize = CellSize): ?!SlotBuilder = cellSize = CellSize): ?!SlotsBuilder =
if not manifest.protected: if not manifest.protected:
return failure("Can only create SlotBuilder using protected manifests.") return failure("Can only create SlotsBuilder using protected manifests.")
if (manifest.blocksCount mod manifest.numSlots) != 0: if (manifest.blocksCount mod manifest.numSlots) != 0:
return failure("Number of blocks must be divisable by number of slots.") return failure("Number of blocks must be divisable by number of slots.")
@ -252,11 +312,11 @@ proc new*(
return failure("Block size must be divisable by cell size.") return failure("Block size must be divisable by cell size.")
let let
strategy = if strategy == nil: strategy = if strategy.isNone:
SteppedIndexingStrategy.new( SteppedIndexingStrategy.new(
0, manifest.blocksCount - 1, manifest.numSlots) 0, manifest.blocksCount - 1, manifest.numSlots)
else: else:
strategy strategy.get
# all trees have to be padded to power of two # all trees have to be padded to power of two
numBlockCells = manifest.blockSize.int div cellSize # number of cells per block numBlockCells = manifest.blockSize.int div cellSize # number of cells per block
@ -264,12 +324,41 @@ proc new*(
numSlotLeafs = (manifest.blocksCount div manifest.numSlots) numSlotLeafs = (manifest.blocksCount div manifest.numSlots)
slotsPadLeafs = newSeqWith(numSlotLeafs.nextPowerOfTwoPad, Poseidon2Zero) # power of two padding for block roots slotsPadLeafs = newSeqWith(numSlotLeafs.nextPowerOfTwoPad, Poseidon2Zero) # power of two padding for block roots
rootsPadLeafs = newSeqWith(manifest.numSlots.nextPowerOfTwoPad, Poseidon2Zero) rootsPadLeafs = newSeqWith(manifest.numSlots.nextPowerOfTwoPad, Poseidon2Zero)
blockEmptyDigest = ? Poseidon2Tree.digest(DefaultEmptyBlock & blockPadBytes, CellSize)
success SlotBuilder( var self = SlotsBuilder(
store: store, store: store,
manifest: manifest, manifest: manifest,
strategy: strategy, strategy: strategy,
cellSize: cellSize, cellSize: cellSize,
blockPadBytes: blockPadBytes, blockPadBytes: blockPadBytes,
slotsPadLeafs: slotsPadLeafs, slotsPadLeafs: slotsPadLeafs,
rootsPadLeafs: rootsPadLeafs) rootsPadLeafs: rootsPadLeafs,
blockEmptyDigest: blockEmptyDigest)
if manifest.verifiable:
if manifest.slotRoots.len == 0 or manifest.slotRoots.len != manifest.numSlots:
return failure "Manifest is verifiable but slot r"
let
slotRoot = ? Poseidon2Hash.fromBytes(
( ? manifest.verifyRoot.mhash.mapFailure ).digestBytes.toArray32
).toFailure
slotRoots = manifest.slotRoots.mapIt(
? Poseidon2Hash.fromBytes(
( ? it.mhash.mapFailure ).digestBytes.toArray32
).toFailure
)
without root =? self.buildRootsTree(slotRoots).?root(), err:
error "Failed to build slot roots tree", err = err.msg
return failure(err)
if slotRoot != root:
return failure "Existing slots root doesn't match reconstructed root."
self.slotRoots = slotRoots
self.verifyRoot = some slotRoot
success self

View File

@ -1,53 +1,8 @@
## 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/poseidon2 from pkg/libp2p import MultiHash
import pkg/poseidon2/io
import pkg/questionable/results
import pkg/libp2p/multihash
import ../merkletree func digestBytes*(mhash: MultiHash): seq[byte] =
## Extract hash digestBytes
func digestTree*(
_: type Poseidon2Tree,
bytes: openArray[byte], chunkSize: int): ?!Poseidon2Tree =
## Hashes chunks of data with a sponge of rate 2, and combines the
## resulting chunk hashes in a merkle root.
## ##
var index = 0 mhash.data.buffer[mhash.dpos..<mhash.dpos + mhash.size]
var leaves: seq[Poseidon2Hash]
while index < bytes.len:
let start = index
let finish = min(index + chunkSize, bytes.len)
let digest = Sponge.digest(bytes.toOpenArray(start, finish - 1), rate = 2)
leaves.add(digest)
index += chunkSize
return Poseidon2Tree.init(leaves)
func digest*(
_: type Poseidon2Tree,
bytes: openArray[byte], chunkSize: int): ?!Poseidon2Hash =
## Hashes chunks of data with a sponge of rate 2, and combines the
## resulting chunk hashes in a merkle root.
##
(? Poseidon2Tree.digestTree(bytes, chunkSize)).root
func digestMhash*(
_: type Poseidon2Tree,
bytes: openArray[byte], chunkSize: int): ?!MultiHash =
## Hashes chunks of data with a sponge of rate 2 and
## returns the multihash of the root
##
let
hash = ? Poseidon2Tree.digest(bytes, chunkSize)
? MultiHash.init(Pos2Bn128MrklCodec, hash).mapFailure

View File

@ -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/poseidon2
import pkg/questionable/results
import pkg/libp2p/multihash
import ../merkletree
func digestTree*(
_: type Poseidon2Tree,
bytes: openArray[byte], chunkSize: int): ?!Poseidon2Tree =
## Hashes chunks of data with a sponge of rate 2, and combines the
## resulting chunk hashes in a merkle root.
##
var index = 0
var leaves: seq[Poseidon2Hash]
while index < bytes.len:
let start = index
let finish = min(index + chunkSize, bytes.len)
let digest = Sponge.digest(bytes.toOpenArray(start, finish - 1), rate = 2)
leaves.add(digest)
index += chunkSize
return Poseidon2Tree.init(leaves)
func digest*(
_: type Poseidon2Tree,
bytes: openArray[byte], chunkSize: int): ?!Poseidon2Hash =
## Hashes chunks of data with a sponge of rate 2, and combines the
## resulting chunk hashes in a merkle root.
##
(? Poseidon2Tree.digestTree(bytes, chunkSize)).root
func digestMhash*(
_: type Poseidon2Tree,
bytes: openArray[byte], chunkSize: int): ?!MultiHash =
## Hashes chunks of data with a sponge of rate 2 and
## returns the multihash of the root
##
let
hash = ? Poseidon2Tree.digest(bytes, chunkSize)
? MultiHash.init(Pos2Bn128MrklCodec, hash).mapFailure

View File

@ -73,6 +73,7 @@ else:
--define:nimTypeNames --define:nimTypeNames
--styleCheck:usages --styleCheck:usages
--styleCheck:error --styleCheck:error
--maxLoopIterationsVM:1000000000
when (NimMajor, NimMinor) >= (1, 4): when (NimMajor, NimMinor) >= (1, 4):
--warning:"ObservableStores:off" --warning:"ObservableStores:off"

View File

@ -9,6 +9,7 @@ import pkg/libp2p
import pkg/codex/codextypes import pkg/codex/codextypes
import pkg/codex/merkletree import pkg/codex/merkletree
import pkg/codex/utils/digest
import ./helpers import ./helpers
import ./generictreetests import ./generictreetests

View File

@ -12,6 +12,7 @@ import pkg/questionable/results
import pkg/codex/merkletree import pkg/codex/merkletree
import pkg/codex/utils/digest import pkg/codex/utils/digest
import pkg/codex/utils/poseidon2digest
import ./helpers import ./helpers

View File

@ -11,6 +11,7 @@ import pkg/codex/rng
import pkg/codex/stores import pkg/codex/stores
import pkg/codex/chunker import pkg/codex/chunker
import pkg/codex/merkletree import pkg/codex/merkletree
import pkg/codex/manifest {.all.}
import pkg/codex/utils import pkg/codex/utils
import pkg/codex/utils/digest import pkg/codex/utils/digest
import pkg/datastore import pkg/datastore
@ -23,7 +24,10 @@ import ../examples
import ../merkletree/helpers import ../merkletree/helpers
import pkg/codex/indexingstrategy {.all.} import pkg/codex/indexingstrategy {.all.}
import pkg/codex/slots/slotbuilder {.all.} import pkg/codex/slots {.all.}
privateAccess(SlotsBuilder) # enable access to private fields
privateAccess(Manifest) # enable access to private fields
suite "Slot builder": suite "Slot builder":
let let
@ -57,7 +61,7 @@ suite "Slot builder":
manifest: Manifest manifest: Manifest
protectedManifest: Manifest protectedManifest: Manifest
expectedEmptyCid: Cid expectedEmptyCid: Cid
slotBuilder: SlotBuilder slotBuilder: SlotsBuilder
chunker: Chunker chunker: Chunker
proc createBlocks(): Future[void] {.async.} = proc createBlocks(): Future[void] {.async.} =
@ -115,8 +119,6 @@ suite "Slot builder":
protectedManifest.hcodec, protectedManifest.hcodec,
protectedManifest.codec).tryGet() protectedManifest.codec).tryGet()
privateAccess(SlotBuilder) # enable access to private fields
setup: setup:
let let
repoDs = SQLiteDatastore.new(Memory).tryGet() repoDs = SQLiteDatastore.new(Memory).tryGet()
@ -130,6 +132,9 @@ suite "Slot builder":
teardown: teardown:
await localStore.close() await localStore.close()
# TODO: THIS IS A BUG IN asynctest, because it doesn't release the
# objects after the test is done, so we need to do it manually
#
# Need to reset all objects because otherwise they get # Need to reset all objects because otherwise they get
# captured by the test runner closures, not good! # captured by the test runner closures, not good!
reset(datasetBlocks) reset(datasetBlocks)
@ -148,8 +153,8 @@ suite "Slot builder":
datasetSize = originalDatasetSize.NBytes) datasetSize = originalDatasetSize.NBytes)
check: check:
SlotBuilder.new(localStore, unprotectedManifest, cellSize = cellSize) SlotsBuilder.new(localStore, unprotectedManifest, cellSize = cellSize)
.error.msg == "Can only create SlotBuilder using protected manifests." .error.msg == "Can only create SlotsBuilder using protected manifests."
test "Number of blocks must be devisable by number of slots": test "Number of blocks must be devisable by number of slots":
let let
@ -164,7 +169,7 @@ suite "Slot builder":
ecM = ecM) ecM = ecM)
check: check:
SlotBuilder.new(localStore, mismatchManifest, cellSize = cellSize) SlotsBuilder.new(localStore, mismatchManifest, cellSize = cellSize)
.error.msg == "Number of blocks must be divisable by number of slots." .error.msg == "Number of blocks must be divisable by number of slots."
test "Block size must be divisable by cell size": test "Block size must be divisable by cell size":
@ -180,11 +185,11 @@ suite "Slot builder":
ecM = ecM) ecM = ecM)
check: check:
SlotBuilder.new(localStore, mismatchManifest, cellSize = cellSize) SlotsBuilder.new(localStore, mismatchManifest, cellSize = cellSize)
.error.msg == "Block size must be divisable by cell size." .error.msg == "Block size must be divisable by cell size."
test "Should build correct slot builder": test "Should build correct slot builder":
slotBuilder = SlotBuilder.new( slotBuilder = SlotsBuilder.new(
localStore, localStore,
protectedManifest, protectedManifest,
cellSize = cellSize).tryGet() cellSize = cellSize).tryGet()
@ -197,7 +202,7 @@ suite "Slot builder":
test "Should build slot hashes for all slots": test "Should build slot hashes for all slots":
let let
steppedStrategy = SteppedIndexingStrategy.new(0, numTotalBlocks - 1, numSlots) steppedStrategy = SteppedIndexingStrategy.new(0, numTotalBlocks - 1, numSlots)
slotBuilder = SlotBuilder.new( slotBuilder = SlotsBuilder.new(
localStore, localStore,
protectedManifest, protectedManifest,
cellSize = cellSize).tryGet() cellSize = cellSize).tryGet()
@ -220,7 +225,7 @@ suite "Slot builder":
test "Should build slot trees for all slots": test "Should build slot trees for all slots":
let let
steppedStrategy = SteppedIndexingStrategy.new(0, numTotalBlocks - 1, numSlots) steppedStrategy = SteppedIndexingStrategy.new(0, numTotalBlocks - 1, numSlots)
slotBuilder = SlotBuilder.new( slotBuilder = SlotsBuilder.new(
localStore, localStore,
protectedManifest, protectedManifest,
cellSize = cellSize).tryGet() cellSize = cellSize).tryGet()
@ -243,7 +248,7 @@ suite "Slot builder":
test "Should persist trees for all slots": test "Should persist trees for all slots":
let let
slotBuilder = SlotBuilder.new( slotBuilder = SlotsBuilder.new(
localStore, localStore,
protectedManifest, protectedManifest,
cellSize = cellSize).tryGet() cellSize = cellSize).tryGet()
@ -268,12 +273,13 @@ suite "Slot builder":
test "Should build correct verification root": test "Should build correct verification root":
let let
steppedStrategy = SteppedIndexingStrategy.new(0, numTotalBlocks - 1, numSlots) steppedStrategy = SteppedIndexingStrategy.new(0, numTotalBlocks - 1, numSlots)
slotBuilder = SlotBuilder.new( slotBuilder = SlotsBuilder.new(
localStore, localStore,
protectedManifest, protectedManifest,
cellSize = cellSize).tryGet() cellSize = cellSize).tryGet()
slotRoots = (await slotBuilder.buildSlots()).tryGet
(await slotBuilder.buildSlots()).tryGet
let
slotsHashes = collect(newSeq): slotsHashes = collect(newSeq):
for i in 0 ..< numSlots: for i in 0 ..< numSlots:
let let
@ -288,7 +294,7 @@ suite "Slot builder":
Merkle.digest(slotHashes & slotsPadLeafs) Merkle.digest(slotHashes & slotsPadLeafs)
expectedRoot = Merkle.digest(slotsHashes & rootsPadLeafs) expectedRoot = Merkle.digest(slotsHashes & rootsPadLeafs)
rootHash = slotBuilder.buildRootsTree(slotRoots).tryGet().root.tryGet() rootHash = slotBuilder.buildRootsTree(slotBuilder.slotRoots).tryGet().root.tryGet()
check: check:
expectedRoot == rootHash expectedRoot == rootHash
@ -296,7 +302,7 @@ suite "Slot builder":
test "Should build correct verification root manifest": test "Should build correct verification root manifest":
let let
steppedStrategy = SteppedIndexingStrategy.new(0, numTotalBlocks - 1, numSlots) steppedStrategy = SteppedIndexingStrategy.new(0, numTotalBlocks - 1, numSlots)
slotBuilder = SlotBuilder.new( slotBuilder = SlotsBuilder.new(
localStore, localStore,
protectedManifest, protectedManifest,
cellSize = cellSize).tryGet() cellSize = cellSize).tryGet()
@ -316,9 +322,78 @@ suite "Slot builder":
expectedRoot = Merkle.digest(slotsHashes & rootsPadLeafs) expectedRoot = Merkle.digest(slotsHashes & rootsPadLeafs)
manifest = (await slotBuilder.buildManifest()).tryGet() manifest = (await slotBuilder.buildManifest()).tryGet()
mhash = manifest.verificationRoot.mhash.tryGet() mhash = manifest.verifyRoot.mhash.tryGet()
mhashBytes = mhash.digestBytes mhashBytes = mhash.digestBytes
rootHash = Poseidon2Hash.fromBytes(mhashBytes.toArray32).toResult.tryGet() rootHash = Poseidon2Hash.fromBytes(mhashBytes.toArray32).get
check: check:
expectedRoot == rootHash expectedRoot == rootHash
test "Should not build from verifiable manifest with 0 slots":
var
slotBuilder = SlotsBuilder.new(
localStore,
protectedManifest,
cellSize = cellSize).tryGet()
verifyManifest = (await slotBuilder.buildManifest()).tryGet()
verifyManifest.slotRoots = @[]
check SlotsBuilder.new(
localStore,
verifyManifest,
cellSize = cellSize).isErr
test "Should not build from verifiable manifest with incorrect number of slots":
var
slotBuilder = SlotsBuilder.new(
localStore,
protectedManifest,
cellSize = cellSize).tryGet()
verifyManifest = (await slotBuilder.buildManifest()).tryGet()
verifyManifest.slotRoots.del(
verifyManifest.slotRoots.len - 1
)
check SlotsBuilder.new(
localStore,
verifyManifest,
cellSize = cellSize).isErr
test "Should not build from verifiable manifest with slots root":
let
slotBuilder = SlotsBuilder.new(
localStore,
protectedManifest,
cellSize = cellSize).tryGet()
verifyManifest = (await slotBuilder.buildManifest()).tryGet()
offset = verifyManifest.verifyRoot.data.buffer.len div 2
rng.shuffle(
Rng.instance,
verifyManifest.verifyRoot.data.buffer)
check SlotsBuilder.new(
localStore,
verifyManifest,
cellSize = cellSize).isErr
test "Should build from verifiable manifest":
let
slotBuilder = SlotsBuilder.new(
localStore,
protectedManifest,
cellSize = cellSize).tryGet()
verifyManifest = (await slotBuilder.buildManifest()).tryGet()
verificationBuilder = SlotsBuilder.new(
localStore,
verifyManifest,
cellSize = cellSize).tryGet()
check:
slotBuilder.slotRoots == verificationBuilder.slotRoots
slotBuilder.verifyRoot == verificationBuilder.verifyRoot

View File

@ -61,7 +61,7 @@ asyncchecksuite "Chunking":
test "should chunk file": test "should chunk file":
let let
(path, _, _) = instantiationInfo(-2, fullPaths = true) # get this file's name path = currentSourcePath()
file = open(path) file = open(path)
fileChunker = FileChunker.new(file = file, chunkSize = 256'nb, pad = false) fileChunker = FileChunker.new(file = file, chunkSize = 256'nb, pad = false)

View File

@ -39,8 +39,8 @@ suite "Erasure encode/decode":
let let
encoded = (await erasure.encode( encoded = (await erasure.encode(
manifest, manifest,
buffers, buffers.Natural,
parity)).tryGet() parity.Natural)).tryGet()
check: check:
encoded.blocksCount mod (buffers + parity) == 0 encoded.blocksCount mod (buffers + parity) == 0

View File

@ -6,6 +6,10 @@ import pkg/asynctest
import pkg/codex/chunker import pkg/codex/chunker
import pkg/codex/blocktype as bt import pkg/codex/blocktype as bt
import pkg/codex/manifest import pkg/codex/manifest
import pkg/poseidon2
import pkg/codex/slots
import pkg/codex/merkletree
import ./helpers import ./helpers
import ./examples import ./examples
@ -17,17 +21,30 @@ checksuite "Manifest":
blockSize = 1.MiBs, blockSize = 1.MiBs,
datasetSize = 100.MiBs datasetSize = 100.MiBs
) )
protectedManifest = Manifest.new( protectedManifest = Manifest.new(
manifest = manifest, manifest = manifest,
treeCid = Cid.example, treeCid = Cid.example,
datasetSize = 200.MiBs, datasetSize = 200.MiBs,
eck = 10, eck = 2,
ecM = 10 ecM = 2
) )
leaves = [
0.toF.Poseidon2Hash,
1.toF.Poseidon2Hash,
2.toF.Poseidon2Hash,
3.toF.Poseidon2Hash]
slotLeavesCids = leaves.toSlotCids().tryGet
tree = Poseidon2Tree.init(leaves).tryGet
slotsRootsCid = tree.root.tryGet.toSlotsRootsCid().tryGet
verifiableManifest = Manifest.new( verifiableManifest = Manifest.new(
manifest = protectedManifest, manifest = protectedManifest,
verificationRoot = Cid.example, verifyRoot = slotsRootsCid,
slotRoots = @[Cid.example, Cid.example] slotRoots = slotLeavesCids
).tryGet() ).tryGet()
proc encodeDecode(manifest: Manifest): Manifest = proc encodeDecode(manifest: Manifest): Manifest =

View File

@ -2,6 +2,8 @@ import std/os
import std/options import std/options
import std/math import std/math
import std/times import std/times
import std/sequtils
import std/importutils
import pkg/asynctest import pkg/asynctest
import pkg/chronos import pkg/chronos
@ -11,6 +13,8 @@ import pkg/datastore
import pkg/questionable import pkg/questionable
import pkg/questionable/results import pkg/questionable/results
import pkg/stint import pkg/stint
import pkg/poseidon2
import pkg/poseidon2/io
import pkg/nitro import pkg/nitro
import pkg/codexdht/discv5/protocol as discv5 import pkg/codexdht/discv5/protocol as discv5
@ -21,23 +25,61 @@ import pkg/codex/contracts
import pkg/codex/systemclock import pkg/codex/systemclock
import pkg/codex/blockexchange import pkg/codex/blockexchange
import pkg/codex/chunker import pkg/codex/chunker
import pkg/codex/node import pkg/codex/slots
import pkg/codex/manifest import pkg/codex/manifest
import pkg/codex/discovery import pkg/codex/discovery
import pkg/codex/erasure
import pkg/codex/merkletree
import pkg/codex/blocktype as bt import pkg/codex/blocktype as bt
import pkg/codex/node {.all.}
import ../examples import ../examples
import ./helpers import ./helpers
import ./helpers/mockmarket import ./helpers/mockmarket
import ./helpers/mockclock import ./helpers/mockclock
privateAccess(CodexNodeRef) # enable access to private fields
proc toTimesDuration(d: chronos.Duration): times.Duration = proc toTimesDuration(d: chronos.Duration): times.Duration =
initDuration(seconds=d.seconds) initDuration(seconds = d.seconds)
proc drain(
stream: LPStream | Result[lpstream.LPStream, ref CatchableError]):
Future[seq[byte]] {.async.} =
asyncchecksuite "Test Node":
let let
(path, _, _) = instantiationInfo(-2, fullPaths = true) # get this file's name stream =
when typeof(stream) is Result[lpstream.LPStream, ref CatchableError]:
stream.tryGet()
else:
stream
defer:
await stream.close()
var data: seq[byte]
while not stream.atEof:
var
buf = newSeq[byte](DefaultBlockSize.int)
res = await stream.readOnce(addr buf[0], DefaultBlockSize.int)
check res <= DefaultBlockSize.int
buf.setLen(res)
data &= buf
data
proc pipeChunker(stream: BufferStream, chunker: Chunker) {.async.} =
try:
while (
let chunk = await chunker.getBytes();
chunk.len > 0):
await stream.pushData(chunk)
finally:
await stream.pushEof()
await stream.close()
template setupAndTearDown() {.dirty.} =
var var
file: File file: File
chunker: Chunker chunker: Chunker
@ -55,33 +97,13 @@ asyncchecksuite "Test Node":
peerStore: PeerCtxStore peerStore: PeerCtxStore
pendingBlocks: PendingBlocksManager pendingBlocks: PendingBlocksManager
discovery: DiscoveryEngine discovery: DiscoveryEngine
erasure: Erasure
proc fetch(T: type Manifest, chunker: Chunker): Future[Manifest] {.async.} = let
# Collect blocks from Chunker into Manifest path = currentSourcePath().parentDir
await storeDataGetManifest(localStore, chunker)
proc retrieve(cid: Cid): Future[seq[byte]] {.async.} =
# Retrieve an entire file contents by file Cid
let
oddChunkSize = math.trunc(DefaultBlockSize.float/1.359).int # Let's check that node.retrieve can correctly rechunk data
stream = (await node.retrieve(cid)).tryGet()
var
data: seq[byte]
defer: await stream.close()
while not stream.atEof:
var
buf = newSeq[byte](oddChunkSize)
res = await stream.readOnce(addr buf[0], oddChunkSize)
check res <= oddChunkSize
buf.setLen(res)
data &= buf
return data
setup: setup:
file = open(path.splitFile().dir /../ "fixtures" / "test.jpg") file = open(path /../ "fixtures" / "test.jpg")
chunker = FileChunker.new(file = file, chunkSize = DefaultBlockSize) chunker = FileChunker.new(file = file, chunkSize = DefaultBlockSize)
switch = newStandardSwitch() switch = newStandardSwitch()
wallet = WalletRef.new(EthPrivateKey.random()) wallet = WalletRef.new(EthPrivateKey.random())
@ -90,7 +112,7 @@ asyncchecksuite "Test Node":
clock = SystemClock.new() clock = SystemClock.new()
localStoreMetaDs = SQLiteDatastore.new(Memory).tryGet() localStoreMetaDs = SQLiteDatastore.new(Memory).tryGet()
localStoreRepoDs = SQLiteDatastore.new(Memory).tryGet() localStoreRepoDs = SQLiteDatastore.new(Memory).tryGet()
localStore = RepoStore.new(localStoreRepoDs, localStoreMetaDs, clock=clock) localStore = RepoStore.new(localStoreRepoDs, localStoreMetaDs, clock = clock)
await localStore.start() await localStore.start()
blockDiscovery = Discovery.new( blockDiscovery = Discovery.new(
@ -102,7 +124,8 @@ asyncchecksuite "Test Node":
discovery = DiscoveryEngine.new(localStore, peerStore, network, blockDiscovery, pendingBlocks) discovery = DiscoveryEngine.new(localStore, peerStore, network, blockDiscovery, pendingBlocks)
engine = BlockExcEngine.new(localStore, wallet, network, discovery, peerStore, pendingBlocks) engine = BlockExcEngine.new(localStore, wallet, network, discovery, peerStore, pendingBlocks)
store = NetworkStore.new(engine, localStore) store = NetworkStore.new(engine, localStore)
node = CodexNodeRef.new(switch, store, engine, nil, blockDiscovery) # TODO: pass `Erasure` erasure = Erasure.new(store, leoEncoderProvider, leoDecoderProvider)
node = CodexNodeRef.new(switch, store, engine, erasure, blockDiscovery)
await node.start() await node.start()
@ -110,9 +133,12 @@ asyncchecksuite "Test Node":
close(file) close(file)
await node.stop() await node.stop()
asyncchecksuite "Test Node - Basic":
setupAndTearDown()
test "Fetch Manifest": test "Fetch Manifest":
let let
manifest = await Manifest.fetch(chunker) manifest = await storeDataGetManifest(localStore, chunker)
manifestBlock = bt.Block.new( manifestBlock = bt.Block.new(
manifest.encode().tryGet(), manifest.encode().tryGet(),
@ -127,7 +153,7 @@ asyncchecksuite "Test Node":
fetched == manifest fetched == manifest
test "Block Batching": test "Block Batching":
let manifest = await Manifest.fetch(chunker) let manifest = await storeDataGetManifest(localStore, chunker)
for batchSize in 1..12: for batchSize in 1..12:
(await node.fetchBatched( (await node.fetchBatched(
@ -142,8 +168,9 @@ asyncchecksuite "Test Node":
let let
stream = BufferStream.new() stream = BufferStream.new()
storeFut = node.store(stream) storeFut = node.store(stream)
oddChunkSize = math.trunc(DefaultBlockSize.float/3.14).NBytes # Let's check that node.store can correctly rechunk these odd chunks oddChunkSize = math.trunc(DefaultBlockSize.float / 3.14).NBytes # Let's check that node.store can correctly rechunk these odd chunks
oddChunker = FileChunker.new(file = file, chunkSize = oddChunkSize, pad = false) # TODO: doesn't work with pad=tue oddChunker = FileChunker.new(file = file, chunkSize = oddChunkSize, pad = false) # TODO: doesn't work with pad=tue
var var
original: seq[byte] original: seq[byte]
@ -159,14 +186,9 @@ asyncchecksuite "Test Node":
let let
manifestCid = (await storeFut).tryGet() manifestCid = (await storeFut).tryGet()
check:
(await localStore.hasBlock(manifestCid)).tryGet()
let
manifestBlock = (await localStore.getBlock(manifestCid)).tryGet() manifestBlock = (await localStore.getBlock(manifestCid)).tryGet()
localManifest = Manifest.decode(manifestBlock).tryGet() localManifest = Manifest.decode(manifestBlock).tryGet()
data = await (await node.retrieve(manifestCid)).drain()
let data = await retrieve(manifestCid)
check: check:
data.len == localManifest.datasetSize.int data.len == localManifest.datasetSize.int
@ -186,80 +208,72 @@ asyncchecksuite "Test Node":
await stream.readExactly(addr data[0], data.len) await stream.readExactly(addr data[0], data.len)
check string.fromBytes(data) == testString check string.fromBytes(data) == testString
test "Setup purchase request":
let
manifest = await storeDataGetManifest(localStore, chunker)
manifestBlock = bt.Block.new(
manifest.encode().tryGet(),
codec = ManifestCodec).tryGet()
asyncchecksuite "Test Node - host contracts": protected = (await erasure.encode(manifest, 3, 2)).tryGet()
let builder = SlotsBuilder.new(localStore, protected).tryGet()
(path, _, _) = instantiationInfo(-2, fullPaths = true) # get this file's name verifiable = (await builder.buildManifest()).tryGet()
verifiableBlock = bt.Block.new(
verifiable.encode().tryGet(),
codec = ManifestCodec).tryGet()
(await localStore.putBlock(manifestBlock)).tryGet()
let
request = (await node.setupRequest(
cid = manifestBlock.cid,
nodes = 5,
tolerance = 2,
duration = 100.u256,
reward = 2.u256,
proofProbability = 3.u256,
expiry = 200.u256,
collateral = 200.u256)).tryGet
check:
(await verifiableBlock.cid in localStore) == true
request.content.cid == $verifiableBlock.cid
request.content.merkleRoot == builder.verifyRoot.get.toBytes
asyncchecksuite "Test Node - Host contracts":
setupAndTearDown()
var var
file: File
chunker: Chunker
switch: Switch
wallet: WalletRef
network: BlockExcNetwork
clock: MockClock
localStore: RepoStore
localStoreRepoDs: DataStore
localStoreMetaDs: DataStore
engine: BlockExcEngine
store: NetworkStore
sales: Sales sales: Sales
node: CodexNodeRef purchasing: Purchasing
blockDiscovery: Discovery
peerStore: PeerCtxStore
pendingBlocks: PendingBlocksManager
discovery: DiscoveryEngine
manifest: Manifest manifest: Manifest
manifestCid: string manifestCidStr: string
manifestCid: Cid
proc fetch(T: type Manifest, chunker: Chunker): Future[Manifest] {.async.} = market: MockMarket
# Collect blocks from Chunker into Manifest
await storeDataGetManifest(localStore, chunker)
setup: setup:
file = open(path.splitFile().dir /../ "fixtures" / "test.jpg")
chunker = FileChunker.new(file = file, chunkSize = DefaultBlockSize)
switch = newStandardSwitch()
wallet = WalletRef.new(EthPrivateKey.random())
network = BlockExcNetwork.new(switch)
clock = MockClock.new()
localStoreMetaDs = SQLiteDatastore.new(Memory).tryGet()
localStoreRepoDs = SQLiteDatastore.new(Memory).tryGet()
localStore = RepoStore.new(localStoreRepoDs, localStoreMetaDs, clock=clock)
await localStore.start()
blockDiscovery = Discovery.new(
switch.peerInfo.privateKey,
announceAddrs = @[MultiAddress.init("/ip4/127.0.0.1/tcp/0")
.expect("Should return multiaddress")])
peerStore = PeerCtxStore.new()
pendingBlocks = PendingBlocksManager.new()
discovery = DiscoveryEngine.new(localStore, peerStore, network, blockDiscovery, pendingBlocks)
engine = BlockExcEngine.new(localStore, wallet, network, discovery, peerStore, pendingBlocks)
store = NetworkStore.new(engine, localStore)
node = CodexNodeRef.new(switch, store, engine, nil, blockDiscovery) # TODO: pass `Erasure`
# Setup Host Contracts and dependencies # Setup Host Contracts and dependencies
let market = MockMarket.new() market = MockMarket.new()
sales = Sales.new(market, clock, localStore) sales = Sales.new(market, clock, localStore)
let hostContracts = some HostInteractions.new(clock, sales)
node.contracts = (ClientInteractions.none, hostContracts, ValidatorInteractions.none) node.contracts = (
none ClientInteractions,
some HostInteractions.new(clock, sales),
none ValidatorInteractions)
await node.start() await node.start()
# Populate manifest in local store # Populate manifest in local store
manifest = await storeDataGetManifest(localStore, chunker) manifest = await storeDataGetManifest(localStore, chunker)
let manifestBlock = bt.Block.new( let
manifestBlock = bt.Block.new(
manifest.encode().tryGet(), manifest.encode().tryGet(),
codec = ManifestCodec codec = ManifestCodec).tryGet()
).tryGet()
manifestCid = $(manifestBlock.cid)
(await localStore.putBlock(manifestBlock)).tryGet()
teardown: manifestCid = manifestBlock.cid
close(file) manifestCidStr = $(manifestCid)
await node.stop()
(await localStore.putBlock(manifestBlock)).tryGet()
test "onExpiryUpdate callback is set": test "onExpiryUpdate callback is set":
check sales.onExpiryUpdate.isSome check sales.onExpiryUpdate.isSome
@ -270,12 +284,13 @@ asyncchecksuite "Test Node - host contracts":
expectedExpiry: SecondsSince1970 = clock.now + DefaultBlockTtl.seconds + 11123 expectedExpiry: SecondsSince1970 = clock.now + DefaultBlockTtl.seconds + 11123
expiryUpdateCallback = !sales.onExpiryUpdate expiryUpdateCallback = !sales.onExpiryUpdate
(await expiryUpdateCallback(manifestCid, expectedExpiry)).tryGet() (await expiryUpdateCallback(manifestCidStr, expectedExpiry)).tryGet()
for index in 0..<manifest.blocksCount: for index in 0..<manifest.blocksCount:
let blk = (await localStore.getBlock(manifest.treeCid, index)).tryGet let
let expiryKey = (createBlockExpirationMetadataKey(blk.cid)).tryGet blk = (await localStore.getBlock(manifest.treeCid, index)).tryGet
let expiry = await localStoreMetaDs.get(expiryKey) expiryKey = (createBlockExpirationMetadataKey(blk.cid)).tryGet
expiry = await localStoreMetaDs.get(expiryKey)
check (expiry.tryGet).toSecondsSince1970 == expectedExpiry check (expiry.tryGet).toSecondsSince1970 == expectedExpiry
@ -285,7 +300,7 @@ asyncchecksuite "Test Node - host contracts":
test "onStore callback": test "onStore callback":
let onStore = !sales.onStore let onStore = !sales.onStore
var request = StorageRequest.example var request = StorageRequest.example
request.content.cid = manifestCid request.content.cid = manifestCidStr
request.expiry = (getTime() + DefaultBlockTtl.toTimesDuration + 1.hours).toUnix.u256 request.expiry = (getTime() + DefaultBlockTtl.toTimesDuration + 1.hours).toUnix.u256
var fetchedBytes: uint = 0 var fetchedBytes: uint = 0
@ -298,8 +313,9 @@ asyncchecksuite "Test Node - host contracts":
check fetchedBytes == 2293760 check fetchedBytes == 2293760
for index in 0..<manifest.blocksCount: for index in 0..<manifest.blocksCount:
let blk = (await localStore.getBlock(manifest.treeCid, index)).tryGet let
let expiryKey = (createBlockExpirationMetadataKey(blk.cid)).tryGet blk = (await localStore.getBlock(manifest.treeCid, index)).tryGet
let expiry = await localStoreMetaDs.get(expiryKey) expiryKey = (createBlockExpirationMetadataKey(blk.cid)).tryGet
expiry = await localStoreMetaDs.get(expiryKey)
check (expiry.tryGet).toSecondsSince1970 == request.expiry.toSecondsSince1970 check (expiry.tryGet).toSecondsSince1970 == request.expiry.toSecondsSince1970

View File

@ -15,7 +15,6 @@ import ../codex/helpers
import ../examples import ../examples
import ./twonodes import ./twonodes
# For debugging you can enable logging output with debugX = true # For debugging you can enable logging output with debugX = true
# You can also pass a string in same format like for the `--log-level` parameter # You can also pass a string in same format like for the `--log-level` parameter
# to enable custom logging levels for specific topics like: debug2 = "INFO; TRACE: marketplace" # to enable custom logging levels for specific topics like: debug2 = "INFO; TRACE: marketplace"
@ -140,7 +139,7 @@ twonodessuite "Integration tests", debug1 = false, debug2 = false:
check request.ask.proofProbability == 3.u256 check request.ask.proofProbability == 3.u256
check request.expiry == expiry check request.expiry == expiry
check request.ask.collateral == 200.u256 check request.ask.collateral == 200.u256
check request.ask.slots == 3'u64 check request.ask.slots == 2'u64
check request.ask.maxSlotLoss == 1'u64 check request.ask.maxSlotLoss == 1'u64
# TODO: We currently do not support encoding single chunks # TODO: We currently do not support encoding single chunks
@ -181,7 +180,6 @@ twonodessuite "Integration tests", debug1 = false, debug2 = false:
check request.ask.slots == 1'u64 check request.ask.slots == 1'u64
check request.ask.maxSlotLoss == 0'u64 check request.ask.maxSlotLoss == 0'u64
test "nodes negotiate contracts on the marketplace": test "nodes negotiate contracts on the marketplace":
let size = 0xFFFFF.u256 let size = 0xFFFFF.u256
# client 2 makes storage available # client 2 makes storage available
@ -227,6 +225,21 @@ twonodessuite "Integration tests", debug1 = false, debug2 = false:
check eventually (await token.balanceOf(account2)) - startBalance == duration*reward check eventually (await token.balanceOf(account2)) - startBalance == duration*reward
test "request storage fails if nodes and tolerance aren't correct":
let cid = client1.upload("some file contents").get
let expiry = (await ethProvider.currentTime()) + 30
let responseBefore = client1.requestStorageRaw(cid,
duration=100.u256,
reward=2.u256,
proofProbability=3.u256,
expiry=expiry,
collateral=200.u256,
nodes=1,
tolerance=1)
check responseBefore.status == "400 Bad Request"
check responseBefore.body == "Tolerance cannot be greater or equal than nodes (nodes - tolerance)"
test "node requires expiry and its value to be in future": test "node requires expiry and its value to be in future":
let currentTime = await ethProvider.currentTime() let currentTime = await ethProvider.currentTime()
let cid = client1.upload("some file contents").get let cid = client1.upload("some file contents").get

@ -1 +1 @@
Subproject commit 3b403b0752790438bed6342b431412cc05474acb Subproject commit eef2603c11eeb5069eaed6c0296f3a1c5d23aad1