# beacon_chain # Copyright (c) 2018-2021 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 std/[options, sequtils, tables, sets], stew/assign2, metrics, snappy, chronicles, ../ssz/[ssz_serialization, merkleization], ../beacon_chain_db, ../extras, ../spec/[ crypto, datatypes, digest, helpers, validator, state_transition, beaconstate], ../beacon_clock, "."/[block_pools_types, block_quarantine] export block_pools_types, helpers, datatypes # https://github.com/ethereum/eth2.0-metrics/blob/master/metrics.md#interop-metrics declareGauge beacon_head_root, "Root of the head block of the beacon chain" declareGauge beacon_head_slot, "Slot of the head block of the beacon chain" # https://github.com/ethereum/eth2.0-metrics/blob/master/metrics.md#interop-metrics declareGauge beacon_finalized_epoch, "Current finalized epoch" # On epoch transition declareGauge beacon_finalized_root, "Current finalized root" # On epoch transition declareGauge beacon_current_justified_epoch, "Current justified epoch" # On epoch transition declareGauge beacon_current_justified_root, "Current justified root" # On epoch transition declareGauge beacon_previous_justified_epoch, "Current previously justified epoch" # On epoch transition declareGauge beacon_previous_justified_root, "Current previously justified root" # On epoch transition declareCounter beacon_reorgs_total, "Total occurrences of reorganizations of the chain" # On fork choice declareCounter beacon_state_data_cache_hits, "EpochRef hits" declareCounter beacon_state_data_cache_misses, "EpochRef misses" declareCounter beacon_state_rewinds, "State database rewinds" declareGauge beacon_active_validators, "Number of validators in the active validator set" declareGauge beacon_pending_deposits, "Number of pending deposits (state.eth1_data.deposit_count - state.eth1_deposit_index)" # On block declareGauge beacon_processed_deposits_total, "Number of total deposits included on chain" # On block logScope: topics = "chaindag" proc putBlock*( dag: var ChainDAGRef, signedBlock: TrustedSignedBeaconBlock) = dag.db.putBlock(signedBlock) proc updateStateData*( dag: ChainDAGRef, state: var StateData, bs: BlockSlot, save: bool, cache: var StateCache) {.gcsafe.} template withStateVars*( stateDataInternal: var StateData, body: untyped): untyped = ## Inject a few more descriptive names for the members of `stateData` - ## the stateData instance may get mutated through these names as well template stateData(): StateData {.inject, used.} = stateDataInternal template hashedState(): HashedBeaconState {.inject, used.} = stateDataInternal.data template state(): BeaconState {.inject, used.} = stateDataInternal.data.data template blck(): BlockRef {.inject, used.} = stateDataInternal.blck template root(): Eth2Digest {.inject, used.} = stateDataInternal.data.root body template withState*( dag: ChainDAGRef, stateData: var StateData, blockSlot: BlockSlot, body: untyped): untyped = ## Helper template that updates stateData to a particular BlockSlot - usage of ## stateData is unsafe outside of block. ## TODO async transformations will lead to a race where stateData gets updated ## while waiting for future to complete - catch this here somehow? var cache {.inject.} = StateCache() updateStateData(dag, stateData, blockSlot, false, cache) withStateVars(stateData): body func parent*(bs: BlockSlot): BlockSlot = ## Return a blockslot representing the previous slot, using the parent block ## if the current slot had a block if bs.slot == Slot(0): BlockSlot(blck: nil, slot: Slot(0)) else: BlockSlot( blck: if bs.slot > bs.blck.slot: bs.blck else: bs.blck.parent, slot: bs.slot - 1 ) func parentOrSlot*(bs: BlockSlot): BlockSlot = ## Return a blockslot representing the previous slot, using the parent block ## with the current slot if the current had a block if bs.blck.isNil(): BlockSlot(blck: nil, slot: Slot(0)) elif bs.slot == bs.blck.slot: BlockSlot(blck: bs.blck.parent, slot: bs.slot) else: BlockSlot(blck: bs.blck, slot: bs.slot - 1) func get_effective_balances*(state: BeaconState): seq[Gwei] = ## Get the balances from a state as counted for fork choice result.newSeq(state.validators.len) # zero-init let epoch = state.get_current_epoch() for i in 0 ..< result.len: # All non-active validators have a 0 balance let validator = unsafeAddr state.validators[i] if validator[].is_active_validator(epoch): result[i] = validator[].effective_balance proc init*( T: type EpochRef, state: BeaconState, cache: var StateCache, prevEpoch: EpochRef): T = let epoch = state.get_current_epoch() epochRef = EpochRef( epoch: epoch, eth1_data: state.eth1_data, eth1_deposit_index: state.eth1_deposit_index, current_justified_checkpoint: state.current_justified_checkpoint, finalized_checkpoint: state.finalized_checkpoint, shuffled_active_validator_indices: cache.get_shuffled_active_validator_indices(state, epoch)) for i in 0'u64..= b.slot or b.parent.isNil: return false doAssert b.slot > b.parent.slot b = b.parent func get_ancestor*(blck: BlockRef, slot: Slot, maxDepth = 100'i64 * 365 * 24 * 60 * 60 div SECONDS_PER_SLOT.int): BlockRef = ## https://github.com/ethereum/eth2.0-specs/blob/v1.0.1/specs/phase0/fork-choice.md#get_ancestor ## Return the most recent block as of the time at `slot` that not more recent ## than `blck` itself doAssert not blck.isNil var blck = blck var depth = 0 while true: if blck.slot <= slot: return blck if blck.parent.isNil: return nil doAssert depth < maxDepth depth += 1 blck = blck.parent func atSlot*(blck: BlockRef, slot: Slot): BlockSlot = ## Return a BlockSlot at a given slot, with the block set to the closest block ## available. If slot comes from before the block, a suitable block ancestor ## will be used, else blck is returned as if all slots after it were empty. ## This helper is useful when imagining what the chain looked like at a ## particular moment in time, or when imagining what it will look like in the ## near future if nothing happens (such as when looking ahead for the next ## block proposal) BlockSlot(blck: blck.get_ancestor(slot), slot: slot) func atEpochStart*(blck: BlockRef, epoch: Epoch): BlockSlot = ## Return the BlockSlot corresponding to the first slot in the given epoch atSlot(blck, epoch.compute_start_slot_at_epoch) func atEpochEnd*(blck: BlockRef, epoch: Epoch): BlockSlot = ## Return the BlockSlot corresponding to the last slot in the given epoch atSlot(blck, (epoch + 1).compute_start_slot_at_epoch - 1) func epochAncestor*(blck: BlockRef, epoch: Epoch): BlockSlot = ## The state transition works by storing information from blocks in a ## "working" area until the epoch transition, then batching work collected ## during the epoch. Thus, last block in the ancestor epochs is the block ## that has an impact on epoch currently considered. ## ## This function returns a BlockSlot pointing to that epoch boundary, ie the ## boundary where the last block has been applied to the state and epoch ## processing has been done. var blck = blck while blck.slot.epoch >= epoch and not blck.parent.isNil: blck = blck.parent blck.atEpochStart(epoch) func findEpochRef*( dag: ChainDAGRef, blck: BlockRef, epoch: Epoch): EpochRef = # may return nil! let ancestor = blck.epochAncestor(epoch) doAssert ancestor.blck != nil for i in 0.. 0: load(epoch - 1) template getStateField*(stateData, fieldName: untyped): untyped = stateData.data.data.fieldName func init(T: type BlockRef, root: Eth2Digest, slot: Slot): BlockRef = BlockRef( root: root, slot: slot ) func init*(T: type BlockRef, root: Eth2Digest, blck: SomeBeaconBlock): BlockRef = BlockRef.init(root, blck.slot) func contains*(dag: ChainDAGRef, root: Eth2Digest): bool = KeyedBlockRef.asLookupKey(root) in dag.blocks proc init*(T: type ChainDAGRef, preset: RuntimePreset, db: BeaconChainDB, updateFlags: UpdateFlags = {}): ChainDAGRef = # TODO we require that the db contains both a head and a tail block - # asserting here doesn't seem like the right way to go about it however.. let tailBlockRoot = db.getTailBlock() headBlockRoot = db.getHeadBlock() doAssert tailBlockRoot.isSome(), "Missing tail block, database corrupt?" doAssert headBlockRoot.isSome(), "Missing head block, database corrupt?" let tailRoot = tailBlockRoot.get() tailBlock = db.getBlock(tailRoot).get() tailRef = BlockRef.init(tailRoot, tailBlock.message) headRoot = headBlockRoot.get() let genesisRef = if tailBlock.message.slot == GENESIS_SLOT: tailRef else: let genesisBlockRoot = db.getGenesisBlockRoot() genesisBlock = db.getBlock(genesisBlockRoot).expect( "preInit should have initialized the database with a genesis block") BlockRef.init(genesisBlockRoot, genesisBlock.message) var blocks: HashSet[KeyedBlockRef] headRef: BlockRef blocks.incl(KeyedBlockRef.init(tailRef)) if genesisRef != tailRef: blocks.incl(KeyedBlockRef.init(genesisRef)) if headRoot != tailRoot: var curRef: BlockRef for blck in db.getAncestorSummaries(headRoot): if blck.root == tailRef.root: doAssert(not curRef.isNil) link(tailRef, curRef) curRef = curRef.parent break let newRef = BlockRef.init(blck.root, blck.summary.slot) if curRef == nil: curRef = newRef headRef = newRef else: link(newRef, curRef) curRef = curRef.parent blocks.incl(KeyedBlockRef.init(curRef)) trace "Populating block dag", key = curRef.root, val = curRef doAssert curRef == tailRef, "head block does not lead to tail, database corrupt?" else: headRef = tailRef var cur = headRef.atSlot(headRef.slot) tmpState = (ref StateData)() # Now that we have a head block, we need to find the most recent state that # we have saved in the database while cur.blck != nil: let root = db.getStateRoot(cur.blck.root, cur.slot) if root.isSome(): if db.getState(root.get(), tmpState.data.data, noRollback): tmpState.data.root = root.get() tmpState.blck = cur.blck break if cur.blck.parent != nil and cur.blck.slot.epoch != epoch(cur.blck.parent.slot): # We store the state of the parent block with the epoch processing applied # in the database! cur = cur.blck.parent.atEpochStart(cur.blck.slot.epoch) else: # Moves back slot by slot, in case a state for an empty slot was saved cur = cur.parent if tmpState.blck == nil: warn "No state found in head history, database corrupt?" # TODO Potentially we could recover from here instead of crashing - what # would be a good recovery model? raiseAssert "No state found in head history, database corrupt?" let res = ChainDAGRef( blocks: blocks, tail: tailRef, genesis: genesisRef, db: db, heads: @[headRef], headState: tmpState[], epochRefState: tmpState[], clearanceState: tmpState[], # The only allowed flag right now is verifyFinalization, as the others all # allow skipping some validation. updateFlags: {verifyFinalization} * updateFlags, runtimePreset: preset, ) doAssert res.updateFlags in [{}, {verifyFinalization}] var cache: StateCache res.updateStateData(res.headState, headRef.atSlot(headRef.slot), false, cache) # We presently save states on the epoch boundary - it means that the latest # state we loaded might be older than head block - nonetheless, it will be # from the same epoch as the head, thus the finalized and justified slots are # the same - these only change on epoch boundaries. # When we start from a snapshot state, the `finalized_checkpoint` in the # snapshot will point to an even older state, but we trust the tail state # (the snapshot) to be finalized, hence the `max` expression below. let finalizedEpoch = max(getStateField(res.headState, finalized_checkpoint).epoch, tailRef.slot.epoch) res.finalizedHead = headRef.atEpochStart(finalizedEpoch) res.clearanceState = res.headState # Pruning metadata res.lastPrunePoint = res.finalizedHead info "Block dag initialized", head = shortLog(headRef), finalizedHead = shortLog(res.finalizedHead), tail = shortLog(tailRef), totalBlocks = blocks.len res proc addEpochRef*(dag: ChainDAGRef, blck: BlockRef, epochRef: EpochRef) = # Because we put a cap on the number of epochRefs we store, we want to # prune the least useful state - for now, we'll assume that to be the oldest # epochRef we know about. var oldest = 0 ancestor = blck.epochAncestor(epochRef.epoch) for x in 0.. 0: var cur = ancestor.blck.epochAncestor(epochRef.epoch - 1) while cur.slot >= dag.finalizedHead.slot: let er = dag.findEpochRef(cur.blck, cur.slot.epoch) if er != nil: er.validator_key_store = epochRef.validator_key_store if cur.slot.epoch == 0: break cur = cur.blck.epochAncestor(cur.slot.epoch - 1) proc getEpochRef*(dag: ChainDAGRef, blck: BlockRef, epoch: Epoch): EpochRef = let epochRef = dag.findEpochRef(blck, epoch) if epochRef != nil: beacon_state_data_cache_hits.inc return epochRef beacon_state_data_cache_misses.inc let ancestor = blck.epochAncestor(epoch) dag.withState(dag.epochRefState, ancestor): let prevEpochRef = if epoch < 1: nil else: dag.findEpochRef(blck, epoch - 1) newEpochRef = EpochRef.init(state, cache, prevEpochRef) if epoch >= dag.finalizedHead.slot.epoch(): # Only cache epoch information for unfinalized blocks - earlier states # are seldomly used (ie RPC), so no need to cache dag.addEpochRef(blck, newEpochRef) newEpochRef proc getFinalizedEpochRef*(dag: ChainDAGRef): EpochRef = dag.getEpochRef(dag.finalizedHead.blck, dag.finalizedHead.slot.epoch) proc getState( dag: ChainDAGRef, state: var StateData, stateRoot: Eth2Digest, blck: BlockRef): bool = let restoreAddr = # Any restore point will do as long as it's not the object being updated if unsafeAddr(state) == unsafeAddr(dag.headState): unsafeAddr dag.clearanceState else: unsafeAddr dag.headState func restore(v: var BeaconState) = assign(v, restoreAddr[].data.data) if not dag.db.getState(stateRoot, state.data.data, restore): return false state.blck = blck state.data.root = stateRoot true func isStateCheckpoint(bs: BlockSlot): bool = ## State checkpoints are the points in time for which we store full state ## snapshots, which later serve as rewind starting points when replaying state ## transitions from database, for example during reorgs. ## # As a policy, we only store epoch boundary states without the epoch block # (if it exists) applied - the rest can be reconstructed by loading an epoch # boundary state and applying the missing blocks. # We also avoid states that were produced with empty slots only - as such, # there is only a checkpoint for the first epoch after a block. # The tail block also counts as a state checkpoint! (bs.slot == bs.blck.slot and bs.blck.parent == nil) or (bs.slot.isEpoch and bs.slot.epoch == (bs.blck.slot.epoch + 1)) func stateCheckpoint*(bs: BlockSlot): BlockSlot = ## The first ancestor BlockSlot that is a state checkpoint var bs = bs while not isStateCheckPoint(bs): bs = bs.parentOrSlot bs proc getState(dag: ChainDAGRef, state: var StateData, bs: BlockSlot): bool = ## Load a state from the database given a block and a slot - this will first ## lookup the state root in the state root table then load the corresponding ## state, if it exists if not bs.isStateCheckpoint(): return false # Only state checkpoints are stored - no need to hit DB if (let stateRoot = dag.db.getStateRoot(bs.blck.root, bs.slot); stateRoot.isSome()): return dag.getState(state, stateRoot.get(), bs.blck) false proc putState*(dag: ChainDAGRef, state: var StateData) = # Store a state and its root logScope: blck = shortLog(state.blck) stateSlot = shortLog(getStateField(state, slot)) stateRoot = shortLog(state.data.root) if not isStateCheckpoint(state.blck.atSlot(getStateField(state, slot))): return if dag.db.containsState(state.data.root): return debug "Storing state" # Ideally we would save the state and the root lookup cache in a single # transaction to prevent database inconsistencies, but the state loading code # is resilient against one or the other going missing dag.db.putState(state.data.root, state.data.data) # Allow backwards-compatible version rollback with bounded recovery cost if getStateField(state, slot).epoch mod 64 == 0: dag.db.putStateFull(state.data.root, state.data.data) dag.db.putStateRoot( state.blck.root, getStateField(state, slot), state.data.root) func getRef*(dag: ChainDAGRef, root: Eth2Digest): BlockRef = ## Retrieve a resolved block reference, if available let key = KeyedBlockRef.asLookupKey(root) # HashSet lacks the api to do check-and-get in one lookup - `[]` will return # the copy of the instance in the set which has more fields than `root` set! if key in dag.blocks: try: dag.blocks[key].blockRef() except KeyError: raiseAssert "contains" else: nil func getBlockRange*( dag: ChainDAGRef, startSlot: Slot, skipStep: uint64, output: var openArray[BlockRef]): Natural = ## This function populates an `output` buffer of blocks ## with a slots ranging from `startSlot` up to, but not including, ## `startSlot + skipStep * output.len`, skipping any slots that don't have ## a block. ## ## Blocks will be written to `output` from the end without gaps, even if ## a block is missing in a particular slot. The return value shows how ## many slots were missing blocks - to iterate over the result, start ## at this index. ## ## If there were no blocks in the range, `output.len` will be returned. let requestedCount = output.lenu64 headSlot = dag.head.slot trace "getBlockRange entered", head = shortLog(dag.head.root), requestedCount, startSlot, skipStep, headSlot if startSlot < dag.tail.slot or headSlot <= startSlot or requestedCount == 0: return output.len # Identical to returning an empty set of block as indicated above let runway = uint64(headSlot - startSlot) # This is the number of blocks that will follow the start block extraBlocks = min(runway div skipStep, requestedCount - 1) # If `skipStep` is very large, `extraBlocks` should be 0 from # the previous line, so `endSlot` will be equal to `startSlot`: endSlot = startSlot + extraBlocks * skipStep var b = dag.head.atSlot(endSlot) o = output.len # Process all blocks that follow the start block (may be zero blocks) for i in 1..extraBlocks: if b.blck.slot == b.slot: dec o output[o] = b.blck for j in 1..skipStep: b = b.parent # We should now be at the start block. # Like any "block slot", it may be a missing/skipped block: if b.blck.slot == b.slot: dec o output[o] = b.blck o # Return the index of the first non-nil item in the output func getBlockBySlot*(dag: ChainDAGRef, slot: Slot): BlockRef = ## Retrieves the first block in the current canonical chain ## with slot number less or equal to `slot`. dag.head.atSlot(slot).blck func getBlockByPreciseSlot*(dag: ChainDAGRef, slot: Slot): BlockRef = ## Retrieves a block from the canonical chain with a slot ## number equal to `slot`. let found = dag.getBlockBySlot(slot) if found.slot != slot: found else: nil proc get*(dag: ChainDAGRef, blck: BlockRef): BlockData = ## Retrieve the associated block body of a block reference doAssert (not blck.isNil), "Trying to get nil BlockRef" let data = dag.db.getBlock(blck.root) doAssert data.isSome, "BlockRef without backing data, database corrupt?" BlockData(data: data.get(), refs: blck) proc get*(dag: ChainDAGRef, root: Eth2Digest): Option[BlockData] = ## Retrieve a resolved block reference and its associated body, if available let refs = dag.getRef(root) if not refs.isNil: some(dag.get(refs)) else: none(BlockData) proc advanceSlots( dag: ChainDAGRef, state: var StateData, slot: Slot, save: bool, cache: var StateCache) = # Given a state, advance it zero or more slots by applying empty slot # processing - the state must be positions at a slot before or equal to the # target doAssert getStateField(state, slot) <= slot while getStateField(state, slot) < slot: doAssert process_slots( state.data, getStateField(state, slot) + 1, cache, dag.updateFlags), "process_slots shouldn't fail when state slot is correct" if save: dag.putState(state) proc applyBlock( dag: ChainDAGRef, state: var StateData, blck: BlockData, flags: UpdateFlags, cache: var StateCache): bool = # Apply a single block to the state - the state must be positioned at the # parent of the block with a slot lower than the one of the block being # applied doAssert state.blck == blck.refs.parent var statePtr = unsafeAddr state # safe because `restore` is locally scoped func restore(v: var HashedBeaconState) = doAssert (addr(statePtr.data) == addr v) statePtr[] = dag.headState loadStateCache(dag, cache, blck.refs, blck.data.message.slot.epoch) let ok = state_transition( dag.runtimePreset, state.data, blck.data, cache, flags + dag.updateFlags + {slotProcessed}, restore) if ok: state.blck = blck.refs ok proc updateStateData*( dag: ChainDAGRef, state: var StateData, bs: BlockSlot, save: bool, cache: var StateCache) = ## Rewind or advance state such that it matches the given block and slot - ## this may include replaying from an earlier snapshot if blck is on a ## different branch or has advanced to a higher slot number than slot ## If slot is higher than blck.slot, replay will fill in with empty/non-block ## slots, else it is ignored # First, see if we're already at the requested block. If we are, also check # that the state has not been advanced past the desired block - if it has, # an earlier state must be loaded since there's no way to undo the slot # transitions let startTime = Moment.now() var ancestors: seq[BlockRef] cur = bs found = false template canAdvance(state: StateData, bs: BlockSlot): bool = # The block is the same and we're at an early enough slot - the state can # be used to arrive at the desired blockslot state.blck == bs.blck and getStateField(state, slot) <= bs.slot # First, run a quick check if we can simply apply a few blocks to an in-memory # state - any in-memory state will be faster than loading from database. # The limit here how many blocks we apply is somewhat arbitrary but two full # epochs (might be more slots if there are skips) seems like a good enough # first guess. # This happens in particular during startup where we replay blocks # sequentially to grab their votes. const RewindBlockThreshold = 64 while ancestors.len < RewindBlockThreshold: if canAdvance(state, cur): found = true break if canAdvance(dag.headState, cur): assign(state, dag.headState) found = true break if canAdvance(dag.clearanceState, cur): assign(state, dag.clearanceState) found = true break if canAdvance(dag.epochRefState, cur): assign(state, dag.epochRefState) found = true break if cur.slot == cur.blck.slot: # This is not an empty slot, so the block will need to be applied to # eventually reach bs ancestors.add(cur.blck) if cur.blck.parent == nil: break # Moving slot by slot helps find states that were advanced with empty slots cur = cur.parentOrSlot # Let's see if we're within a few epochs of the state block - then we can # simply replay blocks without loading the whole state if not found: debug "UpdateStateData cache miss", bs, stateBlock = state.blck, stateSlot = getStateField(state, slot) # Either the state is too new or was created by applying a different block. # We'll now resort to loading the state from the database then reapplying # blocks until we reach the desired point in time. cur = bs ancestors.setLen(0) # Look for a state in the database and load it - as long as it cannot be # found, keep track of the blocks that are needed to reach it from the # state that eventually will be found while not dag.getState(state, cur): # There's no state saved for this particular BlockSlot combination, keep # looking... if cur.blck.parent != nil and cur.blck.slot.epoch != epoch(cur.blck.parent.slot): # We store the state of the parent block with the epoch processing applied # in the database - we'll need to apply the block however! ancestors.add(cur.blck) cur = cur.blck.parent.atEpochStart(cur.blck.slot.epoch) else: if cur.slot == cur.blck.slot: # This is not an empty slot, so the block will need to be applied to # eventually reach bs ancestors.add(cur.blck) # Moves back slot by slot, in case a state for an empty slot was saved cur = cur.parent beacon_state_rewinds.inc() let startSlot {.used.} = getStateField(state, slot) # used in logs below startRoot {.used.} = state.data.root # Time to replay all the blocks between then and now for i in countdown(ancestors.len - 1, 0): # Because the ancestors are in the database, there's no need to persist them # again. Also, because we're applying blocks that were loaded from the # database, we can skip certain checks that have already been performed # before adding the block to the database. let ok = dag.applyBlock(state, dag.get(ancestors[i]), {}, cache) doAssert ok, "Blocks in database should never fail to apply.." loadStateCache(dag, cache, bs.blck, bs.slot.epoch) # ...and make sure to process empty slots as requested dag.advanceSlots(state, bs.slot, save, cache) let diff = Moment.now() - startTime logScope: blocks = ancestors.len slots = getStateField(state, slot) - startSlot stateRoot = shortLog(state.data.root) stateSlot = getStateField(state, slot) startRoot = shortLog(startRoot) startSlot blck = shortLog(bs) found diff = shortLog(diff) if diff >= 1.seconds: # This might indicate there's a cache that's not in order or a disk that is # too slow - for now, it's here for investigative purposes and the cutoff # time might need tuning info "State replayed" elif ancestors.len > 0: debug "State replayed" else: trace "State advanced" # Normal case! proc delState(dag: ChainDAGRef, bs: BlockSlot) = # Delete state state and mapping for a particular block+slot if not bs.slot.isEpoch: return # We only ever save epoch states if (let root = dag.db.getStateRoot(bs.blck.root, bs.slot); root.isSome()): dag.db.delState(root.get()) dag.db.delStateRoot(bs.blck.root, bs.slot) proc pruneBlocksDAG(dag: ChainDAGRef) = ## This prunes the block DAG ## This does NOT prune the cached state checkpoints and EpochRef ## This must be done after a new finalization point is reached ## to invalidate pending blocks or attestations referring ## to a now invalid fork. ## ## This does NOT update the `dag.lastPrunePoint` field. ## as the caches and fork choice can be pruned at a later time. # Clean up block refs, walking block by block let start = Moment.now() # Finalization means that we choose a single chain as the canonical one - # it also means we're no longer interested in any branches from that chain # up to the finalization point let hlen = dag.heads.len for i in 0..