Slot queue (#455)

## Slot queue
Adds a slot queue, as per the [slot queue design](https://github.com/codex-storage/codex-research/blob/master/design/sales.md#slot-queue).

Any time storage is requested, all slots from that request are immediately added to the queue. Finished, Canclled, Failed requests remove all slots with that request id from the queue. SlotFreed events add a new slot to the queue and SlotFilled events remove the slot from the queue. This allows popping of a slot each time one is processed, making things much simpler.

When an entire request of slots is added to the queue, the slot indices are shuffled randomly to hopefully prevent nodes that pick up the same storage requested event from clashing on the first processed slot index. This allowed removal of assigning a random slot index in the SalePreparing state and it also ensured that all SalesAgents will have a slot index assigned to them at the start thus the removal of the optional slotIndex.

Remove slotId from SlotFreed event as it was not being used. RequestId and slotIndex were added to the SlotFreed event earlier and those are now being used

The slot queue invariant that prioritises queue items added to the queue relies on a scoring mechanism to sort them based on the [sort order in the design document](https://github.com/codex-storage/codex-research/blob/master/design/sales.md#sort-order).

When a storage request is handled by the sales module, a slot index was randomly assigned and then the slot was filled. Now, a random slot index is only assigned when adding an entire request to the slot queue. Additionally, the slot is checked that its state is `SlotState.Free` before continuing with the download process.

SlotQueue should always ensure the underlying AsyncHeapQueue has one less than the maximum items, ensuring the SlotQueue can always have space to add an additional item regardless if it’s full or not.

Constructing `SlotQueue.workers` in `SlotQueue.new` calls `newAsyncQueue` which causes side effects, so the construction call had to be moved to `SlotQueue.start`.

Prevent loading request from contract (network request) if there is an existing item in queue for that request.

Check availability before adding request to queue.

Add ability to query market contract for past events. When new availabilities are added, the `onReservationAdded` callback is triggered in which past `StorageRequested` events are queried, and those slots are added to the queue (filtered by availability on `push` and filtered by state in `SalePreparing`).

#### Request Workers
Limit the concurrent requests being processed in the queue by using a limited pool of workers (default = 3). Workers are in a data structure of type `AsyncQueue[SlotQueueWorker]`. This allows us to await a `popFirst` for available workers inside of the main SlotQueue event loop

Add an `onCleanUp` that stops the agents and removes them from the sales module agent list. `onCleanUp` is called from sales end states (eg ignored, cancelled, finished, failed, errored).

Add a `doneProcessing` future to `SlotQueueWorker` to be completed in the `OnProcessSlot` callback. Each `doneProcessing` future created is cancelled and awaited in `SlotQueue.stop` (thanks to `TrackableFuturees`), which forced `stop` to become async.
  - Cancel dispatched workers and the `onProcessSlot` callbacks, prevents zombie callbacks

#### Add TrackableFutures
Allow tracking of futures in a module so they can be cancelled at a later time. Useful for asyncSpawned futures, but works for any future.

### Sales module
The sales module needed to subscribe to request events to ensure that the request queue was managed correctly on each event. In the process of doing this, the sales agents were updated to avoid subscribing to events in each agent, and instead dispatch received events from the sales module to all created sales agents. This would prevent memory leaks on having too many eventemitters subscribed to.
  - prevent removal of agents from sales module while stopping, otherwise the agents seq len is modified while iterating

An additional sales agent state was added, `SalePreparing`, that handles all state machine setup, such as retrieving the request and subscribing to events that were previously in the `SaleDownloading` state.

Once agents have parked in an end state (eg ignored, cancelled, finished, failed, errored), they were not getting cleaned up and the sales module was keeping a handle on their reference. An `onCleanUp` callback was created to be called after the state machine enters an end state, which could prevent a memory leak if the number of requests coming in is high.

Move the SalesAgent callback raises pragmas from the Sales module to the proc definition in SalesAgent. This avoids having to catch `Exception`.
  - remove unneeded error handling as pragmas were moved

Move sales.subscriptions from an object containing named subscriptions to a `seq[Subscription]` directly on the sales object.

Sales tests: shut down repo after sales stop, to fix SIGABRT in CI

### Add async Promise API
  - modelled after JavaScript Promise API
  - alternative to `asyncSpawn` that allows handling of async calls in a synchronous context (including access to the synchronous closure) with less additional procs to be declared
  - Write less code, catch errors that would otherwise defect in asyncspawn, and execute a callback after completion
  - Add cancellation callbacks to utils/then, ensuring cancellations are handled properly

## Dependencies
- bump codex-contracts-eth to support slot queue (https://github.com/codex-storage/codex-contracts-eth/pull/61)
- bump nim-ethers to 0.5.0
- Bump nim-json-rpc submodule to 0bf2bcb

---------

Co-authored-by: Jaremy Creechley <creechley@gmail.com>
This commit is contained in:
Eric 2023-07-25 12:50:30 +10:00 committed by GitHub
parent 14c5270e83
commit 1d161d383e
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
41 changed files with 2331 additions and 218 deletions

View File

@ -13,7 +13,7 @@ requires "bearssl >= 0.1.4"
requires "chronicles >= 0.7.2"
requires "chronos >= 2.5.2"
requires "confutils"
requires "ethers >= 0.2.4 & < 0.3.0"
requires "ethers >= 0.5.0 & < 0.6.0"
requires "libbacktrace"
requires "libp2p"
requires "metrics"

View File

@ -22,14 +22,14 @@ proc start*(clock: OnChainClock) {.async.} =
return
clock.started = true
proc onBlock(blck: Block) {.async, upraises:[].} =
proc onBlock(blck: Block) {.upraises:[].} =
let blockTime = initTime(blck.timestamp.truncate(int64), 0)
let computerTime = getTime()
clock.offset = blockTime - computerTime
clock.newBlock.fire()
if latestBlock =? (await clock.provider.getBlock(BlockTag.latest)):
await onBlock(latestBlock)
onBlock(latestBlock)
clock.subscription = await clock.provider.subscribe(onBlock)

View File

@ -1,5 +1,6 @@
import std/sequtils
import std/strutils
import std/strformat
import std/sugar
import pkg/chronicles
import pkg/ethers
import pkg/ethers/testing
@ -36,7 +37,7 @@ proc approveFunds(market: OnChainMarket, amount: UInt256) {.async.} =
let tokenAddress = await market.contract.token()
let token = Erc20Token.new(tokenAddress, market.signer)
await token.approve(market.contract.address(), amount)
discard await token.approve(market.contract.address(), amount)
method getSigner*(market: OnChainMarket): Future[Address] {.async.} =
return await market.signer.getAddress()
@ -168,11 +169,13 @@ method canProofBeMarkedAsMissing*(
trace "Proof can not be marked as missing", msg = e.msg
return false
method subscribeRequests(market: OnChainMarket,
method subscribeRequests*(market: OnChainMarket,
callback: OnRequest):
Future[MarketSubscription] {.async.} =
proc onEvent(event: StorageRequested) {.upraises:[].} =
callback(event.requestId, event.ask)
callback(event.requestId,
event.ask,
event.expiry)
let subscription = await market.contract.subscribe(StorageRequested, onEvent)
return OnChainMarketSubscription(eventSubscription: subscription)
@ -198,10 +201,18 @@ method subscribeSlotFreed*(market: OnChainMarket,
callback: OnSlotFreed):
Future[MarketSubscription] {.async.} =
proc onEvent(event: SlotFreed) {.upraises:[].} =
callback(event.slotId)
callback(event.requestId, event.slotIndex)
let subscription = await market.contract.subscribe(SlotFreed, onEvent)
return OnChainMarketSubscription(eventSubscription: subscription)
method subscribeFulfillment(market: OnChainMarket,
callback: OnFulfillment):
Future[MarketSubscription] {.async.} =
proc onEvent(event: RequestFulfilled) {.upraises:[].} =
callback(event.requestId)
let subscription = await market.contract.subscribe(RequestFulfilled, onEvent)
return OnChainMarketSubscription(eventSubscription: subscription)
method subscribeFulfillment(market: OnChainMarket,
requestId: RequestId,
callback: OnFulfillment):
@ -212,6 +223,14 @@ method subscribeFulfillment(market: OnChainMarket,
let subscription = await market.contract.subscribe(RequestFulfilled, onEvent)
return OnChainMarketSubscription(eventSubscription: subscription)
method subscribeRequestCancelled*(market: OnChainMarket,
callback: OnRequestCancelled):
Future[MarketSubscription] {.async.} =
proc onEvent(event: RequestCancelled) {.upraises:[].} =
callback(event.requestId)
let subscription = await market.contract.subscribe(RequestCancelled, onEvent)
return OnChainMarketSubscription(eventSubscription: subscription)
method subscribeRequestCancelled*(market: OnChainMarket,
requestId: RequestId,
callback: OnRequestCancelled):
@ -222,6 +241,14 @@ method subscribeRequestCancelled*(market: OnChainMarket,
let subscription = await market.contract.subscribe(RequestCancelled, onEvent)
return OnChainMarketSubscription(eventSubscription: subscription)
method subscribeRequestFailed*(market: OnChainMarket,
callback: OnRequestFailed):
Future[MarketSubscription] {.async.} =
proc onEvent(event: RequestFailed) {.upraises:[]} =
callback(event.requestId)
let subscription = await market.contract.subscribe(RequestFailed, onEvent)
return OnChainMarketSubscription(eventSubscription: subscription)
method subscribeRequestFailed*(market: OnChainMarket,
requestId: RequestId,
callback: OnRequestFailed):
@ -242,3 +269,24 @@ method subscribeProofSubmission*(market: OnChainMarket,
method unsubscribe*(subscription: OnChainMarketSubscription) {.async.} =
await subscription.eventSubscription.unsubscribe()
method queryPastStorageRequests*(market: OnChainMarket,
blocksAgo: int):
Future[seq[PastStorageRequest]] {.async.} =
let contract = market.contract
let provider = contract.provider
let head = await provider.getBlockNumber()
let fromBlock = BlockTag.init(head - blocksAgo.abs.u256)
let events = await contract.queryFilter(StorageRequested,
fromBlock,
BlockTag.latest)
return events.map(event =>
PastStorageRequest(
requestId: event.requestId,
ask: event.ask,
expiry: event.expiry
)
)

View File

@ -18,13 +18,13 @@ type
StorageRequested* = object of Event
requestId*: RequestId
ask*: StorageAsk
expiry*: UInt256
SlotFilled* = object of Event
requestId* {.indexed.}: RequestId
slotIndex* {.indexed.}: UInt256
slotId*: SlotId
slotIndex*: UInt256
SlotFreed* = object of Event
requestId* {.indexed.}: RequestId
slotId*: SlotId
slotIndex*: UInt256
RequestFulfilled* = object of Event
requestId* {.indexed.}: RequestId
RequestCancelled* = object of Event

View File

@ -4,6 +4,8 @@ import pkg/nimcrypto
import pkg/ethers/fields
import pkg/questionable/results
import pkg/stew/byteutils
import pkg/json_serialization
import pkg/upraises
export contractabi
@ -203,3 +205,17 @@ func price*(request: StorageRequest): UInt256 =
func size*(ask: StorageAsk): UInt256 =
ask.slots.u256 * ask.slotSize
proc writeValue*(
writer: var JsonWriter,
value: SlotId | RequestId) {.upraises:[IOError].} =
mixin writeValue
writer.writeValue value.toArray
proc readValue*[T: SlotId | RequestId](
reader: var JsonReader,
value: var T) {.upraises: [SerializationError, IOError].} =
mixin readValue
value = T reader.readValue(T.distinctBase)

View File

@ -15,13 +15,19 @@ export periods
type
Market* = ref object of RootObj
Subscription* = ref object of RootObj
OnRequest* = proc(id: RequestId, ask: StorageAsk) {.gcsafe, upraises:[].}
OnRequest* = proc(id: RequestId,
ask: StorageAsk,
expiry: UInt256) {.gcsafe, upraises:[].}
OnFulfillment* = proc(requestId: RequestId) {.gcsafe, upraises: [].}
OnSlotFilled* = proc(requestId: RequestId, slotIndex: UInt256) {.gcsafe, upraises:[].}
OnSlotFreed* = proc(slotId: SlotId) {.gcsafe, upraises: [].}
OnSlotFreed* = proc(requestId: RequestId, slotIndex: UInt256) {.gcsafe, upraises: [].}
OnRequestCancelled* = proc(requestId: RequestId) {.gcsafe, upraises:[].}
OnRequestFailed* = proc(requestId: RequestId) {.gcsafe, upraises:[].}
OnProofSubmitted* = proc(id: SlotId, proof: seq[byte]) {.gcsafe, upraises:[].}
PastStorageRequest* = object
requestId*: RequestId
ask*: StorageAsk
expiry*: UInt256
method getSigner*(market: Market): Future[Address] {.base, async.} =
raiseAssert("not implemented")
@ -112,6 +118,11 @@ method canProofBeMarkedAsMissing*(market: Market,
period: Period): Future[bool] {.base, async.} =
raiseAssert("not implemented")
method subscribeFulfillment*(market: Market,
callback: OnFulfillment):
Future[Subscription] {.base, async.} =
raiseAssert("not implemented")
method subscribeFulfillment*(market: Market,
requestId: RequestId,
callback: OnFulfillment):
@ -135,12 +146,22 @@ method subscribeSlotFreed*(market: Market,
Future[Subscription] {.base, async.} =
raiseAssert("not implemented")
method subscribeRequestCancelled*(market: Market,
callback: OnRequestCancelled):
Future[Subscription] {.base, async.} =
raiseAssert("not implemented")
method subscribeRequestCancelled*(market: Market,
requestId: RequestId,
callback: OnRequestCancelled):
Future[Subscription] {.base, async.} =
raiseAssert("not implemented")
method subscribeRequestFailed*(market: Market,
callback: OnRequestFailed):
Future[Subscription] {.base, async.} =
raiseAssert("not implemented")
method subscribeRequestFailed*(market: Market,
requestId: RequestId,
callback: OnRequestFailed):
@ -154,3 +175,8 @@ method subscribeProofSubmission*(market: Market,
method unsubscribe*(subscription: Subscription) {.base, async, upraises:[].} =
raiseAssert("not implemented")
method queryPastStorageRequests*(market: Market,
blocksAgo: int):
Future[seq[PastStorageRequest]] {.base, async.} =
raiseAssert("not implemented")

View File

@ -1,20 +1,24 @@
import std/sequtils
import std/sugar
import std/tables
import pkg/questionable
import pkg/upraises
import pkg/stint
import pkg/chronicles
import pkg/datastore
import ./rng
import ./market
import ./clock
import ./proving
import ./stores
import ./contracts/requests
import ./contracts/marketplace
import ./sales/salescontext
import ./sales/salesagent
import ./sales/statemachine
import ./sales/states/downloading
import ./sales/slotqueue
import ./sales/trackedfutures
import ./sales/states/preparing
import ./sales/states/unknown
import ./utils/then
## Sales holds a list of available storage that it may sell.
##
@ -43,8 +47,10 @@ logScope:
type
Sales* = ref object
context*: SalesContext
subscription*: ?market.Subscription
agents*: seq[SalesAgent]
running: bool
subscriptions: seq[market.Subscription]
trackedFutures: TrackedFutures
proc `onStore=`*(sales: Sales, onStore: OnStore) =
sales.context.onStore = some onStore
@ -67,37 +73,47 @@ func new*(_: type Sales,
proving: Proving,
repo: RepoStore): Sales =
Sales(context: SalesContext(
market: market,
clock: clock,
proving: proving,
reservations: Reservations.new(repo)
))
let reservations = Reservations.new(repo)
Sales(
context: SalesContext(
market: market,
clock: clock,
proving: proving,
reservations: reservations,
slotQueue: SlotQueue.new(reservations)
),
trackedFutures: TrackedFutures.new(),
subscriptions: @[]
)
proc randomSlotIndex(numSlots: uint64): UInt256 =
let rng = Rng.instance
let slotIndex = rng.rand(numSlots - 1)
return slotIndex.u256
proc remove(sales: Sales, agent: SalesAgent) {.async.} =
await agent.stop()
if sales.running:
sales.agents.keepItIf(it != agent)
proc handleRequest(sales: Sales,
requestId: RequestId,
ask: StorageAsk) =
proc cleanUp(sales: Sales,
agent: SalesAgent,
processing: Future[void]) {.async.} =
await sales.remove(agent)
# signal back to the slot queue to cycle a worker
if not processing.isNil and not processing.finished():
processing.complete()
debug "handling storage requested",
slots = ask.slots, slotSize = ask.slotSize, duration = ask.duration,
reward = ask.reward, maxSlotLoss = ask.maxSlotLoss
proc processSlot(sales: Sales, item: SlotQueueItem, done: Future[void]) =
debug "processing slot from queue", requestId = $item.requestId,
slot = item.slotIndex
# TODO: check if random slot is actually available (not already filled)
let slotIndex = randomSlotIndex(ask.slots)
let agent = newSalesAgent(
sales.context,
requestId,
slotIndex,
item.requestId,
item.slotIndex.u256,
none StorageRequest
)
agent.context.onIgnored = proc {.gcsafe, upraises:[].} =
sales.agents.keepItIf(it != agent)
agent.start(SaleDownloading())
agent.context.onCleanUp = proc {.async.} =
await sales.cleanUp(agent, done)
agent.start(SalePreparing())
sales.agents.add agent
proc mySlots*(sales: Sales): Future[seq[Slot]] {.async.} =
@ -120,27 +136,272 @@ proc load*(sales: Sales) {.async.} =
slot.request.id,
slot.slotIndex,
some slot.request)
agent.context.onCleanUp = proc {.async.} = await sales.remove(agent)
agent.start(SaleUnknown())
sales.agents.add agent
proc start*(sales: Sales) {.async.} =
doAssert sales.subscription.isNone, "Sales already started"
proc onReservationAdded(sales: Sales, availability: Availability) {.async.} =
## Query last 256 blocks for new requests, adding them to the queue. `push`
## checks for availability before adding to the queue. If processed, the
## sales agent will check if the slot is free.
let context = sales.context
let market = context.market
let queue = context.slotQueue
proc onRequest(requestId: RequestId, ask: StorageAsk) {.gcsafe, upraises:[].} =
sales.handleRequest(requestId, ask)
logScope:
topics = "sales onReservationAdded callback"
trace "reservation added, querying past storage requests to add to queue"
try:
sales.subscription = some await sales.context.market.subscribeRequests(onRequest)
let events = await market.queryPastStorageRequests(256)
let requests = events.map(event =>
SlotQueueItem.init(event.requestId, event.ask, event.expiry)
)
trace "found past storage requested events to add to queue",
events = events.len
for slots in requests:
for slot in slots:
if err =? (await queue.push(slot)).errorOption:
# continue on error
if err of QueueNotRunningError:
warn "cannot push items to queue, queue is not running"
elif err of NoMatchingAvailabilityError:
info "slot in queue had no matching availabilities, ignoring"
elif err of SlotsOutOfRangeError:
warn "Too many slots, cannot add to queue"
elif err of SlotQueueItemExistsError:
trace "item already exists, ignoring"
discard
else: raise err
except CatchableError as e:
error "Unable to start sales", msg = e.msg
warn "Error adding request to SlotQueue", error = e.msg
discard
proc onStorageRequested(sales: Sales,
requestId: RequestId,
ask: StorageAsk,
expiry: UInt256) =
logScope:
topics = "sales onStorageRequested"
requestId
slots = ask.slots
expiry
let slotQueue = sales.context.slotQueue
trace "storage requested, adding slots to queue"
without items =? SlotQueueItem.init(requestId, ask, expiry).catch, err:
if err of SlotsOutOfRangeError:
warn "Too many slots, cannot add to queue"
else:
warn "Failed to create slot queue items from request", error = err.msg
for item in items:
# continue on failure
slotQueue.push(item)
.track(sales)
.catch(proc(err: ref CatchableError) =
if err of NoMatchingAvailabilityError:
info "slot in queue had no matching availabilities, ignoring"
elif err of SlotQueueItemExistsError:
error "Failed to push item to queue becaue it already exists"
elif err of QueueNotRunningError:
warn "Failed to push item to queue becaue queue is not running"
else:
warn "Error adding request to SlotQueue", error = err.msg
)
proc onSlotFreed(sales: Sales,
requestId: RequestId,
slotIndex: UInt256) =
logScope:
topics = "sales onSlotFreed"
requestId
slotIndex
trace "slot freed, adding to queue"
proc addSlotToQueue() {.async.} =
let context = sales.context
let market = context.market
let queue = context.slotQueue
# first attempt to populate request using existing slot metadata in queue
without var found =? queue.populateItem(requestId,
slotIndex.truncate(uint16)):
trace "no existing request metadata, getting request info from contract"
# if there's no existing slot for that request, retrieve the request
# from the contract.
without request =? await market.getRequest(requestId):
error "unknown request in contract"
return
found = SlotQueueItem.init(request, slotIndex.truncate(uint16))
if err =? (await queue.push(found)).errorOption:
raise err
addSlotToQueue()
.track(sales)
.catch(proc(err: ref CatchableError) =
if err of NoMatchingAvailabilityError:
info "slot in queue had no matching availabilities, ignoring"
elif err of SlotQueueItemExistsError:
error "Failed to push item to queue becaue it already exists"
elif err of QueueNotRunningError:
warn "Failed to push item to queue becaue queue is not running"
else:
warn "Error adding request to SlotQueue", error = err.msg
)
proc subscribeRequested(sales: Sales) {.async.} =
let context = sales.context
let market = context.market
proc onStorageRequested(requestId: RequestId,
ask: StorageAsk,
expiry: UInt256) =
sales.onStorageRequested(requestId, ask, expiry)
try:
let sub = await market.subscribeRequests(onStorageRequested)
sales.subscriptions.add(sub)
except CatchableError as e:
error "Unable to subscribe to storage request events", msg = e.msg
proc subscribeCancellation(sales: Sales) {.async.} =
let context = sales.context
let market = context.market
let queue = context.slotQueue
proc onCancelled(requestId: RequestId) =
trace "request cancelled, removing all request slots from queue"
queue.delete(requestId)
try:
let sub = await market.subscribeRequestCancelled(onCancelled)
sales.subscriptions.add(sub)
except CatchableError as e:
error "Unable to subscribe to cancellation events", msg = e.msg
proc subscribeFulfilled*(sales: Sales) {.async.} =
let context = sales.context
let market = context.market
let queue = context.slotQueue
proc onFulfilled(requestId: RequestId) =
trace "request fulfilled, removing all request slots from queue"
queue.delete(requestId)
for agent in sales.agents:
agent.onFulfilled(requestId)
try:
let sub = await market.subscribeFulfillment(onFulfilled)
sales.subscriptions.add(sub)
except CatchableError as e:
error "Unable to subscribe to storage fulfilled events", msg = e.msg
proc subscribeFailure(sales: Sales) {.async.} =
let context = sales.context
let market = context.market
let queue = context.slotQueue
proc onFailed(requestId: RequestId) =
trace "request failed, removing all request slots from queue"
queue.delete(requestId)
for agent in sales.agents:
agent.onFailed(requestId)
try:
let sub = await market.subscribeRequestFailed(onFailed)
sales.subscriptions.add(sub)
except CatchableError as e:
error "Unable to subscribe to storage failure events", msg = e.msg
proc subscribeSlotFilled(sales: Sales) {.async.} =
let context = sales.context
let market = context.market
let queue = context.slotQueue
proc onSlotFilled(requestId: RequestId, slotIndex: UInt256) =
trace "slot filled, removing from slot queue", requestId, slotIndex
queue.delete(requestId, slotIndex.truncate(uint16))
for agent in sales.agents:
agent.onSlotFilled(requestId, slotIndex)
try:
let sub = await market.subscribeSlotFilled(onSlotFilled)
sales.subscriptions.add(sub)
except CatchableError as e:
error "Unable to subscribe to slot filled events", msg = e.msg
proc subscribeSlotFreed(sales: Sales) {.async.} =
let context = sales.context
let market = context.market
proc onSlotFreed(requestId: RequestId, slotIndex: UInt256) =
sales.onSlotFreed(requestId, slotIndex)
try:
let sub = await market.subscribeSlotFreed(onSlotFreed)
sales.subscriptions.add(sub)
except CatchableError as e:
error "Unable to subscribe to slot freed events", msg = e.msg
proc startSlotQueue(sales: Sales) {.async.} =
let slotQueue = sales.context.slotQueue
let reservations = sales.context.reservations
slotQueue.onProcessSlot =
proc(item: SlotQueueItem, done: Future[void]) {.async.} =
sales.processSlot(item, done)
asyncSpawn slotQueue.start()
reservations.onReservationAdded =
proc(availability: Availability) {.async.} =
await sales.onReservationAdded(availability)
proc subscribe(sales: Sales) {.async.} =
await sales.subscribeRequested()
await sales.subscribeFulfilled()
await sales.subscribeFailure()
await sales.subscribeSlotFilled()
await sales.subscribeSlotFreed()
await sales.subscribeCancellation()
proc unsubscribe(sales: Sales) {.async.} =
for sub in sales.subscriptions:
try:
await sub.unsubscribe()
except CatchableError as e:
error "Unable to unsubscribe from subscription", error = e.msg
proc start*(sales: Sales) {.async.} =
await sales.startSlotQueue()
await sales.subscribe()
proc stop*(sales: Sales) {.async.} =
if subscription =? sales.subscription:
sales.subscription = market.Subscription.none
try:
await subscription.unsubscribe()
except CatchableError as e:
warn "Unsubscribe failed", msg = e.msg
trace "stopping sales"
sales.running = false
await sales.context.slotQueue.stop()
await sales.unsubscribe()
await sales.trackedFutures.cancelTracked()
for agent in sales.agents:
await agent.stop()
sales.agents = @[]

View File

@ -42,7 +42,9 @@ type
used*: bool
Reservations* = ref object
repo: RepoStore
onReservationAdded: ?OnReservationAdded
GetNext* = proc(): Future[?Availability] {.upraises: [], gcsafe, closure.}
OnReservationAdded* = proc(availability: Availability): Future[void] {.upraises: [], gcsafe.}
AvailabilityIter* = ref object
finished*: bool
next*: GetNext
@ -96,18 +98,22 @@ proc toErr[E1: ref CatchableError, E2: AvailabilityError](
proc writeValue*(
writer: var JsonWriter,
value: SlotId | AvailabilityId) {.upraises:[IOError].} =
value: AvailabilityId) {.upraises:[IOError].} =
mixin writeValue
writer.writeValue value.toArray
proc readValue*[T: SlotId | AvailabilityId](
proc readValue*[T: AvailabilityId](
reader: var JsonReader,
value: var T) {.upraises: [SerializationError, IOError].} =
mixin readValue
value = T reader.readValue(T.distinctBase)
proc `onReservationAdded=`*(self: Reservations,
onReservationAdded: OnReservationAdded) =
self.onReservationAdded = some onReservationAdded
func key(id: AvailabilityId): ?!Key =
(ReservationsKey / id.toArray.toHex)
@ -210,6 +216,15 @@ proc reserve*(
return failure(updateErr)
if onReservationAdded =? self.onReservationAdded:
try:
await onReservationAdded(availability)
except CatchableError as e:
# we don't have any insight into types of errors that `onProcessSlot` can
# throw because it is caller-defined
warn "Unknown error during 'onReservationAdded' callback",
availabilityId = availability.id, error = e.msg
return success()
proc release*(
@ -320,7 +335,7 @@ proc unused*(r: Reservations): Future[?!seq[Availability]] {.async.} =
proc find*(
self: Reservations,
size, duration, minPrice: UInt256, collateral: UInt256,
size, duration, minPrice, collateral: UInt256,
used: bool): Future[?Availability] {.async.} =

View File

@ -1,8 +1,11 @@
import pkg/chronos
import pkg/chronicles
import pkg/questionable
import pkg/questionable/results
import pkg/stint
import pkg/upraises
import ../contracts/requests
import ../utils/asyncspawn
import ../errors
import ./statemachine
import ./salescontext
import ./salesdata
@ -13,10 +16,13 @@ export reservations
logScope:
topics = "marketplace sales"
type SalesAgent* = ref object of Machine
context*: SalesContext
data*: SalesData
subscribed: bool
type
SalesAgent* = ref object of Machine
context*: SalesContext
data*: SalesData
subscribed: bool
SalesAgentError = object of CodexError
AllSlotsFilledError* = object of SalesAgentError
func `==`*(a, b: SalesAgent): bool =
a.data.requestId == b.data.requestId and
@ -41,7 +47,6 @@ proc retrieveRequest*(agent: SalesAgent) {.async.} =
proc subscribeCancellation(agent: SalesAgent) {.async.} =
let data = agent.data
let market = agent.context.market
let clock = agent.context.clock
proc onCancelled() {.async.} =
@ -49,51 +54,34 @@ proc subscribeCancellation(agent: SalesAgent) {.async.} =
return
await clock.waitUntil(request.expiry.truncate(int64))
if not data.fulfilled.isNil:
asyncSpawn data.fulfilled.unsubscribe(), ignore = CatchableError
agent.schedule(cancelledEvent(request))
data.cancelled = onCancelled()
proc onFulfilled(_: RequestId) =
data.cancelled.cancel()
method onFulfilled*(agent: SalesAgent, requestId: RequestId) {.base, gcsafe, upraises: [].} =
if agent.data.requestId == requestId and
not agent.data.cancelled.isNil:
agent.data.cancelled.cancel()
data.fulfilled =
await market.subscribeFulfillment(data.requestId, onFulfilled)
proc subscribeFailure(agent: SalesAgent) {.async.} =
let data = agent.data
let market = agent.context.market
proc onFailed(_: RequestId) =
without request =? data.request:
return
asyncSpawn data.failed.unsubscribe(), ignore = CatchableError
method onFailed*(agent: SalesAgent, requestId: RequestId) {.base, gcsafe, upraises: [].} =
without request =? agent.data.request:
return
if agent.data.requestId == requestId:
agent.schedule(failedEvent(request))
data.failed =
await market.subscribeRequestFailed(data.requestId, onFailed)
method onSlotFilled*(agent: SalesAgent,
requestId: RequestId,
slotIndex: UInt256) {.base, gcsafe, upraises: [].} =
proc subscribeSlotFilled(agent: SalesAgent) {.async.} =
let data = agent.data
let market = agent.context.market
proc onSlotFilled(requestId: RequestId, slotIndex: UInt256) =
asyncSpawn data.slotFilled.unsubscribe(), ignore = CatchableError
agent.schedule(slotFilledEvent(requestId, data.slotIndex))
data.slotFilled =
await market.subscribeSlotFilled(data.requestId,
data.slotIndex,
onSlotFilled)
if agent.data.requestId == requestId and
agent.data.slotIndex == slotIndex:
agent.schedule(slotFilledEvent(requestId, slotIndex))
proc subscribe*(agent: SalesAgent) {.async.} =
if agent.subscribed:
return
await agent.subscribeCancellation()
await agent.subscribeFailure()
await agent.subscribeSlotFilled()
agent.subscribed = true
proc unsubscribe*(agent: SalesAgent) {.async.} =
@ -101,25 +89,7 @@ proc unsubscribe*(agent: SalesAgent) {.async.} =
return
let data = agent.data
try:
if not data.fulfilled.isNil:
await data.fulfilled.unsubscribe()
data.fulfilled = nil
except CatchableError:
discard
try:
if not data.failed.isNil:
await data.failed.unsubscribe()
data.failed = nil
except CatchableError:
discard
try:
if not data.slotFilled.isNil:
await data.slotFilled.unsubscribe()
data.slotFilled = nil
except CatchableError:
discard
if not data.cancelled.isNil:
if not data.cancelled.isNil and not data.cancelled.finished:
await data.cancelled.cancelAndWait()
data.cancelled = nil

View File

@ -5,6 +5,7 @@ import ../node/batch
import ../market
import ../clock
import ../proving
import ./slotqueue
import ./reservations
type
@ -14,9 +15,10 @@ type
onStore*: ?OnStore
onClear*: ?OnClear
onSale*: ?OnSale
onIgnored*: OnIgnored
onCleanUp*: OnCleanUp
proving*: Proving
reservations*: Reservations
slotQueue*: SlotQueue
OnStore* = proc(request: StorageRequest,
slot: UInt256,
@ -27,4 +29,4 @@ type
slotIndex: UInt256) {.gcsafe, upraises: [].}
OnSale* = proc(request: StorageRequest,
slotIndex: UInt256) {.gcsafe, upraises: [].}
OnIgnored* = proc() {.gcsafe, upraises: [].}
OnCleanUp* = proc: Future[void] {.gcsafe, upraises: [].}

View File

@ -9,7 +9,4 @@ type
ask*: StorageAsk
request*: ?StorageRequest
slotIndex*: UInt256
failed*: market.Subscription
fulfilled*: market.Subscription
slotFilled*: market.Subscription
cancelled*: Future[void]

396
codex/sales/slotqueue.nim Normal file
View File

@ -0,0 +1,396 @@
import std/sequtils
import std/sugar
import std/tables
import pkg/chronicles
import pkg/chronos
import pkg/questionable
import pkg/questionable/results
import pkg/upraises
import ./reservations
import ./trackedfutures
import ../errors
import ../rng
import ../utils
import ../contracts/requests
import ../utils/asyncheapqueue
import ../utils/then
logScope:
topics = "marketplace slotqueue"
type
OnProcessSlot* =
proc(item: SlotQueueItem, done: Future[void]): Future[void] {.gcsafe, upraises:[].}
# Non-ref obj copies value when assigned, preventing accidental modification
# of values which could cause an incorrect order (eg
# ``slotQueue[1].collateral = 1`` would cause ``collateral`` to be updated,
# but the heap invariant would no longer be honoured. When non-ref, the
# compiler can ensure that statement will fail).
SlotQueueWorker = object
doneProcessing*: Future[void]
SlotQueueItem* = object
requestId: RequestId
slotIndex: uint16
slotSize: UInt256
duration: UInt256
reward: UInt256
collateral: UInt256
expiry: UInt256
# don't need to -1 to prevent overflow when adding 1 (to always allow push)
# because AsyncHeapQueue size is of type `int`, which is larger than `uint16`
SlotQueueSize = range[1'u16..uint16.high]
SlotQueue* = ref object
maxWorkers: int
onProcessSlot: ?OnProcessSlot
queue: AsyncHeapQueue[SlotQueueItem]
reservations: Reservations
running: bool
workers: AsyncQueue[SlotQueueWorker]
trackedFutures: TrackedFutures
SlotQueueError = object of CodexError
SlotQueueItemExistsError* = object of SlotQueueError
SlotQueueItemNotExistsError* = object of SlotQueueError
SlotsOutOfRangeError* = object of SlotQueueError
NoMatchingAvailabilityError* = object of SlotQueueError
QueueNotRunningError* = object of SlotQueueError
# Number of concurrent workers used for processing SlotQueueItems
const DefaultMaxWorkers = 3
# Cap slot queue size to prevent unbounded growth and make sifting more
# efficient. Max size is not equivalent to the number of slots a host can
# service, which is limited by host availabilities and new requests circulating
# the network. Additionally, each new request/slot in the network will be
# included in the queue if it is higher priority than any of the exisiting
# items. Older slots should be unfillable over time as other hosts fill the
# slots.
const DefaultMaxSize = 64'u16
proc profitability(item: SlotQueueItem): UInt256 =
StorageAsk(collateral: item.collateral,
duration: item.duration,
reward: item.reward,
slotSize: item.slotSize).pricePerSlot
proc `<`*(a, b: SlotQueueItem): bool =
# for A to have a higher priority than B (in a min queue), A must be less than
# B.
var scoreA: uint8 = 0
var scoreB: uint8 = 0
proc addIf(score: var uint8, condition: bool, addition: int) =
if condition:
score += 1'u8 shl addition
scoreA.addIf(a.profitability > b.profitability, 3)
scoreB.addIf(a.profitability < b.profitability, 3)
scoreA.addIf(a.collateral < b.collateral, 2)
scoreB.addIf(a.collateral > b.collateral, 2)
scoreA.addIf(a.expiry > b.expiry, 1)
scoreB.addIf(a.expiry < b.expiry, 1)
scoreA.addIf(a.slotSize < b.slotSize, 0)
scoreB.addIf(a.slotSize > b.slotSize, 0)
return scoreA > scoreB
proc `==`*(a, b: SlotQueueItem): bool =
a.requestId == b.requestId and
a.slotIndex == b.slotIndex
proc new*(_: type SlotQueue,
reservations: Reservations,
maxWorkers = DefaultMaxWorkers,
maxSize: SlotQueueSize = DefaultMaxSize): SlotQueue =
if maxWorkers <= 0:
raise newException(ValueError, "maxWorkers must be positive")
if maxWorkers.uint16 > maxSize:
raise newException(ValueError, "maxWorkers must be less than maxSize")
SlotQueue(
maxWorkers: maxWorkers,
# Add 1 to always allow for an extra item to be pushed onto the queue
# temporarily. After push (and sort), the bottom-most item will be deleted
queue: newAsyncHeapQueue[SlotQueueItem](maxSize.int + 1),
reservations: reservations,
running: false,
trackedFutures: TrackedFutures.new()
)
# avoid instantiating `workers` in constructor to avoid side effects in
# `newAsyncQueue` procedure
proc init*(_: type SlotQueueWorker): SlotQueueWorker =
SlotQueueWorker(
doneProcessing: newFuture[void]("slotqueue.worker.processing")
)
proc init*(_: type SlotQueueItem,
requestId: RequestId,
slotIndex: uint16,
ask: StorageAsk,
expiry: UInt256): SlotQueueItem =
SlotQueueItem(
requestId: requestId,
slotIndex: slotIndex,
slotSize: ask.slotSize,
duration: ask.duration,
reward: ask.reward,
collateral: ask.collateral,
expiry: expiry
)
proc init*(_: type SlotQueueItem,
request: StorageRequest,
slotIndex: uint16): SlotQueueItem =
SlotQueueItem.init(request.id,
slotIndex,
request.ask,
request.expiry)
proc init*(_: type SlotQueueItem,
requestId: RequestId,
ask: StorageAsk,
expiry: UInt256): seq[SlotQueueItem] =
if not ask.slots.inRange:
raise newException(SlotsOutOfRangeError, "Too many slots")
var i = 0'u16
proc initSlotQueueItem: SlotQueueItem =
let item = SlotQueueItem.init(requestId, i, ask, expiry)
inc i
return item
var items = newSeqWith(ask.slots.int, initSlotQueueItem())
Rng.instance.shuffle(items)
return items
proc init*(_: type SlotQueueItem,
request: StorageRequest): seq[SlotQueueItem] =
return SlotQueueItem.init(request.id, request.ask, request.expiry)
proc inRange*(val: SomeUnsignedInt): bool =
val.uint16 in SlotQueueSize.low..SlotQueueSize.high
proc requestId*(self: SlotQueueItem): RequestId = self.requestId
proc slotIndex*(self: SlotQueueItem): uint16 = self.slotIndex
proc slotSize*(self: SlotQueueItem): UInt256 = self.slotSize
proc duration*(self: SlotQueueItem): UInt256 = self.duration
proc reward*(self: SlotQueueItem): UInt256 = self.reward
proc collateral*(self: SlotQueueItem): UInt256 = self.collateral
proc running*(self: SlotQueue): bool = self.running
proc len*(self: SlotQueue): int = self.queue.len
proc size*(self: SlotQueue): int = self.queue.size - 1
proc `$`*(self: SlotQueue): string = $self.queue
proc `onProcessSlot=`*(self: SlotQueue, onProcessSlot: OnProcessSlot) =
self.onProcessSlot = some onProcessSlot
proc activeWorkers*(self: SlotQueue): int =
if not self.running: return 0
# active = capacity - available
self.maxWorkers - self.workers.len
proc contains*(self: SlotQueue, item: SlotQueueItem): bool =
self.queue.contains(item)
proc populateItem*(self: SlotQueue,
requestId: RequestId,
slotIndex: uint16): ?SlotQueueItem =
trace "populate item, items in queue", len = self.queue.len
for item in self.queue.items:
trace "populate item search", itemRequestId = item.requestId, requestId
if item.requestId == requestId:
return some SlotQueueItem(
requestId: requestId,
slotIndex: slotIndex,
slotSize: item.slotSize,
duration: item.duration,
reward: item.reward,
collateral: item.collateral,
expiry: item.expiry
)
return none SlotQueueItem
proc push*(self: SlotQueue, item: SlotQueueItem): Future[?!void] {.async.} =
trace "pushing item to queue",
requestId = item.requestId, slotIndex = item.slotIndex
if not self.running:
let err = newException(QueueNotRunningError, "queue not running")
return failure(err)
without availability =? await self.reservations.find(item.slotSize,
item.duration,
item.profitability,
item.collateral,
used = false):
let err = newException(NoMatchingAvailabilityError, "no availability")
return failure(err)
if self.contains(item):
let err = newException(SlotQueueItemExistsError, "item already exists")
return failure(err)
if err =? self.queue.pushNoWait(item).mapFailure.errorOption:
return failure(err)
if self.queue.full():
# delete the last item
self.queue.del(self.queue.size - 1)
doAssert self.queue.len <= self.queue.size - 1
return success()
proc push*(self: SlotQueue, items: seq[SlotQueueItem]): Future[?!void] {.async.} =
for item in items:
if err =? (await self.push(item)).errorOption:
return failure(err)
return success()
proc findByRequest(self: SlotQueue, requestId: RequestId): seq[SlotQueueItem] =
var items: seq[SlotQueueItem] = @[]
for item in self.queue.items:
if item.requestId == requestId:
items.add item
return items
proc delete*(self: SlotQueue, item: SlotQueueItem) =
logScope:
requestId = item.requestId
slotIndex = item.slotIndex
trace "removing item from queue"
if not self.running:
trace "cannot delete item from queue, queue not running"
return
self.queue.delete(item)
proc delete*(self: SlotQueue, requestId: RequestId, slotIndex: uint16) =
let item = SlotQueueItem(requestId: requestId, slotIndex: slotIndex)
self.delete(item)
proc delete*(self: SlotQueue, requestId: RequestId) =
let items = self.findByRequest(requestId)
for item in items:
self.delete(item)
proc `[]`*(self: SlotQueue, i: Natural): SlotQueueItem =
self.queue[i]
proc addWorker(self: SlotQueue): ?!void =
if not self.running:
let err = newException(QueueNotRunningError, "queue must be running")
return failure(err)
trace "adding new worker to worker queue"
let worker = SlotQueueWorker.init()
try:
self.workers.addLastNoWait(worker)
except AsyncQueueFullError:
return failure("failed to add worker, worker queue full")
return success()
proc dispatch(self: SlotQueue,
worker: SlotQueueWorker,
item: SlotQueueItem) {.async.} =
logScope:
requestId = item.requestId
slotIndex = item.slotIndex
if not self.running:
warn "Could not dispatch worker because queue is not running"
return
if onProcessSlot =? self.onProcessSlot:
try:
await onProcessSlot(item, worker.doneProcessing)
await worker.doneProcessing
if err =? self.addWorker().errorOption:
raise err # catch below
except QueueNotRunningError as e:
info "could not re-add worker to worker queue, queue not running",
error = e.msg
except CancelledError:
# do not bubble exception up as it is called with `asyncSpawn` which would
# convert the exception into a `FutureDefect`
discard
except CatchableError as e:
# we don't have any insight into types of errors that `onProcessSlot` can
# throw because it is caller-defined
warn "Unknown error processing slot in worker", error = e.msg
proc start*(self: SlotQueue) {.async.} =
if self.running:
return
trace "starting slot queue"
self.running = true
# must be called in `start` to avoid sideeffects in `new`
self.workers = newAsyncQueue[SlotQueueWorker](self.maxWorkers)
# Add initial workers to the `AsyncHeapQueue`. Once a worker has completed its
# task, a new worker will be pushed to the queue
for i in 0..<self.maxWorkers:
if err =? self.addWorker().errorOption:
error "start: error adding new worker", error = err.msg
while self.running:
try:
let worker = await self.workers.popFirst().track(self) # if workers saturated, wait here for new workers
let item = await self.queue.pop().track(self) # if queue empty, wait here for new items
if not self.running: # may have changed after waiting for pop
trace "not running, exiting"
break
self.dispatch(worker, item)
.track(self)
.catch(proc (e: ref CatchableError) =
error "Unknown error dispatching worker", error = e.msg
)
discard worker.doneProcessing.track(self)
await sleepAsync(1.millis) # poll
except CancelledError:
discard
except CatchableError as e: # raised from self.queue.pop() or self.workers.pop()
warn "slot queue error encountered during processing", error = e.msg
proc stop*(self: SlotQueue) {.async.} =
if not self.running:
return
trace "stopping slot queue"
self.running = false
await self.trackedFutures.cancelTracked()

View File

@ -9,12 +9,12 @@ import ./errorhandling
import ./cancelled
import ./failed
import ./filled
import ./ignored
import ./proving
import ./errored
type
SaleDownloading* = ref object of ErrorHandlingState
availability*: Availability
logScope:
topics = "marketplace sales downloading"
@ -36,9 +36,7 @@ method run*(state: SaleDownloading, machine: Machine): Future[?State] {.async.}
let data = agent.data
let context = agent.context
let reservations = context.reservations
await agent.retrieveRequest()
await agent.subscribe()
let availability = state.availability
without onStore =? context.onStore:
raiseAssert "onStore callback not set"
@ -46,21 +44,8 @@ method run*(state: SaleDownloading, machine: Machine): Future[?State] {.async.}
without request =? data.request:
raiseAssert "no sale request"
debug "New request detected, downloading info", requestId = $data.requestId
without availability =? await reservations.find(
request.ask.slotSize,
request.ask.duration,
request.ask.pricePerSlot,
request.ask.collateral,
used = false):
info "No availability found for request, ignoring",
requestId = $data.requestId,
slotSize = request.ask.slotSize,
duration = request.ask.duration,
pricePerSlot = request.ask.pricePerSlot,
used = false
return some State(SaleIgnored())
without slotIndex =? data.slotIndex:
raiseAssert("no slot index assigned")
# mark availability as used so that it is not matched to other requests
if markUsedErr =? (await reservations.markUsed(availability.id)).errorOption:
@ -86,7 +71,7 @@ method run*(state: SaleDownloading, machine: Machine): Future[?State] {.async.}
trace "Starting download"
if err =? (await onStore(request,
data.slotIndex,
slotIndex,
onBatch)).errorOption:
markUnused(availability.id)

View File

@ -21,11 +21,13 @@ method run*(state: SaleErrored, machine: Machine): Future[?State] {.async.} =
let data = agent.data
let context = agent.context
error "Sale error", error=state.error.msg, requestId = $data.requestId
if onClear =? context.onClear and
request =? data.request and
slotIndex =? data.slotIndex:
onClear(request, slotIndex)
await agent.unsubscribe()
if onCleanUp =? context.onCleanUp:
await onCleanUp()
error "Sale error", error=state.error.msg, requestId = $data.requestId

View File

@ -23,7 +23,10 @@ method run*(state: SaleFilled, machine: Machine): Future[?State] {.async.} =
let data = SalesAgent(machine).data
let market = SalesAgent(machine).context.market
let host = await market.getHost(data.requestId, data.slotIndex)
without slotIndex =? data.slotIndex:
raiseAssert("no slot index assigned")
let host = await market.getHost(data.requestId, slotIndex)
let me = await market.getSigner()
if host == me.some:
return some State(SaleFinished())

View File

@ -32,5 +32,8 @@ method run(state: SaleFilling, machine: Machine): Future[?State] {.async.} =
without (collateral =? data.request.?ask.?collateral):
raiseAssert "Request not set"
debug "Filling slot", requestId = $data.requestId, slot = $data.slotIndex
await market.fillSlot(data.requestId, data.slotIndex, state.proof, collateral)
without slotIndex =? data.slotIndex:
raiseAssert("no slot index assigned")
debug "Filling slot", requestId = $data.requestId, slotIndex
await market.fillSlot(data.requestId, slotIndex, state.proof, collateral)

View File

@ -36,4 +36,5 @@ method run*(state: SaleFinished, machine: Machine): Future[?State] {.async.} =
if onSale =? context.onSale:
onSale(request, slotIndex)
await agent.unsubscribe()
if onCleanUp =? context.onCleanUp:
await onCleanUp()

View File

@ -12,7 +12,5 @@ method run*(state: SaleIgnored, machine: Machine): Future[?State] {.async.} =
let agent = SalesAgent(machine)
let context = agent.context
if onIgnored =? context.onIgnored:
onIgnored()
await agent.unsubscribe()
if onCleanUp =? context.onCleanUp:
await onCleanUp()

View File

@ -0,0 +1,69 @@
import pkg/chronicles
import pkg/questionable
import pkg/questionable/results
import ../../market
import ../salesagent
import ../statemachine
import ./errorhandling
import ./cancelled
import ./failed
import ./filled
import ./ignored
import ./downloading
type
SalePreparing* = ref object of ErrorHandlingState
logScope:
topics = "sales preparing"
method `$`*(state: SalePreparing): string = "SalePreparing"
method onCancelled*(state: SalePreparing, request: StorageRequest): ?State =
return some State(SaleCancelled())
method onFailed*(state: SalePreparing, request: StorageRequest): ?State =
return some State(SaleFailed())
method onSlotFilled*(state: SalePreparing, requestId: RequestId,
slotIndex: UInt256): ?State =
return some State(SaleFilled())
method run*(state: SalePreparing, machine: Machine): Future[?State] {.async.} =
let agent = SalesAgent(machine)
let data = agent.data
let context = agent.context
let market = context.market
let reservations = context.reservations
await agent.retrieveRequest()
await agent.subscribe()
without request =? data.request:
raiseAssert "no sale request"
let slotId = slotId(data.requestId, data.slotIndex)
let state = await market.slotState(slotId)
if state != SlotState.Free:
return some State(SaleIgnored())
# TODO: Once implemented, check to ensure the host is allowed to fill the slot,
# due to the [sliding window mechanism](https://github.com/codex-storage/codex-research/blob/master/design/marketplace.md#dispersal)
# availability was checked for this slot when it entered the queue, however
# check to the ensure that there is still availability as they may have
# changed since being added (other slots may have been processed in that time)
without availability =? await reservations.find(
request.ask.slotSize,
request.ask.duration,
request.ask.pricePerSlot,
request.ask.collateral,
used = false):
info "no availability found for request, ignoring",
slotSize = request.ask.slotSize,
duration = request.ask.duration,
pricePerSlot = request.ask.pricePerSlot,
used = false
return some State(SaleIgnored())
return some State(SaleDownloading(availability: availability))

View File

@ -35,8 +35,11 @@ method run*(state: SaleProving, machine: Machine): Future[?State] {.async.} =
without onProve =? context.proving.onProve:
raiseAssert "onProve callback not set"
without slotIndex =? data.slotIndex:
raiseAssert("no slot index assigned")
debug "Start proof generation", requestId = $data.requestId
let proof = await onProve(Slot(request: request, slotIndex: data.slotIndex))
let proof = await onProve(Slot(request: request, slotIndex: slotIndex))
debug "Finished proof generation", requestId = $data.requestId
return some State(SaleFilling(proof: proof))

View File

@ -27,7 +27,10 @@ method run*(state: SaleUnknown, machine: Machine): Future[?State] {.async.} =
await agent.retrieveRequest()
await agent.subscribe()
let slotId = slotId(data.requestId, data.slotIndex)
without slotIndex =? data.slotIndex:
raiseAssert("no slot index assigned")
let slotId = slotId(data.requestId, slotIndex)
without slotState =? await market.slotState(slotId):
let error = newException(SaleUnknownError, "cannot retrieve slot state")

View File

@ -0,0 +1,46 @@
import std/sugar
import std/tables
import pkg/chronicles
import pkg/chronos
import ../utils/then
type
TrackedFutures* = ref object
futures: Table[uint, FutureBase]
cancelling: bool
logScope:
topics = "trackable futures"
proc track*[T](self: TrackedFutures, fut: Future[T]): Future[T] =
logScope:
id = fut.id
proc removeFuture() =
if not self.cancelling and not fut.isNil:
trace "removing tracked future"
self.futures.del(fut.id)
fut
.then((val: T) => removeFuture())
.catch((e: ref CatchableError) => removeFuture())
trace "tracking future"
self.futures[fut.id] = FutureBase(fut)
return fut
proc track*[T, U](future: Future[T], self: U): Future[T] =
## Convenience method that allows chaining future, eg:
## `await someFut().track(sales)`, where `sales` has declared a
## `trackedFutures` property.
self.trackedFutures.track(future)
proc cancelTracked*(self: TrackedFutures) {.async.} =
self.cancelling = true
for future in self.futures.values:
if not future.isNil and not future.finished:
trace "cancelling tracked future", id = future.id
await future.cancelAndWait()
self.cancelling = false

View File

@ -283,7 +283,7 @@ proc len*[T](heap: AsyncHeapQueue[T]): int {.inline.} =
proc size*[T](heap: AsyncHeapQueue[T]): int {.inline.} =
## Return the maximum number of elements in ``heap``.
len(heap.maxsize)
heap.maxsize
proc `[]`*[T](heap: AsyncHeapQueue[T], i: Natural) : T {.inline.} =
## Access the i-th element of ``heap`` by order from first to last.

226
codex/utils/then.nim Normal file
View File

@ -0,0 +1,226 @@
import pkg/chronos
import pkg/questionable
import pkg/questionable/results
import pkg/upraises
# Similar to JavaScript's Promise API, `.then` and `.catch` can be used to
# handle results and errors of async `Futures` within a synchronous closure.
# They can be used as an alternative to `asyncSpawn` which does not return a
# value and will raise a `FutureDefect` if there are unhandled errors
# encountered. Both `.then` and `.catch` act as callbacks that do not block the
# synchronous closure's flow.
# `.then` is called when the `Future` is successfully completed and can be
# chained as many times as desired, calling each `.then` callback in order. When
# the `Future` returns `Result[T, ref CatchableError]` (or `?!T`), the value
# called in the `.then` callback will be unpacked from the `Result` as a
# convenience. In other words, for `Future[?!T]`, the `.then` callback will take
# a single parameter `T`. See `tests/utils/testthen.nim` for more examples. To
# allow for chaining, `.then` returns its future. If the future is already
# complete, the `.then` callback will be executed immediately.
# `.catch` is called when the `Future` fails. In the case when the `Future`
# returns a `Result[T, ref CatchableError` (or `?!T`), `.catch` will be called
# if the `Result` contains an error. If the `Future` is already failed (or
# `Future[?!T]` contains an error), the `.catch` callback will be excuted
# immediately.
# NOTE: Cancelled `Futures` are discarded as bubbling the `CancelledError` to
# the synchronous closure will likely cause an unintended and unhandled
# exception.
# More info on JavaScript's Promise API can be found at:
# https://developer.mozilla.org/en-US/docs/Web/JavaScript/Reference/Global_Objects/Promise
runnableExamples:
proc asyncProc(): Future[int] {.async.} =
await sleepAsync(1.millis)
return 1
asyncProc()
.then(proc(i: int) = echo "returned ", i)
.catch(proc(e: ref CatchableError) = doAssert false, "will not be triggered")
# outputs "returned 1"
proc asyncProcWithError(): Future[int] {.async.} =
await sleepAsync(1.millis)
raise newException(ValueError, "some error")
asyncProcWithError()
.then(proc(i: int) = doAssert false, "will not be triggered")
.catch(proc(e: ref CatchableError) = echo "errored: ", e.msg)
# outputs "errored: some error"
type
OnSuccess*[T] = proc(val: T) {.gcsafe, upraises: [].}
OnError* = proc(err: ref CatchableError) {.gcsafe, upraises: [].}
proc ignoreError(err: ref CatchableError) = discard
template returnOrError(future: FutureBase, onError: OnError) =
if not future.finished:
return
if future.cancelled:
# do not bubble as closure is synchronous
return
if future.failed:
onError(future.error)
return
proc then*(future: Future[void],
onError: OnError):
Future[void] =
proc cb(udata: pointer) =
future.returnOrError(onError)
proc cancellation(udata: pointer) =
if not future.finished():
future.removeCallback(cb)
future.addCallback(cb)
future.cancelCallback = cancellation
return future
proc then*(future: Future[void],
onSuccess: OnSuccess[void],
onError: OnError = ignoreError):
Future[void] =
proc cb(udata: pointer) =
future.returnOrError(onError)
onSuccess()
proc cancellation(udata: pointer) =
if not future.finished():
future.removeCallback(cb)
future.addCallback(cb)
future.cancelCallback = cancellation
return future
proc then*[T](future: Future[T],
onSuccess: OnSuccess[T],
onError: OnError = ignoreError):
Future[T] =
proc cb(udata: pointer) =
future.returnOrError(onError)
without val =? future.read.catch, err:
onError(err)
return
onSuccess(val)
proc cancellation(udata: pointer) =
if not future.finished():
future.removeCallback(cb)
future.addCallback(cb)
future.cancelCallback = cancellation
return future
proc then*[T](future: Future[?!T],
onSuccess: OnSuccess[T],
onError: OnError = ignoreError):
Future[?!T] =
proc cb(udata: pointer) =
future.returnOrError(onError)
try:
without val =? future.read, err:
onError(err)
return
onSuccess(val)
except CatchableError as e:
onError(e)
proc cancellation(udata: pointer) =
if not future.finished():
future.removeCallback(cb)
future.addCallback(cb)
future.cancelCallback = cancellation
return future
proc then*(future: Future[?!void],
onError: OnError = ignoreError):
Future[?!void] =
proc cb(udata: pointer) =
future.returnOrError(onError)
try:
if err =? future.read.errorOption:
onError(err)
except CatchableError as e:
onError(e)
proc cancellation(udata: pointer) =
if not future.finished():
future.removeCallback(cb)
future.addCallback(cb)
future.cancelCallback = cancellation
return future
proc then*(future: Future[?!void],
onSuccess: OnSuccess[void],
onError: OnError = ignoreError):
Future[?!void] =
proc cb(udata: pointer) =
future.returnOrError(onError)
try:
if err =? future.read.errorOption:
onError(err)
return
except CatchableError as e:
onError(e)
return
onSuccess()
proc cancellation(udata: pointer) =
if not future.finished():
future.removeCallback(cb)
future.addCallback(cb)
future.cancelCallback = cancellation
return future
proc catch*[T](future: Future[T], onError: OnError) =
proc cb(udata: pointer) =
future.returnOrError(onError)
proc cancellation(udata: pointer) =
if not future.finished():
future.removeCallback(cb)
future.addCallback(cb)
future.cancelCallback = cancellation
proc catch*[T](future: Future[?!T], onError: OnError) =
proc cb(udata: pointer) =
future.returnOrError(onError)
try:
if err =? future.read.errorOption:
onError(err)
except CatchableError as e:
onError(e)
proc cancellation(udata: pointer) =
if not future.finished():
future.removeCallback(cb)
future.addCallback(cb)
future.cancelCallback = cancellation

View File

@ -11,3 +11,15 @@ template eventually*(condition: untyped, timeout = 5.seconds): bool =
else:
await sleepAsync(1.millis)
await loop()
template always*(condition: untyped, timeout = 50.millis): bool =
proc loop: Future[bool] {.async.} =
let start = Moment.now()
while true:
if not condition:
return false
if Moment.now() > (start + timeout):
return true
else:
await sleepAsync(1.millis)
await loop()

View File

@ -2,6 +2,8 @@ import std/sequtils
import std/tables
import std/hashes
import std/sets
import std/sugar
import pkg/questionable
import pkg/codex/market
import pkg/codex/contracts/requests
import pkg/codex/contracts/config
@ -53,7 +55,7 @@ type
callback: OnRequest
FulfillmentSubscription* = ref object of Subscription
market: MockMarket
requestId: RequestId
requestId: ?RequestId
callback: OnFulfillment
SlotFilledSubscription* = ref object of Subscription
market: MockMarket
@ -65,11 +67,11 @@ type
callback: OnSlotFreed
RequestCancelledSubscription* = ref object of Subscription
market: MockMarket
requestId: RequestId
requestId: ?RequestId
callback: OnRequestCancelled
RequestFailedSubscription* = ref object of Subscription
market: MockMarket
requestId: RequestId
requestId: ?RequestId
callback: OnRequestCancelled
ProofSubmittedSubscription = ref object of Subscription
market: MockMarket
@ -83,7 +85,7 @@ proc hash*(requestId: RequestId): Hash =
proc new*(_: type MockMarket): MockMarket =
## Create a new mocked Market instance
##
##
let config = MarketplaceConfig(
collateral: CollateralConfig(
repairRewardPercentage: 10,
@ -112,7 +114,9 @@ method requestStorage*(market: MockMarket, request: StorageRequest) {.async.} =
market.requested.add(request)
var subscriptions = market.subscriptions.onRequest
for subscription in subscriptions:
subscription.callback(request.id, request.ask)
subscription.callback(request.id,
request.ask,
request.expiry)
method myRequests*(market: MockMarket): Future[seq[RequestId]] {.async.} =
return market.activeRequests[market.signer]
@ -173,28 +177,32 @@ proc emitSlotFilled*(market: MockMarket,
if requestMatches and slotMatches:
subscription.callback(requestId, slotIndex)
proc emitSlotFreed*(market: MockMarket, slotId: SlotId) =
proc emitSlotFreed*(market: MockMarket,
requestId: RequestId,
slotIndex: UInt256) =
var subscriptions = market.subscriptions.onSlotFreed
for subscription in subscriptions:
subscription.callback(slotId)
subscription.callback(requestId, slotIndex)
proc emitRequestCancelled*(market: MockMarket,
requestId: RequestId) =
proc emitRequestCancelled*(market: MockMarket, requestId: RequestId) =
var subscriptions = market.subscriptions.onRequestCancelled
for subscription in subscriptions:
if subscription.requestId == requestId:
if subscription.requestId == requestId.some or
subscription.requestId.isNone:
subscription.callback(requestId)
proc emitRequestFulfilled*(market: MockMarket, requestId: RequestId) =
var subscriptions = market.subscriptions.onFulfillment
for subscription in subscriptions:
if subscription.requestId == requestId:
if subscription.requestId == requestId.some or
subscription.requestId.isNone:
subscription.callback(requestId)
proc emitRequestFailed*(market: MockMarket, requestId: RequestId) =
var subscriptions = market.subscriptions.onRequestFailed
for subscription in subscriptions:
if subscription.requestId == requestId:
if subscription.requestId == requestId.some or
subscription.requestId.isNone:
subscription.callback(requestId)
proc fillSlot*(market: MockMarket,
@ -221,7 +229,12 @@ method fillSlot*(market: MockMarket,
method freeSlot*(market: MockMarket, slotId: SlotId) {.async.} =
market.freed.add(slotId)
market.emitSlotFreed(slotId)
for s in market.filled:
if slotId(s.requestId, s.slotIndex) == slotId:
market.emitSlotFreed(s.requestId, s.slotIndex)
break
market.slotState[slotId] = SlotState.Free
method withdrawFunds*(market: MockMarket,
requestId: RequestId) {.async.} =
@ -281,13 +294,24 @@ method subscribeRequests*(market: MockMarket,
market.subscriptions.onRequest.add(subscription)
return subscription
method subscribeFulfillment*(market: MockMarket,
callback: OnFulfillment):
Future[Subscription] {.async.} =
let subscription = FulfillmentSubscription(
market: market,
requestId: none RequestId,
callback: callback
)
market.subscriptions.onFulfillment.add(subscription)
return subscription
method subscribeFulfillment*(market: MockMarket,
requestId: RequestId,
callback: OnFulfillment):
Future[Subscription] {.async.} =
let subscription = FulfillmentSubscription(
market: market,
requestId: requestId,
requestId: some requestId,
callback: callback
)
market.subscriptions.onFulfillment.add(subscription)
@ -321,25 +345,47 @@ method subscribeSlotFreed*(market: MockMarket,
market.subscriptions.onSlotFreed.add(subscription)
return subscription
method subscribeRequestCancelled*(market: MockMarket,
callback: OnRequestCancelled):
Future[Subscription] {.async.} =
let subscription = RequestCancelledSubscription(
market: market,
requestId: none RequestId,
callback: callback
)
market.subscriptions.onRequestCancelled.add(subscription)
return subscription
method subscribeRequestCancelled*(market: MockMarket,
requestId: RequestId,
callback: OnRequestCancelled):
Future[Subscription] {.async.} =
let subscription = RequestCancelledSubscription(
market: market,
requestId: requestId,
requestId: some requestId,
callback: callback
)
market.subscriptions.onRequestCancelled.add(subscription)
return subscription
method subscribeRequestFailed*(market: MockMarket,
callback: OnRequestFailed):
Future[Subscription] {.async.} =
let subscription = RequestFailedSubscription(
market: market,
requestId: none RequestId,
callback: callback
)
market.subscriptions.onRequestFailed.add(subscription)
return subscription
method subscribeRequestFailed*(market: MockMarket,
requestId: RequestId,
callback: OnRequestFailed):
Future[Subscription] {.async.} =
let subscription = RequestFailedSubscription(
market: market,
requestId: requestId,
requestId: some requestId,
callback: callback
)
market.subscriptions.onRequestFailed.add(subscription)
@ -355,6 +401,17 @@ method subscribeProofSubmission*(mock: MockMarket,
mock.subscriptions.onProofSubmitted.add(subscription)
return subscription
method queryPastStorageRequests*(market: MockMarket,
blocksAgo: int):
Future[seq[PastStorageRequest]] {.async.} =
# MockMarket does not have the concept of blocks, so simply return all
# previous events
return market.requested.map(request =>
PastStorageRequest(requestId: request.id,
ask: request.ask,
expiry: request.expiry)
)
method unsubscribe*(subscription: RequestSubscription) {.async.} =
subscription.market.subscriptions.onRequest.keepItIf(it != subscription)

View File

@ -0,0 +1,16 @@
import pkg/codex/sales/salesagent
type
MockSalesAgent = ref object of SalesAgent
fulfilledCalled*: bool
failedCalled*: bool
slotFilledCalled*: bool
method onFulfilled*(agent: SalesAgent, requestId: RequestId) =
fulfilledCalled = true
method onFailed*(agent: SalesAgent, requestId: RequestId) =
failedCalled = true
method onSlotFilled*(agent: SalesAgent, requestId: RequestId, slotIndex: UInt256) {.base.} =
slotFilledCalled = true

View File

@ -0,0 +1,29 @@
import std/unittest
import pkg/questionable
import pkg/codex/contracts/requests
import pkg/codex/sales/states/downloading
import pkg/codex/sales/states/cancelled
import pkg/codex/sales/states/failed
import pkg/codex/sales/states/filled
import ../../examples
suite "sales state 'preparing'":
let request = StorageRequest.example
let slotIndex = (request.ask.slots div 2).u256
var state: SalePreparing
setup:
state = SalePreparing.new()
test "switches to cancelled state when request expires":
let next = state.onCancelled(request)
check !next of SaleCancelled
test "switches to failed state when request fails":
let next = state.onFailed(request)
check !next of SaleFailed
test "switches to filled state when slot is filled":
let next = state.onSlotFilled(request.id, slotIndex)
check !next of SaleFilled

View File

@ -11,6 +11,7 @@ import pkg/codex/sales
import pkg/codex/sales/salesdata
import pkg/codex/sales/salescontext
import pkg/codex/sales/reservations
import pkg/codex/sales/slotqueue
import pkg/codex/stores/repostore
import pkg/codex/proving
import pkg/codex/blocktype as bt
@ -32,6 +33,8 @@ asyncchecksuite "Sales":
var proving: Proving
var reservations: Reservations
var repo: RepoStore
var queue: SlotQueue
var itemsProcessed: seq[SlotQueueItem]
setup:
availability = Availability.init(
@ -67,22 +70,122 @@ asyncchecksuite "Sales":
slot: UInt256,
onBatch: BatchProc): Future[?!void] {.async.} =
return success()
queue = sales.context.slotQueue
proving.onProve = proc(slot: Slot): Future[seq[byte]] {.async.} =
return proof
await sales.start()
request.expiry = (clock.now() + 42).u256
itemsProcessed = @[]
teardown:
await repo.stop()
await sales.stop()
await repo.stop()
proc getAvailability: ?!Availability =
waitFor reservations.get(availability.id)
proc wasIgnored: Future[bool] {.async.} =
return
eventually sales.agents.len == 1 and # agent created at first
eventually sales.agents.len == 0 # then removed once ignored
proc notProcessed(itemsProcessed: seq[SlotQueueItem],
request: StorageRequest): bool =
let items = SlotQueueItem.init(request)
for i in 0..<items.len:
if itemsProcessed.contains(items[i]):
return false
return true
proc addRequestToSaturatedQueue(): Future[StorageRequest] {.async.} =
queue.onProcessSlot = proc(item: SlotQueueItem, done: Future[void]) {.async.} =
await sleepAsync(10.millis)
itemsProcessed.add item
done.complete()
var request1 = StorageRequest.example
request1.ask.collateral = request.ask.collateral + 1
discard await reservations.reserve(availability)
await market.requestStorage(request)
await market.requestStorage(request1)
await sleepAsync(5.millis) # wait for request slots to be added to queue
return request1
test "processes all request's slots once StorageRequested emitted":
queue.onProcessSlot = proc(item: SlotQueueItem, done: Future[void]) {.async.} =
itemsProcessed.add item
done.complete()
check isOk await reservations.reserve(availability)
await market.requestStorage(request)
let items = SlotQueueItem.init(request)
check eventually items.allIt(itemsProcessed.contains(it))
test "removes slots from slot queue once RequestCancelled emitted":
let request1 = await addRequestToSaturatedQueue()
market.emitRequestCancelled(request1.id)
check always itemsProcessed.notProcessed(request1)
test "removes request from slot queue once RequestFailed emitted":
let request1 = await addRequestToSaturatedQueue()
market.emitRequestFailed(request1.id)
check always itemsProcessed.notProcessed(request1)
test "removes request from slot queue once RequestFulfilled emitted":
let request1 = await addRequestToSaturatedQueue()
market.emitRequestFulfilled(request1.id)
check always itemsProcessed.notProcessed(request1)
test "removes slot index from slot queue once SlotFilled emitted":
let request1 = await addRequestToSaturatedQueue()
market.emitSlotFilled(request1.id, 1.u256)
let expected = SlotQueueItem.init(request1, 1'u16)
check always (not itemsProcessed.contains(expected))
test "adds slot index to slot queue once SlotFreed emitted":
queue.onProcessSlot = proc(item: SlotQueueItem, done: Future[void]) {.async.} =
itemsProcessed.add item
done.complete()
check isOk await reservations.reserve(availability)
market.requested.add request # "contract" must be able to return request
market.emitSlotFreed(request.id, 2.u256)
let expected = SlotQueueItem.init(request, 2.uint16)
check eventually itemsProcessed.contains(expected)
test "request slots are not added to the slot queue when no availabilities exist":
var itemsProcessed: seq[SlotQueueItem] = @[]
queue.onProcessSlot = proc(item: SlotQueueItem, done: Future[void]) {.async.} =
itemsProcessed.add item
done.complete()
await market.requestStorage(request)
# check that request was ignored due to no matching availability
check always itemsProcessed.len == 0
test "non-matching availabilities/requests are not added to the slot queue":
var itemsProcessed: seq[SlotQueueItem] = @[]
queue.onProcessSlot = proc(item: SlotQueueItem, done: Future[void]) {.async.} =
itemsProcessed.add item
done.complete()
let nonMatchingAvailability = Availability.init(
size=100.u256,
duration=60.u256,
minPrice=601.u256, # too high
maxCollateral=400.u256
)
check isOk await reservations.reserve(nonMatchingAvailability)
await market.requestStorage(request)
# check that request was ignored due to no matching availability
check always itemsProcessed.len == 0
test "adds past requests to queue once availability added":
var itemsProcessed: seq[SlotQueueItem] = @[]
queue.onProcessSlot = proc(item: SlotQueueItem, done: Future[void]) {.async.} =
itemsProcessed.add item
done.complete()
await market.requestStorage(request)
# now add matching availability
check isOk await reservations.reserve(availability)
check eventually itemsProcessed.len == request.ask.slots.int
test "makes storage unavailable when downloading a matched request":
var used = false
@ -115,19 +218,19 @@ asyncchecksuite "Sales":
availability.duration = request.ask.duration - 1
check isOk await reservations.reserve(availability)
await market.requestStorage(request)
check await wasIgnored()
check getAvailability().?size == success availability.size
test "ignores request when slot size is too small":
availability.size = request.ask.slotSize - 1
check isOk await reservations.reserve(availability)
await market.requestStorage(request)
check await wasIgnored()
check getAvailability().?size == success availability.size
test "ignores request when reward is too low":
availability.minPrice = request.ask.pricePerSlot + 1
check isOk await reservations.reserve(availability)
await market.requestStorage(request)
check await wasIgnored()
check getAvailability().?size == success availability.size
test "availability remains unused when request is ignored":
availability.minPrice = request.ask.pricePerSlot + 1
@ -140,7 +243,16 @@ asyncchecksuite "Sales":
tooBigCollateral.ask.collateral = availability.maxCollateral + 1
check isOk await reservations.reserve(availability)
await market.requestStorage(tooBigCollateral)
check await wasIgnored()
check getAvailability().?size == success availability.size
test "ignores request when slot state is not free":
check isOk await reservations.reserve(availability)
await market.requestStorage(request)
market.slotState[request.slotId(0.u256)] = SlotState.Filled
market.slotState[request.slotId(1.u256)] = SlotState.Filled
market.slotState[request.slotId(2.u256)] = SlotState.Filled
market.slotState[request.slotId(3.u256)] = SlotState.Filled
check getAvailability().?size == success availability.size
test "retrieves and stores data locally":
var storingRequest: StorageRequest

View File

@ -1,6 +1,3 @@
import std/sets
import std/sequtils
import std/sugar
import std/times
import pkg/asynctest
import pkg/chronos
@ -25,6 +22,7 @@ type
MockErrorState = ref object of ErrorHandlingState
method `$`*(state: MockState): string = "MockState"
method `$`*(state: MockErrorState): string = "MockErrorState"
method onCancelled*(state: MockState, request: StorageRequest): ?State =
onCancelCalled = true
@ -88,45 +86,26 @@ asyncchecksuite "Sales agent":
await agent.retrieveRequest()
check agent.data.request == some request
test "subscribe assigns subscriptions/futures":
test "subscribe assigns cancelled future":
await agent.subscribe()
check not agent.data.cancelled.isNil
check not agent.data.failed.isNil
check not agent.data.fulfilled.isNil
check not agent.data.slotFilled.isNil
test "unsubscribe deassigns subscriptions/futures":
test "unsubscribe deassigns canceleld future":
await agent.subscribe()
await agent.unsubscribe()
check agent.data.cancelled.isNil
check agent.data.failed.isNil
check agent.data.fulfilled.isNil
check agent.data.slotFilled.isNil
test "subscribe can be called multiple times, without overwriting subscriptions/futures":
await agent.subscribe()
let cancelled = agent.data.cancelled
let failed = agent.data.failed
let fulfilled = agent.data.fulfilled
let slotFilled = agent.data.slotFilled
await agent.subscribe()
check cancelled == agent.data.cancelled
check failed == agent.data.failed
check fulfilled == agent.data.fulfilled
check slotFilled == agent.data.slotFilled
test "unsubscribe can be called multiple times":
await agent.subscribe()
await agent.unsubscribe()
await agent.unsubscribe()
test "subscribe can be called when request expiry has lapsed":
# succeeds when agent.data.fulfilled.isNil
request.expiry = (getTime() - initDuration(seconds=1)).toUnix.u256
agent.data.request = some request
check agent.data.fulfilled.isNil
await agent.subscribe()
test "current state onCancelled called when cancel emitted":
let state = MockState.new()
agent.start(state)
@ -134,22 +113,20 @@ asyncchecksuite "Sales agent":
clock.set(request.expiry.truncate(int64))
check eventually onCancelCalled
test "cancelled future is finished (cancelled) when fulfillment emitted":
test "cancelled future is finished (cancelled) when onFulfilled called":
agent.start(MockState.new())
await agent.subscribe()
market.emitRequestFulfilled(request.id)
agent.onFulfilled(request.id)
check eventually agent.data.cancelled.cancelled()
test "current state onFailed called when failed emitted":
test "current state onFailed called when onFailed called":
agent.start(MockState.new())
await agent.subscribe()
market.emitRequestFailed(request.id)
agent.onFailed(request.id)
check eventually onFailedCalled
test "current state onSlotFilled called when slot filled emitted":
agent.start(MockState.new())
await agent.subscribe()
market.emitSlotFilled(request.id, slotIndex)
agent.onSlotFilled(request.id, slotIndex)
check eventually onSlotFilledCalled
test "ErrorHandlingState.onError can be overridden at the state level":

View File

@ -0,0 +1,450 @@
import std/sequtils
import pkg/asynctest
import pkg/chronicles
import pkg/chronos
import pkg/datastore
import pkg/questionable
import pkg/questionable/results
import pkg/codex/sales/reservations
import pkg/codex/sales/slotqueue
import pkg/codex/stores
import ../helpers/mockmarket
import ../helpers/eventually
import ../examples
suite "Slot queue start/stop":
var repo: RepoStore
var repoDs: Datastore
var metaDs: SQLiteDatastore
var reservations: Reservations
var queue: SlotQueue
setup:
repoDs = SQLiteDatastore.new(Memory).tryGet()
metaDs = SQLiteDatastore.new(Memory).tryGet()
repo = RepoStore.new(repoDs, metaDs)
reservations = Reservations.new(repo)
queue = SlotQueue.new(reservations)
teardown:
await queue.stop()
test "starts out not running":
check not queue.running
test "can call start multiple times, and when already running":
asyncSpawn queue.start()
asyncSpawn queue.start()
check queue.running
test "can call stop when alrady stopped":
await queue.stop()
check not queue.running
test "can call stop when running":
asyncSpawn queue.start()
await queue.stop()
check not queue.running
test "can call stop multiple times":
asyncSpawn queue.start()
await queue.stop()
await queue.stop()
check not queue.running
suite "Slot queue workers":
var repo: RepoStore
var repoDs: Datastore
var metaDs: SQLiteDatastore
var availability: Availability
var reservations: Reservations
var queue: SlotQueue
proc onProcessSlot(item: SlotQueueItem, doneProcessing: Future[void]) {.async.} =
await sleepAsync(1000.millis)
# this is not illustrative of the realistic scenario as the
# `doneProcessing` future would be passed to another context before being
# completed and therefore is not as simple as making the callback async
doneProcessing.complete()
setup:
let request = StorageRequest.example
repoDs = SQLiteDatastore.new(Memory).tryGet()
metaDs = SQLiteDatastore.new(Memory).tryGet()
let quota = request.ask.slotSize.truncate(uint) * 100 + 1
repo = RepoStore.new(repoDs, metaDs, quotaMaxBytes = quota)
reservations = Reservations.new(repo)
# create an availability that should always match
availability = Availability.init(
size = request.ask.slotSize * 100,
duration = request.ask.duration * 100,
minPrice = request.ask.pricePerSlot div 100,
maxCollateral = request.ask.collateral * 100
)
queue = SlotQueue.new(reservations, maxSize = 5, maxWorkers = 3)
queue.onProcessSlot = onProcessSlot
discard await reservations.reserve(availability)
proc startQueue = asyncSpawn queue.start()
teardown:
await queue.stop()
test "activeWorkers should be 0 when not running":
check queue.activeWorkers == 0
test "maxWorkers cannot be 0":
expect ValueError:
discard SlotQueue.new(reservations, maxSize = 1, maxWorkers = 0)
test "maxWorkers cannot surpass maxSize":
expect ValueError:
discard SlotQueue.new(reservations, maxSize = 1, maxWorkers = 2)
test "does not surpass max workers":
startQueue()
let item1 = SlotQueueItem.example
let item2 = SlotQueueItem.example
let item3 = SlotQueueItem.example
let item4 = SlotQueueItem.example
check (await queue.push(item1)).isOk
check (await queue.push(item2)).isOk
check (await queue.push(item3)).isOk
check (await queue.push(item4)).isOk
check eventually queue.activeWorkers == 3
test "discards workers once processing completed":
proc processSlot(item: SlotQueueItem, done: Future[void]) {.async.} =
await sleepAsync(1.millis)
done.complete()
queue.onProcessSlot = processSlot
startQueue()
let item1 = SlotQueueItem.example
let item2 = SlotQueueItem.example
let item3 = SlotQueueItem.example
let item4 = SlotQueueItem.example
check (await queue.push(item1)).isOk # finishes after 1.millis
check (await queue.push(item2)).isOk # finishes after 1.millis
check (await queue.push(item3)).isOk # finishes after 1.millis
check (await queue.push(item4)).isOk
check eventually queue.activeWorkers == 1
suite "Slot queue":
var onProcessSlotCalled = false
var onProcessSlotCalledWith: seq[(RequestId, uint16)]
var repo: RepoStore
var repoDs: Datastore
var metaDs: SQLiteDatastore
var availability: Availability
var reservations: Reservations
var queue: SlotQueue
let maxWorkers = 2
var unpauseQueue: Future[void]
var paused: bool
proc newSlotQueue(maxSize, maxWorkers: int, processSlotDelay = 1.millis) =
queue = SlotQueue.new(reservations, maxWorkers, maxSize.uint16)
queue.onProcessSlot = proc(item: SlotQueueItem, done: Future[void]) {.async.} =
await sleepAsync(processSlotDelay)
trace "processing item", requestId = item.requestId, slotIndex = item.slotIndex
onProcessSlotCalled = true
onProcessSlotCalledWith.add (item.requestId, item.slotIndex)
done.complete()
asyncSpawn queue.start()
setup:
onProcessSlotCalled = false
onProcessSlotCalledWith = @[]
let request = StorageRequest.example
repoDs = SQLiteDatastore.new(Memory).tryGet()
metaDs = SQLiteDatastore.new(Memory).tryGet()
let quota = request.ask.slotSize.truncate(uint) * 100 + 1
repo = RepoStore.new(repoDs, metaDs, quotaMaxBytes = quota)
reservations = Reservations.new(repo)
# create an availability that should always match
availability = Availability.init(
size = request.ask.slotSize * 100,
duration = request.ask.duration * 100,
minPrice = request.ask.pricePerSlot div 100,
maxCollateral = request.ask.collateral * 100
)
discard await reservations.reserve(availability)
teardown:
paused = false
await queue.stop()
test "starts out empty":
newSlotQueue(maxSize = 2, maxWorkers = 2)
check queue.len == 0
check $queue == "[]"
test "reports correct size":
newSlotQueue(maxSize = 2, maxWorkers = 2)
check queue.size == 2
test "correctly compares SlotQueueItems":
var requestA = StorageRequest.example
requestA.ask.duration = 1.u256
requestA.ask.reward = 1.u256
check requestA.ask.pricePerSlot == 1.u256
requestA.ask.collateral = 100000.u256
requestA.expiry = 1001.u256
var requestB = StorageRequest.example
requestB.ask.duration = 100.u256
requestB.ask.reward = 1000.u256
check requestB.ask.pricePerSlot == 100000.u256
requestB.ask.collateral = 1.u256
requestB.expiry = 1000.u256
let itemA = SlotQueueItem.init(requestA, 0)
let itemB = SlotQueueItem.init(requestB, 0)
check itemB < itemA # B higher priority than A
check itemA > itemB
test "expands available all possible slot indices on init":
let request = StorageRequest.example
let items = SlotQueueItem.init(request)
check items.len.uint64 == request.ask.slots
var checked = 0
for slotIndex in 0'u16..<request.ask.slots.uint16:
check items.anyIt(it == SlotQueueItem.init(request, slotIndex))
inc checked
check checked == items.len
test "can process items":
newSlotQueue(maxSize = 2, maxWorkers = 2)
let item1 = SlotQueueItem.example
let item2 = SlotQueueItem.example
check (await queue.push(item1)).isOk
check (await queue.push(item2)).isOk
check eventually onProcessSlotCalledWith == @[
(item1.requestId, item1.slotIndex),
(item2.requestId, item2.slotIndex)
]
test "can push items past number of maxWorkers":
newSlotQueue(maxSize = 2, maxWorkers = 2)
let item0 = SlotQueueItem.example
let item1 = SlotQueueItem.example
let item2 = SlotQueueItem.example
let item3 = SlotQueueItem.example
let item4 = SlotQueueItem.example
check isOk (await queue.push(item0))
check isOk (await queue.push(item1))
check isOk (await queue.push(item2))
check isOk (await queue.push(item3))
check isOk (await queue.push(item4))
test "populates item with exisiting request metadata":
newSlotQueue(maxSize = 8, maxWorkers = 1, processSlotDelay = 10.millis)
let request0 = StorageRequest.example
var request1 = StorageRequest.example
request1.ask.collateral += 1.u256
let items0 = SlotQueueItem.init(request0)
let items1 = SlotQueueItem.init(request1)
check (await queue.push(items0)).isOk
check (await queue.push(items1)).isOk
let populated = !queue.populateItem(request1.id, 12'u16)
check populated.requestId == request1.id
check populated.slotIndex == 12'u16
check populated.slotSize == request1.ask.slotSize
check populated.duration == request1.ask.duration
check populated.reward == request1.ask.reward
check populated.collateral == request1.ask.collateral
test "does not find exisiting request metadata":
newSlotQueue(maxSize = 2, maxWorkers = 2)
let item = SlotQueueItem.example
check queue.populateItem(item.requestId, 12'u16).isNone
test "can support uint16.high slots":
var request = StorageRequest.example
let maxUInt16 = uint16.high
let uint64Slots = uint64(maxUInt16)
request.ask.slots = uint64Slots
let items = SlotQueueItem.init(request.id, request.ask, request.expiry)
check items.len.uint16 == maxUInt16
test "cannot support greater than uint16.high slots":
var request = StorageRequest.example
let int32Slots = uint16.high.int32 + 1
let uint64Slots = uint64(int32Slots)
request.ask.slots = uint64Slots
expect SlotsOutOfRangeError:
discard SlotQueueItem.init(request.id, request.ask, request.expiry)
test "cannot push duplicate items":
newSlotQueue(maxSize = 6, maxWorkers = 1, processSlotDelay = 15.millis)
let item0 = SlotQueueItem.example
let item1 = SlotQueueItem.example
let item2 = SlotQueueItem.example
check isOk (await queue.push(item0))
check isOk (await queue.push(item1))
check (await queue.push(@[item2, item2, item2, item2])).error of SlotQueueItemExistsError
test "can add items past max maxSize":
newSlotQueue(maxSize = 4, maxWorkers = 2, processSlotDelay = 10.millis)
let item1 = SlotQueueItem.example
let item2 = SlotQueueItem.example
let item3 = SlotQueueItem.example
let item4 = SlotQueueItem.example
check (await queue.push(item1)).isOk
check (await queue.push(item2)).isOk
check (await queue.push(item3)).isOk
check (await queue.push(item4)).isOk
check eventually onProcessSlotCalledWith.len == 4
test "can delete items":
newSlotQueue(maxSize = 6, maxWorkers = 2, processSlotDelay = 10.millis)
let item0 = SlotQueueItem.example
let item1 = SlotQueueItem.example
let item2 = SlotQueueItem.example
let item3 = SlotQueueItem.example
check (await queue.push(item0)).isOk
check (await queue.push(item1)).isOk
check (await queue.push(item2)).isOk
check (await queue.push(item3)).isOk
queue.delete(item3)
check not queue.contains(item3)
test "can delete item by request id and slot id":
newSlotQueue(maxSize = 8, maxWorkers = 1, processSlotDelay = 10.millis)
let request0 = StorageRequest.example
var request1 = StorageRequest.example
request1.ask.collateral += 1.u256
let items0 = SlotQueueItem.init(request0)
let items1 = SlotQueueItem.init(request1)
check (await queue.push(items0)).isOk
check (await queue.push(items1)).isOk
let last = items1[items1.high]
check eventually queue.contains(last)
queue.delete(last.requestId, last.slotIndex)
check not onProcessSlotCalledWith.anyIt(
it == (last.requestId, last.slotIndex)
)
test "can delete all items by request id":
newSlotQueue(maxSize = 8, maxWorkers = 1, processSlotDelay = 10.millis)
let request0 = StorageRequest.example
var request1 = StorageRequest.example
request1.ask.collateral += 1.u256
let items0 = SlotQueueItem.init(request0)
let items1 = SlotQueueItem.init(request1)
check (await queue.push(items0)).isOk
check (await queue.push(items1)).isOk
queue.delete(request1.id)
check not onProcessSlotCalledWith.anyIt(it[0] == request1.id)
test "can check if contains item":
newSlotQueue(maxSize = 6, maxWorkers = 1, processSlotDelay = 10.millis)
let request0 = StorageRequest.example
var request1 = StorageRequest.example
var request2 = StorageRequest.example
var request3 = StorageRequest.example
var request4 = StorageRequest.example
var request5 = StorageRequest.example
request1.ask.collateral = request0.ask.collateral + 1
request2.ask.collateral = request1.ask.collateral + 1
request3.ask.collateral = request2.ask.collateral + 1
request4.ask.collateral = request3.ask.collateral + 1
request5.ask.collateral = request4.ask.collateral + 1
let item0 = SlotQueueItem.init(request0, 0)
let item1 = SlotQueueItem.init(request1, 0)
let item2 = SlotQueueItem.init(request2, 0)
let item3 = SlotQueueItem.init(request3, 0)
let item4 = SlotQueueItem.init(request4, 0)
let item5 = SlotQueueItem.init(request5, 0)
check queue.contains(item5) == false
check (await queue.push(@[item0, item1, item2, item3, item4, item5])).isOk
check queue.contains(item5)
test "sorts items by profitability ascending (higher pricePerSlot = higher priority)":
var request = StorageRequest.example
let item0 = SlotQueueItem.init(request, 0)
request.ask.reward += 1.u256
let item1 = SlotQueueItem.init(request, 1)
check item1 < item0
test "sorts items by collateral ascending (less required collateral = higher priority)":
var request = StorageRequest.example
let item0 = SlotQueueItem.init(request, 0)
request.ask.collateral -= 1.u256
let item1 = SlotQueueItem.init(request, 1)
check item1 < item0
test "sorts items by expiry descending (longer expiry = higher priority)":
var request = StorageRequest.example
let item0 = SlotQueueItem.init(request, 0)
request.expiry += 1.u256
let item1 = SlotQueueItem.init(request, 1)
check item1 < item0
test "sorts items by slot size ascending (smaller dataset = higher priority)":
var request = StorageRequest.example
let item0 = SlotQueueItem.init(request, 0)
request.ask.slotSize -= 1.u256
let item1 = SlotQueueItem.init(request, 1)
check item1 < item0
test "should call callback once an item is added":
newSlotQueue(maxSize = 2, maxWorkers = 2)
let item = SlotQueueItem.example
check not onProcessSlotCalled
check (await queue.push(item)).isOk
check eventually onProcessSlotCalled
test "should only process item once":
newSlotQueue(maxSize = 2, maxWorkers = 2)
let item = SlotQueueItem.example
check (await queue.push(item)).isOk
check eventually onProcessSlotCalledWith == @[
(item.requestId, item.slotIndex)
]
test "should process items in correct order":
newSlotQueue(maxSize = 2, maxWorkers = 2)
# sleeping after push allows the slotqueue loop to iterate,
# calling the callback for each pushed/updated item
var request = StorageRequest.example
let item0 = SlotQueueItem.init(request, 0)
request.ask.reward += 1.u256
let item1 = SlotQueueItem.init(request, 1)
request.ask.reward += 1.u256
let item2 = SlotQueueItem.init(request, 2)
request.ask.reward += 1.u256
let item3 = SlotQueueItem.init(request, 3)
check (await queue.push(item0)).isOk
await sleepAsync(1.millis)
check (await queue.push(item1)).isOk
await sleepAsync(1.millis)
check (await queue.push(item2)).isOk
await sleepAsync(1.millis)
check (await queue.push(item3)).isOk
check eventually (
onProcessSlotCalledWith == @[
(item0.requestId, item0.slotIndex),
(item1.requestId, item1.slotIndex),
(item2.requestId, item2.slotIndex),
(item3.requestId, item3.slotIndex),
]
)
test "fails to push when there's no matching availability":
newSlotQueue(maxSize = 2, maxWorkers = 2)
discard await reservations.release(availability.id,
availability.size.truncate(uint))
let item = SlotQueueItem.example
check (await queue.push(item)).error of NoMatchingAvailabilityError

View File

@ -1,5 +1,6 @@
import ./sales/testsales
import ./sales/teststates
import ./sales/testreservations
import ./sales/testslotqueue
{.warning[UnusedImport]: off.}

View File

@ -2,5 +2,6 @@ import ./utils/testoptionalcast
import ./utils/testkeyutils
import ./utils/testasyncstatemachine
import ./utils/testtimer
import ./utils/testthen
{.warning[UnusedImport]: off.}

View File

@ -0,0 +1,344 @@
import pkg/asynctest
import pkg/chronos
import pkg/questionable
import pkg/questionable/results
import codex/utils/then
import ../helpers
asyncchecksuite "then - Future[void]":
var returnsVoidWasRun: bool
var error = (ref CatchableError)(msg: "some error")
setup:
returnsVoidWasRun = false
proc returnsVoid() {.async.} =
await sleepAsync 1.millis
returnsVoidWasRun = true
proc returnsVoidError() {.async.} =
raise error
proc returnsVoidCancelled() {.async.} =
await sleepAsync(1.seconds)
proc wasCancelled(error: ref CancelledError): bool =
not error.isNil and error.msg == "Future operation cancelled!"
test "calls async proc when returns Future[void]":
discard returnsVoid().then(
proc(err: ref CatchableError) = discard
)
check eventually returnsVoidWasRun
test "calls onSuccess when Future[void] complete":
var onSuccessCalled = false
discard returnsVoid().then(
proc() = onSuccessCalled = true,
proc(err: ref CatchableError) = discard
)
check eventually returnsVoidWasRun
check eventually onSuccessCalled
test "can pass only onSuccess for Future[void]":
var onSuccessCalled = false
discard returnsVoid().then(
proc() = onSuccessCalled = true
)
check eventually returnsVoidWasRun
check eventually onSuccessCalled
test "can chain onSuccess when Future[void] complete":
var onSuccessCalledTimes = 0
discard returnsVoid()
.then(proc() = inc onSuccessCalledTimes)
.then(proc() = inc onSuccessCalledTimes)
.then(proc() = inc onSuccessCalledTimes)
check eventually onSuccessCalledTimes == 3
test "calls onError when Future[void] fails":
var errorActual: ref CatchableError
discard returnsVoidError().then(
proc() = discard,
proc(e: ref CatchableError) = errorActual = e
)
check eventually error == errorActual
test "calls onError when Future[void] fails":
var errorActual: ref CatchableError
discard returnsVoidError().then(
proc(e: ref CatchableError) = errorActual = e
)
check eventually error == errorActual
test "catch callback fired when Future[void] fails":
var errorActual: ref CatchableError
returnsVoidError().catch(
proc(e: ref CatchableError) = errorActual = e
)
check eventually error == errorActual
test "does not fire onSuccess callback when Future[void] fails":
var onSuccessCalled = false
returnsVoidError()
.then(proc() = onSuccessCalled = true)
.then(proc() = onSuccessCalled = true)
.catch(proc(e: ref CatchableError) = discard)
check always (not onSuccessCalled)
asyncchecksuite "then - Future[T]":
var returnsValWasRun: bool
var error = (ref CatchableError)(msg: "some error")
setup:
returnsValWasRun = false
proc returnsVal(): Future[int] {.async.} =
await sleepAsync 1.millis
returnsValWasRun = true
return 1
proc returnsValError(): Future[int] {.async.} =
raise error
proc returnsValCancelled(): Future[int] {.async.} =
await sleepAsync(1.seconds)
proc wasCancelled(error: ref CancelledError): bool =
not error.isNil and error.msg == "Future operation cancelled!"
test "calls onSuccess when Future[T] complete":
var returnedVal = 0
discard returnsVal().then(
proc(val: int) = returnedVal = val,
proc(err: ref CatchableError) = discard
)
check eventually returnsValWasRun
check eventually returnedVal == 1
test "can pass only onSuccess for Future[T]":
var returnedVal = 0
discard returnsVal().then(
proc(val: int) = returnedVal = val
)
check eventually returnsValWasRun
check eventually returnedVal == 1
test "can chain onSuccess when Future[T] complete":
var onSuccessCalledWith: seq[int] = @[]
discard returnsVal()
.then(proc(val: int) = onSuccessCalledWith.add(val))
.then(proc(val: int) = onSuccessCalledWith.add(val))
.then(proc(val: int) = onSuccessCalledWith.add(val))
check eventually onSuccessCalledWith == @[1, 1, 1]
test "calls onError when Future[T] fails":
var errorActual: ref CatchableError
discard returnsValError().then(
proc(val: int) = discard,
proc(e: ref CatchableError) = errorActual = e
)
check eventually error == errorActual
test "catch callback fired when Future[T] fails":
var errorActual: ref CatchableError
returnsValError().catch(
proc(e: ref CatchableError) = errorActual = e
)
check eventually error == errorActual
test "does not fire onSuccess callback when Future[T] fails":
var onSuccessCalled = false
returnsValError()
.then(proc(val: int) = onSuccessCalled = true)
.then(proc(val: int) = onSuccessCalled = true)
.catch(proc(e: ref CatchableError) = discard)
check always (not onSuccessCalled)
asyncchecksuite "then - Future[?!void]":
var returnsResultVoidWasRun: bool
var error = (ref CatchableError)(msg: "some error")
setup:
returnsResultVoidWasRun = false
proc returnsResultVoid(): Future[?!void] {.async.} =
await sleepAsync 1.millis
returnsResultVoidWasRun = true
return success()
proc returnsResultVoidError(): Future[?!void] {.async.} =
return failure(error)
proc returnsResultVoidErrorUncaught(): Future[?!void] {.async.} =
raise error
proc returnsResultVoidCancelled(): Future[?!void] {.async.} =
await sleepAsync(1.seconds)
return success()
proc wasCancelled(error: ref CancelledError): bool =
not error.isNil and error.msg == "Future operation cancelled!"
test "calls onSuccess when Future[?!void] complete":
var onSuccessCalled = false
discard returnsResultVoid().then(
proc() = onSuccessCalled = true,
proc(err: ref CatchableError) = discard
)
check eventually returnsResultVoidWasRun
check eventually onSuccessCalled
test "can pass only onSuccess for Future[?!void]":
var onSuccessCalled = false
discard returnsResultVoid().then(
proc() = onSuccessCalled = true
)
check eventually returnsResultVoidWasRun
check eventually onSuccessCalled
test "can chain onSuccess when Future[?!void] complete":
var onSuccessCalledTimes = 0
discard returnsResultVoid()
.then(proc() = inc onSuccessCalledTimes)
.then(proc() = inc onSuccessCalledTimes)
.then(proc() = inc onSuccessCalledTimes)
check eventually onSuccessCalledTimes == 3
test "calls onError when Future[?!void] fails":
var errorActual: ref CatchableError
discard returnsResultVoidError().then(
proc() = discard,
proc(e: ref CatchableError) = errorActual = e
)
await sleepAsync(10.millis)
check eventually error == errorActual
test "calls onError when Future[?!void] fails":
var errorActual: ref CatchableError
discard returnsResultVoidError().then(
proc(e: ref CatchableError) = errorActual = e
)
check eventually error == errorActual
test "catch callback fired when Future[?!void] fails":
var errorActual: ref CatchableError
returnsResultVoidError().catch(
proc(e: ref CatchableError) = errorActual = e
)
check eventually error == errorActual
test "does not fire onSuccess callback when Future[?!void] fails":
var onSuccessCalled = false
returnsResultVoidError()
.then(proc() = onSuccessCalled = true)
.then(proc() = onSuccessCalled = true)
.catch(proc(e: ref CatchableError) = discard)
check always (not onSuccessCalled)
test "catch callback fired when Future[?!void] fails with uncaught error":
var errorActual: ref CatchableError
returnsResultVoidErrorUncaught().catch(
proc(e: ref CatchableError) = errorActual = e
)
check eventually error == errorActual
asyncchecksuite "then - Future[?!T]":
var returnsResultValWasRun: bool
var error = (ref CatchableError)(msg: "some error")
setup:
returnsResultValWasRun = false
proc returnsResultVal(): Future[?!int] {.async.} =
await sleepAsync 1.millis
returnsResultValWasRun = true
return success(2)
proc returnsResultValError(): Future[?!int] {.async.} =
return failure(error)
proc returnsResultValErrorUncaught(): Future[?!int] {.async.} =
raise error
proc returnsResultValCancelled(): Future[?!int] {.async.} =
await sleepAsync(1.seconds)
return success(3)
proc wasCancelled(error: ref CancelledError): bool =
not error.isNil and error.msg == "Future operation cancelled!"
test "calls onSuccess when Future[?!T] completes":
var actualVal = 0
discard returnsResultVal().then(
proc(val: int) = actualVal = val,
proc(err: ref CatchableError) = discard
)
check eventually returnsResultValWasRun
check eventually actualVal == 2
test "can pass only onSuccess for Future[?!T]":
var actualVal = 0
discard returnsResultVal().then(
proc(val: int) = actualVal = val
)
check eventually returnsResultValWasRun
check eventually actualVal == 2
test "can chain onSuccess when Future[?!T] complete":
var onSuccessCalledWith: seq[int] = @[]
discard returnsResultVal()
.then(proc(val: int) = onSuccessCalledWith.add val)
.then(proc(val: int) = onSuccessCalledWith.add val)
.then(proc(val: int) = onSuccessCalledWith.add val)
check eventually onSuccessCalledWith == @[2, 2, 2]
test "calls onError when Future[?!T] fails":
var errorActual: ref CatchableError
discard returnsResultValError().then(
proc(val: int) = discard,
proc(e: ref CatchableError) = errorActual = e
)
check eventually error == errorActual
test "calls onError when Future[?!T] fails":
var errorActual: ref CatchableError
discard returnsResultValError().then(
proc(val: int) = discard,
proc(e: ref CatchableError) = errorActual = e
)
check eventually error == errorActual
test "catch callback fired when Future[?!T] fails":
var errorActual: ref CatchableError
returnsResultValError().catch(
proc(e: ref CatchableError) = errorActual = e
)
check eventually error == errorActual
test "does not fire onSuccess callback when Future[?!T] fails":
var onSuccessCalled = false
returnsResultValError()
.then(proc(val: int) = onSuccessCalled = true)
.then(proc(val: int) = onSuccessCalled = true)
.catch(proc(e: ref CatchableError) = discard)
check always (not onSuccessCalled)
test "catch callback fired when Future[?!T] fails with uncaught error":
var errorActual: ref CatchableError
returnsResultValErrorUncaught()
.then(proc(val: int) = discard)
.then(proc(val: int) = discard)
.catch(proc(e: ref CatchableError) = errorActual = e)
check eventually error == errorActual

View File

@ -38,10 +38,10 @@ ethersuite "Marketplace contracts":
request.client = await client.getAddress()
switchAccount(client)
await token.approve(marketplace.address, request.price)
discard await token.approve(marketplace.address, request.price)
await marketplace.requestStorage(request)
switchAccount(host)
await token.approve(marketplace.address, request.ask.collateral)
discard await token.approve(marketplace.address, request.ask.collateral)
await marketplace.fillSlot(request.id, 0.u256, proof)
slotId = request.slotId(0.u256)
@ -56,7 +56,7 @@ ethersuite "Marketplace contracts":
proc startContract() {.async.} =
for slotIndex in 1..<request.ask.slots:
await token.approve(marketplace.address, request.ask.collateral)
discard await token.approve(marketplace.address, request.ask.collateral)
await marketplace.fillSlot(request.id, slotIndex.u256, proof)
test "accept marketplace proofs":

View File

@ -2,6 +2,7 @@ import std/options
import pkg/chronos
import pkg/stew/byteutils
import codex/contracts
import ../codex/helpers/eventually
import ../ethertest
import ./examples
import ./time
@ -76,13 +77,16 @@ ethersuite "On-Chain Market":
test "supports request subscriptions":
var receivedIds: seq[RequestId]
var receivedAsks: seq[StorageAsk]
proc onRequest(id: RequestId, ask: StorageAsk) =
var receivedExpirys: seq[UInt256]
proc onRequest(id: RequestId, ask: StorageAsk, expiry: UInt256) =
receivedIds.add(id)
receivedAsks.add(ask)
receivedExpirys.add(expiry)
let subscription = await market.subscribeRequests(onRequest)
await market.requestStorage(request)
check receivedIds == @[request.id]
check receivedAsks == @[request.ask]
check receivedExpirys == @[request.expiry]
await subscription.unsubscribe()
test "supports filling of slots":
@ -158,12 +162,15 @@ ethersuite "On-Chain Market":
test "supports slot freed subscriptions":
await market.requestStorage(request)
await market.fillSlot(request.id, slotIndex, proof, request.ask.collateral)
var receivedIds: seq[SlotId]
proc onSlotFreed(id: SlotId) =
receivedIds.add(id)
var receivedRequestIds: seq[RequestId] = @[]
var receivedIdxs: seq[UInt256] = @[]
proc onSlotFreed(requestId: RequestId, idx: UInt256) =
receivedRequestIds.add(requestId)
receivedIdxs.add(idx)
let subscription = await market.subscribeSlotFreed(onSlotFreed)
await market.freeSlot(slotId(request.id, slotIndex))
check receivedIds == @[slotId(request.id, slotIndex)]
check receivedRequestIds == @[request.id]
check receivedIdxs == @[slotIndex]
await subscription.unsubscribe()
test "support fulfillment subscriptions":
@ -318,3 +325,34 @@ ethersuite "On-Chain Market":
await market.fillSlot(request.id, slotIndex, proof, request.ask.collateral)
let slotId = request.slotId(slotIndex)
check (await market.slotState(slotId)) == SlotState.Filled
test "can query past events":
var request1 = StorageRequest.example
var request2 = StorageRequest.example
request1.client = accounts[0]
request2.client = accounts[0]
await market.requestStorage(request)
await market.requestStorage(request1)
await market.requestStorage(request2)
# `market.requestStorage` executes an `approve` tx before the
# `requestStorage` tx, so that's two PoA blocks per `requestStorage` call (6
# blocks for 3 calls). `fromBlock` and `toBlock` are inclusive, so to check
# 6 blocks, we only need to check 5 "blocks ago". We don't need to check the
# `approve` for the first `requestStorage` call, so that's 1 less again = 4
# "blocks ago".
check eventually (
(await market.queryPastStorageRequests(5)) ==
@[
PastStorageRequest(requestId: request.id, ask: request.ask, expiry: request.expiry),
PastStorageRequest(requestId: request1.id, ask: request1.ask, expiry: request1.expiry),
PastStorageRequest(requestId: request2.id, ask: request2.ask, expiry: request2.expiry)
])
test "past event query can specify negative `blocksAgo` parameter":
await market.requestStorage(request)
check eventually (
(await market.queryPastStorageRequests(blocksAgo = -2)) ==
(await market.queryPastStorageRequests(blocksAgo = 2))
)

View File

@ -2,6 +2,7 @@ import std/random
import std/sequtils
import std/times
import pkg/codex/contracts/requests
import pkg/codex/sales/slotqueue
import pkg/stint
proc example*[T: SomeInteger](_: type T): T =
@ -53,6 +54,11 @@ proc example*(_: type Slot): Slot =
let slotIndex = rand(request.ask.slots.int).u256
Slot(request: request, slotIndex: slotIndex)
proc example*(_: type SlotQueueItem): SlotQueueItem =
let request = StorageRequest.example
let slot = Slot.example
SlotQueueItem.init(request, slot.slotIndex.truncate(uint16))
proc exampleProof*(): seq[byte] =
var proof: seq[byte]
while proof.len == 0:

View File

@ -177,7 +177,7 @@ multinodesuite "Simulate invalid proofs",
var slotWasFreed = false
proc onSlotFreed(event: SlotFreed) =
if event.slotId == slotId:
if slotId(event.requestId, event.slotIndex) == slotId:
slotWasFreed = true
let subscription = await marketplace.subscribe(SlotFreed, onSlotFreed)
@ -201,7 +201,7 @@ multinodesuite "Simulate invalid proofs",
var slotWasFreed = false
proc onSlotFreed(event: SlotFreed) =
if event.slotId == slotId:
if slotId(event.requestId, event.slotIndex) == slotId:
slotWasFreed = true
let subscription = await marketplace.subscribe(SlotFreed, onSlotFreed)

@ -1 +1 @@
Subproject commit 30affa0da85985f6dc90b62f6293de46a9e26130
Subproject commit 230e7276e271ce53bce36fffdbb25a50621c33b9

2
vendor/nim-ethers vendored

@ -1 +1 @@
Subproject commit 0321e6d7bd9c703c9e9bf31ee8664adac1d6cbe7
Subproject commit 9f4f762e21b433aa31549964d723f47d45da7990

2
vendor/nim-json-rpc vendored

@ -1 +1 @@
Subproject commit 5a281760803907f4989cacf109b516381dfbbe11
Subproject commit 0bf2bcbe74a18a3c7a709d57108bb7b51e748a92