Use the forwards iterator more often (#2376)

## Issue Addressed

NA

## Primary Change

When investigating memory usage, I noticed that retrieving a block from an early slot (e.g., slot 900) would cause a sharp increase in the memory footprint (from 400mb to 800mb+) which seemed to be ever-lasting.

After some investigation, I found that the reverse iteration from the head back to that slot was the likely culprit. To counter this, I've switched the `BeaconChain::block_root_at_slot` to use the forwards iterator, instead of the reverse one.

I also noticed that the networking stack is using `BeaconChain::root_at_slot` to check if a peer is relevant (`check_peer_relevance`). Perhaps the steep, seemingly-random-but-consistent increases in memory usage are caused by the use of this function.

Using the forwards iterator with the HTTP API alleviated the sharp increases in memory usage. It also made the response much faster (before it felt like to took 1-2s, now it feels instant).

## Additional Changes

In the process I also noticed that we have two functions for getting block roots:

- `BeaconChain::block_root_at_slot`: returns `None` for a skip slot.
- `BeaconChain::root_at_slot`: returns the previous root for a skip slot.

I unified these two functions into `block_root_at_slot` and added the `WhenSlotSkipped` enum. Now, the caller must be explicit about the skip-slot behaviour when requesting a root. 

Additionally, I replaced `vec![]` with `Vec::with_capacity` in `store::chunked_vector::range_query`. I stumbled across this whilst debugging and made this modification to see what effect it would have (not much). It seems like a decent change to keep around, but I'm not concerned either way.

Also, `BeaconChain::get_ancestor_block_root` is unused, so I got rid of it 🗑️.

## Additional Info

I haven't also done the same for state roots here. Whilst it's possible and a good idea, it's more work since the fwds iterators are presently block-roots-specific.

Whilst there's a few places a reverse iteration of state roots could be triggered (e.g., attestation production, HTTP API), they're no where near as common as the `check_peer_relevance` call. As such, I think we should get this PR merged first, then come back for the state root iters. I made an issue here https://github.com/sigp/lighthouse/issues/2377.
This commit is contained in:
Paul Hauner 2021-05-31 04:18:20 +00:00
parent 320a683e72
commit 4c7bb4984c
12 changed files with 339 additions and 65 deletions

View File

@ -36,6 +36,7 @@ use eth2::types::{EventKind, SseBlock, SseFinalizedCheckpoint, SseHead};
use fork_choice::ForkChoice;
use futures::channel::mpsc::Sender;
use itertools::process_results;
use itertools::Itertools;
use operation_pool::{OperationPool, PersistedOperationPool};
use parking_lot::{Mutex, RwLock};
use slasher::Slasher;
@ -85,6 +86,18 @@ pub const OP_POOL_DB_KEY: Hash256 = Hash256::zero();
pub const ETH1_CACHE_DB_KEY: Hash256 = Hash256::zero();
pub const FORK_CHOICE_DB_KEY: Hash256 = Hash256::zero();
/// Defines the behaviour when a block/block-root for a skipped slot is requested.
pub enum WhenSlotSkipped {
/// If the slot is a skip slot, return `None`.
///
/// This is how the HTTP API behaves.
None,
/// If the slot it a skip slot, return the previous non-skipped block.
///
/// This is generally how the specification behaves.
Prev,
}
/// The result of a chain segment processing.
pub enum ChainSegmentResult<T: EthSpec> {
/// Processing this chain segment finished successfully.
@ -442,18 +455,6 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
.map(|result| result.map_err(|e| e.into())))
}
/// Traverse backwards from `block_root` to find the root of the ancestor block at `slot`.
pub fn get_ancestor_block_root(
&self,
block_root: Hash256,
slot: Slot,
) -> Result<Option<Hash256>, Error> {
process_results(self.rev_iter_block_roots_from(block_root)?, |mut iter| {
iter.find(|(_, ancestor_slot)| *ancestor_slot == slot)
.map(|(ancestor_block_root, _)| ancestor_block_root)
})
}
/// Iterates across all `(state_root, slot)` pairs from the head of the chain (inclusive) to
/// the earliest reachable ancestor (may or may not be genesis).
///
@ -489,17 +490,17 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
/// Returns the block at the given slot, if any. Only returns blocks in the canonical chain.
///
/// Use the `skips` parameter to define the behaviour when `request_slot` is a skipped slot.
///
/// ## Errors
///
/// May return a database error.
pub fn block_at_slot(
&self,
slot: Slot,
request_slot: Slot,
skips: WhenSlotSkipped,
) -> Result<Option<SignedBeaconBlock<T::EthSpec>>, Error> {
let root = process_results(self.rev_iter_block_roots()?, |mut iter| {
iter.find(|(_, this_slot)| *this_slot == slot)
.map(|(root, _)| root)
})?;
let root = self.block_root_at_slot(request_slot, skips)?;
if let Some(block_root) = root {
Ok(self.store.get_item(&block_root)?)
@ -521,21 +522,132 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
}
/// Returns the block root at the given slot, if any. Only returns roots in the canonical chain.
/// Returns `Ok(None)` if the given `Slot` was skipped.
///
/// ## Notes
///
/// - Use the `skips` parameter to define the behaviour when `request_slot` is a skipped slot.
/// - Returns `Ok(None)` for any slot higher than the current wall-clock slot.
pub fn block_root_at_slot(
&self,
request_slot: Slot,
skips: WhenSlotSkipped,
) -> Result<Option<Hash256>, Error> {
match skips {
WhenSlotSkipped::None => self.block_root_at_slot_skips_none(request_slot),
WhenSlotSkipped::Prev => self.block_root_at_slot_skips_prev(request_slot),
}
}
/// Returns the block root at the given slot, if any. Only returns roots in the canonical chain.
///
/// ## Notes
///
/// - Returns `Ok(None)` if the given `Slot` was skipped.
/// - Returns `Ok(None)` for any slot higher than the current wall-clock slot.
///
/// ## Errors
///
/// May return a database error.
pub fn block_root_at_slot(&self, slot: Slot) -> Result<Option<Hash256>, Error> {
process_results(self.rev_iter_block_roots()?, |mut iter| {
let root_opt = iter
.find(|(_, this_slot)| *this_slot == slot)
.map(|(root, _)| root);
if let (Some(root), Some((prev_root, _))) = (root_opt, iter.next()) {
return (prev_root != root).then(|| root);
fn block_root_at_slot_skips_none(&self, request_slot: Slot) -> Result<Option<Hash256>, Error> {
if request_slot > self.slot()? {
return Ok(None);
} else if request_slot == self.spec.genesis_slot {
return Ok(Some(self.genesis_block_root));
}
let prev_slot = request_slot.saturating_sub(1_u64);
// Try an optimized path of reading the root directly from the head state.
let fast_lookup: Option<Option<Hash256>> = self.with_head(|head| {
let state = &head.beacon_state;
// Try find the root for the `request_slot`.
let request_root_opt = match state.slot.cmp(&request_slot) {
// It's always a skip slot if the head is less than the request slot, return early.
Ordering::Less => return Ok(Some(None)),
// The request slot is the head slot.
Ordering::Equal => Some(head.beacon_block_root),
// Try find the request slot in the state.
Ordering::Greater => state.get_block_root(request_slot).ok().copied(),
};
if let Some(request_root) = request_root_opt {
if let Ok(prev_root) = state.get_block_root(prev_slot) {
return Ok(Some((*prev_root != request_root).then(|| request_root)));
}
}
root_opt
})
// Fast lookup is not possible.
Ok::<_, Error>(None)
})?;
if let Some(root_opt) = fast_lookup {
return Ok(root_opt);
}
if let Some(((prev_root, _), (curr_root, curr_slot))) =
process_results(self.forwards_iter_block_roots(prev_slot)?, |iter| {
iter.tuple_windows().next()
})?
{
// Sanity check.
if curr_slot != request_slot {
return Err(Error::InconsistentForwardsIter {
request_slot,
slot: curr_slot,
});
}
Ok((curr_root != prev_root).then(|| curr_root))
} else {
Ok(None)
}
}
/// Returns the block root at the given slot, if any. Only returns roots in the canonical chain.
///
/// ## Notes
///
/// - Returns the root at the previous non-skipped slot if the given `Slot` was skipped.
/// - Returns `Ok(None)` for any slot higher than the current wall-clock slot.
///
/// ## Errors
///
/// May return a database error.
fn block_root_at_slot_skips_prev(&self, request_slot: Slot) -> Result<Option<Hash256>, Error> {
if request_slot > self.slot()? {
return Ok(None);
} else if request_slot == self.spec.genesis_slot {
return Ok(Some(self.genesis_block_root));
}
// Try an optimized path of reading the root directly from the head state.
let fast_lookup: Option<Hash256> = self.with_head(|head| {
if head.beacon_block.slot() <= request_slot {
// Return the head root if all slots between the request and the head are skipped.
Ok(Some(head.beacon_block_root))
} else if let Ok(root) = head.beacon_state.get_block_root(request_slot) {
// Return the root if it's easily accessible from the head state.
Ok(Some(*root))
} else {
// Fast lookup is not possible.
Ok::<_, Error>(None)
}
})?;
if let Some(root) = fast_lookup {
return Ok(Some(root));
}
process_results(self.forwards_iter_block_roots(request_slot)?, |mut iter| {
if let Some((root, slot)) = iter.next() {
if slot == request_slot {
Ok(Some(root))
} else {
// Sanity check.
Err(Error::InconsistentForwardsIter { request_slot, slot })
}
} else {
Ok(None)
}
})?
}
/// Returns the block at the given root, if any.
@ -825,16 +937,6 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
Ok(map)
}
/// Returns the block canonical root of the current canonical chain at a given slot.
///
/// Returns `None` if the given slot doesn't exist in the chain.
pub fn root_at_slot(&self, target_slot: Slot) -> Result<Option<Hash256>, Error> {
process_results(self.rev_iter_block_roots()?, |mut iter| {
iter.find(|(_, slot)| *slot == target_slot)
.map(|(root, _)| root)
})
}
/// Returns the block canonical root of the current canonical chain at a given slot, starting from the given state.
///
/// Returns `None` if the given slot doesn't exist in the chain.
@ -2324,10 +2426,10 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
if let Some(event_handler) = self.event_handler.as_ref() {
if event_handler.has_head_subscribers() {
if let Ok(Some(current_duty_dependent_root)) =
self.root_at_slot(target_epoch_start_slot - 1)
self.block_root_at_slot(target_epoch_start_slot - 1, WhenSlotSkipped::Prev)
{
if let Ok(Some(previous_duty_dependent_root)) =
self.root_at_slot(prev_target_epoch_start_slot - 1)
if let Ok(Some(previous_duty_dependent_root)) = self
.block_root_at_slot(prev_target_epoch_start_slot - 1, WhenSlotSkipped::Prev)
{
event_handler.register(EventKind::Head(SseHead {
slot: head_slot,

View File

@ -113,6 +113,10 @@ pub enum BeaconChainError {
state_epoch: Epoch,
shuffling_epoch: Epoch,
},
InconsistentForwardsIter {
request_slot: Slot,
slot: Slot,
},
}
easy_from_to!(SlotProcessingError, BeaconChainError);

View File

@ -31,7 +31,7 @@ mod validator_pubkey_cache;
pub use self::beacon_chain::{
AttestationProcessingOutcome, BeaconChain, BeaconChainTypes, BeaconStore, ChainSegmentResult,
ForkChoiceError, StateSkipConfig, MAXIMUM_GOSSIP_CLOCK_DISPARITY,
ForkChoiceError, StateSkipConfig, WhenSlotSkipped, MAXIMUM_GOSSIP_CLOCK_DISPARITY,
};
pub use self::beacon_snapshot::BeaconSnapshot;
pub use self::chain_config::ChainConfig;

View File

@ -5,7 +5,7 @@ extern crate lazy_static;
use beacon_chain::{
test_utils::{AttestationStrategy, BeaconChainHarness, BlockStrategy},
StateSkipConfig,
StateSkipConfig, WhenSlotSkipped,
};
use store::config::StoreConfig;
use tree_hash::TreeHash;
@ -60,7 +60,7 @@ fn produces_attestations() {
};
let block = chain
.block_at_slot(block_slot)
.block_at_slot(block_slot, WhenSlotSkipped::Prev)
.expect("should get block")
.expect("block should not be skipped");
let block_root = block.message.tree_hash_root();

View File

@ -6,7 +6,7 @@ extern crate lazy_static;
use beacon_chain::{
attestation_verification::Error as AttnError,
test_utils::{AttestationStrategy, BeaconChainHarness, BlockStrategy, EphemeralHarnessType},
BeaconChain, BeaconChainTypes,
BeaconChain, BeaconChainTypes, WhenSlotSkipped,
};
use int_to_bytes::int_to_bytes32;
use state_processing::{
@ -912,7 +912,7 @@ fn attestation_that_skips_epochs() {
let earlier_slot = (current_epoch - 2).start_slot(MainnetEthSpec::slots_per_epoch());
let earlier_block = harness
.chain
.block_at_slot(earlier_slot)
.block_at_slot(earlier_slot, WhenSlotSkipped::Prev)
.expect("should not error getting block at slot")
.expect("should find block at slot");

View File

@ -9,6 +9,7 @@ use beacon_chain::{
AttestationStrategy, BeaconChainHarness, BlockStrategy, EphemeralHarnessType,
OP_POOL_DB_KEY,
},
WhenSlotSkipped,
};
use operation_pool::PersistedOperationPool;
use state_processing::{
@ -609,3 +610,147 @@ fn produces_and_processes_with_genesis_skip_slots() {
run_skip_slot_test(i)
}
}
#[test]
fn block_roots_skip_slot_behaviour() {
let harness = get_harness(VALIDATOR_COUNT);
// Test should be longer than the block roots to ensure a DB lookup is triggered.
let chain_length = harness.chain.head().unwrap().beacon_state.block_roots.len() as u64 * 3;
let skipped_slots = [1, 6, 7, 10, chain_length];
// Build a chain with some skip slots.
for i in 1..=chain_length {
if i > 1 {
harness.advance_slot();
}
let slot = harness.chain.slot().unwrap().as_u64();
if !skipped_slots.contains(&slot) {
harness.extend_chain(
1,
BlockStrategy::OnCanonicalHead,
AttestationStrategy::AllValidators,
);
}
}
let mut prev_unskipped_root = None;
for target_slot in 0..=chain_length {
if skipped_slots.contains(&target_slot) {
/*
* A skip slot
*/
assert!(
harness
.chain
.block_root_at_slot(target_slot.into(), WhenSlotSkipped::None)
.unwrap()
.is_none(),
"WhenSlotSkipped::None should return None on a skip slot"
);
let skipped_root = harness
.chain
.block_root_at_slot(target_slot.into(), WhenSlotSkipped::Prev)
.unwrap()
.expect("WhenSlotSkipped::Prev should always return Some");
assert_eq!(
skipped_root,
prev_unskipped_root.expect("test is badly formed"),
"WhenSlotSkipped::Prev should accurately return the prior skipped block"
);
let expected_block = harness.chain.get_block(&skipped_root).unwrap().unwrap();
assert_eq!(
harness
.chain
.block_at_slot(target_slot.into(), WhenSlotSkipped::Prev)
.unwrap()
.unwrap(),
expected_block,
);
assert!(
harness
.chain
.block_at_slot(target_slot.into(), WhenSlotSkipped::None)
.unwrap()
.is_none(),
"WhenSlotSkipped::None should return None on a skip slot"
);
} else {
/*
* Not a skip slot
*/
let skips_none = harness
.chain
.block_root_at_slot(target_slot.into(), WhenSlotSkipped::None)
.unwrap()
.expect("WhenSlotSkipped::None should return Some for non-skipped block");
let skips_prev = harness
.chain
.block_root_at_slot(target_slot.into(), WhenSlotSkipped::Prev)
.unwrap()
.expect("WhenSlotSkipped::Prev should always return Some");
assert_eq!(
skips_none, skips_prev,
"WhenSlotSkipped::None and WhenSlotSkipped::Prev should be equal on non-skipped slot"
);
let expected_block = harness.chain.get_block(&skips_prev).unwrap().unwrap();
assert_eq!(
harness
.chain
.block_at_slot(target_slot.into(), WhenSlotSkipped::Prev)
.unwrap()
.unwrap(),
expected_block
);
assert_eq!(
harness
.chain
.block_at_slot(target_slot.into(), WhenSlotSkipped::None)
.unwrap()
.unwrap(),
expected_block
);
prev_unskipped_root = Some(skips_prev);
}
}
/*
* A future, non-existent slot.
*/
let future_slot = harness.chain.slot().unwrap() + 1;
assert_eq!(
harness.chain.head().unwrap().beacon_block.slot(),
future_slot - 2,
"test precondition"
);
assert!(
harness
.chain
.block_root_at_slot(future_slot, WhenSlotSkipped::None)
.unwrap()
.is_none(),
"WhenSlotSkipped::None should return None on a future slot"
);
assert!(
harness
.chain
.block_root_at_slot(future_slot, WhenSlotSkipped::Prev)
.unwrap()
.is_none(),
"WhenSlotSkipped::Prev should return None on a future slot"
);
}

View File

@ -1,4 +1,4 @@
use beacon_chain::{BeaconChain, BeaconChainTypes};
use beacon_chain::{BeaconChain, BeaconChainTypes, WhenSlotSkipped};
use eth2::types::BlockId as CoreBlockId;
use std::str::FromStr;
use types::{Hash256, SignedBeaconBlock, Slot};
@ -37,7 +37,7 @@ impl BlockId {
.map(|head| head.current_justified_checkpoint.root)
.map_err(warp_utils::reject::beacon_chain_error),
CoreBlockId::Slot(slot) => chain
.block_root_at_slot(*slot)
.block_root_at_slot(*slot, WhenSlotSkipped::None)
.map_err(warp_utils::reject::beacon_chain_error)
.and_then(|root_opt| {
root_opt.ok_or_else(|| {

View File

@ -17,6 +17,7 @@ use beacon_chain::{
observed_operations::ObservationOutcome,
validator_monitor::{get_block_delay_ms, timestamp_now},
AttestationError as AttnError, BeaconChain, BeaconChainError, BeaconChainTypes,
WhenSlotSkipped,
};
use block_id::BlockId;
use eth2::types::{self as api_types, ValidatorId};
@ -751,7 +752,7 @@ pub fn serve<T: BeaconChainTypes>(
let block = BlockId::from_root(root).block(&chain)?;
let canonical = chain
.block_root_at_slot(block.slot())
.block_root_at_slot(block.slot(), WhenSlotSkipped::None)
.map_err(warp_utils::reject::beacon_chain_error)?
.map_or(false, |canonical| root == canonical);

View File

@ -2,7 +2,7 @@
use beacon_chain::{
test_utils::{AttestationStrategy, BeaconChainHarness, BlockStrategy, EphemeralHarnessType},
BeaconChain, StateSkipConfig, MAXIMUM_GOSSIP_CLOCK_DISPARITY,
BeaconChain, StateSkipConfig, WhenSlotSkipped, MAXIMUM_GOSSIP_CLOCK_DISPARITY,
};
use environment::null_logger;
use eth2::Error;
@ -791,7 +791,10 @@ impl ApiTester {
.current_justified_checkpoint
.root,
),
BlockId::Slot(slot) => self.chain.block_root_at_slot(slot).unwrap(),
BlockId::Slot(slot) => self
.chain
.block_root_at_slot(slot, WhenSlotSkipped::None)
.unwrap(),
BlockId::Root(root) => Some(root),
}
}
@ -812,14 +815,21 @@ impl ApiTester {
.unwrap()
.map(|res| res.data);
let root = self.chain.block_root_at_slot(slot).unwrap();
let root = self
.chain
.block_root_at_slot(slot, WhenSlotSkipped::None)
.unwrap();
if root.is_none() && result.is_none() {
continue;
}
let root = root.unwrap();
let block = self.chain.block_at_slot(slot).unwrap().unwrap();
let block = self
.chain
.block_at_slot(slot, WhenSlotSkipped::Prev)
.unwrap()
.unwrap();
let header = BlockHeaderData {
root,
canonical: true,
@ -900,7 +910,7 @@ impl ApiTester {
let block_root = block_root_opt.unwrap();
let canonical = self
.chain
.block_root_at_slot(block.slot())
.block_root_at_slot(block.slot(), WhenSlotSkipped::None)
.unwrap()
.map_or(false, |canonical| block_root == canonical);
@ -1532,7 +1542,10 @@ impl ApiTester {
let dependent_root = self
.chain
.root_at_slot((epoch - 1).start_slot(E::slots_per_epoch()) - 1)
.block_root_at_slot(
(epoch - 1).start_slot(E::slots_per_epoch()) - 1,
WhenSlotSkipped::Prev,
)
.unwrap()
.unwrap_or(self.chain.head_beacon_block_root().unwrap());
@ -1604,7 +1617,10 @@ impl ApiTester {
let dependent_root = self
.chain
.root_at_slot(epoch.start_slot(E::slots_per_epoch()) - 1)
.block_root_at_slot(
epoch.start_slot(E::slots_per_epoch()) - 1,
WhenSlotSkipped::Prev,
)
.unwrap()
.unwrap_or(self.chain.head_beacon_block_root().unwrap());
@ -2186,7 +2202,7 @@ impl ApiTester {
current_duty_dependent_root,
previous_duty_dependent_root: self
.chain
.root_at_slot(current_slot - E::slots_per_epoch())
.block_root_at_slot(current_slot - E::slots_per_epoch(), WhenSlotSkipped::Prev)
.unwrap()
.unwrap(),
epoch_transition: true,
@ -2195,7 +2211,7 @@ impl ApiTester {
let expected_finalized = EventKind::FinalizedCheckpoint(SseFinalizedCheckpoint {
block: self
.chain
.root_at_slot(next_slot - finalization_distance)
.block_root_at_slot(next_slot - finalization_distance, WhenSlotSkipped::Prev)
.unwrap()
.unwrap(),
state: self

View File

@ -2,7 +2,7 @@ use crate::beacon_processor::worker::FUTURE_SLOT_TOLERANCE;
use crate::service::NetworkMessage;
use crate::status::ToStatusMessage;
use crate::sync::SyncMessage;
use beacon_chain::{BeaconChainError, BeaconChainTypes};
use beacon_chain::{BeaconChainError, BeaconChainTypes, WhenSlotSkipped};
use eth2_libp2p::rpc::StatusMessage;
use eth2_libp2p::rpc::*;
use eth2_libp2p::{PeerId, PeerRequestId, ReportSource, Response, SyncInfo};
@ -72,7 +72,7 @@ impl<T: BeaconChainTypes> Worker<T> {
&& local.finalized_root != Hash256::zero()
&& self
.chain
.root_at_slot(start_slot(remote.finalized_epoch))
.block_root_at_slot(start_slot(remote.finalized_epoch), WhenSlotSkipped::Prev)
.map(|root_opt| root_opt != Some(remote.finalized_root))?
{
// The remote's finalized epoch is less than or equal to ours, but the block root is

View File

@ -431,9 +431,15 @@ fn range_query<S: KeyValueStore<E>, E: EthSpec, T: Decode + Encode>(
start_index: usize,
end_index: usize,
) -> Result<Vec<Chunk<T>>, Error> {
let mut result = vec![];
let range = start_index..=end_index;
let len = range
.end()
// Add one to account for inclusive range.
.saturating_add(1)
.saturating_sub(*range.start());
let mut result = Vec::with_capacity(len);
for chunk_index in start_index..=end_index {
for chunk_index in range {
let key = &chunk_key(chunk_index as u64)[..];
let chunk = Chunk::load(store, column, key)?.ok_or(ChunkError::Missing { chunk_index })?;
result.push(chunk);

View File

@ -3,7 +3,7 @@
use beacon_chain::{
test_utils::{AttestationStrategy, BeaconChainHarness, BlockStrategy, EphemeralHarnessType},
BeaconChain, BeaconChainError, BeaconForkChoiceStore, ChainConfig, ForkChoiceError,
StateSkipConfig,
StateSkipConfig, WhenSlotSkipped,
};
use fork_choice::{
ForkChoiceStore, InvalidAttestation, InvalidBlock, QueuedAttestation,
@ -872,7 +872,7 @@ fn invalid_attestation_future_block() {
MutationDelay::Blocks(1),
|attestation, chain| {
attestation.data.beacon_block_root = chain
.block_at_slot(chain.slot().unwrap())
.block_at_slot(chain.slot().unwrap(), WhenSlotSkipped::Prev)
.unwrap()
.unwrap()
.canonical_root();
@ -901,7 +901,7 @@ fn invalid_attestation_inconsistent_ffg_vote() {
MutationDelay::NoDelay,
|attestation, chain| {
attestation.data.target.root = chain
.block_at_slot(Slot::new(1))
.block_at_slot(Slot::new(1), WhenSlotSkipped::Prev)
.unwrap()
.unwrap()
.canonical_root();
@ -909,7 +909,7 @@ fn invalid_attestation_inconsistent_ffg_vote() {
*attestation_opt.lock().unwrap() = Some(attestation.data.target.root);
*local_opt.lock().unwrap() = Some(
chain
.block_at_slot(Slot::new(0))
.block_at_slot(Slot::new(0), WhenSlotSkipped::Prev)
.unwrap()
.unwrap()
.canonical_root(),