feat: introduce blockstore manager
Implement blockstore manager which executes block storage operations on its block stores, in the order to which they were added to the manager, typically in the order of most local (fastest, eg cache) to least local (slowest, eg filesystem or perhaps a network filesystem). As an example, given a `BlockStoreManager` instantiated with a `@[MemoryStore, FSStore]`, retrieving a block would first attempt to get from the `MemoryStore`, and if not found, attempt to get from the `FSStore`. Remove all dependencies on `BlockStores` (typically in the shape of `localstore`) and instead depend on `BlockStoreManager` via the `BlockExcEngine`. Modify the role of the `BlockExcEngine` to make a “local vs remote” decision on block access/storage. For all operations other than retrieving blocks, this means simply going to the `BlockStoreManager`. For retrieving blocks, however, this means going first to the `BlockStoreManager`, and then if not found, going to the Dagger network (via pending block and want/have lists). Remove `NetworkStore` as its two purposes were to defer block retrieval from a local store first, then go to the block exchange to requeest a block from the Dagger network. `BlockStoreManager` takes care of going to local storage first, and the block exchange engine handles going to Dagger network if retrieval from the store manager fails. ### Notes 1. Future work may want to consider breaking up `BlockExcEngine` further in to three modules: - `BlockExcEngine` (depends on `WantHave`, `DHT`) - `WantHave` - `DHT` (work is in progress) Co-authored-by: Michael Bradley <michaelsbradleyjr@gmail.com>
This commit is contained in:
parent
ec66e42e73
commit
4a70312ee9
|
@ -12,8 +12,11 @@ import std/sequtils
|
||||||
import pkg/chronos
|
import pkg/chronos
|
||||||
import pkg/chronicles
|
import pkg/chronicles
|
||||||
import pkg/libp2p
|
import pkg/libp2p
|
||||||
|
import pkg/questionable
|
||||||
|
import pkg/questionable/results
|
||||||
|
|
||||||
import ../stores/blockstore
|
import ../stores/blockstore
|
||||||
|
import ../stores/manager
|
||||||
import ../blocktype as bt
|
import ../blocktype as bt
|
||||||
import ../utils/asyncheapqueue
|
import ../utils/asyncheapqueue
|
||||||
|
|
||||||
|
@ -25,7 +28,7 @@ import ./pendingblocks
|
||||||
import ./peercontext
|
import ./peercontext
|
||||||
import ./engine/payments
|
import ./engine/payments
|
||||||
|
|
||||||
export peercontext, payments, pendingblocks
|
export asyncheapqueue, peercontext, payments, pendingblocks
|
||||||
|
|
||||||
logScope:
|
logScope:
|
||||||
topics = "dagger blockexc engine"
|
topics = "dagger blockexc engine"
|
||||||
|
@ -42,7 +45,7 @@ type
|
||||||
TaskScheduler* = proc(task: BlockExcPeerCtx): bool {.gcsafe.}
|
TaskScheduler* = proc(task: BlockExcPeerCtx): bool {.gcsafe.}
|
||||||
|
|
||||||
BlockExcEngine* = ref object of RootObj
|
BlockExcEngine* = ref object of RootObj
|
||||||
localStore*: BlockStore # where we localStore blocks for this instance
|
blockStoreMgr: BlockStoreManager # manages storing/accessing blocks for this instance
|
||||||
network*: BlockExcNetwork # network interface
|
network*: BlockExcNetwork # network interface
|
||||||
peers*: seq[BlockExcPeerCtx] # peers we're currently actively exchanging with
|
peers*: seq[BlockExcPeerCtx] # peers we're currently actively exchanging with
|
||||||
wantList*: seq[Cid] # local wants list
|
wantList*: seq[Cid] # local wants list
|
||||||
|
@ -98,7 +101,7 @@ proc stop*(b: BlockExcEngine) {.async.} =
|
||||||
## Stop the blockexc blockexc
|
## Stop the blockexc blockexc
|
||||||
##
|
##
|
||||||
|
|
||||||
trace "NetworkStore stop"
|
trace "Block Exchange Engine stop"
|
||||||
if not b.blockexcRunning:
|
if not b.blockexcRunning:
|
||||||
warn "Stopping blockexc without starting it"
|
warn "Stopping blockexc without starting it"
|
||||||
return
|
return
|
||||||
|
@ -110,7 +113,7 @@ proc stop*(b: BlockExcEngine) {.async.} =
|
||||||
await t.cancelAndWait()
|
await t.cancelAndWait()
|
||||||
trace "Task stopped"
|
trace "Task stopped"
|
||||||
|
|
||||||
trace "NetworkStore stopped"
|
trace "Block Exchange Engine stopped"
|
||||||
|
|
||||||
proc requestBlock*(
|
proc requestBlock*(
|
||||||
b: BlockExcEngine,
|
b: BlockExcEngine,
|
||||||
|
@ -169,6 +172,44 @@ proc requestBlock*(
|
||||||
|
|
||||||
return blk
|
return blk
|
||||||
|
|
||||||
|
proc retrieve*(
|
||||||
|
b: BlockExcEngine,
|
||||||
|
cid: Cid): Future[?!bt.Block] {.async.} =
|
||||||
|
## Attempt to retrieve block from local stores, and if not retrieved,
|
||||||
|
## request from Dagger network
|
||||||
|
##
|
||||||
|
var
|
||||||
|
blk: bt.Block
|
||||||
|
blkResult = await b.blockStoreMgr.getBlock(cid)
|
||||||
|
|
||||||
|
if blkResult.isOk:
|
||||||
|
blk = blkResult.get
|
||||||
|
else:
|
||||||
|
blk = try:
|
||||||
|
await b.requestBlock(cid)
|
||||||
|
except AsyncTimeoutError:
|
||||||
|
trace "Block request timed out", cid
|
||||||
|
return failure("Block request timed out")
|
||||||
|
except CatchableError as exc:
|
||||||
|
trace "Exception requesting block", cid, exc = exc.msg
|
||||||
|
return failure(exc.msg)
|
||||||
|
|
||||||
|
return success(blk)
|
||||||
|
|
||||||
|
proc store*(
|
||||||
|
self: BlockExcEngine,
|
||||||
|
blk: bt.Block): Future[bool] {.async.} =
|
||||||
|
## Store the block *locally* using the block store manager
|
||||||
|
|
||||||
|
return await self.blockStoreMgr.putBlock(blk)
|
||||||
|
|
||||||
|
proc exists*(
|
||||||
|
self: BlockExcEngine,
|
||||||
|
cid: Cid): bool =
|
||||||
|
## Check if the block exists *locally* using the block store manager
|
||||||
|
|
||||||
|
return self.blockStoreMgr.hasBlock(cid)
|
||||||
|
|
||||||
proc blockPresenceHandler*(
|
proc blockPresenceHandler*(
|
||||||
b: BlockExcEngine,
|
b: BlockExcEngine,
|
||||||
peer: PeerID,
|
peer: PeerID,
|
||||||
|
@ -193,7 +234,7 @@ proc scheduleTasks(b: BlockExcEngine, blocks: seq[bt.Block]) =
|
||||||
for c in cids: # for each cid
|
for c in cids: # for each cid
|
||||||
# schedule a peer if it wants at least one
|
# schedule a peer if it wants at least one
|
||||||
# cid and we have it in our local store
|
# cid and we have it in our local store
|
||||||
if c in p.peerWants and c in b.localStore:
|
if c in p.peerWants and c in b.blockStoreMgr:
|
||||||
if not b.scheduleTask(p):
|
if not b.scheduleTask(p):
|
||||||
trace "Unable to schedule task for peer", peer = p.id
|
trace "Unable to schedule task for peer", peer = p.id
|
||||||
break # do next peer
|
break # do next peer
|
||||||
|
@ -226,10 +267,8 @@ proc blocksHandler*(
|
||||||
##
|
##
|
||||||
|
|
||||||
trace "Got blocks from peer", peer, len = blocks.len
|
trace "Got blocks from peer", peer, len = blocks.len
|
||||||
for blk in blocks:
|
if not (await b.blockStoreMgr.putBlocks(blocks)):
|
||||||
if not (await b.localStore.putBlock(blk)):
|
trace "Unable to store blocks"
|
||||||
trace "Unable to store block", cid = blk.cid
|
|
||||||
continue
|
|
||||||
|
|
||||||
b.resolveBlocks(blocks)
|
b.resolveBlocks(blocks)
|
||||||
let peerCtx = b.getPeerCtx(peer)
|
let peerCtx = b.getPeerCtx(peer)
|
||||||
|
@ -266,7 +305,7 @@ proc wantListHandler*(
|
||||||
|
|
||||||
# peer might want to ask for the same cid with
|
# peer might want to ask for the same cid with
|
||||||
# different want params
|
# different want params
|
||||||
if e.sendDontHave and e.cid notin b.localStore:
|
if e.sendDontHave and e.cid notin b.blockStoreMgr:
|
||||||
dontHaves.add(e.cid)
|
dontHaves.add(e.cid)
|
||||||
|
|
||||||
# send don't have's to remote
|
# send don't have's to remote
|
||||||
|
@ -338,13 +377,9 @@ proc taskHandler*(b: BlockExcEngine, task: BlockExcPeerCtx) {.gcsafe, async.} =
|
||||||
# TODO: There should be all sorts of accounting of
|
# TODO: There should be all sorts of accounting of
|
||||||
# bytes sent/received here
|
# bytes sent/received here
|
||||||
if wantsBlocks.len > 0:
|
if wantsBlocks.len > 0:
|
||||||
let blockFuts = await allFinished(wantsBlocks.mapIt(
|
let
|
||||||
b.localStore.getBlock(it.cid)
|
cids = wantsBlocks.mapIt(it.cid)
|
||||||
))
|
blocks = await b.blockStoreMgr.getBlocks(cids)
|
||||||
|
|
||||||
let blocks = blockFuts
|
|
||||||
.filterIt((not it.failed) and it.read.isOk)
|
|
||||||
.mapIt(!it.read)
|
|
||||||
|
|
||||||
if blocks.len > 0:
|
if blocks.len > 0:
|
||||||
b.network.request.sendBlocks(
|
b.network.request.sendBlocks(
|
||||||
|
@ -363,7 +398,7 @@ proc taskHandler*(b: BlockExcEngine, task: BlockExcPeerCtx) {.gcsafe, async.} =
|
||||||
for e in task.peerWants:
|
for e in task.peerWants:
|
||||||
if e.wantType == WantType.wantHave:
|
if e.wantType == WantType.wantHave:
|
||||||
var presence = Presence(cid: e.cid)
|
var presence = Presence(cid: e.cid)
|
||||||
presence.have = b.localStore.hasblock(presence.cid)
|
presence.have = b.blockStoreMgr.hasblock(presence.cid)
|
||||||
if presence.have and price =? b.pricing.?price:
|
if presence.have and price =? b.pricing.?price:
|
||||||
presence.price = price
|
presence.price = price
|
||||||
wants.add(BlockPresence.init(presence))
|
wants.add(BlockPresence.init(presence))
|
||||||
|
@ -383,18 +418,18 @@ proc blockexcTaskRunner(b: BlockExcEngine) {.async.} =
|
||||||
|
|
||||||
proc new*(
|
proc new*(
|
||||||
T: type BlockExcEngine,
|
T: type BlockExcEngine,
|
||||||
localStore: BlockStore,
|
|
||||||
wallet: WalletRef,
|
wallet: WalletRef,
|
||||||
network: BlockExcNetwork,
|
network: BlockExcNetwork,
|
||||||
|
storeMgr: BlockStoreManager,
|
||||||
concurrentTasks = DefaultConcurrentTasks,
|
concurrentTasks = DefaultConcurrentTasks,
|
||||||
maxRetries = DefaultMaxRetries,
|
maxRetries = DefaultMaxRetries,
|
||||||
peersPerRequest = DefaultMaxPeersPerRequest): T =
|
peersPerRequest = DefaultMaxPeersPerRequest): T =
|
||||||
|
|
||||||
let engine = BlockExcEngine(
|
let engine = BlockExcEngine(
|
||||||
localStore: localStore,
|
|
||||||
pendingBlocks: PendingBlocksManager.new(),
|
pendingBlocks: PendingBlocksManager.new(),
|
||||||
peersPerRequest: peersPerRequest,
|
peersPerRequest: peersPerRequest,
|
||||||
network: network,
|
network: network,
|
||||||
|
blockStoreMgr: storeMgr,
|
||||||
wallet: wallet,
|
wallet: wallet,
|
||||||
concurrentTasks: concurrentTasks,
|
concurrentTasks: concurrentTasks,
|
||||||
maxRetries: maxRetries,
|
maxRetries: maxRetries,
|
||||||
|
|
|
@ -23,8 +23,7 @@ import ./node
|
||||||
import ./conf
|
import ./conf
|
||||||
import ./rng
|
import ./rng
|
||||||
import ./rest/api
|
import ./rest/api
|
||||||
import ./stores/fsstore
|
import ./stores
|
||||||
import ./stores/networkstore
|
|
||||||
import ./blockexchange
|
import ./blockexchange
|
||||||
import ./utils/fileutils
|
import ./utils/fileutils
|
||||||
|
|
||||||
|
@ -65,10 +64,12 @@ proc new*(T: type DaggerServer, config: DaggerConf): T =
|
||||||
let
|
let
|
||||||
wallet = WalletRef.new(EthPrivateKey.random())
|
wallet = WalletRef.new(EthPrivateKey.random())
|
||||||
network = BlockExcNetwork.new(switch)
|
network = BlockExcNetwork.new(switch)
|
||||||
|
cacheStore = MemoryStore.new()
|
||||||
localStore = FSStore.new(config.dataDir / "repo")
|
localStore = FSStore.new(config.dataDir / "repo")
|
||||||
engine = BlockExcEngine.new(localStore, wallet, network)
|
# Most Local > Most Remote: order is important!
|
||||||
store = NetworkStore.new(engine, localStore)
|
blockStoreMgr = BlockStoreManager.new(@[cacheStore, localStore])
|
||||||
daggerNode = DaggerNodeRef.new(switch, store, engine)
|
engine = BlockExcEngine.new(wallet, network, blockStoreMgr)
|
||||||
|
daggerNode = DaggerNodeRef.new(switch, engine)
|
||||||
restServer = RestServerRef.new(
|
restServer = RestServerRef.new(
|
||||||
daggerNode.initRestApi(),
|
daggerNode.initRestApi(),
|
||||||
initTAddress("127.0.0.1" , config.apiPort),
|
initTAddress("127.0.0.1" , config.apiPort),
|
||||||
|
|
|
@ -22,7 +22,7 @@ import pkg/libp2p/signed_envelope
|
||||||
import ./chunker
|
import ./chunker
|
||||||
import ./blocktype as bt
|
import ./blocktype as bt
|
||||||
import ./blocksmanifest
|
import ./blocksmanifest
|
||||||
import ./stores/blockstore
|
import ./stores/manager
|
||||||
import ./blockexchange
|
import ./blockexchange
|
||||||
|
|
||||||
logScope:
|
logScope:
|
||||||
|
@ -37,7 +37,6 @@ type
|
||||||
DaggerNodeRef* = ref object
|
DaggerNodeRef* = ref object
|
||||||
switch*: Switch
|
switch*: Switch
|
||||||
networkId*: PeerID
|
networkId*: PeerID
|
||||||
blockStore*: BlockStore
|
|
||||||
engine*: BlockExcEngine
|
engine*: BlockExcEngine
|
||||||
|
|
||||||
proc start*(node: DaggerNodeRef) {.async.} =
|
proc start*(node: DaggerNodeRef) {.async.} =
|
||||||
|
@ -71,7 +70,7 @@ proc streamBlocks*(
|
||||||
# to prevent slurping the entire dataset
|
# to prevent slurping the entire dataset
|
||||||
# since disk IO is blocking
|
# since disk IO is blocking
|
||||||
for c in blockManifest:
|
for c in blockManifest:
|
||||||
without blk =? (await node.blockStore.getBlock(c)):
|
without blk =? (await node.engine.retrieve(c)):
|
||||||
warn "Couldn't retrieve block", cid = c
|
warn "Couldn't retrieve block", cid = c
|
||||||
break # abort if we couldn't get a block
|
break # abort if we couldn't get a block
|
||||||
|
|
||||||
|
@ -88,7 +87,7 @@ proc retrieve*(
|
||||||
cid: Cid): Future[?!void] {.async.} =
|
cid: Cid): Future[?!void] {.async.} =
|
||||||
|
|
||||||
trace "Received retrieval request", cid
|
trace "Received retrieval request", cid
|
||||||
without blk =? await node.blockStore.getBlock(cid):
|
without blk =? await node.engine.retrieve(cid):
|
||||||
return failure(
|
return failure(
|
||||||
newException(DaggerError, "Couldn't retrieve block for Cid!"))
|
newException(DaggerError, "Couldn't retrieve block for Cid!"))
|
||||||
|
|
||||||
|
@ -136,9 +135,10 @@ proc store*(
|
||||||
return failure("Unable to init block from chunk!")
|
return failure("Unable to init block from chunk!")
|
||||||
|
|
||||||
blockManifest.put(blk.cid)
|
blockManifest.put(blk.cid)
|
||||||
if not (await node.blockStore.putBlock(blk)):
|
if not (await node.engine.store(blk)):
|
||||||
# trace "Unable to store block", cid = blk.cid
|
# trace "Unable to store block", cid = blk.cid
|
||||||
return failure("Unable to store block " & $blk.cid)
|
return failure("Unable to store block " & $blk.cid)
|
||||||
|
node.engine.resolveBlocks(@[blk])
|
||||||
|
|
||||||
except CancelledError as exc:
|
except CancelledError as exc:
|
||||||
raise exc
|
raise exc
|
||||||
|
@ -157,9 +157,10 @@ proc store*(
|
||||||
trace "Unable to init block from manifest data!"
|
trace "Unable to init block from manifest data!"
|
||||||
return failure("Unable to init block from manifest data!")
|
return failure("Unable to init block from manifest data!")
|
||||||
|
|
||||||
if not (await node.blockStore.putBlock(manifest)):
|
if not (await node.engine.store(manifest)):
|
||||||
trace "Unable to store manifest", cid = manifest.cid
|
trace "Unable to store manifest", cid = manifest.cid
|
||||||
return failure("Unable to store manifest " & $manifest.cid)
|
return failure("Unable to store manifest " & $manifest.cid)
|
||||||
|
node.engine.resolveBlocks(@[manifest])
|
||||||
|
|
||||||
var cid: ?!Cid
|
var cid: ?!Cid
|
||||||
if (cid = blockManifest.cid; cid.isErr):
|
if (cid = blockManifest.cid; cid.isErr):
|
||||||
|
@ -175,9 +176,7 @@ proc store*(
|
||||||
proc new*(
|
proc new*(
|
||||||
T: type DaggerNodeRef,
|
T: type DaggerNodeRef,
|
||||||
switch: Switch,
|
switch: Switch,
|
||||||
store: BlockStore,
|
|
||||||
engine: BlockExcEngine): T =
|
engine: BlockExcEngine): T =
|
||||||
T(
|
T(
|
||||||
switch: switch,
|
switch: switch,
|
||||||
blockStore: store,
|
|
||||||
engine: engine)
|
engine: engine)
|
||||||
|
|
|
@ -1,7 +1,7 @@
|
||||||
import ./stores/[
|
import ./stores/[
|
||||||
|
manager,
|
||||||
memorystore,
|
memorystore,
|
||||||
blockstore,
|
blockstore,
|
||||||
networkstore,
|
|
||||||
fsstore]
|
fsstore]
|
||||||
|
|
||||||
export memorystore, blockstore, networkstore, fsstore
|
export manager, memorystore, blockstore, fsstore
|
||||||
|
|
|
@ -18,7 +18,13 @@ import ../blocktype
|
||||||
export blocktype, libp2p
|
export blocktype, libp2p
|
||||||
|
|
||||||
type
|
type
|
||||||
|
PutFail* = proc(self: BlockStore, blk: Block): Future[void] {.gcsafe.}
|
||||||
|
DelFail* = proc(self: BlockStore, cid: Cid): Future[void] {.gcsafe.}
|
||||||
|
|
||||||
BlockStore* = ref object of RootObj
|
BlockStore* = ref object of RootObj
|
||||||
|
canFail*: bool # Allow put/del operations to fail optimistically
|
||||||
|
onPutFail*: PutFail
|
||||||
|
onDelFail*: DelFail
|
||||||
|
|
||||||
method getBlock*(
|
method getBlock*(
|
||||||
b: BlockStore,
|
b: BlockStore,
|
||||||
|
@ -50,5 +56,5 @@ method hasBlock*(s: BlockStore, cid: Cid): bool {.base.} =
|
||||||
|
|
||||||
return false
|
return false
|
||||||
|
|
||||||
proc contains*(s: BlockStore, blk: Cid): bool =
|
method contains*(s: BlockStore, blk: Cid): bool {.base.} =
|
||||||
s.hasBlock(blk)
|
s.hasBlock(blk)
|
||||||
|
|
|
@ -29,7 +29,6 @@ logScope:
|
||||||
|
|
||||||
type
|
type
|
||||||
FSStore* = ref object of BlockStore
|
FSStore* = ref object of BlockStore
|
||||||
cache: BlockStore
|
|
||||||
repoDir: string
|
repoDir: string
|
||||||
postfixLen*: int
|
postfixLen*: int
|
||||||
|
|
||||||
|
@ -105,9 +104,7 @@ method hasBlock*(self: FSStore, cid: Cid): bool =
|
||||||
proc new*(
|
proc new*(
|
||||||
T: type FSStore,
|
T: type FSStore,
|
||||||
repoDir: string,
|
repoDir: string,
|
||||||
postfixLen = 2,
|
postfixLen = 2): T =
|
||||||
cache: BlockStore = MemoryStore.new()): T =
|
|
||||||
T(
|
T(
|
||||||
postfixLen: postfixLen,
|
postfixLen: postfixLen,
|
||||||
repoDir: repoDir,
|
repoDir: repoDir)
|
||||||
cache: cache)
|
|
||||||
|
|
|
@ -0,0 +1,177 @@
|
||||||
|
## Nim-Dagger
|
||||||
|
## Copyright (c) 2021 Status Research & Development GmbH
|
||||||
|
## Licensed under either of
|
||||||
|
## * Apache License, version 2.0, ([LICENSE-APACHE](LICENSE-APACHE))
|
||||||
|
## * MIT license ([LICENSE-MIT](LICENSE-MIT))
|
||||||
|
## at your option.
|
||||||
|
## This file may not be copied, modified, or distributed except according to
|
||||||
|
## those terms.
|
||||||
|
|
||||||
|
{.push raises: [Defect].}
|
||||||
|
|
||||||
|
import std/sequtils
|
||||||
|
import std/sugar
|
||||||
|
|
||||||
|
import pkg/chronicles
|
||||||
|
import pkg/chronos
|
||||||
|
import pkg/questionable
|
||||||
|
import pkg/questionable/results
|
||||||
|
|
||||||
|
import ./blockstore
|
||||||
|
|
||||||
|
logScope:
|
||||||
|
topics = "dagger blockstoremanager"
|
||||||
|
|
||||||
|
type
|
||||||
|
BlockStoreManager* = ref object
|
||||||
|
stores: seq[BlockStore]
|
||||||
|
|
||||||
|
method getBlock*(
|
||||||
|
self: BlockStoreManager,
|
||||||
|
cid: Cid): Future[?!Block] {.async.} =
|
||||||
|
## Cycle through stores, in order of insertion, to get a block.
|
||||||
|
## Cycling short circuits once a block is found.
|
||||||
|
## In practice, this should query from most local to most remote, eg:
|
||||||
|
## MemoryStore > FSStore
|
||||||
|
##
|
||||||
|
|
||||||
|
for store in self.stores:
|
||||||
|
logScope:
|
||||||
|
cid
|
||||||
|
store = $(typeof store)
|
||||||
|
trace "Getting block"
|
||||||
|
let blk = await store.getBlock(cid)
|
||||||
|
if blk.isOk:
|
||||||
|
trace "Retrieved block from store"
|
||||||
|
return blk
|
||||||
|
else:
|
||||||
|
trace "Couldn't get from store"
|
||||||
|
|
||||||
|
return Block.failure("Couldn't find block in any stores")
|
||||||
|
|
||||||
|
method getBlocks*(
|
||||||
|
self: BlockStoreManager,
|
||||||
|
cids: seq[Cid]): Future[seq[Block]] {.async.} =
|
||||||
|
## Gets blocks from each local store in the BlockStoreManager.
|
||||||
|
## Cycle through local stores, in order of insertion, to get a block.
|
||||||
|
## In practice, this should query from most local to least local, eg:
|
||||||
|
## MemoryStore > FSStore
|
||||||
|
## Each block request stops cycling BlockStores once a block is found.
|
||||||
|
##
|
||||||
|
|
||||||
|
let getFuts = await allFinished(cids.map(cid => self.getBlock(cid)))
|
||||||
|
return getFuts
|
||||||
|
.filterIt((not it.failed) and it.read.isOk)
|
||||||
|
.mapIt(!it.read) # extract Block value
|
||||||
|
|
||||||
|
method putBlock*(
|
||||||
|
self: BlockStoreManager,
|
||||||
|
blk: Block): Future[bool] {.async.} =
|
||||||
|
## Put a block to each local store in the BlockStoreManager.
|
||||||
|
## Cycle through local stores, in order of insertion, to put a block.
|
||||||
|
## In practice, this should query from most local to least local, eg:
|
||||||
|
## MemoryStore > FSStore
|
||||||
|
##
|
||||||
|
|
||||||
|
var success = true
|
||||||
|
for store in self.stores:
|
||||||
|
logScope:
|
||||||
|
cid = blk.cid
|
||||||
|
store = $(typeof store)
|
||||||
|
trace "Putting block in store"
|
||||||
|
# TODO: Could we use asyncSpawn here as we likely don't need to check
|
||||||
|
# if putBlock failed or not (think in terms of a network-based storage
|
||||||
|
# where an operation may take a long time)?
|
||||||
|
var storeSuccess = await store.putBlock(blk)
|
||||||
|
if not storeSuccess:
|
||||||
|
trace "Couldn't put block in store"
|
||||||
|
|
||||||
|
# allow the operation to fail without affecting the return value
|
||||||
|
# (ie which indicatees if the put operation was successful or not)
|
||||||
|
if store.canFail:
|
||||||
|
storeSuccess = true
|
||||||
|
|
||||||
|
if not store.onPutFail.isNil:
|
||||||
|
asyncSpawn store.onPutFail(store, blk)
|
||||||
|
|
||||||
|
else: trace "Put block in store"
|
||||||
|
success = success and storeSuccess
|
||||||
|
|
||||||
|
return success
|
||||||
|
|
||||||
|
method putBlocks*(
|
||||||
|
self: BlockStoreManager,
|
||||||
|
blks: seq[Block]): Future[bool] {.async.} =
|
||||||
|
## Put blocks to each local store in the BlockStoreManager.
|
||||||
|
## Cycle through local stores, in order of insertion, to put a block.
|
||||||
|
## In practice, this should query from most local to least local, eg:
|
||||||
|
## MemoryStore > FSStore
|
||||||
|
##
|
||||||
|
|
||||||
|
let
|
||||||
|
putFuts = await allFinished(blks.map(blk => self.putBlock(blk)))
|
||||||
|
success = putFuts.allIt(not it.failed and it.read) # extract bool value
|
||||||
|
|
||||||
|
return success
|
||||||
|
|
||||||
|
method delBlock*(
|
||||||
|
self: BlockStoreManager,
|
||||||
|
cid: Cid): Future[bool] {.async.} =
|
||||||
|
## Delete a block from each local block store in the BlockStoreManager.
|
||||||
|
## Cycle through local stores, in order of insertion, to delete a block.
|
||||||
|
## In practice, this should query from most local to least local, eg:
|
||||||
|
## MemoryStore > FSStore
|
||||||
|
##
|
||||||
|
|
||||||
|
var success = true
|
||||||
|
for store in self.stores:
|
||||||
|
logScope:
|
||||||
|
cid
|
||||||
|
store = $(typeof store)
|
||||||
|
trace "Deleting block from store"
|
||||||
|
# TODO: Could we use asyncSpawn here as we likely don't need to check
|
||||||
|
# if deletion failed or not?
|
||||||
|
var storeSuccess = await store.delBlock(cid)
|
||||||
|
if not storeSuccess:
|
||||||
|
trace "Couldn't delete from store"
|
||||||
|
|
||||||
|
# allow the operation to fail without affecting the return value
|
||||||
|
# (ie which indicatees if the put operation was successful or not)
|
||||||
|
if store.canFail:
|
||||||
|
storeSuccess = true
|
||||||
|
|
||||||
|
if not store.onDelFail.isNil:
|
||||||
|
asyncSpawn store.onDelFail(store, cid)
|
||||||
|
|
||||||
|
else: trace "Deleted block from store"
|
||||||
|
success = success and storeSuccess
|
||||||
|
|
||||||
|
return success
|
||||||
|
|
||||||
|
method hasBlock*(self: BlockStoreManager, cid: Cid): bool =
|
||||||
|
## Check if the block exists in the BlockStoreManager
|
||||||
|
##
|
||||||
|
|
||||||
|
for store in self.stores:
|
||||||
|
logScope:
|
||||||
|
cid
|
||||||
|
store = $(typeof store)
|
||||||
|
|
||||||
|
trace "Checking has block"
|
||||||
|
if store.hasBlock(cid):
|
||||||
|
trace "Store has block"
|
||||||
|
return true
|
||||||
|
else:
|
||||||
|
trace "Store doesn't have block"
|
||||||
|
|
||||||
|
method contains*(self: BlockStoreManager, blk: Cid): bool =
|
||||||
|
self.hasBlock(blk)
|
||||||
|
|
||||||
|
proc new*(
|
||||||
|
T: type BlockStoreManager,
|
||||||
|
stores: seq[BlockStore]): T =
|
||||||
|
|
||||||
|
let b = BlockStoreManager(
|
||||||
|
stores: stores)
|
||||||
|
|
||||||
|
return b
|
|
@ -1,90 +0,0 @@
|
||||||
## Nim-Dagger
|
|
||||||
## Copyright (c) 2021 Status Research & Development GmbH
|
|
||||||
## Licensed under either of
|
|
||||||
## * Apache License, version 2.0, ([LICENSE-APACHE](LICENSE-APACHE))
|
|
||||||
## * MIT license ([LICENSE-MIT](LICENSE-MIT))
|
|
||||||
## at your option.
|
|
||||||
## This file may not be copied, modified, or distributed except according to
|
|
||||||
## those terms.
|
|
||||||
|
|
||||||
{.push raises: [Defect].}
|
|
||||||
|
|
||||||
import pkg/chronicles
|
|
||||||
import pkg/chronos
|
|
||||||
import pkg/libp2p
|
|
||||||
|
|
||||||
import ../blocktype as bt
|
|
||||||
import ../utils/asyncheapqueue
|
|
||||||
|
|
||||||
import ./blockstore
|
|
||||||
import ../blockexchange/network
|
|
||||||
import ../blockexchange/engine
|
|
||||||
import ../blockexchange/peercontext
|
|
||||||
|
|
||||||
export blockstore, network, engine, asyncheapqueue
|
|
||||||
|
|
||||||
logScope:
|
|
||||||
topics = "dagger networkstore"
|
|
||||||
|
|
||||||
type
|
|
||||||
NetworkStore* = ref object of BlockStore
|
|
||||||
engine*: BlockExcEngine # blockexc decision engine
|
|
||||||
localStore*: BlockStore # local block store
|
|
||||||
|
|
||||||
method getBlock*(
|
|
||||||
self: NetworkStore,
|
|
||||||
cid: Cid): Future[?!bt.Block] {.async.} =
|
|
||||||
## Get a block from a remote peer
|
|
||||||
##
|
|
||||||
|
|
||||||
trace "Getting block", cid
|
|
||||||
without var blk =? (await self.localStore.getBlock(cid)):
|
|
||||||
trace "Couldn't get from local store", cid
|
|
||||||
blk = try:
|
|
||||||
await self.engine.requestBlock(cid)
|
|
||||||
except CatchableError as exc:
|
|
||||||
trace "Exception requestig block", cid, exc = exc.msg
|
|
||||||
return failure(exc.msg)
|
|
||||||
|
|
||||||
trace "Retrieved block from local store", cid
|
|
||||||
return blk.success
|
|
||||||
|
|
||||||
method putBlock*(
|
|
||||||
self: NetworkStore,
|
|
||||||
blk: bt.Block): Future[bool] {.async.} =
|
|
||||||
trace "Puting block", cid = blk.cid
|
|
||||||
|
|
||||||
if not (await self.localStore.putBlock(blk)):
|
|
||||||
return false
|
|
||||||
|
|
||||||
self.engine.resolveBlocks(@[blk])
|
|
||||||
return true
|
|
||||||
|
|
||||||
method delBlock*(
|
|
||||||
self: NetworkStore,
|
|
||||||
cid: Cid): Future[bool] =
|
|
||||||
## Delete a block/s from the block store
|
|
||||||
##
|
|
||||||
|
|
||||||
self.localStore.delBlock(cid)
|
|
||||||
|
|
||||||
{.pop.}
|
|
||||||
|
|
||||||
method hasBlock*(
|
|
||||||
self: NetworkStore,
|
|
||||||
cid: Cid): bool =
|
|
||||||
## Check if the block exists in the blockstore
|
|
||||||
##
|
|
||||||
|
|
||||||
self.localStore.hasBlock(cid)
|
|
||||||
|
|
||||||
proc new*(
|
|
||||||
T: type NetworkStore,
|
|
||||||
engine: BlockExcEngine,
|
|
||||||
localStore: BlockStore): T =
|
|
||||||
|
|
||||||
let b = NetworkStore(
|
|
||||||
localStore: localStore,
|
|
||||||
engine: engine)
|
|
||||||
|
|
||||||
return b
|
|
|
@ -1,5 +1,6 @@
|
||||||
import std/unittest
|
import std/unittest
|
||||||
|
|
||||||
|
import pkg/dagger/blockexchange
|
||||||
import pkg/dagger/stores
|
import pkg/dagger/stores
|
||||||
import ../../examples
|
import ../../examples
|
||||||
|
|
||||||
|
|
|
@ -2,6 +2,7 @@ import pkg/asynctest
|
||||||
import pkg/chronos
|
import pkg/chronos
|
||||||
import pkg/stew/byteutils
|
import pkg/stew/byteutils
|
||||||
import ../../examples
|
import ../../examples
|
||||||
|
import pkg/dagger/blockexchange
|
||||||
import pkg/dagger/stores
|
import pkg/dagger/stores
|
||||||
|
|
||||||
suite "account protobuf messages":
|
suite "account protobuf messages":
|
||||||
|
|
|
@ -17,7 +17,7 @@ import pkg/dagger/blocktype as bt
|
||||||
import ../helpers
|
import ../helpers
|
||||||
import ../examples
|
import ../examples
|
||||||
|
|
||||||
suite "NetworkStore engine - 2 nodes":
|
suite "Block exchange engine - 2 nodes":
|
||||||
|
|
||||||
let
|
let
|
||||||
chunker1 = RandomChunker.new(Rng.instance(), size = 1024, chunkSize = 256)
|
chunker1 = RandomChunker.new(Rng.instance(), size = 1024, chunkSize = 256)
|
||||||
|
@ -28,12 +28,12 @@ suite "NetworkStore engine - 2 nodes":
|
||||||
wallet1, wallet2: WalletRef
|
wallet1, wallet2: WalletRef
|
||||||
pricing1, pricing2: Pricing
|
pricing1, pricing2: Pricing
|
||||||
network1, network2: BlockExcNetwork
|
network1, network2: BlockExcNetwork
|
||||||
blockexc1, blockexc2: NetworkStore
|
|
||||||
peerId1, peerId2: PeerID
|
peerId1, peerId2: PeerID
|
||||||
peerCtx1, peerCtx2: BlockExcPeerCtx
|
peerCtx1, peerCtx2: BlockExcPeerCtx
|
||||||
blocks1, blocks2: seq[bt.Block]
|
blocks1, blocks2: seq[bt.Block]
|
||||||
engine1, engine2: BlockExcEngine
|
engine1, engine2: BlockExcEngine
|
||||||
localStore1, localStore2: BlockStore
|
localStore1, localStore2: BlockStore
|
||||||
|
blockStoreMgr1, blockStoreMgr2: BlockStoreManager
|
||||||
|
|
||||||
setup:
|
setup:
|
||||||
while true:
|
while true:
|
||||||
|
@ -64,14 +64,14 @@ suite "NetworkStore engine - 2 nodes":
|
||||||
|
|
||||||
localStore1 = MemoryStore.new(blocks1.mapIt( it ))
|
localStore1 = MemoryStore.new(blocks1.mapIt( it ))
|
||||||
network1 = BlockExcNetwork.new(switch = switch1)
|
network1 = BlockExcNetwork.new(switch = switch1)
|
||||||
engine1 = BlockExcEngine.new(localStore1, wallet1, network1)
|
blockStoreMgr1 = BlockStoreManager.new(@[localStore1])
|
||||||
blockexc1 = NetworkStore.new(engine1, localStore1)
|
engine1 = BlockExcEngine.new(wallet1, network1, blockStoreMgr1)
|
||||||
switch1.mount(network1)
|
switch1.mount(network1)
|
||||||
|
|
||||||
localStore2 = MemoryStore.new(blocks2.mapIt( it ))
|
localStore2 = MemoryStore.new(blocks2.mapIt( it ))
|
||||||
network2 = BlockExcNetwork.new(switch = switch2)
|
network2 = BlockExcNetwork.new(switch = switch2)
|
||||||
engine2 = BlockExcEngine.new(localStore2, wallet2, network2)
|
blockStoreMgr2 = BlockStoreManager.new(@[localStore2])
|
||||||
blockexc2 = NetworkStore.new(engine2, localStore2)
|
engine2 = BlockExcEngine.new(wallet2, network2, blockStoreMgr2)
|
||||||
switch2.mount(network2)
|
switch2.mount(network2)
|
||||||
|
|
||||||
await allFuturesThrowing(
|
await allFuturesThrowing(
|
||||||
|
@ -80,21 +80,21 @@ suite "NetworkStore engine - 2 nodes":
|
||||||
)
|
)
|
||||||
|
|
||||||
# initialize our want lists
|
# initialize our want lists
|
||||||
blockexc1.engine.wantList = blocks2.mapIt( it.cid )
|
engine1.wantList = blocks2.mapIt( it.cid )
|
||||||
blockexc2.engine.wantList = blocks1.mapIt( it.cid )
|
engine2.wantList = blocks1.mapIt( it.cid )
|
||||||
|
|
||||||
pricing1.address = wallet1.address
|
pricing1.address = wallet1.address
|
||||||
pricing2.address = wallet2.address
|
pricing2.address = wallet2.address
|
||||||
blockexc1.engine.pricing = pricing1.some
|
engine1.pricing = pricing1.some
|
||||||
blockexc2.engine.pricing = pricing2.some
|
engine2.pricing = pricing2.some
|
||||||
|
|
||||||
await switch1.connect(
|
await switch1.connect(
|
||||||
switch2.peerInfo.peerId,
|
switch2.peerInfo.peerId,
|
||||||
switch2.peerInfo.addrs)
|
switch2.peerInfo.addrs)
|
||||||
|
|
||||||
await sleepAsync(1.seconds) # give some time to exchange lists
|
await sleepAsync(1.seconds) # give some time to exchange lists
|
||||||
peerCtx2 = blockexc1.engine.getPeerCtx(peerId2)
|
peerCtx2 = engine1.getPeerCtx(peerId2)
|
||||||
peerCtx1 = blockexc2.engine.getPeerCtx(peerId1)
|
peerCtx1 = engine2.getPeerCtx(peerId1)
|
||||||
|
|
||||||
teardown:
|
teardown:
|
||||||
await allFuturesThrowing(
|
await allFuturesThrowing(
|
||||||
|
@ -109,10 +109,10 @@ suite "NetworkStore engine - 2 nodes":
|
||||||
|
|
||||||
check:
|
check:
|
||||||
peerCtx1.peerHave.mapIt( $it ).sorted(cmp[string]) ==
|
peerCtx1.peerHave.mapIt( $it ).sorted(cmp[string]) ==
|
||||||
blockexc2.engine.wantList.mapIt( $it ).sorted(cmp[string])
|
engine2.wantList.mapIt( $it ).sorted(cmp[string])
|
||||||
|
|
||||||
peerCtx2.peerHave.mapIt( $it ).sorted(cmp[string]) ==
|
peerCtx2.peerHave.mapIt( $it ).sorted(cmp[string]) ==
|
||||||
blockexc1.engine.wantList.mapIt( $it ).sorted(cmp[string])
|
engine1.wantList.mapIt( $it ).sorted(cmp[string])
|
||||||
|
|
||||||
test "exchanges accounts on connect":
|
test "exchanges accounts on connect":
|
||||||
check peerCtx1.account.?address == pricing1.address.some
|
check peerCtx1.account.?address == pricing1.address.some
|
||||||
|
@ -120,7 +120,7 @@ suite "NetworkStore engine - 2 nodes":
|
||||||
|
|
||||||
test "should send want-have for block":
|
test "should send want-have for block":
|
||||||
let blk = bt.Block.init("Block 1".toBytes).tryGet()
|
let blk = bt.Block.init("Block 1".toBytes).tryGet()
|
||||||
check await blockexc2.engine.localStore.putBlock(blk)
|
check await engine2.store(blk)
|
||||||
|
|
||||||
let entry = Entry(
|
let entry = Entry(
|
||||||
`block`: blk.cid.data.buffer,
|
`block`: blk.cid.data.buffer,
|
||||||
|
@ -130,51 +130,48 @@ suite "NetworkStore engine - 2 nodes":
|
||||||
sendDontHave: false)
|
sendDontHave: false)
|
||||||
|
|
||||||
peerCtx1.peerWants.add(entry)
|
peerCtx1.peerWants.add(entry)
|
||||||
check blockexc2
|
check engine2
|
||||||
.engine
|
|
||||||
.taskQueue
|
.taskQueue
|
||||||
.pushOrUpdateNoWait(peerCtx1).isOk
|
.pushOrUpdateNoWait(peerCtx1).isOk
|
||||||
await sleepAsync(100.millis)
|
await sleepAsync(100.millis)
|
||||||
|
|
||||||
check blockexc1.engine.localStore.hasBlock(blk.cid)
|
check engine1.exists(blk.cid)
|
||||||
|
|
||||||
test "should get blocks from remote":
|
test "should get blocks from remote":
|
||||||
let blocks = await allFinished(
|
let blocks = await allFinished(
|
||||||
blocks2.mapIt( blockexc1.getBlock(it.cid) ))
|
blocks2.mapIt( engine1.retrieve(it.cid) ))
|
||||||
check blocks.mapIt( !it.read ) == blocks2
|
check blocks.mapIt( !it.read ) == blocks2
|
||||||
|
|
||||||
test "remote should send blocks when available":
|
test "remote should send blocks when available":
|
||||||
let blk = bt.Block.init("Block 1".toBytes).tryGet()
|
let blk = bt.Block.init("Block 1".toBytes).tryGet()
|
||||||
|
|
||||||
# should fail retrieving block from remote
|
# should fail retrieving block from remote
|
||||||
check not await blockexc1.getBlock(blk.cid)
|
check not await engine1.retrieve(blk.cid)
|
||||||
.withTimeout(100.millis) # should expire
|
.withTimeout(100.millis) # should expire
|
||||||
|
|
||||||
# first put the required block in the local store
|
# First, put the required block in the local store.
|
||||||
check await blockexc2.engine.localStore.putBlock(blk)
|
check await engine2.store(blk)
|
||||||
|
# Second, trigger blockexc to resolve any pending requests for the block.
|
||||||
# second trigger blockexc to resolve any pending requests
|
engine2.resolveBlocks(@[blk])
|
||||||
# for the block
|
|
||||||
check await blockexc2.putBlock(blk)
|
|
||||||
|
|
||||||
# should succeed retrieving block from remote
|
# should succeed retrieving block from remote
|
||||||
check await blockexc1.getBlock(blk.cid)
|
check await engine1.retrieve(blk.cid)
|
||||||
.withTimeout(100.millis) # should succede
|
.withTimeout(100.millis) # should succede
|
||||||
|
|
||||||
test "receives payments for blocks that were sent":
|
test "receives payments for blocks that were sent":
|
||||||
let blocks = await allFinished(
|
let blocks = await allFinished(
|
||||||
blocks2.mapIt( blockexc1.getBlock(it.cid) ))
|
blocks2.mapIt( engine1.retrieve(it.cid) ))
|
||||||
await sleepAsync(100.millis)
|
await sleepAsync(100.millis)
|
||||||
let channel = !peerCtx1.paymentChannel
|
let channel = !peerCtx1.paymentChannel
|
||||||
check wallet2.balance(channel, Asset) > 0
|
check wallet2.balance(channel, Asset) > 0
|
||||||
|
|
||||||
suite "NetworkStore - multiple nodes":
|
suite "Block exchange engine - multiple nodes":
|
||||||
let
|
let
|
||||||
chunker = RandomChunker.new(Rng.instance(), size = 4096, chunkSize = 256)
|
chunker = RandomChunker.new(Rng.instance(), size = 4096, chunkSize = 256)
|
||||||
|
|
||||||
var
|
var
|
||||||
switch: seq[Switch]
|
switch: seq[Switch]
|
||||||
blockexc: seq[NetworkStore]
|
blockexc: seq[BlockExcEngine]
|
||||||
blocks: seq[bt.Block]
|
blocks: seq[bt.Block]
|
||||||
|
|
||||||
setup:
|
setup:
|
||||||
|
@ -188,7 +185,7 @@ suite "NetworkStore - multiple nodes":
|
||||||
for e in generateNodes(5):
|
for e in generateNodes(5):
|
||||||
switch.add(e.switch)
|
switch.add(e.switch)
|
||||||
blockexc.add(e.blockexc)
|
blockexc.add(e.blockexc)
|
||||||
await e.blockexc.engine.start()
|
await e.blockexc.start()
|
||||||
|
|
||||||
await allFuturesThrowing(
|
await allFuturesThrowing(
|
||||||
switch.mapIt( it.start() )
|
switch.mapIt( it.start() )
|
||||||
|
@ -203,22 +200,20 @@ suite "NetworkStore - multiple nodes":
|
||||||
blockexc = @[]
|
blockexc = @[]
|
||||||
|
|
||||||
test "should receive haves for own want list":
|
test "should receive haves for own want list":
|
||||||
let
|
let engine = blockexc[4]
|
||||||
downloader = blockexc[4]
|
|
||||||
engine = downloader.engine
|
|
||||||
|
|
||||||
# Add blocks from 1st peer to want list
|
# Add blocks from 1st peer to want list
|
||||||
engine.wantList &= blocks[0..3].mapIt( it.cid )
|
engine.wantList &= blocks[0..3].mapIt( it.cid )
|
||||||
engine.wantList &= blocks[12..15].mapIt( it.cid )
|
engine.wantList &= blocks[12..15].mapIt( it.cid )
|
||||||
|
|
||||||
await allFutures(
|
await allFutures(
|
||||||
blocks[0..3].mapIt( blockexc[0].engine.localStore.putBlock(it) ))
|
blocks[0..3].mapIt( blockexc[0].store(it) ))
|
||||||
await allFutures(
|
await allFutures(
|
||||||
blocks[4..7].mapIt( blockexc[1].engine.localStore.putBlock(it) ))
|
blocks[4..7].mapIt( blockexc[1].store(it) ))
|
||||||
await allFutures(
|
await allFutures(
|
||||||
blocks[8..11].mapIt( blockexc[2].engine.localStore.putBlock(it) ))
|
blocks[8..11].mapIt( blockexc[2].store(it) ))
|
||||||
await allFutures(
|
await allFutures(
|
||||||
blocks[12..15].mapIt( blockexc[3].engine.localStore.putBlock(it) ))
|
blocks[12..15].mapIt( blockexc[3].store(it) ))
|
||||||
|
|
||||||
await connectNodes(switch)
|
await connectNodes(switch)
|
||||||
await sleepAsync(1.seconds)
|
await sleepAsync(1.seconds)
|
||||||
|
@ -231,26 +226,24 @@ suite "NetworkStore - multiple nodes":
|
||||||
blocks[12..15].mapIt( it.cid ).mapIt($it).sorted(cmp[string])
|
blocks[12..15].mapIt( it.cid ).mapIt($it).sorted(cmp[string])
|
||||||
|
|
||||||
test "should exchange blocks with multiple nodes":
|
test "should exchange blocks with multiple nodes":
|
||||||
let
|
let engine = blockexc[4]
|
||||||
downloader = blockexc[4]
|
|
||||||
engine = downloader.engine
|
|
||||||
|
|
||||||
# Add blocks from 1st peer to want list
|
# Add blocks from 1st peer to want list
|
||||||
engine.wantList &= blocks[0..3].mapIt( it.cid )
|
engine.wantList &= blocks[0..3].mapIt( it.cid )
|
||||||
engine.wantList &= blocks[12..15].mapIt( it.cid )
|
engine.wantList &= blocks[12..15].mapIt( it.cid )
|
||||||
|
|
||||||
await allFutures(
|
await allFutures(
|
||||||
blocks[0..3].mapIt( blockexc[0].engine.localStore.putBlock(it) ))
|
blocks[0..3].mapIt( blockexc[0].store(it) ))
|
||||||
await allFutures(
|
await allFutures(
|
||||||
blocks[4..7].mapIt( blockexc[1].engine.localStore.putBlock(it) ))
|
blocks[4..7].mapIt( blockexc[1].store(it) ))
|
||||||
await allFutures(
|
await allFutures(
|
||||||
blocks[8..11].mapIt( blockexc[2].engine.localStore.putBlock(it) ))
|
blocks[8..11].mapIt( blockexc[2].store(it) ))
|
||||||
await allFutures(
|
await allFutures(
|
||||||
blocks[12..15].mapIt( blockexc[3].engine.localStore.putBlock(it) ))
|
blocks[12..15].mapIt( blockexc[3].store(it) ))
|
||||||
|
|
||||||
await connectNodes(switch)
|
await connectNodes(switch)
|
||||||
await sleepAsync(1.seconds)
|
await sleepAsync(1.seconds)
|
||||||
|
|
||||||
let wantListBlocks = await allFinished(
|
let wantListBlocks = await allFinished(
|
||||||
blocks[0..3].mapIt( downloader.getBlock(it.cid) ))
|
blocks[0..3].mapIt( engine.retrieve(it.cid) ))
|
||||||
check wantListBlocks.mapIt( !it.read ) == blocks[0..3]
|
check wantListBlocks.mapIt( !it.read ) == blocks[0..3]
|
||||||
|
|
|
@ -55,11 +55,12 @@ suite "NetworkStore engine basic":
|
||||||
network = BlockExcNetwork(request: BlockExcRequest(
|
network = BlockExcNetwork(request: BlockExcRequest(
|
||||||
sendWantList: sendWantList,
|
sendWantList: sendWantList,
|
||||||
))
|
))
|
||||||
|
memStore: BlockStore = MemoryStore.new(blocks.mapIt( it ))
|
||||||
|
blockStoreMgr = BlockStoreManager.new(@[memStore])
|
||||||
engine = BlockExcEngine.new(
|
engine = BlockExcEngine.new(
|
||||||
MemoryStore.new(blocks.mapIt( it )),
|
|
||||||
wallet,
|
wallet,
|
||||||
network)
|
network,
|
||||||
|
blockStoreMgr)
|
||||||
engine.wantList = blocks.mapIt( it.cid )
|
engine.wantList = blocks.mapIt( it.cid )
|
||||||
engine.setupPeer(peerId)
|
engine.setupPeer(peerId)
|
||||||
|
|
||||||
|
@ -76,8 +77,10 @@ suite "NetworkStore engine basic":
|
||||||
network = BlockExcNetwork(request: BlockExcRequest(
|
network = BlockExcNetwork(request: BlockExcRequest(
|
||||||
sendAccount: sendAccount,
|
sendAccount: sendAccount,
|
||||||
))
|
))
|
||||||
|
memStore: BlockStore = MemoryStore.new()
|
||||||
|
blockStoreMgr = BlockStoreManager.new(@[memStore])
|
||||||
|
|
||||||
engine = BlockExcEngine.new(MemoryStore.new, wallet, network)
|
engine = BlockExcEngine.new(wallet, network, blockStoreMgr)
|
||||||
|
|
||||||
engine.pricing = pricing.some
|
engine.pricing = pricing.some
|
||||||
engine.setupPeer(peerId)
|
engine.setupPeer(peerId)
|
||||||
|
@ -94,6 +97,7 @@ suite "NetworkStore engine handlers":
|
||||||
var
|
var
|
||||||
engine: BlockExcEngine
|
engine: BlockExcEngine
|
||||||
peerCtx: BlockExcPeerCtx
|
peerCtx: BlockExcPeerCtx
|
||||||
|
blockStoreMgr: BlockStoreManager
|
||||||
done: Future[void]
|
done: Future[void]
|
||||||
blocks: seq[bt.Block]
|
blocks: seq[bt.Block]
|
||||||
|
|
||||||
|
@ -106,7 +110,9 @@ suite "NetworkStore engine handlers":
|
||||||
blocks.add(bt.Block.init(chunk).tryGet())
|
blocks.add(bt.Block.init(chunk).tryGet())
|
||||||
|
|
||||||
done = newFuture[void]()
|
done = newFuture[void]()
|
||||||
engine = BlockExcEngine.new(MemoryStore.new(), wallet, BlockExcNetwork())
|
let memStore: BlockStore = MemoryStore.new()
|
||||||
|
blockStoreMgr = BlockStoreManager.new(@[memStore])
|
||||||
|
engine = BlockExcEngine.new(wallet, BlockExcNetwork(), blockStoreMgr)
|
||||||
peerCtx = BlockExcPeerCtx(
|
peerCtx = BlockExcPeerCtx(
|
||||||
id: peerId
|
id: peerId
|
||||||
)
|
)
|
||||||
|
@ -155,8 +161,8 @@ suite "NetworkStore engine handlers":
|
||||||
sendPresence: sendPresence
|
sendPresence: sendPresence
|
||||||
))
|
))
|
||||||
|
|
||||||
check await engine.localStore.putBlock(blocks[0])
|
check await engine.store(blocks[0])
|
||||||
check await engine.localStore.putBlock(blocks[1])
|
check await engine.store(blocks[1])
|
||||||
await engine.wantListHandler(peerId, wantList)
|
await engine.wantListHandler(peerId, wantList)
|
||||||
|
|
||||||
await done
|
await done
|
||||||
|
@ -170,7 +176,7 @@ suite "NetworkStore engine handlers":
|
||||||
let resolved = await allFinished(pending)
|
let resolved = await allFinished(pending)
|
||||||
check resolved.mapIt( it.read ) == blocks
|
check resolved.mapIt( it.read ) == blocks
|
||||||
for b in blocks:
|
for b in blocks:
|
||||||
check engine.localStore.hasBlock(b.cid)
|
check engine.exists(b.cid)
|
||||||
|
|
||||||
test "sends payments for received blocks":
|
test "sends payments for received blocks":
|
||||||
let account = Account(address: EthAddress.example)
|
let account = Account(address: EthAddress.example)
|
||||||
|
@ -216,6 +222,7 @@ suite "Task Handler":
|
||||||
|
|
||||||
var
|
var
|
||||||
engine: BlockExcEngine
|
engine: BlockExcEngine
|
||||||
|
blockStoreMgr: BlockStoreManager
|
||||||
peersCtx: seq[BlockExcPeerCtx]
|
peersCtx: seq[BlockExcPeerCtx]
|
||||||
peers: seq[PeerID]
|
peers: seq[PeerID]
|
||||||
done: Future[void]
|
done: Future[void]
|
||||||
|
@ -230,7 +237,9 @@ suite "Task Handler":
|
||||||
blocks.add(bt.Block.init(chunk).tryGet())
|
blocks.add(bt.Block.init(chunk).tryGet())
|
||||||
|
|
||||||
done = newFuture[void]()
|
done = newFuture[void]()
|
||||||
engine = BlockExcEngine.new(MemoryStore.new(), wallet, BlockExcNetwork())
|
let memStore: BlockStore = MemoryStore.new()
|
||||||
|
blockStoreMgr = BlockStoreManager.new(@[memStore])
|
||||||
|
engine = BlockExcEngine.new(wallet, BlockExcNetwork(), blockStoreMgr)
|
||||||
peersCtx = @[]
|
peersCtx = @[]
|
||||||
|
|
||||||
for i in 0..3:
|
for i in 0..3:
|
||||||
|
@ -254,7 +263,7 @@ suite "Task Handler":
|
||||||
blks[0].cid == blocks[1].cid
|
blks[0].cid == blocks[1].cid
|
||||||
|
|
||||||
for blk in blocks:
|
for blk in blocks:
|
||||||
check await engine.localStore.putBlock(blk)
|
check await engine.store(blk)
|
||||||
engine.network.request.sendBlocks = sendBlocks
|
engine.network.request.sendBlocks = sendBlocks
|
||||||
|
|
||||||
# second block to send by priority
|
# second block to send by priority
|
||||||
|
@ -292,7 +301,7 @@ suite "Task Handler":
|
||||||
]
|
]
|
||||||
|
|
||||||
for blk in blocks:
|
for blk in blocks:
|
||||||
check await engine.localStore.putBlock(blk)
|
check await engine.store(blk)
|
||||||
engine.network.request.sendPresence = sendPresence
|
engine.network.request.sendPresence = sendPresence
|
||||||
|
|
||||||
# have block
|
# have block
|
||||||
|
|
|
@ -2,9 +2,10 @@ import std/random
|
||||||
import std/sequtils
|
import std/sequtils
|
||||||
import pkg/libp2p
|
import pkg/libp2p
|
||||||
import pkg/nitro
|
import pkg/nitro
|
||||||
|
import pkg/dagger/blockexchange
|
||||||
import pkg/dagger/rng
|
import pkg/dagger/rng
|
||||||
import pkg/dagger/stores
|
import pkg/dagger/stores
|
||||||
import pkg/dagger/blocktype
|
import pkg/dagger/blocktype as bt
|
||||||
|
|
||||||
proc example*(_: type EthAddress): EthAddress =
|
proc example*(_: type EthAddress): EthAddress =
|
||||||
EthPrivateKey.random().toPublicKey.toAddress
|
EthPrivateKey.random().toPublicKey.toAddress
|
||||||
|
@ -39,7 +40,7 @@ proc example*(_: type Pricing): Pricing =
|
||||||
price: uint32.rand.u256
|
price: uint32.rand.u256
|
||||||
)
|
)
|
||||||
|
|
||||||
proc example*(_: type Block): Block =
|
proc example*(_: type bt.Block): bt.Block =
|
||||||
let length = rand(4096)
|
let length = rand(4096)
|
||||||
let bytes = newSeqWith(length, rand(uint8))
|
let bytes = newSeqWith(length, rand(uint8))
|
||||||
Block.init(bytes).tryGet()
|
Block.init(bytes).tryGet()
|
||||||
|
|
|
@ -1,9 +1,12 @@
|
||||||
import std/sequtils
|
import std/sequtils
|
||||||
|
|
||||||
|
import pkg/chronicles
|
||||||
import pkg/chronos
|
import pkg/chronos
|
||||||
import pkg/libp2p
|
import pkg/libp2p
|
||||||
|
import pkg/nitro/wallet
|
||||||
|
|
||||||
import pkg/dagger/stores
|
import pkg/dagger/stores
|
||||||
|
import pkg/dagger/blockexchange
|
||||||
import pkg/dagger/blocktype as bt
|
import pkg/dagger/blocktype as bt
|
||||||
|
|
||||||
import ../examples
|
import ../examples
|
||||||
|
@ -13,22 +16,22 @@ proc generateNodes*(
|
||||||
blocks: openArray[bt.Block] = [],
|
blocks: openArray[bt.Block] = [],
|
||||||
secureManagers: openarray[SecureProtocol] = [
|
secureManagers: openarray[SecureProtocol] = [
|
||||||
SecureProtocol.Noise,
|
SecureProtocol.Noise,
|
||||||
]): seq[tuple[switch: Switch, blockexc: NetworkStore]] =
|
]): seq[tuple[switch: Switch, blockexc: BlockExcEngine]] =
|
||||||
for i in 0..<num:
|
for i in 0..<num:
|
||||||
let
|
let
|
||||||
switch = newStandardSwitch(transportFlags = {ServerFlags.ReuseAddr})
|
switch = newStandardSwitch(transportFlags = {ServerFlags.ReuseAddr})
|
||||||
wallet = WalletRef.example
|
wallet = WalletRef.example
|
||||||
network = BlockExcNetwork.new(switch)
|
network = BlockExcNetwork.new(switch)
|
||||||
localStore = MemoryStore.new(blocks.mapIt( it ))
|
localStore: BlockStore = MemoryStore.new(blocks.mapIt( it ))
|
||||||
engine = BlockExcEngine.new(localStore, wallet, network)
|
blockStoreMgr = BlockStoreManager.new(@[localStore])
|
||||||
networkStore = NetworkStore.new(engine, localStore)
|
engine = BlockExcEngine.new(wallet, network, blockStoreMgr)
|
||||||
|
|
||||||
switch.mount(network)
|
switch.mount(network)
|
||||||
|
|
||||||
# initialize our want lists
|
# initialize our want lists
|
||||||
engine.wantList = blocks.mapIt( it.cid )
|
engine.wantList = blocks.mapIt( it.cid )
|
||||||
switch.mount(network)
|
switch.mount(network)
|
||||||
result.add((switch, networkStore))
|
result.add((switch, engine))
|
||||||
|
|
||||||
proc connectNodes*(nodes: seq[Switch]) {.async.} =
|
proc connectNodes*(nodes: seq[Switch]) {.async.} =
|
||||||
for dialer in nodes:
|
for dialer in nodes:
|
||||||
|
|
|
@ -0,0 +1,80 @@
|
||||||
|
|
||||||
|
{.push raises: [Defect].}
|
||||||
|
|
||||||
|
import chronicles
|
||||||
|
import chronos
|
||||||
|
|
||||||
|
import pkg/dagger/stores
|
||||||
|
import pkg/questionable
|
||||||
|
import pkg/questionable/results
|
||||||
|
|
||||||
|
logScope:
|
||||||
|
topics = "blockstore test mock"
|
||||||
|
|
||||||
|
type
|
||||||
|
GetBlockMock* = proc(self: BlockStoreMock, cid: Cid): Future[?!Block] {.gcsafe.}
|
||||||
|
PutBlockMock* = proc(self: BlockStoreMock, blk: Block): Future[bool] {.gcsafe.}
|
||||||
|
DelBlockMock* = proc(self: BlockStoreMock, cid: Cid): Future[bool] {.gcsafe.}
|
||||||
|
HasBlockMock* = proc(self: BlockStoreMock, cid: Cid): bool
|
||||||
|
|
||||||
|
BlockStoreMock* = ref object of BlockStore
|
||||||
|
getBlock*: GetBlockMock
|
||||||
|
putBlock*: PutBlockMock
|
||||||
|
delBlock*: DelBlockMock
|
||||||
|
hasBlock*: HasBlockMock
|
||||||
|
|
||||||
|
method getBlock*(
|
||||||
|
self: BlockStoreMock,
|
||||||
|
cid: Cid): Future[?!Block] {.async.} =
|
||||||
|
## Get a block from the stores
|
||||||
|
##
|
||||||
|
if self.getBlock.isNil:
|
||||||
|
return await procCall BlockStore(self).getBlock(cid)
|
||||||
|
|
||||||
|
return await self.getBlock(self, cid)
|
||||||
|
|
||||||
|
method hasBlock*(
|
||||||
|
self: BlockStoreMock,
|
||||||
|
cid: Cid): bool {.raises: [Defect, AssertionError].} =
|
||||||
|
## check if the block exists
|
||||||
|
##
|
||||||
|
if self.hasBlock.isNil:
|
||||||
|
return procCall BlockStore(self).hasBlock(cid)
|
||||||
|
|
||||||
|
return self.hasBlock(self, cid)
|
||||||
|
|
||||||
|
method putBlock*(
|
||||||
|
self: BlockStoreMock,
|
||||||
|
blk: Block): Future[bool] {.async.} =
|
||||||
|
## Put a block to the blockstore
|
||||||
|
##
|
||||||
|
if self.putBlock.isNil:
|
||||||
|
return await procCall BlockStore(self).putBlock(blk)
|
||||||
|
|
||||||
|
return await self.putBlock(self, blk)
|
||||||
|
|
||||||
|
method delBlock*(
|
||||||
|
self: BlockStoreMock,
|
||||||
|
cid: Cid): Future[bool] {.async.} =
|
||||||
|
## delete a block/s from the block store
|
||||||
|
##
|
||||||
|
if self.delBlock.isNil:
|
||||||
|
return await procCall BlockStore(self).delBlock(cid)
|
||||||
|
|
||||||
|
return await self.delBlock(self, cid)
|
||||||
|
|
||||||
|
func new*(_: type BlockStoreMock,
|
||||||
|
getBlock: GetBlockMock = nil,
|
||||||
|
putBlock: PutBlockMock = nil,
|
||||||
|
delBlock: DelBlockMock = nil,
|
||||||
|
hasBlock: HasBlockMock = nil,
|
||||||
|
): BlockStoreMock =
|
||||||
|
|
||||||
|
return BlockStoreMock(
|
||||||
|
getBlock: getBlock,
|
||||||
|
putBlock: putBlock,
|
||||||
|
delBlock: delBlock,
|
||||||
|
hasBlock: hasBlock
|
||||||
|
)
|
||||||
|
|
||||||
|
|
|
@ -8,7 +8,6 @@ import pkg/asynctest
|
||||||
import pkg/libp2p
|
import pkg/libp2p
|
||||||
import pkg/stew/byteutils
|
import pkg/stew/byteutils
|
||||||
|
|
||||||
import pkg/dagger/stores/memorystore
|
|
||||||
import pkg/dagger/chunker
|
import pkg/dagger/chunker
|
||||||
import pkg/dagger/stores
|
import pkg/dagger/stores
|
||||||
|
|
||||||
|
|
|
@ -0,0 +1,468 @@
|
||||||
|
import std/os
|
||||||
|
|
||||||
|
import pkg/asynctest
|
||||||
|
import pkg/chronos
|
||||||
|
import pkg/dagger/stores
|
||||||
|
import pkg/questionable
|
||||||
|
import pkg/questionable/results
|
||||||
|
# import pkg/libp2p
|
||||||
|
# import pkg/stew/byteutils
|
||||||
|
|
||||||
|
# import pkg/dagger/chunker
|
||||||
|
|
||||||
|
import ./blockstoremock
|
||||||
|
import ../examples
|
||||||
|
|
||||||
|
suite "BlockStore manager":
|
||||||
|
|
||||||
|
var
|
||||||
|
blockStore1: BlockStoreMock
|
||||||
|
blockStore2: BlockStoreMock
|
||||||
|
mgr: BlockStoreManager
|
||||||
|
|
||||||
|
setup:
|
||||||
|
blockStore1 = BlockStoreMock.new()
|
||||||
|
blockStore2 = BlockStoreMock.new()
|
||||||
|
mgr = BlockStoreManager.new(
|
||||||
|
@[BlockStore(blockStore1), BlockStore(blockStore2)])
|
||||||
|
|
||||||
|
teardown:
|
||||||
|
discard
|
||||||
|
|
||||||
|
test "getBlock, should get from second block store":
|
||||||
|
let blk = Block.example
|
||||||
|
|
||||||
|
blockStore1.getBlock =
|
||||||
|
proc(self: BlockStoreMock, cid: Cid): Future[?!Block] {.async.} =
|
||||||
|
check cid == blk.cid
|
||||||
|
return failure("block not found")
|
||||||
|
|
||||||
|
blockStore2.getBlock =
|
||||||
|
proc(self: BlockStoreMock, cid: Cid): Future[?!Block] {.async.} =
|
||||||
|
return success blk
|
||||||
|
|
||||||
|
let blkResult = await mgr.getBlock(blk.cid)
|
||||||
|
check:
|
||||||
|
blkResult.isOk
|
||||||
|
!blkResult == blk
|
||||||
|
|
||||||
|
test "getBlock, should get from first block store":
|
||||||
|
let blk = Block.example
|
||||||
|
|
||||||
|
blockStore1.getBlock =
|
||||||
|
proc(self: BlockStoreMock, cid: Cid): Future[?!Block] {.async.} =
|
||||||
|
check cid == blk.cid
|
||||||
|
return success blk
|
||||||
|
|
||||||
|
blockStore2.getBlock =
|
||||||
|
proc(self: BlockStoreMock, cid: Cid): Future[?!Block] {.async.} =
|
||||||
|
fail()
|
||||||
|
return failure("shouldn't get here")
|
||||||
|
|
||||||
|
let blkResult = await mgr.getBlock(blk.cid)
|
||||||
|
check:
|
||||||
|
blkResult.isOk
|
||||||
|
!blkResult == blk
|
||||||
|
|
||||||
|
test "getBlock, no block found":
|
||||||
|
let blk = Block.example
|
||||||
|
|
||||||
|
blockStore1.getBlock =
|
||||||
|
proc(self: BlockStoreMock, cid: Cid): Future[?!Block] {.async.} =
|
||||||
|
check cid == blk.cid
|
||||||
|
return failure("couldn't find block")
|
||||||
|
|
||||||
|
blockStore2.getBlock =
|
||||||
|
proc(self: BlockStoreMock, cid: Cid): Future[?!Block] {.async.} =
|
||||||
|
check cid == blk.cid
|
||||||
|
return failure("couldn't find block")
|
||||||
|
|
||||||
|
let blkResult = await mgr.getBlock(blk.cid)
|
||||||
|
check:
|
||||||
|
blkResult.isErr
|
||||||
|
blkResult.error.msg == "Couldn't find block in any stores"
|
||||||
|
|
||||||
|
test "getBlocks, no blocks found":
|
||||||
|
let
|
||||||
|
blk1 = Block.example
|
||||||
|
blk2 = Block.example
|
||||||
|
|
||||||
|
blockStore1.getBlock =
|
||||||
|
proc(self: BlockStoreMock, cid: Cid): Future[?!Block] {.async.} =
|
||||||
|
return failure("couldn't find block")
|
||||||
|
|
||||||
|
blockStore2.getBlock =
|
||||||
|
proc(self: BlockStoreMock, cid: Cid): Future[?!Block] {.async.} =
|
||||||
|
return failure("couldn't find block")
|
||||||
|
|
||||||
|
let blks = await mgr.getBlocks(@[blk1.cid, blk2.cid])
|
||||||
|
check:
|
||||||
|
blks.len == 0
|
||||||
|
|
||||||
|
test "getBlocks, some blocks found":
|
||||||
|
let
|
||||||
|
blk1 = Block.example
|
||||||
|
blk2 = Block.example
|
||||||
|
|
||||||
|
blockStore1.getBlock =
|
||||||
|
proc(self: BlockStoreMock, cid: Cid): Future[?!Block] {.async.} =
|
||||||
|
return failure("couldn't find block")
|
||||||
|
|
||||||
|
blockStore2.getBlock =
|
||||||
|
proc(self: BlockStoreMock, cid: Cid): Future[?!Block] {.async.} =
|
||||||
|
return success blk2
|
||||||
|
|
||||||
|
let blks = await mgr.getBlocks(@[blk1.cid, blk2.cid])
|
||||||
|
check:
|
||||||
|
blks[0] == blk2
|
||||||
|
|
||||||
|
test "getBlocks, all blocks found":
|
||||||
|
let
|
||||||
|
blk1 = Block.example
|
||||||
|
blk2 = Block.example
|
||||||
|
|
||||||
|
blockStore1.getBlock =
|
||||||
|
proc(self: BlockStoreMock, cid: Cid): Future[?!Block] {.async.} =
|
||||||
|
if cid == blk2.cid:
|
||||||
|
return failure("block not found")
|
||||||
|
else: return success blk1
|
||||||
|
|
||||||
|
blockStore2.getBlock =
|
||||||
|
proc(self: BlockStoreMock, cid: Cid): Future[?!Block] {.async.} =
|
||||||
|
if cid == blk1.cid:
|
||||||
|
return failure("block not found")
|
||||||
|
else: return success blk2
|
||||||
|
|
||||||
|
let blks = await mgr.getBlocks(@[blk1.cid, blk2.cid])
|
||||||
|
check:
|
||||||
|
blks == @[blk1, blk2]
|
||||||
|
|
||||||
|
test "putBlock, all stores should successfully put block":
|
||||||
|
let blk = Block.example
|
||||||
|
|
||||||
|
blockStore1.putBlock =
|
||||||
|
proc(self: BlockStoreMock, b: Block): Future[bool] {.async.} =
|
||||||
|
check b == blk
|
||||||
|
return true
|
||||||
|
|
||||||
|
blockStore2.putBlock =
|
||||||
|
proc(self: BlockStoreMock, b: Block): Future[bool] {.async.} =
|
||||||
|
check b == blk
|
||||||
|
return true
|
||||||
|
|
||||||
|
let blkResult = await mgr.putBlock(blk)
|
||||||
|
check:
|
||||||
|
blkResult
|
||||||
|
|
||||||
|
test "putBlock, one store should fail, result is failure":
|
||||||
|
let blk = Block.example
|
||||||
|
|
||||||
|
blockStore1.putBlock =
|
||||||
|
proc(self: BlockStoreMock, b: Block): Future[bool] {.async.} =
|
||||||
|
check b == blk
|
||||||
|
return true
|
||||||
|
|
||||||
|
blockStore2.putBlock =
|
||||||
|
proc(self: BlockStoreMock, b: Block): Future[bool] {.async.} =
|
||||||
|
check b == blk
|
||||||
|
return false
|
||||||
|
|
||||||
|
let blkResult = await mgr.putBlock(blk)
|
||||||
|
check:
|
||||||
|
not blkResult
|
||||||
|
|
||||||
|
test "putBlock, one store should fail, result is failure, callback called":
|
||||||
|
let
|
||||||
|
blk = Block.example
|
||||||
|
fut = newFuture[bool]("putBlock test")
|
||||||
|
|
||||||
|
blockStore1.putBlock =
|
||||||
|
proc(self: BlockStoreMock, b: Block): Future[bool] {.async.} =
|
||||||
|
check b == blk
|
||||||
|
return true
|
||||||
|
|
||||||
|
blockStore2.onPutFail = proc(self: BlockStore, b: Block): Future[void] {.async.} =
|
||||||
|
fut.complete(true)
|
||||||
|
|
||||||
|
blockStore2.putBlock =
|
||||||
|
proc(self: BlockStoreMock, b: Block): Future[bool] {.async.} =
|
||||||
|
check b == blk
|
||||||
|
return false
|
||||||
|
|
||||||
|
let
|
||||||
|
blkWasPut = await mgr.putBlock(blk)
|
||||||
|
putFailCalled = await fut.wait(5.seconds)
|
||||||
|
|
||||||
|
check:
|
||||||
|
not blkWasPut
|
||||||
|
putFailCalled
|
||||||
|
|
||||||
|
test "putBlock, one store should fail, result is success, callback called":
|
||||||
|
let
|
||||||
|
blk = Block.example
|
||||||
|
fut = newFuture[bool]("putBlock test")
|
||||||
|
|
||||||
|
blockStore1.putBlock =
|
||||||
|
proc(self: BlockStoreMock, b: Block): Future[bool] {.async.} =
|
||||||
|
check b == blk
|
||||||
|
return true
|
||||||
|
|
||||||
|
blockStore2.canFail = true
|
||||||
|
|
||||||
|
blockStore2.onPutFail = proc(self: BlockStore, b: Block): Future[void] {.async.} =
|
||||||
|
fut.complete(true)
|
||||||
|
|
||||||
|
blockStore2.putBlock =
|
||||||
|
proc(self: BlockStoreMock, b: Block): Future[bool] {.async.} =
|
||||||
|
check b == blk
|
||||||
|
return false
|
||||||
|
|
||||||
|
let
|
||||||
|
blkWasPut = await mgr.putBlock(blk)
|
||||||
|
putFailCalled = await fut.wait(5.seconds)
|
||||||
|
|
||||||
|
check:
|
||||||
|
blkWasPut
|
||||||
|
putFailCalled
|
||||||
|
|
||||||
|
test "putBlock, all stores fail, result should be false":
|
||||||
|
let blk = Block.example
|
||||||
|
|
||||||
|
blockStore1.putBlock =
|
||||||
|
proc(self: BlockStoreMock, b: Block): Future[bool] {.async.} =
|
||||||
|
check b == blk
|
||||||
|
return false
|
||||||
|
|
||||||
|
blockStore2.putBlock =
|
||||||
|
proc(self: BlockStoreMock, b: Block): Future[bool] {.async.} =
|
||||||
|
check b == blk
|
||||||
|
return false
|
||||||
|
|
||||||
|
let blkWasPut = await mgr.putBlock(blk)
|
||||||
|
check:
|
||||||
|
not blkWasPut
|
||||||
|
|
||||||
|
test "putBlocks, no blocks stored":
|
||||||
|
let
|
||||||
|
blk1 = Block.example
|
||||||
|
blk2 = Block.example
|
||||||
|
|
||||||
|
blockStore1.putBlock =
|
||||||
|
proc(self: BlockStoreMock, blk: Block): Future[bool] {.async.} =
|
||||||
|
return false
|
||||||
|
|
||||||
|
blockStore2.putBlock =
|
||||||
|
proc(self: BlockStoreMock, blk: Block): Future[bool] {.async.} =
|
||||||
|
return false
|
||||||
|
|
||||||
|
let blksWerePut = await mgr.putBlocks(@[blk1, blk2])
|
||||||
|
check:
|
||||||
|
not blksWerePut
|
||||||
|
|
||||||
|
test "putBlocks, some puts failed, overall result is failure":
|
||||||
|
let
|
||||||
|
blk1 = Block.example
|
||||||
|
blk2 = Block.example
|
||||||
|
|
||||||
|
blockStore1.putBlock =
|
||||||
|
proc(self: BlockStoreMock, blk: Block): Future[bool] {.async.} =
|
||||||
|
return false
|
||||||
|
|
||||||
|
blockStore2.putBlock =
|
||||||
|
proc(self: BlockStoreMock, blk: Block): Future[bool] {.async.} =
|
||||||
|
return true
|
||||||
|
|
||||||
|
let blksWerePut = await mgr.putBlocks(@[blk1, blk2])
|
||||||
|
check:
|
||||||
|
not blksWerePut
|
||||||
|
|
||||||
|
test "putBlocks, some puts failed, overall result is success":
|
||||||
|
let
|
||||||
|
blk1 = Block.example
|
||||||
|
blk2 = Block.example
|
||||||
|
|
||||||
|
blockStore1.canFail = true
|
||||||
|
blockStore1.putBlock =
|
||||||
|
proc(self: BlockStoreMock, blk: Block): Future[bool] {.async.} =
|
||||||
|
return false
|
||||||
|
|
||||||
|
blockStore2.putBlock =
|
||||||
|
proc(self: BlockStoreMock, blk: Block): Future[bool] {.async.} =
|
||||||
|
return true
|
||||||
|
|
||||||
|
let blksWerePut = await mgr.putBlocks(@[blk1, blk2])
|
||||||
|
check:
|
||||||
|
blksWerePut
|
||||||
|
|
||||||
|
test "putBlocks, all blocks stored":
|
||||||
|
let
|
||||||
|
blk1 = Block.example
|
||||||
|
blk2 = Block.example
|
||||||
|
|
||||||
|
blockStore1.putBlock =
|
||||||
|
proc(self: BlockStoreMock, blk: Block): Future[bool] {.async.} =
|
||||||
|
return true
|
||||||
|
|
||||||
|
blockStore2.putBlock =
|
||||||
|
proc(self: BlockStoreMock, blk: Block): Future[bool] {.async.} =
|
||||||
|
return true
|
||||||
|
|
||||||
|
let blksWerePut = await mgr.putBlocks(@[blk1, blk2])
|
||||||
|
check:
|
||||||
|
blksWerePut
|
||||||
|
|
||||||
|
test "delBlock, all stores should successfully put block":
|
||||||
|
let blk = Block.example
|
||||||
|
|
||||||
|
blockStore1.delBlock =
|
||||||
|
proc(self: BlockStoreMock, cid: Cid): Future[bool] {.async.} =
|
||||||
|
check cid == blk.cid
|
||||||
|
return true
|
||||||
|
|
||||||
|
blockStore2.delBlock =
|
||||||
|
proc(self: BlockStoreMock, cid: Cid): Future[bool] {.async.} =
|
||||||
|
check cid == blk.cid
|
||||||
|
return true
|
||||||
|
|
||||||
|
let blkWasDeleted = await mgr.delBlock(blk.cid)
|
||||||
|
check:
|
||||||
|
blkWasDeleted
|
||||||
|
|
||||||
|
test "delBlock, one store should fail, result is failure":
|
||||||
|
let blk = Block.example
|
||||||
|
|
||||||
|
blockStore1.delBlock =
|
||||||
|
proc(self: BlockStoreMock, cid: Cid): Future[bool] {.async.} =
|
||||||
|
check cid == blk.cid
|
||||||
|
return true
|
||||||
|
|
||||||
|
blockStore2.delBlock =
|
||||||
|
proc(self: BlockStoreMock, cid: Cid): Future[bool] {.async.} =
|
||||||
|
check cid == blk.cid
|
||||||
|
return false
|
||||||
|
|
||||||
|
let blkWasDeleted = await mgr.delBlock(blk.cid)
|
||||||
|
check:
|
||||||
|
not blkWasDeleted
|
||||||
|
|
||||||
|
test "delBlock, one store should fail, result is failure, callback called":
|
||||||
|
let
|
||||||
|
blk = Block.example
|
||||||
|
fut = newFuture[bool]("delBlock test")
|
||||||
|
|
||||||
|
blockStore1.delBlock =
|
||||||
|
proc(self: BlockStoreMock, cid: Cid): Future[bool] {.async.} =
|
||||||
|
check cid == blk.cid
|
||||||
|
return true
|
||||||
|
|
||||||
|
blockStore2.onDelFail = proc(self: BlockStore, cid: Cid): Future[void] {.async.} =
|
||||||
|
fut.complete(true)
|
||||||
|
|
||||||
|
blockStore2.delBlock =
|
||||||
|
proc(self: BlockStoreMock, cid: Cid): Future[bool] {.async.} =
|
||||||
|
check cid == blk.cid
|
||||||
|
return false
|
||||||
|
|
||||||
|
let
|
||||||
|
blkWasDeleted = await mgr.delBlock(blk.cid)
|
||||||
|
delFailCalled = await fut.wait(5.seconds)
|
||||||
|
|
||||||
|
check:
|
||||||
|
not blkWasDeleted
|
||||||
|
delFailCalled
|
||||||
|
|
||||||
|
test "delBlock, one store should fail, result is success, callback called":
|
||||||
|
let
|
||||||
|
blk = Block.example
|
||||||
|
fut = newFuture[bool]("delBlock test")
|
||||||
|
|
||||||
|
blockStore1.delBlock =
|
||||||
|
proc(self: BlockStoreMock, cid: Cid): Future[bool] {.async.} =
|
||||||
|
check cid == blk.cid
|
||||||
|
return true
|
||||||
|
|
||||||
|
blockStore2.canFail = true
|
||||||
|
|
||||||
|
blockStore2.onDelFail = proc(self: BlockStore, cid: Cid): Future[void] {.async.} =
|
||||||
|
fut.complete(true)
|
||||||
|
|
||||||
|
blockStore2.delBlock =
|
||||||
|
proc(self: BlockStoreMock, cid: Cid): Future[bool] {.async.} =
|
||||||
|
check cid == blk.cid
|
||||||
|
return false
|
||||||
|
|
||||||
|
let
|
||||||
|
blkWasDeleted = await mgr.delBlock(blk.cid)
|
||||||
|
delFailCalled = await fut.wait(5.seconds)
|
||||||
|
|
||||||
|
check:
|
||||||
|
blkWasDeleted
|
||||||
|
delFailCalled
|
||||||
|
|
||||||
|
test "delBlock, all stores fail, result should be false":
|
||||||
|
let blk = Block.example
|
||||||
|
|
||||||
|
blockStore1.delBlock =
|
||||||
|
proc(self: BlockStoreMock, cid: Cid): Future[bool] {.async.} =
|
||||||
|
check cid == blk.cid
|
||||||
|
return false
|
||||||
|
|
||||||
|
blockStore2.delBlock =
|
||||||
|
proc(self: BlockStoreMock, cid: Cid): Future[bool] {.async.} =
|
||||||
|
check cid == blk.cid
|
||||||
|
return false
|
||||||
|
|
||||||
|
let blkWasDeleted = await mgr.delBlock(blk.cid)
|
||||||
|
check:
|
||||||
|
not blkWasDeleted
|
||||||
|
|
||||||
|
test "hasBlock, should have block in second block store":
|
||||||
|
let blk = Block.example
|
||||||
|
|
||||||
|
blockStore1.hasBlock =
|
||||||
|
proc(self: BlockStoreMock, cid: Cid): bool
|
||||||
|
{.raises: [Defect, AssertionError].} =
|
||||||
|
return false
|
||||||
|
|
||||||
|
blockStore2.hasBlock =
|
||||||
|
proc(self: BlockStoreMock, cid: Cid): bool
|
||||||
|
{.raises: [Defect, AssertionError].} =
|
||||||
|
return true
|
||||||
|
|
||||||
|
check:
|
||||||
|
mgr.hasBlock(blk.cid)
|
||||||
|
mgr.contains(blk.cid) # alias to hasBlock
|
||||||
|
|
||||||
|
test "hasBlock, should have block in first block store":
|
||||||
|
let blk = Block.example
|
||||||
|
var wasChecked = false
|
||||||
|
|
||||||
|
blockStore1.hasBlock =
|
||||||
|
proc(self: BlockStoreMock, cid: Cid): bool
|
||||||
|
{.raises: [Defect, AssertionError].} =
|
||||||
|
return true
|
||||||
|
|
||||||
|
blockStore2.hasBlock =
|
||||||
|
proc(self: BlockStoreMock, cid: Cid): bool
|
||||||
|
{.raises: [Defect, AssertionError].} =
|
||||||
|
wasChecked = true
|
||||||
|
return false
|
||||||
|
|
||||||
|
check:
|
||||||
|
mgr.hasBlock(blk.cid)
|
||||||
|
not wasChecked
|
||||||
|
|
||||||
|
test "hasBlock, no block found":
|
||||||
|
let blk = Block.example
|
||||||
|
|
||||||
|
blockStore1.hasBlock =
|
||||||
|
proc(self: BlockStoreMock, cid: Cid): bool
|
||||||
|
{.raises: [Defect, AssertionError].} =
|
||||||
|
return false
|
||||||
|
|
||||||
|
blockStore2.hasBlock =
|
||||||
|
proc(self: BlockStoreMock, cid: Cid): bool
|
||||||
|
{.raises: [Defect, AssertionError].} =
|
||||||
|
return false
|
||||||
|
|
||||||
|
check not mgr.hasBlock(blk.cid)
|
|
@ -27,9 +27,9 @@ suite "Test Node":
|
||||||
switch: Switch
|
switch: Switch
|
||||||
wallet: WalletRef
|
wallet: WalletRef
|
||||||
network: BlockExcNetwork
|
network: BlockExcNetwork
|
||||||
localStore: MemoryStore
|
localStore: BlockStore
|
||||||
engine: BlockExcEngine
|
engine: BlockExcEngine
|
||||||
store: NetworkStore
|
blockStoreMgr: BlockStoreManager
|
||||||
node: DaggerNodeRef
|
node: DaggerNodeRef
|
||||||
|
|
||||||
setup:
|
setup:
|
||||||
|
@ -39,9 +39,9 @@ suite "Test Node":
|
||||||
wallet = WalletRef.new(EthPrivateKey.random())
|
wallet = WalletRef.new(EthPrivateKey.random())
|
||||||
network = BlockExcNetwork.new(switch)
|
network = BlockExcNetwork.new(switch)
|
||||||
localStore = MemoryStore.new()
|
localStore = MemoryStore.new()
|
||||||
engine = BlockExcEngine.new(localStore, wallet, network)
|
blockStoreMgr = BlockStoreManager.new(@[localStore])
|
||||||
store = NetworkStore.new(engine, localStore)
|
engine = BlockExcEngine.new(wallet, network, blockStoreMgr)
|
||||||
node = DaggerNodeRef.new(switch, store, engine)
|
node = DaggerNodeRef.new(switch, engine)
|
||||||
|
|
||||||
await node.start()
|
await node.start()
|
||||||
|
|
||||||
|
|
|
@ -1,4 +1,5 @@
|
||||||
import ./stores/testfsstore
|
import ./stores/testfsstore
|
||||||
import ./stores/testmemorystore
|
import ./stores/testmemorystore
|
||||||
|
import ./stores/testmanager
|
||||||
|
|
||||||
{.warning[UnusedImport]: off.}
|
{.warning[UnusedImport]: off.}
|
||||||
|
|
Loading…
Reference in New Issue