rename `BlockError` -> `VerifierError` (#4310)

We currently use `BlockError` for both beacon blocks and LC objects.
In light of EIP4844, we will likely also use it for blob sidecars.
To avoid confusion, renaming it to a more generic `VerifierError`,
and update its documentation to be more generic.

To avoid long lines as a followup, also renaming the `block_processor`'s
`BlockProcessingCompleted.completed`->`ProcessingStatus.completed` and
`BlockProcessingCompleted.notCompleted`->`ProcessingStatus.notCompleted`
This commit is contained in:
Etan Kissling 2022-11-10 18:40:27 +01:00 committed by GitHub
parent 2256a22594
commit 48994f67d3
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
17 changed files with 176 additions and 170 deletions

View File

@ -121,7 +121,7 @@ proc addResolvedHeadBlock(
proc checkStateTransition(
dag: ChainDAGRef, signedBlock: ForkySigVerifiedSignedBeaconBlock,
cache: var StateCache): Result[void, BlockError] =
cache: var StateCache): Result[void, VerifierError] =
## Ensure block can be applied on a state
func restore(v: var ForkedHashedBeaconState) =
assign(dag.clearanceState, dag.headState)
@ -136,7 +136,7 @@ proc checkStateTransition(
blck = shortLog(signedBlock.message),
error = res.error()
err(BlockError.Invalid)
err(VerifierError.Invalid)
else:
ok()
@ -167,7 +167,7 @@ proc addHeadBlock*(
signedBlock: ForkySignedBeaconBlock,
blockVerified: bool,
onBlockAdded: OnForkyBlockAdded
): Result[BlockRef, BlockError] =
): Result[BlockRef, VerifierError] =
## Try adding a block to the chain, verifying first that it passes the state
## transition function and contains correct cryptographic signature.
##
@ -193,7 +193,7 @@ proc addHeadBlock*(
if existing.get().bid.slot == blck.slot and
existing.get().bid.root == blockRoot:
debug "Duplicate block"
return err(BlockError.Duplicate)
return err(VerifierError.Duplicate)
# Block is older than finalized, but different from the block in our
# canonical history: it must be from an unviable branch
@ -202,11 +202,11 @@ proc addHeadBlock*(
finalizedHead = shortLog(dag.finalizedHead),
tail = shortLog(dag.tail)
return err(BlockError.UnviableFork)
return err(VerifierError.UnviableFork)
# Check non-finalized blocks as well
if dag.containsForkBlock(blockRoot):
return err(BlockError.Duplicate)
return err(VerifierError.Duplicate)
let parent = dag.getBlockRef(blck.parent_root).valueOr:
# There are two cases where the parent won't be found: we don't have it or
@ -218,10 +218,10 @@ proc addHeadBlock*(
if parentId.isSome() and parentId.get.slot < dag.finalizedHead.slot:
debug "Block unviable due to pre-finalized-checkpoint parent",
parentId = parentId.get()
return err(BlockError.UnviableFork)
return err(VerifierError.UnviableFork)
debug "Block parent unknown or finalized already", parentId
return err(BlockError.MissingParent)
return err(VerifierError.MissingParent)
if parent.slot >= blck.slot:
# A block whose parent is newer than the block itself is clearly invalid -
@ -229,7 +229,7 @@ proc addHeadBlock*(
debug "Block older than parent",
parent = shortLog(parent)
return err(BlockError.Invalid)
return err(VerifierError.Invalid)
# The block is resolved, now it's time to validate it to ensure that the
# blocks we add to the database are clean for the given state
@ -254,7 +254,7 @@ proc addHeadBlock*(
# load its corresponding state
error "Unable to load clearance state for parent block, database corrupt?",
clearanceBlock = shortLog(clearanceBlock)
return err(BlockError.MissingParent)
return err(VerifierError.MissingParent)
let stateDataTick = Moment.now()
@ -268,12 +268,12 @@ proc addHeadBlock*(
# A PublicKey or Signature isn't on the BLS12-381 curve
info "Unable to load signature sets",
err = e.error()
return err(BlockError.Invalid)
return err(VerifierError.Invalid)
if not verifier.batchVerify(sigs):
info "Block signature verification failed",
signature = shortLog(signedBlock.signature)
return err(BlockError.Invalid)
return err(VerifierError.Invalid)
let sigVerifyTick = Moment.now()
@ -296,14 +296,14 @@ proc addHeadBlock*(
dag: ChainDAGRef, verifier: var BatchVerifier,
signedBlock: ForkySignedBeaconBlock,
onBlockAdded: OnForkyBlockAdded
): Result[BlockRef, BlockError] =
): Result[BlockRef, VerifierError] =
addHeadBlock(
dag, verifier, signedBlock, blockVerified = true, onBlockAdded)
proc addBackfillBlock*(
dag: ChainDAGRef,
signedBlock: ForkySignedBeaconBlock | ForkySigVerifiedSignedBeaconBlock):
Result[void, BlockError] =
Result[void, VerifierError] =
## When performing checkpoint sync, we need to backfill historical blocks
## in order to respond to GetBlocksByRange requests. Backfill blocks are
## added in backwards order, one by one, based on the `parent_root` of the
@ -328,7 +328,7 @@ proc addBackfillBlock*(
# The genesis block must have an empty signature (since there's no proposer)
if signedBlock.signature != ValidatorSig():
info "Invalid genesis block signature"
return err(BlockError.Invalid)
return err(VerifierError.Invalid)
else:
let proposerKey = dag.validatorKey(blck.proposer_index)
if proposerKey.isNone():
@ -351,7 +351,7 @@ proc addBackfillBlock*(
proposerKey.get(),
signedBlock.signature):
info "Block signature verification failed"
return err(BlockError.Invalid)
return err(VerifierError.Invalid)
let startTick = Moment.now()
@ -371,7 +371,7 @@ proc addBackfillBlock*(
return ok()
debug "Duplicate block"
return err(BlockError.Duplicate)
return err(VerifierError.Duplicate)
# Block is older than finalized, but different from the block in our
# canonical history: it must be from an unviable branch
@ -379,7 +379,7 @@ proc addBackfillBlock*(
existing = shortLog(existing.get()),
finalizedHead = shortLog(dag.finalizedHead)
return err(BlockError.UnviableFork)
return err(VerifierError.UnviableFork)
if dag.frontfill.isSome():
let frontfill = dag.frontfill.get()
@ -405,11 +405,11 @@ proc addBackfillBlock*(
# Backfill done - dag.backfill.slot now points to genesis block just like
# it would if we loaded a fully synced database - returning duplicate
# here is appropriate, though one could also call it ... ok?
return err(BlockError.Duplicate)
return err(VerifierError.Duplicate)
if dag.backfill.parent_root != blockRoot:
debug "Block does not match expected backfill root"
return err(BlockError.MissingParent) # MissingChild really, but ..
return err(VerifierError.MissingParent) # MissingChild really, but ..
checkSignature()

View File

@ -34,22 +34,22 @@ export
# relationships and allowing various forms of lookups
type
BlockError* {.pure.} = enum
VerifierError* {.pure.} = enum
Invalid
## Block is broken / doesn't apply cleanly - whoever sent it is fishy (or
## Value is broken / doesn't apply cleanly - whoever sent it is fishy (or
## we're buggy)
MissingParent
## We don't know the parent of this block so we can't tell if it's valid
## We don't know the parent of this value so we can't tell if it's valid
## or not - it'll go into the quarantine and be reexamined when the parent
## appears or be discarded if finality obsoletes it
UnviableFork
## Block is from a history / fork that does not include our most current
## Value is from a history / fork that does not include our most current
## finalized checkpoint
Duplicate
## We've seen this block already, can't add again
## We've seen this value already, can't add again
OnBlockCallback* =
proc(data: ForkedTrustedSignedBeaconBlock) {.gcsafe, raises: [Defect].}

View File

@ -23,7 +23,8 @@ from ../consensus_object_pools/consensus_manager import
updateHeadWithExecution
from ../beacon_clock import GetBeaconTimeFn, toFloatSeconds
from ../consensus_object_pools/block_dag import BlockRef, root, slot
from ../consensus_object_pools/block_pools_types import BlockError, EpochRef
from ../consensus_object_pools/block_pools_types import
EpochRef, VerifierError
from ../consensus_object_pools/block_quarantine import
addOrphan, addUnviable, pop, removeOrphan
from ../validators/validator_monitor import
@ -42,7 +43,7 @@ declareHistogram beacon_store_block_duration_seconds,
type
BlockEntry* = object
blck*: ForkedSignedBeaconBlock
resfut*: Future[Result[void, BlockError]]
resfut*: Future[Result[void, VerifierError]]
queueTick*: Moment # Moment when block was enqueued
validationDur*: Duration # Time it took to perform gossip validation
src*: MsgSource
@ -95,13 +96,13 @@ type
invalid
noResponse
BlockProcessingCompleted {.pure.} = enum
ProcessingStatus {.pure.} = enum
completed
notCompleted
proc addBlock*(
self: var BlockProcessor, src: MsgSource, blck: ForkedSignedBeaconBlock,
resfut: Future[Result[void, BlockError]] = nil,
resfut: Future[Result[void, VerifierError]] = nil,
validationDur = Duration())
# Initialization
@ -144,12 +145,12 @@ proc dumpInvalidBlock*(
proc dumpBlock[T](
self: BlockProcessor,
signedBlock: ForkySignedBeaconBlock,
res: Result[T, BlockError]) =
res: Result[T, VerifierError]) =
if self.dumpEnabled and res.isErr:
case res.error
of BlockError.Invalid:
of VerifierError.Invalid:
self.dumpInvalidBlock(signedBlock)
of BlockError.MissingParent:
of VerifierError.MissingParent:
dump(self.dumpDirIncoming, signedBlock)
else:
discard
@ -159,7 +160,7 @@ from ../consensus_object_pools/block_clearance import
proc storeBackfillBlock(
self: var BlockProcessor,
signedBlock: ForkySignedBeaconBlock): Result[void, BlockError] =
signedBlock: ForkySignedBeaconBlock): Result[void, VerifierError] =
# The block is certainly not missing any more
self.consensusManager.quarantine[].missing.del(signedBlock.root)
@ -168,15 +169,15 @@ proc storeBackfillBlock(
if res.isErr():
case res.error
of BlockError.MissingParent:
of VerifierError.MissingParent:
if signedBlock.message.parent_root in
self.consensusManager.quarantine[].unviable:
# DAG doesn't know about unviable ancestor blocks - we do! Translate
# this to the appropriate error so that sync etc doesn't retry the block
self.consensusManager.quarantine[].addUnviable(signedBlock.root)
return err(BlockError.UnviableFork)
of BlockError.UnviableFork:
return err(VerifierError.UnviableFork)
of VerifierError.UnviableFork:
# Track unviables so that descendants can be discarded properly
self.consensusManager.quarantine[].addUnviable(signedBlock.root)
else: discard
@ -348,7 +349,7 @@ proc storeBlock*(
self: ref BlockProcessor, src: MsgSource, wallTime: BeaconTime,
signedBlock: ForkySignedBeaconBlock, queueTick: Moment = Moment.now(),
validationDur = Duration()):
Future[Result[BlockRef, (BlockError, BlockProcessingCompleted)]] {.async.} =
Future[Result[BlockRef, (VerifierError, ProcessingStatus)]] {.async.} =
## storeBlock is the main entry point for unvalidated blocks - all untrusted
## blocks, regardless of origin, pass through here. When storing a block,
## we will add it to the dag and pass it to all block consumers that need
@ -367,7 +368,7 @@ proc storeBlock*(
if NewPayloadStatus.invalid == payloadStatus:
self.consensusManager.quarantine[].addUnviable(signedBlock.root)
return err((BlockError.UnviableFork, BlockProcessingCompleted.completed))
return err((VerifierError.UnviableFork, ProcessingStatus.completed))
if NewPayloadStatus.noResponse == payloadStatus and not self[].optimistic:
# Disallow the `MissingParent` from leaking to the sync/request managers
@ -376,7 +377,7 @@ proc storeBlock*(
# directly, so is exposed to this, but only cares about whether there is
# an error or not.
return err((
BlockError.MissingParent, BlockProcessingCompleted.notCompleted))
VerifierError.MissingParent, ProcessingStatus.notCompleted))
# We'll also remove the block as an orphan: it's unlikely the parent is
# missing if we get this far - should that be the case, the block will
@ -414,14 +415,14 @@ proc storeBlock*(
# was pruned from the ForkChoice.
if blck.isErr():
case blck.error()
of BlockError.MissingParent:
of VerifierError.MissingParent:
if signedBlock.message.parent_root in
self.consensusManager.quarantine[].unviable:
# DAG doesn't know about unviable ancestor blocks - we do! Translate
# this to the appropriate error so that sync etc doesn't retry the block
self.consensusManager.quarantine[].addUnviable(signedBlock.root)
return err((BlockError.UnviableFork, BlockProcessingCompleted.completed))
return err((VerifierError.UnviableFork, ProcessingStatus.completed))
if not self.consensusManager.quarantine[].addOrphan(
dag.finalizedHead.slot, ForkedSignedBeaconBlock.init(signedBlock)):
@ -429,12 +430,12 @@ proc storeBlock*(
blockRoot = shortLog(signedBlock.root),
blck = shortLog(signedBlock.message),
signature = shortLog(signedBlock.signature)
of BlockError.UnviableFork:
of VerifierError.UnviableFork:
# Track unviables so that descendants can be discarded properly
self.consensusManager.quarantine[].addUnviable(signedBlock.root)
else: discard
return err((blck.error, BlockProcessingCompleted.completed))
return err((blck.error, ProcessingStatus.completed))
let storeBlockTick = Moment.now()
@ -533,14 +534,14 @@ proc storeBlock*(
# Process the blocks that had the newly accepted block as parent
self[].addBlock(MsgSource.gossip, quarantined)
return Result[BlockRef, (BlockError, BlockProcessingCompleted)].ok blck.get
return Result[BlockRef, (VerifierError, ProcessingStatus)].ok blck.get
# Enqueue
# ------------------------------------------------------------------------------
proc addBlock*(
self: var BlockProcessor, src: MsgSource, blck: ForkedSignedBeaconBlock,
resfut: Future[Result[void, BlockError]] = nil,
resfut: Future[Result[void, VerifierError]] = nil,
validationDur = Duration()) =
## Enqueue a Gossip-validated block for consensus verification
# Backpressure:
@ -590,7 +591,7 @@ proc processBlock(
await self.storeBlock(
entry.src, wallTime, blck, entry.queueTick, entry.validationDur)
if res.isErr and res.error[1] == BlockProcessingCompleted.notCompleted:
if res.isErr and res.error[1] == ProcessingStatus.notCompleted:
# When an execution engine returns an error or fails to respond to a
# payload validity request for some block, a consensus engine:
# - MUST NOT optimistically import the block.
@ -606,8 +607,8 @@ proc processBlock(
if entry.resfut != nil:
entry.resfut.complete(
if res.isOk(): Result[void, BlockError].ok()
else: Result[void, BlockError].err(res.error()[0]))
if res.isOk(): Result[void, VerifierError].ok()
else: Result[void, VerifierError].err(res.error()[0]))
proc runQueueProcessingLoop*(self: ref BlockProcessor) {.async.} =
while true:

View File

@ -155,12 +155,12 @@ proc dumpInvalidObject(
proc dumpObject[T](
self: LightClientProcessor,
obj: SomeLightClientObject,
res: Result[T, BlockError]) =
res: Result[T, VerifierError]) =
if self.dumpEnabled and res.isErr:
case res.error
of BlockError.Invalid:
of VerifierError.Invalid:
self.dumpInvalidObject(obj)
of BlockError.MissingParent:
of VerifierError.MissingParent:
dump(self.dumpDirIncoming, obj)
else:
discard
@ -190,18 +190,18 @@ proc tryForceUpdate(
proc processObject(
self: var LightClientProcessor,
obj: SomeLightClientObject,
wallTime: BeaconTime): Result[void, BlockError] =
wallTime: BeaconTime): Result[void, VerifierError] =
let
wallSlot = wallTime.slotOrZero()
store = self.store
res =
when obj is altair.LightClientBootstrap:
if store[].isSome:
err(BlockError.Duplicate)
err(VerifierError.Duplicate)
else:
let trustedBlockRoot = self.getTrustedBlockRoot()
if trustedBlockRoot.isNone:
err(BlockError.MissingParent)
err(VerifierError.MissingParent)
else:
let initRes =
initialize_light_client_store(trustedBlockRoot.get, obj)
@ -212,7 +212,7 @@ proc processObject(
ok()
elif obj is SomeLightClientUpdate:
if store[].isNone:
err(BlockError.MissingParent)
err(VerifierError.MissingParent)
else:
store[].get.process_light_client_update(
obj, wallSlot, self.cfg, self.genesis_validators_root)
@ -228,7 +228,7 @@ proc processObject(
# If none is made available within reasonable time, the light client
# is force-updated using the best known data to ensure sync progress.
case res.error
of BlockError.Duplicate:
of VerifierError.Duplicate:
if wallTime >= self.lastDuplicateTick + duplicateRateLimit:
if self.numDuplicatesSinceProgress < minForceUpdateDuplicates:
if obj.matches(store[].get.best_valid_update.get):
@ -296,7 +296,7 @@ template withReportedProgress(body: untyped): bool =
proc storeObject*(
self: var LightClientProcessor,
src: MsgSource, wallTime: BeaconTime,
obj: SomeLightClientObject): Result[bool, BlockError] =
obj: SomeLightClientObject): Result[bool, VerifierError] =
## storeObject is the main entry point for unvalidated light client objects -
## all untrusted objects pass through here. When storing an object, we will
## update the `LightClientStore` accordingly
@ -353,7 +353,7 @@ proc addObject*(
self: var LightClientProcessor,
src: MsgSource,
obj: SomeLightClientObject,
resfut: Future[Result[void, BlockError]] = nil) =
resfut: Future[Result[void, VerifierError]] = nil) =
## Enqueue a Gossip-validated light client object for verification
# Backpressure:
# Only one object is validated at any time -
@ -380,16 +380,16 @@ proc addObject*(
if resfut != nil:
if res.isOk:
resfut.complete(Result[void, BlockError].ok())
resfut.complete(Result[void, VerifierError].ok())
else:
resfut.complete(Result[void, BlockError].err(res.error))
resfut.complete(Result[void, VerifierError].err(res.error))
# Message validators
# ------------------------------------------------------------------------------
func toValidationError(
self: var LightClientProcessor,
r: Result[bool, BlockError],
r: Result[bool, VerifierError],
wallTime: BeaconTime,
obj: SomeLightClientObject): Result[void, ValidationError] =
if r.isOk:
@ -420,11 +420,13 @@ func toValidationError(
errIgnore(typeof(obj).name & ": no significant progress")
else:
case r.error
of BlockError.Invalid:
of VerifierError.Invalid:
# [REJECT] The `finality_update` is valid.
# [REJECT] The `optimistic_update` is valid.
errReject($r.error)
of BlockError.MissingParent, BlockError.UnviableFork, BlockError.Duplicate:
of VerifierError.MissingParent,
VerifierError.UnviableFork,
VerifierError.Duplicate:
# [IGNORE] No other `finality_update` with a lower or equal
# `finalized_header.slot` was already forwarded on the network.
# [IGNORE] No other `optimistic_update` with a lower or equal

View File

@ -94,8 +94,8 @@ proc createLightClient(
onStoreInitialized, onFinalizedHeader, onOptimisticHeader)
proc lightClientVerifier(obj: SomeLightClientObject):
Future[Result[void, BlockError]] =
let resfut = newFuture[Result[void, BlockError]]("lightClientVerifier")
Future[Result[void, VerifierError]] =
let resfut = newFuture[Result[void, VerifierError]]("lightClientVerifier")
lightClient.processor[].addObject(MsgSource.gossip, obj, resfut)
resfut
proc bootstrapVerifier(obj: altair.LightClientBootstrap): auto =

View File

@ -303,12 +303,12 @@ proc initFullNode(
rng, taskpool, consensusManager, node.validatorMonitor, getBeaconTime,
optimistic = config.optimistic)
blockVerifier = proc(signedBlock: ForkedSignedBeaconBlock):
Future[Result[void, BlockError]] =
Future[Result[void, VerifierError]] =
# The design with a callback for block verification is unusual compared
# to the rest of the application, but fits with the general approach
# taken in the sync/request managers - this is an architectural compromise
# that should probably be reimagined more holistically in the future.
let resfut = newFuture[Result[void, BlockError]]("blockVerifier")
let resfut = newFuture[Result[void, VerifierError]]("blockVerifier")
blockProcessor[].addBlock(MsgSource.gossip, signedBlock, resfut)
resfut
processor = Eth2Processor.new(

View File

@ -15,16 +15,16 @@ import
datatypes/altair,
helpers
from ../consensus_object_pools/block_pools_types import BlockError
export block_pools_types.BlockError
from ../consensus_object_pools/block_pools_types import VerifierError
export block_pools_types.VerifierError
# https://github.com/ethereum/consensus-specs/blob/v1.3.0-alpha.0/specs/altair/light-client/sync-protocol.md#initialize_light_client_store
func initialize_light_client_store*(
trusted_block_root: Eth2Digest,
bootstrap: altair.LightClientBootstrap
): Result[LightClientStore, BlockError] =
): Result[LightClientStore, VerifierError] =
if hash_tree_root(bootstrap.header) != trusted_block_root:
return err(BlockError.Invalid)
return err(VerifierError.Invalid)
if not is_valid_merkle_branch(
hash_tree_root(bootstrap.current_sync_committee),
@ -32,7 +32,7 @@ func initialize_light_client_store*(
log2trunc(altair.CURRENT_SYNC_COMMITTEE_INDEX),
get_subtree_index(altair.CURRENT_SYNC_COMMITTEE_INDEX),
bootstrap.header.state_root):
return err(BlockError.Invalid)
return err(VerifierError.Invalid)
return ok(LightClientStore(
finalized_header: bootstrap.header,
@ -45,32 +45,32 @@ proc validate_light_client_update*(
update: SomeLightClientUpdate,
current_slot: Slot,
cfg: RuntimeConfig,
genesis_validators_root: Eth2Digest): Result[void, BlockError] =
genesis_validators_root: Eth2Digest): Result[void, VerifierError] =
# Verify sync committee has sufficient participants
template sync_aggregate(): auto = update.sync_aggregate
template sync_committee_bits(): auto = sync_aggregate.sync_committee_bits
let num_active_participants = countOnes(sync_committee_bits).uint64
if num_active_participants < MIN_SYNC_COMMITTEE_PARTICIPANTS:
return err(BlockError.Invalid)
return err(VerifierError.Invalid)
# Verify update does not skip a sync committee period
when update is SomeLightClientUpdateWithFinality:
if update.attested_header.slot < update.finalized_header.slot:
return err(BlockError.Invalid)
return err(VerifierError.Invalid)
if update.signature_slot <= update.attested_header.slot:
return err(BlockError.Invalid)
return err(VerifierError.Invalid)
if current_slot < update.signature_slot:
return err(BlockError.UnviableFork)
return err(VerifierError.UnviableFork)
let
store_period = store.finalized_header.slot.sync_committee_period
signature_period = update.signature_slot.sync_committee_period
is_next_sync_committee_known = store.is_next_sync_committee_known
if is_next_sync_committee_known:
if signature_period notin [store_period, store_period + 1]:
return err(BlockError.MissingParent)
return err(VerifierError.MissingParent)
else:
if signature_period != store_period:
return err(BlockError.MissingParent)
return err(VerifierError.MissingParent)
# Verify update is relevant
let attested_period = update.attested_header.slot.sync_committee_period
@ -79,18 +79,18 @@ proc validate_light_client_update*(
if update.attested_header.slot <= store.finalized_header.slot:
when update is SomeLightClientUpdateWithSyncCommittee:
if is_next_sync_committee_known:
return err(BlockError.Duplicate)
return err(VerifierError.Duplicate)
if attested_period != store_period or not is_sync_committee_update:
return err(BlockError.Duplicate)
return err(VerifierError.Duplicate)
else:
return err(BlockError.Duplicate)
return err(VerifierError.Duplicate)
# Verify that the `finalized_header`, if present, actually is the
# finalized header saved in the state of the `attested_header`
when update is SomeLightClientUpdateWithFinality:
if not update.is_finality_update:
if not update.finalized_header.isZeroMemory:
return err(BlockError.Invalid)
return err(VerifierError.Invalid)
else:
var finalized_root {.noinit.}: Eth2Digest
if update.finalized_header.slot != GENESIS_SLOT:
@ -98,32 +98,32 @@ proc validate_light_client_update*(
elif update.finalized_header.isZeroMemory:
finalized_root.reset()
else:
return err(BlockError.Invalid)
return err(VerifierError.Invalid)
if not is_valid_merkle_branch(
finalized_root,
update.finality_branch,
log2trunc(altair.FINALIZED_ROOT_INDEX),
get_subtree_index(altair.FINALIZED_ROOT_INDEX),
update.attested_header.state_root):
return err(BlockError.Invalid)
return err(VerifierError.Invalid)
# Verify that the `next_sync_committee`, if present, actually is the
# next sync committee saved in the state of the `attested_header`
when update is SomeLightClientUpdateWithSyncCommittee:
if not is_sync_committee_update:
if not update.next_sync_committee.isZeroMemory:
return err(BlockError.Invalid)
return err(VerifierError.Invalid)
else:
if attested_period == store_period and is_next_sync_committee_known:
if update.next_sync_committee != store.next_sync_committee:
return err(BlockError.UnviableFork)
return err(VerifierError.UnviableFork)
if not is_valid_merkle_branch(
hash_tree_root(update.next_sync_committee),
update.next_sync_committee_branch,
log2trunc(altair.NEXT_SYNC_COMMITTEE_INDEX),
get_subtree_index(altair.NEXT_SYNC_COMMITTEE_INDEX),
update.attested_header.state_root):
return err(BlockError.Invalid)
return err(VerifierError.Invalid)
# Verify sync committee aggregate signature
let sync_committee =
@ -144,7 +144,7 @@ proc validate_light_client_update*(
if not blsFastAggregateVerify(
participant_pubkeys, signing_root.data,
sync_aggregate.sync_committee_signature):
return err(BlockError.UnviableFork)
return err(VerifierError.UnviableFork)
ok()
@ -213,7 +213,7 @@ proc process_light_client_update*(
update: SomeLightClientUpdate,
current_slot: Slot,
cfg: RuntimeConfig,
genesis_validators_root: Eth2Digest): Result[void, BlockError] =
genesis_validators_root: Eth2Digest): Result[void, VerifierError] =
? validate_light_client_update(
store, update, current_slot, cfg, genesis_validators_root)
@ -257,5 +257,5 @@ proc process_light_client_update*(
store.best_valid_update.reset()
if not didProgress:
return err(BlockError.Duplicate)
return err(VerifierError.Duplicate)
ok()

View File

@ -37,7 +37,7 @@ type
Endpoint[Nothing, altair.LightClientOptimisticUpdate]
ValueVerifier[V] =
proc(v: V): Future[Result[void, BlockError]] {.gcsafe, raises: [Defect].}
proc(v: V): Future[Result[void, VerifierError]] {.gcsafe, raises: [Defect].}
BootstrapVerifier* =
ValueVerifier[altair.LightClientBootstrap]
UpdateVerifier* =
@ -226,22 +226,22 @@ proc workerTask[E](
let res = await self.valueVerifier(E)(val)
if res.isErr:
case res.error
of BlockError.MissingParent:
of VerifierError.MissingParent:
# Stop, requires different request to progress
return didProgress
of BlockError.Duplicate:
of VerifierError.Duplicate:
# Ignore, a concurrent request may have already fulfilled this
when E.V is altair.LightClientBootstrap:
didProgress = true
else:
discard
of BlockError.UnviableFork:
of VerifierError.UnviableFork:
# Descore, peer is on an incompatible fork version
notice "Received value from an unviable fork", value = val.shortLog,
endpoint = E.name, peer, peer_score = peer.getScore()
peer.updateScore(PeerScoreUnviableFork)
return didProgress
of BlockError.Invalid:
of VerifierError.Invalid:
# Descore, received data is malformed
warn "Received invalid value", value = val.shortLog,
endpoint = E.name, peer, peer_score = peer.getScore()

View File

@ -33,7 +33,7 @@ const
type
BlockVerifier* =
proc(signedBlock: ForkedSignedBeaconBlock):
Future[Result[void, BlockError]] {.gcsafe, raises: [Defect].}
Future[Result[void, VerifierError]] {.gcsafe, raises: [Defect].}
RequestManager* = object
network*: Eth2Node
@ -90,21 +90,21 @@ proc fetchAncestorBlocksFromNetwork(rman: RequestManager,
let ver = await rman.blockVerifier(b[])
if ver.isErr():
case ver.error()
of BlockError.MissingParent:
of VerifierError.MissingParent:
# Ignoring because the order of the blocks that
# we requested may be different from the order in which we need
# these blocks to apply.
discard
of BlockError.Duplicate:
of VerifierError.Duplicate:
# Ignoring because these errors could occur due to the
# concurrent/parallel requests we made.
discard
of BlockError.UnviableFork:
of VerifierError.UnviableFork:
# If they're working a different fork, we'll want to descore them
# but also process the other blocks (in case we can register the
# other blocks as unviable)
gotUnviableBlock = true
of BlockError.Invalid:
of VerifierError.Invalid:
# We stop processing blocks because peer is either sending us
# junk or working a different fork
notice "Received invalid block",

View File

@ -30,7 +30,7 @@ type
ProcessingCallback* = proc() {.gcsafe, raises: [Defect].}
BlockVerifier* =
proc(signedBlock: ForkedSignedBeaconBlock):
Future[Result[void, BlockError]] {.gcsafe, raises: [Defect].}
Future[Result[void, VerifierError]] {.gcsafe, raises: [Defect].}
SyncQueueKind* {.pure.} = enum
Forward, Backward
@ -666,7 +666,7 @@ proc push*[T](sq: SyncQueue[T], sr: SyncRequest[T],
# compiler segfault if this is moved into the for loop, at time of writing
# TODO this does segfault in 1.2 but not 1.6, so remove workaround when 1.2
# is dropped.
res: Result[void, BlockError]
res: Result[void, VerifierError]
for blk in sq.blocks(item):
res = await sq.blockVerifier(blk[])
@ -674,20 +674,20 @@ proc push*[T](sq: SyncQueue[T], sr: SyncRequest[T],
goodBlock = some(blk[].slot)
else:
case res.error()
of BlockError.MissingParent:
of VerifierError.MissingParent:
missingParentSlot = some(blk[].slot)
break
of BlockError.Duplicate:
of VerifierError.Duplicate:
# Keep going, happens naturally
discard
of BlockError.UnviableFork:
of VerifierError.UnviableFork:
# Keep going so as to register other unviable blocks with the
# quarantine
if unviableBlock.isNone:
# Remember the first unviable block, so we can log it
unviableBlock = some((blk[].root, blk[].slot))
of BlockError.Invalid:
of VerifierError.Invalid:
hasInvalidBlock = true
let req = item.request
@ -747,10 +747,10 @@ proc push*[T](sq: SyncQueue[T], sr: SyncRequest[T],
resetSlot: Option[Slot]
failSlot = missingParentSlot.get()
# If we got `BlockError.MissingParent` it means that peer returns chain
# of blocks with holes or `block_pool` is in incomplete state. We going
# to rewind the SyncQueue some distance back (2ⁿ, where n∈[0,∞], but
# no more than `finalized_epoch`).
# If we got `VerifierError.MissingParent` it means that peer returns
# chain of blocks with holes or `block_pool` is in incomplete state. We
# going to rewind the SyncQueue some distance back (2ⁿ, where n∈[0,∞],
# but no more than `finalized_epoch`).
let
req = item.request
safeSlot = sq.getSafeSlot()
@ -763,7 +763,7 @@ proc push*[T](sq: SyncQueue[T], sr: SyncRequest[T],
case sq.kind
of SyncQueueKind.Forward:
if goodBlock.isSome():
# `BlockError.MissingParent` and `Success` present in response,
# `VerifierError.MissingParent` and `Success` present in response,
# it means that we just need to request this range one more time.
debug "Unexpected missing parent, but no rewind needed",
request = req, finalized_slot = safeSlot,

View File

@ -223,11 +223,13 @@ proc doTrustedNodeSync*(
withBlck(data[]):
if (let res = dag.addBackfillBlock(blck.asSigVerified()); res.isErr()):
case res.error()
of BlockError.Invalid, BlockError.MissingParent, BlockError.UnviableFork:
of VerifierError.Invalid,
VerifierError.MissingParent,
VerifierError.UnviableFork:
error "Got invalid block from trusted node - is it on the right network?",
blck = shortLog(blck), err = res.error()
quit 1
of BlockError.Duplicate:
of VerifierError.Duplicate:
discard
# Download blocks backwards from the backfill slot, ie the first slot for

View File

@ -202,7 +202,7 @@ proc stepOnBlock(
signedBlock: ForkySignedBeaconBlock,
time: BeaconTime,
invalidatedRoots: Table[Eth2Digest, Eth2Digest]):
Result[BlockRef, BlockError] =
Result[BlockRef, VerifierError] =
# 1. Move state to proper slot.
doAssert dag.updateState(
state,
@ -246,7 +246,7 @@ proc stepOnBlock(
fkChoice[].mark_root_invalid(dag.getEarliestInvalidBlockRoot(
signedBlock.message.parent_root, lvh, executionPayloadHash))
return err BlockError.Invalid
return err VerifierError.Invalid
let blockAdded = dag.addHeadBlock(verifier, signedBlock) do (
blckRef: BlockRef, signedBlock: TrustedBlock,

View File

@ -635,7 +635,7 @@ suite "Attestation pool processing" & preset():
epochRef, blckRef, unrealized, signedBlock.message,
blckRef.slot.start_beacon_time)
doAssert: b10Add_clone.error == BlockError.Duplicate
doAssert: b10Add_clone.error == VerifierError.Duplicate
test "Trying to add a duplicate block from an old pruned epoch is tagged as an error":
# Note: very sensitive to stack usage
@ -725,4 +725,4 @@ suite "Attestation pool processing" & preset():
epochRef, blckRef, unrealized, signedBlock.message,
blckRef.slot.start_beacon_time)
doAssert: b10Add_clone.error == BlockError.Duplicate
doAssert: b10Add_clone.error == VerifierError.Duplicate

View File

@ -60,7 +60,7 @@ suite "Block processor" & preset():
asyncTest "Reverse order block add & get" & preset():
let missing = await processor.storeBlock(
MsgSource.gossip, b2.message.slot.start_beacon_time(), b2)
check: missing.error[0] == BlockError.MissingParent
check: missing.error[0] == VerifierError.MissingParent
check:
not dag.containsForkBlock(b2.root) # Unresolved, shouldn't show up

View File

@ -29,8 +29,8 @@ proc pruneAtFinalization(dag: ChainDAGRef) =
dag.pruneStateCachesDAG()
type
AddHeadRes = Result[BlockRef, BlockError]
AddBackRes = Result[void, BlockError]
AddHeadRes = Result[BlockRef, VerifierError]
AddBackRes = Result[void, VerifierError]
suite "Block pool processing" & preset():
setup:
@ -273,7 +273,7 @@ suite "Block pool processing" & preset():
b11 = dag.addHeadBlock(verifier, b1, nilPhase0Callback)
check:
b11 == AddHeadRes.err BlockError.Duplicate
b11 == AddHeadRes.err VerifierError.Duplicate
not b10[].isNil
test "updateHead updates head and headState" & preset():
@ -382,7 +382,7 @@ suite "Block pool altair processing" & preset():
let
bAdd = dag.addHeadBlock(verifier, b, nilAltairCallback)
check:
bAdd == AddHeadRes.err BlockError.Invalid
bAdd == AddHeadRes.err VerifierError.Invalid
block: # Randao reveal
var b = b2
@ -390,7 +390,7 @@ suite "Block pool altair processing" & preset():
let
bAdd = dag.addHeadBlock(verifier, b, nilAltairCallback)
check:
bAdd == AddHeadRes.err BlockError.Invalid
bAdd == AddHeadRes.err VerifierError.Invalid
block: # Attestations
var b = b2
@ -398,7 +398,7 @@ suite "Block pool altair processing" & preset():
let
bAdd = dag.addHeadBlock(verifier, b, nilAltairCallback)
check:
bAdd == AddHeadRes.err BlockError.Invalid
bAdd == AddHeadRes.err VerifierError.Invalid
block: # SyncAggregate empty
var b = b2
@ -406,7 +406,7 @@ suite "Block pool altair processing" & preset():
let
bAdd = dag.addHeadBlock(verifier, b, nilAltairCallback)
check:
bAdd == AddHeadRes.err BlockError.Invalid
bAdd == AddHeadRes.err VerifierError.Invalid
block: # SyncAggregate junk
var b = b2
@ -416,7 +416,7 @@ suite "Block pool altair processing" & preset():
let
bAdd = dag.addHeadBlock(verifier, b, nilAltairCallback)
check:
bAdd == AddHeadRes.err BlockError.Invalid
bAdd == AddHeadRes.err VerifierError.Invalid
suite "chain DAG finalization tests" & preset():
setup:
@ -556,7 +556,7 @@ suite "chain DAG finalization tests" & preset():
let status = dag.addHeadBlock(verifier, lateBlock, nilPhase0Callback)
# This _should_ be Unviable, but we can't tell, from the data that we have
# so MissingParent is the least wrong thing to reply
check: status == AddHeadRes.err BlockError.UnviableFork
check: status == AddHeadRes.err VerifierError.UnviableFork
block:
let
@ -871,13 +871,13 @@ suite "Backfill":
badBlock.signature = blocks[^3].phase0Data.signature
check:
dag.addBackfillBlock(badBlock) == AddBackRes.err BlockError.Invalid
dag.addBackfillBlock(badBlock) == AddBackRes.err VerifierError.Invalid
check:
dag.addBackfillBlock(blocks[^3].phase0Data) ==
AddBackRes.err BlockError.MissingParent
AddBackRes.err VerifierError.MissingParent
dag.addBackfillBlock(genBlock.phase0Data.asSigned()) ==
AddBackRes.err BlockError.MissingParent
AddBackRes.err VerifierError.MissingParent
dag.addBackfillBlock(tailBlock.phase0Data).isOk()
@ -909,7 +909,7 @@ suite "Backfill":
check:
dag.addBackfillBlock(genBlock.phase0Data.asSigned) ==
AddBackRes.err BlockError.Duplicate
AddBackRes.err VerifierError.Duplicate
dag.backfill.slot == GENESIS_SLOT
@ -972,7 +972,7 @@ suite "Backfill":
check:
dag.addBackfillBlock(genBlock.phase0Data.asSigned).isOk()
dag.addBackfillBlock(
genBlock.phase0Data.asSigned) == AddBackRes.err BlockError.Duplicate
genBlock.phase0Data.asSigned) == AddBackRes.err VerifierError.Duplicate
var
cache: StateCache
@ -1064,13 +1064,13 @@ suite "Starting states":
badBlock = blocks[^2].phase0Data
badBlock.signature = blocks[^3].phase0Data.signature
check:
dag.addBackfillBlock(badBlock) == AddBackRes.err BlockError.Invalid
dag.addBackfillBlock(badBlock) == AddBackRes.err VerifierError.Invalid
check:
dag.addBackfillBlock(blocks[^3].phase0Data) ==
AddBackRes.err BlockError.MissingParent
AddBackRes.err VerifierError.MissingParent
dag.addBackfillBlock(genBlock.phase0Data.asSigned()) ==
AddBackRes.err BlockError.MissingParent
AddBackRes.err VerifierError.MissingParent
dag.addBackfillBlock(tailBlock.phase0Data) == AddBackRes.ok()

View File

@ -99,7 +99,7 @@ suite "Light client processor" & preset():
processor = LightClientProcessor.new(
false, "", "", cfg, genesis_validators_root, finalizationMode,
store, getBeaconTime, getTrustedBlockRoot, onStoreInitialized)
res: Result[bool, BlockError]
res: Result[bool, VerifierError]
test "Sync" & testNameSuffix:
let bootstrap = dag.getLightClientBootstrap(trustedBlockRoot)
@ -151,14 +151,14 @@ suite "Light client processor" & preset():
else:
check:
res.isErr
res.error == BlockError.Duplicate
res.error == VerifierError.Duplicate
store[].isSome
store[].get.best_valid_update.isSome
store[].get.best_valid_update.get == update.get
else:
check:
res.isErr
res.error == BlockError.MissingParent
res.error == VerifierError.MissingParent
store[].isSome
store[].get.best_valid_update.isSome
store[].get.best_valid_update.get != update.get
@ -170,14 +170,14 @@ suite "Light client processor" & preset():
period == lastPeriodWithSupermajority + 1:
check:
res.isErr
res.error == BlockError.Duplicate
res.error == VerifierError.Duplicate
store[].isSome
store[].get.best_valid_update.isSome
store[].get.best_valid_update.get == update.get
else:
check:
res.isErr
res.error == BlockError.MissingParent
res.error == VerifierError.MissingParent
store[].isSome
store[].get.best_valid_update.isSome
store[].get.best_valid_update.get != update.get
@ -194,7 +194,7 @@ suite "Light client processor" & preset():
if finalizationMode == LightClientFinalizationMode.Optimistic:
check:
res.isErr
res.error == BlockError.Duplicate
res.error == VerifierError.Duplicate
store[].isSome
store[].get.best_valid_update.isNone
if store[].get.finalized_header == update.get.attested_header:
@ -203,14 +203,14 @@ suite "Light client processor" & preset():
elif period == lastPeriodWithSupermajority + 1:
check:
res.isErr
res.error == BlockError.Duplicate
res.error == VerifierError.Duplicate
store[].isSome
store[].get.best_valid_update.isSome
store[].get.best_valid_update.get == update.get
else:
check:
res.isErr
res.error == BlockError.MissingParent
res.error == VerifierError.MissingParent
store[].isSome
store[].get.best_valid_update.isSome
store[].get.best_valid_update.get != update.get
@ -238,9 +238,9 @@ suite "Light client processor" & preset():
store[].get.best_valid_update.get.matches(finalityUpdate.get)
store[].get.optimistic_header == finalityUpdate.get.attested_header
elif finalizationMode == LightClientFinalizationMode.Optimistic:
check res.error == BlockError.Duplicate
check res.error == VerifierError.Duplicate
else:
check res.error == BlockError.MissingParent
check res.error == VerifierError.MissingParent
check numOnStoreInitializedCalls == 1
test "Invalid bootstrap" & testNameSuffix:
@ -252,7 +252,7 @@ suite "Light client processor" & preset():
MsgSource.gossip, getBeaconTime(), bootstrap.get)
check:
res.isErr
res.error == BlockError.Invalid
res.error == VerifierError.Invalid
numOnStoreInitializedCalls == 0
test "Duplicate bootstrap" & testNameSuffix:
@ -268,7 +268,7 @@ suite "Light client processor" & preset():
MsgSource.gossip, getBeaconTime(), bootstrap.get)
check:
res.isErr
res.error == BlockError.Duplicate
res.error == VerifierError.Duplicate
numOnStoreInitializedCalls == 1
test "Missing bootstrap (update)" & testNameSuffix:
@ -279,7 +279,7 @@ suite "Light client processor" & preset():
MsgSource.gossip, getBeaconTime(), update.get)
check:
res.isErr
res.error == BlockError.MissingParent
res.error == VerifierError.MissingParent
numOnStoreInitializedCalls == 0
test "Missing bootstrap (finality update)" & testNameSuffix:
@ -290,7 +290,7 @@ suite "Light client processor" & preset():
MsgSource.gossip, getBeaconTime(), finalityUpdate.get)
check:
res.isErr
res.error == BlockError.MissingParent
res.error == VerifierError.MissingParent
numOnStoreInitializedCalls == 0
test "Missing bootstrap (optimistic update)" & testNameSuffix:
@ -301,5 +301,5 @@ suite "Light client processor" & preset():
MsgSource.gossip, getBeaconTime(), optimisticUpdate.get)
check:
res.isErr
res.error == BlockError.MissingParent
res.error == VerifierError.MissingParent
numOnStoreInitializedCalls == 0

View File

@ -42,15 +42,16 @@ func getStaticSlotCb(slot: Slot): GetSlotCallback =
type
BlockEntry = object
blck*: ForkedSignedBeaconBlock
resfut*: Future[Result[void, BlockError]]
resfut*: Future[Result[void, VerifierError]]
proc collector(queue: AsyncQueue[BlockEntry]): BlockVerifier =
# This sets up a fake block verifiation collector that simply puts the blocks
# in the async queue, similar to how BlockProcessor does it - as far as
# testing goes, this is risky because it might introduce differences between
# the BlockProcessor and this test
proc verify(signedBlock: ForkedSignedBeaconBlock): Future[Result[void, BlockError]] =
let fut = newFuture[Result[void, BlockError]]()
proc verify(signedBlock: ForkedSignedBeaconBlock):
Future[Result[void, VerifierError]] =
let fut = newFuture[Result[void, VerifierError]]()
try: queue.addLastNoWait(BlockEntry(blck: signedBlock, resfut: fut))
except CatchableError as exc: raiseAssert exc.msg
return fut
@ -262,9 +263,9 @@ suite "SyncManager test suite":
r22.slot == Slot(0) and r22.count == 1'u64
template done(b: BlockEntry) =
b.resfut.complete(Result[void, BlockError].ok())
b.resfut.complete(Result[void, VerifierError].ok())
template fail(b: BlockEntry, e: untyped) =
b.resfut.complete(Result[void, BlockError].err(e))
b.resfut.complete(Result[void, VerifierError].err(e))
template smokeTest(kkind: SyncQueueKind, start, finish: Slot,
chunkSize: uint64) =
@ -283,7 +284,7 @@ suite "SyncManager test suite":
if sblock.blck.slot == Slot(counter):
sblock.done()
else:
sblock.fail(BlockError.Invalid)
sblock.fail(VerifierError.Invalid)
dec(counter)
proc forwardValidator(aq: AsyncQueue[BlockEntry]) {.async.} =
@ -293,7 +294,7 @@ suite "SyncManager test suite":
inc(counter)
sblock.done()
else:
sblock.fail(BlockError.Invalid)
sblock.fail(VerifierError.Invalid)
var
queue =
@ -352,7 +353,7 @@ suite "SyncManager test suite":
if sblock.blck.slot == Slot(counter):
sblock.done()
else:
sblock.fail(BlockError.Invalid)
sblock.fail(VerifierError.Invalid)
dec(counter)
proc forwardValidator(aq: AsyncQueue[BlockEntry]) {.async.} =
@ -362,7 +363,7 @@ suite "SyncManager test suite":
inc(counter)
sblock.done()
else:
sblock.fail(BlockError.Invalid)
sblock.fail(VerifierError.Invalid)
var
chain = createChain(startSlot, finishSlot)
@ -448,9 +449,9 @@ suite "SyncManager test suite":
sblock.done()
elif sblock.blck.slot < Slot(counter):
# There was a gap, report missing parent
sblock.fail(BlockError.MissingParent)
sblock.fail(VerifierError.MissingParent)
else:
sblock.fail(BlockError.Duplicate)
sblock.fail(VerifierError.Duplicate)
proc getBackwardSafeSlotCb(): Slot =
min((Slot(counter).epoch + 1).start_slot, finish)
@ -463,9 +464,9 @@ suite "SyncManager test suite":
sblock.done()
elif sblock.blck.slot > Slot(counter):
# There was a gap, report missing parent
sblock.fail(BlockError.MissingParent)
sblock.fail(VerifierError.MissingParent)
else:
sblock.fail(BlockError.Duplicate)
sblock.fail(VerifierError.Duplicate)
proc getFowardSafeSlotCb(): Slot =
max(Slot(max(counter, 1) - 1).epoch.start_slot, start)
@ -540,7 +541,7 @@ suite "SyncManager test suite":
proc failingValidator(aq: AsyncQueue[BlockEntry]) {.async.} =
while true:
let sblock = await aq.popFirst()
sblock.fail(BlockError.Invalid)
sblock.fail(VerifierError.Invalid)
proc getBackwardSafeSlotCb(): Slot =
let progress = (uint64(int(finish) - counter) div chunkSize) * chunkSize
@ -709,12 +710,12 @@ suite "SyncManager test suite":
if sblock.blck.slot == Slot(counter):
withBlck(sblock.blck):
if blck.message.proposer_index == 0xDEADBEAF'u64:
sblock.fail(BlockError.MissingParent)
sblock.fail(VerifierError.MissingParent)
else:
inc(counter)
sblock.done()
else:
sblock.fail(BlockError.Invalid)
sblock.fail(VerifierError.Invalid)
var
chain = createChain(startSlot, finishSlot)
@ -821,7 +822,7 @@ suite "SyncManager test suite":
while true:
let sblock = await aq.popFirst()
withBlck(sblock.blck):
sblock.fail(BlockError.UnviableFork)
sblock.fail(VerifierError.UnviableFork)
inc(counter)
var
@ -874,13 +875,13 @@ suite "SyncManager test suite":
if sblock.blck.slot == Slot(counter):
withBlck(sblock.blck):
if blck.message.proposer_index == 0xDEADBEAF'u64:
sblock.fail(BlockError.MissingParent)
sblock.fail(VerifierError.MissingParent)
else:
lastSafeSlot = sblock.blck.slot
dec(counter)
sblock.done()
else:
sblock.fail(BlockError.Invalid)
sblock.fail(VerifierError.Invalid)
var
chain = createChain(startSlot, finishSlot)