Add timeouts to canonical head rwlock (#759)

* Add TimeoutRwLock to BeaconChain

* Update network crate

* Update rest api

* Fix beacon chain tests

* Fix rest api tests

* Set test back to !debug_assertions
This commit is contained in:
Paul Hauner 2020-01-06 17:30:37 +11:00 committed by GitHub
parent b0c8b2b700
commit f04c55075e
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
21 changed files with 391 additions and 156 deletions

View File

@ -7,9 +7,9 @@ use crate::fork_choice::{Error as ForkChoiceError, ForkChoice};
use crate::head_tracker::HeadTracker;
use crate::metrics;
use crate::persisted_beacon_chain::{PersistedBeaconChain, BEACON_CHAIN_DB_KEY};
use crate::timeout_rw_lock::TimeoutRwLock;
use lmd_ghost::LmdGhost;
use operation_pool::{OperationPool, PersistedOperationPool};
use parking_lot::RwLock;
use slog::{debug, error, info, trace, warn, Logger};
use slot_clock::SlotClock;
use ssz::Encode;
@ -49,6 +49,10 @@ const WRITE_BLOCK_PROCESSING_SSZ: bool = cfg!(feature = "write_ssz_files");
/// Maximum block slot number. Block with slots bigger than this constant will NOT be processed.
const MAXIMUM_BLOCK_SLOT_NUMBER: u64 = 4_294_967_296; // 2^32
/// The time-out before failure during an operation to take a read/write RwLock on the canonical
/// head.
const HEAD_LOCK_TIMEOUT: Duration = Duration::from_secs(1);
#[derive(Debug, PartialEq)]
pub enum BlockProcessingOutcome {
/// Block was valid and imported into the block graph.
@ -103,6 +107,7 @@ pub struct HeadInfo {
pub block_root: Hash256,
pub state_root: Hash256,
pub finalized_checkpoint: types::Checkpoint,
pub fork: Fork,
}
pub trait BeaconChainTypes: Send + Sync + 'static {
@ -131,7 +136,7 @@ pub struct BeaconChain<T: BeaconChainTypes> {
/// Provides information from the Ethereum 1 (PoW) chain.
pub eth1_chain: Option<Eth1Chain<T::Eth1Chain, T::EthSpec>>,
/// Stores a "snapshot" of the chain at the time the head-of-the-chain block was received.
pub(crate) canonical_head: RwLock<CheckPoint<T::EthSpec>>,
pub(crate) canonical_head: TimeoutRwLock<CheckPoint<T::EthSpec>>,
/// The root of the genesis block.
pub genesis_block_root: Hash256,
/// A state-machine that is updated with information from the network and chooses a canonical
@ -154,7 +159,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
pub fn persist(&self) -> Result<(), Error> {
let timer = metrics::start_timer(&metrics::PERSIST_CHAIN);
let canonical_head = self.head();
let canonical_head = self.head()?;
let finalized_checkpoint = {
let beacon_block_root = canonical_head.beacon_state.finalized_checkpoint.root;
@ -256,26 +261,32 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
/// - Iterator returns `(Hash256, Slot)`.
/// - As this iterator starts at the `head` of the chain (viz., the best block), the first slot
/// returned may be earlier than the wall-clock slot.
pub fn rev_iter_block_roots(&self) -> ReverseBlockRootIterator<T::EthSpec, T::Store> {
let head = self.head();
pub fn rev_iter_block_roots(
&self,
) -> Result<ReverseBlockRootIterator<T::EthSpec, T::Store>, Error> {
let head = self.head()?;
let iter = BlockRootsIterator::owned(self.store.clone(), head.beacon_state);
ReverseBlockRootIterator::new((head.beacon_block_root, head.beacon_block.slot), iter)
Ok(ReverseBlockRootIterator::new(
(head.beacon_block_root, head.beacon_block.slot),
iter,
))
}
pub fn forwards_iter_block_roots(
&self,
start_slot: Slot,
) -> <T::Store as Store<T::EthSpec>>::ForwardsBlockRootsIterator {
let local_head = self.head();
T::Store::forwards_block_roots_iterator(
) -> Result<<T::Store as Store<T::EthSpec>>::ForwardsBlockRootsIterator, Error> {
let local_head = self.head()?;
Ok(T::Store::forwards_block_roots_iterator(
self.store.clone(),
start_slot,
local_head.beacon_state,
local_head.beacon_block_root,
&self.spec,
)
))
}
/// Traverse backwards from `block_root` to find the block roots of its ancestors.
@ -325,13 +336,18 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
/// - Iterator returns `(Hash256, Slot)`.
/// - As this iterator starts at the `head` of the chain (viz., the best block), the first slot
/// returned may be earlier than the wall-clock slot.
pub fn rev_iter_state_roots(&self) -> ReverseStateRootIterator<T::EthSpec, T::Store> {
let head = self.head();
pub fn rev_iter_state_roots(
&self,
) -> Result<ReverseStateRootIterator<T::EthSpec, T::Store>, Error> {
let head = self.head()?;
let slot = head.beacon_state.slot;
let iter = StateRootsIterator::owned(self.store.clone(), head.beacon_state);
ReverseStateRootIterator::new((head.beacon_state_root, slot), iter)
Ok(ReverseStateRootIterator::new(
(head.beacon_state_root, slot),
iter,
))
}
/// Returns the block at the given root, if any.
@ -353,7 +369,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
/// May return a database error.
pub fn block_at_slot(&self, slot: Slot) -> Result<Option<BeaconBlock<T::EthSpec>>, Error> {
let root = self
.rev_iter_block_roots()
.rev_iter_block_roots()?
.find(|(_, this_slot)| *this_slot == slot)
.map(|(root, _)| root);
@ -452,22 +468,29 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
/// It is important to note that the `beacon_state` returned may not match the present slot. It
/// is the state as it was when the head block was received, which could be some slots prior to
/// now.
pub fn head(&self) -> CheckPoint<T::EthSpec> {
self.canonical_head.read().clone()
pub fn head(&self) -> Result<CheckPoint<T::EthSpec>, Error> {
self.canonical_head
.try_read_for(HEAD_LOCK_TIMEOUT)
.ok_or_else(|| Error::CanonicalHeadLockTimeout)
.map(|v| v.clone())
}
/// Returns info representing the head block and state.
///
/// A summarized version of `Self::head` that involves less cloning.
pub fn head_info(&self) -> HeadInfo {
let head = self.canonical_head.read();
pub fn head_info(&self) -> Result<HeadInfo, Error> {
let head = self
.canonical_head
.try_read_for(HEAD_LOCK_TIMEOUT)
.ok_or_else(|| Error::CanonicalHeadLockTimeout)?;
HeadInfo {
Ok(HeadInfo {
slot: head.beacon_block.slot,
block_root: head.beacon_block_root,
state_root: head.beacon_state_root,
finalized_checkpoint: head.beacon_state.finalized_checkpoint.clone(),
}
fork: head.beacon_state.fork.clone(),
})
}
/// Returns the current heads of the `BeaconChain`. For the canonical head, see `Self::head`.
@ -482,7 +505,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
/// Returns `None` when the state is not found in the database or there is an error skipping
/// to a future state.
pub fn state_at_slot(&self, slot: Slot) -> Result<BeaconState<T::EthSpec>, Error> {
let head_state = self.head().beacon_state;
let head_state = self.head()?.beacon_state;
if slot == head_state.slot {
Ok(head_state)
@ -534,7 +557,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
Ok(state)
} else {
let state_root = self
.rev_iter_state_roots()
.rev_iter_state_roots()?
.take_while(|(_root, current_slot)| *current_slot >= slot)
.find(|(_root, current_slot)| *current_slot == slot)
.map(|(root, _slot)| root)
@ -559,29 +582,33 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
}
/// Returns the slot of the highest block in the canonical chain.
pub fn best_slot(&self) -> Slot {
self.canonical_head.read().beacon_block.slot
pub fn best_slot(&self) -> Result<Slot, Error> {
self.canonical_head
.try_read_for(HEAD_LOCK_TIMEOUT)
.map(|head| head.beacon_block.slot)
.ok_or_else(|| Error::CanonicalHeadLockTimeout)
}
/// Returns the validator index (if any) for the given public key.
///
/// Information is retrieved from the present `beacon_state.validators`.
pub fn validator_index(&self, pubkey: &PublicKeyBytes) -> Option<usize> {
for (i, validator) in self.head().beacon_state.validators.iter().enumerate() {
pub fn validator_index(&self, pubkey: &PublicKeyBytes) -> Result<Option<usize>, Error> {
for (i, validator) in self.head()?.beacon_state.validators.iter().enumerate() {
if validator.pubkey == *pubkey {
return Some(i);
return Ok(Some(i));
}
}
None
Ok(None)
}
/// Returns the block canonical root of the current canonical chain at a given slot.
///
/// Returns None if a block doesn't exist at the slot.
pub fn root_at_slot(&self, target_slot: Slot) -> Option<Hash256> {
self.rev_iter_block_roots()
pub fn root_at_slot(&self, target_slot: Slot) -> Result<Option<Hash256>, Error> {
Ok(self
.rev_iter_block_roots()?
.find(|(_root, slot)| *slot == target_slot)
.map(|(root, _slot)| root)
.map(|(root, _slot)| root))
}
/// Reads the slot clock (see `self.read_slot_clock()` and returns the number of slots since
@ -603,10 +630,10 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
/// present epoch is available.
pub fn block_proposer(&self, slot: Slot) -> Result<usize, Error> {
let epoch = |slot: Slot| slot.epoch(T::EthSpec::slots_per_epoch());
let head_state = &self.head().beacon_state;
let head_state = &self.head()?.beacon_state;
let mut state = if epoch(slot) == epoch(head_state.slot) {
self.head().beacon_state.clone()
self.head()?.beacon_state.clone()
} else {
self.state_at_slot(slot)?
};
@ -636,10 +663,10 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
epoch: Epoch,
) -> Result<Option<(Slot, u64)>, Error> {
let as_epoch = |slot: Slot| slot.epoch(T::EthSpec::slots_per_epoch());
let head_state = &self.head().beacon_state;
let head_state = &self.head()?.beacon_state;
let mut state = if epoch == as_epoch(head_state.slot) {
self.head().beacon_state.clone()
self.head()?.beacon_state.clone()
} else {
self.state_at_slot(epoch.start_slot(T::EthSpec::slots_per_epoch()))?
};
@ -672,7 +699,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
index: CommitteeIndex,
) -> Result<Attestation<T::EthSpec>, Error> {
let state = self.state_at_slot(slot)?;
let head = self.head();
let head = self.head()?;
let data = self.produce_attestation_data_for_block(
index,
@ -699,7 +726,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
index: CommitteeIndex,
) -> Result<AttestationData, Error> {
let state = self.state_at_slot(slot)?;
let head = self.head();
let head = self.head()?;
self.produce_attestation_data_for_block(
index,
@ -915,7 +942,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
//
// This is likely overly restrictive, we could store the attestation for later
// processing.
let head_epoch = self.head_info().slot.epoch(T::EthSpec::slots_per_epoch());
let head_epoch = self.head_info()?.slot.epoch(T::EthSpec::slots_per_epoch());
let attestation_epoch = attestation.data.slot.epoch(T::EthSpec::slots_per_epoch());
// Only log a warning if our head is in a reasonable place to verify this attestation.
@ -977,7 +1004,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
// - The highest valid finalized epoch we've ever seen (i.e., the head).
// - The finalized epoch that this attestation was created against.
let finalized_epoch = std::cmp::max(
self.head_info().finalized_checkpoint.epoch,
self.head_info()?.finalized_checkpoint.epoch,
state.finalized_checkpoint.epoch,
);
@ -1175,7 +1202,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
let full_timer = metrics::start_timer(&metrics::BLOCK_PROCESSING_TIMES);
let finalized_slot = self
.head_info()
.head_info()?
.finalized_checkpoint
.epoch
.start_slot(T::EthSpec::slots_per_epoch());
@ -1518,7 +1545,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
let beacon_block_root = self.fork_choice.find_head(&self)?;
// If a new head was chosen.
let result = if beacon_block_root != self.head_info().block_root {
let result = if beacon_block_root != self.head_info()?.block_root {
metrics::inc_counter(&metrics::FORK_CHOICE_CHANGED_HEAD);
let beacon_block: BeaconBlock<T::EthSpec> = self
@ -1530,15 +1557,15 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
.get_state_caching(&beacon_state_root, Some(beacon_block.slot))?
.ok_or_else(|| Error::MissingBeaconState(beacon_state_root))?;
let previous_slot = self.head_info().slot;
let previous_slot = self.head_info()?.slot;
let new_slot = beacon_block.slot;
// Note: this will declare a re-org if we skip `SLOTS_PER_HISTORICAL_ROOT` blocks
// between calls to fork choice without swapping between chains. This seems like an
// extreme-enough scenario that a warning is fine.
let is_reorg = self.head_info().block_root
let is_reorg = self.head_info()?.block_root
!= beacon_state
.get_block_root(self.head_info().slot)
.get_block_root(self.head_info()?.slot)
.map(|root| *root)
.unwrap_or_else(|_| Hash256::random());
@ -1548,7 +1575,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
warn!(
self.log,
"Beacon chain re-org";
"previous_head" => format!("{}", self.head_info().block_root),
"previous_head" => format!("{}", self.head_info()?.block_root),
"previous_slot" => previous_slot,
"new_head_parent" => format!("{}", beacon_block.parent_root),
"new_head" => format!("{}", beacon_block_root),
@ -1567,7 +1594,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
);
};
let old_finalized_epoch = self.head_info().finalized_checkpoint.epoch;
let old_finalized_epoch = self.head_info()?.finalized_checkpoint.epoch;
let new_finalized_epoch = beacon_state.finalized_checkpoint.epoch;
let finalized_root = beacon_state.finalized_checkpoint.root;
@ -1578,7 +1605,11 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
new_epoch: new_finalized_epoch,
})
} else {
let previous_head_beacon_block_root = self.canonical_head.read().beacon_block_root;
let previous_head_beacon_block_root = self
.canonical_head
.try_read_for(HEAD_LOCK_TIMEOUT)
.ok_or_else(|| Error::CanonicalHeadLockTimeout)?
.beacon_block_root;
let current_head_beacon_block_root = beacon_block_root;
let mut new_head = CheckPoint {
@ -1599,7 +1630,10 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
// Update the checkpoint that stores the head of the chain at the time it received the
// block.
*self.canonical_head.write() = new_head;
*self
.canonical_head
.try_write_for(HEAD_LOCK_TIMEOUT)
.ok_or_else(|| Error::CanonicalHeadLockTimeout)? = new_head;
metrics::stop_timer(timer);
@ -1695,10 +1729,10 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
let mut dump = vec![];
let mut last_slot = CheckPoint {
beacon_block: self.head().beacon_block.clone(),
beacon_block_root: self.head().beacon_block_root,
beacon_state: self.head().beacon_state.clone(),
beacon_state_root: self.head().beacon_state_root,
beacon_block: self.head()?.beacon_block.clone(),
beacon_block_root: self.head()?.beacon_block_root,
beacon_state: self.head()?.beacon_state.clone(),
beacon_state_root: self.head()?.beacon_state_root,
};
dump.push(last_slot.clone());

View File

@ -3,6 +3,7 @@ use crate::eth1_chain::CachingEth1Backend;
use crate::events::NullEventHandler;
use crate::head_tracker::HeadTracker;
use crate::persisted_beacon_chain::{PersistedBeaconChain, BEACON_CHAIN_DB_KEY};
use crate::timeout_rw_lock::TimeoutRwLock;
use crate::{
BeaconChain, BeaconChainTypes, CheckPoint, Eth1Chain, Eth1ChainBackend, EventHandler,
ForkChoice,
@ -10,7 +11,6 @@ use crate::{
use eth1::Config as Eth1Config;
use lmd_ghost::{LmdGhost, ThreadSafeReducedTree};
use operation_pool::OperationPool;
use parking_lot::RwLock;
use slog::{info, Logger};
use slot_clock::{SlotClock, TestingSlotClock};
use std::marker::PhantomData;
@ -364,7 +364,7 @@ where
.op_pool
.ok_or_else(|| "Cannot build without op pool".to_string())?,
eth1_chain: self.eth1_chain,
canonical_head: RwLock::new(canonical_head),
canonical_head: TimeoutRwLock::new(canonical_head),
genesis_block_root: self
.genesis_block_root
.ok_or_else(|| "Cannot build without a genesis block root".to_string())?,
@ -379,12 +379,16 @@ where
log: log.clone(),
};
let head = beacon_chain
.head()
.map_err(|e| format!("Failed to get head: {:?}", e))?;
info!(
log,
"Beacon chain initialized";
"head_state" => format!("{}", beacon_chain.head().beacon_state_root),
"head_block" => format!("{}", beacon_chain.head().beacon_block_root),
"head_slot" => format!("{}", beacon_chain.head().beacon_block.slot),
"head_state" => format!("{}", head.beacon_state_root),
"head_block" => format!("{}", head.beacon_block_root),
"head_slot" => format!("{}", head.beacon_block.slot),
);
Ok(beacon_chain)
@ -629,7 +633,8 @@ mod test {
.build()
.expect("should build");
let head = chain.head();
let head = chain.head().expect("should get head");
let state = head.beacon_state;
let block = head.beacon_block;

View File

@ -48,6 +48,7 @@ pub enum BeaconChainError {
/// Returned when an internal check fails, indicating corrupt data.
InvariantViolated(String),
SszTypesError(SszTypesError),
CanonicalHeadLockTimeout,
}
easy_from_to!(SlotProcessingError, BeaconChainError);

View File

@ -14,6 +14,7 @@ mod head_tracker;
mod metrics;
mod persisted_beacon_chain;
pub mod test_utils;
mod timeout_rw_lock;
pub use self::beacon_chain::{
AttestationProcessingOutcome, BeaconChain, BeaconChainTypes, BlockProcessingOutcome,

View File

@ -199,10 +199,9 @@ lazy_static! {
/// Scrape the `beacon_chain` for metrics that are not constantly updated (e.g., the present slot,
/// head state info, etc) and update the Prometheus `DEFAULT_REGISTRY`.
pub fn scrape_for_metrics<T: BeaconChainTypes>(beacon_chain: &BeaconChain<T>) {
scrape_head_state::<T>(
&beacon_chain.head().beacon_state,
beacon_chain.head().beacon_state_root,
);
if let Ok(head) = beacon_chain.head() {
scrape_head_state::<T>(&head.beacon_state, head.beacon_state_root)
}
}
/// Scrape the given `state` assuming it's the head state, updating the `DEFAULT_REGISTRY`.

View File

@ -461,7 +461,12 @@ where
honest_fork_blocks: usize,
faulty_fork_blocks: usize,
) -> (Hash256, Hash256) {
let initial_head_slot = self.chain.head().beacon_block.slot;
let initial_head_slot = self
.chain
.head()
.expect("should get head")
.beacon_block
.slot;
// Move to the next slot so we may produce some more blocks on the head.
self.advance_slot();

View File

@ -0,0 +1,20 @@
use parking_lot::{RwLock, RwLockReadGuard, RwLockWriteGuard};
use std::time::Duration;
/// A simple wrapper around `parking_lot::RwLock` that only permits read/write access with a
/// time-out (i.e., no indefinitely-blocking operations).
pub struct TimeoutRwLock<T>(RwLock<T>);
impl<T> TimeoutRwLock<T> {
pub fn new(inner: T) -> Self {
Self(RwLock::new(inner))
}
pub fn try_read_for(&self, timeout: Duration) -> Option<RwLockReadGuard<T>> {
self.0.try_read_for(timeout)
}
pub fn try_write_for(&self, timeout: Duration) -> Option<RwLockWriteGuard<T>> {
self.0.try_write_for(timeout)
}
}

View File

@ -59,7 +59,14 @@ fn finalizes_after_resuming_from_db() {
);
assert!(
harness.chain.head().beacon_state.finalized_checkpoint.epoch > 0,
harness
.chain
.head()
.expect("should read head")
.beacon_state
.finalized_checkpoint
.epoch
> 0,
"the chain should have already finalized"
);
@ -95,7 +102,11 @@ fn finalizes_after_resuming_from_db() {
AttestationStrategy::AllValidators,
);
let state = &resumed_harness.chain.head().beacon_state;
let state = &resumed_harness
.chain
.head()
.expect("should read head")
.beacon_state;
assert_eq!(
state.slot, num_blocks_produced,
"head should be at the current slot"

View File

@ -96,7 +96,7 @@ fn randomised_skips() {
}
}
let state = &harness.chain.head().beacon_state;
let state = &harness.chain.head().expect("should get head").beacon_state;
assert_eq!(state.slot, num_slots, "head should be at the current slot");
@ -166,6 +166,7 @@ fn randao_genesis_storage() {
let genesis_value = *harness
.chain
.head()
.expect("should get head")
.beacon_state
.get_randao_mix(Epoch::new(0))
.expect("randao mix ok");
@ -181,6 +182,7 @@ fn randao_genesis_storage() {
assert!(harness
.chain
.head()
.expect("should get head")
.beacon_state
.randao_mixes
.iter()
@ -197,6 +199,7 @@ fn randao_genesis_storage() {
assert!(harness
.chain
.head()
.expect("should get head")
.beacon_state
.randao_mixes
.iter()
@ -238,7 +241,7 @@ fn split_slot_restore() {
/// Check that the head state's slot matches `expected_slot`.
fn check_slot(harness: &TestHarness, expected_slot: u64) {
let state = &harness.chain.head().beacon_state;
let state = &harness.chain.head().expect("should get head").beacon_state;
assert_eq!(
state.slot, expected_slot,
@ -248,7 +251,7 @@ fn check_slot(harness: &TestHarness, expected_slot: u64) {
/// Check that the chain has finalized under best-case assumptions, and check the head slot.
fn check_finalization(harness: &TestHarness, expected_slot: u64) {
let state = &harness.chain.head().beacon_state;
let state = &harness.chain.head().expect("should get head").beacon_state;
check_slot(harness, expected_slot);
@ -271,6 +274,7 @@ fn check_split_slot(harness: &TestHarness, store: Arc<DiskStore<E>>) {
harness
.chain
.head()
.expect("should get head")
.beacon_state
.finalized_checkpoint
.epoch
@ -314,7 +318,7 @@ fn check_chain_dump(harness: &TestHarness, expected_len: u64) {
.map(|checkpoint| (checkpoint.beacon_block_root, checkpoint.beacon_block.slot))
.collect::<Vec<_>>();
let head = harness.chain.head();
let head = harness.chain.head().expect("should get head");
let mut forward_block_roots = Store::forwards_block_roots_iterator(
harness.chain.store.clone(),
Slot::new(0),
@ -343,6 +347,7 @@ fn check_iterators(harness: &TestHarness) {
harness
.chain
.rev_iter_state_roots()
.expect("should get iter")
.last()
.map(|(_, slot)| slot),
Some(Slot::new(0))
@ -351,6 +356,7 @@ fn check_iterators(harness: &TestHarness) {
harness
.chain
.rev_iter_block_roots()
.expect("should get iter")
.last()
.map(|(_, slot)| slot),
Some(Slot::new(0))

View File

@ -37,7 +37,7 @@ fn get_harness(validator_count: usize) -> BeaconChainHarness<HarnessType<Minimal
fn massive_skips() {
let harness = get_harness(8);
let spec = &MinimalEthSpec::default_spec();
let mut state = harness.chain.head().beacon_state;
let mut state = harness.chain.head().expect("should get head").beacon_state;
// Run per_slot_processing until it returns an error.
let error = loop {
@ -70,8 +70,16 @@ fn iterators() {
AttestationStrategy::SomeValidators(vec![]),
);
let block_roots: Vec<(Hash256, Slot)> = harness.chain.rev_iter_block_roots().collect();
let state_roots: Vec<(Hash256, Slot)> = harness.chain.rev_iter_state_roots().collect();
let block_roots: Vec<(Hash256, Slot)> = harness
.chain
.rev_iter_block_roots()
.expect("should get iter")
.collect();
let state_roots: Vec<(Hash256, Slot)> = harness
.chain
.rev_iter_state_roots()
.expect("should get iter")
.collect();
assert_eq!(
block_roots.len(),
@ -109,7 +117,7 @@ fn iterators() {
)
});
let head = &harness.chain.head();
let head = &harness.chain.head().expect("should get head");
assert_eq!(
*block_roots.first().expect("should have some block roots"),
@ -154,7 +162,7 @@ fn chooses_fork() {
assert!(honest_head != faulty_head, "forks should be distinct");
let state = &harness.chain.head().beacon_state;
let state = &harness.chain.head().expect("should get head").beacon_state;
assert_eq!(
state.slot,
@ -163,7 +171,11 @@ fn chooses_fork() {
);
assert_eq!(
harness.chain.head().beacon_block_root,
harness
.chain
.head()
.expect("should get head")
.beacon_block_root,
honest_head,
"the honest chain should be the canonical chain"
);
@ -181,7 +193,7 @@ fn finalizes_with_full_participation() {
AttestationStrategy::AllValidators,
);
let state = &harness.chain.head().beacon_state;
let state = &harness.chain.head().expect("should get head").beacon_state;
assert_eq!(
state.slot, num_blocks_produced,
@ -219,7 +231,7 @@ fn finalizes_with_two_thirds_participation() {
AttestationStrategy::SomeValidators(attesters),
);
let state = &harness.chain.head().beacon_state;
let state = &harness.chain.head().expect("should get head").beacon_state;
assert_eq!(
state.slot, num_blocks_produced,
@ -263,7 +275,7 @@ fn does_not_finalize_with_less_than_two_thirds_participation() {
AttestationStrategy::SomeValidators(attesters),
);
let state = &harness.chain.head().beacon_state;
let state = &harness.chain.head().expect("should get head").beacon_state;
assert_eq!(
state.slot, num_blocks_produced,
@ -296,7 +308,7 @@ fn does_not_finalize_without_attestation() {
AttestationStrategy::SomeValidators(vec![]),
);
let state = &harness.chain.head().beacon_state;
let state = &harness.chain.head().expect("should get head").beacon_state;
assert_eq!(
state.slot, num_blocks_produced,
@ -357,7 +369,7 @@ fn free_attestations_added_to_fork_choice_some_none() {
AttestationStrategy::AllValidators,
);
let state = &harness.chain.head().beacon_state;
let state = &harness.chain.head().expect("should get head").beacon_state;
let fork_choice = &harness.chain.fork_choice;
let validator_slots: Vec<(usize, Slot)> = (0..VALIDATOR_COUNT)
@ -409,12 +421,23 @@ fn attestations_with_increasing_slots() {
AttestationStrategy::SomeValidators(vec![]),
);
attestations.append(&mut harness.get_free_attestations(
attestations.append(
&mut harness.get_free_attestations(
&AttestationStrategy::AllValidators,
&harness.chain.head().beacon_state,
harness.chain.head().beacon_block_root,
harness.chain.head().beacon_block.slot,
));
&harness.chain.head().expect("should get head").beacon_state,
harness
.chain
.head()
.expect("should get head")
.beacon_block_root,
harness
.chain
.head()
.expect("should get head")
.beacon_block
.slot,
),
);
harness.advance_slot();
}
@ -439,7 +462,7 @@ fn free_attestations_added_to_fork_choice_all_updated() {
AttestationStrategy::AllValidators,
);
let state = &harness.chain.head().beacon_state;
let state = &harness.chain.head().expect("should get head").beacon_state;
let fork_choice = &harness.chain.fork_choice;
let validators: Vec<usize> = (0..VALIDATOR_COUNT).collect();
@ -496,17 +519,39 @@ fn run_skip_slot_test(skip_slots: u64) {
);
assert_eq!(
harness_a.chain.head().beacon_block.slot,
harness_a
.chain
.head()
.expect("should get head")
.beacon_block
.slot,
Slot::new(skip_slots + 1)
);
assert_eq!(harness_b.chain.head().beacon_block.slot, Slot::new(0));
assert_eq!(
harness_b
.chain
.process_block(harness_a.chain.head().beacon_block.clone()),
.head()
.expect("should get head")
.beacon_block
.slot,
Slot::new(0)
);
assert_eq!(
harness_b.chain.process_block(
harness_a
.chain
.head()
.expect("should get head")
.beacon_block
.clone()
),
Ok(BlockProcessingOutcome::Processed {
block_root: harness_a.chain.head().beacon_block_root
block_root: harness_a
.chain
.head()
.expect("should get head")
.beacon_block_root
})
);
@ -516,7 +561,12 @@ fn run_skip_slot_test(skip_slots: u64) {
.expect("should run fork choice");
assert_eq!(
harness_b.chain.head().beacon_block.slot,
harness_b
.chain
.head()
.expect("should get head")
.beacon_block
.slot,
Slot::new(skip_slots + 1)
);
}

View File

@ -69,7 +69,12 @@ pub fn spawn_notifier<T: BeaconChainTypes>(
usize::max_value()
};
let head = beacon_chain.head();
let head = beacon_chain.head()
.map_err(|e| error!(
log,
"Failed to get beacon chain head";
"error" => format!("{:?}", e)
))?;
let head_slot = head.beacon_block.slot;
let head_epoch = head_slot.epoch(T::EthSpec::slots_per_epoch());

View File

@ -45,9 +45,9 @@ impl From<StatusMessage> for PeerSyncInfo {
}
}
impl<T: BeaconChainTypes> From<&Arc<BeaconChain<T>>> for PeerSyncInfo {
fn from(chain: &Arc<BeaconChain<T>>) -> PeerSyncInfo {
Self::from(status_message(chain))
impl PeerSyncInfo {
pub fn from_chain<T: BeaconChainTypes>(chain: &Arc<BeaconChain<T>>) -> Option<PeerSyncInfo> {
Some(Self::from(status_message(chain)?))
}
}
@ -119,8 +119,10 @@ impl<T: BeaconChainTypes> MessageProcessor<T> {
///
/// Sends a `Status` message to the peer.
pub fn on_connect(&mut self, peer_id: PeerId) {
if let Some(status_message) = status_message(&self.chain) {
self.network
.send_rpc_request(peer_id, RPCRequest::Status(status_message(&self.chain)));
.send_rpc_request(peer_id, RPCRequest::Status(status_message));
}
}
/// Handle a `Status` request.
@ -135,12 +137,14 @@ impl<T: BeaconChainTypes> MessageProcessor<T> {
// ignore status responses if we are shutting down
trace!(self.log, "StatusRequest"; "peer" => format!("{:?}", peer_id));
if let Some(status_message) = status_message(&self.chain) {
// Say status back.
self.network.send_rpc_response(
peer_id.clone(),
request_id,
RPCResponse::Status(status_message(&self.chain)),
RPCResponse::Status(status_message),
);
}
self.process_status(peer_id, status);
}
@ -158,7 +162,16 @@ impl<T: BeaconChainTypes> MessageProcessor<T> {
/// Disconnects the peer if required.
fn process_status(&mut self, peer_id: PeerId, status: StatusMessage) {
let remote = PeerSyncInfo::from(status);
let local = PeerSyncInfo::from(&self.chain);
let local = match PeerSyncInfo::from_chain(&self.chain) {
Some(local) => local,
None => {
return error!(
self.log,
"Failed to get peer sync info";
"msg" => "likely due to head lock contention"
)
}
};
let start_slot = |epoch: Epoch| epoch.start_slot(T::EthSpec::slots_per_epoch());
@ -191,8 +204,11 @@ impl<T: BeaconChainTypes> MessageProcessor<T> {
} else if remote.finalized_epoch <= local.finalized_epoch
&& remote.finalized_root != Hash256::zero()
&& local.finalized_root != Hash256::zero()
&& (self.chain.root_at_slot(start_slot(remote.finalized_epoch))
!= Some(remote.finalized_root))
&& self
.chain
.root_at_slot(start_slot(remote.finalized_epoch))
.map(|root_opt| root_opt != Some(remote.finalized_root))
.unwrap_or_else(|_| false)
{
// The remotes finalized epoch is less than or greater than ours, but the block root is
// different to the one in our chain.
@ -321,9 +337,21 @@ impl<T: BeaconChainTypes> MessageProcessor<T> {
return;
}
let mut block_roots = self
let forwards_block_root_iter = match self
.chain
.forwards_iter_block_roots(Slot::from(req.start_slot))
{
Ok(iter) => iter,
Err(e) => {
return error!(
self.log,
"Unable to obtain root iter";
"error" => format!("{:?}", e)
)
}
};
let mut block_roots = forwards_block_root_iter
.take_while(|(_root, slot)| slot.as_u64() < req.start_slot + req.count * req.step)
.step_by(req.step as usize)
.map(|(root, _slot)| root)
@ -552,16 +580,18 @@ impl<T: BeaconChainTypes> MessageProcessor<T> {
}
/// Build a `StatusMessage` representing the state of the given `beacon_chain`.
pub(crate) fn status_message<T: BeaconChainTypes>(beacon_chain: &BeaconChain<T>) -> StatusMessage {
let state = &beacon_chain.head().beacon_state;
pub(crate) fn status_message<T: BeaconChainTypes>(
beacon_chain: &BeaconChain<T>,
) -> Option<StatusMessage> {
let head_info = beacon_chain.head_info().ok()?;
StatusMessage {
fork_version: state.fork.current_version,
finalized_root: state.finalized_checkpoint.root,
finalized_epoch: state.finalized_checkpoint.epoch,
head_root: beacon_chain.head().beacon_block_root,
head_slot: state.slot,
}
Some(StatusMessage {
fork_version: head_info.fork.current_version,
finalized_root: head_info.finalized_checkpoint.root,
finalized_epoch: head_info.finalized_checkpoint.epoch,
head_root: head_info.block_root,
head_slot: head_info.slot,
})
}
/// Wraps a Network Channel to employ various RPC related network functionality for the message

View File

@ -229,7 +229,16 @@ impl<T: BeaconChainTypes> SyncManager<T> {
}
};
let local = PeerSyncInfo::from(&chain);
let local = match PeerSyncInfo::from_chain(&chain) {
Some(local) => local,
None => {
return error!(
self.log,
"Failed to get peer sync info";
"msg" => "likely due to head lock contention"
)
}
};
// If a peer is within SLOT_IMPORT_TOLERANCE from our head slot, ignore a batch/range sync,
// consider it a fully-sync'd peer.

View File

@ -43,7 +43,9 @@ impl SyncNetworkContext {
"peer" => format!("{:?}", peer_id)
);
if let Some(chain) = chain.upgrade() {
let _ = self.send_rpc_request(peer_id, RPCRequest::Status(status_message(&chain)));
if let Some(status_message) = status_message(&chain) {
let _ = self.send_rpc_request(peer_id, RPCRequest::Status(status_message));
}
}
}

View File

@ -8,7 +8,7 @@ use crate::message_processor::PeerSyncInfo;
use crate::sync::network_context::SyncNetworkContext;
use beacon_chain::{BeaconChain, BeaconChainTypes};
use eth2_libp2p::PeerId;
use slog::{debug, warn};
use slog::{debug, error, warn};
use std::sync::Weak;
use types::EthSpec;
use types::{Hash256, Slot};
@ -103,9 +103,22 @@ impl<T: BeaconChainTypes> ChainCollection<T> {
/// updates the state of the collection.
pub fn update_finalized(&mut self, network: &mut SyncNetworkContext, log: &slog::Logger) {
let local_slot = match self.beacon_chain.upgrade() {
Some(chain) => PeerSyncInfo::from(&chain)
Some(chain) => {
let local = match PeerSyncInfo::from_chain(&chain) {
Some(local) => local,
None => {
return error!(
log,
"Failed to get peer sync info";
"msg" => "likely due to head lock contention"
)
}
};
local
.finalized_epoch
.start_slot(T::EthSpec::slots_per_epoch()),
.start_slot(T::EthSpec::slots_per_epoch())
}
None => {
warn!(log, "Beacon chain dropped. Chains not updated");
return;
@ -113,7 +126,7 @@ impl<T: BeaconChainTypes> ChainCollection<T> {
};
// Remove any outdated finalized chains
self.purge_outdated_chains(network);
self.purge_outdated_chains(network, log);
// Check if any chains become the new syncing chain
if let Some(index) = self.finalized_syncing_index() {
@ -248,14 +261,23 @@ impl<T: BeaconChainTypes> ChainCollection<T> {
///
/// This removes chains with no peers, or chains whose start block slot is less than our current
/// finalized block slot.
pub fn purge_outdated_chains(&mut self, network: &mut SyncNetworkContext) {
pub fn purge_outdated_chains(&mut self, network: &mut SyncNetworkContext, log: &slog::Logger) {
// Remove any chains that have no peers
self.finalized_chains
.retain(|chain| !chain.peer_pool.is_empty());
self.head_chains.retain(|chain| !chain.peer_pool.is_empty());
let local_info = match self.beacon_chain.upgrade() {
Some(chain) => PeerSyncInfo::from(&chain),
Some(chain) => match PeerSyncInfo::from_chain(&chain) {
Some(local) => local,
None => {
return error!(
log,
"Failed to get peer sync info";
"msg" => "likely due to head lock contention"
)
}
},
None => {
return;
}

View File

@ -46,7 +46,7 @@ use crate::sync::network_context::SyncNetworkContext;
use beacon_chain::{BeaconChain, BeaconChainTypes};
use eth2_libp2p::rpc::RequestId;
use eth2_libp2p::PeerId;
use slog::{debug, trace, warn};
use slog::{debug, error, trace, warn};
use std::collections::HashSet;
use std::sync::Weak;
use types::{BeaconBlock, EthSpec};
@ -106,7 +106,16 @@ impl<T: BeaconChainTypes> RangeSync<T> {
// determine if we need to run a sync to the nearest finalized state or simply sync to
// its current head
let local_info = match self.beacon_chain.upgrade() {
Some(chain) => PeerSyncInfo::from(&chain),
Some(chain) => match PeerSyncInfo::from_chain(&chain) {
Some(local) => local,
None => {
return error!(
self.log,
"Failed to get peer sync info";
"msg" => "likely due to head lock contention"
)
}
},
None => {
warn!(self.log,
"Beacon chain dropped. Peer not considered for sync";
@ -127,7 +136,7 @@ impl<T: BeaconChainTypes> RangeSync<T> {
self.remove_peer(network, &peer_id);
// remove any out-of-date chains
self.chains.purge_outdated_chains(network);
self.chains.purge_outdated_chains(network, &self.log);
if remote_finalized_slot > local_info.head_slot {
debug!(self.log, "Finalization sync peer joined"; "peer_id" => format!("{:?}", peer_id));

View File

@ -33,7 +33,7 @@ pub fn get_head<T: BeaconChainTypes>(
req: Request<Body>,
beacon_chain: Arc<BeaconChain<T>>,
) -> ApiResult {
let chain_head = beacon_chain.head();
let chain_head = beacon_chain.head()?;
let head = CanonicalHeadResponse {
slot: chain_head.beacon_state.slot,
@ -106,7 +106,7 @@ pub fn get_block<T: BeaconChainTypes>(
("slot", value) => {
let target = parse_slot(&value)?;
block_root_at_slot(&beacon_chain, target).ok_or_else(|| {
block_root_at_slot(&beacon_chain, target)?.ok_or_else(|| {
ApiError::NotFound(format!("Unable to find BeaconBlock for slot {:?}", target))
})?
}
@ -140,7 +140,7 @@ pub fn get_block_root<T: BeaconChainTypes>(
let slot_string = UrlQuery::from_request(&req)?.only_one("slot")?;
let target = parse_slot(&slot_string)?;
let root = block_root_at_slot(&beacon_chain, target).ok_or_else(|| {
let root = block_root_at_slot(&beacon_chain, target)?.ok_or_else(|| {
ApiError::NotFound(format!("Unable to find BeaconBlock for slot {:?}", target))
})?;
@ -152,7 +152,7 @@ pub fn get_fork<T: BeaconChainTypes>(
req: Request<Body>,
beacon_chain: Arc<BeaconChain<T>>,
) -> ApiResult {
ResponseBuilder::new(&req)?.body(&beacon_chain.head().beacon_state.fork)
ResponseBuilder::new(&req)?.body(&beacon_chain.head()?.beacon_state.fork)
}
#[derive(Clone, Debug, PartialEq, Serialize, Deserialize, Encode, Decode)]
@ -302,7 +302,7 @@ fn get_state_from_root_opt<T: BeaconChainTypes>(
})?
.ok_or_else(|| ApiError::NotFound(format!("No state exists with root: {}", state_root)))
} else {
Ok(beacon_chain.head().beacon_state)
Ok(beacon_chain.head()?.beacon_state)
}
}
@ -417,7 +417,7 @@ pub fn get_state<T: BeaconChainTypes>(
req: Request<Body>,
beacon_chain: Arc<BeaconChain<T>>,
) -> ApiResult {
let head_state = beacon_chain.head().beacon_state;
let head_state = beacon_chain.head()?.beacon_state;
let (key, value) = match UrlQuery::from_request(&req) {
Ok(query) => {
@ -491,5 +491,5 @@ pub fn get_genesis_time<T: BeaconChainTypes>(
req: Request<Body>,
beacon_chain: Arc<BeaconChain<T>>,
) -> ApiResult {
ResponseBuilder::new(&req)?.body(&beacon_chain.head().beacon_state.genesis_time)
ResponseBuilder::new(&req)?.body(&beacon_chain.head()?.beacon_state.genesis_time)
}

View File

@ -60,6 +60,12 @@ impl From<types::BeaconStateError> for ApiError {
}
}
impl From<beacon_chain::BeaconChainError> for ApiError {
fn from(e: beacon_chain::BeaconChainError) -> ApiError {
ApiError::ServerError(format!("BeaconChainError error: {:?}", e))
}
}
impl From<state_processing::per_slot_processing::Error> for ApiError {
fn from(e: state_processing::per_slot_processing::Error) -> ApiError {
ApiError::ServerError(format!("PerSlotProcessing error: {:?}", e))

View File

@ -120,12 +120,12 @@ pub fn parse_pubkey_bytes(string: &str) -> Result<PublicKeyBytes, ApiError> {
pub fn block_root_at_slot<T: BeaconChainTypes>(
beacon_chain: &BeaconChain<T>,
target: Slot,
) -> Option<Hash256> {
beacon_chain
.rev_iter_block_roots()
) -> Result<Option<Hash256>, ApiError> {
Ok(beacon_chain
.rev_iter_block_roots()?
.take_while(|(_root, slot)| *slot >= target)
.find(|(_root, slot)| *slot == target)
.map(|(root, _slot)| root)
.map(|(root, _slot)| root))
}
/// Returns a `BeaconState` and it's root in the canonical chain of `beacon_chain` at the given
@ -137,15 +137,15 @@ pub fn state_at_slot<T: BeaconChainTypes>(
beacon_chain: &BeaconChain<T>,
slot: Slot,
) -> Result<(Hash256, BeaconState<T::EthSpec>), ApiError> {
let head_state = &beacon_chain.head().beacon_state;
let head_state = &beacon_chain.head()?.beacon_state;
if head_state.slot == slot {
// The request slot is the same as the best block (head) slot.
// I'm not sure if this `.clone()` will be optimized out. If not, it seems unnecessary.
Ok((
beacon_chain.head().beacon_state_root,
beacon_chain.head().beacon_state.clone(),
beacon_chain.head()?.beacon_state_root,
beacon_chain.head()?.beacon_state.clone(),
))
} else {
let root = state_root_at_slot(beacon_chain, slot)?;
@ -168,7 +168,7 @@ pub fn state_root_at_slot<T: BeaconChainTypes>(
beacon_chain: &BeaconChain<T>,
slot: Slot,
) -> Result<Hash256, ApiError> {
let head_state = &beacon_chain.head().beacon_state;
let head_state = &beacon_chain.head()?.beacon_state;
let current_slot = beacon_chain
.slot()
.map_err(|_| ApiError::ServerError("Unable to read slot clock".to_string()))?;
@ -192,7 +192,7 @@ pub fn state_root_at_slot<T: BeaconChainTypes>(
// 2. The request slot is the same as the best block (head) slot.
//
// The head state root is stored in memory, return a reference.
Ok(beacon_chain.head().beacon_state_root)
Ok(beacon_chain.head()?.beacon_state_root)
} else if head_state.slot > slot {
// 3. The request slot is prior to the head slot.
//
@ -209,7 +209,7 @@ pub fn state_root_at_slot<T: BeaconChainTypes>(
//
// Use `per_slot_processing` to advance the head state to the present slot,
// assuming that all slots do not contain a block (i.e., they are skipped slots).
let mut state = beacon_chain.head().beacon_state.clone();
let mut state = beacon_chain.head()?.beacon_state.clone();
let spec = &T::EthSpec::default_spec();
for _ in state.slot.as_u64()..slot.as_u64() {

View File

@ -122,10 +122,10 @@ pub fn get_state_for_epoch<T: BeaconChainTypes>(
epoch: Epoch,
) -> Result<BeaconState<T::EthSpec>, ApiError> {
let slots_per_epoch = T::EthSpec::slots_per_epoch();
let head_epoch = beacon_chain.head().beacon_state.current_epoch();
let head_epoch = beacon_chain.head()?.beacon_state.current_epoch();
if RelativeEpoch::from_epoch(head_epoch, epoch).is_ok() {
Ok(beacon_chain.head().beacon_state)
Ok(beacon_chain.head()?.beacon_state)
} else {
let slot = if epoch > head_epoch {
// Move to the first slot of the epoch prior to the request.
@ -308,7 +308,7 @@ pub fn publish_beacon_block<T: BeaconChainTypes>(
// - Excessive time between block produce and publish.
// - A validator is using another beacon node to produce blocks and
// submitting them here.
if beacon_chain.head().beacon_block_root != block_root {
if beacon_chain.head()?.beacon_block_root != block_root {
warn!(
log,
"Block from validator is not head";

View File

@ -43,7 +43,12 @@ fn get_randao_reveal<T: BeaconChainTypes>(
slot: Slot,
spec: &ChainSpec,
) -> Signature {
let fork = beacon_chain.head().beacon_state.fork.clone();
let fork = beacon_chain
.head()
.expect("should get head")
.beacon_state
.fork
.clone();
let proposer_index = beacon_chain
.block_proposer(slot)
.expect("should get proposer index");
@ -60,7 +65,12 @@ fn sign_block<T: BeaconChainTypes>(
block: &mut BeaconBlock<T::EthSpec>,
spec: &ChainSpec,
) {
let fork = beacon_chain.head().beacon_state.fork.clone();
let fork = beacon_chain
.head()
.expect("should get head")
.beacon_state
.fork
.clone();
let proposer_index = beacon_chain
.block_proposer(block.slot)
.expect("should get proposer index");
@ -81,7 +91,11 @@ fn validator_produce_attestation() {
.client
.beacon_chain()
.expect("client should have beacon chain");
let state = beacon_chain.head().beacon_state.clone();
let state = beacon_chain
.head()
.expect("should get head")
.beacon_state
.clone();
let validator_index = 0;
let duties = state
@ -182,6 +196,7 @@ fn validator_duties() {
let validators = beacon_chain
.head()
.expect("should get head")
.beacon_state
.validators
.iter()
@ -534,6 +549,7 @@ fn genesis_time() {
.beacon_chain()
.expect("should have beacon chain")
.head()
.expect("should get head")
.beacon_state
.genesis_time,
genesis_time,
@ -558,6 +574,7 @@ fn fork() {
.beacon_chain()
.expect("should have beacon chain")
.head()
.expect("should get head")
.beacon_state
.fork,
fork,
@ -623,6 +640,7 @@ fn get_genesis_state_root() {
.beacon_chain()
.expect("should have beacon chain")
.rev_iter_state_roots()
.expect("should get iter")
.find(|(_cur_root, cur_slot)| slot == *cur_slot)
.map(|(cur_root, _)| cur_root)
.expect("chain should have state root at slot");
@ -649,6 +667,7 @@ fn get_genesis_block_root() {
.beacon_chain()
.expect("should have beacon chain")
.rev_iter_block_roots()
.expect("should get iter")
.find(|(_cur_root, cur_slot)| slot == *cur_slot)
.map(|(cur_root, _)| cur_root)
.expect("chain should have state root at slot");
@ -666,7 +685,7 @@ fn get_validators() {
.client
.beacon_chain()
.expect("node should have beacon chain");
let state = &chain.head().beacon_state;
let state = &chain.head().expect("should get head").beacon_state;
let validators = state.validators.iter().take(2).collect::<Vec<_>>();
let pubkeys = validators
@ -695,7 +714,7 @@ fn get_all_validators() {
.client
.beacon_chain()
.expect("node should have beacon chain");
let state = &chain.head().beacon_state;
let state = &chain.head().expect("should get head").beacon_state;
let result = env
.runtime()
@ -718,7 +737,7 @@ fn get_active_validators() {
.client
.beacon_chain()
.expect("node should have beacon chain");
let state = &chain.head().beacon_state;
let state = &chain.head().expect("should get head").beacon_state;
let result = env
.runtime()
@ -764,6 +783,7 @@ fn get_committees() {
let expected = chain
.head()
.expect("should get head")
.beacon_state
.get_beacon_committees_at_epoch(RelativeEpoch::Current)
.expect("should get committees")