Wire sampler (#676)
* Setting up testfixture for proof datasampler * Sets up calculating number of cells in a slot * Sets up tests for bitwise modulo * Implements cell index collection * setting up slot blocks module * Implements getting treeCID from slot * implements getting slot blocks by index * Implements out-of-range check for slot index * cleanup * Sets up getting sample from block * Implements selecting a cell sample from a block * Implements building a minitree for block cells * Adds method to get dataset block index from slot block index * It's running * splits up indexing * almost there * Fixes test. Implementation is now functional * Refactoring to object-oriented * Cleanup * Lining up output type with updated reference code. * setting up * Updates expected samples * Updates proof checking test to match new format * move builder to own dir * move sampler to own dir * fix paths * various changes to add support for the sampler * wip sampler implementation * don't use upraises * wip sampler integration * misc * move tests around * Various fixes to select correct slot and block index * removing old tests * cleanup * misc fix tests that work with correct cell indices * remove unused file * fixup logging * add logscope * truncate entropy to 31 bytes, otherwise it might be > than mod * forwar getCidAndProof to local store * misc * Adds missing test for initial-proving state * reverting back to correct slot/block indexing * fix tests for revert * misc * misc --------- Co-authored-by: benbierens <thatbenbierens@gmail.com>
This commit is contained in:
parent
2fc7c75fd2
commit
72da534856
|
@ -450,7 +450,7 @@ proc wantListHandler*(
|
||||||
trace "Sending presence to remote", items = presence.len
|
trace "Sending presence to remote", items = presence.len
|
||||||
await b.network.request.sendPresence(peer, presence)
|
await b.network.request.sendPresence(peer, presence)
|
||||||
|
|
||||||
trace "Scheduling a task for this peer, to look over their want-list", peer
|
trace "Scheduling a task to check want-list", peer
|
||||||
if not b.scheduleTask(peerCtx):
|
if not b.scheduleTask(peerCtx):
|
||||||
trace "Unable to schedule task for peer", peer
|
trace "Unable to schedule task for peer", peer
|
||||||
|
|
||||||
|
|
|
@ -103,5 +103,4 @@ proc new*(
|
||||||
firstIndex: firstIndex,
|
firstIndex: firstIndex,
|
||||||
lastIndex: lastIndex,
|
lastIndex: lastIndex,
|
||||||
numberOfIterations: numberOfIterations,
|
numberOfIterations: numberOfIterations,
|
||||||
step: divUp((lastIndex - firstIndex), numberOfIterations)
|
step: divUp((lastIndex - firstIndex), numberOfIterations))
|
||||||
)
|
|
||||||
|
|
|
@ -102,20 +102,6 @@ proc storeManifest*(
|
||||||
|
|
||||||
success blk
|
success blk
|
||||||
|
|
||||||
proc findPeer*(
|
|
||||||
self: CodexNodeRef,
|
|
||||||
peerId: PeerId): Future[?PeerRecord] {.async.} =
|
|
||||||
## Find peer using the discovery service from the given CodexNode
|
|
||||||
##
|
|
||||||
return await self.discovery.findPeer(peerId)
|
|
||||||
|
|
||||||
proc connect*(
|
|
||||||
self: CodexNodeRef,
|
|
||||||
peerId: PeerId,
|
|
||||||
addrs: seq[MultiAddress]
|
|
||||||
): Future[void] =
|
|
||||||
self.switch.connect(peerId, addrs)
|
|
||||||
|
|
||||||
proc fetchManifest*(
|
proc fetchManifest*(
|
||||||
self: CodexNodeRef,
|
self: CodexNodeRef,
|
||||||
cid: Cid): Future[?!Manifest] {.async.} =
|
cid: Cid): Future[?!Manifest] {.async.} =
|
||||||
|
@ -141,6 +127,20 @@ proc fetchManifest*(
|
||||||
|
|
||||||
return manifest.success
|
return manifest.success
|
||||||
|
|
||||||
|
proc findPeer*(
|
||||||
|
self: CodexNodeRef,
|
||||||
|
peerId: PeerId): Future[?PeerRecord] {.async.} =
|
||||||
|
## Find peer using the discovery service from the given CodexNode
|
||||||
|
##
|
||||||
|
return await self.discovery.findPeer(peerId)
|
||||||
|
|
||||||
|
proc connect*(
|
||||||
|
self: CodexNodeRef,
|
||||||
|
peerId: PeerId,
|
||||||
|
addrs: seq[MultiAddress]
|
||||||
|
): Future[void] =
|
||||||
|
self.switch.connect(peerId, addrs)
|
||||||
|
|
||||||
proc updateExpiry*(
|
proc updateExpiry*(
|
||||||
self: CodexNodeRef,
|
self: CodexNodeRef,
|
||||||
manifestCid: Cid,
|
manifestCid: Cid,
|
||||||
|
@ -493,9 +493,8 @@ proc onStore(
|
||||||
|
|
||||||
trace "Received a request to store a slot!"
|
trace "Received a request to store a slot!"
|
||||||
|
|
||||||
without cid =? Cid.init(request.content.cid):
|
without cid =? Cid.init(request.content.cid).mapFailure, err:
|
||||||
trace "Unable to parse Cid", cid
|
trace "Unable to parse Cid", cid
|
||||||
let err = newException(CodexError, "Unable to parse Cid")
|
|
||||||
return failure(err)
|
return failure(err)
|
||||||
|
|
||||||
without manifest =? (await self.fetchManifest(cid)), err:
|
without manifest =? (await self.fetchManifest(cid)), err:
|
||||||
|
@ -521,15 +520,17 @@ proc onStore(
|
||||||
if updateExpiryErr =? (await allFutureResult(ensureExpiryFutures)).errorOption:
|
if updateExpiryErr =? (await allFutureResult(ensureExpiryFutures)).errorOption:
|
||||||
return failure(updateExpiryErr)
|
return failure(updateExpiryErr)
|
||||||
|
|
||||||
echo "blocksCb.isNil: ", blocksCb.isNil
|
|
||||||
if not blocksCb.isNil and err =? (await blocksCb(blocks)).errorOption:
|
if not blocksCb.isNil and err =? (await blocksCb(blocks)).errorOption:
|
||||||
trace "Unable to process blocks", err = err.msg
|
trace "Unable to process blocks", err = err.msg
|
||||||
return failure(err)
|
return failure(err)
|
||||||
|
|
||||||
return success()
|
return success()
|
||||||
|
|
||||||
if blksIter =? builder.slotIndicies(slotIdx) and
|
if blksIter =? builder.slotIndiciesIter(slotIdx) and
|
||||||
err =? (await self.fetchBatched(manifest.treeCid, blksIter, onBatch = updateExpiry)).errorOption:
|
err =? (await self.fetchBatched(
|
||||||
|
manifest.treeCid,
|
||||||
|
blksIter,
|
||||||
|
onBatch = updateExpiry)).errorOption:
|
||||||
trace "Unable to fetch blocks", err = err.msg
|
trace "Unable to fetch blocks", err = err.msg
|
||||||
return failure(err)
|
return failure(err)
|
||||||
|
|
||||||
|
@ -543,6 +544,47 @@ proc onStore(
|
||||||
|
|
||||||
return success()
|
return success()
|
||||||
|
|
||||||
|
proc onProve(
|
||||||
|
self: CodexNodeRef,
|
||||||
|
slot: Slot,
|
||||||
|
challenge: ProofChallenge): Future[?!seq[byte]] {.async.} =
|
||||||
|
## Generats a proof for a given slot and challenge
|
||||||
|
##
|
||||||
|
|
||||||
|
let
|
||||||
|
cidStr = slot.request.content.cid
|
||||||
|
slotIdx = slot.slotIndex.truncate(Natural)
|
||||||
|
|
||||||
|
logScope:
|
||||||
|
cid = cidStr
|
||||||
|
slot = slotIdx
|
||||||
|
challenge = challenge
|
||||||
|
|
||||||
|
trace "Received proof challenge"
|
||||||
|
|
||||||
|
without cid =? Cid.init(cidStr).mapFailure, err:
|
||||||
|
error "Unable to parse Cid", cid, err = err.msg
|
||||||
|
return failure(err)
|
||||||
|
|
||||||
|
without manifest =? await self.fetchManifest(cid), err:
|
||||||
|
error "Unable to fetch manifest for cid", err = err.msg
|
||||||
|
return failure(err)
|
||||||
|
|
||||||
|
without builder =? SlotsBuilder.new(self.blockStore, manifest), err:
|
||||||
|
error "Unable to create slots builder", err = err.msg
|
||||||
|
return failure(err)
|
||||||
|
|
||||||
|
without sampler =? DataSampler.new(slotIdx, self.blockStore, builder), err:
|
||||||
|
error "Unable to create data sampler", err = err.msg
|
||||||
|
return failure(err)
|
||||||
|
|
||||||
|
without proofInput =? await sampler.getProofInput(challenge, nSamples = 3), err:
|
||||||
|
error "Unable to get proof input for slot", err = err.msg
|
||||||
|
return failure(err)
|
||||||
|
|
||||||
|
# Todo: send proofInput to circuit. Get proof. (Profit, repeat.)
|
||||||
|
success(@[42'u8])
|
||||||
|
|
||||||
proc onExpiryUpdate(
|
proc onExpiryUpdate(
|
||||||
self: CodexNodeRef,
|
self: CodexNodeRef,
|
||||||
rootCid: string,
|
rootCid: string,
|
||||||
|
@ -561,13 +603,6 @@ proc onClear(
|
||||||
# TODO: remove data from local storage
|
# TODO: remove data from local storage
|
||||||
discard
|
discard
|
||||||
|
|
||||||
proc onProve(
|
|
||||||
self: CodexNodeRef,
|
|
||||||
slot: Slot,
|
|
||||||
challenge: ProofChallenge): Future[seq[byte]] {.async.} =
|
|
||||||
# TODO: generate proof
|
|
||||||
return @[42'u8]
|
|
||||||
|
|
||||||
proc start*(self: CodexNodeRef) {.async.} =
|
proc start*(self: CodexNodeRef) {.async.} =
|
||||||
if not self.engine.isNil:
|
if not self.engine.isNil:
|
||||||
await self.engine.start()
|
await self.engine.start()
|
||||||
|
@ -598,7 +633,7 @@ proc start*(self: CodexNodeRef) {.async.} =
|
||||||
self.onClear(request, slotIndex)
|
self.onClear(request, slotIndex)
|
||||||
|
|
||||||
hostContracts.sales.onProve =
|
hostContracts.sales.onProve =
|
||||||
proc(slot: Slot, challenge: ProofChallenge): Future[seq[byte]] =
|
proc(slot: Slot, challenge: ProofChallenge): Future[?!seq[byte]] =
|
||||||
# TODO: generate proof
|
# TODO: generate proof
|
||||||
self.onProve(slot, challenge)
|
self.onProve(slot, challenge)
|
||||||
|
|
||||||
|
|
|
@ -301,7 +301,7 @@ proc onStorageRequested(sales: Sales,
|
||||||
expiry: UInt256) =
|
expiry: UInt256) =
|
||||||
|
|
||||||
logScope:
|
logScope:
|
||||||
topics = " marketplace sales onStorageRequested"
|
topics = "marketplace sales onStorageRequested"
|
||||||
requestId
|
requestId
|
||||||
slots = ask.slots
|
slots = ask.slots
|
||||||
expiry
|
expiry
|
||||||
|
|
|
@ -27,7 +27,7 @@ type
|
||||||
OnStore* = proc(request: StorageRequest,
|
OnStore* = proc(request: StorageRequest,
|
||||||
slot: UInt256,
|
slot: UInt256,
|
||||||
blocksCb: BlocksCb): Future[?!void] {.gcsafe, upraises: [].}
|
blocksCb: BlocksCb): Future[?!void] {.gcsafe, upraises: [].}
|
||||||
OnProve* = proc(slot: Slot, challenge: ProofChallenge): Future[seq[byte]] {.gcsafe, upraises: [].}
|
OnProve* = proc(slot: Slot, challenge: ProofChallenge): Future[?!seq[byte]] {.gcsafe, upraises: [].}
|
||||||
OnExpiryUpdate* = proc(rootCid: string, expiry: SecondsSince1970): Future[?!void] {.gcsafe, upraises: [].}
|
OnExpiryUpdate* = proc(rootCid: string, expiry: SecondsSince1970): Future[?!void] {.gcsafe, upraises: [].}
|
||||||
OnClear* = proc(request: StorageRequest,
|
OnClear* = proc(request: StorageRequest,
|
||||||
slotIndex: UInt256) {.gcsafe, upraises: [].}
|
slotIndex: UInt256) {.gcsafe, upraises: [].}
|
||||||
|
|
|
@ -1,9 +1,11 @@
|
||||||
import pkg/chronicles
|
import pkg/chronicles
|
||||||
|
import pkg/questionable/results
|
||||||
import ../statemachine
|
import ../statemachine
|
||||||
import ../salesagent
|
import ../salesagent
|
||||||
import ./errorhandling
|
import ./errorhandling
|
||||||
import ./filling
|
import ./filling
|
||||||
import ./cancelled
|
import ./cancelled
|
||||||
|
import ./errored
|
||||||
import ./failed
|
import ./failed
|
||||||
|
|
||||||
logScope:
|
logScope:
|
||||||
|
@ -34,7 +36,10 @@ method run*(state: SaleInitialProving, machine: Machine): Future[?State] {.async
|
||||||
let
|
let
|
||||||
slot = Slot(request: request, slotIndex: data.slotIndex)
|
slot = Slot(request: request, slotIndex: data.slotIndex)
|
||||||
challenge = await context.market.getChallenge(slot.id)
|
challenge = await context.market.getChallenge(slot.id)
|
||||||
proof = await onProve(slot, challenge)
|
without proof =? (await onProve(slot, challenge)), err:
|
||||||
|
error "Failed to generate initial proof", error = err.msg
|
||||||
|
return some State(SaleErrored(error: err))
|
||||||
|
|
||||||
debug "Finished proof calculation", requestId = $data.requestId
|
debug "Finished proof calculation", requestId = $data.requestId
|
||||||
|
|
||||||
return some State(SaleFilling(proof: proof))
|
return some State(SaleFilling(proof: proof))
|
||||||
|
|
|
@ -1,5 +1,6 @@
|
||||||
import std/options
|
import std/options
|
||||||
import pkg/chronicles
|
import pkg/chronicles
|
||||||
|
import pkg/questionable/results
|
||||||
import ../../clock
|
import ../../clock
|
||||||
import ../statemachine
|
import ../statemachine
|
||||||
import ../salesagent
|
import ../salesagent
|
||||||
|
@ -27,7 +28,10 @@ method prove*(
|
||||||
currentPeriod: Period
|
currentPeriod: Period
|
||||||
) {.base, async.} =
|
) {.base, async.} =
|
||||||
try:
|
try:
|
||||||
let proof = await onProve(slot, challenge)
|
without proof =? (await onProve(slot, challenge)), err:
|
||||||
|
error "Failed to generate proof", error = err.msg
|
||||||
|
# In this state, there's nothing we can do except try again next time.
|
||||||
|
return
|
||||||
debug "Submitting proof", currentPeriod = currentPeriod, slotId = $slot.id
|
debug "Submitting proof", currentPeriod = currentPeriod, slotId = $slot.id
|
||||||
await market.submitProof(slot.id, proof)
|
await market.submitProof(slot.id, proof)
|
||||||
except CatchableError as e:
|
except CatchableError as e:
|
||||||
|
|
|
@ -1,3 +1,4 @@
|
||||||
import ./slots/builder
|
import ./slots/builder
|
||||||
|
import ./slots/sampler
|
||||||
|
|
||||||
export builder
|
export builder, sampler
|
||||||
|
|
|
@ -1,327 +1,4 @@
|
||||||
## Nim-Codex
|
import ./builder/builder
|
||||||
## 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.
|
|
||||||
|
|
||||||
{.push raises: [].}
|
|
||||||
|
|
||||||
import std/math
|
|
||||||
import std/sequtils
|
|
||||||
import std/sugar
|
|
||||||
|
|
||||||
import pkg/libp2p
|
|
||||||
import pkg/chronos
|
|
||||||
import pkg/chronicles
|
|
||||||
import pkg/questionable/results
|
|
||||||
import pkg/poseidon2
|
|
||||||
import pkg/poseidon2/io
|
|
||||||
import pkg/constantine/math/arithmetic/finite_fields
|
|
||||||
|
|
||||||
import ../indexingstrategy
|
|
||||||
import ../merkletree
|
|
||||||
import ../stores
|
|
||||||
import ../manifest
|
|
||||||
import ../utils
|
|
||||||
import ../utils/asynciter
|
|
||||||
import ../utils/digest
|
|
||||||
import ../utils/poseidon2digest
|
|
||||||
|
|
||||||
import ./converters
|
import ./converters
|
||||||
|
|
||||||
export converters
|
export builder, converters
|
||||||
|
|
||||||
const
|
|
||||||
# TODO: Unified with the CellSize specified in branch "data-sampler"
|
|
||||||
# in the proving circuit.
|
|
||||||
CellSize* = 2048
|
|
||||||
|
|
||||||
DefaultEmptyBlock* = newSeq[byte](DefaultBlockSize.int)
|
|
||||||
DefaultEmptyCell* = newSeq[byte](DefaultCellSize.int)
|
|
||||||
|
|
||||||
type
|
|
||||||
# TODO: should be a generic type that
|
|
||||||
# supports all merkle trees
|
|
||||||
SlotsBuilder* = ref object of RootObj
|
|
||||||
store: BlockStore
|
|
||||||
manifest: Manifest
|
|
||||||
strategy: IndexingStrategy
|
|
||||||
cellSize: int
|
|
||||||
blockEmptyDigest: Poseidon2Hash
|
|
||||||
blockPadBytes: seq[byte]
|
|
||||||
slotsPadLeafs: seq[Poseidon2Hash]
|
|
||||||
rootsPadLeafs: seq[Poseidon2Hash]
|
|
||||||
slotRoots: seq[Poseidon2Hash]
|
|
||||||
verifyRoot: ?Poseidon2Hash
|
|
||||||
|
|
||||||
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
|
|
||||||
## merkle tree for each block.
|
|
||||||
##
|
|
||||||
|
|
||||||
self.blockPadBytes.len
|
|
||||||
|
|
||||||
func numSlotsPadLeafs*(self: SlotsBuilder): Natural =
|
|
||||||
## Number of padding field elements required for a pow2
|
|
||||||
## merkle tree for each slot.
|
|
||||||
##
|
|
||||||
|
|
||||||
self.slotsPadLeafs.len
|
|
||||||
|
|
||||||
func numRootsPadLeafs*(self: SlotsBuilder): Natural =
|
|
||||||
## Number of padding field elements required for a pow2
|
|
||||||
## merkle tree for the slot roots.
|
|
||||||
##
|
|
||||||
|
|
||||||
self.rootsPadLeafs.len
|
|
||||||
|
|
||||||
func numSlots*(self: SlotsBuilder): Natural =
|
|
||||||
## Number of slots.
|
|
||||||
##
|
|
||||||
|
|
||||||
self.manifest.numSlots
|
|
||||||
|
|
||||||
func numSlotBlocks*(self: SlotsBuilder): Natural =
|
|
||||||
## Number of blocks per slot.
|
|
||||||
##
|
|
||||||
|
|
||||||
self.manifest.blocksCount div self.manifest.numSlots
|
|
||||||
|
|
||||||
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.
|
|
||||||
##
|
|
||||||
|
|
||||||
self.manifest.blockSize.int div self.cellSize
|
|
||||||
|
|
||||||
func slotIndicies*(self: SlotsBuilder, slot: Natural): ?!Iter[int] =
|
|
||||||
## Returns the slot indices.
|
|
||||||
## TODO: should return an iterator
|
|
||||||
##
|
|
||||||
|
|
||||||
self.strategy.getIndicies(slot).catch
|
|
||||||
|
|
||||||
proc getCellHashes*(
|
|
||||||
self: SlotsBuilder,
|
|
||||||
slotIndex: int): Future[?!seq[Poseidon2Hash]] {.async.} =
|
|
||||||
|
|
||||||
let
|
|
||||||
treeCid = self.manifest.treeCid
|
|
||||||
blockCount = self.manifest.blocksCount
|
|
||||||
numberOfSlots = self.manifest.numSlots
|
|
||||||
|
|
||||||
logScope:
|
|
||||||
treeCid = treeCid
|
|
||||||
blockCount = blockCount
|
|
||||||
numberOfSlots = numberOfSlots
|
|
||||||
index = blockIndex
|
|
||||||
slotIndex = slotIndex
|
|
||||||
|
|
||||||
let
|
|
||||||
hashes: seq[Poseidon2Hash] = collect(newSeq):
|
|
||||||
for blockIndex in self.strategy.getIndicies(slotIndex):
|
|
||||||
trace "Getting block CID for tree at index"
|
|
||||||
|
|
||||||
without blk =? (await self.store.getBlock(treeCid, blockIndex)), err:
|
|
||||||
error "Failed to get block CID for tree at index"
|
|
||||||
return failure(err)
|
|
||||||
|
|
||||||
if blk.isEmpty:
|
|
||||||
self.blockEmptyDigest
|
|
||||||
else:
|
|
||||||
without digest =? Poseidon2Tree.digest(blk.data & self.blockPadBytes, self.cellSize), err:
|
|
||||||
error "Failed to create digest for block"
|
|
||||||
return failure(err)
|
|
||||||
|
|
||||||
digest
|
|
||||||
|
|
||||||
success hashes
|
|
||||||
|
|
||||||
proc buildSlotTree*(
|
|
||||||
self: SlotsBuilder,
|
|
||||||
slotIndex: int): Future[?!Poseidon2Tree] {.async.} =
|
|
||||||
without cellHashes =? (await self.getCellHashes(slotIndex)), err:
|
|
||||||
error "Failed to select slot blocks", err = err.msg
|
|
||||||
return failure(err)
|
|
||||||
|
|
||||||
Poseidon2Tree.init(cellHashes & self.slotsPadLeafs)
|
|
||||||
|
|
||||||
proc buildSlot*(
|
|
||||||
self: SlotsBuilder,
|
|
||||||
slotIndex: Natural): Future[?!Poseidon2Hash] {.async.} =
|
|
||||||
## 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
|
|
||||||
treeCid =? tree.root.?toSlotCid, err:
|
|
||||||
error "Failed to build slot tree", err = err.msg
|
|
||||||
return failure(err)
|
|
||||||
|
|
||||||
trace "Storing slot tree", treeCid, slotIndex, leaves = tree.leavesCount
|
|
||||||
for i, leaf in tree.leaves:
|
|
||||||
without cellCid =? leaf.toCellCid, err:
|
|
||||||
error "Failed to get CID for slot cell", err = err.msg
|
|
||||||
return failure(err)
|
|
||||||
|
|
||||||
without proof =? tree.getProof(i) and
|
|
||||||
encodableProof =? proof.toEncodableProof, err:
|
|
||||||
error "Failed to get proof for slot tree", err = err.msg
|
|
||||||
return failure(err)
|
|
||||||
|
|
||||||
if err =? (await self.store.putCidAndProof(
|
|
||||||
treeCid, i, cellCid, encodableProof)).errorOption:
|
|
||||||
error "Failed to store slot tree", err = err.msg
|
|
||||||
return failure(err)
|
|
||||||
|
|
||||||
tree.root()
|
|
||||||
|
|
||||||
func buildRootsTree*(
|
|
||||||
self: SlotsBuilder,
|
|
||||||
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:
|
|
||||||
without slotRoot =? (await self.buildSlot(i)), err:
|
|
||||||
error "Failed to build slot", err = err.msg, index = i
|
|
||||||
return failure(err)
|
|
||||||
slotRoot
|
|
||||||
|
|
||||||
without root =? self.buildRootsTree(self.slotRoots).?root(), err:
|
|
||||||
error "Failed to build slot roots tree", err = err.msg
|
|
||||||
return failure(err)
|
|
||||||
|
|
||||||
if self.verifyRoot.isSome and self.verifyRoot.get != root: # TODO: `!=` doesn't work for SecretBool
|
|
||||||
return failure "Existing slots root doesn't match reconstructed root."
|
|
||||||
else:
|
|
||||||
self.verifyRoot = some root
|
|
||||||
|
|
||||||
success()
|
|
||||||
|
|
||||||
proc buildManifest*(self: SlotsBuilder): Future[?!Manifest] {.async.} =
|
|
||||||
if err =? (await self.buildSlots()).errorOption:
|
|
||||||
error "Failed to build slot roots", err = err.msg
|
|
||||||
return failure(err)
|
|
||||||
|
|
||||||
without rootCids =? self.slotRoots.toSlotCids(), err:
|
|
||||||
error "Failed to map slot roots to CIDs", err = err.msg
|
|
||||||
return failure(err)
|
|
||||||
|
|
||||||
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)
|
|
||||||
|
|
||||||
Manifest.new(self.manifest, rootProvingCid, rootCids)
|
|
||||||
|
|
||||||
proc new*(
|
|
||||||
T: type SlotsBuilder,
|
|
||||||
store: BlockStore,
|
|
||||||
manifest: Manifest,
|
|
||||||
strategy: ?IndexingStrategy = none IndexingStrategy,
|
|
||||||
cellSize = CellSize): ?!SlotsBuilder =
|
|
||||||
|
|
||||||
if not manifest.protected:
|
|
||||||
return failure("Can only create SlotsBuilder using protected manifests.")
|
|
||||||
|
|
||||||
if (manifest.blocksCount mod manifest.numSlots) != 0:
|
|
||||||
return failure("Number of blocks must be divisable by number of slots.")
|
|
||||||
|
|
||||||
if (manifest.blockSize.int mod cellSize) != 0:
|
|
||||||
return failure("Block size must be divisable by cell size.")
|
|
||||||
|
|
||||||
let
|
|
||||||
strategy = if strategy.isNone:
|
|
||||||
? SteppedIndexingStrategy.new(
|
|
||||||
0, manifest.blocksCount - 1, manifest.numSlots).catch
|
|
||||||
else:
|
|
||||||
strategy.get
|
|
||||||
|
|
||||||
# all trees have to be padded to power of two
|
|
||||||
numBlockCells = manifest.blockSize.int div cellSize # number of cells per block
|
|
||||||
blockPadBytes = newSeq[byte](numBlockCells.nextPowerOfTwoPad * cellSize) # power of two padding for blocks
|
|
||||||
numSlotLeafs = (manifest.blocksCount div manifest.numSlots)
|
|
||||||
slotsPadLeafs = newSeqWith(numSlotLeafs.nextPowerOfTwoPad, Poseidon2Zero) # power of two padding for block roots
|
|
||||||
rootsPadLeafs = newSeqWith(manifest.numSlots.nextPowerOfTwoPad, Poseidon2Zero)
|
|
||||||
blockEmptyDigest = ? Poseidon2Tree.digest(DefaultEmptyBlock & blockPadBytes, CellSize)
|
|
||||||
|
|
||||||
var self = SlotsBuilder(
|
|
||||||
store: store,
|
|
||||||
manifest: manifest,
|
|
||||||
strategy: strategy,
|
|
||||||
cellSize: cellSize,
|
|
||||||
blockPadBytes: blockPadBytes,
|
|
||||||
slotsPadLeafs: slotsPadLeafs,
|
|
||||||
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 roots are missing or invalid."
|
|
||||||
|
|
||||||
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
|
|
||||||
|
|
|
@ -0,0 +1,370 @@
|
||||||
|
## 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.
|
||||||
|
|
||||||
|
{.push raises: [].}
|
||||||
|
|
||||||
|
import std/math
|
||||||
|
import std/sequtils
|
||||||
|
import std/sugar
|
||||||
|
|
||||||
|
import pkg/libp2p
|
||||||
|
import pkg/chronos
|
||||||
|
import pkg/chronicles
|
||||||
|
import pkg/questionable
|
||||||
|
import pkg/questionable/results
|
||||||
|
import pkg/poseidon2
|
||||||
|
import pkg/poseidon2/io
|
||||||
|
import pkg/constantine/math/arithmetic/finite_fields
|
||||||
|
|
||||||
|
import ../../indexingstrategy
|
||||||
|
import ../../merkletree
|
||||||
|
import ../../stores
|
||||||
|
import ../../manifest
|
||||||
|
import ../../utils
|
||||||
|
import ../../utils/asynciter
|
||||||
|
import ../../utils/digest
|
||||||
|
import ../../utils/poseidon2digest
|
||||||
|
import ../converters
|
||||||
|
|
||||||
|
export converters
|
||||||
|
|
||||||
|
logScope:
|
||||||
|
topics = "codex slotsbuilder"
|
||||||
|
|
||||||
|
const
|
||||||
|
# TODO: Unified with the DefaultCellSize specified in branch "data-sampler"
|
||||||
|
# in the proving circuit.
|
||||||
|
|
||||||
|
DefaultEmptyBlock* = newSeq[byte](DefaultBlockSize.int)
|
||||||
|
DefaultEmptyCell* = newSeq[byte](DefaultCellSize.int)
|
||||||
|
|
||||||
|
type
|
||||||
|
# TODO: should be a generic type that
|
||||||
|
# supports all merkle trees
|
||||||
|
SlotsBuilder* = ref object of RootObj
|
||||||
|
store: BlockStore
|
||||||
|
manifest: Manifest
|
||||||
|
strategy: IndexingStrategy
|
||||||
|
cellSize: NBytes
|
||||||
|
emptyDigestTree: Poseidon2Tree
|
||||||
|
blockPadBytes: seq[byte]
|
||||||
|
slotsPadLeafs: seq[Poseidon2Hash]
|
||||||
|
rootsPadLeafs: seq[Poseidon2Hash]
|
||||||
|
slotRoots: seq[Poseidon2Hash]
|
||||||
|
verifyTree: ?Poseidon2Tree
|
||||||
|
|
||||||
|
func slotRoots*(self: SlotsBuilder): seq[Poseidon2Hash] {.inline.} =
|
||||||
|
## Returns the slot roots.
|
||||||
|
##
|
||||||
|
|
||||||
|
self.slotRoots
|
||||||
|
|
||||||
|
func verifyTree*(self: SlotsBuilder): ?Poseidon2Tree {.inline.} =
|
||||||
|
## Returns the slots tree (verification tree).
|
||||||
|
##
|
||||||
|
|
||||||
|
self.verifyTree
|
||||||
|
|
||||||
|
func verifyRoot*(self: SlotsBuilder): ?Poseidon2Hash {.inline.} =
|
||||||
|
## Returns the slots root (verification root).
|
||||||
|
##
|
||||||
|
|
||||||
|
self.verifyTree.?root().?toOption
|
||||||
|
|
||||||
|
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
|
||||||
|
## merkle tree for each block.
|
||||||
|
##
|
||||||
|
|
||||||
|
self.blockPadBytes.len
|
||||||
|
|
||||||
|
func numSlotsPadLeafs*(self: SlotsBuilder): Natural =
|
||||||
|
## Number of padding field elements required for a pow2
|
||||||
|
## merkle tree for each slot.
|
||||||
|
##
|
||||||
|
|
||||||
|
self.slotsPadLeafs.len
|
||||||
|
|
||||||
|
func numRootsPadLeafs*(self: SlotsBuilder): Natural =
|
||||||
|
## Number of padding field elements required for a pow2
|
||||||
|
## merkle tree for the slot roots.
|
||||||
|
##
|
||||||
|
|
||||||
|
self.rootsPadLeafs.len
|
||||||
|
|
||||||
|
func numSlots*(self: SlotsBuilder): Natural =
|
||||||
|
## Number of slots.
|
||||||
|
##
|
||||||
|
|
||||||
|
self.manifest.numSlots
|
||||||
|
|
||||||
|
func numSlotBlocks*(self: SlotsBuilder): Natural =
|
||||||
|
## Number of blocks per slot.
|
||||||
|
##
|
||||||
|
|
||||||
|
self.manifest.blocksCount div self.manifest.numSlots
|
||||||
|
|
||||||
|
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.
|
||||||
|
##
|
||||||
|
|
||||||
|
(self.manifest.blockSize div self.cellSize).Natural
|
||||||
|
|
||||||
|
func cellSize*(self: SlotsBuilder): NBytes =
|
||||||
|
## Cell size.
|
||||||
|
##
|
||||||
|
|
||||||
|
self.cellSize
|
||||||
|
|
||||||
|
func numSlotCells*(self: SlotsBuilder): Natural =
|
||||||
|
## Number of cells per slot.
|
||||||
|
##
|
||||||
|
|
||||||
|
self.numBlockCells * self.numSlotBlocks
|
||||||
|
|
||||||
|
func slotIndiciesIter*(self: SlotsBuilder, slot: Natural): ?!Iter[int] =
|
||||||
|
## Returns the slot indices.
|
||||||
|
##
|
||||||
|
|
||||||
|
self.strategy.getIndicies(slot).catch
|
||||||
|
|
||||||
|
func slotIndicies*(self: SlotsBuilder, slot: Natural): seq[int] =
|
||||||
|
## Returns the slot indices.
|
||||||
|
##
|
||||||
|
|
||||||
|
if iter =? self.strategy.getIndicies(slot).catch:
|
||||||
|
toSeq(iter)
|
||||||
|
else:
|
||||||
|
trace "Failed to get slot indicies"
|
||||||
|
newSeq[int]()
|
||||||
|
|
||||||
|
func manifest*(self: SlotsBuilder): Manifest =
|
||||||
|
## Returns the manifest.
|
||||||
|
##
|
||||||
|
|
||||||
|
self.manifest
|
||||||
|
|
||||||
|
proc buildBlockTree*(
|
||||||
|
self: SlotsBuilder,
|
||||||
|
blkIdx: Natural): Future[?!(seq[byte], Poseidon2Tree)] {.async.} =
|
||||||
|
without blk =? await self.store.getBlock(self.manifest.treeCid, blkIdx), err:
|
||||||
|
error "Failed to get block CID for tree at index"
|
||||||
|
return failure(err)
|
||||||
|
|
||||||
|
if blk.isEmpty:
|
||||||
|
success (DefaultEmptyBlock & self.blockPadBytes, self.emptyDigestTree)
|
||||||
|
else:
|
||||||
|
without tree =?
|
||||||
|
Poseidon2Tree.digestTree(blk.data & self.blockPadBytes, self.cellSize.int), err:
|
||||||
|
error "Failed to create digest for block"
|
||||||
|
return failure(err)
|
||||||
|
|
||||||
|
success (blk.data, tree)
|
||||||
|
|
||||||
|
proc getCellHashes*(
|
||||||
|
self: SlotsBuilder,
|
||||||
|
slotIndex: Natural): Future[?!seq[Poseidon2Hash]] {.async.} =
|
||||||
|
|
||||||
|
let
|
||||||
|
treeCid = self.manifest.treeCid
|
||||||
|
blockCount = self.manifest.blocksCount
|
||||||
|
numberOfSlots = self.manifest.numSlots
|
||||||
|
|
||||||
|
logScope:
|
||||||
|
treeCid = treeCid
|
||||||
|
blockCount = blockCount
|
||||||
|
numberOfSlots = numberOfSlots
|
||||||
|
index = blkIdx
|
||||||
|
slotIndex = slotIndex
|
||||||
|
|
||||||
|
let
|
||||||
|
hashes: seq[Poseidon2Hash] = collect(newSeq):
|
||||||
|
for blkIdx in self.strategy.getIndicies(slotIndex):
|
||||||
|
trace "Getting block CID for tree at index"
|
||||||
|
|
||||||
|
without (_, tree) =? (await self.buildBlockTree(blkIdx)) and
|
||||||
|
digest =? tree.root, err:
|
||||||
|
error "Failed to get block CID for tree at index", err = err.msg
|
||||||
|
return failure(err)
|
||||||
|
|
||||||
|
digest
|
||||||
|
|
||||||
|
success hashes
|
||||||
|
|
||||||
|
proc buildSlotTree*(
|
||||||
|
self: SlotsBuilder,
|
||||||
|
slotIndex: Natural): Future[?!Poseidon2Tree] {.async.} =
|
||||||
|
without cellHashes =? (await self.getCellHashes(slotIndex)), err:
|
||||||
|
error "Failed to select slot blocks", err = err.msg
|
||||||
|
return failure(err)
|
||||||
|
|
||||||
|
Poseidon2Tree.init(cellHashes & self.slotsPadLeafs)
|
||||||
|
|
||||||
|
proc buildSlot*(
|
||||||
|
self: SlotsBuilder,
|
||||||
|
slotIndex: Natural): Future[?!Poseidon2Hash] {.async.} =
|
||||||
|
## 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
|
||||||
|
treeCid =? tree.root.?toSlotCid, err:
|
||||||
|
error "Failed to build slot tree", err = err.msg
|
||||||
|
return failure(err)
|
||||||
|
|
||||||
|
trace "Storing slot tree", treeCid, slotIndex, leaves = tree.leavesCount
|
||||||
|
for i, leaf in tree.leaves:
|
||||||
|
without cellCid =? leaf.toCellCid, err:
|
||||||
|
error "Failed to get CID for slot cell", err = err.msg
|
||||||
|
return failure(err)
|
||||||
|
|
||||||
|
without proof =? tree.getProof(i) and
|
||||||
|
encodableProof =? proof.toEncodableProof, err:
|
||||||
|
error "Failed to get proof for slot tree", err = err.msg
|
||||||
|
return failure(err)
|
||||||
|
|
||||||
|
if err =? (await self.store.putCidAndProof(
|
||||||
|
treeCid, i, cellCid, encodableProof)).errorOption:
|
||||||
|
error "Failed to store slot tree", err = err.msg
|
||||||
|
return failure(err)
|
||||||
|
|
||||||
|
tree.root()
|
||||||
|
|
||||||
|
func buildVerifyTree*(
|
||||||
|
self: SlotsBuilder,
|
||||||
|
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:
|
||||||
|
without slotRoot =? (await self.buildSlot(i)), err:
|
||||||
|
error "Failed to build slot", err = err.msg, index = i
|
||||||
|
return failure(err)
|
||||||
|
slotRoot
|
||||||
|
|
||||||
|
without tree =? self.buildVerifyTree(self.slotRoots) and root =? tree.root, err:
|
||||||
|
error "Failed to build slot roots tree", err = err.msg
|
||||||
|
return failure(err)
|
||||||
|
|
||||||
|
if verifyTree =? self.verifyTree and verifyRoot =? verifyTree.root:
|
||||||
|
if verifyRoot != root: # TODO: `!=` doesn't work for SecretBool
|
||||||
|
return failure "Existing slots root doesn't match reconstructed root."
|
||||||
|
|
||||||
|
self.verifyTree = some tree
|
||||||
|
|
||||||
|
success()
|
||||||
|
|
||||||
|
proc buildManifest*(self: SlotsBuilder): Future[?!Manifest] {.async.} =
|
||||||
|
if err =? (await self.buildSlots()).errorOption:
|
||||||
|
error "Failed to build slot roots", err = err.msg
|
||||||
|
return failure(err)
|
||||||
|
|
||||||
|
without rootCids =? self.slotRoots.toSlotCids(), err:
|
||||||
|
error "Failed to map slot roots to CIDs", err = err.msg
|
||||||
|
return failure(err)
|
||||||
|
|
||||||
|
without rootProvingCidRes =? self.verifyRoot.?toVerifyCid() 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)
|
||||||
|
|
||||||
|
Manifest.new(self.manifest, rootProvingCid, rootCids)
|
||||||
|
|
||||||
|
proc new*(
|
||||||
|
T: type SlotsBuilder,
|
||||||
|
store: BlockStore,
|
||||||
|
manifest: Manifest,
|
||||||
|
strategy: ?IndexingStrategy = none IndexingStrategy,
|
||||||
|
cellSize = DefaultCellSize): ?!SlotsBuilder =
|
||||||
|
|
||||||
|
if not manifest.protected:
|
||||||
|
return failure("Can only create SlotsBuilder using protected manifests.")
|
||||||
|
|
||||||
|
if (manifest.blocksCount mod manifest.numSlots) != 0:
|
||||||
|
return failure("Number of blocks must be divisable by number of slots.")
|
||||||
|
|
||||||
|
if (manifest.blockSize mod cellSize) != 0.NBytes:
|
||||||
|
return failure("Block size must be divisable by cell size.")
|
||||||
|
|
||||||
|
let
|
||||||
|
strategy = if strategy.isNone:
|
||||||
|
? SteppedIndexingStrategy.new(
|
||||||
|
0, manifest.blocksCount - 1, manifest.numSlots).catch
|
||||||
|
else:
|
||||||
|
strategy.get
|
||||||
|
|
||||||
|
# all trees have to be padded to power of two
|
||||||
|
numBlockCells = (manifest.blockSize div cellSize).int # number of cells per block
|
||||||
|
blockPadBytes = newSeq[byte](numBlockCells.nextPowerOfTwoPad * cellSize.int) # power of two padding for blocks
|
||||||
|
numSlotLeafs = (manifest.blocksCount div manifest.numSlots)
|
||||||
|
slotsPadLeafs = newSeqWith(numSlotLeafs.nextPowerOfTwoPad, Poseidon2Zero) # power of two padding for block roots
|
||||||
|
rootsPadLeafs = newSeqWith(manifest.numSlots.nextPowerOfTwoPad, Poseidon2Zero)
|
||||||
|
emptyDigestTree = ? Poseidon2Tree.digestTree(DefaultEmptyBlock & blockPadBytes, DefaultCellSize.int)
|
||||||
|
|
||||||
|
var self = SlotsBuilder(
|
||||||
|
store: store,
|
||||||
|
manifest: manifest,
|
||||||
|
strategy: strategy,
|
||||||
|
cellSize: cellSize,
|
||||||
|
blockPadBytes: blockPadBytes,
|
||||||
|
slotsPadLeafs: slotsPadLeafs,
|
||||||
|
rootsPadLeafs: rootsPadLeafs,
|
||||||
|
emptyDigestTree: emptyDigestTree)
|
||||||
|
|
||||||
|
if manifest.verifiable:
|
||||||
|
if manifest.slotRoots.len == 0 or manifest.slotRoots.len != manifest.numSlots:
|
||||||
|
return failure "Manifest is verifiable but slot roots are missing or invalid."
|
||||||
|
|
||||||
|
let slotRoots = manifest.slotRoots.mapIt( (? it.fromSlotCid()))
|
||||||
|
|
||||||
|
without tree =? self.buildVerifyTree(slotRoots), err:
|
||||||
|
error "Failed to build slot roots tree", err = err.msg
|
||||||
|
return failure(err)
|
||||||
|
|
||||||
|
without expectedRoot =? manifest.verifyRoot.fromVerifyCid(), err:
|
||||||
|
error "Unable to convert manifest verifyRoot to hash", error = err.msg
|
||||||
|
return failure(err)
|
||||||
|
|
||||||
|
if verifyRoot =? tree.root:
|
||||||
|
if verifyRoot != expectedRoot:
|
||||||
|
return failure "Existing slots root doesn't match reconstructed root."
|
||||||
|
|
||||||
|
self.slotRoots = slotRoots
|
||||||
|
self.verifyTree = some tree
|
||||||
|
|
||||||
|
success self
|
|
@ -10,6 +10,7 @@
|
||||||
import std/sequtils
|
import std/sequtils
|
||||||
|
|
||||||
import pkg/libp2p
|
import pkg/libp2p
|
||||||
|
import pkg/stew/arrayops
|
||||||
import pkg/questionable
|
import pkg/questionable
|
||||||
import pkg/questionable/results
|
import pkg/questionable/results
|
||||||
import pkg/poseidon2
|
import pkg/poseidon2
|
||||||
|
@ -18,37 +19,55 @@ import pkg/poseidon2/io
|
||||||
import ../codextypes
|
import ../codextypes
|
||||||
import ../merkletree
|
import ../merkletree
|
||||||
import ../errors
|
import ../errors
|
||||||
|
import ../utils/digest
|
||||||
|
|
||||||
func toCellCid*(cell: Poseidon2Hash): ?!Cid =
|
func toCid(hash: Poseidon2Hash, mcodec: MultiCodec, cidCodec: MultiCodec): ?!Cid =
|
||||||
let
|
let
|
||||||
cellMhash = ? MultiHash.init(Pos2Bn128MrklCodec, cell.toBytes).mapFailure
|
mhash = ? MultiHash.init(mcodec, hash.toBytes).mapFailure
|
||||||
cellCid = ? Cid.init(CIDv1, CodexSlotCellCodec, cellMhash).mapFailure
|
treeCid = ? Cid.init(CIDv1, cidCodec, mhash).mapFailure
|
||||||
|
|
||||||
success cellCid
|
|
||||||
|
|
||||||
func toSlotCid*(root: Poseidon2Hash): ?!Cid =
|
|
||||||
let
|
|
||||||
mhash = ? MultiHash.init($multiCodec("identity"), root.toBytes).mapFailure
|
|
||||||
treeCid = ? Cid.init(CIDv1, SlotRootCodec, mhash).mapFailure
|
|
||||||
|
|
||||||
success treeCid
|
success treeCid
|
||||||
|
|
||||||
|
proc toPoseidon2Hash(cid: Cid, mcodec: MultiCodec, cidCodec: MultiCodec): ?!Poseidon2Hash =
|
||||||
|
if cid.cidver != CIDv1:
|
||||||
|
return failure("Unexpected CID version")
|
||||||
|
|
||||||
|
if cid.mcodec != cidCodec:
|
||||||
|
return failure("Cid is not of expected codec. Was: " & $cid.mcodec & " but expected: " & $cidCodec)
|
||||||
|
|
||||||
|
let
|
||||||
|
mhash = ? cid.mhash.mapFailure
|
||||||
|
bytes: array[32, byte] = array[32, byte].initCopyFrom(mhash.digestBytes())
|
||||||
|
hash = ? Poseidon2Hash.fromBytes(bytes).toFailure
|
||||||
|
|
||||||
|
success hash
|
||||||
|
|
||||||
|
func toCellCid*(hash: Poseidon2Hash): ?!Cid =
|
||||||
|
toCid(hash, Pos2Bn128MrklCodec, CodexSlotCellCodec)
|
||||||
|
|
||||||
|
func fromCellCid*(cid: Cid): ?!Poseidon2Hash =
|
||||||
|
toPoseidon2Hash(cid, Pos2Bn128MrklCodec, CodexSlotCellCodec)
|
||||||
|
|
||||||
|
func toSlotCid*(hash: Poseidon2Hash): ?!Cid =
|
||||||
|
toCid(hash, multiCodec("identity"), SlotRootCodec)
|
||||||
|
|
||||||
func toSlotCids*(slotRoots: openArray[Poseidon2Hash]): ?!seq[Cid] =
|
func toSlotCids*(slotRoots: openArray[Poseidon2Hash]): ?!seq[Cid] =
|
||||||
success slotRoots.mapIt( ? it.toSlotCid )
|
success slotRoots.mapIt( ? it.toSlotCid )
|
||||||
|
|
||||||
func toSlotsRootsCid*(root: Poseidon2Hash): ?!Cid =
|
func fromSlotCid*(cid: Cid): ?!Poseidon2Hash =
|
||||||
let
|
toPoseidon2Hash(cid, multiCodec("identity"), SlotRootCodec)
|
||||||
mhash = ? MultiHash.init($multiCodec("identity"), root.toBytes).mapFailure
|
|
||||||
treeCid = ? Cid.init(CIDv1, SlotProvingRootCodec, mhash).mapFailure
|
|
||||||
|
|
||||||
success treeCid
|
func toVerifyCid*(hash: Poseidon2Hash): ?!Cid =
|
||||||
|
toCid(hash, multiCodec("identity"), SlotProvingRootCodec)
|
||||||
|
|
||||||
|
func fromVerifyCid*(cid: Cid): ?!Poseidon2Hash =
|
||||||
|
toPoseidon2Hash(cid, multiCodec("identity"), SlotProvingRootCodec)
|
||||||
|
|
||||||
func toEncodableProof*(
|
func toEncodableProof*(
|
||||||
proof: Poseidon2Proof): ?!CodexProof =
|
proof: Poseidon2Proof): ?!CodexProof =
|
||||||
|
|
||||||
let
|
let
|
||||||
encodableProof = CodexProof(
|
encodableProof = CodexProof(
|
||||||
mcodec: multiCodec("identity"), # copy bytes as is
|
mcodec: multiCodec("identity"),
|
||||||
index: proof.index,
|
index: proof.index,
|
||||||
nleaves: proof.nleaves,
|
nleaves: proof.nleaves,
|
||||||
path: proof.path.mapIt( @( it.toBytes ) ))
|
path: proof.path.mapIt( @( it.toBytes ) ))
|
||||||
|
|
|
@ -0,0 +1,4 @@
|
||||||
|
import ./sampler/sampler
|
||||||
|
import ./sampler/utils
|
||||||
|
|
||||||
|
export sampler, utils
|
|
@ -0,0 +1,163 @@
|
||||||
|
## 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 std/sugar
|
||||||
|
import std/sequtils
|
||||||
|
|
||||||
|
import pkg/chronicles
|
||||||
|
import pkg/chronos
|
||||||
|
import pkg/questionable
|
||||||
|
import pkg/questionable/results
|
||||||
|
import pkg/constantine/math/arithmetic
|
||||||
|
import pkg/poseidon2
|
||||||
|
import pkg/poseidon2/types
|
||||||
|
import pkg/poseidon2/io
|
||||||
|
import pkg/stew/arrayops
|
||||||
|
|
||||||
|
import ../../market
|
||||||
|
import ../../blocktype as bt
|
||||||
|
import ../../merkletree
|
||||||
|
import ../../manifest
|
||||||
|
import ../../stores
|
||||||
|
|
||||||
|
import ../builder
|
||||||
|
|
||||||
|
import ./utils
|
||||||
|
|
||||||
|
logScope:
|
||||||
|
topics = "codex datasampler"
|
||||||
|
|
||||||
|
type
|
||||||
|
Cell* = seq[byte]
|
||||||
|
|
||||||
|
Sample* = object
|
||||||
|
data*: Cell
|
||||||
|
slotProof*: Poseidon2Proof
|
||||||
|
cellProof*: Poseidon2Proof
|
||||||
|
slotBlockIdx*: Natural
|
||||||
|
blockCellIdx*: Natural
|
||||||
|
|
||||||
|
ProofInput* = object
|
||||||
|
entropy*: Poseidon2Hash
|
||||||
|
verifyRoot*: Poseidon2Hash
|
||||||
|
verifyProof*: Poseidon2Proof
|
||||||
|
numSlots*: Natural
|
||||||
|
numCells*: Natural
|
||||||
|
slotIndex*: Natural
|
||||||
|
samples*: seq[Sample]
|
||||||
|
|
||||||
|
DataSampler* = ref object of RootObj
|
||||||
|
index: Natural
|
||||||
|
blockStore: BlockStore
|
||||||
|
# The following data is invariant over time for a given slot:
|
||||||
|
builder: SlotsBuilder
|
||||||
|
|
||||||
|
proc new*(
|
||||||
|
T: type DataSampler,
|
||||||
|
index: Natural,
|
||||||
|
blockStore: BlockStore,
|
||||||
|
builder: SlotsBuilder): ?!DataSampler =
|
||||||
|
|
||||||
|
if index > builder.slotRoots.high:
|
||||||
|
error "Slot index is out of range"
|
||||||
|
return failure("Slot index is out of range")
|
||||||
|
|
||||||
|
success DataSampler(
|
||||||
|
index: index,
|
||||||
|
blockStore: blockStore,
|
||||||
|
builder: builder)
|
||||||
|
|
||||||
|
proc getCell*(self: DataSampler, blkBytes: seq[byte], blkCellIdx: Natural): Cell =
|
||||||
|
let
|
||||||
|
cellSize = self.builder.cellSize.uint64
|
||||||
|
dataStart = cellSize * blkCellIdx.uint64
|
||||||
|
dataEnd = dataStart + cellSize
|
||||||
|
return blkBytes[dataStart ..< dataEnd]
|
||||||
|
|
||||||
|
proc getProofInput*(
|
||||||
|
self: DataSampler,
|
||||||
|
entropy: ProofChallenge,
|
||||||
|
nSamples: Natural): Future[?!ProofInput] {.async.} =
|
||||||
|
## Generate proofs as input to the proving circuit.
|
||||||
|
##
|
||||||
|
|
||||||
|
let
|
||||||
|
entropy = Poseidon2Hash.fromBytes(
|
||||||
|
array[31, byte].initCopyFrom(entropy[0..30])) # truncate to 31 bytes, otherwise it _might_ be greater than mod
|
||||||
|
|
||||||
|
without verifyTree =? self.builder.verifyTree and
|
||||||
|
verifyProof =? verifyTree.getProof(self.index) and
|
||||||
|
verifyRoot =? verifyTree.root(), err:
|
||||||
|
error "Failed to get slot proof from verify tree", err = err.msg
|
||||||
|
return failure(err)
|
||||||
|
|
||||||
|
let
|
||||||
|
slotTreeCid = self.builder.manifest.slotRoots[self.index]
|
||||||
|
cellsPerBlock = self.builder.numBlockCells
|
||||||
|
cellIdxs = entropy.cellIndices(
|
||||||
|
self.builder.slotRoots[self.index],
|
||||||
|
self.builder.numSlotCells,
|
||||||
|
nSamples)
|
||||||
|
|
||||||
|
logScope:
|
||||||
|
index = self.index
|
||||||
|
samples = nSamples
|
||||||
|
cells = cellIdxs
|
||||||
|
slotTreeCid = slotTreeCid
|
||||||
|
|
||||||
|
trace "Collecting input for proof"
|
||||||
|
let samples = collect(newSeq):
|
||||||
|
for cellIdx in cellIdxs:
|
||||||
|
let
|
||||||
|
blkCellIdx = cellIdx.toBlockCellIdx(cellsPerBlock) # block cell index
|
||||||
|
slotCellIdx = cellIdx.toBlockIdx(cellsPerBlock) # slot tree index
|
||||||
|
|
||||||
|
logScope:
|
||||||
|
cellIdx = cellIdx
|
||||||
|
slotCellIdx = slotCellIdx
|
||||||
|
blkCellIdx = blkCellIdx
|
||||||
|
|
||||||
|
without (cid, proof) =? await self.blockStore.getCidAndProof(
|
||||||
|
slotTreeCid,
|
||||||
|
slotCellIdx.Natural), err:
|
||||||
|
error "Failed to get block from block store", err = err.msg
|
||||||
|
return failure(err)
|
||||||
|
|
||||||
|
without slotProof =? proof.toVerifiableProof(), err:
|
||||||
|
error "Unable to convert slot proof to poseidon proof", error = err.msg
|
||||||
|
return failure(err)
|
||||||
|
|
||||||
|
# This converts our slotBlockIndex to a datasetBlockIndex using the
|
||||||
|
# indexing-strategy used by the builder.
|
||||||
|
# We need this to fetch the block data. We can't do it by slotTree + slotBlkIdx.
|
||||||
|
let datasetBlockIndex = self.builder.slotIndicies(self.index)[slotCellIdx]
|
||||||
|
|
||||||
|
without (bytes, blkTree) =? await self.builder.buildBlockTree(datasetBlockIndex), err:
|
||||||
|
error "Failed to build block tree", err = err.msg
|
||||||
|
return failure(err)
|
||||||
|
|
||||||
|
without blockProof =? blkTree.getProof(blkCellIdx), err:
|
||||||
|
error "Failed to get proof from block tree", err = err.msg
|
||||||
|
return failure(err)
|
||||||
|
|
||||||
|
Sample(
|
||||||
|
data: self.getCell(bytes, blkCellIdx),
|
||||||
|
slotProof: slotProof,
|
||||||
|
cellProof: blockProof,
|
||||||
|
slotBlockIdx: slotCellIdx.Natural,
|
||||||
|
blockCellIdx: blkCellIdx.Natural)
|
||||||
|
|
||||||
|
success ProofInput(
|
||||||
|
entropy: entropy,
|
||||||
|
verifyRoot: verifyRoot,
|
||||||
|
verifyProof: verifyProof,
|
||||||
|
numSlots: self.builder.numSlots,
|
||||||
|
numCells: self.builder.numSlotCells,
|
||||||
|
slotIndex: self.index,
|
||||||
|
samples: samples)
|
|
@ -0,0 +1,83 @@
|
||||||
|
## Nim-Codex
|
||||||
|
## Copyright (c) 2024 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 std/sugar
|
||||||
|
import std/bitops
|
||||||
|
|
||||||
|
import pkg/poseidon2
|
||||||
|
import pkg/poseidon2/io
|
||||||
|
|
||||||
|
import pkg/constantine/math/arithmetic
|
||||||
|
|
||||||
|
import pkg/constantine/math/io/io_fields
|
||||||
|
|
||||||
|
import ../../merkletree
|
||||||
|
|
||||||
|
func extractLowBits*[n: static int](elm: BigInt[n], k: int): uint64 =
|
||||||
|
assert( k > 0 and k <= 64 )
|
||||||
|
var r = 0'u64
|
||||||
|
for i in 0..<k:
|
||||||
|
let b = bit[n](elm, i)
|
||||||
|
let y = uint64(b)
|
||||||
|
if (y != 0):
|
||||||
|
r = bitor( r, 1'u64 shl i )
|
||||||
|
r
|
||||||
|
|
||||||
|
func extractLowBits(fld: Poseidon2Hash, k: int): uint64 =
|
||||||
|
let elm : BigInt[254] = fld.toBig()
|
||||||
|
return extractLowBits(elm, k);
|
||||||
|
|
||||||
|
func floorLog2*(x : int) : int =
|
||||||
|
var k = -1
|
||||||
|
var y = x
|
||||||
|
while (y > 0):
|
||||||
|
k += 1
|
||||||
|
y = y shr 1
|
||||||
|
return k
|
||||||
|
|
||||||
|
func ceilingLog2*(x : int) : int =
|
||||||
|
if (x == 0):
|
||||||
|
return -1
|
||||||
|
else:
|
||||||
|
return (floorLog2(x-1) + 1)
|
||||||
|
|
||||||
|
func toBlockIdx*(cell: Natural, numCells: Natural): Natural =
|
||||||
|
let log2 = ceilingLog2(numCells)
|
||||||
|
doAssert( 1 shl log2 == numCells , "`numCells` is assumed to be a power of two" )
|
||||||
|
|
||||||
|
return cell div numCells
|
||||||
|
|
||||||
|
func toBlockCellIdx*(cell: Natural, numCells: Natural): Natural =
|
||||||
|
let log2 = ceilingLog2(numCells)
|
||||||
|
doAssert( 1 shl log2 == numCells , "`numCells` is assumed to be a power of two" )
|
||||||
|
|
||||||
|
return cell mod numCells
|
||||||
|
|
||||||
|
func cellIndex*(
|
||||||
|
entropy: Poseidon2Hash,
|
||||||
|
slotRoot: Poseidon2Hash,
|
||||||
|
numCells: Natural, counter: Natural): Natural =
|
||||||
|
let log2 = ceilingLog2(numCells)
|
||||||
|
doAssert( 1 shl log2 == numCells , "`numCells` is assumed to be a power of two" )
|
||||||
|
|
||||||
|
let hash = Sponge.digest( @[ slotRoot, entropy, counter.toF ], rate = 2 )
|
||||||
|
|
||||||
|
return int( extractLowBits(hash, log2) )
|
||||||
|
|
||||||
|
func cellIndices*(
|
||||||
|
entropy: Poseidon2Hash,
|
||||||
|
slotRoot: Poseidon2Hash,
|
||||||
|
numCells: Natural, nSamples: Natural): seq[Natural] =
|
||||||
|
|
||||||
|
var indices: seq[Natural]
|
||||||
|
while (indices.len < nSamples):
|
||||||
|
let idx = cellIndex(entropy, slotRoot, numCells, indices.len + 1)
|
||||||
|
indices.add(idx.Natural)
|
||||||
|
indices
|
||||||
|
|
|
@ -89,7 +89,7 @@ method getCidAndProof*(
|
||||||
## Get a block proof from the blockstore
|
## Get a block proof from the blockstore
|
||||||
##
|
##
|
||||||
|
|
||||||
raiseAssert("putCidAndProof not implemented!")
|
raiseAssert("getCidAndProof not implemented!")
|
||||||
|
|
||||||
method ensureExpiry*(
|
method ensureExpiry*(
|
||||||
self: BlockStore,
|
self: BlockStore,
|
||||||
|
|
|
@ -86,6 +86,15 @@ method putCidAndProof*(
|
||||||
proof: CodexProof): Future[?!void] =
|
proof: CodexProof): Future[?!void] =
|
||||||
self.localStore.putCidAndProof(treeCid, index, blockCid, proof)
|
self.localStore.putCidAndProof(treeCid, index, blockCid, proof)
|
||||||
|
|
||||||
|
method getCidAndProof*(
|
||||||
|
self: NetworkStore,
|
||||||
|
treeCid: Cid,
|
||||||
|
index: Natural): Future[?!(Cid, CodexProof)] =
|
||||||
|
## Get a block proof from the blockstore
|
||||||
|
##
|
||||||
|
|
||||||
|
self.localStore.getCidAndProof(treeCid, index)
|
||||||
|
|
||||||
method ensureExpiry*(
|
method ensureExpiry*(
|
||||||
self: NetworkStore,
|
self: NetworkStore,
|
||||||
cid: Cid,
|
cid: Cid,
|
||||||
|
|
|
@ -241,7 +241,7 @@ method ensureExpiry*(
|
||||||
cid: Cid,
|
cid: Cid,
|
||||||
expiry: SecondsSince1970
|
expiry: SecondsSince1970
|
||||||
): Future[?!void] {.async.} =
|
): Future[?!void] {.async.} =
|
||||||
## Ensure that block's assosicated expiry is at least given timestamp
|
## Ensure that block's associated expiry is at least given timestamp
|
||||||
## If the current expiry is lower then it is updated to the given one, otherwise it is left intact
|
## If the current expiry is lower then it is updated to the given one, otherwise it is left intact
|
||||||
##
|
##
|
||||||
|
|
||||||
|
@ -262,8 +262,12 @@ method ensureExpiry*(
|
||||||
error "Could not read datastore key", err = err.msg
|
error "Could not read datastore key", err = err.msg
|
||||||
return failure(err)
|
return failure(err)
|
||||||
|
|
||||||
|
logScope:
|
||||||
|
current = currentExpiry.toSecondsSince1970
|
||||||
|
ensuring = expiry
|
||||||
|
|
||||||
if expiry <= currentExpiry.toSecondsSince1970:
|
if expiry <= currentExpiry.toSecondsSince1970:
|
||||||
trace "Current expiry is larger than or equal to the specified one, no action needed", current = currentExpiry.toSecondsSince1970, ensuring = expiry
|
trace "Expiry is larger than or equal to requested"
|
||||||
return success()
|
return success()
|
||||||
|
|
||||||
if err =? (await self.metaDs.put(expiryKey, expiry.toBytes)).errorOption:
|
if err =? (await self.metaDs.put(expiryKey, expiry.toBytes)).errorOption:
|
||||||
|
|
|
@ -2,12 +2,11 @@ import std/sugar
|
||||||
|
|
||||||
import pkg/questionable
|
import pkg/questionable
|
||||||
import pkg/chronos
|
import pkg/chronos
|
||||||
import pkg/upraises
|
|
||||||
|
|
||||||
type
|
type
|
||||||
Function*[T, U] = proc(fut: T): U {.upraises: [CatchableError], gcsafe, closure.}
|
Function*[T, U] = proc(fut: T): U {.raises: [CatchableError], gcsafe, closure.}
|
||||||
IsFinished* = proc(): bool {.upraises: [], gcsafe, closure.}
|
IsFinished* = proc(): bool {.raises: [], gcsafe, closure.}
|
||||||
GenNext*[T] = proc(): T {.upraises: [CatchableError], gcsafe, closure.}
|
GenNext*[T] = proc(): T {.raises: [CatchableError], gcsafe, closure.}
|
||||||
Iter*[T] = ref object
|
Iter*[T] = ref object
|
||||||
finished: bool
|
finished: bool
|
||||||
next*: GenNext[T]
|
next*: GenNext[T]
|
||||||
|
@ -36,7 +35,7 @@ proc map*[T, U](fut: Future[T], fn: Function[T, U]): Future[U] {.async.} =
|
||||||
proc new*[T](_: type Iter, genNext: GenNext[T], isFinished: IsFinished, finishOnErr: bool = true): Iter[T] =
|
proc new*[T](_: type Iter, genNext: GenNext[T], isFinished: IsFinished, finishOnErr: bool = true): Iter[T] =
|
||||||
var iter = Iter[T]()
|
var iter = Iter[T]()
|
||||||
|
|
||||||
proc next(): T {.upraises: [CatchableError].} =
|
proc next(): T {.raises: [CatchableError].} =
|
||||||
if not iter.finished:
|
if not iter.finished:
|
||||||
var item: T
|
var item: T
|
||||||
try:
|
try:
|
||||||
|
|
|
@ -7,6 +7,7 @@ import pkg/codex/rng
|
||||||
import pkg/codex/stores
|
import pkg/codex/stores
|
||||||
import pkg/codex/blocktype as bt
|
import pkg/codex/blocktype as bt
|
||||||
import pkg/codex/sales
|
import pkg/codex/sales
|
||||||
|
import pkg/codex/merkletree
|
||||||
import ../examples
|
import ../examples
|
||||||
|
|
||||||
export examples
|
export examples
|
||||||
|
@ -71,3 +72,11 @@ proc example*(_: type Reservation): Reservation =
|
||||||
size = uint16.example.u256,
|
size = uint16.example.u256,
|
||||||
slotId = SlotId.example
|
slotId = SlotId.example
|
||||||
)
|
)
|
||||||
|
|
||||||
|
proc example*(_: type MerkleProof): MerkleProof =
|
||||||
|
MerkleProof.init(3, @[MultiHash.example]).tryget()
|
||||||
|
|
||||||
|
proc example*(_: type Poseidon2Proof): Poseidon2Proof =
|
||||||
|
var example = MerkleProof[Poseidon2Hash, PoseidonKeysEnum]()
|
||||||
|
example.index = 123
|
||||||
|
example
|
||||||
|
|
|
@ -129,7 +129,7 @@ asyncchecksuite "Test Node - Host contracts":
|
||||||
(await onStore(request, 1.u256, onBlocks)).tryGet()
|
(await onStore(request, 1.u256, onBlocks)).tryGet()
|
||||||
check fetchedBytes == 786432
|
check fetchedBytes == 786432
|
||||||
|
|
||||||
for index in !builder.slotIndicies(1):
|
for index in builder.slotIndicies(1):
|
||||||
let
|
let
|
||||||
blk = (await localStore.getBlock(verifiable.treeCid, index)).tryGet
|
blk = (await localStore.getBlock(verifiable.treeCid, index)).tryGet
|
||||||
expiryKey = (createBlockExpirationMetadataKey(blk.cid)).tryGet
|
expiryKey = (createBlockExpirationMetadataKey(blk.cid)).tryGet
|
||||||
|
|
|
@ -6,6 +6,7 @@ import pkg/codex/sales/states/initialproving
|
||||||
import pkg/codex/sales/states/cancelled
|
import pkg/codex/sales/states/cancelled
|
||||||
import pkg/codex/sales/states/failed
|
import pkg/codex/sales/states/failed
|
||||||
import pkg/codex/sales/states/filling
|
import pkg/codex/sales/states/filling
|
||||||
|
import pkg/codex/sales/states/errored
|
||||||
import pkg/codex/sales/salesagent
|
import pkg/codex/sales/salesagent
|
||||||
import pkg/codex/sales/salescontext
|
import pkg/codex/sales/salescontext
|
||||||
import pkg/codex/market
|
import pkg/codex/market
|
||||||
|
@ -24,9 +25,9 @@ asyncchecksuite "sales state 'initialproving'":
|
||||||
var receivedChallenge: ProofChallenge
|
var receivedChallenge: ProofChallenge
|
||||||
|
|
||||||
setup:
|
setup:
|
||||||
let onProve = proc (slot: Slot, challenge: ProofChallenge): Future[seq[byte]] {.async.} =
|
let onProve = proc (slot: Slot, challenge: ProofChallenge): Future[?!seq[byte]] {.async.} =
|
||||||
receivedChallenge = challenge
|
receivedChallenge = challenge
|
||||||
return proof
|
return success(proof)
|
||||||
let context = SalesContext(
|
let context = SalesContext(
|
||||||
onProve: onProve.some,
|
onProve: onProve.some,
|
||||||
market: market
|
market: market
|
||||||
|
@ -56,3 +57,19 @@ asyncchecksuite "sales state 'initialproving'":
|
||||||
let future = state.run(agent)
|
let future = state.run(agent)
|
||||||
|
|
||||||
check receivedChallenge == market.proofChallenge
|
check receivedChallenge == market.proofChallenge
|
||||||
|
|
||||||
|
test "switches to errored state when onProve callback fails":
|
||||||
|
let onProveFailed: OnProve = proc(slot: Slot, challenge: ProofChallenge): Future[?!seq[byte]] {.async.} =
|
||||||
|
return failure("oh no!")
|
||||||
|
|
||||||
|
let proofFailedContext = SalesContext(
|
||||||
|
onProve: onProveFailed.some,
|
||||||
|
market: market
|
||||||
|
)
|
||||||
|
agent = newSalesAgent(proofFailedContext,
|
||||||
|
request.id,
|
||||||
|
slotIndex,
|
||||||
|
request.some)
|
||||||
|
|
||||||
|
let next = await state.run(agent)
|
||||||
|
check !next of SaleErrored
|
||||||
|
|
|
@ -28,9 +28,9 @@ asyncchecksuite "sales state 'proving'":
|
||||||
setup:
|
setup:
|
||||||
clock = MockClock.new()
|
clock = MockClock.new()
|
||||||
market = MockMarket.new()
|
market = MockMarket.new()
|
||||||
let onProve = proc (slot: Slot, challenge: ProofChallenge): Future[seq[byte]] {.async.} =
|
let onProve = proc (slot: Slot, challenge: ProofChallenge): Future[?!seq[byte]] {.async.} =
|
||||||
receivedChallenge = challenge
|
receivedChallenge = challenge
|
||||||
return proof
|
return success(proof)
|
||||||
let context = SalesContext(market: market, clock: clock, onProve: onProve.some)
|
let context = SalesContext(market: market, clock: clock, onProve: onProve.some)
|
||||||
agent = newSalesAgent(context,
|
agent = newSalesAgent(context,
|
||||||
request.id,
|
request.id,
|
||||||
|
|
|
@ -44,8 +44,8 @@ asyncchecksuite "sales state 'simulated-proving'":
|
||||||
market.setProofRequired(slot.id, true)
|
market.setProofRequired(slot.id, true)
|
||||||
subscription = await market.subscribeProofSubmission(onProofSubmission)
|
subscription = await market.subscribeProofSubmission(onProofSubmission)
|
||||||
|
|
||||||
let onProve = proc (slot: Slot, challenge: ProofChallenge): Future[seq[byte]] {.async.} =
|
let onProve = proc (slot: Slot, challenge: ProofChallenge): Future[?!seq[byte]] {.async.} =
|
||||||
return proof
|
return success(proof)
|
||||||
let context = SalesContext(market: market, clock: clock, onProve: onProve.some)
|
let context = SalesContext(market: market, clock: clock, onProve: onProve.some)
|
||||||
agent = newSalesAgent(context,
|
agent = newSalesAgent(context,
|
||||||
request.id,
|
request.id,
|
||||||
|
|
|
@ -64,8 +64,8 @@ asyncchecksuite "Sales - start":
|
||||||
return success()
|
return success()
|
||||||
|
|
||||||
queue = sales.context.slotQueue
|
queue = sales.context.slotQueue
|
||||||
sales.onProve = proc(slot: Slot, challenge: ProofChallenge): Future[seq[byte]] {.async.} =
|
sales.onProve = proc(slot: Slot, challenge: ProofChallenge): Future[?!seq[byte]] {.async.} =
|
||||||
return proof
|
return success(proof)
|
||||||
itemsProcessed = @[]
|
itemsProcessed = @[]
|
||||||
request.expiry = (clock.now() + 42).u256
|
request.expiry = (clock.now() + 42).u256
|
||||||
|
|
||||||
|
@ -167,8 +167,8 @@ asyncchecksuite "Sales":
|
||||||
return success()
|
return success()
|
||||||
|
|
||||||
queue = sales.context.slotQueue
|
queue = sales.context.slotQueue
|
||||||
sales.onProve = proc(slot: Slot, challenge: ProofChallenge): Future[seq[byte]] {.async.} =
|
sales.onProve = proc(slot: Slot, challenge: ProofChallenge): Future[?!seq[byte]] {.async.} =
|
||||||
return proof
|
return success(proof)
|
||||||
await sales.start()
|
await sales.start()
|
||||||
itemsProcessed = @[]
|
itemsProcessed = @[]
|
||||||
|
|
||||||
|
@ -369,7 +369,7 @@ asyncchecksuite "Sales":
|
||||||
|
|
||||||
test "handles errors during state run":
|
test "handles errors during state run":
|
||||||
var saleFailed = false
|
var saleFailed = false
|
||||||
sales.onProve = proc(slot: Slot, challenge: ProofChallenge): Future[seq[byte]] {.async.} =
|
sales.onProve = proc(slot: Slot, challenge: ProofChallenge): Future[?!seq[byte]] {.async.} =
|
||||||
# raise exception so machine.onError is called
|
# raise exception so machine.onError is called
|
||||||
raise newException(ValueError, "some error")
|
raise newException(ValueError, "some error")
|
||||||
|
|
||||||
|
@ -394,9 +394,10 @@ asyncchecksuite "Sales":
|
||||||
test "generates proof of storage":
|
test "generates proof of storage":
|
||||||
var provingRequest: StorageRequest
|
var provingRequest: StorageRequest
|
||||||
var provingSlot: UInt256
|
var provingSlot: UInt256
|
||||||
sales.onProve = proc(slot: Slot, challenge: ProofChallenge): Future[seq[byte]] {.async.} =
|
sales.onProve = proc(slot: Slot, challenge: ProofChallenge): Future[?!seq[byte]] {.async.} =
|
||||||
provingRequest = slot.request
|
provingRequest = slot.request
|
||||||
provingSlot = slot.slotIndex
|
provingSlot = slot.slotIndex
|
||||||
|
return success(exampleProof())
|
||||||
createAvailability()
|
createAvailability()
|
||||||
await market.requestStorage(request)
|
await market.requestStorage(request)
|
||||||
check eventually provingRequest == request
|
check eventually provingRequest == request
|
||||||
|
@ -426,7 +427,7 @@ asyncchecksuite "Sales":
|
||||||
test "calls onClear when storage becomes available again":
|
test "calls onClear when storage becomes available again":
|
||||||
# fail the proof intentionally to trigger `agent.finish(success=false)`,
|
# fail the proof intentionally to trigger `agent.finish(success=false)`,
|
||||||
# which then calls the onClear callback
|
# which then calls the onClear callback
|
||||||
sales.onProve = proc(slot: Slot, challenge: ProofChallenge): Future[seq[byte]] {.async.} =
|
sales.onProve = proc(slot: Slot, challenge: ProofChallenge): Future[?!seq[byte]] {.async.} =
|
||||||
raise newException(IOError, "proof failed")
|
raise newException(IOError, "proof failed")
|
||||||
var clearedRequest: StorageRequest
|
var clearedRequest: StorageRequest
|
||||||
var clearedSlotIndex: UInt256
|
var clearedSlotIndex: UInt256
|
||||||
|
|
|
@ -0,0 +1,172 @@
|
||||||
|
import std/sequtils
|
||||||
|
|
||||||
|
import pkg/questionable/results
|
||||||
|
import pkg/poseidon2/io
|
||||||
|
import pkg/poseidon2
|
||||||
|
import pkg/chronos
|
||||||
|
import pkg/codex/stores/cachestore
|
||||||
|
import pkg/codex/chunker
|
||||||
|
import pkg/codex/stores
|
||||||
|
import pkg/codex/blocktype as bt
|
||||||
|
import pkg/codex/contracts/requests
|
||||||
|
import pkg/codex/contracts
|
||||||
|
import pkg/codex/merkletree
|
||||||
|
import pkg/codex/stores/cachestore
|
||||||
|
import pkg/codex/indexingstrategy
|
||||||
|
|
||||||
|
import pkg/codex/slots/converters
|
||||||
|
import pkg/codex/slots/builder/builder
|
||||||
|
import pkg/codex/utils/poseidon2digest
|
||||||
|
import pkg/codex/utils/asynciter
|
||||||
|
|
||||||
|
import ../helpers
|
||||||
|
import ../merkletree/helpers
|
||||||
|
|
||||||
|
const
|
||||||
|
# The number of slot blocks and number of slots, combined with
|
||||||
|
# the bytes per block, make it so that there are exactly 256 cells
|
||||||
|
# in the dataset.
|
||||||
|
bytesPerBlock* = 64 * 1024
|
||||||
|
cellsPerBlock* = bytesPerBlock div DefaultCellSize.int
|
||||||
|
numberOfSlotBlocks* = 4
|
||||||
|
totalNumberOfSlots* = 2
|
||||||
|
datasetSlotIndex* = 1
|
||||||
|
cellsPerSlot* = (bytesPerBlock * numberOfSlotBlocks) div DefaultCellSize.int
|
||||||
|
totalNumCells = ((numberOfSlotBlocks * totalNumberOfSlots * bytesPerBlock) div DefaultCellSize.int)
|
||||||
|
|
||||||
|
type
|
||||||
|
ProvingTestEnvironment* = ref object
|
||||||
|
# Invariant:
|
||||||
|
challenge*: Poseidon2Hash
|
||||||
|
# Variant:
|
||||||
|
localStore*: CacheStore
|
||||||
|
manifest*: Manifest
|
||||||
|
manifestBlock*: bt.Block
|
||||||
|
slot*: Slot
|
||||||
|
datasetBlocks*: seq[bt.Block]
|
||||||
|
slotTree*: Poseidon2Tree
|
||||||
|
slotRootCid*: Cid
|
||||||
|
slotRoots*: seq[Poseidon2Hash]
|
||||||
|
datasetToSlotTree*: Poseidon2Tree
|
||||||
|
datasetRootHash*: Poseidon2Hash
|
||||||
|
|
||||||
|
proc createDatasetBlocks(self: ProvingTestEnvironment): Future[void] {.async.} =
|
||||||
|
var data: seq[byte] = @[]
|
||||||
|
|
||||||
|
# This generates a number of blocks that have different data, such that
|
||||||
|
# Each cell in each block is unique, but nothing is random.
|
||||||
|
for i in 0 ..< totalNumCells:
|
||||||
|
data = data & (i.byte).repeat(DefaultCellSize.uint64)
|
||||||
|
|
||||||
|
let chunker = MockChunker.new(
|
||||||
|
dataset = data,
|
||||||
|
chunkSize = bytesPerBlock)
|
||||||
|
|
||||||
|
while true:
|
||||||
|
let chunk = await chunker.getBytes()
|
||||||
|
if chunk.len <= 0:
|
||||||
|
break
|
||||||
|
let b = bt.Block.new(chunk).tryGet()
|
||||||
|
self.datasetBlocks.add(b)
|
||||||
|
discard await self.localStore.putBlock(b)
|
||||||
|
|
||||||
|
proc createSlotTree(self: ProvingTestEnvironment, dSlotIndex: uint64): Future[Poseidon2Tree] {.async.} =
|
||||||
|
let
|
||||||
|
slotSize = (bytesPerBlock * numberOfSlotBlocks).uint64
|
||||||
|
blocksInSlot = slotSize div bytesPerBlock.uint64
|
||||||
|
datasetBlockIndexingStrategy = SteppedIndexingStrategy.new(0, self.datasetBlocks.len - 1, totalNumberOfSlots)
|
||||||
|
datasetBlockIndices = toSeq(datasetBlockIndexingStrategy.getIndicies(dSlotIndex.int))
|
||||||
|
|
||||||
|
let
|
||||||
|
slotBlocks = datasetBlockIndices.mapIt(self.datasetBlocks[it])
|
||||||
|
numBlockCells = bytesPerBlock.int div DefaultCellSize.int
|
||||||
|
blockPadBytes = newSeq[byte](numBlockCells.nextPowerOfTwoPad * DefaultCellSize.int)
|
||||||
|
slotBlockRoots = slotBlocks.mapIt(Poseidon2Tree.digest(it.data & blockPadBytes, DefaultCellSize.int).tryGet())
|
||||||
|
tree = Poseidon2Tree.init(slotBlockRoots).tryGet()
|
||||||
|
treeCid = tree.root().tryGet().toSlotCid().tryGet()
|
||||||
|
|
||||||
|
for i in 0 ..< numberOfSlotBlocks:
|
||||||
|
let
|
||||||
|
blkCid = slotBlockRoots[i].toCellCid().tryGet()
|
||||||
|
proof = tree.getProof(i).tryGet().toEncodableProof().tryGet()
|
||||||
|
|
||||||
|
discard await self.localStore.putCidAndProof(treeCid, i, blkCid, proof)
|
||||||
|
|
||||||
|
return tree
|
||||||
|
|
||||||
|
proc createDatasetRootHashAndSlotTree(self: ProvingTestEnvironment): Future[void] {.async.} =
|
||||||
|
var slotTrees = newSeq[Poseidon2Tree]()
|
||||||
|
for i in 0 ..< totalNumberOfSlots:
|
||||||
|
slotTrees.add(await self.createSlotTree(i.uint64))
|
||||||
|
self.slotTree = slotTrees[datasetSlotIndex]
|
||||||
|
self.slotRootCid = slotTrees[datasetSlotIndex].root().tryGet().toSlotCid().tryGet()
|
||||||
|
self.slotRoots = slotTrees.mapIt(it.root().tryGet())
|
||||||
|
let rootsPadLeafs = newSeqWith(totalNumberOfSlots.nextPowerOfTwoPad, Poseidon2Zero)
|
||||||
|
self.datasetToSlotTree = Poseidon2Tree.init(self.slotRoots & rootsPadLeafs).tryGet()
|
||||||
|
self.datasetRootHash = self.datasetToSlotTree.root().tryGet()
|
||||||
|
|
||||||
|
proc createManifest(self: ProvingTestEnvironment): Future[void] {.async.} =
|
||||||
|
let
|
||||||
|
cids = self.datasetBlocks.mapIt(it.cid)
|
||||||
|
tree = CodexTree.init(cids).tryGet()
|
||||||
|
treeCid = tree.rootCid(CIDv1, BlockCodec).tryGet()
|
||||||
|
|
||||||
|
for i in 0 ..< self.datasetBlocks.len:
|
||||||
|
let
|
||||||
|
blk = self.datasetBlocks[i]
|
||||||
|
leafCid = blk.cid
|
||||||
|
proof = tree.getProof(i).tryGet()
|
||||||
|
discard await self.localStore.putBlock(blk)
|
||||||
|
discard await self.localStore.putCidAndProof(treeCid, i, leafCid, proof)
|
||||||
|
|
||||||
|
# Basic manifest:
|
||||||
|
self.manifest = Manifest.new(
|
||||||
|
treeCid = treeCid,
|
||||||
|
blockSize = bytesPerBlock.NBytes,
|
||||||
|
datasetSize = (bytesPerBlock * numberOfSlotBlocks * totalNumberOfSlots).NBytes)
|
||||||
|
|
||||||
|
# Protected manifest:
|
||||||
|
self.manifest = Manifest.new(
|
||||||
|
manifest = self.manifest,
|
||||||
|
treeCid = treeCid,
|
||||||
|
datasetSize = self.manifest.datasetSize,
|
||||||
|
ecK = totalNumberOfSlots,
|
||||||
|
ecM = 0
|
||||||
|
)
|
||||||
|
|
||||||
|
# Verifiable manifest:
|
||||||
|
self.manifest = Manifest.new(
|
||||||
|
manifest = self.manifest,
|
||||||
|
verifyRoot = self.datasetRootHash.toVerifyCid().tryGet(),
|
||||||
|
slotRoots = self.slotRoots.mapIt(it.toSlotCid().tryGet())
|
||||||
|
).tryGet()
|
||||||
|
|
||||||
|
self.manifestBlock = bt.Block.new(self.manifest.encode().tryGet(), codec = ManifestCodec).tryGet()
|
||||||
|
discard await self.localStore.putBlock(self.manifestBlock)
|
||||||
|
|
||||||
|
proc createSlot(self: ProvingTestEnvironment): void =
|
||||||
|
self.slot = Slot(
|
||||||
|
request: StorageRequest(
|
||||||
|
ask: StorageAsk(
|
||||||
|
slots: totalNumberOfSlots.uint64,
|
||||||
|
slotSize: u256(bytesPerBlock * numberOfSlotBlocks)
|
||||||
|
),
|
||||||
|
content: StorageContent(
|
||||||
|
cid: $self.manifestBlock.cid
|
||||||
|
),
|
||||||
|
),
|
||||||
|
slotIndex: u256(datasetSlotIndex)
|
||||||
|
)
|
||||||
|
|
||||||
|
proc createProvingTestEnvironment*(): Future[ProvingTestEnvironment] {.async.} =
|
||||||
|
var testEnv = ProvingTestEnvironment(
|
||||||
|
challenge: toF(12345)
|
||||||
|
)
|
||||||
|
|
||||||
|
testEnv.localStore = CacheStore.new()
|
||||||
|
await testEnv.createDatasetBlocks()
|
||||||
|
await testEnv.createDatasetRootHashAndSlotTree()
|
||||||
|
await testEnv.createManifest()
|
||||||
|
testEnv.createSlot()
|
||||||
|
|
||||||
|
return testEnv
|
|
@ -0,0 +1,47 @@
|
||||||
|
import pkg/chronos
|
||||||
|
import pkg/asynctest
|
||||||
|
import pkg/poseidon2
|
||||||
|
import pkg/poseidon2/io
|
||||||
|
import pkg/constantine/math/io/io_fields
|
||||||
|
import pkg/questionable/results
|
||||||
|
import pkg/codex/merkletree
|
||||||
|
|
||||||
|
import pkg/codex/slots/converters
|
||||||
|
import ../examples
|
||||||
|
import ../merkletree/helpers
|
||||||
|
|
||||||
|
let
|
||||||
|
hash: Poseidon2Hash = toF(12345)
|
||||||
|
|
||||||
|
suite "Converters":
|
||||||
|
test "CellBlock cid":
|
||||||
|
let
|
||||||
|
cid = toCellCid(hash).tryGet()
|
||||||
|
value = fromCellCid(cid).tryGet()
|
||||||
|
|
||||||
|
check:
|
||||||
|
hash.toDecimal() == value.toDecimal()
|
||||||
|
|
||||||
|
test "Slot cid":
|
||||||
|
let
|
||||||
|
cid = toSlotCid(hash).tryGet()
|
||||||
|
value = fromSlotCid(cid).tryGet()
|
||||||
|
|
||||||
|
check:
|
||||||
|
hash.toDecimal() == value.toDecimal()
|
||||||
|
|
||||||
|
test "Verify cid":
|
||||||
|
let
|
||||||
|
cid = toVerifyCid(hash).tryGet()
|
||||||
|
value = fromVerifyCid(cid).tryGet()
|
||||||
|
|
||||||
|
check:
|
||||||
|
hash.toDecimal() == value.toDecimal()
|
||||||
|
|
||||||
|
test "Proof":
|
||||||
|
let
|
||||||
|
codexProof = toEncodableProof(Poseidon2Proof.example).tryGet()
|
||||||
|
poseidonProof = toVerifiableProof(codexProof).tryGet()
|
||||||
|
|
||||||
|
check:
|
||||||
|
Poseidon2Proof.example == poseidonProof
|
|
@ -0,0 +1,118 @@
|
||||||
|
import std/sequtils
|
||||||
|
import std/sugar
|
||||||
|
import std/random
|
||||||
|
import std/strutils
|
||||||
|
|
||||||
|
import pkg/questionable/results
|
||||||
|
import pkg/constantine/math/arithmetic
|
||||||
|
import pkg/constantine/math/io/io_fields
|
||||||
|
import pkg/poseidon2/io
|
||||||
|
import pkg/poseidon2
|
||||||
|
import pkg/chronos
|
||||||
|
import pkg/asynctest
|
||||||
|
import pkg/codex/stores/cachestore
|
||||||
|
import pkg/codex/chunker
|
||||||
|
import pkg/codex/stores
|
||||||
|
import pkg/codex/blocktype as bt
|
||||||
|
import pkg/codex/contracts/requests
|
||||||
|
import pkg/codex/contracts
|
||||||
|
import pkg/codex/merkletree
|
||||||
|
import pkg/codex/stores/cachestore
|
||||||
|
|
||||||
|
import pkg/codex/slots/sampler
|
||||||
|
import pkg/codex/slots/builder/builder
|
||||||
|
|
||||||
|
import ../helpers
|
||||||
|
import ../examples
|
||||||
|
import ../merkletree/helpers
|
||||||
|
import testsampler_expected
|
||||||
|
import ./provingtestenv
|
||||||
|
|
||||||
|
asyncchecksuite "Test DataSampler":
|
||||||
|
var
|
||||||
|
env: ProvingTestEnvironment
|
||||||
|
dataSampler: DataSampler
|
||||||
|
blk: bt.Block
|
||||||
|
cell0Bytes: seq[byte]
|
||||||
|
cell1Bytes: seq[byte]
|
||||||
|
cell2Bytes: seq[byte]
|
||||||
|
|
||||||
|
proc createDataSampler(): Future[void] {.async.} =
|
||||||
|
dataSampler = DataSampler.new(
|
||||||
|
datasetSlotIndex,
|
||||||
|
env.localStore,
|
||||||
|
SlotsBuilder.new(env.localStore, env.manifest).tryGet()).tryGet()
|
||||||
|
|
||||||
|
setup:
|
||||||
|
randomize()
|
||||||
|
env = await createProvingTestEnvironment()
|
||||||
|
let bytes = newSeqWith(bytesPerBlock, rand(uint8))
|
||||||
|
blk = bt.Block.new(bytes).tryGet()
|
||||||
|
cell0Bytes = bytes[0..<DefaultCellSize.uint64]
|
||||||
|
cell1Bytes = bytes[DefaultCellSize.uint64..<(DefaultCellSize.uint64*2)]
|
||||||
|
cell2Bytes = bytes[(DefaultCellSize.uint64*2)..<(DefaultCellSize.uint64*3)]
|
||||||
|
|
||||||
|
await createDataSampler()
|
||||||
|
|
||||||
|
teardown:
|
||||||
|
reset(env)
|
||||||
|
reset(dataSampler)
|
||||||
|
|
||||||
|
test "Can get cell from block":
|
||||||
|
let
|
||||||
|
sample0 = dataSampler.getCell(blk.data, 0)
|
||||||
|
sample1 = dataSampler.getCell(blk.data, 1)
|
||||||
|
sample2 = dataSampler.getCell(blk.data, 2)
|
||||||
|
|
||||||
|
check:
|
||||||
|
sample0 == cell0Bytes
|
||||||
|
sample1 == cell1Bytes
|
||||||
|
sample2 == cell2Bytes
|
||||||
|
|
||||||
|
test "Can gather proof input":
|
||||||
|
let
|
||||||
|
nSamples = 3
|
||||||
|
challengeBytes = env.challenge.toBytes()
|
||||||
|
input = (await dataSampler.getProofInput(challengeBytes, nSamples)).tryget()
|
||||||
|
|
||||||
|
proc equal(a: Poseidon2Hash, b: Poseidon2Hash): bool =
|
||||||
|
a.toDecimal() == b.toDecimal()
|
||||||
|
|
||||||
|
proc toStr(proof: Poseidon2Proof): string =
|
||||||
|
let a = proof.path.mapIt(toHex(it))
|
||||||
|
join(a)
|
||||||
|
|
||||||
|
let
|
||||||
|
expectedBlockSlotProofs = getExpectedBlockSlotProofs()
|
||||||
|
expectedCellBlockProofs = getExpectedCellBlockProofs()
|
||||||
|
expectedCellData = getExpectedCellData()
|
||||||
|
expectedProof = env.datasetToSlotTree.getProof(datasetSlotIndex).tryGet()
|
||||||
|
|
||||||
|
check:
|
||||||
|
equal(input.verifyRoot, env.datasetRootHash)
|
||||||
|
equal(input.entropy, env.challenge)
|
||||||
|
input.numCells == ((bytesPerBlock * numberOfSlotBlocks) div DefaultCellSize.int).Natural
|
||||||
|
input.numSlots == totalNumberOfSlots.Natural
|
||||||
|
input.slotIndex == env.slot.slotIndex.truncate(Natural)
|
||||||
|
input.verifyProof == expectedProof
|
||||||
|
|
||||||
|
# block-slot proofs
|
||||||
|
input.samples[0].slotBlockIdx == 2
|
||||||
|
input.samples[1].slotBlockIdx == 2
|
||||||
|
input.samples[2].slotBlockIdx == 0
|
||||||
|
toStr(input.samples[0].slotProof) == expectedBlockSlotProofs[0]
|
||||||
|
toStr(input.samples[1].slotProof) == expectedBlockSlotProofs[1]
|
||||||
|
toStr(input.samples[2].slotProof) == expectedBlockSlotProofs[2]
|
||||||
|
|
||||||
|
# cell-block proofs
|
||||||
|
input.samples[0].blockCellIdx == 26
|
||||||
|
input.samples[1].blockCellIdx == 29
|
||||||
|
input.samples[2].blockCellIdx == 29
|
||||||
|
toStr(input.samples[0].cellProof) == expectedCellBlockProofs[0]
|
||||||
|
toStr(input.samples[1].cellProof) == expectedCellBlockProofs[1]
|
||||||
|
toStr(input.samples[2].cellProof) == expectedCellBlockProofs[2]
|
||||||
|
|
||||||
|
# # cell data
|
||||||
|
toHex(input.samples[0].data) == expectedCellData[0]
|
||||||
|
toHex(input.samples[1].data) == expectedCellData[1]
|
||||||
|
toHex(input.samples[2].data) == expectedCellData[2]
|
|
@ -0,0 +1,25 @@
|
||||||
|
# Snapshot of expected values for testsampler.
|
||||||
|
|
||||||
|
import std/strutils
|
||||||
|
import pkg/codex/codextypes
|
||||||
|
|
||||||
|
proc getExpectedCellBlockProofs*(): seq[string] =
|
||||||
|
@[
|
||||||
|
"0x189890bedf2a40f2757554c5f089811e07601543a576e2d40d68a1bd295adbee0x059f227fe687a7abd9c3d9878c0b812aa7829c85d30c23154b8824a907909b060x2c685fc951f1684fd3979f7e3a558fa6aeed3f960ef0a9e2ba51cc62cff7de0e0x10ebae3fb12d502044216e40319726ed36308b9ae4ab3fb0a36c77e5614c6fbd0x1decd3fb7ff458261149731115657cecd7eb2fe4a6cf84f3c6761aa8b0dd6b9a",
|
||||||
|
"0x2567cd93b3fe058b31908656c05d3a09fd33cc0d7df3c7c005d991a8cda60ba80x16e4788105082295706c49c604216518f16ca9dd106012c7c98e6ee138893f6e0x1c258af996aecf9bba249130182ccfe44a9090bc58fe59063a06db67efb7b5240x10ebae3fb12d502044216e40319726ed36308b9ae4ab3fb0a36c77e5614c6fbd0x1decd3fb7ff458261149731115657cecd7eb2fe4a6cf84f3c6761aa8b0dd6b9a",
|
||||||
|
"0x0568735989a51526104eddbcf386b8aaef186a2d31afce0c2671c8ce8dd8cd1a0x20d06082668338924981a9e0e4f18e7ec6e2b7912e7fb74c1b6dc921b824def60x2fd45662152ae87192971a0e9b7d50de48d7bc8ab22e5711680173a302120bf00x0f528a58c839889e4bb9195e2bcbc2addb7022e47c8fb11bbdeba0a0e9c6f4cb0x0edf43ec0f277500371537a4d566f3f352d0c49bfa9d4659e07d776ffe119437"
|
||||||
|
]
|
||||||
|
|
||||||
|
proc getExpectedBlockSlotProofs*(): seq[string] =
|
||||||
|
@[
|
||||||
|
"0x0684458ea77eca59be05e368bb26b7ca318b8e836100c415e60136876c01ba170x2a66917fa49371e835376fcece0d854c77008ac1195740963b1ac4491ee1aaf1",
|
||||||
|
"0x0684458ea77eca59be05e368bb26b7ca318b8e836100c415e60136876c01ba170x2a66917fa49371e835376fcece0d854c77008ac1195740963b1ac4491ee1aaf1",
|
||||||
|
"0x03883ad2637a4c68f29bc0910400259291d9c3d730de7e3925adbf26c80b7f440x2d6a888f50b14b0c686f64c4bd0d8389cd555cdf0e3d6f387682c4722ac2a674"
|
||||||
|
]
|
||||||
|
|
||||||
|
proc getExpectedCellData*(): seq[string] =
|
||||||
|
@[
|
||||||
|
"BA".repeat(DefaultCellSize.int),
|
||||||
|
"BD".repeat(DefaultCellSize.int),
|
||||||
|
"3D".repeat(DefaultCellSize.int)
|
||||||
|
]
|
|
@ -31,23 +31,23 @@ privateAccess(Manifest) # enable access to private fields
|
||||||
|
|
||||||
suite "Slot builder":
|
suite "Slot builder":
|
||||||
let
|
let
|
||||||
blockSize = 1024
|
blockSize = NBytes 1024
|
||||||
cellSize = 64
|
cellSize = NBytes 64
|
||||||
ecK = 3
|
ecK = 3
|
||||||
ecM = 2
|
ecM = 2
|
||||||
|
|
||||||
numSlots = ecK + ecM
|
numSlots = ecK + ecM
|
||||||
numDatasetBlocks = 100
|
numDatasetBlocks = 100
|
||||||
numBlockCells = blockSize div cellSize
|
numBlockCells = (blockSize div cellSize).int
|
||||||
|
|
||||||
numTotalBlocks = calcEcBlocksCount(numDatasetBlocks, ecK, ecM) # total number of blocks in the dataset after
|
numTotalBlocks = calcEcBlocksCount(numDatasetBlocks, ecK, ecM) # total number of blocks in the dataset after
|
||||||
# EC (should will match number of slots)
|
# EC (should will match number of slots)
|
||||||
originalDatasetSize = numDatasetBlocks * blockSize # size of the dataset before EC
|
originalDatasetSize = numDatasetBlocks * blockSize.int # size of the dataset before EC
|
||||||
totalDatasetSize = numTotalBlocks * blockSize # size of the dataset after EC
|
totalDatasetSize = numTotalBlocks * blockSize.int # size of the dataset after EC
|
||||||
numTotalSlotBlocks = nextPowerOfTwo(numTotalBlocks div numSlots)
|
numTotalSlotBlocks = nextPowerOfTwo(numTotalBlocks div numSlots)
|
||||||
|
|
||||||
blockPadBytes =
|
blockPadBytes =
|
||||||
newSeq[byte](numBlockCells.nextPowerOfTwoPad * cellSize) # power of two padding for blocks
|
newSeq[byte](numBlockCells.nextPowerOfTwoPad * cellSize.int) # power of two padding for blocks
|
||||||
|
|
||||||
slotsPadLeafs =
|
slotsPadLeafs =
|
||||||
newSeqWith((numTotalBlocks div numSlots).nextPowerOfTwoPad, Poseidon2Zero) # power of two padding for block roots
|
newSeqWith((numTotalBlocks div numSlots).nextPowerOfTwoPad, Poseidon2Zero) # power of two padding for block roots
|
||||||
|
@ -215,7 +215,7 @@ suite "Slot builder":
|
||||||
|
|
||||||
expectedHashes: seq[Poseidon2Hash] = collect(newSeq):
|
expectedHashes: seq[Poseidon2Hash] = collect(newSeq):
|
||||||
for blk in expectedBlock:
|
for blk in expectedBlock:
|
||||||
SpongeMerkle.digest(blk.data & blockPadBytes, cellSize)
|
SpongeMerkle.digest(blk.data & blockPadBytes, cellSize.int)
|
||||||
|
|
||||||
cellHashes = (await slotBuilder.getCellHashes(i)).tryGet()
|
cellHashes = (await slotBuilder.getCellHashes(i)).tryGet()
|
||||||
|
|
||||||
|
@ -238,7 +238,7 @@ suite "Slot builder":
|
||||||
|
|
||||||
expectedHashes: seq[Poseidon2Hash] = collect(newSeq):
|
expectedHashes: seq[Poseidon2Hash] = collect(newSeq):
|
||||||
for blk in expectedBlock:
|
for blk in expectedBlock:
|
||||||
SpongeMerkle.digest(blk.data & blockPadBytes, cellSize)
|
SpongeMerkle.digest(blk.data & blockPadBytes, cellSize.int)
|
||||||
expectedRoot = Merkle.digest(expectedHashes & slotsPadLeafs)
|
expectedRoot = Merkle.digest(expectedHashes & slotsPadLeafs)
|
||||||
|
|
||||||
slotTree = (await slotBuilder.buildSlotTree(i)).tryGet()
|
slotTree = (await slotBuilder.buildSlotTree(i)).tryGet()
|
||||||
|
@ -289,12 +289,12 @@ suite "Slot builder":
|
||||||
|
|
||||||
slotHashes: seq[Poseidon2Hash] = collect(newSeq):
|
slotHashes: seq[Poseidon2Hash] = collect(newSeq):
|
||||||
for blk in expectedBlocks:
|
for blk in expectedBlocks:
|
||||||
SpongeMerkle.digest(blk.data & blockPadBytes, cellSize)
|
SpongeMerkle.digest(blk.data & blockPadBytes, cellSize.int)
|
||||||
|
|
||||||
Merkle.digest(slotHashes & slotsPadLeafs)
|
Merkle.digest(slotHashes & slotsPadLeafs)
|
||||||
|
|
||||||
expectedRoot = Merkle.digest(slotsHashes & rootsPadLeafs)
|
expectedRoot = Merkle.digest(slotsHashes & rootsPadLeafs)
|
||||||
rootHash = slotBuilder.buildRootsTree(slotBuilder.slotRoots).tryGet().root.tryGet()
|
rootHash = slotBuilder.buildVerifyTree(slotBuilder.slotRoots).tryGet().root.tryGet()
|
||||||
|
|
||||||
check:
|
check:
|
||||||
expectedRoot == rootHash
|
expectedRoot == rootHash
|
||||||
|
@ -316,7 +316,7 @@ suite "Slot builder":
|
||||||
|
|
||||||
slotHashes: seq[Poseidon2Hash] = collect(newSeq):
|
slotHashes: seq[Poseidon2Hash] = collect(newSeq):
|
||||||
for blk in expectedBlocks:
|
for blk in expectedBlocks:
|
||||||
SpongeMerkle.digest(blk.data & blockPadBytes, cellSize)
|
SpongeMerkle.digest(blk.data & blockPadBytes, cellSize.int)
|
||||||
|
|
||||||
Merkle.digest(slotHashes & slotsPadLeafs)
|
Merkle.digest(slotHashes & slotsPadLeafs)
|
||||||
|
|
||||||
|
@ -361,7 +361,7 @@ suite "Slot builder":
|
||||||
verifyManifest,
|
verifyManifest,
|
||||||
cellSize = cellSize).isErr
|
cellSize = cellSize).isErr
|
||||||
|
|
||||||
test "Should not build from verifiable manifest with slots root":
|
test "Should not build from verifiable manifest with invalid verify root":
|
||||||
let
|
let
|
||||||
slotBuilder = SlotsBuilder.new(
|
slotBuilder = SlotsBuilder.new(
|
||||||
localStore,
|
localStore,
|
|
@ -0,0 +1,104 @@
|
||||||
|
import std/sequtils
|
||||||
|
import std/sugar
|
||||||
|
import std/random
|
||||||
|
import std/strutils
|
||||||
|
|
||||||
|
import pkg/questionable/results
|
||||||
|
import pkg/constantine/math/arithmetic
|
||||||
|
import pkg/constantine/math/io/io_fields
|
||||||
|
import pkg/poseidon2/io
|
||||||
|
import pkg/poseidon2
|
||||||
|
import pkg/chronos
|
||||||
|
import pkg/asynctest
|
||||||
|
import pkg/codex/stores/cachestore
|
||||||
|
import pkg/codex/chunker
|
||||||
|
import pkg/codex/stores
|
||||||
|
import pkg/codex/blocktype as bt
|
||||||
|
import pkg/codex/contracts/requests
|
||||||
|
import pkg/codex/contracts
|
||||||
|
import pkg/codex/merkletree
|
||||||
|
import pkg/codex/stores/cachestore
|
||||||
|
|
||||||
|
import pkg/codex/slots/sampler/utils
|
||||||
|
|
||||||
|
import ../helpers
|
||||||
|
import ../examples
|
||||||
|
import ../merkletree/helpers
|
||||||
|
import ./provingtestenv
|
||||||
|
|
||||||
|
asyncchecksuite "Test proof sampler utils":
|
||||||
|
let knownIndices: seq[Natural] = @[90, 93, 29]
|
||||||
|
|
||||||
|
var
|
||||||
|
env: ProvingTestEnvironment
|
||||||
|
slotRoot: Poseidon2Hash
|
||||||
|
numCells: Natural
|
||||||
|
|
||||||
|
setup:
|
||||||
|
env = await createProvingTestEnvironment()
|
||||||
|
slotRoot = env.slotRoots[datasetSlotIndex]
|
||||||
|
numCells = cellsPerSlot
|
||||||
|
|
||||||
|
teardown:
|
||||||
|
reset(env)
|
||||||
|
|
||||||
|
test "Extract low bits":
|
||||||
|
proc extract(value: uint64, nBits: int): uint64 =
|
||||||
|
let big = toF(value).toBig()
|
||||||
|
return extractLowBits(big, nBits)
|
||||||
|
|
||||||
|
check:
|
||||||
|
extract(0x88, 4) == 0x8.uint64
|
||||||
|
extract(0x88, 7) == 0x8.uint64
|
||||||
|
extract(0x9A, 5) == 0x1A.uint64
|
||||||
|
extract(0x9A, 7) == 0x1A.uint64
|
||||||
|
extract(0x1248, 10) == 0x248.uint64
|
||||||
|
extract(0x1248, 12) == 0x248.uint64
|
||||||
|
extract(0x1248306A560C9AC0.uint64, 10) == 0x2C0.uint64
|
||||||
|
extract(0x1248306A560C9AC0.uint64, 12) == 0xAC0.uint64
|
||||||
|
extract(0x1248306A560C9AC0.uint64, 50) == 0x306A560C9AC0.uint64
|
||||||
|
extract(0x1248306A560C9AC0.uint64, 52) == 0x8306A560C9AC0.uint64
|
||||||
|
|
||||||
|
test "Can find single slot-cell index":
|
||||||
|
proc slotCellIndex(i: Natural): Natural =
|
||||||
|
return cellIndex(env.challenge, slotRoot, numCells, i)
|
||||||
|
|
||||||
|
proc getExpectedIndex(i: int): Natural =
|
||||||
|
let
|
||||||
|
numberOfCellsInSlot = (bytesPerBlock * numberOfSlotBlocks) div DefaultCellSize.uint64.int
|
||||||
|
hash = Sponge.digest(@[slotRoot, env.challenge, toF(i)], rate = 2)
|
||||||
|
|
||||||
|
return int(extractLowBits(hash.toBig(), ceilingLog2(numberOfCellsInSlot)))
|
||||||
|
|
||||||
|
check:
|
||||||
|
slotCellIndex(1) == getExpectedIndex(1)
|
||||||
|
slotCellIndex(1) == knownIndices[0]
|
||||||
|
slotCellIndex(2) == getExpectedIndex(2)
|
||||||
|
slotCellIndex(2) == knownIndices[1]
|
||||||
|
slotCellIndex(3) == getExpectedIndex(3)
|
||||||
|
slotCellIndex(3) == knownIndices[2]
|
||||||
|
|
||||||
|
test "Can find sequence of slot-cell indices":
|
||||||
|
proc slotCellIndices(n: int): seq[Natural] =
|
||||||
|
cellIndices(env.challenge, slotRoot, numCells, n)
|
||||||
|
|
||||||
|
proc getExpectedIndices(n: int): seq[Natural] =
|
||||||
|
return collect(newSeq, (for i in 1..n: cellIndex(env.challenge, slotRoot, numCells, i)))
|
||||||
|
|
||||||
|
check:
|
||||||
|
slotCellIndices(3) == getExpectedIndices(3)
|
||||||
|
slotCellIndices(3) == knownIndices
|
||||||
|
|
||||||
|
for (input, expected) in [(10, 0), (31, 0), (32, 1), (63, 1), (64, 2)]:
|
||||||
|
test "Can get slotBlockIndex from slotCellIndex (" & $input & " -> " & $expected & ")":
|
||||||
|
let slotBlockIndex = toBlockIdx(input, numCells = 32)
|
||||||
|
|
||||||
|
check:
|
||||||
|
slotBlockIndex == expected
|
||||||
|
|
||||||
|
for (input, expected) in [(10, 10), (31, 31), (32, 0), (63, 31), (64, 0)]:
|
||||||
|
test "Can get blockCellIndex from slotCellIndex (" & $input & " -> " & $expected & ")":
|
||||||
|
let blockCellIndex = toBlockCellIdx(input, numCells = 32)
|
||||||
|
|
||||||
|
check:
|
||||||
|
blockCellIndex == expected
|
|
@ -39,11 +39,11 @@ checksuite "Manifest":
|
||||||
slotLeavesCids = leaves.toSlotCids().tryGet
|
slotLeavesCids = leaves.toSlotCids().tryGet
|
||||||
|
|
||||||
tree = Poseidon2Tree.init(leaves).tryGet
|
tree = Poseidon2Tree.init(leaves).tryGet
|
||||||
slotsRootsCid = tree.root.tryGet.toSlotsRootsCid().tryGet
|
verifyCid = tree.root.tryGet.toVerifyCid().tryGet
|
||||||
|
|
||||||
verifiableManifest = Manifest.new(
|
verifiableManifest = Manifest.new(
|
||||||
manifest = protectedManifest,
|
manifest = protectedManifest,
|
||||||
verifyRoot = slotsRootsCid,
|
verifyRoot = verifyCid,
|
||||||
slotRoots = slotLeavesCids
|
slotRoots = slotLeavesCids
|
||||||
).tryGet()
|
).tryGet()
|
||||||
|
|
||||||
|
|
|
@ -1,3 +0,0 @@
|
||||||
import ./slotbuilder/testslotbuilder
|
|
||||||
|
|
||||||
{.warning[UnusedImport]: off.}
|
|
|
@ -0,0 +1,6 @@
|
||||||
|
import ./slots/testslotbuilder
|
||||||
|
import ./slots/testutils
|
||||||
|
import ./slots/testsampler
|
||||||
|
import ./slots/testconverters
|
||||||
|
|
||||||
|
{.warning[UnusedImport]: off.}
|
|
@ -14,7 +14,7 @@ import ./codex/testsystemclock
|
||||||
import ./codex/testvalidation
|
import ./codex/testvalidation
|
||||||
import ./codex/testasyncstreamwrapper
|
import ./codex/testasyncstreamwrapper
|
||||||
import ./codex/testmerkletree
|
import ./codex/testmerkletree
|
||||||
import ./codex/testslotbuilder
|
import ./codex/testslots
|
||||||
import ./codex/testindexingstrategy
|
import ./codex/testindexingstrategy
|
||||||
|
|
||||||
{.warning[UnusedImport]: off.}
|
{.warning[UnusedImport]: off.}
|
||||||
|
|
Loading…
Reference in New Issue