nimbus-eth2/beacon_chain/consensus_object_pools/block_clearance.nim

373 lines
14 KiB
Nim
Raw Normal View History

# beacon_chain
# Copyright (c) 2018-2022 Status Research & Development GmbH
# Licensed and distributed under either of
# * MIT license (license terms in the root directory or at https://opensource.org/licenses/MIT).
# * Apache v2 license (license terms in the root directory or at https://www.apache.org/licenses/LICENSE-2.0).
# at your option. This file may not be copied, modified, or distributed except according to those terms.
{.push raises: [Defect].}
import
chronicles,
stew/[assign2, results],
../spec/[beaconstate, forks, signatures, signatures_batch, state_transition],
"."/[block_dag, blockchain_dag, blockchain_dag_light_client]
limit by-root requests to non-finalized blocks (#3293) * limit by-root requests to non-finalized blocks Presently, we keep a mapping from block root to `BlockRef` in memory - this has simplified reasoning about the dag, but is not sustainable with the chain growing. We can distinguish between two cases where by-root access is useful: * unfinalized blocks - this is where the beacon chain is operating generally, by validating incoming data as interesting for future fork choice decisions - bounded by the length of the unfinalized period * finalized blocks - historical access in the REST API etc - no bounds, really In this PR, we limit the by-root block index to the first use case: finalized chain data can more efficiently be addressed by slot number. Future work includes: * limiting the `BlockRef` horizon in general - each instance is 40 bytes+overhead which adds up - this needs further refactoring to deal with the tail vs state problem * persisting the finalized slot-to-hash index - this one also keeps growing unbounded (albeit slowly) Anyway, this PR easily shaves ~128mb of memory usage at the time of writing. * No longer honor `BeaconBlocksByRoot` requests outside of the non-finalized period - previously, Nimbus would generously return any block through this libp2p request - per the spec, finalized blocks should be fetched via `BeaconBlocksByRange` instead. * return `Opt[BlockRef]` instead of `nil` when blocks can't be found - this becomes a lot more common now and thus deserves more attention * `dag.blocks` -> `dag.forkBlocks` - this index only carries unfinalized blocks from now - `finalizedBlocks` covers the other `BlockRef` instances * in backfill, verify that the last backfilled block leads back to genesis, or panic * add backfill timings to log * fix missing check that `BlockRef` block can be fetched with `getForkedBlock` reliably * shortcut doppelganger check when feature is not enabled * in REST/JSON-RPC, fetch blocks without involving `BlockRef` * fix dag.blocks ref
2022-01-21 11:33:16 +00:00
export results, signatures_batch, block_dag, blockchain_dag
2020-05-21 17:08:31 +00:00
# Clearance
# ---------------------------------------------
#
# This module is in charge of making the
# "quarantined" network blocks
# pass the firewall and be stored in the chain DAG
logScope:
topics = "clearance"
proc addResolvedHeadBlock(
dag: ChainDAGRef,
state: var ForkedHashedBeaconState,
trustedBlock: ForkyTrustedSignedBeaconBlock,
parent: BlockRef, cache: var StateCache,
onBlockAdded: OnPhase0BlockAdded | OnAltairBlockAdded | OnBellatrixBlockAdded,
stateDataDur, sigVerifyDur, stateVerifyDur: Duration
): BlockRef =
doAssert state.matches_block_slot(
trustedBlock.root, trustedBlock.message.slot),
"Given state must have the new block applied"
let
blockRoot = trustedBlock.root
blockRef = BlockRef.init(blockRoot, trustedBlock.message)
startTick = Moment.now()
link(parent, blockRef)
limit by-root requests to non-finalized blocks (#3293) * limit by-root requests to non-finalized blocks Presently, we keep a mapping from block root to `BlockRef` in memory - this has simplified reasoning about the dag, but is not sustainable with the chain growing. We can distinguish between two cases where by-root access is useful: * unfinalized blocks - this is where the beacon chain is operating generally, by validating incoming data as interesting for future fork choice decisions - bounded by the length of the unfinalized period * finalized blocks - historical access in the REST API etc - no bounds, really In this PR, we limit the by-root block index to the first use case: finalized chain data can more efficiently be addressed by slot number. Future work includes: * limiting the `BlockRef` horizon in general - each instance is 40 bytes+overhead which adds up - this needs further refactoring to deal with the tail vs state problem * persisting the finalized slot-to-hash index - this one also keeps growing unbounded (albeit slowly) Anyway, this PR easily shaves ~128mb of memory usage at the time of writing. * No longer honor `BeaconBlocksByRoot` requests outside of the non-finalized period - previously, Nimbus would generously return any block through this libp2p request - per the spec, finalized blocks should be fetched via `BeaconBlocksByRange` instead. * return `Opt[BlockRef]` instead of `nil` when blocks can't be found - this becomes a lot more common now and thus deserves more attention * `dag.blocks` -> `dag.forkBlocks` - this index only carries unfinalized blocks from now - `finalizedBlocks` covers the other `BlockRef` instances * in backfill, verify that the last backfilled block leads back to genesis, or panic * add backfill timings to log * fix missing check that `BlockRef` block can be fetched with `getForkedBlock` reliably * shortcut doppelganger check when feature is not enabled * in REST/JSON-RPC, fetch blocks without involving `BlockRef` * fix dag.blocks ref
2022-01-21 11:33:16 +00:00
dag.forkBlocks.incl(KeyedBlockRef.init(blockRef))
# Resolved blocks should be stored in database
dag.putBlock(trustedBlock)
let putBlockTick = Moment.now()
var foundHead: bool
for head in dag.heads.mitems():
if head.isAncestorOf(blockRef):
head = blockRef
foundHead = true
break
if not foundHead:
dag.heads.add(blockRef)
# Regardless of the chain we're on, the deposits come in the same order so
# as soon as we import a block, we'll also update the shared public key
# cache
dag.updateValidatorKeys(getStateField(state, validators).asSeq())
# Getting epochRef with the state will potentially create a new EpochRef
let
Prune `BlockRef` on finalization (#3513) Up til now, the block dag has been using `BlockRef`, a structure adapted for a full DAG, to represent all of chain history. This is a correct and simple design, but does not exploit the linearity of the chain once parts of it finalize. By pruning the in-memory `BlockRef` structure at finalization, we save, at the time of writing, a cool ~250mb (or 25%:ish) chunk of memory landing us at a steady state of ~750mb normal memory usage for a validating node. Above all though, we prevent memory usage from growing proportionally with the length of the chain, something that would not be sustainable over time - instead, the steady state memory usage is roughly determined by the validator set size which grows much more slowly. With these changes, the core should remain sustainable memory-wise post-merge all the way to withdrawals (when the validator set is expected to grow). In-memory indices are still used for the "hot" unfinalized portion of the chain - this ensure that consensus performance remains unchanged. What changes is that for historical access, we use a db-based linear slot index which is cache-and-disk-friendly, keeping the cost for accessing historical data at a similar level as before, achieving the savings at no percievable cost to functionality or performance. A nice collateral benefit is the almost-instant startup since we no longer load any large indicies at dag init. The cost of this functionality instead can be found in the complexity of having to deal with two ways of traversing the chain - by `BlockRef` and by slot. * use `BlockId` instead of `BlockRef` where finalized / historical data may be required * simplify clearance pre-advancement * remove dag.finalizedBlocks (~50:ish mb) * remove `getBlockAtSlot` - use `getBlockIdAtSlot` instead * `parent` and `atSlot` for `BlockId` now require a `ChainDAGRef` instance, unlike `BlockRef` traversal * prune `BlockRef` parents on finality (~200:ish mb) * speed up ChainDAG init by not loading finalized history index * mess up light client server error handling - this need revisiting :)
2022-03-17 17:42:56 +00:00
epochRef = dag.getEpochRef(state, cache)
epochRefTick = Moment.now()
debug "Block resolved",
blockRoot = shortLog(blockRoot),
blck = shortLog(trustedBlock.message),
heads = dag.heads.len(),
stateDataDur, sigVerifyDur, stateVerifyDur,
putBlockDur = putBlockTick - startTick,
epochRefDur = epochRefTick - putBlockTick
# Update light client data
dag.processNewBlockForLightClient(state, trustedBlock, parent.bid)
# Notify others of the new block before processing the quarantine, such that
# notifications for parents happens before those of the children
if onBlockAdded != nil:
onBlockAdded(blockRef, trustedBlock, epochRef)
if not(isNil(dag.onBlockAdded)):
dag.onBlockAdded(ForkedTrustedSignedBeaconBlock.init(trustedBlock))
blockRef
proc checkStateTransition(
dag: ChainDAGRef, signedBlock: ForkySigVerifiedSignedBeaconBlock,
cache: var StateCache): Result[void, BlockError] =
## Ensure block can be applied on a state
func restore(v: var ForkedHashedBeaconState) =
assign(dag.clearanceState, dag.headState)
let res = state_transition_block(
dag.cfg, dag.clearanceState, signedBlock,
cache, dag.updateFlags, restore)
if res.isErr():
info "Invalid block",
blockRoot = shortLog(signedBlock.root),
blck = shortLog(signedBlock.message),
error = res.error()
err(BlockError.Invalid)
else:
ok()
proc advanceClearanceState*(dag: ChainDAGRef) =
# When the chain is synced, the most likely block to be produced is the block
# right after head - we can exploit this assumption and advance the state
# to that slot before the block arrives, thus allowing us to do the expensive
# epoch transition ahead of time.
# Notably, we use the clearance state here because that's where the block will
# first be seen - later, this state will be copied to the head state!
Prune `BlockRef` on finalization (#3513) Up til now, the block dag has been using `BlockRef`, a structure adapted for a full DAG, to represent all of chain history. This is a correct and simple design, but does not exploit the linearity of the chain once parts of it finalize. By pruning the in-memory `BlockRef` structure at finalization, we save, at the time of writing, a cool ~250mb (or 25%:ish) chunk of memory landing us at a steady state of ~750mb normal memory usage for a validating node. Above all though, we prevent memory usage from growing proportionally with the length of the chain, something that would not be sustainable over time - instead, the steady state memory usage is roughly determined by the validator set size which grows much more slowly. With these changes, the core should remain sustainable memory-wise post-merge all the way to withdrawals (when the validator set is expected to grow). In-memory indices are still used for the "hot" unfinalized portion of the chain - this ensure that consensus performance remains unchanged. What changes is that for historical access, we use a db-based linear slot index which is cache-and-disk-friendly, keeping the cost for accessing historical data at a similar level as before, achieving the savings at no percievable cost to functionality or performance. A nice collateral benefit is the almost-instant startup since we no longer load any large indicies at dag init. The cost of this functionality instead can be found in the complexity of having to deal with two ways of traversing the chain - by `BlockRef` and by slot. * use `BlockId` instead of `BlockRef` where finalized / historical data may be required * simplify clearance pre-advancement * remove dag.finalizedBlocks (~50:ish mb) * remove `getBlockAtSlot` - use `getBlockIdAtSlot` instead * `parent` and `atSlot` for `BlockId` now require a `ChainDAGRef` instance, unlike `BlockRef` traversal * prune `BlockRef` parents on finality (~200:ish mb) * speed up ChainDAG init by not loading finalized history index * mess up light client server error handling - this need revisiting :)
2022-03-17 17:42:56 +00:00
let advanced = withState(dag.clearanceState):
state.data.slot > state.data.latest_block_header.slot
if not advanced:
let next = getStateField(dag.clearanceState, slot) + 1
let startTick = Moment.now()
Prune `BlockRef` on finalization (#3513) Up til now, the block dag has been using `BlockRef`, a structure adapted for a full DAG, to represent all of chain history. This is a correct and simple design, but does not exploit the linearity of the chain once parts of it finalize. By pruning the in-memory `BlockRef` structure at finalization, we save, at the time of writing, a cool ~250mb (or 25%:ish) chunk of memory landing us at a steady state of ~750mb normal memory usage for a validating node. Above all though, we prevent memory usage from growing proportionally with the length of the chain, something that would not be sustainable over time - instead, the steady state memory usage is roughly determined by the validator set size which grows much more slowly. With these changes, the core should remain sustainable memory-wise post-merge all the way to withdrawals (when the validator set is expected to grow). In-memory indices are still used for the "hot" unfinalized portion of the chain - this ensure that consensus performance remains unchanged. What changes is that for historical access, we use a db-based linear slot index which is cache-and-disk-friendly, keeping the cost for accessing historical data at a similar level as before, achieving the savings at no percievable cost to functionality or performance. A nice collateral benefit is the almost-instant startup since we no longer load any large indicies at dag init. The cost of this functionality instead can be found in the complexity of having to deal with two ways of traversing the chain - by `BlockRef` and by slot. * use `BlockId` instead of `BlockRef` where finalized / historical data may be required * simplify clearance pre-advancement * remove dag.finalizedBlocks (~50:ish mb) * remove `getBlockAtSlot` - use `getBlockIdAtSlot` instead * `parent` and `atSlot` for `BlockId` now require a `ChainDAGRef` instance, unlike `BlockRef` traversal * prune `BlockRef` parents on finality (~200:ish mb) * speed up ChainDAG init by not loading finalized history index * mess up light client server error handling - this need revisiting :)
2022-03-17 17:42:56 +00:00
var
cache = StateCache()
info = ForkedEpochInfo()
dag.advanceSlots(dag.clearanceState, next, true, cache, info)
debug "Prepared clearance state for next block",
next, updateStateDur = Moment.now() - startTick
proc addHeadBlock*(
dag: ChainDAGRef, verifier: var BatchVerifier,
signedBlock: ForkySignedBeaconBlock,
onBlockAdded: OnPhase0BlockAdded | OnAltairBlockAdded |
OnBellatrixBlockAdded
): Result[BlockRef, BlockError] =
## Try adding a block to the chain, verifying first that it passes the state
## transition function and contains correct cryptographic signature.
##
2022-04-14 15:39:37 +00:00
## Cryptographic checks can be skipped by adding skipBlsValidation to
## dag.updateFlags
logScope:
blockRoot = shortLog(signedBlock.root)
blck = shortLog(signedBlock.message)
chaindag: don't keep backfill block table in memory (#3429) This PR names and documents the concept of the archive: a range of slots for which we have degraded functionality in terms of historical access - in particular: * we don't support rewinding to states in this range * we don't keep an in-memory representation of the block dag The archive de-facto exists in a trusted-node-synced node, but this PR gives it a name and drops the in-memory digest index. In order to satisfy `GetBlocksByRange` requests, we ensure that we have blocks for the entire archive period via backfill. Future versions may relax this further, adding a "pre-archive" period that is fully pruned. During by-slot searches in the archive (both for libp2p and rest requests), an extra database lookup is used to covert the given `slot` to a `root` - future versions will avoid this using era files which natively are indexed by `slot`. That said, the lookup is quite fast compared to the actual block loading given how trivial the table is - it's hard to measure, even. A collateral benefit of this PR is that checkpoint-synced nodes will see 100-200MB memory usage savings, thanks to the dropped in-memory cache - future pruning work will bring this benefit to full nodes as well. * document chaindag storage architecture and assumptions * look up parent using block id instead of full block in clearance (future-proofing the code against a future in which blocks come from era files) * simplify finalized block init, always writing the backfill portion to db at startup (to ensure lookups work as expected) * preallocate some extra memory for finalized blocks, to avoid immediate realloc
2022-02-26 18:16:19 +00:00
signature = shortLog(signedBlock.signature)
template blck(): untyped = signedBlock.message # shortcuts without copy
template blockRoot(): untyped = signedBlock.root
# If the block we get is older than what we finalized already, we drop it.
# One way this can happen is that we start request a block and finalization
# happens in the meantime - the block we requested will then be stale
# by the time it gets here.
if blck.slot <= dag.finalizedHead.slot:
chaindag: don't keep backfill block table in memory (#3429) This PR names and documents the concept of the archive: a range of slots for which we have degraded functionality in terms of historical access - in particular: * we don't support rewinding to states in this range * we don't keep an in-memory representation of the block dag The archive de-facto exists in a trusted-node-synced node, but this PR gives it a name and drops the in-memory digest index. In order to satisfy `GetBlocksByRange` requests, we ensure that we have blocks for the entire archive period via backfill. Future versions may relax this further, adding a "pre-archive" period that is fully pruned. During by-slot searches in the archive (both for libp2p and rest requests), an extra database lookup is used to covert the given `slot` to a `root` - future versions will avoid this using era files which natively are indexed by `slot`. That said, the lookup is quite fast compared to the actual block loading given how trivial the table is - it's hard to measure, even. A collateral benefit of this PR is that checkpoint-synced nodes will see 100-200MB memory usage savings, thanks to the dropped in-memory cache - future pruning work will bring this benefit to full nodes as well. * document chaindag storage architecture and assumptions * look up parent using block id instead of full block in clearance (future-proofing the code against a future in which blocks come from era files) * simplify finalized block init, always writing the backfill portion to db at startup (to ensure lookups work as expected) * preallocate some extra memory for finalized blocks, to avoid immediate realloc
2022-02-26 18:16:19 +00:00
let existing = dag.getBlockIdAtSlot(blck.slot)
# The exact slot match ensures we reject blocks that were orphaned in
# the finalized chain
if existing.isSome:
if existing.get().bid.slot == blck.slot and
existing.get().bid.root == blockRoot:
debug "Duplicate block"
return err(BlockError.Duplicate)
# Block is older than finalized, but different from the block in our
# canonical history: it must be from an unviable branch
debug "Block from unviable fork",
existing = shortLog(existing.get()),
finalizedHead = shortLog(dag.finalizedHead),
tail = shortLog(dag.tail)
limit by-root requests to non-finalized blocks (#3293) * limit by-root requests to non-finalized blocks Presently, we keep a mapping from block root to `BlockRef` in memory - this has simplified reasoning about the dag, but is not sustainable with the chain growing. We can distinguish between two cases where by-root access is useful: * unfinalized blocks - this is where the beacon chain is operating generally, by validating incoming data as interesting for future fork choice decisions - bounded by the length of the unfinalized period * finalized blocks - historical access in the REST API etc - no bounds, really In this PR, we limit the by-root block index to the first use case: finalized chain data can more efficiently be addressed by slot number. Future work includes: * limiting the `BlockRef` horizon in general - each instance is 40 bytes+overhead which adds up - this needs further refactoring to deal with the tail vs state problem * persisting the finalized slot-to-hash index - this one also keeps growing unbounded (albeit slowly) Anyway, this PR easily shaves ~128mb of memory usage at the time of writing. * No longer honor `BeaconBlocksByRoot` requests outside of the non-finalized period - previously, Nimbus would generously return any block through this libp2p request - per the spec, finalized blocks should be fetched via `BeaconBlocksByRange` instead. * return `Opt[BlockRef]` instead of `nil` when blocks can't be found - this becomes a lot more common now and thus deserves more attention * `dag.blocks` -> `dag.forkBlocks` - this index only carries unfinalized blocks from now - `finalizedBlocks` covers the other `BlockRef` instances * in backfill, verify that the last backfilled block leads back to genesis, or panic * add backfill timings to log * fix missing check that `BlockRef` block can be fetched with `getForkedBlock` reliably * shortcut doppelganger check when feature is not enabled * in REST/JSON-RPC, fetch blocks without involving `BlockRef` * fix dag.blocks ref
2022-01-21 11:33:16 +00:00
return err(BlockError.UnviableFork)
limit by-root requests to non-finalized blocks (#3293) * limit by-root requests to non-finalized blocks Presently, we keep a mapping from block root to `BlockRef` in memory - this has simplified reasoning about the dag, but is not sustainable with the chain growing. We can distinguish between two cases where by-root access is useful: * unfinalized blocks - this is where the beacon chain is operating generally, by validating incoming data as interesting for future fork choice decisions - bounded by the length of the unfinalized period * finalized blocks - historical access in the REST API etc - no bounds, really In this PR, we limit the by-root block index to the first use case: finalized chain data can more efficiently be addressed by slot number. Future work includes: * limiting the `BlockRef` horizon in general - each instance is 40 bytes+overhead which adds up - this needs further refactoring to deal with the tail vs state problem * persisting the finalized slot-to-hash index - this one also keeps growing unbounded (albeit slowly) Anyway, this PR easily shaves ~128mb of memory usage at the time of writing. * No longer honor `BeaconBlocksByRoot` requests outside of the non-finalized period - previously, Nimbus would generously return any block through this libp2p request - per the spec, finalized blocks should be fetched via `BeaconBlocksByRange` instead. * return `Opt[BlockRef]` instead of `nil` when blocks can't be found - this becomes a lot more common now and thus deserves more attention * `dag.blocks` -> `dag.forkBlocks` - this index only carries unfinalized blocks from now - `finalizedBlocks` covers the other `BlockRef` instances * in backfill, verify that the last backfilled block leads back to genesis, or panic * add backfill timings to log * fix missing check that `BlockRef` block can be fetched with `getForkedBlock` reliably * shortcut doppelganger check when feature is not enabled * in REST/JSON-RPC, fetch blocks without involving `BlockRef` * fix dag.blocks ref
2022-01-21 11:33:16 +00:00
# Check non-finalized blocks as well
if dag.containsForkBlock(blockRoot):
return err(BlockError.Duplicate)
limit by-root requests to non-finalized blocks (#3293) * limit by-root requests to non-finalized blocks Presently, we keep a mapping from block root to `BlockRef` in memory - this has simplified reasoning about the dag, but is not sustainable with the chain growing. We can distinguish between two cases where by-root access is useful: * unfinalized blocks - this is where the beacon chain is operating generally, by validating incoming data as interesting for future fork choice decisions - bounded by the length of the unfinalized period * finalized blocks - historical access in the REST API etc - no bounds, really In this PR, we limit the by-root block index to the first use case: finalized chain data can more efficiently be addressed by slot number. Future work includes: * limiting the `BlockRef` horizon in general - each instance is 40 bytes+overhead which adds up - this needs further refactoring to deal with the tail vs state problem * persisting the finalized slot-to-hash index - this one also keeps growing unbounded (albeit slowly) Anyway, this PR easily shaves ~128mb of memory usage at the time of writing. * No longer honor `BeaconBlocksByRoot` requests outside of the non-finalized period - previously, Nimbus would generously return any block through this libp2p request - per the spec, finalized blocks should be fetched via `BeaconBlocksByRange` instead. * return `Opt[BlockRef]` instead of `nil` when blocks can't be found - this becomes a lot more common now and thus deserves more attention * `dag.blocks` -> `dag.forkBlocks` - this index only carries unfinalized blocks from now - `finalizedBlocks` covers the other `BlockRef` instances * in backfill, verify that the last backfilled block leads back to genesis, or panic * add backfill timings to log * fix missing check that `BlockRef` block can be fetched with `getForkedBlock` reliably * shortcut doppelganger check when feature is not enabled * in REST/JSON-RPC, fetch blocks without involving `BlockRef` * fix dag.blocks ref
2022-01-21 11:33:16 +00:00
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
# it has been finalized already, and as a result the branch the new block
# is on is no longer a viable fork candidate - we can't tell which is which
# at this stage, but we can check if we've seen the parent block previously
# and thus prevent requests for it to be downloaded again.
chaindag: don't keep backfill block table in memory (#3429) This PR names and documents the concept of the archive: a range of slots for which we have degraded functionality in terms of historical access - in particular: * we don't support rewinding to states in this range * we don't keep an in-memory representation of the block dag The archive de-facto exists in a trusted-node-synced node, but this PR gives it a name and drops the in-memory digest index. In order to satisfy `GetBlocksByRange` requests, we ensure that we have blocks for the entire archive period via backfill. Future versions may relax this further, adding a "pre-archive" period that is fully pruned. During by-slot searches in the archive (both for libp2p and rest requests), an extra database lookup is used to covert the given `slot` to a `root` - future versions will avoid this using era files which natively are indexed by `slot`. That said, the lookup is quite fast compared to the actual block loading given how trivial the table is - it's hard to measure, even. A collateral benefit of this PR is that checkpoint-synced nodes will see 100-200MB memory usage savings, thanks to the dropped in-memory cache - future pruning work will bring this benefit to full nodes as well. * document chaindag storage architecture and assumptions * look up parent using block id instead of full block in clearance (future-proofing the code against a future in which blocks come from era files) * simplify finalized block init, always writing the backfill portion to db at startup (to ensure lookups work as expected) * preallocate some extra memory for finalized blocks, to avoid immediate realloc
2022-02-26 18:16:19 +00:00
let parentId = dag.getBlockId(blck.parent_root)
if parentId.isSome():
debug "Block unviable due to pre-finalized-checkpoint parent",
parentId = parentId.get()
limit by-root requests to non-finalized blocks (#3293) * limit by-root requests to non-finalized blocks Presently, we keep a mapping from block root to `BlockRef` in memory - this has simplified reasoning about the dag, but is not sustainable with the chain growing. We can distinguish between two cases where by-root access is useful: * unfinalized blocks - this is where the beacon chain is operating generally, by validating incoming data as interesting for future fork choice decisions - bounded by the length of the unfinalized period * finalized blocks - historical access in the REST API etc - no bounds, really In this PR, we limit the by-root block index to the first use case: finalized chain data can more efficiently be addressed by slot number. Future work includes: * limiting the `BlockRef` horizon in general - each instance is 40 bytes+overhead which adds up - this needs further refactoring to deal with the tail vs state problem * persisting the finalized slot-to-hash index - this one also keeps growing unbounded (albeit slowly) Anyway, this PR easily shaves ~128mb of memory usage at the time of writing. * No longer honor `BeaconBlocksByRoot` requests outside of the non-finalized period - previously, Nimbus would generously return any block through this libp2p request - per the spec, finalized blocks should be fetched via `BeaconBlocksByRange` instead. * return `Opt[BlockRef]` instead of `nil` when blocks can't be found - this becomes a lot more common now and thus deserves more attention * `dag.blocks` -> `dag.forkBlocks` - this index only carries unfinalized blocks from now - `finalizedBlocks` covers the other `BlockRef` instances * in backfill, verify that the last backfilled block leads back to genesis, or panic * add backfill timings to log * fix missing check that `BlockRef` block can be fetched with `getForkedBlock` reliably * shortcut doppelganger check when feature is not enabled * in REST/JSON-RPC, fetch blocks without involving `BlockRef` * fix dag.blocks ref
2022-01-21 11:33:16 +00:00
return err(BlockError.UnviableFork)
chaindag: don't keep backfill block table in memory (#3429) This PR names and documents the concept of the archive: a range of slots for which we have degraded functionality in terms of historical access - in particular: * we don't support rewinding to states in this range * we don't keep an in-memory representation of the block dag The archive de-facto exists in a trusted-node-synced node, but this PR gives it a name and drops the in-memory digest index. In order to satisfy `GetBlocksByRange` requests, we ensure that we have blocks for the entire archive period via backfill. Future versions may relax this further, adding a "pre-archive" period that is fully pruned. During by-slot searches in the archive (both for libp2p and rest requests), an extra database lookup is used to covert the given `slot` to a `root` - future versions will avoid this using era files which natively are indexed by `slot`. That said, the lookup is quite fast compared to the actual block loading given how trivial the table is - it's hard to measure, even. A collateral benefit of this PR is that checkpoint-synced nodes will see 100-200MB memory usage savings, thanks to the dropped in-memory cache - future pruning work will bring this benefit to full nodes as well. * document chaindag storage architecture and assumptions * look up parent using block id instead of full block in clearance (future-proofing the code against a future in which blocks come from era files) * simplify finalized block init, always writing the backfill portion to db at startup (to ensure lookups work as expected) * preallocate some extra memory for finalized blocks, to avoid immediate realloc
2022-02-26 18:16:19 +00:00
debug "Block parent unknown or finalized already", parentId
return err(BlockError.MissingParent)
chaindag: don't keep backfill block table in memory (#3429) This PR names and documents the concept of the archive: a range of slots for which we have degraded functionality in terms of historical access - in particular: * we don't support rewinding to states in this range * we don't keep an in-memory representation of the block dag The archive de-facto exists in a trusted-node-synced node, but this PR gives it a name and drops the in-memory digest index. In order to satisfy `GetBlocksByRange` requests, we ensure that we have blocks for the entire archive period via backfill. Future versions may relax this further, adding a "pre-archive" period that is fully pruned. During by-slot searches in the archive (both for libp2p and rest requests), an extra database lookup is used to covert the given `slot` to a `root` - future versions will avoid this using era files which natively are indexed by `slot`. That said, the lookup is quite fast compared to the actual block loading given how trivial the table is - it's hard to measure, even. A collateral benefit of this PR is that checkpoint-synced nodes will see 100-200MB memory usage savings, thanks to the dropped in-memory cache - future pruning work will bring this benefit to full nodes as well. * document chaindag storage architecture and assumptions * look up parent using block id instead of full block in clearance (future-proofing the code against a future in which blocks come from era files) * simplify finalized block init, always writing the backfill portion to db at startup (to ensure lookups work as expected) * preallocate some extra memory for finalized blocks, to avoid immediate realloc
2022-02-26 18:16:19 +00:00
if parent.slot >= blck.slot:
# A block whose parent is newer than the block itself is clearly invalid -
# discard it immediately
chaindag: don't keep backfill block table in memory (#3429) This PR names and documents the concept of the archive: a range of slots for which we have degraded functionality in terms of historical access - in particular: * we don't support rewinding to states in this range * we don't keep an in-memory representation of the block dag The archive de-facto exists in a trusted-node-synced node, but this PR gives it a name and drops the in-memory digest index. In order to satisfy `GetBlocksByRange` requests, we ensure that we have blocks for the entire archive period via backfill. Future versions may relax this further, adding a "pre-archive" period that is fully pruned. During by-slot searches in the archive (both for libp2p and rest requests), an extra database lookup is used to covert the given `slot` to a `root` - future versions will avoid this using era files which natively are indexed by `slot`. That said, the lookup is quite fast compared to the actual block loading given how trivial the table is - it's hard to measure, even. A collateral benefit of this PR is that checkpoint-synced nodes will see 100-200MB memory usage savings, thanks to the dropped in-memory cache - future pruning work will bring this benefit to full nodes as well. * document chaindag storage architecture and assumptions * look up parent using block id instead of full block in clearance (future-proofing the code against a future in which blocks come from era files) * simplify finalized block init, always writing the backfill portion to db at startup (to ensure lookups work as expected) * preallocate some extra memory for finalized blocks, to avoid immediate realloc
2022-02-26 18:16:19 +00:00
debug "Block older than parent",
parent = shortLog(parent)
return err(BlockError.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
let startTick = Moment.now()
# The clearance state works as the canonical
# "let's make things permanent" point and saves things to the database -
# storing things is slow, so we don't want to do so before there's a
# reasonable chance that the information will become more permanently useful -
# by the time a new block reaches this point, the parent block will already
# have "established" itself in the network to some degree at least.
var cache = StateCache()
# We've verified that the slot of the new block is newer than that of the
# parent, so we should now be able to create an approriate clearance state
# onto which we can apply the new block
let clearanceBlock = BlockSlotId.init(parent.bid, signedBlock.message.slot)
if not updateState(
Prune `BlockRef` on finalization (#3513) Up til now, the block dag has been using `BlockRef`, a structure adapted for a full DAG, to represent all of chain history. This is a correct and simple design, but does not exploit the linearity of the chain once parts of it finalize. By pruning the in-memory `BlockRef` structure at finalization, we save, at the time of writing, a cool ~250mb (or 25%:ish) chunk of memory landing us at a steady state of ~750mb normal memory usage for a validating node. Above all though, we prevent memory usage from growing proportionally with the length of the chain, something that would not be sustainable over time - instead, the steady state memory usage is roughly determined by the validator set size which grows much more slowly. With these changes, the core should remain sustainable memory-wise post-merge all the way to withdrawals (when the validator set is expected to grow). In-memory indices are still used for the "hot" unfinalized portion of the chain - this ensure that consensus performance remains unchanged. What changes is that for historical access, we use a db-based linear slot index which is cache-and-disk-friendly, keeping the cost for accessing historical data at a similar level as before, achieving the savings at no percievable cost to functionality or performance. A nice collateral benefit is the almost-instant startup since we no longer load any large indicies at dag init. The cost of this functionality instead can be found in the complexity of having to deal with two ways of traversing the chain - by `BlockRef` and by slot. * use `BlockId` instead of `BlockRef` where finalized / historical data may be required * simplify clearance pre-advancement * remove dag.finalizedBlocks (~50:ish mb) * remove `getBlockAtSlot` - use `getBlockIdAtSlot` instead * `parent` and `atSlot` for `BlockId` now require a `ChainDAGRef` instance, unlike `BlockRef` traversal * prune `BlockRef` parents on finality (~200:ish mb) * speed up ChainDAG init by not loading finalized history index * mess up light client server error handling - this need revisiting :)
2022-03-17 17:42:56 +00:00
dag, dag.clearanceState, clearanceBlock, true, cache):
# We should never end up here - the parent must be a block no older than and
# rooted in the finalized checkpoint, hence we should always be able to
# load its corresponding state
error "Unable to load clearance state for parent block, database corrupt?",
Prune `BlockRef` on finalization (#3513) Up til now, the block dag has been using `BlockRef`, a structure adapted for a full DAG, to represent all of chain history. This is a correct and simple design, but does not exploit the linearity of the chain once parts of it finalize. By pruning the in-memory `BlockRef` structure at finalization, we save, at the time of writing, a cool ~250mb (or 25%:ish) chunk of memory landing us at a steady state of ~750mb normal memory usage for a validating node. Above all though, we prevent memory usage from growing proportionally with the length of the chain, something that would not be sustainable over time - instead, the steady state memory usage is roughly determined by the validator set size which grows much more slowly. With these changes, the core should remain sustainable memory-wise post-merge all the way to withdrawals (when the validator set is expected to grow). In-memory indices are still used for the "hot" unfinalized portion of the chain - this ensure that consensus performance remains unchanged. What changes is that for historical access, we use a db-based linear slot index which is cache-and-disk-friendly, keeping the cost for accessing historical data at a similar level as before, achieving the savings at no percievable cost to functionality or performance. A nice collateral benefit is the almost-instant startup since we no longer load any large indicies at dag init. The cost of this functionality instead can be found in the complexity of having to deal with two ways of traversing the chain - by `BlockRef` and by slot. * use `BlockId` instead of `BlockRef` where finalized / historical data may be required * simplify clearance pre-advancement * remove dag.finalizedBlocks (~50:ish mb) * remove `getBlockAtSlot` - use `getBlockIdAtSlot` instead * `parent` and `atSlot` for `BlockId` now require a `ChainDAGRef` instance, unlike `BlockRef` traversal * prune `BlockRef` parents on finality (~200:ish mb) * speed up ChainDAG init by not loading finalized history index * mess up light client server error handling - this need revisiting :)
2022-03-17 17:42:56 +00:00
clearanceBlock = shortLog(clearanceBlock)
return err(BlockError.MissingParent)
let stateDataTick = Moment.now()
# First, batch-verify all signatures in block
2022-04-14 15:39:37 +00:00
if skipBlsValidation notin dag.updateFlags:
# TODO: remove skipBlsValidation
var sigs: seq[SignatureSet]
if (let e = sigs.collectSignatureSets(
signedBlock, dag.db.immutableValidators,
dag.clearanceState, cache); e.isErr()):
limit by-root requests to non-finalized blocks (#3293) * limit by-root requests to non-finalized blocks Presently, we keep a mapping from block root to `BlockRef` in memory - this has simplified reasoning about the dag, but is not sustainable with the chain growing. We can distinguish between two cases where by-root access is useful: * unfinalized blocks - this is where the beacon chain is operating generally, by validating incoming data as interesting for future fork choice decisions - bounded by the length of the unfinalized period * finalized blocks - historical access in the REST API etc - no bounds, really In this PR, we limit the by-root block index to the first use case: finalized chain data can more efficiently be addressed by slot number. Future work includes: * limiting the `BlockRef` horizon in general - each instance is 40 bytes+overhead which adds up - this needs further refactoring to deal with the tail vs state problem * persisting the finalized slot-to-hash index - this one also keeps growing unbounded (albeit slowly) Anyway, this PR easily shaves ~128mb of memory usage at the time of writing. * No longer honor `BeaconBlocksByRoot` requests outside of the non-finalized period - previously, Nimbus would generously return any block through this libp2p request - per the spec, finalized blocks should be fetched via `BeaconBlocksByRange` instead. * return `Opt[BlockRef]` instead of `nil` when blocks can't be found - this becomes a lot more common now and thus deserves more attention * `dag.blocks` -> `dag.forkBlocks` - this index only carries unfinalized blocks from now - `finalizedBlocks` covers the other `BlockRef` instances * in backfill, verify that the last backfilled block leads back to genesis, or panic * add backfill timings to log * fix missing check that `BlockRef` block can be fetched with `getForkedBlock` reliably * shortcut doppelganger check when feature is not enabled * in REST/JSON-RPC, fetch blocks without involving `BlockRef` * fix dag.blocks ref
2022-01-21 11:33:16 +00:00
# A PublicKey or Signature isn't on the BLS12-381 curve
info "Unable to load signature sets",
err = e.error()
return err(BlockError.Invalid)
chaindag: don't keep backfill block table in memory (#3429) This PR names and documents the concept of the archive: a range of slots for which we have degraded functionality in terms of historical access - in particular: * we don't support rewinding to states in this range * we don't keep an in-memory representation of the block dag The archive de-facto exists in a trusted-node-synced node, but this PR gives it a name and drops the in-memory digest index. In order to satisfy `GetBlocksByRange` requests, we ensure that we have blocks for the entire archive period via backfill. Future versions may relax this further, adding a "pre-archive" period that is fully pruned. During by-slot searches in the archive (both for libp2p and rest requests), an extra database lookup is used to covert the given `slot` to a `root` - future versions will avoid this using era files which natively are indexed by `slot`. That said, the lookup is quite fast compared to the actual block loading given how trivial the table is - it's hard to measure, even. A collateral benefit of this PR is that checkpoint-synced nodes will see 100-200MB memory usage savings, thanks to the dropped in-memory cache - future pruning work will bring this benefit to full nodes as well. * document chaindag storage architecture and assumptions * look up parent using block id instead of full block in clearance (future-proofing the code against a future in which blocks come from era files) * simplify finalized block init, always writing the backfill portion to db at startup (to ensure lookups work as expected) * preallocate some extra memory for finalized blocks, to avoid immediate realloc
2022-02-26 18:16:19 +00:00
if not verifier.batchVerify(sigs):
limit by-root requests to non-finalized blocks (#3293) * limit by-root requests to non-finalized blocks Presently, we keep a mapping from block root to `BlockRef` in memory - this has simplified reasoning about the dag, but is not sustainable with the chain growing. We can distinguish between two cases where by-root access is useful: * unfinalized blocks - this is where the beacon chain is operating generally, by validating incoming data as interesting for future fork choice decisions - bounded by the length of the unfinalized period * finalized blocks - historical access in the REST API etc - no bounds, really In this PR, we limit the by-root block index to the first use case: finalized chain data can more efficiently be addressed by slot number. Future work includes: * limiting the `BlockRef` horizon in general - each instance is 40 bytes+overhead which adds up - this needs further refactoring to deal with the tail vs state problem * persisting the finalized slot-to-hash index - this one also keeps growing unbounded (albeit slowly) Anyway, this PR easily shaves ~128mb of memory usage at the time of writing. * No longer honor `BeaconBlocksByRoot` requests outside of the non-finalized period - previously, Nimbus would generously return any block through this libp2p request - per the spec, finalized blocks should be fetched via `BeaconBlocksByRange` instead. * return `Opt[BlockRef]` instead of `nil` when blocks can't be found - this becomes a lot more common now and thus deserves more attention * `dag.blocks` -> `dag.forkBlocks` - this index only carries unfinalized blocks from now - `finalizedBlocks` covers the other `BlockRef` instances * in backfill, verify that the last backfilled block leads back to genesis, or panic * add backfill timings to log * fix missing check that `BlockRef` block can be fetched with `getForkedBlock` reliably * shortcut doppelganger check when feature is not enabled * in REST/JSON-RPC, fetch blocks without involving `BlockRef` * fix dag.blocks ref
2022-01-21 11:33:16 +00:00
info "Block signature verification failed",
signature = shortLog(signedBlock.signature)
return err(BlockError.Invalid)
let sigVerifyTick = Moment.now()
? checkStateTransition(dag, signedBlock.asSigVerified(), cache)
let stateVerifyTick = Moment.now()
# Careful, clearanceState.data has been updated but not blck - we need to
# create the BlockRef first!
ok addResolvedHeadBlock(
dag, dag.clearanceState,
signedBlock.asTrusted(),
parent, cache,
onBlockAdded,
stateDataDur = stateDataTick - startTick,
sigVerifyDur = sigVerifyTick - stateDataTick,
stateVerifyDur = stateVerifyTick - sigVerifyTick)
proc addBackfillBlock*(
dag: ChainDAGRef,
signedBlock: ForkySignedBeaconBlock): Result[void, BlockError] =
## 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
## earliest block we know about.
##
## Because only one history is relevant when backfilling, one doesn't have to
## consider forks or other finalization-related issues - a block is either
## valid and finalized, or not.
logScope:
blockRoot = shortLog(signedBlock.root)
blck = shortLog(signedBlock.message)
chaindag: don't keep backfill block table in memory (#3429) This PR names and documents the concept of the archive: a range of slots for which we have degraded functionality in terms of historical access - in particular: * we don't support rewinding to states in this range * we don't keep an in-memory representation of the block dag The archive de-facto exists in a trusted-node-synced node, but this PR gives it a name and drops the in-memory digest index. In order to satisfy `GetBlocksByRange` requests, we ensure that we have blocks for the entire archive period via backfill. Future versions may relax this further, adding a "pre-archive" period that is fully pruned. During by-slot searches in the archive (both for libp2p and rest requests), an extra database lookup is used to covert the given `slot` to a `root` - future versions will avoid this using era files which natively are indexed by `slot`. That said, the lookup is quite fast compared to the actual block loading given how trivial the table is - it's hard to measure, even. A collateral benefit of this PR is that checkpoint-synced nodes will see 100-200MB memory usage savings, thanks to the dropped in-memory cache - future pruning work will bring this benefit to full nodes as well. * document chaindag storage architecture and assumptions * look up parent using block id instead of full block in clearance (future-proofing the code against a future in which blocks come from era files) * simplify finalized block init, always writing the backfill portion to db at startup (to ensure lookups work as expected) * preallocate some extra memory for finalized blocks, to avoid immediate realloc
2022-02-26 18:16:19 +00:00
signature = shortLog(signedBlock.signature)
backfill = (dag.backfill.slot, shortLog(dag.backfill.parent_root))
template blck(): untyped = signedBlock.message # shortcuts without copy
template blockRoot(): untyped = signedBlock.root
limit by-root requests to non-finalized blocks (#3293) * limit by-root requests to non-finalized blocks Presently, we keep a mapping from block root to `BlockRef` in memory - this has simplified reasoning about the dag, but is not sustainable with the chain growing. We can distinguish between two cases where by-root access is useful: * unfinalized blocks - this is where the beacon chain is operating generally, by validating incoming data as interesting for future fork choice decisions - bounded by the length of the unfinalized period * finalized blocks - historical access in the REST API etc - no bounds, really In this PR, we limit the by-root block index to the first use case: finalized chain data can more efficiently be addressed by slot number. Future work includes: * limiting the `BlockRef` horizon in general - each instance is 40 bytes+overhead which adds up - this needs further refactoring to deal with the tail vs state problem * persisting the finalized slot-to-hash index - this one also keeps growing unbounded (albeit slowly) Anyway, this PR easily shaves ~128mb of memory usage at the time of writing. * No longer honor `BeaconBlocksByRoot` requests outside of the non-finalized period - previously, Nimbus would generously return any block through this libp2p request - per the spec, finalized blocks should be fetched via `BeaconBlocksByRange` instead. * return `Opt[BlockRef]` instead of `nil` when blocks can't be found - this becomes a lot more common now and thus deserves more attention * `dag.blocks` -> `dag.forkBlocks` - this index only carries unfinalized blocks from now - `finalizedBlocks` covers the other `BlockRef` instances * in backfill, verify that the last backfilled block leads back to genesis, or panic * add backfill timings to log * fix missing check that `BlockRef` block can be fetched with `getForkedBlock` reliably * shortcut doppelganger check when feature is not enabled * in REST/JSON-RPC, fetch blocks without involving `BlockRef` * fix dag.blocks ref
2022-01-21 11:33:16 +00:00
let startTick = Moment.now()
if blck.slot >= dag.backfill.slot:
chaindag: don't keep backfill block table in memory (#3429) This PR names and documents the concept of the archive: a range of slots for which we have degraded functionality in terms of historical access - in particular: * we don't support rewinding to states in this range * we don't keep an in-memory representation of the block dag The archive de-facto exists in a trusted-node-synced node, but this PR gives it a name and drops the in-memory digest index. In order to satisfy `GetBlocksByRange` requests, we ensure that we have blocks for the entire archive period via backfill. Future versions may relax this further, adding a "pre-archive" period that is fully pruned. During by-slot searches in the archive (both for libp2p and rest requests), an extra database lookup is used to covert the given `slot` to a `root` - future versions will avoid this using era files which natively are indexed by `slot`. That said, the lookup is quite fast compared to the actual block loading given how trivial the table is - it's hard to measure, even. A collateral benefit of this PR is that checkpoint-synced nodes will see 100-200MB memory usage savings, thanks to the dropped in-memory cache - future pruning work will bring this benefit to full nodes as well. * document chaindag storage architecture and assumptions * look up parent using block id instead of full block in clearance (future-proofing the code against a future in which blocks come from era files) * simplify finalized block init, always writing the backfill portion to db at startup (to ensure lookups work as expected) * preallocate some extra memory for finalized blocks, to avoid immediate realloc
2022-02-26 18:16:19 +00:00
let existing = dag.getBlockIdAtSlot(blck.slot)
if existing.isSome:
if existing.get().bid.slot == blck.slot and
existing.get().bid.root == blockRoot:
# We should not call the block added callback for blocks that already
# existed in the pool, as that may confuse consumers such as the fork
# choice.
debug "Duplicate block"
return err(BlockError.Duplicate)
# Block is older than finalized, but different from the block in our
# canonical history: it must be from an unviable branch
debug "Block from unviable fork",
existing = shortLog(existing.get()),
finalizedHead = shortLog(dag.finalizedHead)
return err(BlockError.UnviableFork)
era: load blocks and states (#3394) * era: load blocks and states Era files contain finalized history and can be thought of as an alternative source for block and state data that allows clients to avoid syncing this information from the P2P network - the P2P network is then used to "top up" the client with the most recent data. They can be freely shared in the community via whatever means (http, torrent, etc) and serve as a permanent cold store of consensus data (and, after the merge, execution data) for history buffs and bean counters alike. This PR gently introduces support for loading blocks and states in two cases: block requests from rest/p2p and frontfilling when doing checkpoint sync. The era files are used as a secondary source if the information is not found in the database - compared to the database, there are a few key differences: * the database stores the block indexed by block root while the era file indexes by slot - the former is used only in rest, while the latter is used both by p2p and rest. * when loading blocks from era files, the root is no longer trivially available - if it is needed, it must either be computed (slow) or cached (messy) - the good news is that for p2p requests, it is not needed * in era files, "framed" snappy encoding is used while in the database we store unframed snappy - for p2p2 requests, the latter requires recompression while the former could avoid it * front-filling is the process of using era files to replace backfilling - in theory this front-filling could happen from any block and front-fills with gaps could also be entertained, but our backfilling algorithm cannot take advantage of this because there's no (simple) way to tell it to "skip" a range. * front-filling, as implemented, is a bit slow (10s to load mainnet): we load the full BeaconState for every era to grab the roots of the blocks - it would be better to partially load the state - as such, it would also be good to be able to partially decompress snappy blobs * lookups from REST via root are served by first looking up a block summary in the database, then using the slot to load the block data from the era file - however, there needs to be an option to create the summary table from era files to fully support historical queries To test this, `ncli_db` has an era file exporter: the files it creates should be placed in an `era` folder next to `db` in the data directory. What's interesting in particular about this setup is that `db` remains as the source of truth for security purposes - it stores the latest synced head root which in turn determines where a node "starts" its consensus participation - the era directory however can be freely shared between nodes / people without any (significant) security implications, assuming the era files are consistent / not broken. There's lots of future improvements to be had: * we can drop the in-memory `BlockRef` index almost entirely - at this point, resident memory usage of Nimbus should drop to a cool 500-600 mb * we could serve era files via REST trivially: this would drop backfill times to whatever time it takes to download the files - unlike the current implementation that downloads block by block, downloading an era at a time almost entirely cuts out request overhead * we can "reasonably" recreate detailed state history from almost any point in time, turning an O(slot) process into O(1) effectively - we'll still need caches and indices to do this with sufficient efficiency for the rest api, but at least it cuts the whole process down to minutes instead of hours, for arbitrary points in time * CI: ignore failures with Nim-1.6 (temporary) * test fixes Co-authored-by: Ștefan Talpalaru <stefantalpalaru@yahoo.com>
2022-03-23 08:58:17 +00:00
if blck.slot == dag.frontfill.slot and
dag.backfill.parent_root == dag.frontfill.root:
if blockRoot != dag.frontfill.root:
# We've matched the backfill blocks all the way back to frontfill via the
# `parent_root` chain and ended up at a different block - one way this
limit by-root requests to non-finalized blocks (#3293) * limit by-root requests to non-finalized blocks Presently, we keep a mapping from block root to `BlockRef` in memory - this has simplified reasoning about the dag, but is not sustainable with the chain growing. We can distinguish between two cases where by-root access is useful: * unfinalized blocks - this is where the beacon chain is operating generally, by validating incoming data as interesting for future fork choice decisions - bounded by the length of the unfinalized period * finalized blocks - historical access in the REST API etc - no bounds, really In this PR, we limit the by-root block index to the first use case: finalized chain data can more efficiently be addressed by slot number. Future work includes: * limiting the `BlockRef` horizon in general - each instance is 40 bytes+overhead which adds up - this needs further refactoring to deal with the tail vs state problem * persisting the finalized slot-to-hash index - this one also keeps growing unbounded (albeit slowly) Anyway, this PR easily shaves ~128mb of memory usage at the time of writing. * No longer honor `BeaconBlocksByRoot` requests outside of the non-finalized period - previously, Nimbus would generously return any block through this libp2p request - per the spec, finalized blocks should be fetched via `BeaconBlocksByRange` instead. * return `Opt[BlockRef]` instead of `nil` when blocks can't be found - this becomes a lot more common now and thus deserves more attention * `dag.blocks` -> `dag.forkBlocks` - this index only carries unfinalized blocks from now - `finalizedBlocks` covers the other `BlockRef` instances * in backfill, verify that the last backfilled block leads back to genesis, or panic * add backfill timings to log * fix missing check that `BlockRef` block can be fetched with `getForkedBlock` reliably * shortcut doppelganger check when feature is not enabled * in REST/JSON-RPC, fetch blocks without involving `BlockRef` * fix dag.blocks ref
2022-01-21 11:33:16 +00:00
# can happen is when an invalid `--network` parameter is given during
# startup (though in theory, we check that - maybe the database was
# swapped or something?).
chaindag: don't keep backfill block table in memory (#3429) This PR names and documents the concept of the archive: a range of slots for which we have degraded functionality in terms of historical access - in particular: * we don't support rewinding to states in this range * we don't keep an in-memory representation of the block dag The archive de-facto exists in a trusted-node-synced node, but this PR gives it a name and drops the in-memory digest index. In order to satisfy `GetBlocksByRange` requests, we ensure that we have blocks for the entire archive period via backfill. Future versions may relax this further, adding a "pre-archive" period that is fully pruned. During by-slot searches in the archive (both for libp2p and rest requests), an extra database lookup is used to covert the given `slot` to a `root` - future versions will avoid this using era files which natively are indexed by `slot`. That said, the lookup is quite fast compared to the actual block loading given how trivial the table is - it's hard to measure, even. A collateral benefit of this PR is that checkpoint-synced nodes will see 100-200MB memory usage savings, thanks to the dropped in-memory cache - future pruning work will bring this benefit to full nodes as well. * document chaindag storage architecture and assumptions * look up parent using block id instead of full block in clearance (future-proofing the code against a future in which blocks come from era files) * simplify finalized block init, always writing the backfill portion to db at startup (to ensure lookups work as expected) * preallocate some extra memory for finalized blocks, to avoid immediate realloc
2022-02-26 18:16:19 +00:00
fatal "Checkpoint given during initial startup inconsistent with genesis block - wrong network used when starting the node?",
genesis = shortLog(dag.genesis), tail = shortLog(dag.tail),
head = shortLog(dag.head)
limit by-root requests to non-finalized blocks (#3293) * limit by-root requests to non-finalized blocks Presently, we keep a mapping from block root to `BlockRef` in memory - this has simplified reasoning about the dag, but is not sustainable with the chain growing. We can distinguish between two cases where by-root access is useful: * unfinalized blocks - this is where the beacon chain is operating generally, by validating incoming data as interesting for future fork choice decisions - bounded by the length of the unfinalized period * finalized blocks - historical access in the REST API etc - no bounds, really In this PR, we limit the by-root block index to the first use case: finalized chain data can more efficiently be addressed by slot number. Future work includes: * limiting the `BlockRef` horizon in general - each instance is 40 bytes+overhead which adds up - this needs further refactoring to deal with the tail vs state problem * persisting the finalized slot-to-hash index - this one also keeps growing unbounded (albeit slowly) Anyway, this PR easily shaves ~128mb of memory usage at the time of writing. * No longer honor `BeaconBlocksByRoot` requests outside of the non-finalized period - previously, Nimbus would generously return any block through this libp2p request - per the spec, finalized blocks should be fetched via `BeaconBlocksByRange` instead. * return `Opt[BlockRef]` instead of `nil` when blocks can't be found - this becomes a lot more common now and thus deserves more attention * `dag.blocks` -> `dag.forkBlocks` - this index only carries unfinalized blocks from now - `finalizedBlocks` covers the other `BlockRef` instances * in backfill, verify that the last backfilled block leads back to genesis, or panic * add backfill timings to log * fix missing check that `BlockRef` block can be fetched with `getForkedBlock` reliably * shortcut doppelganger check when feature is not enabled * in REST/JSON-RPC, fetch blocks without involving `BlockRef` * fix dag.blocks ref
2022-01-21 11:33:16 +00:00
quit 1
dag.backfill = blck.toBeaconBlockSummary()
chaindag: don't keep backfill block table in memory (#3429) This PR names and documents the concept of the archive: a range of slots for which we have degraded functionality in terms of historical access - in particular: * we don't support rewinding to states in this range * we don't keep an in-memory representation of the block dag The archive de-facto exists in a trusted-node-synced node, but this PR gives it a name and drops the in-memory digest index. In order to satisfy `GetBlocksByRange` requests, we ensure that we have blocks for the entire archive period via backfill. Future versions may relax this further, adding a "pre-archive" period that is fully pruned. During by-slot searches in the archive (both for libp2p and rest requests), an extra database lookup is used to covert the given `slot` to a `root` - future versions will avoid this using era files which natively are indexed by `slot`. That said, the lookup is quite fast compared to the actual block loading given how trivial the table is - it's hard to measure, even. A collateral benefit of this PR is that checkpoint-synced nodes will see 100-200MB memory usage savings, thanks to the dropped in-memory cache - future pruning work will bring this benefit to full nodes as well. * document chaindag storage architecture and assumptions * look up parent using block id instead of full block in clearance (future-proofing the code against a future in which blocks come from era files) * simplify finalized block init, always writing the backfill portion to db at startup (to ensure lookups work as expected) * preallocate some extra memory for finalized blocks, to avoid immediate realloc
2022-02-26 18:16:19 +00:00
dag.db.finalizedBlocks.insert(blck.slot, blockRoot)
era: load blocks and states (#3394) * era: load blocks and states Era files contain finalized history and can be thought of as an alternative source for block and state data that allows clients to avoid syncing this information from the P2P network - the P2P network is then used to "top up" the client with the most recent data. They can be freely shared in the community via whatever means (http, torrent, etc) and serve as a permanent cold store of consensus data (and, after the merge, execution data) for history buffs and bean counters alike. This PR gently introduces support for loading blocks and states in two cases: block requests from rest/p2p and frontfilling when doing checkpoint sync. The era files are used as a secondary source if the information is not found in the database - compared to the database, there are a few key differences: * the database stores the block indexed by block root while the era file indexes by slot - the former is used only in rest, while the latter is used both by p2p and rest. * when loading blocks from era files, the root is no longer trivially available - if it is needed, it must either be computed (slow) or cached (messy) - the good news is that for p2p requests, it is not needed * in era files, "framed" snappy encoding is used while in the database we store unframed snappy - for p2p2 requests, the latter requires recompression while the former could avoid it * front-filling is the process of using era files to replace backfilling - in theory this front-filling could happen from any block and front-fills with gaps could also be entertained, but our backfilling algorithm cannot take advantage of this because there's no (simple) way to tell it to "skip" a range. * front-filling, as implemented, is a bit slow (10s to load mainnet): we load the full BeaconState for every era to grab the roots of the blocks - it would be better to partially load the state - as such, it would also be good to be able to partially decompress snappy blobs * lookups from REST via root are served by first looking up a block summary in the database, then using the slot to load the block data from the era file - however, there needs to be an option to create the summary table from era files to fully support historical queries To test this, `ncli_db` has an era file exporter: the files it creates should be placed in an `era` folder next to `db` in the data directory. What's interesting in particular about this setup is that `db` remains as the source of truth for security purposes - it stores the latest synced head root which in turn determines where a node "starts" its consensus participation - the era directory however can be freely shared between nodes / people without any (significant) security implications, assuming the era files are consistent / not broken. There's lots of future improvements to be had: * we can drop the in-memory `BlockRef` index almost entirely - at this point, resident memory usage of Nimbus should drop to a cool 500-600 mb * we could serve era files via REST trivially: this would drop backfill times to whatever time it takes to download the files - unlike the current implementation that downloads block by block, downloading an era at a time almost entirely cuts out request overhead * we can "reasonably" recreate detailed state history from almost any point in time, turning an O(slot) process into O(1) effectively - we'll still need caches and indices to do this with sufficient efficiency for the rest api, but at least it cuts the whole process down to minutes instead of hours, for arbitrary points in time * CI: ignore failures with Nim-1.6 (temporary) * test fixes Co-authored-by: Ștefan Talpalaru <stefantalpalaru@yahoo.com>
2022-03-23 08:58:17 +00:00
dag.updateFrontfillBlocks()
limit by-root requests to non-finalized blocks (#3293) * limit by-root requests to non-finalized blocks Presently, we keep a mapping from block root to `BlockRef` in memory - this has simplified reasoning about the dag, but is not sustainable with the chain growing. We can distinguish between two cases where by-root access is useful: * unfinalized blocks - this is where the beacon chain is operating generally, by validating incoming data as interesting for future fork choice decisions - bounded by the length of the unfinalized period * finalized blocks - historical access in the REST API etc - no bounds, really In this PR, we limit the by-root block index to the first use case: finalized chain data can more efficiently be addressed by slot number. Future work includes: * limiting the `BlockRef` horizon in general - each instance is 40 bytes+overhead which adds up - this needs further refactoring to deal with the tail vs state problem * persisting the finalized slot-to-hash index - this one also keeps growing unbounded (albeit slowly) Anyway, this PR easily shaves ~128mb of memory usage at the time of writing. * No longer honor `BeaconBlocksByRoot` requests outside of the non-finalized period - previously, Nimbus would generously return any block through this libp2p request - per the spec, finalized blocks should be fetched via `BeaconBlocksByRange` instead. * return `Opt[BlockRef]` instead of `nil` when blocks can't be found - this becomes a lot more common now and thus deserves more attention * `dag.blocks` -> `dag.forkBlocks` - this index only carries unfinalized blocks from now - `finalizedBlocks` covers the other `BlockRef` instances * in backfill, verify that the last backfilled block leads back to genesis, or panic * add backfill timings to log * fix missing check that `BlockRef` block can be fetched with `getForkedBlock` reliably * shortcut doppelganger check when feature is not enabled * in REST/JSON-RPC, fetch blocks without involving `BlockRef` * fix dag.blocks ref
2022-01-21 11:33:16 +00:00
chaindag: don't keep backfill block table in memory (#3429) This PR names and documents the concept of the archive: a range of slots for which we have degraded functionality in terms of historical access - in particular: * we don't support rewinding to states in this range * we don't keep an in-memory representation of the block dag The archive de-facto exists in a trusted-node-synced node, but this PR gives it a name and drops the in-memory digest index. In order to satisfy `GetBlocksByRange` requests, we ensure that we have blocks for the entire archive period via backfill. Future versions may relax this further, adding a "pre-archive" period that is fully pruned. During by-slot searches in the archive (both for libp2p and rest requests), an extra database lookup is used to covert the given `slot` to a `root` - future versions will avoid this using era files which natively are indexed by `slot`. That said, the lookup is quite fast compared to the actual block loading given how trivial the table is - it's hard to measure, even. A collateral benefit of this PR is that checkpoint-synced nodes will see 100-200MB memory usage savings, thanks to the dropped in-memory cache - future pruning work will bring this benefit to full nodes as well. * document chaindag storage architecture and assumptions * look up parent using block id instead of full block in clearance (future-proofing the code against a future in which blocks come from era files) * simplify finalized block init, always writing the backfill portion to db at startup (to ensure lookups work as expected) * preallocate some extra memory for finalized blocks, to avoid immediate realloc
2022-02-26 18:16:19 +00:00
notice "Received final block during backfill, backfill complete"
limit by-root requests to non-finalized blocks (#3293) * limit by-root requests to non-finalized blocks Presently, we keep a mapping from block root to `BlockRef` in memory - this has simplified reasoning about the dag, but is not sustainable with the chain growing. We can distinguish between two cases where by-root access is useful: * unfinalized blocks - this is where the beacon chain is operating generally, by validating incoming data as interesting for future fork choice decisions - bounded by the length of the unfinalized period * finalized blocks - historical access in the REST API etc - no bounds, really In this PR, we limit the by-root block index to the first use case: finalized chain data can more efficiently be addressed by slot number. Future work includes: * limiting the `BlockRef` horizon in general - each instance is 40 bytes+overhead which adds up - this needs further refactoring to deal with the tail vs state problem * persisting the finalized slot-to-hash index - this one also keeps growing unbounded (albeit slowly) Anyway, this PR easily shaves ~128mb of memory usage at the time of writing. * No longer honor `BeaconBlocksByRoot` requests outside of the non-finalized period - previously, Nimbus would generously return any block through this libp2p request - per the spec, finalized blocks should be fetched via `BeaconBlocksByRange` instead. * return `Opt[BlockRef]` instead of `nil` when blocks can't be found - this becomes a lot more common now and thus deserves more attention * `dag.blocks` -> `dag.forkBlocks` - this index only carries unfinalized blocks from now - `finalizedBlocks` covers the other `BlockRef` instances * in backfill, verify that the last backfilled block leads back to genesis, or panic * add backfill timings to log * fix missing check that `BlockRef` block can be fetched with `getForkedBlock` reliably * shortcut doppelganger check when feature is not enabled * in REST/JSON-RPC, fetch blocks without involving `BlockRef` * fix dag.blocks ref
2022-01-21 11:33:16 +00:00
# Backfill done - dag.backfill.slot now points to genesis block just like
chaindag: don't keep backfill block table in memory (#3429) This PR names and documents the concept of the archive: a range of slots for which we have degraded functionality in terms of historical access - in particular: * we don't support rewinding to states in this range * we don't keep an in-memory representation of the block dag The archive de-facto exists in a trusted-node-synced node, but this PR gives it a name and drops the in-memory digest index. In order to satisfy `GetBlocksByRange` requests, we ensure that we have blocks for the entire archive period via backfill. Future versions may relax this further, adding a "pre-archive" period that is fully pruned. During by-slot searches in the archive (both for libp2p and rest requests), an extra database lookup is used to covert the given `slot` to a `root` - future versions will avoid this using era files which natively are indexed by `slot`. That said, the lookup is quite fast compared to the actual block loading given how trivial the table is - it's hard to measure, even. A collateral benefit of this PR is that checkpoint-synced nodes will see 100-200MB memory usage savings, thanks to the dropped in-memory cache - future pruning work will bring this benefit to full nodes as well. * document chaindag storage architecture and assumptions * look up parent using block id instead of full block in clearance (future-proofing the code against a future in which blocks come from era files) * simplify finalized block init, always writing the backfill portion to db at startup (to ensure lookups work as expected) * preallocate some extra memory for finalized blocks, to avoid immediate realloc
2022-02-26 18:16:19 +00:00
# it would if we loaded a fully synced database - returning duplicate
limit by-root requests to non-finalized blocks (#3293) * limit by-root requests to non-finalized blocks Presently, we keep a mapping from block root to `BlockRef` in memory - this has simplified reasoning about the dag, but is not sustainable with the chain growing. We can distinguish between two cases where by-root access is useful: * unfinalized blocks - this is where the beacon chain is operating generally, by validating incoming data as interesting for future fork choice decisions - bounded by the length of the unfinalized period * finalized blocks - historical access in the REST API etc - no bounds, really In this PR, we limit the by-root block index to the first use case: finalized chain data can more efficiently be addressed by slot number. Future work includes: * limiting the `BlockRef` horizon in general - each instance is 40 bytes+overhead which adds up - this needs further refactoring to deal with the tail vs state problem * persisting the finalized slot-to-hash index - this one also keeps growing unbounded (albeit slowly) Anyway, this PR easily shaves ~128mb of memory usage at the time of writing. * No longer honor `BeaconBlocksByRoot` requests outside of the non-finalized period - previously, Nimbus would generously return any block through this libp2p request - per the spec, finalized blocks should be fetched via `BeaconBlocksByRange` instead. * return `Opt[BlockRef]` instead of `nil` when blocks can't be found - this becomes a lot more common now and thus deserves more attention * `dag.blocks` -> `dag.forkBlocks` - this index only carries unfinalized blocks from now - `finalizedBlocks` covers the other `BlockRef` instances * in backfill, verify that the last backfilled block leads back to genesis, or panic * add backfill timings to log * fix missing check that `BlockRef` block can be fetched with `getForkedBlock` reliably * shortcut doppelganger check when feature is not enabled * in REST/JSON-RPC, fetch blocks without involving `BlockRef` * fix dag.blocks ref
2022-01-21 11:33:16 +00:00
# here is appropriate, though one could also call it ... ok?
return err(BlockError.Duplicate)
if dag.backfill.parent_root != blockRoot:
debug "Block does not match expected backfill root"
return err(BlockError.MissingParent) # MissingChild really, but ..
# If the hash is correct, the block itself must be correct, but the root does
# not cover the signature, which we check next
let proposerKey = dag.validatorKey(blck.proposer_index)
if proposerKey.isNone():
chaindag: don't keep backfill block table in memory (#3429) This PR names and documents the concept of the archive: a range of slots for which we have degraded functionality in terms of historical access - in particular: * we don't support rewinding to states in this range * we don't keep an in-memory representation of the block dag The archive de-facto exists in a trusted-node-synced node, but this PR gives it a name and drops the in-memory digest index. In order to satisfy `GetBlocksByRange` requests, we ensure that we have blocks for the entire archive period via backfill. Future versions may relax this further, adding a "pre-archive" period that is fully pruned. During by-slot searches in the archive (both for libp2p and rest requests), an extra database lookup is used to covert the given `slot` to a `root` - future versions will avoid this using era files which natively are indexed by `slot`. That said, the lookup is quite fast compared to the actual block loading given how trivial the table is - it's hard to measure, even. A collateral benefit of this PR is that checkpoint-synced nodes will see 100-200MB memory usage savings, thanks to the dropped in-memory cache - future pruning work will bring this benefit to full nodes as well. * document chaindag storage architecture and assumptions * look up parent using block id instead of full block in clearance (future-proofing the code against a future in which blocks come from era files) * simplify finalized block init, always writing the backfill portion to db at startup (to ensure lookups work as expected) * preallocate some extra memory for finalized blocks, to avoid immediate realloc
2022-02-26 18:16:19 +00:00
# We've verified that the block root matches our expectations by following
# the chain of parents all the way from checkpoint. If all those blocks
# were valid, the proposer_index in this block must also be valid, and we
# should have a key for it but we don't: this is either a bug on our from
# which we cannot recover, or an invalid checkpoint state was given in which
# case we're in trouble.
fatal "Invalid proposer in backfill block - checkpoint state corrupt?",
head = shortLog(dag.head), tail = shortLog(dag.tail),
genesis = shortLog(dag.genesis)
quit 1
if not verify_block_signature(
dag.forkAtEpoch(blck.slot.epoch),
getStateField(dag.headState, genesis_validators_root),
blck.slot,
signedBlock.root,
proposerKey.get(),
signedBlock.signature):
chaindag: don't keep backfill block table in memory (#3429) This PR names and documents the concept of the archive: a range of slots for which we have degraded functionality in terms of historical access - in particular: * we don't support rewinding to states in this range * we don't keep an in-memory representation of the block dag The archive de-facto exists in a trusted-node-synced node, but this PR gives it a name and drops the in-memory digest index. In order to satisfy `GetBlocksByRange` requests, we ensure that we have blocks for the entire archive period via backfill. Future versions may relax this further, adding a "pre-archive" period that is fully pruned. During by-slot searches in the archive (both for libp2p and rest requests), an extra database lookup is used to covert the given `slot` to a `root` - future versions will avoid this using era files which natively are indexed by `slot`. That said, the lookup is quite fast compared to the actual block loading given how trivial the table is - it's hard to measure, even. A collateral benefit of this PR is that checkpoint-synced nodes will see 100-200MB memory usage savings, thanks to the dropped in-memory cache - future pruning work will bring this benefit to full nodes as well. * document chaindag storage architecture and assumptions * look up parent using block id instead of full block in clearance (future-proofing the code against a future in which blocks come from era files) * simplify finalized block init, always writing the backfill portion to db at startup (to ensure lookups work as expected) * preallocate some extra memory for finalized blocks, to avoid immediate realloc
2022-02-26 18:16:19 +00:00
info "Block signature verification failed"
return err(BlockError.Invalid)
chaindag: don't keep backfill block table in memory (#3429) This PR names and documents the concept of the archive: a range of slots for which we have degraded functionality in terms of historical access - in particular: * we don't support rewinding to states in this range * we don't keep an in-memory representation of the block dag The archive de-facto exists in a trusted-node-synced node, but this PR gives it a name and drops the in-memory digest index. In order to satisfy `GetBlocksByRange` requests, we ensure that we have blocks for the entire archive period via backfill. Future versions may relax this further, adding a "pre-archive" period that is fully pruned. During by-slot searches in the archive (both for libp2p and rest requests), an extra database lookup is used to covert the given `slot` to a `root` - future versions will avoid this using era files which natively are indexed by `slot`. That said, the lookup is quite fast compared to the actual block loading given how trivial the table is - it's hard to measure, even. A collateral benefit of this PR is that checkpoint-synced nodes will see 100-200MB memory usage savings, thanks to the dropped in-memory cache - future pruning work will bring this benefit to full nodes as well. * document chaindag storage architecture and assumptions * look up parent using block id instead of full block in clearance (future-proofing the code against a future in which blocks come from era files) * simplify finalized block init, always writing the backfill portion to db at startup (to ensure lookups work as expected) * preallocate some extra memory for finalized blocks, to avoid immediate realloc
2022-02-26 18:16:19 +00:00
limit by-root requests to non-finalized blocks (#3293) * limit by-root requests to non-finalized blocks Presently, we keep a mapping from block root to `BlockRef` in memory - this has simplified reasoning about the dag, but is not sustainable with the chain growing. We can distinguish between two cases where by-root access is useful: * unfinalized blocks - this is where the beacon chain is operating generally, by validating incoming data as interesting for future fork choice decisions - bounded by the length of the unfinalized period * finalized blocks - historical access in the REST API etc - no bounds, really In this PR, we limit the by-root block index to the first use case: finalized chain data can more efficiently be addressed by slot number. Future work includes: * limiting the `BlockRef` horizon in general - each instance is 40 bytes+overhead which adds up - this needs further refactoring to deal with the tail vs state problem * persisting the finalized slot-to-hash index - this one also keeps growing unbounded (albeit slowly) Anyway, this PR easily shaves ~128mb of memory usage at the time of writing. * No longer honor `BeaconBlocksByRoot` requests outside of the non-finalized period - previously, Nimbus would generously return any block through this libp2p request - per the spec, finalized blocks should be fetched via `BeaconBlocksByRange` instead. * return `Opt[BlockRef]` instead of `nil` when blocks can't be found - this becomes a lot more common now and thus deserves more attention * `dag.blocks` -> `dag.forkBlocks` - this index only carries unfinalized blocks from now - `finalizedBlocks` covers the other `BlockRef` instances * in backfill, verify that the last backfilled block leads back to genesis, or panic * add backfill timings to log * fix missing check that `BlockRef` block can be fetched with `getForkedBlock` reliably * shortcut doppelganger check when feature is not enabled * in REST/JSON-RPC, fetch blocks without involving `BlockRef` * fix dag.blocks ref
2022-01-21 11:33:16 +00:00
let sigVerifyTick = Moment.now
dag.putBlock(signedBlock.asTrusted())
chaindag: don't keep backfill block table in memory (#3429) This PR names and documents the concept of the archive: a range of slots for which we have degraded functionality in terms of historical access - in particular: * we don't support rewinding to states in this range * we don't keep an in-memory representation of the block dag The archive de-facto exists in a trusted-node-synced node, but this PR gives it a name and drops the in-memory digest index. In order to satisfy `GetBlocksByRange` requests, we ensure that we have blocks for the entire archive period via backfill. Future versions may relax this further, adding a "pre-archive" period that is fully pruned. During by-slot searches in the archive (both for libp2p and rest requests), an extra database lookup is used to covert the given `slot` to a `root` - future versions will avoid this using era files which natively are indexed by `slot`. That said, the lookup is quite fast compared to the actual block loading given how trivial the table is - it's hard to measure, even. A collateral benefit of this PR is that checkpoint-synced nodes will see 100-200MB memory usage savings, thanks to the dropped in-memory cache - future pruning work will bring this benefit to full nodes as well. * document chaindag storage architecture and assumptions * look up parent using block id instead of full block in clearance (future-proofing the code against a future in which blocks come from era files) * simplify finalized block init, always writing the backfill portion to db at startup (to ensure lookups work as expected) * preallocate some extra memory for finalized blocks, to avoid immediate realloc
2022-02-26 18:16:19 +00:00
dag.db.finalizedBlocks.insert(blck.slot, blockRoot)
limit by-root requests to non-finalized blocks (#3293) * limit by-root requests to non-finalized blocks Presently, we keep a mapping from block root to `BlockRef` in memory - this has simplified reasoning about the dag, but is not sustainable with the chain growing. We can distinguish between two cases where by-root access is useful: * unfinalized blocks - this is where the beacon chain is operating generally, by validating incoming data as interesting for future fork choice decisions - bounded by the length of the unfinalized period * finalized blocks - historical access in the REST API etc - no bounds, really In this PR, we limit the by-root block index to the first use case: finalized chain data can more efficiently be addressed by slot number. Future work includes: * limiting the `BlockRef` horizon in general - each instance is 40 bytes+overhead which adds up - this needs further refactoring to deal with the tail vs state problem * persisting the finalized slot-to-hash index - this one also keeps growing unbounded (albeit slowly) Anyway, this PR easily shaves ~128mb of memory usage at the time of writing. * No longer honor `BeaconBlocksByRoot` requests outside of the non-finalized period - previously, Nimbus would generously return any block through this libp2p request - per the spec, finalized blocks should be fetched via `BeaconBlocksByRange` instead. * return `Opt[BlockRef]` instead of `nil` when blocks can't be found - this becomes a lot more common now and thus deserves more attention * `dag.blocks` -> `dag.forkBlocks` - this index only carries unfinalized blocks from now - `finalizedBlocks` covers the other `BlockRef` instances * in backfill, verify that the last backfilled block leads back to genesis, or panic * add backfill timings to log * fix missing check that `BlockRef` block can be fetched with `getForkedBlock` reliably * shortcut doppelganger check when feature is not enabled * in REST/JSON-RPC, fetch blocks without involving `BlockRef` * fix dag.blocks ref
2022-01-21 11:33:16 +00:00
dag.backfill = blck.toBeaconBlockSummary()
limit by-root requests to non-finalized blocks (#3293) * limit by-root requests to non-finalized blocks Presently, we keep a mapping from block root to `BlockRef` in memory - this has simplified reasoning about the dag, but is not sustainable with the chain growing. We can distinguish between two cases where by-root access is useful: * unfinalized blocks - this is where the beacon chain is operating generally, by validating incoming data as interesting for future fork choice decisions - bounded by the length of the unfinalized period * finalized blocks - historical access in the REST API etc - no bounds, really In this PR, we limit the by-root block index to the first use case: finalized chain data can more efficiently be addressed by slot number. Future work includes: * limiting the `BlockRef` horizon in general - each instance is 40 bytes+overhead which adds up - this needs further refactoring to deal with the tail vs state problem * persisting the finalized slot-to-hash index - this one also keeps growing unbounded (albeit slowly) Anyway, this PR easily shaves ~128mb of memory usage at the time of writing. * No longer honor `BeaconBlocksByRoot` requests outside of the non-finalized period - previously, Nimbus would generously return any block through this libp2p request - per the spec, finalized blocks should be fetched via `BeaconBlocksByRange` instead. * return `Opt[BlockRef]` instead of `nil` when blocks can't be found - this becomes a lot more common now and thus deserves more attention * `dag.blocks` -> `dag.forkBlocks` - this index only carries unfinalized blocks from now - `finalizedBlocks` covers the other `BlockRef` instances * in backfill, verify that the last backfilled block leads back to genesis, or panic * add backfill timings to log * fix missing check that `BlockRef` block can be fetched with `getForkedBlock` reliably * shortcut doppelganger check when feature is not enabled * in REST/JSON-RPC, fetch blocks without involving `BlockRef` * fix dag.blocks ref
2022-01-21 11:33:16 +00:00
let putBlockTick = Moment.now
debug "Block backfilled",
sigVerifyDur = sigVerifyTick - startTick,
putBlockDur = putBlockTick - sigVerifyTick
ok()