Integrate erasure (#73)
* wip: adding request for storage endpoint * wire in erasure coding * fix tests for erasure coding * put type definitions into separate file * integrate erasure coding * change run/shutdown to start/stop * temporary sleep, otherwise the fsstore blocks
This commit is contained in:
parent
22c6705312
commit
ffa9b624f1
|
@ -57,7 +57,7 @@ when isMainModule:
|
|||
setupForeignThreadGc()
|
||||
except Exception as exc: raiseAssert exc.msg # shouldn't happen
|
||||
notice "Shutting down after having received SIGINT"
|
||||
waitFor server.shutdown()
|
||||
waitFor server.stop()
|
||||
|
||||
try:
|
||||
setControlCHook(controlCHandler)
|
||||
|
@ -68,10 +68,10 @@ when isMainModule:
|
|||
when defined(posix):
|
||||
proc SIGTERMHandler(signal: cint) {.noconv.} =
|
||||
notice "Shutting down after having received SIGTERM"
|
||||
waitFor server.shutdown()
|
||||
waitFor server.stop()
|
||||
|
||||
c_signal(SIGTERM, SIGTERMHandler)
|
||||
|
||||
waitFor server.run()
|
||||
waitFor server.start()
|
||||
of StartUpCommand.initNode:
|
||||
discard
|
||||
|
|
|
@ -26,6 +26,7 @@ import ./rest/api
|
|||
import ./stores
|
||||
import ./blockexchange
|
||||
import ./utils/fileutils
|
||||
import ./erasure
|
||||
|
||||
type
|
||||
DaggerServer* = ref object
|
||||
|
@ -34,14 +35,14 @@ type
|
|||
restServer: RestServerRef
|
||||
daggerNode: DaggerNodeRef
|
||||
|
||||
proc run*(s: DaggerServer) {.async.} =
|
||||
proc start*(s: DaggerServer) {.async.} =
|
||||
s.restServer.start()
|
||||
await s.daggerNode.start()
|
||||
|
||||
s.runHandle = newFuture[void]()
|
||||
await s.runHandle
|
||||
|
||||
proc shutdown*(s: DaggerServer) {.async.} =
|
||||
proc stop*(s: DaggerServer) {.async.} =
|
||||
await allFuturesThrowing(
|
||||
s.restServer.stop(), s.daggerNode.stop())
|
||||
|
||||
|
@ -73,7 +74,8 @@ proc new*(T: type DaggerServer, config: DaggerConf): T =
|
|||
localStore = FSStore.new(config.dataDir / "repo", cache = cache)
|
||||
engine = BlockExcEngine.new(localStore, wallet, network)
|
||||
store = NetworkStore.new(engine, localStore)
|
||||
daggerNode = DaggerNodeRef.new(switch, store, engine)
|
||||
erasure = Erasure.new(store, leoEncoderProvider, leoDecoderProvider)
|
||||
daggerNode = DaggerNodeRef.new(switch, store, engine, erasure)
|
||||
restServer = RestServerRef.new(
|
||||
daggerNode.initRestApi(),
|
||||
initTAddress("127.0.0.1" , config.apiPort),
|
||||
|
|
|
@ -106,6 +106,11 @@ proc encode*(
|
|||
dataBlocks = await allFinished(
|
||||
blockIdx.mapIt( self.store.getBlock(encoded[it]) ))
|
||||
|
||||
# TODO: this is a tight blocking loop so we sleep here to allow
|
||||
# other events to be processed, this should be addressed
|
||||
# by threading
|
||||
await sleepAsync(10.millis)
|
||||
|
||||
for j in 0..<blocks:
|
||||
let idx = blockIdx[j]
|
||||
if idx < manifest.len:
|
||||
|
@ -177,6 +182,11 @@ proc decode*(
|
|||
self.store.getBlock(encoded[it]) # Get the data blocks (first K)
|
||||
)
|
||||
|
||||
# TODO: this is a tight blocking loop so we sleep here to allow
|
||||
# other events to be processed, this should be addressed
|
||||
# by threading
|
||||
await sleepAsync(10.millis)
|
||||
|
||||
var
|
||||
data = newSeq[seq[byte]](encoded.K) # number of blocks to encode
|
||||
parityData = newSeq[seq[byte]](encoded.M)
|
||||
|
|
|
@ -1,4 +1,5 @@
|
|||
import ./manifest/coders
|
||||
import ./manifest/manifest
|
||||
import ./manifest/types
|
||||
|
||||
export manifest, coders
|
||||
export types, manifest, coders
|
||||
|
|
|
@ -21,19 +21,7 @@ import pkg/chronos
|
|||
|
||||
import ./manifest
|
||||
import ../errors
|
||||
|
||||
const
|
||||
DagPBCodec* = multiCodec("dag-pb")
|
||||
|
||||
type
|
||||
ManifestCoderType*[codec: static MultiCodec] = object
|
||||
DagPBCoder* = ManifestCoderType[multiCodec("dag-pb")]
|
||||
|
||||
const
|
||||
# TODO: move somewhere better?
|
||||
ManifestContainers* = {
|
||||
$DagPBCodec: DagPBCoder()
|
||||
}.toTable
|
||||
import ./types
|
||||
|
||||
func encode*(_: DagPBCoder, manifest: Manifest): ?!seq[byte] =
|
||||
## Encode the manifest into a ``ManifestCodec``
|
||||
|
|
|
@ -19,23 +19,8 @@ import pkg/chronicles
|
|||
|
||||
import ../errors
|
||||
import ../blocktype
|
||||
|
||||
type
|
||||
Manifest* = ref object of RootObj
|
||||
rootHash*: ?Cid # root (tree) hash of the contained data set
|
||||
blockSize*: int # size of each contained block (might not be needed if blocks are len-prefixed)
|
||||
blocks*: seq[Cid] # block Cid
|
||||
version*: CidVersion # Cid version
|
||||
hcodec*: MultiCodec # Multihash codec
|
||||
codec*: MultiCodec # Data set codec
|
||||
case protected*: bool # Protected datasets have erasure coded info
|
||||
of true:
|
||||
K*: int # Number of blocks to encode
|
||||
M*: int # Number of resulting parity blocks
|
||||
originalCid*: Cid # The original Cid of the dataset being erasure coded
|
||||
originalLen*: int # The length of the original manifest
|
||||
else:
|
||||
discard
|
||||
import ./types
|
||||
import ./coders
|
||||
|
||||
func len*(self: Manifest): int =
|
||||
self.blocks.len
|
||||
|
@ -189,5 +174,6 @@ proc new*(
|
|||
|
||||
proc new*(
|
||||
T: type Manifest,
|
||||
data: openArray[byte]): ?!T =
|
||||
Manifest.decode(data)
|
||||
data: openArray[byte],
|
||||
decoder = ManifestContainers[$DagPBCodec]): ?!T =
|
||||
Manifest.decode(data, decoder)
|
||||
|
|
|
@ -0,0 +1,42 @@
|
|||
## Nim-Dagger
|
||||
## Copyright (c) 2022 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/tables
|
||||
import pkg/libp2p
|
||||
import pkg/questionable
|
||||
|
||||
const
|
||||
DagPBCodec* = multiCodec("dag-pb")
|
||||
|
||||
type
|
||||
ManifestCoderType*[codec: static MultiCodec] = object
|
||||
DagPBCoder* = ManifestCoderType[multiCodec("dag-pb")]
|
||||
|
||||
const
|
||||
# TODO: move somewhere better?
|
||||
ManifestContainers* = {
|
||||
$DagPBCodec: DagPBCoder()
|
||||
}.toTable
|
||||
|
||||
type
|
||||
Manifest* = ref object of RootObj
|
||||
rootHash*: ?Cid # root (tree) hash of the contained data set
|
||||
blockSize*: int # size of each contained block (might not be needed if blocks are len-prefixed)
|
||||
blocks*: seq[Cid] # block Cid
|
||||
version*: CidVersion # Cid version
|
||||
hcodec*: MultiCodec # Multihash codec
|
||||
codec*: MultiCodec # Data set codec
|
||||
case protected*: bool # Protected datasets have erasure coded info
|
||||
of true:
|
||||
K*: int # Number of blocks to encode
|
||||
M*: int # Number of resulting parity blocks
|
||||
originalCid*: Cid # The original Cid of the dataset being erasure coded
|
||||
originalLen*: int # The length of the original manifest
|
||||
else:
|
||||
discard
|
|
@ -26,6 +26,7 @@ import ./manifest
|
|||
import ./stores/blockstore
|
||||
import ./blockexchange
|
||||
import ./streams
|
||||
import ./erasure
|
||||
|
||||
logScope:
|
||||
topics = "dagger node"
|
||||
|
@ -38,21 +39,22 @@ type
|
|||
networkId*: PeerID
|
||||
blockStore*: BlockStore
|
||||
engine*: BlockExcEngine
|
||||
erasure*: Erasure
|
||||
|
||||
proc start*(node: DaggerNodeRef) {.async.} =
|
||||
await node.switch.start()
|
||||
await node.engine.start()
|
||||
await node.erasure.start()
|
||||
|
||||
node.networkId = node.switch.peerInfo.peerId
|
||||
notice "Started dagger node", id = $node.networkId, addrs = node.switch.peerInfo.addrs
|
||||
|
||||
proc stop*(node: DaggerNodeRef) {.async.} =
|
||||
trace "Stopping node"
|
||||
|
||||
if not node.engine.isNil:
|
||||
await node.engine.stop()
|
||||
|
||||
if not node.switch.isNil:
|
||||
await node.switch.stop()
|
||||
await node.engine.stop()
|
||||
await node.switch.stop()
|
||||
await node.erasure.stop()
|
||||
|
||||
proc findPeer*(
|
||||
node: DaggerNodeRef,
|
||||
|
@ -85,6 +87,16 @@ proc retrieve*(
|
|||
without manifest =? Manifest.decode(blk.data, ManifestContainers[$mc]):
|
||||
return failure("Unable to construct manifest!")
|
||||
|
||||
if manifest.protected:
|
||||
proc erasureJob(): Future[void] {.async.} =
|
||||
try:
|
||||
without res =? (await node.erasure.decode(manifest)), error: # spawn an erasure decoding job
|
||||
trace "Unable to erasure decode manigest", cid, exc = error.msg
|
||||
except CatchableError as exc:
|
||||
trace "Exception decoding manifest", cid
|
||||
|
||||
asyncSpawn erasureJob()
|
||||
|
||||
return LPStream(StoreStream.new(node.blockStore, manifest)).success
|
||||
|
||||
let
|
||||
|
@ -158,12 +170,69 @@ proc store*(
|
|||
|
||||
return manifest.cid.success
|
||||
|
||||
proc requestStorage*(
|
||||
self: DaggerNodeRef,
|
||||
cid: Cid,
|
||||
ppb: uint,
|
||||
duration: Duration,
|
||||
nodes: uint,
|
||||
tolerance: uint,
|
||||
autoRenew: bool = false): Future[?!Cid] {.async.} =
|
||||
## Initiate a request for storage sequence, this might
|
||||
## be a multistep procedure.
|
||||
##
|
||||
## Roughly the flow is as follows:
|
||||
## - Get the original cid from the store (should have already been uploaded)
|
||||
## - Erasure code it according to the nodes and tolerance parameters
|
||||
## - Run the PoR setup on the erasure dataset
|
||||
## - Call into the marketplace and purchasing contracts
|
||||
##
|
||||
trace "Received a request for storage!", cid, ppb, duration, nodes, tolerance, autoRenew
|
||||
|
||||
without blk =? (await self.blockStore.getBlock(cid)), error:
|
||||
trace "Unable to retrieve manifest block", cid
|
||||
return failure(error)
|
||||
|
||||
without mc =? blk.cid.contentType():
|
||||
trace "Couldn't identify Cid!", cid
|
||||
return failure("Couldn't identify Cid! " & $cid)
|
||||
|
||||
# if we got a manifest, stream the blocks
|
||||
if $mc notin ManifestContainers:
|
||||
trace "Not a manifest type!", cid, mc
|
||||
return failure("Not a manifest type!")
|
||||
|
||||
without var manifest =? Manifest.decode(blk.data), error:
|
||||
trace "Unable to decode manifest from block", cid
|
||||
return failure(error)
|
||||
|
||||
# Erasure code the dataset according to provided parameters
|
||||
without encoded =? (await self.erasure.encode(manifest, nodes.int, tolerance.int)), error:
|
||||
trace "Unable to erasure code dataset", cid
|
||||
return failure(error)
|
||||
|
||||
without encodedData =? encoded.encode(), error:
|
||||
trace "Unable to encode protected manifest"
|
||||
return failure(error)
|
||||
|
||||
without encodedBlk =? bt.Block.new(data = encodedData, codec = DagPBCodec), error:
|
||||
trace "Unable to create block from encoded manifest"
|
||||
return failure(error)
|
||||
|
||||
if not (await self.blockStore.putBlock(encodedBlk)):
|
||||
trace "Unable to store encoded manifest block", cid = encodedBlk.cid
|
||||
return failure("Unable to store encoded manifest block")
|
||||
|
||||
return encodedBlk.cid.success
|
||||
|
||||
proc new*(
|
||||
T: type DaggerNodeRef,
|
||||
switch: Switch,
|
||||
store: BlockStore,
|
||||
engine: BlockExcEngine): T =
|
||||
engine: BlockExcEngine,
|
||||
erasure: Erasure): T =
|
||||
T(
|
||||
switch: switch,
|
||||
blockStore: store,
|
||||
engine: engine)
|
||||
engine: engine,
|
||||
erasure: erasure)
|
||||
|
|
|
@ -20,10 +20,12 @@ import pkg/chronicles
|
|||
import pkg/chronos
|
||||
import pkg/presto
|
||||
import pkg/libp2p
|
||||
import pkg/stew/base10
|
||||
|
||||
import pkg/libp2p/routing_record
|
||||
|
||||
import ../node
|
||||
import ../blocktype
|
||||
|
||||
proc validate(
|
||||
pattern: string,
|
||||
|
@ -35,13 +37,14 @@ proc encodeString(cid: type Cid): Result[string, cstring] =
|
|||
ok($cid)
|
||||
|
||||
proc decodeString(T: type Cid, value: string): Result[Cid, cstring] =
|
||||
Cid.init(value)
|
||||
.mapErr do(e: CidError) -> cstring:
|
||||
case e
|
||||
of CidError.Incorrect: "Incorrect Cid"
|
||||
of CidError.Unsupported: "Unsupported Cid"
|
||||
of CidError.Overrun: "Overrun Cid"
|
||||
else: "Error parsing Cid"
|
||||
Cid
|
||||
.init(value)
|
||||
.mapErr do(e: CidError) -> cstring:
|
||||
case e
|
||||
of CidError.Incorrect: "Incorrect Cid"
|
||||
of CidError.Unsupported: "Unsupported Cid"
|
||||
of CidError.Overrun: "Overrun Cid"
|
||||
else: "Error parsing Cid"
|
||||
|
||||
proc encodeString(peerId: PeerID): Result[string, cstring] =
|
||||
ok($peerId)
|
||||
|
@ -57,6 +60,29 @@ proc decodeString(T: type MultiAddress, value: string): Result[MultiAddress, cst
|
|||
.init(value)
|
||||
.mapErr do(e: string) -> cstring: cstring(e)
|
||||
|
||||
proc decodeString(T: type SomeUnsignedInt, value: string): Result[T, cstring] =
|
||||
Base10.decode(T, value)
|
||||
|
||||
proc encodeString(value: SomeUnsignedInt): Result[string, cstring] =
|
||||
ok(Base10.toString(value))
|
||||
|
||||
proc decodeString(T: type Duration, value: string): Result[T, cstring] =
|
||||
let v = ? Base10.decode(uint32, value)
|
||||
ok(v.minutes)
|
||||
|
||||
proc encodeString(value: Duration): Result[string, cstring] =
|
||||
ok($value)
|
||||
|
||||
proc decodeString(T: type bool, value: string): Result[T, cstring] =
|
||||
try:
|
||||
ok(value.parseBool())
|
||||
except CatchableError as exc:
|
||||
let s: cstring = exc.msg
|
||||
err(s) # err(exc.msg) won't compile
|
||||
|
||||
proc encodeString(value: bool): Result[string, cstring] =
|
||||
ok($value)
|
||||
|
||||
proc initRestApi*(node: DaggerNodeRef): RestRouter =
|
||||
var router = RestRouter.init(validate)
|
||||
router.api(
|
||||
|
@ -97,7 +123,6 @@ proc initRestApi*(node: DaggerNodeRef): RestRouter =
|
|||
except CatchableError as e:
|
||||
return RestApiResponse.error(Http400, "Unknown error dialling peer")
|
||||
|
||||
|
||||
router.api(
|
||||
MethodGet,
|
||||
"/api/dagger/v1/download/{id}") do (
|
||||
|
@ -116,18 +141,15 @@ proc initRestApi*(node: DaggerNodeRef): RestRouter =
|
|||
|
||||
var bytes = 0
|
||||
try:
|
||||
if (
|
||||
let retr = await node.retrieve(id.get());
|
||||
retr.isErr):
|
||||
return RestApiResponse.error(Http404, retr.error.msg)
|
||||
without stream =? (await node.retrieve(id.get())), error:
|
||||
return RestApiResponse.error(Http404, error.msg)
|
||||
|
||||
resp.addHeader("Content-Type", "application/octet-stream")
|
||||
await resp.prepareChunked()
|
||||
|
||||
stream = retr.get()
|
||||
while not stream.atEof:
|
||||
var
|
||||
buff = newSeqUninitialized[byte](FileChunkSize)
|
||||
buff = newSeqUninitialized[byte](BlockSize)
|
||||
len = await stream.readOnce(addr buff[0], buff.len)
|
||||
|
||||
buff.setLen(len)
|
||||
|
@ -146,6 +168,91 @@ proc initRestApi*(node: DaggerNodeRef): RestRouter =
|
|||
if not stream.isNil:
|
||||
await stream.close()
|
||||
|
||||
router.api(
|
||||
MethodPost,
|
||||
"/api/dagger/v1/storage/request/{cid}") do (
|
||||
cid: Cid,
|
||||
ppb: Option[uint],
|
||||
duration: Option[Duration],
|
||||
nodes: Option[uint],
|
||||
loss: Option[uint],
|
||||
renew: Option[bool]) -> RestApiResponse:
|
||||
## Create a request for storage
|
||||
##
|
||||
## Cid - the cid of the previously uploaded dataset
|
||||
## ppb - the price per byte the client is willing to pay
|
||||
## duration - the duration of the contract
|
||||
## nodeCount - the total amount of the nodes storing the dataset, including `lossTolerance`
|
||||
## lossTolerance - the number of nodes losses the user is willing to tolerate
|
||||
## autoRenew - should the contract be autorenewed -
|
||||
## will fail unless the user has enough funds lockedup
|
||||
##
|
||||
|
||||
var
|
||||
cid =
|
||||
if cid.isErr:
|
||||
return RestApiResponse.error(Http400, $cid.error())
|
||||
else:
|
||||
cid.get()
|
||||
|
||||
ppb =
|
||||
if ppb.isNone:
|
||||
return RestApiResponse.error(Http400, "Missing ppb")
|
||||
else:
|
||||
if ppb.get().isErr:
|
||||
return RestApiResponse.error(Http500, $ppb.get().error)
|
||||
else:
|
||||
ppb.get().get()
|
||||
|
||||
duration =
|
||||
if duration.isNone:
|
||||
return RestApiResponse.error(Http400, "Missing duration")
|
||||
else:
|
||||
if duration.get().isErr:
|
||||
return RestApiResponse.error(Http500, $duration.get().error)
|
||||
else:
|
||||
duration.get().get()
|
||||
|
||||
nodes =
|
||||
if nodes.isNone:
|
||||
return RestApiResponse.error(Http400, "Missing node count")
|
||||
else:
|
||||
if nodes.get().isErr:
|
||||
return RestApiResponse.error(Http500, $nodes.get().error)
|
||||
else:
|
||||
nodes.get().get()
|
||||
|
||||
loss =
|
||||
if loss.isNone:
|
||||
return RestApiResponse.error(Http400, "Missing loss tolerance")
|
||||
else:
|
||||
if loss.get().isErr:
|
||||
return RestApiResponse.error(Http500, $loss.get().error)
|
||||
else:
|
||||
loss.get().get()
|
||||
|
||||
renew = if renew.isNone:
|
||||
false
|
||||
else:
|
||||
if renew.get().isErr:
|
||||
return RestApiResponse.error(Http500, $renew.get().error)
|
||||
else:
|
||||
renew.get().get()
|
||||
|
||||
try:
|
||||
without storageCid =? (await node.requestStorage(
|
||||
cid,
|
||||
ppb,
|
||||
duration,
|
||||
nodes,
|
||||
loss,
|
||||
renew)), error:
|
||||
return RestApiResponse.error(Http500, error.msg)
|
||||
|
||||
return RestApiResponse.response($storageCid)
|
||||
except CatchableError as exc:
|
||||
return RestApiResponse.error(Http500, exc.msg)
|
||||
|
||||
router.rawApi(
|
||||
MethodPost,
|
||||
"/api/dagger/v1/upload") do (
|
||||
|
@ -173,7 +280,7 @@ proc initRestApi*(node: DaggerNodeRef): RestRouter =
|
|||
try:
|
||||
while not reader.atEof:
|
||||
var
|
||||
buff = newSeqUninitialized[byte](FileChunkSize)
|
||||
buff = newSeqUninitialized[byte](BlockSize)
|
||||
len = await reader.readOnce(addr buff[0], buff.len)
|
||||
|
||||
buff.setLen(len)
|
||||
|
@ -185,8 +292,8 @@ proc initRestApi*(node: DaggerNodeRef): RestRouter =
|
|||
bytes += len
|
||||
|
||||
await stream.pushEof()
|
||||
without cid =? (await storeFut):
|
||||
return RestApiResponse.error(Http500)
|
||||
without cid =? (await storeFut), error:
|
||||
return RestApiResponse.error(Http500, error.msg)
|
||||
|
||||
trace "Uploaded file", bytes, cid = $cid
|
||||
return RestApiResponse.response($cid)
|
||||
|
|
|
@ -42,7 +42,7 @@ suite "Test Node":
|
|||
localStore = CacheStore.new()
|
||||
engine = BlockExcEngine.new(localStore, wallet, network)
|
||||
store = NetworkStore.new(engine, localStore)
|
||||
node = DaggerNodeRef.new(switch, store, engine)
|
||||
node = DaggerNodeRef.new(switch, store, engine, nil) # TODO: pass `Erasure`
|
||||
|
||||
await node.start()
|
||||
|
||||
|
|
Loading…
Reference in New Issue