lighthouse/eth2/proto_array_fork_choice/src/proto_array.rs

411 lines
16 KiB
Rust
Raw Normal View History

Add proto_array fork choice (#804) * Start implementing proto_array * Add progress * Add unfinished progress * Add further progress * Add progress * Add tree filtering * Add half-finished modifications * Add refactored version * Tidy, add incomplete LmdGhost impl * Move impls in LmdGhost trait def * Remove old reduced_tree fork choice * Combine two functions in to `compute_deltas` * Start testing * Add more compute_deltas tests * Add fork choice testing * Add more fork choice testing * Add more fork choice tests * Add more testing to proto-array * Remove old tests * Modify tests * Add more tests * Add more testing * Add comments and fixes * Re-organise crate * Tidy, finish pruning tests * Add ssz encoding, other pub fns * Rename lmd_ghost > proto_array_fork_choice * Integrate proto_array into lighthouse * Add first pass at fixing filter * Clean out old comments * Add more comments * Attempt to fix prune error * Adjust TODO * Fix test compile errors * Add extra justification change check * Update cargo.lock * Fix fork choice test compile errors * Most remove ffg_update_required * Fix bug with epoch of attestation votes * Start adding new test format * Make fork choice tests declarative * Create test def concept * Move test defs into crate * Add binary, re-org crate * Shuffle files * Start adding ffg tests * Add more fork choice tests * Add fork choice JSON dumping * Add more detail to best node error * Ensure fin+just checkpoints from from same block * Rename JustificationManager * Move checkpoint manager into own file * Tidy * Add targetted logging for sneaky sync bug * Fix justified balances bug * Add cache metrics * Add metrics for log levels * Fix bug in checkpoint manager * Fix compile error in fork choice tests * Ignore duplicate blocks in fork choice * Add block to fock choice before db * Rename on_new_block fn * Fix spec inconsistency in `CheckpointManager` * Remove BlockRootTree * Remove old reduced_tree code fragment * Add API endpoint for fork choice * Add more ffg tests * Remove block_root_tree reminents * Ensure effective balances are used * Remove old debugging code, fix API fault * Add check to ensure parent block is in fork choice * Update readme dates * Fix readme * Tidy checkpoint manager * Remove fork choice yaml files from repo * Remove fork choice yaml from repo * General tidy * Address majority of Michael's comments * Tidy bin/lib business * Remove dangling file * Undo changes for rpc/handler from master * Revert "Undo changes for rpc/handler from master" This reverts commit 876edff0e4a501aafbb47113454852826dcc24e8. Co-authored-by: Age Manning <Age@AgeManning.com>
2020-01-29 04:05:00 +00:00
use crate::error::Error;
use serde_derive::{Deserialize, Serialize};
use ssz_derive::{Decode, Encode};
use std::collections::HashMap;
use types::{Epoch, Hash256, Slot};
#[derive(Clone, PartialEq, Debug, Encode, Decode, Serialize, Deserialize)]
pub struct ProtoNode {
/// The `slot` is not necessary for `ProtoArray`, it just exists so external components can
/// easily query the block slot. This is useful for upstream fork choice logic.
pub slot: Slot,
Optimize attestation processing (#841) * Start updating types * WIP * Signature hacking * Existing EF tests passing with fake_crypto * Updates * Delete outdated API spec * The refactor continues * It compiles * WIP test fixes * All release tests passing bar genesis state parsing * Update and test YamlConfig * Update to spec v0.10 compatible BLS * Updates to BLS EF tests * Add EF test for AggregateVerify And delete unused hash2curve tests for uncompressed points * Update EF tests to v0.10.1 * Use optional block root correctly in block proc * Use genesis fork in deposit domain. All tests pass * Cargo fmt * Fast aggregate verify test * Update REST API docs * Cargo fmt * Fix unused import * Bump spec tags to v0.10.1 * Add `seconds_per_eth1_block` to chainspec * Update to timestamp based eth1 voting scheme * Return None from `get_votes_to_consider` if block cache is empty * Handle overflows in `is_candidate_block` * Revert to failing tests * Fix eth1 data sets test * Choose default vote according to spec * Fix collect_valid_votes tests * Fix `get_votes_to_consider` to choose all eligible blocks * Uncomment winning_vote tests * Add comments; remove unused code * Reduce seconds_per_eth1_block for simulation * Addressed review comments * Add test for default vote case * Fix logs * Remove unused functions * Meter default eth1 votes * Fix comments * Address review comments; remove unused dependency * Add first attempt at attestation proc. re-write * Add version 2 of attestation processing * Minor fixes * Add validator pubkey cache * Make get_indexed_attestation take a committee * Link signature processing into new attn verification * First working version * Ensure pubkey cache is updated * Add more metrics, slight optimizations * Clone committee cache during attestation processing * Update shuffling cache during block processing * Remove old commented-out code * Fix shuffling cache insert bug * Used indexed attestation in fork choice * Restructure attn processing, add metrics * Add more detailed metrics * Tidy, fix failing tests * Fix failing tests, tidy * Disable/delete two outdated tests * Tidy * Add pubkey cache persistence file * Add more comments * Integrate persistence file into builder * Add pubkey cache tests * Add data_dir to beacon chain builder * Remove Option in pubkey cache persistence file * Ensure consistency between datadir/data_dir * Fix failing network test * Tidy * Fix todos * Add attestation processing tests * Add another test * Only run attestation tests in release * Make attestation tests MainnetEthSpec * Address Michael's comments * Remove redundant check * Fix warning * Fix failing test Co-authored-by: Michael Sproul <micsproul@gmail.com> Co-authored-by: Pawan Dhananjay <pawandhananjay@gmail.com>
2020-03-05 06:19:35 +00:00
/// The `state_root` is not necessary for `ProtoArray` either, it also just exists for upstream
/// components (namely attestation verification).
pub state_root: Hash256,
Add proto_array fork choice (#804) * Start implementing proto_array * Add progress * Add unfinished progress * Add further progress * Add progress * Add tree filtering * Add half-finished modifications * Add refactored version * Tidy, add incomplete LmdGhost impl * Move impls in LmdGhost trait def * Remove old reduced_tree fork choice * Combine two functions in to `compute_deltas` * Start testing * Add more compute_deltas tests * Add fork choice testing * Add more fork choice testing * Add more fork choice tests * Add more testing to proto-array * Remove old tests * Modify tests * Add more tests * Add more testing * Add comments and fixes * Re-organise crate * Tidy, finish pruning tests * Add ssz encoding, other pub fns * Rename lmd_ghost > proto_array_fork_choice * Integrate proto_array into lighthouse * Add first pass at fixing filter * Clean out old comments * Add more comments * Attempt to fix prune error * Adjust TODO * Fix test compile errors * Add extra justification change check * Update cargo.lock * Fix fork choice test compile errors * Most remove ffg_update_required * Fix bug with epoch of attestation votes * Start adding new test format * Make fork choice tests declarative * Create test def concept * Move test defs into crate * Add binary, re-org crate * Shuffle files * Start adding ffg tests * Add more fork choice tests * Add fork choice JSON dumping * Add more detail to best node error * Ensure fin+just checkpoints from from same block * Rename JustificationManager * Move checkpoint manager into own file * Tidy * Add targetted logging for sneaky sync bug * Fix justified balances bug * Add cache metrics * Add metrics for log levels * Fix bug in checkpoint manager * Fix compile error in fork choice tests * Ignore duplicate blocks in fork choice * Add block to fock choice before db * Rename on_new_block fn * Fix spec inconsistency in `CheckpointManager` * Remove BlockRootTree * Remove old reduced_tree code fragment * Add API endpoint for fork choice * Add more ffg tests * Remove block_root_tree reminents * Ensure effective balances are used * Remove old debugging code, fix API fault * Add check to ensure parent block is in fork choice * Update readme dates * Fix readme * Tidy checkpoint manager * Remove fork choice yaml files from repo * Remove fork choice yaml from repo * General tidy * Address majority of Michael's comments * Tidy bin/lib business * Remove dangling file * Undo changes for rpc/handler from master * Revert "Undo changes for rpc/handler from master" This reverts commit 876edff0e4a501aafbb47113454852826dcc24e8. Co-authored-by: Age Manning <Age@AgeManning.com>
2020-01-29 04:05:00 +00:00
root: Hash256,
parent: Option<usize>,
justified_epoch: Epoch,
finalized_epoch: Epoch,
weight: u64,
best_child: Option<usize>,
best_descendant: Option<usize>,
}
#[derive(PartialEq, Debug, Serialize, Deserialize)]
pub struct ProtoArray {
/// Do not attempt to prune the tree unless it has at least this many nodes. Small prunes
/// simply waste time.
pub prune_threshold: usize,
pub justified_epoch: Epoch,
pub finalized_epoch: Epoch,
pub nodes: Vec<ProtoNode>,
pub indices: HashMap<Hash256, usize>,
}
impl ProtoArray {
/// Iterate backwards through the array, touching all nodes and their parents and potentially
/// the best-child of each parent.
///
/// The structure of the `self.nodes` array ensures that the child of each node is always
/// touched before its parent.
///
/// For each node, the following is done:
///
/// - Update the node's weight with the corresponding delta.
/// - Back-propagate each node's delta to its parents delta.
/// - Compare the current node with the parents best-child, updating it if the current node
/// should become the best child.
/// - If required, update the parents best-descendant with the current node or its best-descendant.
pub fn apply_score_changes(
&mut self,
mut deltas: Vec<i64>,
justified_epoch: Epoch,
finalized_epoch: Epoch,
) -> Result<(), Error> {
if deltas.len() != self.indices.len() {
return Err(Error::InvalidDeltaLen {
deltas: deltas.len(),
indices: self.indices.len(),
});
}
if justified_epoch != self.justified_epoch || finalized_epoch != self.finalized_epoch {
self.justified_epoch = justified_epoch;
self.finalized_epoch = finalized_epoch;
}
// Iterate backwards through all indices in `self.nodes`.
for node_index in (0..self.nodes.len()).rev() {
let node = self
.nodes
.get_mut(node_index)
.ok_or_else(|| Error::InvalidNodeIndex(node_index))?;
// There is no need to adjust the balances or manage parent of the zero hash since it
// is an alias to the genesis block. The weight applied to the genesis block is
// irrelevant as we _always_ choose it and it's impossible for it to have a parent.
if node.root == Hash256::zero() {
continue;
}
let node_delta = deltas
.get(node_index)
.copied()
.ok_or_else(|| Error::InvalidNodeDelta(node_index))?;
// Apply the delta to the node.
if node_delta < 0 {
// Note: I am conflicted about whether to use `saturating_sub` or `checked_sub`
// here.
//
// I can't think of any valid reason why `node_delta.abs()` should be greater than
// `node.weight`, so I have chosen `checked_sub` to try and fail-fast if there is
// some error.
//
// However, I am not fully convinced that some valid case for `saturating_sub` does
// not exist.
node.weight = node
.weight
.checked_sub(node_delta.abs() as u64)
.ok_or_else(|| Error::DeltaOverflow(node_index))?;
} else {
node.weight = node
.weight
.checked_add(node_delta as u64)
.ok_or_else(|| Error::DeltaOverflow(node_index))?;
}
// If the node has a parent, try to update its best-child and best-descendant.
if let Some(parent_index) = node.parent {
let parent_delta = deltas
.get_mut(parent_index)
.ok_or_else(|| Error::InvalidParentDelta(parent_index))?;
// Back-propagate the nodes delta to its parent.
*parent_delta += node_delta;
self.maybe_update_best_child_and_descendant(parent_index, node_index)?;
}
}
Ok(())
}
/// Register a block with the fork choice.
///
/// It is only sane to supply a `None` parent for the genesis block.
pub fn on_block(
&mut self,
slot: Slot,
root: Hash256,
parent_opt: Option<Hash256>,
Optimize attestation processing (#841) * Start updating types * WIP * Signature hacking * Existing EF tests passing with fake_crypto * Updates * Delete outdated API spec * The refactor continues * It compiles * WIP test fixes * All release tests passing bar genesis state parsing * Update and test YamlConfig * Update to spec v0.10 compatible BLS * Updates to BLS EF tests * Add EF test for AggregateVerify And delete unused hash2curve tests for uncompressed points * Update EF tests to v0.10.1 * Use optional block root correctly in block proc * Use genesis fork in deposit domain. All tests pass * Cargo fmt * Fast aggregate verify test * Update REST API docs * Cargo fmt * Fix unused import * Bump spec tags to v0.10.1 * Add `seconds_per_eth1_block` to chainspec * Update to timestamp based eth1 voting scheme * Return None from `get_votes_to_consider` if block cache is empty * Handle overflows in `is_candidate_block` * Revert to failing tests * Fix eth1 data sets test * Choose default vote according to spec * Fix collect_valid_votes tests * Fix `get_votes_to_consider` to choose all eligible blocks * Uncomment winning_vote tests * Add comments; remove unused code * Reduce seconds_per_eth1_block for simulation * Addressed review comments * Add test for default vote case * Fix logs * Remove unused functions * Meter default eth1 votes * Fix comments * Address review comments; remove unused dependency * Add first attempt at attestation proc. re-write * Add version 2 of attestation processing * Minor fixes * Add validator pubkey cache * Make get_indexed_attestation take a committee * Link signature processing into new attn verification * First working version * Ensure pubkey cache is updated * Add more metrics, slight optimizations * Clone committee cache during attestation processing * Update shuffling cache during block processing * Remove old commented-out code * Fix shuffling cache insert bug * Used indexed attestation in fork choice * Restructure attn processing, add metrics * Add more detailed metrics * Tidy, fix failing tests * Fix failing tests, tidy * Disable/delete two outdated tests * Tidy * Add pubkey cache persistence file * Add more comments * Integrate persistence file into builder * Add pubkey cache tests * Add data_dir to beacon chain builder * Remove Option in pubkey cache persistence file * Ensure consistency between datadir/data_dir * Fix failing network test * Tidy * Fix todos * Add attestation processing tests * Add another test * Only run attestation tests in release * Make attestation tests MainnetEthSpec * Address Michael's comments * Remove redundant check * Fix warning * Fix failing test Co-authored-by: Michael Sproul <micsproul@gmail.com> Co-authored-by: Pawan Dhananjay <pawandhananjay@gmail.com>
2020-03-05 06:19:35 +00:00
state_root: Hash256,
Add proto_array fork choice (#804) * Start implementing proto_array * Add progress * Add unfinished progress * Add further progress * Add progress * Add tree filtering * Add half-finished modifications * Add refactored version * Tidy, add incomplete LmdGhost impl * Move impls in LmdGhost trait def * Remove old reduced_tree fork choice * Combine two functions in to `compute_deltas` * Start testing * Add more compute_deltas tests * Add fork choice testing * Add more fork choice testing * Add more fork choice tests * Add more testing to proto-array * Remove old tests * Modify tests * Add more tests * Add more testing * Add comments and fixes * Re-organise crate * Tidy, finish pruning tests * Add ssz encoding, other pub fns * Rename lmd_ghost > proto_array_fork_choice * Integrate proto_array into lighthouse * Add first pass at fixing filter * Clean out old comments * Add more comments * Attempt to fix prune error * Adjust TODO * Fix test compile errors * Add extra justification change check * Update cargo.lock * Fix fork choice test compile errors * Most remove ffg_update_required * Fix bug with epoch of attestation votes * Start adding new test format * Make fork choice tests declarative * Create test def concept * Move test defs into crate * Add binary, re-org crate * Shuffle files * Start adding ffg tests * Add more fork choice tests * Add fork choice JSON dumping * Add more detail to best node error * Ensure fin+just checkpoints from from same block * Rename JustificationManager * Move checkpoint manager into own file * Tidy * Add targetted logging for sneaky sync bug * Fix justified balances bug * Add cache metrics * Add metrics for log levels * Fix bug in checkpoint manager * Fix compile error in fork choice tests * Ignore duplicate blocks in fork choice * Add block to fock choice before db * Rename on_new_block fn * Fix spec inconsistency in `CheckpointManager` * Remove BlockRootTree * Remove old reduced_tree code fragment * Add API endpoint for fork choice * Add more ffg tests * Remove block_root_tree reminents * Ensure effective balances are used * Remove old debugging code, fix API fault * Add check to ensure parent block is in fork choice * Update readme dates * Fix readme * Tidy checkpoint manager * Remove fork choice yaml files from repo * Remove fork choice yaml from repo * General tidy * Address majority of Michael's comments * Tidy bin/lib business * Remove dangling file * Undo changes for rpc/handler from master * Revert "Undo changes for rpc/handler from master" This reverts commit 876edff0e4a501aafbb47113454852826dcc24e8. Co-authored-by: Age Manning <Age@AgeManning.com>
2020-01-29 04:05:00 +00:00
justified_epoch: Epoch,
finalized_epoch: Epoch,
) -> Result<(), Error> {
// If the block is already known, simply ignore it.
if self.indices.contains_key(&root) {
return Ok(());
}
let node_index = self.nodes.len();
let node = ProtoNode {
slot,
Optimize attestation processing (#841) * Start updating types * WIP * Signature hacking * Existing EF tests passing with fake_crypto * Updates * Delete outdated API spec * The refactor continues * It compiles * WIP test fixes * All release tests passing bar genesis state parsing * Update and test YamlConfig * Update to spec v0.10 compatible BLS * Updates to BLS EF tests * Add EF test for AggregateVerify And delete unused hash2curve tests for uncompressed points * Update EF tests to v0.10.1 * Use optional block root correctly in block proc * Use genesis fork in deposit domain. All tests pass * Cargo fmt * Fast aggregate verify test * Update REST API docs * Cargo fmt * Fix unused import * Bump spec tags to v0.10.1 * Add `seconds_per_eth1_block` to chainspec * Update to timestamp based eth1 voting scheme * Return None from `get_votes_to_consider` if block cache is empty * Handle overflows in `is_candidate_block` * Revert to failing tests * Fix eth1 data sets test * Choose default vote according to spec * Fix collect_valid_votes tests * Fix `get_votes_to_consider` to choose all eligible blocks * Uncomment winning_vote tests * Add comments; remove unused code * Reduce seconds_per_eth1_block for simulation * Addressed review comments * Add test for default vote case * Fix logs * Remove unused functions * Meter default eth1 votes * Fix comments * Address review comments; remove unused dependency * Add first attempt at attestation proc. re-write * Add version 2 of attestation processing * Minor fixes * Add validator pubkey cache * Make get_indexed_attestation take a committee * Link signature processing into new attn verification * First working version * Ensure pubkey cache is updated * Add more metrics, slight optimizations * Clone committee cache during attestation processing * Update shuffling cache during block processing * Remove old commented-out code * Fix shuffling cache insert bug * Used indexed attestation in fork choice * Restructure attn processing, add metrics * Add more detailed metrics * Tidy, fix failing tests * Fix failing tests, tidy * Disable/delete two outdated tests * Tidy * Add pubkey cache persistence file * Add more comments * Integrate persistence file into builder * Add pubkey cache tests * Add data_dir to beacon chain builder * Remove Option in pubkey cache persistence file * Ensure consistency between datadir/data_dir * Fix failing network test * Tidy * Fix todos * Add attestation processing tests * Add another test * Only run attestation tests in release * Make attestation tests MainnetEthSpec * Address Michael's comments * Remove redundant check * Fix warning * Fix failing test Co-authored-by: Michael Sproul <micsproul@gmail.com> Co-authored-by: Pawan Dhananjay <pawandhananjay@gmail.com>
2020-03-05 06:19:35 +00:00
state_root,
Add proto_array fork choice (#804) * Start implementing proto_array * Add progress * Add unfinished progress * Add further progress * Add progress * Add tree filtering * Add half-finished modifications * Add refactored version * Tidy, add incomplete LmdGhost impl * Move impls in LmdGhost trait def * Remove old reduced_tree fork choice * Combine two functions in to `compute_deltas` * Start testing * Add more compute_deltas tests * Add fork choice testing * Add more fork choice testing * Add more fork choice tests * Add more testing to proto-array * Remove old tests * Modify tests * Add more tests * Add more testing * Add comments and fixes * Re-organise crate * Tidy, finish pruning tests * Add ssz encoding, other pub fns * Rename lmd_ghost > proto_array_fork_choice * Integrate proto_array into lighthouse * Add first pass at fixing filter * Clean out old comments * Add more comments * Attempt to fix prune error * Adjust TODO * Fix test compile errors * Add extra justification change check * Update cargo.lock * Fix fork choice test compile errors * Most remove ffg_update_required * Fix bug with epoch of attestation votes * Start adding new test format * Make fork choice tests declarative * Create test def concept * Move test defs into crate * Add binary, re-org crate * Shuffle files * Start adding ffg tests * Add more fork choice tests * Add fork choice JSON dumping * Add more detail to best node error * Ensure fin+just checkpoints from from same block * Rename JustificationManager * Move checkpoint manager into own file * Tidy * Add targetted logging for sneaky sync bug * Fix justified balances bug * Add cache metrics * Add metrics for log levels * Fix bug in checkpoint manager * Fix compile error in fork choice tests * Ignore duplicate blocks in fork choice * Add block to fock choice before db * Rename on_new_block fn * Fix spec inconsistency in `CheckpointManager` * Remove BlockRootTree * Remove old reduced_tree code fragment * Add API endpoint for fork choice * Add more ffg tests * Remove block_root_tree reminents * Ensure effective balances are used * Remove old debugging code, fix API fault * Add check to ensure parent block is in fork choice * Update readme dates * Fix readme * Tidy checkpoint manager * Remove fork choice yaml files from repo * Remove fork choice yaml from repo * General tidy * Address majority of Michael's comments * Tidy bin/lib business * Remove dangling file * Undo changes for rpc/handler from master * Revert "Undo changes for rpc/handler from master" This reverts commit 876edff0e4a501aafbb47113454852826dcc24e8. Co-authored-by: Age Manning <Age@AgeManning.com>
2020-01-29 04:05:00 +00:00
root,
parent: parent_opt.and_then(|parent| self.indices.get(&parent).copied()),
justified_epoch,
finalized_epoch,
weight: 0,
best_child: None,
best_descendant: None,
};
self.indices.insert(node.root, node_index);
self.nodes.push(node.clone());
if let Some(parent_index) = node.parent {
self.maybe_update_best_child_and_descendant(parent_index, node_index)?;
}
Ok(())
}
/// Follows the best-descendant links to find the best-block (i.e., head-block).
///
/// ## Notes
///
/// The result of this function is not guaranteed to be accurate if `Self::on_new_block` has
/// been called without a subsequent `Self::apply_score_changes` call. This is because
/// `on_new_block` does not attempt to walk backwards through the tree and update the
/// best-child/best-descendant links.
pub fn find_head(&self, justified_root: &Hash256) -> Result<Hash256, Error> {
let justified_index = self
.indices
.get(justified_root)
.copied()
.ok_or_else(|| Error::JustifiedNodeUnknown(*justified_root))?;
let justified_node = self
.nodes
.get(justified_index)
.ok_or_else(|| Error::InvalidJustifiedIndex(justified_index))?;
let best_descendant_index = justified_node
.best_descendant
.unwrap_or_else(|| justified_index);
let best_node = self
.nodes
.get(best_descendant_index)
.ok_or_else(|| Error::InvalidBestDescendant(best_descendant_index))?;
// Perform a sanity check that the node is indeed valid to be the head.
if !self.node_is_viable_for_head(&best_node) {
return Err(Error::InvalidBestNode {
start_root: *justified_root,
justified_epoch: self.justified_epoch,
finalized_epoch: self.finalized_epoch,
head_root: justified_node.root,
head_justified_epoch: justified_node.justified_epoch,
head_finalized_epoch: justified_node.finalized_epoch,
});
}
Ok(best_node.root)
}
/// Update the tree with new finalization information. The tree is only actually pruned if both
/// of the two following criteria are met:
///
/// - The supplied finalized epoch and root are different to the current values.
/// - The number of nodes in `self` is at least `self.prune_threshold`.
///
/// # Errors
///
/// Returns errors if:
///
/// - The finalized epoch is less than the current one.
/// - The finalized epoch is equal to the current one, but the finalized root is different.
/// - There is some internal error relating to invalid indices inside `self`.
pub fn maybe_prune(&mut self, finalized_root: Hash256) -> Result<(), Error> {
let finalized_index = *self
.indices
.get(&finalized_root)
.ok_or_else(|| Error::FinalizedNodeUnknown(finalized_root))?;
if finalized_index < self.prune_threshold {
// Pruning at small numbers incurs more cost than benefit.
return Ok(());
}
// Remove the `self.indices` key/values for all the to-be-deleted nodes.
for node_index in 0..finalized_index {
let root = &self
.nodes
.get(node_index)
.ok_or_else(|| Error::InvalidNodeIndex(node_index))?
.root;
self.indices.remove(root);
}
// Drop all the nodes prior to finalization.
self.nodes = self.nodes.split_off(finalized_index);
// Adjust the indices map.
for (_root, index) in self.indices.iter_mut() {
*index = index
.checked_sub(finalized_index)
.ok_or_else(|| Error::IndexOverflow("indices"))?;
}
// Iterate through all the existing nodes and adjust their indices to match the new layout
// of `self.nodes`.
for node in self.nodes.iter_mut() {
if let Some(parent) = node.parent {
// If `node.parent` is less than `finalized_index`, set it to `None`.
node.parent = parent.checked_sub(finalized_index);
}
if let Some(best_child) = node.best_child {
node.best_child = Some(
best_child
.checked_sub(finalized_index)
.ok_or_else(|| Error::IndexOverflow("best_child"))?,
);
}
if let Some(best_descendant) = node.best_descendant {
node.best_descendant = Some(
best_descendant
.checked_sub(finalized_index)
.ok_or_else(|| Error::IndexOverflow("best_descendant"))?,
);
}
}
Ok(())
}
/// Observe the parent at `parent_index` with respect to the child at `child_index` and
/// potentially modify the `parent.best_child` and `parent.best_descendant` values.
///
/// ## Detail
///
/// There are four outcomes:
///
/// - The child is already the best child but it's now invalid due to a FFG change and should be removed.
/// - The child is already the best child and the parent is updated with the new
/// best-descendant.
/// - The child is not the best child but becomes the best child.
/// - The child is not the best child and does not become the best child.
fn maybe_update_best_child_and_descendant(
&mut self,
parent_index: usize,
child_index: usize,
) -> Result<(), Error> {
let child = self
.nodes
.get(child_index)
.ok_or_else(|| Error::InvalidNodeIndex(child_index))?;
let parent = self
.nodes
.get(parent_index)
.ok_or_else(|| Error::InvalidNodeIndex(parent_index))?;
let child_leads_to_viable_head = self.node_leads_to_viable_head(&child)?;
// These three variables are aliases to the three options that we may set the
// `parent.best_child` and `parent.best_descendant` to.
//
// I use the aliases to assist readability.
let change_to_none = (None, None);
let change_to_child = (
Some(child_index),
child.best_descendant.or(Some(child_index)),
);
let no_change = (parent.best_child, parent.best_descendant);
let (new_best_child, new_best_descendant) =
if let Some(best_child_index) = parent.best_child {
if best_child_index == child_index && !child_leads_to_viable_head {
// If the child is already the best-child of the parent but it's not viable for
// the head, remove it.
change_to_none
} else if best_child_index == child_index {
// If the child is the best-child already, set it again to ensure that the
// best-descendant of the parent is updated.
change_to_child
} else {
let best_child = self
.nodes
.get(best_child_index)
.ok_or_else(|| Error::InvalidBestDescendant(best_child_index))?;
let best_child_leads_to_viable_head =
self.node_leads_to_viable_head(&best_child)?;
if child_leads_to_viable_head && !best_child_leads_to_viable_head {
// The child leads to a viable head, but the current best-child doesn't.
change_to_child
} else if !child_leads_to_viable_head && best_child_leads_to_viable_head {
// The best child leads to a viable head, but the child doesn't.
no_change
} else if child.weight == best_child.weight {
// Tie-breaker of equal weights by root.
if child.root >= best_child.root {
change_to_child
} else {
no_change
}
} else {
// Choose the winner by weight.
if child.weight >= best_child.weight {
change_to_child
} else {
no_change
}
}
}
} else {
if child_leads_to_viable_head {
// There is no current best-child and the child is viable.
change_to_child
} else {
// There is no current best-child but the child is not viable.
no_change
}
};
let parent = self
.nodes
.get_mut(parent_index)
.ok_or_else(|| Error::InvalidNodeIndex(parent_index))?;
parent.best_child = new_best_child;
parent.best_descendant = new_best_descendant;
Ok(())
}
/// Indicates if the node itself is viable for the head, or if it's best descendant is viable
/// for the head.
fn node_leads_to_viable_head(&self, node: &ProtoNode) -> Result<bool, Error> {
let best_descendant_is_viable_for_head =
if let Some(best_descendant_index) = node.best_descendant {
let best_descendant = self
.nodes
.get(best_descendant_index)
.ok_or_else(|| Error::InvalidBestDescendant(best_descendant_index))?;
self.node_is_viable_for_head(best_descendant)
} else {
false
};
Ok(best_descendant_is_viable_for_head || self.node_is_viable_for_head(node))
}
/// This is the equivalent to the `filter_block_tree` function in the eth2 spec:
///
/// https://github.com/ethereum/eth2.0-specs/blob/v0.10.0/specs/phase0/fork-choice.md#filter_block_tree
///
/// Any node that has a different finalized or justified epoch should not be viable for the
/// head.
fn node_is_viable_for_head(&self, node: &ProtoNode) -> bool {
(node.justified_epoch == self.justified_epoch || self.justified_epoch == Epoch::new(0))
&& (node.finalized_epoch == self.finalized_epoch
|| self.finalized_epoch == Epoch::new(0))
}
}