Optimize validator duties (#2243)

## Issue Addressed

Closes #2052

## Proposed Changes

- Refactor the attester/proposer duties endpoints in the BN
    - Performance improvements
    - Fixes some potential inconsistencies with the dependent root fields.
    - Removes `http_api::beacon_proposer_cache` and just uses the one on the `BeaconChain` instead.
    - Move the code for the proposer/attester duties endpoints into separate files, for readability.
- Refactor the `DutiesService` in the VC
    - Required to reduce the delay on broadcasting new blocks.
    - Gets rid of the `ValidatorDuty` shim struct that came about when we adopted the standard API.
    - Separate block/attestation duty tasks so that they don't block each other when one is slow.
- In the VC, use `PublicKeyBytes` to represent validators instead of `PublicKey`. `PublicKey` is a legit crypto object whilst `PublicKeyBytes` is just a byte-array, it's much faster to clone/hash `PublicKeyBytes` and this change has had a significant impact on runtimes.
    - Unfortunately this has created lots of dust changes.
 - In the BN, store `PublicKeyBytes` in the `beacon_proposer_cache` and allow access to them. The HTTP API always sends `PublicKeyBytes` over the wire and the conversion from `PublicKey` -> `PublickeyBytes` is non-trivial, especially when queries have 100s/1000s of validators (like Pyrmont).
 - Add the `state_processing::state_advance` mod which dedups a lot of the "apply `n` skip slots to the state" code.
    - This also fixes a bug with some functions which were failing to include a state root as per [this comment](072695284f/consensus/state_processing/src/state_advance.rs (L69-L74)). I couldn't find any instance of this bug that resulted in anything more severe than keying a shuffling cache by the wrong block root.
 - Swap the VC block service to use `mpsc` from `tokio` instead of `futures`. This is consistent with the rest of the code base.
    
~~This PR *reduces* the size of the codebase 🎉~~ It *used* to reduce the size of the code base before I added more comments. 

## Observations on Prymont

- Proposer duties times down from peaks of 450ms to consistent <1ms.
- Current epoch attester duties times down from >1s peaks to a consistent 20-30ms.
- Block production down from +600ms to 100-200ms.

## Additional Info

- ~~Blocked on #2241~~
- ~~Blocked on #2234~~

## TODO

- [x] ~~Refactor this into some smaller PRs?~~ Leaving this as-is for now.
- [x] Address `per_slot_processing` roots.
- [x] Investigate slow next epoch times. Not getting added to cache on block processing?
- [x] Consider [this](072695284f/beacon_node/store/src/hot_cold_store.rs (L811-L812)) in the scenario of replacing the state roots


Co-authored-by: pawan <pawandhananjay@gmail.com>
Co-authored-by: Michael Sproul <michael@sigmaprime.io>
This commit is contained in:
Paul Hauner 2021-03-17 05:09:57 +00:00
parent 6a69b20be1
commit 015ab7d0a7
49 changed files with 2201 additions and 1833 deletions

1
Cargo.lock generated
View File

@ -7073,6 +7073,7 @@ dependencies = [
"rand 0.7.3",
"rayon",
"ring",
"safe_arith",
"scrypt",
"serde",
"serde_derive",

View File

@ -229,7 +229,7 @@ pub fn cli_run<T: EthSpec>(
})?;
slashing_protection
.register_validator(&voting_pubkey)
.register_validator(voting_pubkey.compress())
.map_err(|e| {
format!(
"Error registering validator {}: {:?}",

View File

@ -236,7 +236,7 @@ pub fn cli_run(matches: &ArgMatches, validator_dir: PathBuf) -> Result<(), Strin
.public_key()
.ok_or_else(|| format!("Keystore public key is invalid: {}", keystore.pubkey()))?;
slashing_protection
.register_validator(&voting_pubkey)
.register_validator(voting_pubkey.compress())
.map_err(|e| {
format!(
"Error registering validator {}: {:?}",

View File

@ -1073,7 +1073,7 @@ where
}
chain
.with_committee_cache(target.root, attestation_epoch, |committee_cache| {
.with_committee_cache(target.root, attestation_epoch, |committee_cache, _| {
let committees_per_slot = committee_cache.committees_per_slot();
Ok(committee_cache

View File

@ -42,8 +42,11 @@ use slasher::Slasher;
use slog::{crit, debug, error, info, trace, warn, Logger};
use slot_clock::SlotClock;
use state_processing::{
common::get_indexed_attestation, per_block_processing,
per_block_processing::errors::AttestationValidationError, per_slot_processing,
common::get_indexed_attestation,
per_block_processing,
per_block_processing::errors::AttestationValidationError,
per_slot_processing,
state_advance::{complete_state_advance, partial_state_advance},
BlockSignatureStrategy, SigVerifiedOp,
};
use std::borrow::Cow;
@ -156,6 +159,7 @@ pub struct HeadInfo {
pub fork: Fork,
pub genesis_time: u64,
pub genesis_validators_root: Hash256,
pub proposer_shuffling_decision_root: Hash256,
}
pub trait BeaconChainTypes: Send + Sync + 'static {
@ -243,7 +247,7 @@ pub struct BeaconChain<T: BeaconChainTypes> {
/// Caches the attester shuffling for a given epoch and shuffling key root.
pub(crate) shuffling_cache: TimeoutRwLock<ShufflingCache>,
/// Caches the beacon block proposer shuffling for a given epoch and shuffling key root.
pub(crate) beacon_proposer_cache: Mutex<BeaconProposerCache>,
pub beacon_proposer_cache: Mutex<BeaconProposerCache>,
/// Caches a map of `validator_index -> validator_pubkey`.
pub(crate) validator_pubkey_cache: TimeoutRwLock<ValidatorPubkeyCache<T>>,
/// A list of any hard-coded forks that have been disabled.
@ -606,6 +610,10 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
/// A summarized version of `Self::head` that involves less cloning.
pub fn head_info(&self) -> Result<HeadInfo, Error> {
self.with_head(|head| {
let proposer_shuffling_decision_root = head
.beacon_state
.proposer_shuffling_decision_root(head.beacon_block_root)?;
Ok(HeadInfo {
slot: head.beacon_block.slot(),
block_root: head.beacon_block_root,
@ -615,6 +623,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
fork: head.beacon_state.fork,
genesis_time: head.beacon_state.genesis_time,
genesis_validators_root: head.beacon_state.genesis_validators_root,
proposer_shuffling_decision_root,
})
})
}
@ -773,6 +782,42 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
Ok(pubkey_cache.get(validator_index).cloned())
}
/// As per `Self::validator_pubkey`, but returns `PublicKeyBytes`.
pub fn validator_pubkey_bytes(
&self,
validator_index: usize,
) -> Result<Option<PublicKeyBytes>, Error> {
let pubkey_cache = self
.validator_pubkey_cache
.try_read_for(VALIDATOR_PUBKEY_CACHE_LOCK_TIMEOUT)
.ok_or(Error::ValidatorPubkeyCacheLockTimeout)?;
Ok(pubkey_cache.get_pubkey_bytes(validator_index).copied())
}
/// As per `Self::validator_pubkey_bytes` but will resolve multiple indices at once to avoid
/// bouncing the read-lock on the pubkey cache.
///
/// Returns a map that may have a length less than `validator_indices.len()` if some indices
/// were unable to be resolved.
pub fn validator_pubkey_bytes_many(
&self,
validator_indices: &[usize],
) -> Result<HashMap<usize, PublicKeyBytes>, Error> {
let pubkey_cache = self
.validator_pubkey_cache
.try_read_for(VALIDATOR_PUBKEY_CACHE_LOCK_TIMEOUT)
.ok_or(Error::ValidatorPubkeyCacheLockTimeout)?;
let mut map = HashMap::with_capacity(validator_indices.len());
for &validator_index in validator_indices {
if let Some(pubkey) = pubkey_cache.get_pubkey_bytes(validator_index) {
map.insert(validator_index, *pubkey);
}
}
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.
@ -803,19 +848,35 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
})
}
/// Returns the attestation duties for a given validator index.
/// Returns the attestation duties for the given validator indices using the shuffling cache.
///
/// Information is read from the current state, so only information from the present and prior
/// epoch is available.
pub fn validator_attestation_duty(
/// An error may be returned if `head_block_root` is a finalized block, this function is only
/// designed for operations at the head of the chain.
///
/// The returned `Vec` will have the same length as `validator_indices`, any
/// non-existing/inactive validators will have `None` values.
///
/// ## Notes
///
/// This function will try to use the shuffling cache to return the value. If the value is not
/// in the shuffling cache, it will be added. Care should be taken not to wash out the
/// shuffling cache with historical/useless values.
pub fn validator_attestation_duties(
&self,
validator_index: usize,
validator_indices: &[u64],
epoch: Epoch,
) -> Result<Option<AttestationDuty>, Error> {
let head_block_root = self.head_beacon_block_root()?;
head_block_root: Hash256,
) -> Result<(Vec<Option<AttestationDuty>>, Hash256), Error> {
self.with_committee_cache(head_block_root, epoch, |committee_cache, dependent_root| {
let duties = validator_indices
.iter()
.map(|validator_index| {
let validator_index = *validator_index as usize;
committee_cache.get_attestation_duties(validator_index)
})
.collect();
self.with_committee_cache(head_block_root, epoch, |committee_cache| {
Ok(committee_cache.get_attestation_duties(validator_index))
Ok((duties, dependent_root))
})
}
@ -867,6 +928,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
index,
head.beacon_block_root,
Cow::Borrowed(&head.beacon_state),
head.beacon_state_root(),
)
} else {
// We disallow producing attestations *prior* to the current head since such an
@ -902,6 +964,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
index: CommitteeIndex,
beacon_block_root: Hash256,
mut state: Cow<BeaconState<T::EthSpec>>,
state_root: Hash256,
) -> Result<Attestation<T::EthSpec>, Error> {
let epoch = slot.epoch(T::EthSpec::slots_per_epoch());
@ -909,13 +972,14 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
return Err(Error::CannotAttestToFutureState);
} else if state.current_epoch() < epoch {
let mut_state = state.to_mut();
while mut_state.current_epoch() < epoch {
// Note: here we provide `Hash256::zero()` as the root of the current state. This
// has the effect of setting the values of all historic state roots to the zero
// hash. This is an optimization, we don't need the state roots so why calculate
// them?
per_slot_processing(mut_state, Some(Hash256::zero()), &self.spec)?;
}
// Only perform a "partial" state advance since we do not require the state roots to be
// accurate.
partial_state_advance(
mut_state,
Some(state_root),
epoch.start_slot(T::EthSpec::slots_per_epoch()),
&self.spec,
)?;
mut_state.build_committee_cache(RelativeEpoch::Current, &self.spec)?;
}
@ -1861,7 +1925,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
pub fn produce_block_on_state(
&self,
mut state: BeaconState<T::EthSpec>,
mut state_root_opt: Option<Hash256>,
state_root_opt: Option<Hash256>,
produce_at_slot: Slot,
randao_reveal: Signature,
validator_graffiti: Option<Graffiti>,
@ -1880,15 +1944,10 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
}
let slot_timer = metrics::start_timer(&metrics::BLOCK_PRODUCTION_SLOT_PROCESS_TIMES);
// If required, transition the new state to the present slot.
//
// Note: supplying some `state_root` when it it is known would be a cheap and easy
// optimization.
while state.slot < produce_at_slot {
// Using `state_root.take()` here ensures that we consume the `state_root` on the first
// iteration and never use it again.
per_slot_processing(&mut state, state_root_opt.take(), &self.spec)?;
}
// Ensure the state has performed a complete transition into the required slot.
complete_state_advance(&mut state, state_root_opt, produce_at_slot, &self.spec)?;
drop(slot_timer);
state.build_committee_cache(RelativeEpoch::Current, &self.spec)?;
@ -2363,7 +2422,10 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
}
/// Runs the `map_fn` with the committee cache for `shuffling_epoch` from the chain with head
/// `head_block_root`.
/// `head_block_root`. The `map_fn` will be supplied two values:
///
/// - `&CommitteeCache`: the committee cache that serves the given parameters.
/// - `Hash256`: the "shuffling decision root" which uniquely identifies the `CommitteeCache`.
///
/// It's not necessary that `head_block_root` matches our current view of the chain, it can be
/// any block that is:
@ -2376,7 +2438,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
///
/// ## Important
///
/// This function is **not** suitable for determining proposer duties.
/// This function is **not** suitable for determining proposer duties (only attester duties).
///
/// ## Notes
///
@ -2394,7 +2456,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
map_fn: F,
) -> Result<R, Error>
where
F: Fn(&CommitteeCache) -> Result<R, Error>,
F: Fn(&CommitteeCache, Hash256) -> Result<R, Error>,
{
let head_block = self
.fork_choice
@ -2425,7 +2487,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
metrics::stop_timer(cache_wait_timer);
if let Some(committee_cache) = shuffling_cache.get(&shuffling_id) {
map_fn(committee_cache)
map_fn(committee_cache, shuffling_id.shuffling_decision_block)
} else {
// Drop the shuffling cache to avoid holding the lock for any longer than
// required.
@ -2434,33 +2496,80 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
debug!(
self.log,
"Committee cache miss";
"shuffling_epoch" => shuffling_epoch.as_u64(),
"shuffling_id" => ?shuffling_epoch,
"head_block_root" => head_block_root.to_string(),
);
let state_read_timer =
metrics::start_timer(&metrics::ATTESTATION_PROCESSING_STATE_READ_TIMES);
let mut state = self
.store
.get_inconsistent_state_for_attestation_verification_only(
&head_block.state_root,
Some(head_block.slot),
)?
.ok_or(Error::MissingBeaconState(head_block.state_root))?;
// If the head of the chain can serve this request, use it.
//
// This code is a little awkward because we need to ensure that the head we read and
// the head we copy is identical. Taking one lock to read the head values and another
// to copy the head is liable to race-conditions.
let head_state_opt = self.with_head(|head| {
if head.beacon_block_root == head_block_root {
Ok(Some((
head.beacon_state
.clone_with(CloneConfig::committee_caches_only()),
head.beacon_state_root(),
)))
} else {
Ok::<_, Error>(None)
}
})?;
// If the head state is useful for this request, use it. Otherwise, read a state from
// disk.
let (mut state, state_root) = if let Some((state, state_root)) = head_state_opt {
(state, state_root)
} else {
let state_root = head_block.state_root;
let state = self
.store
.get_inconsistent_state_for_attestation_verification_only(
&state_root,
Some(head_block.slot),
)?
.ok_or(Error::MissingBeaconState(head_block.state_root))?;
(state, state_root)
};
/*
* IMPORTANT
*
* Since it's possible that
* `Store::get_inconsistent_state_for_attestation_verification_only` was used to obtain
* the state, we cannot rely upon the following fields:
*
* - `state.state_roots`
* - `state.block_roots`
*
* These fields should not be used for the rest of this function.
*/
metrics::stop_timer(state_read_timer);
let state_skip_timer =
metrics::start_timer(&metrics::ATTESTATION_PROCESSING_STATE_SKIP_TIMES);
while state.current_epoch() + 1 < shuffling_epoch {
// Here we tell `per_slot_processing` to skip hashing the state and just
// use the zero hash instead.
//
// The state roots are not useful for the shuffling, so there's no need to
// compute them.
per_slot_processing(&mut state, Some(Hash256::zero()), &self.spec)
.map_err(Error::from)?;
// If the state is in an earlier epoch, advance it. If it's from a later epoch, reject
// it.
if state.current_epoch() + 1 < shuffling_epoch {
// Since there's a one-epoch look-ahead on the attester shuffling, it suffices to
// only advance into the slot prior to the `shuffling_epoch`.
let target_slot = shuffling_epoch
.saturating_sub(1_u64)
.start_slot(T::EthSpec::slots_per_epoch());
// Advance the state into the required slot, using the "partial" method since the state
// roots are not relevant for the shuffling.
partial_state_advance(&mut state, Some(state_root), target_slot, &self.spec)?;
} else if state.current_epoch() > shuffling_epoch {
return Err(Error::InvalidStateForShuffling {
state_epoch: state.current_epoch(),
shuffling_epoch,
});
}
metrics::stop_timer(state_skip_timer);
@ -2473,6 +2582,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
state.build_committee_cache(relative_epoch, &self.spec)?;
let committee_cache = state.committee_cache(relative_epoch)?;
let shuffling_decision_block = shuffling_id.shuffling_decision_block;
self.shuffling_cache
.try_write_for(ATTESTATION_CACHE_LOCK_TIMEOUT)
@ -2481,7 +2591,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
metrics::stop_timer(committee_building_timer);
map_fn(&committee_cache)
map_fn(&committee_cache, shuffling_decision_block)
}
}

View File

@ -59,7 +59,7 @@ impl Default for BeaconProposerCache {
impl BeaconProposerCache {
/// If it is cached, returns the proposer for the block at `slot` where the block has the
/// ancestor block root of `shuffling_decision_block` at `end_slot(slot.epoch() - 1)`.
pub fn get<T: EthSpec>(
pub fn get_slot<T: EthSpec>(
&mut self,
shuffling_decision_block: Hash256,
slot: Slot,
@ -84,6 +84,20 @@ impl BeaconProposerCache {
}
}
/// As per `Self::get_slot`, but returns all proposers in all slots for the given `epoch`.
///
/// The nth slot in the returned `SmallVec` will be equal to the nth slot in the given `epoch`.
/// E.g., if `epoch == 1` then `smallvec[0]` refers to slot 32 (assuming `SLOTS_PER_EPOCH ==
/// 32`).
pub fn get_epoch<T: EthSpec>(
&mut self,
shuffling_decision_block: Hash256,
epoch: Epoch,
) -> Option<&SmallVec<[usize; TYPICAL_SLOTS_PER_EPOCH]>> {
let key = (epoch, shuffling_decision_block);
self.cache.get(&key).map(|cache| &cache.proposers)
}
/// Insert the proposers into the cache.
///
/// See `Self::get` for a description of `shuffling_decision_block`.

View File

@ -60,7 +60,9 @@ use state_processing::{
block_signature_verifier::{BlockSignatureVerifier, Error as BlockSignatureVerifierError},
per_block_processing,
per_epoch_processing::EpochProcessingSummary,
per_slot_processing, BlockProcessingError, BlockSignatureStrategy, SlotProcessingError,
per_slot_processing,
state_advance::partial_state_advance,
BlockProcessingError, BlockSignatureStrategy, SlotProcessingError,
};
use std::borrow::Cow;
use std::convert::TryFrom;
@ -351,8 +353,12 @@ pub fn signature_verify_chain_segment<T: BeaconChainTypes>(
.map(|(_, block)| block.slot())
.unwrap_or_else(|| slot);
let state =
cheap_state_advance_to_obtain_committees(&mut parent.pre_state, highest_slot, &chain.spec)?;
let state = cheap_state_advance_to_obtain_committees(
&mut parent.pre_state,
parent.beacon_state_root,
highest_slot,
&chain.spec,
)?;
let pubkey_cache = get_validator_pubkey_cache(chain)?;
let mut signature_verifier = get_signature_verifier(&state, &pubkey_cache, &chain.spec);
@ -564,7 +570,7 @@ impl<T: BeaconChainTypes> GossipVerifiedBlock<T> {
let proposer_opt = chain
.beacon_proposer_cache
.lock()
.get::<T::EthSpec>(proposer_shuffling_decision_block, block.slot());
.get_slot::<T::EthSpec>(proposer_shuffling_decision_block, block.slot());
let (expected_proposer, fork, parent, block) = if let Some(proposer) = proposer_opt {
// The proposer index was cached and we can return it without needing to load the
// parent.
@ -586,6 +592,7 @@ impl<T: BeaconChainTypes> GossipVerifiedBlock<T> {
// The state produced is only valid for determining proposer/attester shuffling indices.
let state = cheap_state_advance_to_obtain_committees(
&mut parent.pre_state,
parent.beacon_state_root,
block.slot(),
&chain.spec,
)?;
@ -694,6 +701,7 @@ impl<T: BeaconChainTypes> SignatureVerifiedBlock<T> {
let state = cheap_state_advance_to_obtain_committees(
&mut parent.pre_state,
parent.beacon_state_root,
block.slot(),
&chain.spec,
)?;
@ -738,6 +746,7 @@ impl<T: BeaconChainTypes> SignatureVerifiedBlock<T> {
let state = cheap_state_advance_to_obtain_committees(
&mut parent.pre_state,
parent.beacon_state_root,
block.slot(),
&chain.spec,
)?;
@ -1280,6 +1289,7 @@ fn load_parent<T: BeaconChainTypes>(
beacon_block: parent_block,
beacon_block_root: root,
pre_state: parent_state,
beacon_state_root: Some(parent_state_root),
},
block,
))
@ -1303,6 +1313,7 @@ fn load_parent<T: BeaconChainTypes>(
/// mutated to be invalid (in fact, it is never changed beyond a simple committee cache build).
fn cheap_state_advance_to_obtain_committees<'a, E: EthSpec>(
state: &'a mut BeaconState<E>,
state_root_opt: Option<Hash256>,
block_slot: Slot,
spec: &ChainSpec,
) -> Result<Cow<'a, BeaconState<E>>, BlockError<E>> {
@ -1319,14 +1330,12 @@ fn cheap_state_advance_to_obtain_committees<'a, E: EthSpec>(
})
} else {
let mut state = state.clone_with(CloneConfig::committee_caches_only());
let target_slot = block_epoch.start_slot(E::slots_per_epoch());
while state.current_epoch() < block_epoch {
// Don't calculate state roots since they aren't required for calculating
// shuffling (achieved by providing Hash256::zero()).
per_slot_processing(&mut state, Some(Hash256::zero()), spec).map_err(|e| {
BlockError::BeaconChainError(BeaconChainError::SlotProcessingError(e))
})?;
}
// Advance the state into the same epoch as the block. Use the "partial" method since state
// roots are not important for proposer/attester shuffling.
partial_state_advance(&mut state, state_root_opt, target_slot, spec)
.map_err(|e| BlockError::BeaconChainError(BeaconChainError::from(e)))?;
state.build_committee_cache(RelativeEpoch::Current, spec)?;

View File

@ -17,6 +17,7 @@ use state_processing::{
ProposerSlashingValidationError,
},
signature_sets::Error as SignatureSetError,
state_advance::Error as StateAdvanceError,
BlockProcessingError, SlotProcessingError,
};
use std::time::Duration;
@ -51,6 +52,7 @@ pub enum BeaconChainError {
MissingBeaconBlock(Hash256),
MissingBeaconState(Hash256),
SlotProcessingError(SlotProcessingError),
StateAdvanceError(StateAdvanceError),
UnableToAdvanceState(String),
NoStateForAttestation {
beacon_block_root: Hash256,
@ -81,6 +83,7 @@ pub enum BeaconChainError {
BlockSignatureVerifierError(state_processing::block_signature_verifier::Error),
DuplicateValidatorPublicKey,
ValidatorPubkeyCacheFileError(String),
ValidatorIndexUnknown(usize),
OpPoolError(OpPoolError),
NaiveAggregationError(NaiveAggregationError),
ObservedAttestationsError(ObservedAttestationsError),
@ -105,6 +108,10 @@ pub enum BeaconChainError {
block_slot: Slot,
state_slot: Slot,
},
InvalidStateForShuffling {
state_epoch: Epoch,
shuffling_epoch: Epoch,
},
}
easy_from_to!(SlotProcessingError, BeaconChainError);
@ -122,6 +129,7 @@ easy_from_to!(BlockSignatureVerifierError, BeaconChainError);
easy_from_to!(PruningError, BeaconChainError);
easy_from_to!(ArithError, BeaconChainError);
easy_from_to!(ForkChoiceStoreError, BeaconChainError);
easy_from_to!(StateAdvanceError, BeaconChainError);
#[derive(Debug)]
pub enum BlockProductionError {
@ -133,6 +141,7 @@ pub enum BlockProductionError {
BlockProcessingError(BlockProcessingError),
Eth1ChainError(Eth1ChainError),
BeaconStateError(BeaconStateError),
StateAdvanceError(StateAdvanceError),
OpPoolError(OpPoolError),
/// The `BeaconChain` was explicitly configured _without_ a connection to eth1, therefore it
/// cannot produce blocks.
@ -147,3 +156,4 @@ easy_from_to!(BlockProcessingError, BlockProductionError);
easy_from_to!(BeaconStateError, BlockProductionError);
easy_from_to!(SlotProcessingError, BlockProductionError);
easy_from_to!(Eth1ChainError, BlockProductionError);
easy_from_to!(StateAdvanceError, BlockProductionError);

View File

@ -14,14 +14,18 @@ pub struct PreProcessingSnapshot<T: EthSpec> {
/// advanced forward one slot using `per_slot_processing`. This state is "primed and ready" for
/// the application of another block.
pub pre_state: BeaconState<T>,
/// This value is only set to `Some` if the `pre_state` was *not* advanced forward.
pub beacon_state_root: Option<Hash256>,
pub beacon_block: SignedBeaconBlock<T>,
pub beacon_block_root: Hash256,
}
impl<T: EthSpec> From<BeaconSnapshot<T>> for PreProcessingSnapshot<T> {
fn from(snapshot: BeaconSnapshot<T>) -> Self {
let beacon_state_root = Some(snapshot.beacon_state_root());
Self {
pre_state: snapshot.beacon_state,
beacon_state_root,
beacon_block: snapshot.beacon_block,
beacon_block_root: snapshot.beacon_block_root,
}
@ -47,10 +51,15 @@ impl<T: EthSpec> CacheItem<T> {
}
pub fn into_pre_state(self) -> PreProcessingSnapshot<T> {
// Do not include the beacon state root if the state has been advanced.
let beacon_state_root =
Some(self.beacon_block.state_root()).filter(|_| self.pre_state.is_none());
PreProcessingSnapshot {
beacon_block: self.beacon_block,
beacon_block_root: self.beacon_block_root,
pre_state: self.pre_state.unwrap_or(self.beacon_state),
beacon_state_root,
}
}
}

View File

@ -15,8 +15,9 @@
//! 2. There's a possibility that the head block is never built upon, causing wasted CPU cycles.
use crate::validator_monitor::HISTORIC_EPOCHS as VALIDATOR_MONITOR_HISTORIC_EPOCHS;
use crate::{
beacon_chain::BLOCK_PROCESSING_CACHE_LOCK_TIMEOUT, snapshot_cache::StateAdvance, BeaconChain,
BeaconChainError, BeaconChainTypes,
beacon_chain::{ATTESTATION_CACHE_LOCK_TIMEOUT, BLOCK_PROCESSING_CACHE_LOCK_TIMEOUT},
snapshot_cache::StateAdvance,
BeaconChain, BeaconChainError, BeaconChainTypes,
};
use slog::{debug, error, warn, Logger};
use slot_clock::SlotClock;
@ -27,7 +28,7 @@ use std::sync::{
};
use task_executor::TaskExecutor;
use tokio::time::sleep;
use types::{EthSpec, Hash256, Slot};
use types::{AttestationShufflingId, EthSpec, Hash256, RelativeEpoch, Slot};
/// If the head slot is more than `MAX_ADVANCE_DISTANCE` from the current slot, then don't perform
/// the state advancement.
@ -252,16 +253,22 @@ fn advance_head<T: BeaconChainTypes>(
"current_slot" => current_slot,
);
// If the advanced state is in a later epoch than where it started, pre-emptively add the
// proposer shuffling for the new epoch into the cache.
if state.current_epoch() > initial_epoch {
debug!(
log,
"Priming proposer cache";
"head_root" => ?head_root,
"state_epoch" => state.current_epoch(),
"current_epoch" => current_slot.epoch(T::EthSpec::slots_per_epoch()),
);
// Build the current epoch cache, to prepare to compute proposer duties.
state
.build_committee_cache(RelativeEpoch::Current, &beacon_chain.spec)
.map_err(BeaconChainError::from)?;
// Build the next epoch cache, to prepare to compute attester duties.
state
.build_committee_cache(RelativeEpoch::Next, &beacon_chain.spec)
.map_err(BeaconChainError::from)?;
// If the `pre_state` is in a later epoch than `state`, pre-emptively add the proposer shuffling
// for the state's current epoch and the committee cache for the state's next epoch.
if initial_epoch < state.current_epoch() {
// Update the proposer cache.
//
// We supply the `head_root` as the decision block since the prior `if` statement guarantees
// the head root is the latest block from the prior epoch.
beacon_chain
.beacon_proposer_cache
.lock()
@ -274,6 +281,27 @@ fn advance_head<T: BeaconChainTypes>(
state.fork,
)
.map_err(BeaconChainError::from)?;
// Update the attester cache.
let shuffling_id = AttestationShufflingId::new(head_root, &state, RelativeEpoch::Next)
.map_err(BeaconChainError::from)?;
let committee_cache = state
.committee_cache(RelativeEpoch::Next)
.map_err(BeaconChainError::from)?;
beacon_chain
.shuffling_cache
.try_write_for(ATTESTATION_CACHE_LOCK_TIMEOUT)
.ok_or(BeaconChainError::AttestationCacheLockTimeout)?
.insert(shuffling_id.clone(), committee_cache);
debug!(
log,
"Primed proposer and attester caches";
"head_root" => ?head_root,
"next_epoch_shuffling_root" => ?shuffling_id.shuffling_decision_block,
"state_epoch" => state.current_epoch(),
"current_epoch" => current_slot.epoch(T::EthSpec::slots_per_epoch()),
);
}
let final_slot = state.slot;

View File

@ -19,7 +19,7 @@ use rand::SeedableRng;
use rayon::prelude::*;
use slog::Logger;
use slot_clock::TestingSlotClock;
use state_processing::per_slot_processing;
use state_processing::state_advance::complete_state_advance;
use std::borrow::Cow;
use std::collections::{HashMap, HashSet};
use std::sync::Arc;
@ -331,6 +331,12 @@ where
self.chain.head().unwrap().beacon_state
}
pub fn get_current_state_and_root(&self) -> (BeaconState<E>, Hash256) {
let head = self.chain.head().unwrap();
let state_root = head.beacon_state_root();
(head.beacon_state, state_root)
}
pub fn get_current_slot(&self) -> Slot {
self.chain.slot().unwrap()
}
@ -377,10 +383,8 @@ where
assert_ne!(slot, 0, "can't produce a block at slot 0");
assert!(slot >= state.slot);
while state.slot < slot {
per_slot_processing(&mut state, None, &self.spec)
.expect("should be able to advance state to slot");
}
complete_state_advance(&mut state, None, slot, &self.spec)
.expect("should be able to advance state to slot");
state
.build_all_caches(&self.spec)
@ -430,6 +434,7 @@ where
&self,
attesting_validators: &[usize],
state: &BeaconState<E>,
state_root: Hash256,
head_block_root: SignedBeaconBlockHash,
attestation_slot: Slot,
) -> Vec<Vec<(Attestation<E>, SubnetId)>> {
@ -454,6 +459,7 @@ where
bc.index,
head_block_root.into(),
Cow::Borrowed(state),
state_root,
)
.unwrap();
@ -503,6 +509,7 @@ where
&self,
attestation_strategy: &AttestationStrategy,
state: &BeaconState<E>,
state_root: Hash256,
head_block_root: Hash256,
attestation_slot: Slot,
) -> Vec<Vec<(Attestation<E>, SubnetId)>> {
@ -513,6 +520,7 @@ where
self.make_unaggregated_attestations(
&validators,
state,
state_root,
head_block_root.into(),
attestation_slot,
)
@ -522,11 +530,17 @@ where
&self,
attesting_validators: &[usize],
state: &BeaconState<E>,
state_root: Hash256,
block_hash: SignedBeaconBlockHash,
slot: Slot,
) -> HarnessAttestations<E> {
let unaggregated_attestations =
self.make_unaggregated_attestations(&attesting_validators, &state, block_hash, slot);
let unaggregated_attestations = self.make_unaggregated_attestations(
&attesting_validators,
&state,
state_root,
block_hash,
slot,
);
let aggregated_attestations: Vec<Option<SignedAggregateAndProof<E>>> = unaggregated_attestations
.iter()
@ -754,12 +768,18 @@ where
pub fn attest_block(
&self,
state: &BeaconState<E>,
state_root: Hash256,
block_hash: SignedBeaconBlockHash,
block: &SignedBeaconBlock<E>,
validators: &[usize],
) {
let attestations =
self.make_attestations(validators, &state, block_hash, block.message.slot);
let attestations = self.make_attestations(
validators,
&state,
state_root,
block_hash,
block.message.slot,
);
self.process_attestations(attestations);
}
@ -767,26 +787,29 @@ where
&self,
slot: Slot,
state: BeaconState<E>,
state_root: Hash256,
validators: &[usize],
) -> Result<(SignedBeaconBlockHash, BeaconState<E>), BlockError<E>> {
let (block_hash, block, state) = self.add_block_at_slot(slot, state)?;
self.attest_block(&state, block_hash, &block, validators);
self.attest_block(&state, state_root, block_hash, &block, validators);
Ok((block_hash, state))
}
pub fn add_attested_blocks_at_slots(
&self,
state: BeaconState<E>,
state_root: Hash256,
slots: &[Slot],
validators: &[usize],
) -> AddBlocksResult<E> {
assert!(!slots.is_empty());
self.add_attested_blocks_at_slots_given_lbh(state, slots, validators, None)
self.add_attested_blocks_at_slots_given_lbh(state, state_root, slots, validators, None)
}
fn add_attested_blocks_at_slots_given_lbh(
&self,
mut state: BeaconState<E>,
state_root: Hash256,
slots: &[Slot],
validators: &[usize],
mut latest_block_hash: Option<SignedBeaconBlockHash>,
@ -799,7 +822,7 @@ where
let mut state_hash_from_slot: HashMap<Slot, BeaconStateHash> = HashMap::new();
for slot in slots {
let (block_hash, new_state) = self
.add_attested_block_at_slot(*slot, state, validators)
.add_attested_block_at_slot(*slot, state, state_root, validators)
.unwrap();
state = new_state;
block_hash_from_slot.insert(*slot, block_hash);
@ -857,8 +880,14 @@ where
for epoch in min_epoch.as_u64()..=max_epoch.as_u64() {
let mut new_chains = vec![];
for (head_state, slots, validators, mut block_hashes, mut state_hashes, head_block) in
chains
for (
mut head_state,
slots,
validators,
mut block_hashes,
mut state_hashes,
head_block,
) in chains
{
let epoch_slots = slots
.iter()
@ -866,9 +895,11 @@ where
.copied()
.collect::<Vec<_>>();
let head_state_root = head_state.update_tree_hash_cache().unwrap();
let (new_block_hashes, new_state_hashes, new_head_block, new_head_state) = self
.add_attested_blocks_at_slots_given_lbh(
head_state,
head_state_root,
&epoch_slots,
&validators,
Some(head_block),
@ -947,7 +978,7 @@ where
block_strategy: BlockStrategy,
attestation_strategy: AttestationStrategy,
) -> Hash256 {
let (state, slots) = match block_strategy {
let (mut state, slots) = match block_strategy {
BlockStrategy::OnCanonicalHead => {
let current_slot: u64 = self.get_current_slot().into();
let slots: Vec<Slot> = (current_slot..(current_slot + (num_blocks as u64)))
@ -975,8 +1006,9 @@ where
AttestationStrategy::AllValidators => self.get_all_validators(),
AttestationStrategy::SomeValidators(vals) => vals,
};
let state_root = state.update_tree_hash_cache().unwrap();
let (_, _, last_produced_block_hash, _) =
self.add_attested_blocks_at_slots(state, &slots, &validators);
self.add_attested_blocks_at_slots(state, state_root, &slots, &validators);
last_produced_block_hash.into()
}

View File

@ -23,6 +23,7 @@ use types::{BeaconState, Hash256, PublicKey, PublicKeyBytes};
pub struct ValidatorPubkeyCache<T: BeaconChainTypes> {
pubkeys: Vec<PublicKey>,
indices: HashMap<PublicKeyBytes, usize>,
pubkey_bytes: Vec<PublicKeyBytes>,
backing: PubkeyCacheBacking<T>,
}
@ -46,6 +47,7 @@ impl<T: BeaconChainTypes> ValidatorPubkeyCache<T> {
let mut cache = Self {
pubkeys: vec![],
indices: HashMap::new(),
pubkey_bytes: vec![],
backing: PubkeyCacheBacking::Database(store),
};
@ -58,12 +60,14 @@ impl<T: BeaconChainTypes> ValidatorPubkeyCache<T> {
pub fn load_from_store(store: BeaconStore<T>) -> Result<Self, BeaconChainError> {
let mut pubkeys = vec![];
let mut indices = HashMap::new();
let mut pubkey_bytes = vec![];
for validator_index in 0.. {
if let Some(DatabasePubkey(pubkey)) =
store.get_item(&DatabasePubkey::key_for_index(validator_index))?
{
pubkeys.push((&pubkey).try_into().map_err(Error::PubkeyDecode)?);
pubkey_bytes.push(pubkey);
indices.insert(pubkey, validator_index);
} else {
break;
@ -73,6 +77,7 @@ impl<T: BeaconChainTypes> ValidatorPubkeyCache<T> {
Ok(ValidatorPubkeyCache {
pubkeys,
indices,
pubkey_bytes,
backing: PubkeyCacheBacking::Database(store),
})
}
@ -91,6 +96,7 @@ impl<T: BeaconChainTypes> ValidatorPubkeyCache<T> {
let mut result = ValidatorPubkeyCache {
pubkeys: Vec::with_capacity(existing_cache.pubkeys.len()),
indices: HashMap::with_capacity(existing_cache.indices.len()),
pubkey_bytes: Vec::with_capacity(existing_cache.indices.len()),
backing: PubkeyCacheBacking::Database(store),
};
result.import(existing_cache.pubkeys.iter().map(PublicKeyBytes::from))?;
@ -120,6 +126,7 @@ impl<T: BeaconChainTypes> ValidatorPubkeyCache<T> {
where
I: Iterator<Item = PublicKeyBytes> + ExactSizeIterator,
{
self.pubkey_bytes.reserve(validator_keys.len());
self.pubkeys.reserve(validator_keys.len());
self.indices.reserve(validator_keys.len());
@ -153,6 +160,7 @@ impl<T: BeaconChainTypes> ValidatorPubkeyCache<T> {
.try_into()
.map_err(BeaconChainError::InvalidValidatorPubkeyBytes)?,
);
self.pubkey_bytes.push(pubkey);
self.indices.insert(pubkey, i);
}
@ -165,6 +173,11 @@ impl<T: BeaconChainTypes> ValidatorPubkeyCache<T> {
self.pubkeys.get(i)
}
/// Get the public key (in bytes form) for a validator with index `i`.
pub fn get_pubkey_bytes(&self, i: usize) -> Option<&PublicKeyBytes> {
self.pubkey_bytes.get(i)
}
/// Get the index of a validator with `pubkey`.
pub fn get_index(&self, pubkey: &PublicKeyBytes) -> Option<usize> {
self.indices.get(pubkey).copied()
@ -264,13 +277,15 @@ impl ValidatorPubkeyCacheFile {
let mut last = None;
let mut pubkeys = Vec::with_capacity(list.len());
let mut indices = HashMap::new();
let mut indices = HashMap::with_capacity(list.len());
let mut pubkey_bytes = Vec::with_capacity(list.len());
for (index, pubkey) in list {
let expected = last.map(|n| n + 1);
if expected.map_or(true, |expected| index == expected) {
last = Some(index);
pubkeys.push((&pubkey).try_into().map_err(Error::PubkeyDecode)?);
pubkey_bytes.push(pubkey);
indices.insert(pubkey, index);
} else {
return Err(Error::InconsistentIndex {
@ -283,6 +298,7 @@ impl ValidatorPubkeyCacheFile {
Ok(ValidatorPubkeyCache {
pubkeys,
indices,
pubkey_bytes,
backing: PubkeyCacheBacking::File(self),
})
}

View File

@ -909,10 +909,13 @@ fn attestation_that_skips_epochs() {
per_slot_processing(&mut state, None, &harness.spec).expect("should process slot");
}
let state_root = state.update_tree_hash_cache().unwrap();
let (attestation, subnet_id) = harness
.get_unaggregated_attestations(
&AttestationStrategy::AllValidators,
&state,
state_root,
earlier_block.canonical_root(),
current_slot,
)

View File

@ -276,6 +276,7 @@ fn epoch_boundary_state_attestation_processing() {
late_attestations.extend(harness.get_unaggregated_attestations(
&AttestationStrategy::SomeValidators(late_validators.clone()),
&head.beacon_state,
head.beacon_state_root(),
head.beacon_block_root,
head.beacon_block.slot(),
));
@ -353,9 +354,9 @@ fn delete_blocks_and_states() {
// Finalize an initial portion of the chain.
let initial_slots: Vec<Slot> = (1..=unforked_blocks).map(Into::into).collect();
let state = harness.get_current_state();
let (state, state_root) = harness.get_current_state_and_root();
let all_validators = harness.get_all_validators();
harness.add_attested_blocks_at_slots(state, &initial_slots, &all_validators);
harness.add_attested_blocks_at_slots(state, state_root, &initial_slots, &all_validators);
// Create a fork post-finalization.
let two_thirds = (LOW_VALIDATOR_COUNT / 3) * 2;
@ -478,9 +479,9 @@ fn multi_epoch_fork_valid_blocks_test(
// Create the initial portion of the chain
if initial_blocks > 0 {
let initial_slots: Vec<Slot> = (1..=initial_blocks).map(Into::into).collect();
let state = harness.get_current_state();
let (state, state_root) = harness.get_current_state_and_root();
let all_validators = harness.get_all_validators();
harness.add_attested_blocks_at_slots(state, &initial_slots, &all_validators);
harness.add_attested_blocks_at_slots(state, state_root, &initial_slots, &all_validators);
}
assert!(num_fork1_validators <= LOW_VALIDATOR_COUNT);
@ -759,19 +760,26 @@ fn prunes_abandoned_fork_between_two_finalized_checkpoints() {
let adversarial_validators: Vec<usize> = (HONEST_VALIDATOR_COUNT..VALIDATOR_COUNT).collect();
let rig = BeaconChainHarness::new(MinimalEthSpec, validators_keypairs);
let slots_per_epoch = rig.slots_per_epoch();
let mut state = rig.get_current_state();
let (mut state, state_root) = rig.get_current_state_and_root();
let canonical_chain_slots: Vec<Slot> = (1..=rig.epoch_start_slot(1)).map(Slot::new).collect();
let (canonical_chain_blocks_pre_finalization, _, _, new_state) =
rig.add_attested_blocks_at_slots(state, &canonical_chain_slots, &honest_validators);
let (canonical_chain_blocks_pre_finalization, _, _, new_state) = rig
.add_attested_blocks_at_slots(
state,
state_root,
&canonical_chain_slots,
&honest_validators,
);
state = new_state;
let canonical_chain_slot: u64 = rig.get_current_slot().into();
let stray_slots: Vec<Slot> = (canonical_chain_slot + 1..rig.epoch_start_slot(2))
.map(Slot::new)
.collect();
let (current_state, current_state_root) = rig.get_current_state_and_root();
let (stray_blocks, stray_states, stray_head, _) = rig.add_attested_blocks_at_slots(
rig.get_current_state(),
current_state,
current_state_root,
&stray_slots,
&adversarial_validators,
);
@ -803,8 +811,13 @@ fn prunes_abandoned_fork_between_two_finalized_checkpoints() {
..=(canonical_chain_slot + slots_per_epoch * 5))
.map(Slot::new)
.collect();
let (canonical_chain_blocks_post_finalization, _, _, _) =
rig.add_attested_blocks_at_slots(state, &finalization_slots, &honest_validators);
let state_root = state.update_tree_hash_cache().unwrap();
let (canonical_chain_blocks_post_finalization, _, _, _) = rig.add_attested_blocks_at_slots(
state,
state_root,
&finalization_slots,
&honest_validators,
);
// Postcondition: New blocks got finalized
assert_eq!(
@ -852,13 +865,14 @@ fn pruning_does_not_touch_abandoned_block_shared_with_canonical_chain() {
let adversarial_validators: Vec<usize> = (HONEST_VALIDATOR_COUNT..VALIDATOR_COUNT).collect();
let rig = BeaconChainHarness::new(MinimalEthSpec, validators_keypairs);
let slots_per_epoch = rig.slots_per_epoch();
let state = rig.get_current_state();
let (state, state_root) = rig.get_current_state_and_root();
// Fill up 0th epoch
let canonical_chain_slots_zeroth_epoch: Vec<Slot> =
(1..rig.epoch_start_slot(1)).map(Slot::new).collect();
let (_, _, _, state) = rig.add_attested_blocks_at_slots(
let (_, _, _, mut state) = rig.add_attested_blocks_at_slots(
state,
state_root,
&canonical_chain_slots_zeroth_epoch,
&honest_validators,
);
@ -868,9 +882,11 @@ fn pruning_does_not_touch_abandoned_block_shared_with_canonical_chain() {
..=rig.epoch_start_slot(1) + 1)
.map(Slot::new)
.collect();
let (canonical_chain_blocks_first_epoch, _, shared_head, state) = rig
let state_root = state.update_tree_hash_cache().unwrap();
let (canonical_chain_blocks_first_epoch, _, shared_head, mut state) = rig
.add_attested_blocks_at_slots(
state.clone(),
state_root,
&canonical_chain_slots_first_epoch,
&honest_validators,
);
@ -880,8 +896,10 @@ fn pruning_does_not_touch_abandoned_block_shared_with_canonical_chain() {
..=rig.epoch_start_slot(1) + 2)
.map(Slot::new)
.collect();
let state_root = state.update_tree_hash_cache().unwrap();
let (stray_blocks, stray_states, stray_head, _) = rig.add_attested_blocks_at_slots(
state.clone(),
state_root,
&stray_chain_slots_first_epoch,
&adversarial_validators,
);
@ -917,8 +935,13 @@ fn pruning_does_not_touch_abandoned_block_shared_with_canonical_chain() {
..=(canonical_chain_slot + slots_per_epoch * 5))
.map(Slot::new)
.collect();
let (canonical_chain_blocks, _, _, _) =
rig.add_attested_blocks_at_slots(state, &finalization_slots, &honest_validators);
let state_root = state.update_tree_hash_cache().unwrap();
let (canonical_chain_blocks, _, _, _) = rig.add_attested_blocks_at_slots(
state,
state_root,
&finalization_slots,
&honest_validators,
);
// Postconditions
assert_eq!(
@ -967,12 +990,16 @@ fn pruning_does_not_touch_blocks_prior_to_finalization() {
let adversarial_validators: Vec<usize> = (HONEST_VALIDATOR_COUNT..VALIDATOR_COUNT).collect();
let rig = BeaconChainHarness::new(MinimalEthSpec, validators_keypairs);
let slots_per_epoch = rig.slots_per_epoch();
let mut state = rig.get_current_state();
let (mut state, state_root) = rig.get_current_state_and_root();
// Fill up 0th epoch with canonical chain blocks
let zeroth_epoch_slots: Vec<Slot> = (1..=rig.epoch_start_slot(1)).map(Slot::new).collect();
let (canonical_chain_blocks, _, _, new_state) =
rig.add_attested_blocks_at_slots(state, &zeroth_epoch_slots, &honest_validators);
let (canonical_chain_blocks, _, _, new_state) = rig.add_attested_blocks_at_slots(
state,
state_root,
&zeroth_epoch_slots,
&honest_validators,
);
state = new_state;
let canonical_chain_slot: u64 = rig.get_current_slot().into();
@ -980,8 +1007,10 @@ fn pruning_does_not_touch_blocks_prior_to_finalization() {
let first_epoch_slots: Vec<Slot> = ((rig.epoch_start_slot(1) + 1)..(rig.epoch_start_slot(2)))
.map(Slot::new)
.collect();
let state_root = state.update_tree_hash_cache().unwrap();
let (stray_blocks, stray_states, stray_head, _) = rig.add_attested_blocks_at_slots(
state.clone(),
state_root,
&first_epoch_slots,
&adversarial_validators,
);
@ -1011,7 +1040,9 @@ fn pruning_does_not_touch_blocks_prior_to_finalization() {
..=(canonical_chain_slot + slots_per_epoch * 4))
.map(Slot::new)
.collect();
let (_, _, _, _) = rig.add_attested_blocks_at_slots(state, &slots, &honest_validators);
let state_root = state.update_tree_hash_cache().unwrap();
let (_, _, _, _) =
rig.add_attested_blocks_at_slots(state, state_root, &slots, &honest_validators);
// Postconditions
assert_eq!(
@ -1048,30 +1079,42 @@ fn prunes_fork_growing_past_youngest_finalized_checkpoint() {
let honest_validators: Vec<usize> = (0..HONEST_VALIDATOR_COUNT).collect();
let adversarial_validators: Vec<usize> = (HONEST_VALIDATOR_COUNT..VALIDATOR_COUNT).collect();
let rig = BeaconChainHarness::new(MinimalEthSpec, validators_keypairs);
let state = rig.get_current_state();
let (state, state_root) = rig.get_current_state_and_root();
// Fill up 0th epoch with canonical chain blocks
let zeroth_epoch_slots: Vec<Slot> = (1..=rig.epoch_start_slot(1)).map(Slot::new).collect();
let (canonical_blocks_zeroth_epoch, _, _, state) =
rig.add_attested_blocks_at_slots(state, &zeroth_epoch_slots, &honest_validators);
let (canonical_blocks_zeroth_epoch, _, _, mut state) = rig.add_attested_blocks_at_slots(
state,
state_root,
&zeroth_epoch_slots,
&honest_validators,
);
// Fill up 1st epoch. Contains a fork.
let slots_first_epoch: Vec<Slot> = (rig.epoch_start_slot(1) + 1..rig.epoch_start_slot(2))
.map(Into::into)
.collect();
let (stray_blocks_first_epoch, stray_states_first_epoch, _, stray_state) = rig
.add_attested_blocks_at_slots(state.clone(), &slots_first_epoch, &adversarial_validators);
let (canonical_blocks_first_epoch, _, _, canonical_state) =
rig.add_attested_blocks_at_slots(state, &slots_first_epoch, &honest_validators);
let state_root = state.update_tree_hash_cache().unwrap();
let (stray_blocks_first_epoch, stray_states_first_epoch, _, mut stray_state) = rig
.add_attested_blocks_at_slots(
state.clone(),
state_root,
&slots_first_epoch,
&adversarial_validators,
);
let (canonical_blocks_first_epoch, _, _, mut canonical_state) =
rig.add_attested_blocks_at_slots(state, state_root, &slots_first_epoch, &honest_validators);
// Fill up 2nd epoch. Extends both the canonical chain and the fork.
let stray_slots_second_epoch: Vec<Slot> = (rig.epoch_start_slot(2)
..=rig.epoch_start_slot(2) + 1)
.map(Into::into)
.collect();
let stray_state_root = stray_state.update_tree_hash_cache().unwrap();
let (stray_blocks_second_epoch, stray_states_second_epoch, stray_head, _) = rig
.add_attested_blocks_at_slots(
stray_state,
stray_state_root,
&stray_slots_second_epoch,
&adversarial_validators,
);
@ -1113,8 +1156,13 @@ fn prunes_fork_growing_past_youngest_finalized_checkpoint() {
let canonical_slots: Vec<Slot> = (rig.epoch_start_slot(2)..=rig.epoch_start_slot(6))
.map(Into::into)
.collect();
let (canonical_blocks, _, _, _) =
rig.add_attested_blocks_at_slots(canonical_state, &canonical_slots, &honest_validators);
let canonical_state_root = canonical_state.update_tree_hash_cache().unwrap();
let (canonical_blocks, _, _, _) = rig.add_attested_blocks_at_slots(
canonical_state,
canonical_state_root,
&canonical_slots,
&honest_validators,
);
// Postconditions
let canonical_blocks: HashMap<Slot, SignedBeaconBlockHash> = canonical_blocks_zeroth_epoch
@ -1169,23 +1217,27 @@ fn prunes_skipped_slots_states() {
let honest_validators: Vec<usize> = (0..HONEST_VALIDATOR_COUNT).collect();
let adversarial_validators: Vec<usize> = (HONEST_VALIDATOR_COUNT..VALIDATOR_COUNT).collect();
let rig = BeaconChainHarness::new(MinimalEthSpec, validators_keypairs);
let state = rig.get_current_state();
let (state, state_root) = rig.get_current_state_and_root();
let canonical_slots_zeroth_epoch: Vec<Slot> =
(1..=rig.epoch_start_slot(1)).map(Into::into).collect();
let (canonical_blocks_zeroth_epoch, _, _, canonical_state) = rig.add_attested_blocks_at_slots(
state.clone(),
&canonical_slots_zeroth_epoch,
&honest_validators,
);
let (canonical_blocks_zeroth_epoch, _, _, mut canonical_state) = rig
.add_attested_blocks_at_slots(
state.clone(),
state_root,
&canonical_slots_zeroth_epoch,
&honest_validators,
);
let skipped_slot: Slot = (rig.epoch_start_slot(1) + 1).into();
let stray_slots: Vec<Slot> = ((skipped_slot + 1).into()..rig.epoch_start_slot(2))
.map(Into::into)
.collect();
let canonical_state_root = canonical_state.update_tree_hash_cache().unwrap();
let (stray_blocks, stray_states, _, stray_state) = rig.add_attested_blocks_at_slots(
canonical_state.clone(),
canonical_state_root,
&stray_slots,
&adversarial_validators,
);
@ -1225,8 +1277,13 @@ fn prunes_skipped_slots_states() {
let canonical_slots: Vec<Slot> = ((skipped_slot + 1).into()..rig.epoch_start_slot(7))
.map(Into::into)
.collect();
let (canonical_blocks_post_finalization, _, _, _) =
rig.add_attested_blocks_at_slots(canonical_state, &canonical_slots, &honest_validators);
let canonical_state_root = canonical_state.update_tree_hash_cache().unwrap();
let (canonical_blocks_post_finalization, _, _, _) = rig.add_attested_blocks_at_slots(
canonical_state,
canonical_state_root,
&canonical_slots,
&honest_validators,
);
// Postconditions
let canonical_blocks: HashMap<Slot, SignedBeaconBlockHash> = canonical_blocks_zeroth_epoch
@ -1279,23 +1336,27 @@ fn finalizes_non_epoch_start_slot() {
let honest_validators: Vec<usize> = (0..HONEST_VALIDATOR_COUNT).collect();
let adversarial_validators: Vec<usize> = (HONEST_VALIDATOR_COUNT..VALIDATOR_COUNT).collect();
let rig = BeaconChainHarness::new(MinimalEthSpec, validators_keypairs);
let state = rig.get_current_state();
let (state, state_root) = rig.get_current_state_and_root();
let canonical_slots_zeroth_epoch: Vec<Slot> =
(1..rig.epoch_start_slot(1)).map(Into::into).collect();
let (canonical_blocks_zeroth_epoch, _, _, canonical_state) = rig.add_attested_blocks_at_slots(
state.clone(),
&canonical_slots_zeroth_epoch,
&honest_validators,
);
let (canonical_blocks_zeroth_epoch, _, _, mut canonical_state) = rig
.add_attested_blocks_at_slots(
state.clone(),
state_root,
&canonical_slots_zeroth_epoch,
&honest_validators,
);
let skipped_slot: Slot = rig.epoch_start_slot(1).into();
let stray_slots: Vec<Slot> = ((skipped_slot + 1).into()..rig.epoch_start_slot(2))
.map(Into::into)
.collect();
let canonical_state_root = canonical_state.update_tree_hash_cache().unwrap();
let (stray_blocks, stray_states, _, stray_state) = rig.add_attested_blocks_at_slots(
canonical_state.clone(),
canonical_state_root,
&stray_slots,
&adversarial_validators,
);
@ -1335,8 +1396,13 @@ fn finalizes_non_epoch_start_slot() {
let canonical_slots: Vec<Slot> = ((skipped_slot + 1).into()..rig.epoch_start_slot(7))
.map(Into::into)
.collect();
let (canonical_blocks_post_finalization, _, _, _) =
rig.add_attested_blocks_at_slots(canonical_state, &canonical_slots, &honest_validators);
let canonical_state_root = canonical_state.update_tree_hash_cache().unwrap();
let (canonical_blocks_post_finalization, _, _, _) = rig.add_attested_blocks_at_slots(
canonical_state,
canonical_state_root,
&canonical_slots,
&honest_validators,
);
// Postconditions
let canonical_blocks: HashMap<Slot, SignedBeaconBlockHash> = canonical_blocks_zeroth_epoch
@ -1530,8 +1596,10 @@ fn pruning_test(
let start_slot = Slot::new(1);
let divergence_slot = start_slot + num_initial_blocks;
let (state, state_root) = harness.get_current_state_and_root();
let (_, _, _, divergence_state) = harness.add_attested_blocks_at_slots(
harness.get_current_state(),
state,
state_root,
&slots(start_slot, num_initial_blocks)[..],
&honest_validators,
);
@ -1553,7 +1621,7 @@ fn pruning_test(
faulty_validators,
),
]);
let (_, _, _, canonical_state) = chains.remove(0);
let (_, _, _, mut canonical_state) = chains.remove(0);
let (stray_blocks, stray_states, _, stray_head_state) = chains.remove(0);
let stray_head_slot = divergence_slot + num_fork_skips + num_fork_blocks - 1;
@ -1577,8 +1645,10 @@ fn pruning_test(
// Trigger finalization
let num_finalization_blocks = 4 * E::slots_per_epoch();
let canonical_slot = divergence_slot + num_canonical_skips + num_canonical_middle_blocks;
let canonical_state_root = canonical_state.update_tree_hash_cache().unwrap();
harness.add_attested_blocks_at_slots(
canonical_state,
canonical_state_root,
&slots(canonical_slot, num_finalization_blocks),
&honest_validators,
);

View File

@ -436,23 +436,16 @@ fn attestations_with_increasing_slots() {
AttestationStrategy::SomeValidators(vec![]),
);
attestations.extend(
harness.get_unaggregated_attestations(
&AttestationStrategy::AllValidators,
&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(),
),
);
let head = harness.chain.head().unwrap();
let head_state_root = head.beacon_state_root();
attestations.extend(harness.get_unaggregated_attestations(
&AttestationStrategy::AllValidators,
&head.beacon_state,
head_state_root,
head.beacon_block_root,
head.beacon_block.slot(),
));
harness.advance_slot();
}

View File

@ -0,0 +1,201 @@
//! Contains the handler for the `GET validator/duties/attester/{epoch}` endpoint.
use crate::state_id::StateId;
use beacon_chain::{BeaconChain, BeaconChainError, BeaconChainTypes};
use eth2::types::{self as api_types};
use state_processing::state_advance::partial_state_advance;
use types::{
AttestationDuty, BeaconState, ChainSpec, CloneConfig, Epoch, EthSpec, Hash256, RelativeEpoch,
};
/// The struct that is returned to the requesting HTTP client.
type ApiDuties = api_types::DutiesResponse<Vec<api_types::AttesterData>>;
/// Handles a request from the HTTP API for attester duties.
pub fn attester_duties<T: BeaconChainTypes>(
request_epoch: Epoch,
request_indices: &[u64],
chain: &BeaconChain<T>,
) -> Result<ApiDuties, warp::reject::Rejection> {
let current_epoch = chain
.epoch()
.map_err(warp_utils::reject::beacon_chain_error)?;
let next_epoch = current_epoch + 1;
if request_epoch > next_epoch {
Err(warp_utils::reject::custom_bad_request(format!(
"request epoch {} is more than one epoch past the current epoch {}",
request_epoch, current_epoch
)))
} else if request_epoch == current_epoch || request_epoch == next_epoch {
cached_attestation_duties(request_epoch, request_indices, chain)
} else {
compute_historic_attester_duties(request_epoch, request_indices, chain)
}
}
fn cached_attestation_duties<T: BeaconChainTypes>(
request_epoch: Epoch,
request_indices: &[u64],
chain: &BeaconChain<T>,
) -> Result<ApiDuties, warp::reject::Rejection> {
let head = chain
.head_info()
.map_err(warp_utils::reject::beacon_chain_error)?;
let (duties, dependent_root) = chain
.validator_attestation_duties(&request_indices, request_epoch, head.block_root)
.map_err(warp_utils::reject::beacon_chain_error)?;
convert_to_api_response(duties, request_indices, dependent_root, chain)
}
/// Compute some attester duties by reading a `BeaconState` from disk, completely ignoring the
/// shuffling cache.
fn compute_historic_attester_duties<T: BeaconChainTypes>(
request_epoch: Epoch,
request_indices: &[u64],
chain: &BeaconChain<T>,
) -> Result<ApiDuties, warp::reject::Rejection> {
// If the head is quite old then it might still be relevant for a historical request.
//
// Use the `with_head` function to read & clone in a single call to avoid race conditions.
let state_opt = chain
.with_head(|head| {
if head.beacon_state.current_epoch() <= request_epoch {
Ok(Some((
head.beacon_state_root(),
head.beacon_state
.clone_with(CloneConfig::committee_caches_only()),
)))
} else {
Ok(None)
}
})
.map_err(warp_utils::reject::beacon_chain_error)?;
let mut state = if let Some((state_root, mut state)) = state_opt {
// If we've loaded the head state it might be from a previous epoch, ensure it's in a
// suitable epoch.
ensure_state_knows_attester_duties_for_epoch(
&mut state,
state_root,
request_epoch,
&chain.spec,
)?;
state
} else {
StateId::slot(request_epoch.start_slot(T::EthSpec::slots_per_epoch())).state(&chain)?
};
// Sanity-check the state lookup.
if !(state.current_epoch() == request_epoch || state.current_epoch() + 1 == request_epoch) {
return Err(warp_utils::reject::custom_server_error(format!(
"state epoch {} not suitable for request epoch {}",
state.current_epoch(),
request_epoch
)));
}
let relative_epoch =
RelativeEpoch::from_epoch(state.current_epoch(), request_epoch).map_err(|e| {
warp_utils::reject::custom_server_error(format!("invalid epoch for state: {:?}", e))
})?;
state
.build_committee_cache(relative_epoch, &chain.spec)
.map_err(BeaconChainError::from)
.map_err(warp_utils::reject::beacon_chain_error)?;
let dependent_root = state
// The only block which decides its own shuffling is the genesis block.
.attester_shuffling_decision_root(chain.genesis_block_root, relative_epoch)
.map_err(BeaconChainError::from)
.map_err(warp_utils::reject::beacon_chain_error)?;
let duties = request_indices
.iter()
.map(|&validator_index| {
state
.get_attestation_duties(validator_index as usize, relative_epoch)
.map_err(BeaconChainError::from)
})
.collect::<Result<_, _>>()
.map_err(warp_utils::reject::beacon_chain_error)?;
convert_to_api_response(duties, request_indices, dependent_root, chain)
}
fn ensure_state_knows_attester_duties_for_epoch<E: EthSpec>(
state: &mut BeaconState<E>,
state_root: Hash256,
target_epoch: Epoch,
spec: &ChainSpec,
) -> Result<(), warp::reject::Rejection> {
// Protect against an inconsistent slot clock.
if state.current_epoch() > target_epoch {
return Err(warp_utils::reject::custom_server_error(format!(
"state epoch {} is later than target epoch {}",
state.current_epoch(),
target_epoch
)));
} else if state.current_epoch() + 1 < target_epoch {
// Since there's a one-epoch look-head on attester duties, it suffices to only advance to
// the prior epoch.
let target_slot = target_epoch
.saturating_sub(1_u64)
.start_slot(E::slots_per_epoch());
// A "partial" state advance is adequate since attester duties don't rely on state roots.
partial_state_advance(state, Some(state_root), target_slot, spec)
.map_err(BeaconChainError::from)
.map_err(warp_utils::reject::beacon_chain_error)?;
}
Ok(())
}
/// Convert the internal representation of attester duties into the format returned to the HTTP
/// client.
fn convert_to_api_response<T: BeaconChainTypes>(
duties: Vec<Option<AttestationDuty>>,
indices: &[u64],
dependent_root: Hash256,
chain: &BeaconChain<T>,
) -> Result<ApiDuties, warp::reject::Rejection> {
// Protect against an inconsistent slot clock.
if duties.len() != indices.len() {
return Err(warp_utils::reject::custom_server_error(format!(
"duties length {} does not match indices length {}",
duties.len(),
indices.len()
)));
}
let usize_indices = indices.iter().map(|i| *i as usize).collect::<Vec<_>>();
let index_to_pubkey_map = chain
.validator_pubkey_bytes_many(&usize_indices)
.map_err(warp_utils::reject::beacon_chain_error)?;
let data = duties
.into_iter()
.zip(indices)
.filter_map(|(duty_opt, &validator_index)| {
let duty = duty_opt?;
Some(api_types::AttesterData {
pubkey: *index_to_pubkey_map.get(&(validator_index as usize))?,
validator_index,
committees_at_slot: duty.committees_at_slot,
committee_index: duty.index,
committee_length: duty.committee_len as u64,
validator_committee_index: duty.committee_position as u64,
slot: duty.slot,
})
})
.collect::<Vec<_>>();
Ok(api_types::DutiesResponse {
dependent_root,
data,
})
}

View File

@ -1,186 +0,0 @@
use crate::metrics;
use beacon_chain::{BeaconChain, BeaconChainError, BeaconChainTypes};
use eth2::types::ProposerData;
use fork_choice::ProtoBlock;
use slot_clock::SlotClock;
use state_processing::per_slot_processing;
use types::{BeaconState, Epoch, EthSpec, Hash256, PublicKeyBytes};
/// This sets a maximum bound on the number of epochs to skip whilst instantiating the cache for
/// the first time.
const EPOCHS_TO_SKIP: u64 = 2;
/// Caches the beacon block proposers for a given `epoch` and `epoch_boundary_root`.
///
/// This cache is only able to contain a single set of proposers and is only
/// intended to cache the proposers for the current epoch according to the head
/// of the chain. A change in epoch or re-org to a different chain may cause a
/// cache miss and rebuild.
pub struct BeaconProposerCache {
epoch: Epoch,
decision_block_root: Hash256,
proposers: Vec<ProposerData>,
}
impl BeaconProposerCache {
/// Create a new cache for the current epoch of the `chain`.
pub fn new<T: BeaconChainTypes>(chain: &BeaconChain<T>) -> Result<Self, BeaconChainError> {
let head_root = chain.head_beacon_block_root()?;
let head_block = chain
.fork_choice
.read()
.get_block(&head_root)
.ok_or(BeaconChainError::MissingBeaconBlock(head_root))?;
// If the head epoch is more than `EPOCHS_TO_SKIP` in the future, just build the cache at
// the epoch of the head. This prevents doing a massive amount of skip slots when starting
// a new database from genesis.
let epoch = {
let epoch_now = chain
.epoch()
.unwrap_or_else(|_| chain.spec.genesis_slot.epoch(T::EthSpec::slots_per_epoch()));
let head_epoch = head_block.slot.epoch(T::EthSpec::slots_per_epoch());
if epoch_now > head_epoch + EPOCHS_TO_SKIP {
head_epoch
} else {
epoch_now
}
};
Self::for_head_block(chain, epoch, head_root, head_block)
}
/// Create a new cache that contains the shuffling for `current_epoch`,
/// assuming that `head_root` and `head_block` represents the most recent
/// canonical block.
fn for_head_block<T: BeaconChainTypes>(
chain: &BeaconChain<T>,
current_epoch: Epoch,
head_root: Hash256,
head_block: ProtoBlock,
) -> Result<Self, BeaconChainError> {
let _timer = metrics::start_timer(&metrics::HTTP_API_BEACON_PROPOSER_CACHE_TIMES);
let mut head_state = chain
.get_state(&head_block.state_root, Some(head_block.slot))?
.ok_or(BeaconChainError::MissingBeaconState(head_block.state_root))?;
let decision_block_root = Self::decision_block_root(current_epoch, head_root, &head_state)?;
// We *must* skip forward to the current epoch to obtain valid proposer
// duties. We cannot skip to the previous epoch, like we do with
// attester duties.
while head_state.current_epoch() < current_epoch {
// Skip slots until the current epoch, providing `Hash256::zero()` as the state root
// since we don't require it to be valid to identify producers.
per_slot_processing(&mut head_state, Some(Hash256::zero()), &chain.spec)?;
}
let proposers = current_epoch
.slot_iter(T::EthSpec::slots_per_epoch())
.map(|slot| {
head_state
.get_beacon_proposer_index(slot, &chain.spec)
.map_err(BeaconChainError::from)
.and_then(|i| {
let pubkey = chain
.validator_pubkey(i)?
.ok_or(BeaconChainError::ValidatorPubkeyCacheIncomplete(i))?;
Ok(ProposerData {
pubkey: PublicKeyBytes::from(pubkey),
validator_index: i as u64,
slot,
})
})
})
.collect::<Result<_, _>>()?;
Ok(Self {
epoch: current_epoch,
decision_block_root,
proposers,
})
}
/// Returns a block root which can be used to key the shuffling obtained from the following
/// parameters:
///
/// - `shuffling_epoch`: the epoch for which the shuffling pertains.
/// - `head_block_root`: the block root at the head of the chain.
/// - `head_block_state`: the state of `head_block_root`.
pub fn decision_block_root<E: EthSpec>(
shuffling_epoch: Epoch,
head_block_root: Hash256,
head_block_state: &BeaconState<E>,
) -> Result<Hash256, BeaconChainError> {
let decision_slot = shuffling_epoch
.start_slot(E::slots_per_epoch())
.saturating_sub(1_u64);
// If decision slot is equal to or ahead of the head, the block root is the head block root
if decision_slot >= head_block_state.slot {
Ok(head_block_root)
} else {
head_block_state
.get_block_root(decision_slot)
.map(|root| *root)
.map_err(Into::into)
}
}
/// Return the proposers for the given `Epoch`.
///
/// The cache may be rebuilt if:
///
/// - The epoch has changed since the last cache build.
/// - There has been a re-org that crosses an epoch boundary.
pub fn get_proposers<T: BeaconChainTypes>(
&mut self,
chain: &BeaconChain<T>,
epoch: Epoch,
) -> Result<Vec<ProposerData>, warp::Rejection> {
let current_epoch = chain
.slot_clock
.now_or_genesis()
.ok_or_else(|| {
warp_utils::reject::custom_server_error("unable to read slot clock".to_string())
})?
.epoch(T::EthSpec::slots_per_epoch());
// Disallow requests that are outside the current epoch. This ensures the cache doesn't get
// washed-out with old values.
if current_epoch != epoch {
return Err(warp_utils::reject::custom_bad_request(format!(
"requested epoch is {} but only current epoch {} is allowed",
epoch, current_epoch
)));
}
let (head_block_root, head_decision_block_root) = chain
.with_head(|head| {
Self::decision_block_root(current_epoch, head.beacon_block_root, &head.beacon_state)
.map(|decision_root| (head.beacon_block_root, decision_root))
})
.map_err(warp_utils::reject::beacon_chain_error)?;
let head_block = chain
.fork_choice
.read()
.get_block(&head_block_root)
.ok_or(BeaconChainError::MissingBeaconBlock(head_block_root))
.map_err(warp_utils::reject::beacon_chain_error)?;
// Rebuild the cache if this call causes a cache-miss.
if self.epoch != current_epoch || self.decision_block_root != head_decision_block_root {
metrics::inc_counter(&metrics::HTTP_API_BEACON_PROPOSER_CACHE_MISSES_TOTAL);
*self = Self::for_head_block(chain, current_epoch, head_block_root, head_block)
.map_err(warp_utils::reject::beacon_chain_error)?;
} else {
metrics::inc_counter(&metrics::HTTP_API_BEACON_PROPOSER_CACHE_HITS_TOTAL);
}
Ok(self.proposers.clone())
}
}

View File

@ -5,9 +5,10 @@
//! There are also some additional, non-standard endpoints behind the `/lighthouse/` path which are
//! used for development.
mod beacon_proposer_cache;
mod attester_duties;
mod block_id;
mod metrics;
mod proposer_duties;
mod state_id;
mod validator_inclusion;
@ -17,19 +18,16 @@ use beacon_chain::{
validator_monitor::{get_block_delay_ms, timestamp_now},
AttestationError as AttnError, BeaconChain, BeaconChainError, BeaconChainTypes,
};
use beacon_proposer_cache::BeaconProposerCache;
use block_id::BlockId;
use eth2::types::{self as api_types, ValidatorId};
use eth2_libp2p::{types::SyncState, EnrExt, NetworkGlobals, PeerId, PubsubMessage};
use lighthouse_version::version_with_platform;
use network::NetworkMessage;
use parking_lot::Mutex;
use serde::{Deserialize, Serialize};
use slog::{crit, debug, error, info, warn, Logger};
use slot_clock::SlotClock;
use ssz::Encode;
use state_id::StateId;
use state_processing::per_slot_processing;
use std::borrow::Cow;
use std::convert::TryInto;
use std::future::Future;
@ -38,9 +36,8 @@ use std::sync::Arc;
use tokio::sync::mpsc::UnboundedSender;
use tokio_stream::{wrappers::BroadcastStream, StreamExt};
use types::{
Attestation, AttestationDuty, AttesterSlashing, CloneConfig, CommitteeCache, Epoch, EthSpec,
Hash256, ProposerSlashing, PublicKey, PublicKeyBytes, RelativeEpoch, SignedAggregateAndProof,
SignedBeaconBlock, SignedVoluntaryExit, Slot, YamlConfig,
Attestation, AttesterSlashing, CommitteeCache, Epoch, EthSpec, ProposerSlashing, RelativeEpoch,
SignedAggregateAndProof, SignedBeaconBlock, SignedVoluntaryExit, Slot, YamlConfig,
};
use warp::http::StatusCode;
use warp::sse::Event;
@ -240,30 +237,6 @@ pub fn serve<T: BeaconChainTypes>(
let eth1_v1 = warp::path(API_PREFIX).and(warp::path(API_VERSION));
// Instantiate the beacon proposer cache.
let beacon_proposer_cache = ctx
.chain
.as_ref()
.map(|chain| BeaconProposerCache::new(&chain))
.transpose()
.map_err(|e| format!("Unable to initialize beacon proposer cache: {:?}", e))?
.map(Mutex::new)
.map(Arc::new);
// Create a `warp` filter that provides access to the proposer cache.
let beacon_proposer_cache = || {
warp::any()
.map(move || beacon_proposer_cache.clone())
.and_then(|beacon_proposer_cache| async move {
match beacon_proposer_cache {
Some(cache) => Ok(cache),
None => Err(warp_utils::reject::custom_not_found(
"Beacon proposer cache is not initialized.".to_string(),
)),
}
})
};
// Create a `warp` filter that provides access to the network globals.
let inner_network_globals = ctx.network_globals.clone();
let network_globals = warp::any()
@ -1674,89 +1647,10 @@ pub fn serve<T: BeaconChainTypes>(
.and(warp::path::end())
.and(not_while_syncing_filter.clone())
.and(chain_filter.clone())
.and(beacon_proposer_cache())
.and_then(
|epoch: Epoch,
chain: Arc<BeaconChain<T>>,
beacon_proposer_cache: Arc<Mutex<BeaconProposerCache>>| {
blocking_json_task(move || {
let current_epoch = chain
.epoch()
.map_err(warp_utils::reject::beacon_chain_error)?;
if epoch > current_epoch {
return Err(warp_utils::reject::custom_bad_request(format!(
"request epoch {} is ahead of the current epoch {}",
epoch, current_epoch
)));
}
if epoch == current_epoch {
let dependent_root_slot = current_epoch
.start_slot(T::EthSpec::slots_per_epoch()) - 1;
let dependent_root = if dependent_root_slot > chain.best_slot().map_err(warp_utils::reject::beacon_chain_error)? {
chain.head_beacon_block_root().map_err(warp_utils::reject::beacon_chain_error)?
} else {
chain
.root_at_slot(dependent_root_slot)
.map_err(warp_utils::reject::beacon_chain_error)?
.unwrap_or(chain.genesis_block_root)
};
beacon_proposer_cache
.lock()
.get_proposers(&chain, epoch)
.map(|duties| api_types::DutiesResponse { data: duties, dependent_root })
} else {
let state =
StateId::slot(epoch.start_slot(T::EthSpec::slots_per_epoch()))
.state(&chain)?;
let dependent_root_slot = state.current_epoch()
.start_slot(T::EthSpec::slots_per_epoch()) - 1;
let dependent_root = if dependent_root_slot > chain.best_slot().map_err(warp_utils::reject::beacon_chain_error)? {
chain.head_beacon_block_root().map_err(warp_utils::reject::beacon_chain_error)?
} else {
chain
.root_at_slot(dependent_root_slot)
.map_err(warp_utils::reject::beacon_chain_error)?
.unwrap_or(chain.genesis_block_root)
};
epoch
.slot_iter(T::EthSpec::slots_per_epoch())
.map(|slot| {
state
.get_beacon_proposer_index(slot, &chain.spec)
.map_err(warp_utils::reject::beacon_state_error)
.and_then(|i| {
let pubkey =
chain.validator_pubkey(i)
.map_err(warp_utils::reject::beacon_chain_error)?
.ok_or_else(||
warp_utils::reject::beacon_chain_error(
BeaconChainError::ValidatorPubkeyCacheIncomplete(i)
)
)?;
Ok(api_types::ProposerData {
pubkey: PublicKeyBytes::from(pubkey),
validator_index: i as u64,
slot,
})
})
})
.collect::<Result<Vec<api_types::ProposerData>, _>>()
.map(|duties| {
api_types::DutiesResponse {
dependent_root,
data: duties,
}
})
}
})
},
);
.and(log_filter.clone())
.and_then(|epoch: Epoch, chain: Arc<BeaconChain<T>>, log: Logger| {
blocking_json_task(move || proposer_duties::proposer_duties(epoch, &chain, &log))
});
// GET validator/blocks/{slot}
let get_validator_blocks = eth1_v1
@ -1865,188 +1759,7 @@ pub fn serve<T: BeaconChainTypes>(
.and_then(
|epoch: Epoch, indices: api_types::ValidatorIndexData, chain: Arc<BeaconChain<T>>| {
blocking_json_task(move || {
let current_epoch = chain
.epoch()
.map_err(warp_utils::reject::beacon_chain_error)?;
if epoch > current_epoch + 1 {
return Err(warp_utils::reject::custom_bad_request(format!(
"request epoch {} is more than one epoch past the current epoch {}",
epoch, current_epoch
)));
}
let validator_count = StateId::head()
.map_state(&chain, |state| Ok(state.validators.len() as u64))?;
let pubkeys = indices
.0
.iter()
.filter(|i| **i < validator_count as u64)
.map(|i| {
let pubkey = chain
.validator_pubkey(*i as usize)
.map_err(warp_utils::reject::beacon_chain_error)?
.ok_or_else(|| {
warp_utils::reject::custom_bad_request(format!(
"unknown validator index {}",
*i
))
})?;
Ok((*i, pubkey))
})
.collect::<Result<Vec<_>, warp::Rejection>>()?;
// Converts the internal Lighthouse `AttestationDuty` struct into an
// API-conforming `AttesterData` struct.
let convert = |validator_index: u64,
pubkey: PublicKey,
duty: AttestationDuty|
-> api_types::AttesterData {
api_types::AttesterData {
pubkey: pubkey.into(),
validator_index,
committees_at_slot: duty.committees_at_slot,
committee_index: duty.index,
committee_length: duty.committee_len as u64,
validator_committee_index: duty.committee_position as u64,
slot: duty.slot,
}
};
// Here we have two paths:
//
// ## Fast
//
// If the request epoch is the current epoch, use the cached beacon chain
// method.
//
// ## Slow
//
// If the request epoch is prior to the current epoch, load a beacon state from
// disk
//
// The idea is to stop historical requests from washing out the cache on the
// beacon chain, whilst allowing a VC to request duties quickly.
let (duties, dependent_root) = if epoch == current_epoch {
// Fast path.
let duties = pubkeys
.into_iter()
// Exclude indices which do not represent a known public key and a
// validator duty.
.filter_map(|(i, pubkey)| {
Some(
chain
.validator_attestation_duty(i as usize, epoch)
.transpose()?
.map_err(warp_utils::reject::beacon_chain_error)
.map(|duty| convert(i, pubkey, duty)),
)
})
.collect::<Result<Vec<_>, warp::Rejection>>()?;
let dependent_root_slot =
(epoch - 1).start_slot(T::EthSpec::slots_per_epoch()) - 1;
let dependent_root = if dependent_root_slot
> chain
.best_slot()
.map_err(warp_utils::reject::beacon_chain_error)?
{
chain
.head_beacon_block_root()
.map_err(warp_utils::reject::beacon_chain_error)?
} else {
chain
.root_at_slot(dependent_root_slot)
.map_err(warp_utils::reject::beacon_chain_error)?
.unwrap_or(chain.genesis_block_root)
};
(duties, dependent_root)
} else {
// If the head state is equal to or earlier than the request epoch, use it.
let mut state = chain
.with_head(|head| {
if head.beacon_state.current_epoch() <= epoch {
Ok(Some(
head.beacon_state
.clone_with(CloneConfig::committee_caches_only()),
))
} else {
Ok(None)
}
})
.map_err(warp_utils::reject::beacon_chain_error)?
.map(Result::Ok)
.unwrap_or_else(|| {
StateId::slot(epoch.start_slot(T::EthSpec::slots_per_epoch()))
.state(&chain)
})?;
// Only skip forward to the epoch prior to the request, since we have a
// one-epoch look-ahead on shuffling.
while state
.next_epoch()
.map_err(warp_utils::reject::beacon_state_error)?
< epoch
{
// Don't calculate state roots since they aren't required for calculating
// shuffling (achieved by providing Hash256::zero()).
per_slot_processing(&mut state, Some(Hash256::zero()), &chain.spec)
.map_err(warp_utils::reject::slot_processing_error)?;
}
let relative_epoch =
RelativeEpoch::from_epoch(state.current_epoch(), epoch).map_err(
|e| {
warp_utils::reject::custom_server_error(format!(
"unable to obtain suitable state: {:?}",
e
))
},
)?;
state
.build_committee_cache(relative_epoch, &chain.spec)
.map_err(warp_utils::reject::beacon_state_error)?;
let duties = pubkeys
.into_iter()
.filter_map(|(i, pubkey)| {
Some(
state
.get_attestation_duties(i as usize, relative_epoch)
.transpose()?
.map_err(warp_utils::reject::beacon_state_error)
.map(|duty| convert(i, pubkey, duty)),
)
})
.collect::<Result<Vec<_>, warp::Rejection>>()?;
let dependent_root_slot =
(epoch - 1).start_slot(T::EthSpec::slots_per_epoch()) - 1;
let dependent_root = if dependent_root_slot
> chain
.best_slot()
.map_err(warp_utils::reject::beacon_chain_error)?
{
chain
.head_beacon_block_root()
.map_err(warp_utils::reject::beacon_chain_error)?
} else {
chain
.root_at_slot(dependent_root_slot)
.map_err(warp_utils::reject::beacon_chain_error)?
.unwrap_or(chain.genesis_block_root)
};
(duties, dependent_root)
};
Ok(api_types::DutiesResponse {
dependent_root,
data: duties,
})
attester_duties::attester_duties(epoch, &indices.0, &chain)
})
},
);

View File

@ -0,0 +1,280 @@
//! Contains the handler for the `GET validator/duties/proposer/{epoch}` endpoint.
use crate::state_id::StateId;
use beacon_chain::{BeaconChain, BeaconChainError, BeaconChainTypes};
use eth2::types::{self as api_types};
use slog::{debug, Logger};
use state_processing::state_advance::partial_state_advance;
use std::cmp::Ordering;
use types::{BeaconState, ChainSpec, CloneConfig, Epoch, EthSpec, Hash256, Slot};
/// The struct that is returned to the requesting HTTP client.
type ApiDuties = api_types::DutiesResponse<Vec<api_types::ProposerData>>;
/// Handles a request from the HTTP API for proposer duties.
pub fn proposer_duties<T: BeaconChainTypes>(
request_epoch: Epoch,
chain: &BeaconChain<T>,
log: &Logger,
) -> Result<ApiDuties, warp::reject::Rejection> {
let current_epoch = chain
.epoch()
.map_err(warp_utils::reject::beacon_chain_error)?;
match request_epoch.cmp(&current_epoch) {
// request_epoch > current_epoch
//
// Reject queries about the future as they're very expensive there's no look-ahead for
// proposer duties.
Ordering::Greater => Err(warp_utils::reject::custom_bad_request(format!(
"request epoch {} is ahead of the current epoch {}",
request_epoch, current_epoch
))),
// request_epoch == current_epoch
//
// Queries about the current epoch should attempt to find the value in the cache. If it
// can't be found, it should be computed and then stored in the cache for future gains.
Ordering::Equal => {
if let Some(duties) = try_proposer_duties_from_cache(request_epoch, chain)? {
Ok(duties)
} else {
debug!(
log,
"Proposer cache miss";
"request_epoch" => request_epoch,
);
compute_and_cache_proposer_duties(request_epoch, chain)
}
}
// request_epoch < current_epoch
//
// Queries about the past are handled with a slow path.
Ordering::Less => compute_historic_proposer_duties(request_epoch, chain),
}
}
/// Attempt to load the proposer duties from the `chain.beacon_proposer_cache`, returning `Ok(None)`
/// if there is a cache miss.
///
/// ## Notes
///
/// The `current_epoch` value should equal the current epoch on the slot clock, otherwise we risk
/// washing out the proposer cache at the expense of block processing.
fn try_proposer_duties_from_cache<T: BeaconChainTypes>(
current_epoch: Epoch,
chain: &BeaconChain<T>,
) -> Result<Option<ApiDuties>, warp::reject::Rejection> {
let head = chain
.head_info()
.map_err(warp_utils::reject::beacon_chain_error)?;
let head_epoch = head.slot.epoch(T::EthSpec::slots_per_epoch());
let dependent_root = match head_epoch.cmp(&current_epoch) {
// head_epoch == current_epoch
Ordering::Equal => head.proposer_shuffling_decision_root,
// head_epoch < current_epoch
Ordering::Less => head.block_root,
// head_epoch > current_epoch
Ordering::Greater => {
return Err(warp_utils::reject::custom_server_error(format!(
"head epoch {} is later than current epoch {}",
head_epoch, current_epoch
)))
}
};
chain
.beacon_proposer_cache
.lock()
.get_epoch::<T::EthSpec>(dependent_root, current_epoch)
.cloned()
.map(|indices| {
convert_to_api_response(chain, current_epoch, dependent_root, indices.to_vec())
})
.transpose()
}
/// Compute the proposer duties using the head state, add the duties to the proposer cache and
/// return the proposers.
///
/// This method does *not* attempt to read the values from the cache before computing them. See
/// `try_proposer_duties_from_cache` to read values.
///
/// ## Notes
///
/// The `current_epoch` value should equal the current epoch on the slot clock, otherwise we risk
/// washing out the proposer cache at the expense of block processing.
fn compute_and_cache_proposer_duties<T: BeaconChainTypes>(
current_epoch: Epoch,
chain: &BeaconChain<T>,
) -> Result<ApiDuties, warp::reject::Rejection> {
// Take a copy of the head of the chain.
let head = chain
.head()
.map_err(warp_utils::reject::beacon_chain_error)?;
let mut state = head.beacon_state;
let head_state_root = head.beacon_block.state_root();
// Advance the state into the requested epoch.
ensure_state_is_in_epoch(&mut state, head_state_root, current_epoch, &chain.spec)?;
let indices = state
.get_beacon_proposer_indices(&chain.spec)
.map_err(BeaconChainError::from)
.map_err(warp_utils::reject::beacon_chain_error)?;
let dependent_root = state
// The only block which decides its own shuffling is the genesis block.
.proposer_shuffling_decision_root(chain.genesis_block_root)
.map_err(BeaconChainError::from)
.map_err(warp_utils::reject::beacon_chain_error)?;
// Prime the proposer shuffling cache with the newly-learned value.
chain
.beacon_proposer_cache
.lock()
.insert(
state.current_epoch(),
dependent_root,
indices.clone(),
state.fork,
)
.map_err(BeaconChainError::from)
.map_err(warp_utils::reject::beacon_chain_error)?;
convert_to_api_response(chain, current_epoch, dependent_root, indices)
}
/// Compute some proposer duties by reading a `BeaconState` from disk, completely ignoring the
/// `beacon_proposer_cache`.
fn compute_historic_proposer_duties<T: BeaconChainTypes>(
epoch: Epoch,
chain: &BeaconChain<T>,
) -> Result<ApiDuties, warp::reject::Rejection> {
// If the head is quite old then it might still be relevant for a historical request.
//
// Use the `with_head` function to read & clone in a single call to avoid race conditions.
let state_opt = chain
.with_head(|head| {
if head.beacon_state.current_epoch() <= epoch {
Ok(Some((
head.beacon_state_root(),
head.beacon_state
.clone_with(CloneConfig::committee_caches_only()),
)))
} else {
Ok(None)
}
})
.map_err(warp_utils::reject::beacon_chain_error)?;
let state = if let Some((state_root, mut state)) = state_opt {
// If we've loaded the head state it might be from a previous epoch, ensure it's in a
// suitable epoch.
ensure_state_is_in_epoch(&mut state, state_root, epoch, &chain.spec)?;
state
} else {
StateId::slot(epoch.start_slot(T::EthSpec::slots_per_epoch())).state(&chain)?
};
// Ensure the state lookup was correct.
if state.current_epoch() != epoch {
return Err(warp_utils::reject::custom_server_error(format!(
"state epoch {} not equal to request epoch {}",
state.current_epoch(),
epoch
)));
}
let indices = state
.get_beacon_proposer_indices(&chain.spec)
.map_err(BeaconChainError::from)
.map_err(warp_utils::reject::beacon_chain_error)?;
// We can supply the genesis block root as the block root since we know that the only block that
// decides its own root is the genesis block.
let dependent_root = state
.proposer_shuffling_decision_root(chain.genesis_block_root)
.map_err(BeaconChainError::from)
.map_err(warp_utils::reject::beacon_chain_error)?;
convert_to_api_response(chain, epoch, dependent_root, indices)
}
/// If required, advance `state` to `target_epoch`.
///
/// ## Details
///
/// - Returns an error if `state.current_epoch() > target_epoch`.
/// - No-op if `state.current_epoch() == target_epoch`.
/// - It must be the case that `state.canonical_root() == state_root`, but this function will not
/// check that.
fn ensure_state_is_in_epoch<E: EthSpec>(
state: &mut BeaconState<E>,
state_root: Hash256,
target_epoch: Epoch,
spec: &ChainSpec,
) -> Result<(), warp::reject::Rejection> {
match state.current_epoch().cmp(&target_epoch) {
// Protects against an inconsistent slot clock.
Ordering::Greater => Err(warp_utils::reject::custom_server_error(format!(
"state epoch {} is later than target epoch {}",
state.current_epoch(),
target_epoch
))),
// The state needs to be advanced.
Ordering::Less => {
let target_slot = target_epoch.start_slot(E::slots_per_epoch());
partial_state_advance(state, Some(state_root), target_slot, spec)
.map_err(BeaconChainError::from)
.map_err(warp_utils::reject::beacon_chain_error)
}
// The state is suitable, nothing to do.
Ordering::Equal => Ok(()),
}
}
/// Converts the internal representation of proposer duties into one that is compatible with the
/// standard API.
fn convert_to_api_response<T: BeaconChainTypes>(
chain: &BeaconChain<T>,
epoch: Epoch,
dependent_root: Hash256,
indices: Vec<usize>,
) -> Result<ApiDuties, warp::reject::Rejection> {
let index_to_pubkey_map = chain
.validator_pubkey_bytes_many(&indices)
.map_err(warp_utils::reject::beacon_chain_error)?;
// Map our internal data structure into the API structure.
let proposer_data = indices
.iter()
.enumerate()
.filter_map(|(i, &validator_index)| {
// Offset the index in `indices` to determine the slot for which these
// duties apply.
let slot = epoch.start_slot(T::EthSpec::slots_per_epoch()) + Slot::from(i);
Some(api_types::ProposerData {
pubkey: *index_to_pubkey_map.get(&validator_index)?,
validator_index: validator_index as u64,
slot,
})
})
.collect::<Vec<_>>();
// Consistency check.
let slots_per_epoch = T::EthSpec::slots_per_epoch() as usize;
if proposer_data.len() != slots_per_epoch {
Err(warp_utils::reject::custom_server_error(format!(
"{} proposers is not enough for {} slots",
proposer_data.len(),
slots_per_epoch,
)))
} else {
Ok(api_types::DutiesResponse {
dependent_root,
data: proposer_data,
})
}
}

View File

@ -103,10 +103,12 @@ impl ApiTester {
let (next_block, _next_state) =
harness.make_block(head.beacon_state.clone(), harness.chain.slot().unwrap());
let head_state_root = head.beacon_state_root();
let attestations = harness
.get_unaggregated_attestations(
&AttestationStrategy::AllValidators,
&head.beacon_state,
head_state_root,
head.beacon_block_root,
harness.chain.slot().unwrap(),
)
@ -234,10 +236,12 @@ impl ApiTester {
let (next_block, _next_state) =
harness.make_block(head.beacon_state.clone(), harness.chain.slot().unwrap());
let head_state_root = head.beacon_state_root();
let attestations = harness
.get_unaggregated_attestations(
&AttestationStrategy::AllValidators,
&head.beacon_state,
head_state_root,
head.beacon_block_root,
harness.chain.slot().unwrap(),
)
@ -1563,50 +1567,117 @@ impl ApiTester {
pub async fn test_get_validator_duties_proposer(self) -> Self {
let current_epoch = self.chain.epoch().unwrap();
let dependent_root = self
.chain
.root_at_slot(current_epoch.start_slot(E::slots_per_epoch()) - 1)
.unwrap()
.unwrap_or(self.chain.head_beacon_block_root().unwrap());
for epoch in 0..=self.chain.epoch().unwrap().as_u64() {
let epoch = Epoch::from(epoch);
let result = self
.client
.get_validator_duties_proposer(current_epoch)
.await
.unwrap();
let dependent_root = self
.chain
.root_at_slot(epoch.start_slot(E::slots_per_epoch()) - 1)
.unwrap()
.unwrap_or(self.chain.head_beacon_block_root().unwrap());
let mut state = self.chain.head_beacon_state().unwrap();
// Presently, the beacon chain harness never runs the code that primes the proposer
// cache. If this changes in the future then we'll need some smarter logic here, but
// this is succinct and effective for the time being.
assert!(
self.chain
.beacon_proposer_cache
.lock()
.get_epoch::<E>(dependent_root, epoch)
.is_none(),
"the proposer cache should miss initially"
);
while state.current_epoch() < current_epoch {
per_slot_processing(&mut state, None, &self.chain.spec).unwrap();
let result = self
.client
.get_validator_duties_proposer(epoch)
.await
.unwrap();
// Check that current-epoch requests prime the proposer cache, whilst non-current
// requests don't.
if epoch == current_epoch {
assert!(
self.chain
.beacon_proposer_cache
.lock()
.get_epoch::<E>(dependent_root, epoch)
.is_some(),
"a current-epoch request should prime the proposer cache"
);
} else {
assert!(
self.chain
.beacon_proposer_cache
.lock()
.get_epoch::<E>(dependent_root, epoch)
.is_none(),
"a non-current-epoch request should not prime the proposer cache"
);
}
let mut state = self
.chain
.state_at_slot(
epoch.start_slot(E::slots_per_epoch()),
StateSkipConfig::WithStateRoots,
)
.unwrap();
state
.build_committee_cache(RelativeEpoch::Current, &self.chain.spec)
.unwrap();
let expected_duties = epoch
.slot_iter(E::slots_per_epoch())
.map(|slot| {
let index = state
.get_beacon_proposer_index(slot, &self.chain.spec)
.unwrap();
let pubkey = state.validators[index].pubkey.clone().into();
ProposerData {
pubkey,
validator_index: index as u64,
slot,
}
})
.collect::<Vec<_>>();
let expected = DutiesResponse {
data: expected_duties,
dependent_root,
};
assert_eq!(result, expected);
// If it's the current epoch, check the function with a primed proposer cache.
if epoch == current_epoch {
// This is technically a double-check, but it's defensive.
assert!(
self.chain
.beacon_proposer_cache
.lock()
.get_epoch::<E>(dependent_root, epoch)
.is_some(),
"the request should prime the proposer cache"
);
let result = self
.client
.get_validator_duties_proposer(epoch)
.await
.unwrap();
assert_eq!(result, expected);
}
}
state
.build_committee_cache(RelativeEpoch::Current, &self.chain.spec)
.unwrap();
let expected_duties = current_epoch
.slot_iter(E::slots_per_epoch())
.map(|slot| {
let index = state
.get_beacon_proposer_index(slot, &self.chain.spec)
.unwrap();
let pubkey = state.validators[index].pubkey.clone().into();
ProposerData {
pubkey,
validator_index: index as u64,
slot,
}
})
.collect::<Vec<_>>();
let expected = DutiesResponse {
data: expected_duties,
dependent_root,
};
assert_eq!(result, expected);
// Requests to future epochs should fail.
self.client
.get_validator_duties_proposer(current_epoch + 1)
.await
.unwrap_err();
self
}

View File

@ -92,15 +92,16 @@ impl TestRig {
let (next_block, _next_state) =
harness.make_block(head.beacon_state.clone(), harness.chain.slot().unwrap());
let head_state_root = head.beacon_state_root();
let attestations = harness
.get_unaggregated_attestations(
&AttestationStrategy::AllValidators,
&head.beacon_state,
head_state_root,
head.beacon_block_root,
harness.chain.slot().unwrap(),
)
.into_iter()
// .map(|vec| vec.into_iter().map(|(attestation, _subnet_id)| attestation))
.flatten()
.collect::<Vec<_>>();

View File

@ -333,7 +333,11 @@ impl<E: EthSpec, Hot: ItemStore<E>, Cold: ItemStore<E>> HotColdDB<E, Hot, Cold>
/// ## Warning
///
/// The returned state **is not a valid beacon state**, it can only be used for obtaining
/// shuffling to process attestations.
/// shuffling to process attestations. At least the following components of the state will be
/// broken/invalid:
///
/// - `state.state_roots`
/// - `state.block_roots`
pub fn get_inconsistent_state_for_attestation_verification_only(
&self,
state_root: &Hash256,

View File

@ -183,8 +183,13 @@ impl ForkChoiceTest {
break;
}
if let Ok(block_hash) = self.harness.process_block_result(block.clone()) {
self.harness
.attest_block(&state, block_hash, &block, &validators);
self.harness.attest_block(
&state,
block.state_root(),
block_hash,
&block,
&validators,
);
self.harness.advance_slot();
} else {
return Err(self);

View File

@ -9,6 +9,7 @@ pub mod genesis;
pub mod per_block_processing;
pub mod per_epoch_processing;
pub mod per_slot_processing;
pub mod state_advance;
pub mod test_utils;
pub mod verify_operation;

View File

@ -0,0 +1,105 @@
//! This module contains functions for advancing a `BeaconState` forward some number of slots
//! without blocks (i.e., skip slots).
//!
//! These functions are not in the specification, however they're defined here to reduce code
//! duplication and protect against some easy-to-make mistakes when performing state advances.
use crate::*;
use types::{BeaconState, ChainSpec, EthSpec, Hash256, Slot};
#[derive(Debug, PartialEq)]
pub enum Error {
BadTargetSlot { target_slot: Slot, state_slot: Slot },
PerSlotProcessing(per_slot_processing::Error),
StateRootNotProvided,
}
/// Advances the `state` to the given `target_slot`, assuming that there were no blocks between
/// these slots.
///
/// ## Errors
///
/// - If `state.slot > target_slot`, an error will be returned.
///
/// ## Notes
///
/// This state advance method is "complete"; it outputs a perfectly valid `BeaconState` and doesn't
/// do anything hacky like the "partial" method (see `partial_state_advance`).
pub fn complete_state_advance<T: EthSpec>(
state: &mut BeaconState<T>,
mut state_root_opt: Option<Hash256>,
target_slot: Slot,
spec: &ChainSpec,
) -> Result<(), Error> {
check_target_slot(state.slot, target_slot)?;
while state.slot < target_slot {
// Use the initial state root on the first iteration of the loop, then use `None` for any
// future iterations.
let state_root_opt = state_root_opt.take();
per_slot_processing(state, state_root_opt, spec).map_err(Error::PerSlotProcessing)?;
}
Ok(())
}
/// Advances the `state` to the given `target_slot`, assuming that there were no blocks between
/// these slots.
///
/// This is a "partial" state advance which outputs an **invalid** `BeaconState`. The state is
/// invalid because the intermediate state roots are not computed. Avoiding computing state roots
/// saves *a lot* of compute time and can be a useful optimization when a state only needs to be
/// advanced to obtain proposer/attester shuffling as they are indifferent to state roots.
///
/// For clarity, **be careful with this function as it produces invalid states**.
///
/// ## Errors
///
/// - If `state.slot > target_slot`, an error will be returned.
/// - If `state_root_opt.is_none()` but the latest block header requires a state root.
pub fn partial_state_advance<T: EthSpec>(
state: &mut BeaconState<T>,
state_root_opt: Option<Hash256>,
target_slot: Slot,
spec: &ChainSpec,
) -> Result<(), Error> {
check_target_slot(state.slot, target_slot)?;
// The only time that a state root is mandatory is if a block has been applied to the state
// without it yet being advanced another slot.
//
// Failing to provide a state root in this scenario would result in corrupting the
// `state.block_roots` array, since the `state.latest_block_header` would contain an invalid
// (all-zeros) state root.
let mut initial_state_root = Some(if state.slot > state.latest_block_header.slot {
state_root_opt.unwrap_or_else(Hash256::zero)
} else {
state_root_opt.ok_or(Error::StateRootNotProvided)?
});
while state.slot < target_slot {
// Use the initial state root on the first iteration of the loop, then use `[0; 32]` for any
// later iterations.
//
// Failing to provide the correct state root on the initial iteration may result in
// corrupting the `state.block_roots` array since the latest block header may not be updated
// with the correct state root.
let state_root = initial_state_root.take().unwrap_or_else(Hash256::zero);
per_slot_processing(state, Some(state_root), spec).map_err(Error::PerSlotProcessing)?;
}
Ok(())
}
fn check_target_slot(state_slot: Slot, target_slot: Slot) -> Result<(), Error> {
if state_slot > target_slot {
Err(Error::BadTargetSlot {
target_slot,
state_slot,
})
} else {
Ok(())
}
}

View File

@ -444,6 +444,62 @@ impl<T: EthSpec> BeaconState<T> {
cache.get_all_beacon_committees()
}
/// Returns the block root which decided the proposer shuffling for the current epoch. This root
/// can be used to key this proposer shuffling.
///
/// ## Notes
///
/// The `block_root` covers the one-off scenario where the genesis block decides its own
/// shuffling. It should be set to the latest block applied to `self` or the genesis block root.
pub fn proposer_shuffling_decision_root(&self, block_root: Hash256) -> Result<Hash256, Error> {
let decision_slot = self.proposer_shuffling_decision_slot();
if self.slot == decision_slot {
Ok(block_root)
} else {
self.get_block_root(decision_slot).map(|root| *root)
}
}
/// Returns the slot at which the proposer shuffling was decided. The block root at this slot
/// can be used to key the proposer shuffling for the current epoch.
fn proposer_shuffling_decision_slot(&self) -> Slot {
self.current_epoch()
.start_slot(T::slots_per_epoch())
.saturating_sub(1_u64)
}
/// Returns the block root which decided the attester shuffling for the given `relative_epoch`.
/// This root can be used to key that attester shuffling.
///
/// ## Notes
///
/// The `block_root` covers the one-off scenario where the genesis block decides its own
/// shuffling. It should be set to the latest block applied to `self` or the genesis block root.
pub fn attester_shuffling_decision_root(
&self,
block_root: Hash256,
relative_epoch: RelativeEpoch,
) -> Result<Hash256, Error> {
let decision_slot = self.attester_shuffling_decision_slot(relative_epoch);
if self.slot == decision_slot {
Ok(block_root)
} else {
self.get_block_root(decision_slot).map(|root| *root)
}
}
/// Returns the slot at which the proposer shuffling was decided. The block root at this slot
/// can be used to key the proposer shuffling for the current epoch.
fn attester_shuffling_decision_slot(&self, relative_epoch: RelativeEpoch) -> Slot {
match relative_epoch {
RelativeEpoch::Next => self.current_epoch(),
RelativeEpoch::Current => self.previous_epoch(),
RelativeEpoch::Previous => self.previous_epoch().saturating_sub(1_u64),
}
.start_slot(T::slots_per_epoch())
.saturating_sub(1_u64)
}
/// Compute the proposer (not necessarily for the Beacon chain) from a list of indices.
///
/// Spec v0.12.1

View File

@ -35,16 +35,8 @@ impl AttestationShufflingId {
) -> Result<Self, BeaconStateError> {
let shuffling_epoch = relative_epoch.into_epoch(state.current_epoch());
let shuffling_decision_slot = shuffling_epoch
.saturating_sub(1_u64)
.start_slot(E::slots_per_epoch())
.saturating_sub(1_u64);
let shuffling_decision_block = if state.slot == shuffling_decision_slot {
block_root
} else {
*state.get_block_root(shuffling_decision_slot)?
};
let shuffling_decision_block =
state.attester_shuffling_decision_root(block_root, relative_epoch)?;
Ok(Self {
shuffling_epoch,

View File

@ -1,3 +1,4 @@
use crate::generic_public_key_bytes::GenericPublicKeyBytes;
use crate::Error;
use serde::de::{Deserialize, Deserializer};
use serde::ser::{Serialize, Serializer};
@ -54,6 +55,11 @@ where
format!("{:?}", self)
}
/// Returns `self` in the compressed `PublicKeyBytes` representation.
pub fn compress(&self) -> GenericPublicKeyBytes<Pub> {
GenericPublicKeyBytes::from(self)
}
/// Serialize `self` as compressed bytes.
pub fn serialize(&self) -> [u8; PUBLIC_KEY_BYTES_LEN] {
self.point.serialize()

View File

@ -69,6 +69,11 @@ impl<Pub> GenericPublicKeyBytes<Pub> {
self.bytes
}
/// Returns `self.serialize()` as a `0x`-prefixed hex string.
pub fn to_hex_string(&self) -> String {
format!("{:?}", self)
}
/// Instantiates `Self` from bytes.
///
/// The bytes are not fully verified (i.e., they may not represent a valid BLS point). Only the

View File

@ -599,6 +599,8 @@ fn check_slashing_protection(validator_dir: &TempDir, pubkeys: impl Iterator<Ite
let slashing_db_path = validator_dir.path().join(SLASHING_PROTECTION_FILENAME);
let slashing_db = SlashingDatabase::open(&slashing_db_path).unwrap();
for validator_pk in pubkeys {
slashing_db.get_validator_id(&validator_pk).unwrap();
slashing_db
.get_validator_id(&validator_pk.compress())
.unwrap();
}
}

View File

@ -21,6 +21,7 @@ eth2_interop_keypairs = { path = "../common/eth2_interop_keypairs" }
slashing_protection = { path = "./slashing_protection" }
slot_clock = { path = "../common/slot_clock" }
types = { path = "../consensus/types" }
safe_arith = { path = "../consensus/safe_arith" }
serde = "1.0.116"
serde_derive = "1.0.116"
bincode = "1.3.1"

View File

@ -1,6 +1,6 @@
use serde_derive::{Deserialize, Serialize};
use std::collections::HashSet;
use types::{Epoch, Hash256, PublicKey, Slot};
use types::{Epoch, Hash256, PublicKeyBytes, Slot};
#[derive(Debug, Clone, PartialEq, Deserialize, Serialize)]
#[serde(deny_unknown_fields)]
@ -13,7 +13,7 @@ pub struct InterchangeMetadata {
#[derive(Debug, Clone, PartialEq, Eq, Hash, Deserialize, Serialize)]
#[serde(deny_unknown_fields)]
pub struct InterchangeData {
pub pubkey: PublicKey,
pub pubkey: PublicKeyBytes,
pub signed_blocks: Vec<SignedBlock>,
pub signed_attestations: Vec<SignedAttestation>,
}

View File

@ -5,7 +5,7 @@ use crate::{
};
use serde_derive::{Deserialize, Serialize};
use tempfile::tempdir;
use types::{Epoch, Hash256, PublicKey, Slot};
use types::{Epoch, Hash256, PublicKeyBytes, Slot};
#[derive(Debug, Clone, Deserialize, Serialize)]
pub struct MultiTestCase {
@ -25,7 +25,7 @@ pub struct TestCase {
#[derive(Debug, Clone, Deserialize, Serialize)]
pub struct TestBlock {
pub pubkey: PublicKey,
pub pubkey: PublicKeyBytes,
pub slot: Slot,
pub signing_root: Hash256,
pub should_succeed: bool,
@ -33,7 +33,7 @@ pub struct TestBlock {
#[derive(Debug, Clone, Deserialize, Serialize)]
pub struct TestAttestation {
pub pubkey: PublicKey,
pub pubkey: PublicKeyBytes,
pub source_epoch: Epoch,
pub target_epoch: Epoch,
pub signing_root: Hash256,

View File

@ -17,7 +17,7 @@ pub use crate::slashing_database::{
use rusqlite::Error as SQLError;
use std::io::{Error as IOError, ErrorKind};
use std::string::ToString;
use types::{Hash256, PublicKey};
use types::{Hash256, PublicKeyBytes};
/// The filename within the `validators` directory that contains the slashing protection DB.
pub const SLASHING_PROTECTION_FILENAME: &str = "slashing_protection.sqlite";
@ -27,7 +27,7 @@ pub const SLASHING_PROTECTION_FILENAME: &str = "slashing_protection.sqlite";
/// This could be because it's slashable, or because an error occurred.
#[derive(PartialEq, Debug)]
pub enum NotSafe {
UnregisteredValidator(PublicKey),
UnregisteredValidator(PublicKeyBytes),
InvalidBlock(InvalidBlock),
InvalidAttestation(InvalidAttestation),
IOError(ErrorKind),

View File

@ -16,7 +16,7 @@ fn block_same_slot() {
let pk = pubkey(0);
slashing_db.register_validator(&pk).unwrap();
slashing_db.register_validator(pk).unwrap();
// A stream of blocks all with the same slot.
let num_blocks = 10;
@ -37,7 +37,7 @@ fn attestation_same_target() {
let pk = pubkey(0);
slashing_db.register_validator(&pk).unwrap();
slashing_db.register_validator(pk).unwrap();
// A stream of attestations all with the same target.
let num_attestations = 10;
@ -64,7 +64,7 @@ fn attestation_surround_fest() {
let pk = pubkey(0);
slashing_db.register_validator(&pk).unwrap();
slashing_db.register_validator(pk).unwrap();
// A stream of attestations that all surround each other.
let num_attestations = 10;

View File

@ -10,7 +10,7 @@ use rusqlite::{params, OptionalExtension, Transaction, TransactionBehavior};
use std::fs::{File, OpenOptions};
use std::path::Path;
use std::time::Duration;
use types::{AttestationData, BeaconBlockHeader, Epoch, Hash256, PublicKey, SignedRoot, Slot};
use types::{AttestationData, BeaconBlockHeader, Epoch, Hash256, PublicKeyBytes, SignedRoot, Slot};
type Pool = r2d2::Pool<SqliteConnectionManager>;
@ -147,14 +147,14 @@ impl SlashingDatabase {
///
/// This allows the validator to record their signatures in the database, and check
/// for slashings.
pub fn register_validator(&self, validator_pk: &PublicKey) -> Result<(), NotSafe> {
self.register_validators(std::iter::once(validator_pk))
pub fn register_validator(&self, validator_pk: PublicKeyBytes) -> Result<(), NotSafe> {
self.register_validators(std::iter::once(&validator_pk))
}
/// Register multiple validators with the slashing protection database.
pub fn register_validators<'a>(
&self,
public_keys: impl Iterator<Item = &'a PublicKey>,
public_keys: impl Iterator<Item = &'a PublicKeyBytes>,
) -> Result<(), NotSafe> {
let mut conn = self.conn_pool.get()?;
let txn = conn.transaction()?;
@ -168,7 +168,7 @@ impl SlashingDatabase {
/// The caller must commit the transaction for the changes to be persisted.
pub fn register_validators_in_txn<'a>(
&self,
public_keys: impl Iterator<Item = &'a PublicKey>,
public_keys: impl Iterator<Item = &'a PublicKeyBytes>,
txn: &Transaction,
) -> Result<(), NotSafe> {
let mut stmt = txn.prepare("INSERT INTO validators (public_key) VALUES (?1)")?;
@ -183,7 +183,7 @@ impl SlashingDatabase {
/// Check that all of the given validators are registered.
pub fn check_validator_registrations<'a>(
&self,
mut public_keys: impl Iterator<Item = &'a PublicKey>,
mut public_keys: impl Iterator<Item = &'a PublicKeyBytes>,
) -> Result<(), NotSafe> {
let mut conn = self.conn_pool.get()?;
let txn = conn.transaction()?;
@ -195,7 +195,7 @@ impl SlashingDatabase {
///
/// This is NOT the same as a validator index, and depends on the ordering that validators
/// are registered with the slashing protection database (and may vary between machines).
pub fn get_validator_id(&self, public_key: &PublicKey) -> Result<i64, NotSafe> {
pub fn get_validator_id(&self, public_key: &PublicKeyBytes) -> Result<i64, NotSafe> {
let mut conn = self.conn_pool.get()?;
let txn = conn.transaction()?;
self.get_validator_id_in_txn(&txn, public_key)
@ -204,17 +204,17 @@ impl SlashingDatabase {
fn get_validator_id_in_txn(
&self,
txn: &Transaction,
public_key: &PublicKey,
public_key: &PublicKeyBytes,
) -> Result<i64, NotSafe> {
self.get_validator_id_opt(txn, public_key)?
.ok_or_else(|| NotSafe::UnregisteredValidator(public_key.clone()))
.ok_or_else(|| NotSafe::UnregisteredValidator(*public_key))
}
/// Optional version of `get_validator_id`.
fn get_validator_id_opt(
&self,
txn: &Transaction,
public_key: &PublicKey,
public_key: &PublicKeyBytes,
) -> Result<Option<i64>, NotSafe> {
Ok(txn
.query_row(
@ -229,7 +229,7 @@ impl SlashingDatabase {
fn check_block_proposal(
&self,
txn: &Transaction,
validator_pubkey: &PublicKey,
validator_pubkey: &PublicKeyBytes,
slot: Slot,
signing_root: SigningRoot,
) -> Result<Safe, NotSafe> {
@ -278,7 +278,7 @@ impl SlashingDatabase {
fn check_attestation(
&self,
txn: &Transaction,
validator_pubkey: &PublicKey,
validator_pubkey: &PublicKeyBytes,
att_source_epoch: Epoch,
att_target_epoch: Epoch,
att_signing_root: SigningRoot,
@ -408,7 +408,7 @@ impl SlashingDatabase {
fn insert_block_proposal(
&self,
txn: &Transaction,
validator_pubkey: &PublicKey,
validator_pubkey: &PublicKeyBytes,
slot: Slot,
signing_root: SigningRoot,
) -> Result<(), NotSafe> {
@ -429,7 +429,7 @@ impl SlashingDatabase {
fn insert_attestation(
&self,
txn: &Transaction,
validator_pubkey: &PublicKey,
validator_pubkey: &PublicKeyBytes,
att_source_epoch: Epoch,
att_target_epoch: Epoch,
att_signing_root: SigningRoot,
@ -457,7 +457,7 @@ impl SlashingDatabase {
/// This is the safe, externally-callable interface for checking block proposals.
pub fn check_and_insert_block_proposal(
&self,
validator_pubkey: &PublicKey,
validator_pubkey: &PublicKeyBytes,
block_header: &BeaconBlockHeader,
domain: Hash256,
) -> Result<Safe, NotSafe> {
@ -471,7 +471,7 @@ impl SlashingDatabase {
/// As for `check_and_insert_block_proposal` but without requiring the whole `BeaconBlockHeader`.
pub fn check_and_insert_block_signing_root(
&self,
validator_pubkey: &PublicKey,
validator_pubkey: &PublicKeyBytes,
slot: Slot,
signing_root: SigningRoot,
) -> Result<Safe, NotSafe> {
@ -490,7 +490,7 @@ impl SlashingDatabase {
/// Transactional variant of `check_and_insert_block_signing_root`.
pub fn check_and_insert_block_signing_root_txn(
&self,
validator_pubkey: &PublicKey,
validator_pubkey: &PublicKeyBytes,
slot: Slot,
signing_root: SigningRoot,
txn: &Transaction,
@ -511,7 +511,7 @@ impl SlashingDatabase {
/// This is the safe, externally-callable interface for checking attestations.
pub fn check_and_insert_attestation(
&self,
validator_pubkey: &PublicKey,
validator_pubkey: &PublicKeyBytes,
attestation: &AttestationData,
domain: Hash256,
) -> Result<Safe, NotSafe> {
@ -527,7 +527,7 @@ impl SlashingDatabase {
/// As for `check_and_insert_attestation` but without requiring the whole `AttestationData`.
pub fn check_and_insert_attestation_signing_root(
&self,
validator_pubkey: &PublicKey,
validator_pubkey: &PublicKeyBytes,
att_source_epoch: Epoch,
att_target_epoch: Epoch,
att_signing_root: SigningRoot,
@ -548,7 +548,7 @@ impl SlashingDatabase {
/// Transactional variant of `check_and_insert_attestation_signing_root`.
fn check_and_insert_attestation_signing_root_txn(
&self,
validator_pubkey: &PublicKey,
validator_pubkey: &PublicKeyBytes,
att_source_epoch: Epoch,
att_target_epoch: Epoch,
att_signing_root: SigningRoot,
@ -600,7 +600,7 @@ impl SlashingDatabase {
let mut import_outcomes = vec![];
for record in interchange.data {
let pubkey = record.pubkey.clone();
let pubkey = record.pubkey;
let txn = conn.transaction()?;
match self.import_interchange_record(record, &txn) {
Ok(summary) => {
@ -757,7 +757,7 @@ impl SlashingDatabase {
/// Remove all blocks for `public_key` with slots less than `new_min_slot`.
fn prune_signed_blocks(
&self,
public_key: &PublicKey,
public_key: &PublicKeyBytes,
new_min_slot: Slot,
txn: &Transaction,
) -> Result<(), NotSafe> {
@ -780,7 +780,7 @@ impl SlashingDatabase {
/// Prune the signed blocks table for the given public keys.
pub fn prune_all_signed_blocks<'a>(
&self,
mut public_keys: impl Iterator<Item = &'a PublicKey>,
mut public_keys: impl Iterator<Item = &'a PublicKeyBytes>,
new_min_slot: Slot,
) -> Result<(), NotSafe> {
let mut conn = self.conn_pool.get()?;
@ -803,7 +803,7 @@ impl SlashingDatabase {
/// attestations in the database.
fn prune_signed_attestations(
&self,
public_key: &PublicKey,
public_key: &PublicKeyBytes,
new_min_target: Epoch,
txn: &Transaction,
) -> Result<(), NotSafe> {
@ -830,7 +830,7 @@ impl SlashingDatabase {
/// Prune the signed attestations table for the given validator keys.
pub fn prune_all_signed_attestations<'a>(
&self,
mut public_keys: impl Iterator<Item = &'a PublicKey>,
mut public_keys: impl Iterator<Item = &'a PublicKeyBytes>,
new_min_target: Epoch,
) -> Result<(), NotSafe> {
let mut conn = self.conn_pool.get()?;
@ -853,7 +853,7 @@ impl SlashingDatabase {
/// Get a summary of a validator's slashing protection data for consumption by the user.
pub fn validator_summary(
&self,
public_key: &PublicKey,
public_key: &PublicKeyBytes,
txn: &Transaction,
) -> Result<ValidatorSummary, NotSafe> {
let validator_id = self.get_validator_id_in_txn(txn, public_key)?;
@ -906,11 +906,11 @@ pub struct ValidatorSummary {
#[derive(Debug)]
pub enum InterchangeImportOutcome {
Success {
pubkey: PublicKey,
pubkey: PublicKeyBytes,
summary: ValidatorSummary,
},
Failure {
pubkey: PublicKey,
pubkey: PublicKeyBytes,
error: NotSafe,
},
}
@ -981,7 +981,7 @@ mod tests {
let _db1 = SlashingDatabase::create(&file).unwrap();
let db2 = SlashingDatabase::open(&file).unwrap();
db2.register_validator(&pubkey(0)).unwrap_err();
db2.register_validator(pubkey(0)).unwrap_err();
}
// Attempting to create the same database twice should error.

View File

@ -2,18 +2,19 @@ use crate::*;
use tempfile::{tempdir, TempDir};
use types::{
test_utils::generate_deterministic_keypair, AttestationData, BeaconBlockHeader, Hash256,
PublicKeyBytes,
};
pub const DEFAULT_VALIDATOR_INDEX: usize = 0;
pub const DEFAULT_DOMAIN: Hash256 = Hash256::zero();
pub const DEFAULT_GENESIS_VALIDATORS_ROOT: Hash256 = Hash256::zero();
pub fn pubkey(index: usize) -> PublicKey {
generate_deterministic_keypair(index).pk
pub fn pubkey(index: usize) -> PublicKeyBytes {
generate_deterministic_keypair(index).pk.compress()
}
pub struct Test<T> {
pubkey: PublicKey,
pubkey: PublicKeyBytes,
data: T,
domain: Hash256,
expected: Result<Safe, NotSafe>,
@ -24,7 +25,7 @@ impl<T> Test<T> {
Self::with_pubkey(pubkey(DEFAULT_VALIDATOR_INDEX), data)
}
pub fn with_pubkey(pubkey: PublicKey, data: T) -> Self {
pub fn with_pubkey(pubkey: PublicKeyBytes, data: T) -> Self {
Self {
pubkey,
data,
@ -58,7 +59,7 @@ impl<T> Test<T> {
pub struct StreamTest<T> {
/// Validators to register.
pub registered_validators: Vec<PublicKey>,
pub registered_validators: Vec<PublicKeyBytes>,
/// Vector of cases and the value expected when calling `check_and_insert_X`.
pub cases: Vec<Test<T>>,
}
@ -89,7 +90,7 @@ impl StreamTest<AttestationData> {
let slashing_db = SlashingDatabase::create(&slashing_db_file).unwrap();
for pubkey in &self.registered_validators {
slashing_db.register_validator(pubkey).unwrap();
slashing_db.register_validator(*pubkey).unwrap();
}
for (i, test) in self.cases.iter().enumerate() {
@ -112,7 +113,7 @@ impl StreamTest<BeaconBlockHeader> {
let slashing_db = SlashingDatabase::create(&slashing_db_file).unwrap();
for pubkey in &self.registered_validators {
slashing_db.register_validator(pubkey).unwrap();
slashing_db.register_validator(*pubkey).unwrap();
}
for (i, test) in self.cases.iter().enumerate() {

View File

@ -11,7 +11,7 @@ use slot_clock::SlotClock;
use std::collections::HashMap;
use std::ops::Deref;
use std::sync::Arc;
use tokio::time::{interval_at, sleep_until, Duration, Instant};
use tokio::time::{sleep, sleep_until, Duration, Instant};
use tree_hash::TreeHash;
use types::{
AggregateSignature, Attestation, AttestationData, BitList, ChainSpec, CommitteeIndex, EthSpec,
@ -20,7 +20,7 @@ use types::{
/// Builds an `AttestationService`.
pub struct AttestationServiceBuilder<T, E: EthSpec> {
duties_service: Option<DutiesService<T, E>>,
duties_service: Option<Arc<DutiesService<T, E>>>,
validator_store: Option<ValidatorStore<T, E>>,
slot_clock: Option<T>,
beacon_nodes: Option<Arc<BeaconNodeFallback<T, E>>>,
@ -38,7 +38,7 @@ impl<T: SlotClock + 'static, E: EthSpec> AttestationServiceBuilder<T, E> {
}
}
pub fn duties_service(mut self, service: DutiesService<T, E>) -> Self {
pub fn duties_service(mut self, service: Arc<DutiesService<T, E>>) -> Self {
self.duties_service = Some(service);
self
}
@ -88,7 +88,7 @@ impl<T: SlotClock + 'static, E: EthSpec> AttestationServiceBuilder<T, E> {
/// Helper to minimise `Arc` usage.
pub struct Inner<T, E: EthSpec> {
duties_service: DutiesService<T, E>,
duties_service: Arc<DutiesService<T, E>>,
validator_store: ValidatorStore<T, E>,
slot_clock: T,
beacon_nodes: Arc<BeaconNodeFallback<T, E>>,
@ -137,32 +137,31 @@ impl<T: SlotClock + 'static, E: EthSpec> AttestationService<T, E> {
"next_update_millis" => duration_to_next_slot.as_millis()
);
let mut interval = {
// Note: `interval_at` panics if `slot_duration` is 0
interval_at(
Instant::now() + duration_to_next_slot + slot_duration / 3,
slot_duration,
)
};
let executor = self.context.executor.clone();
let interval_fut = async move {
loop {
interval.tick().await;
let log = self.context.log();
if let Some(duration_to_next_slot) = self.slot_clock.duration_to_next_slot() {
sleep(duration_to_next_slot + slot_duration / 3).await;
let log = self.context.log();
if let Err(e) = self.spawn_attestation_tasks(slot_duration) {
crit!(
log,
"Failed to spawn attestation tasks";
"error" => e
)
if let Err(e) = self.spawn_attestation_tasks(slot_duration) {
crit!(
log,
"Failed to spawn attestation tasks";
"error" => e
)
} else {
trace!(
log,
"Spawned attestation tasks";
)
}
} else {
trace!(
log,
"Spawned attestation tasks";
)
error!(log, "Failed to read slot clock");
// If we can't read the slot clock, just wait another slot.
sleep(slot_duration).await;
continue;
}
}
};
@ -192,12 +191,9 @@ impl<T: SlotClock + 'static, E: EthSpec> AttestationService<T, E> {
.attesters(slot)
.into_iter()
.fold(HashMap::new(), |mut map, duty_and_proof| {
if let Some(committee_index) = duty_and_proof.duty.attestation_committee_index {
let validator_duties = map.entry(committee_index).or_insert_with(Vec::new);
validator_duties.push(duty_and_proof);
}
map.entry(duty_and_proof.duty.committee_index)
.or_insert_with(Vec::new)
.push(duty_and_proof);
map
});
@ -355,43 +351,27 @@ impl<T: SlotClock + 'static, E: EthSpec> AttestationService<T, E> {
let mut attestations = Vec::with_capacity(validator_duties.len());
for duty in validator_duties {
// Ensure that all required fields are present in the validator duty.
let (
duty_slot,
duty_committee_index,
validator_committee_position,
_,
_,
committee_length,
) = if let Some(tuple) = duty.attestation_duties() {
tuple
} else {
crit!(
log,
"Missing validator duties when signing";
"duties" => format!("{:?}", duty)
);
continue;
};
for duty_and_proof in validator_duties {
let duty = &duty_and_proof.duty;
// Ensure that the attestation matches the duties.
if duty_slot != attestation_data.slot || duty_committee_index != attestation_data.index
#[allow(clippy::suspicious_operation_groupings)]
if duty.slot != attestation_data.slot || duty.committee_index != attestation_data.index
{
crit!(
log,
"Inconsistent validator duties during signing";
"validator" => format!("{:?}", duty.validator_pubkey()),
"duty_slot" => duty_slot,
"validator" => ?duty.pubkey,
"duty_slot" => duty.slot,
"attestation_slot" => attestation_data.slot,
"duty_index" => duty_committee_index,
"duty_index" => duty.committee_index,
"attestation_index" => attestation_data.index,
);
continue;
}
let mut attestation = Attestation {
aggregation_bits: BitList::with_capacity(committee_length as usize).unwrap(),
aggregation_bits: BitList::with_capacity(duty.committee_length as usize).unwrap(),
data: attestation_data.clone(),
signature: AggregateSignature::infinity(),
};
@ -399,8 +379,8 @@ impl<T: SlotClock + 'static, E: EthSpec> AttestationService<T, E> {
if self
.validator_store
.sign_attestation(
duty.validator_pubkey(),
validator_committee_position,
&duty.pubkey,
duty.validator_committee_index as usize,
&mut attestation,
current_epoch,
)
@ -490,6 +470,8 @@ impl<T: SlotClock + 'static, E: EthSpec> AttestationService<T, E> {
let mut signed_aggregate_and_proofs = Vec::new();
for duty_and_proof in validator_duties {
let duty = &duty_and_proof.duty;
let selection_proof = if let Some(proof) = duty_and_proof.selection_proof.as_ref() {
proof
} else {
@ -497,26 +479,18 @@ impl<T: SlotClock + 'static, E: EthSpec> AttestationService<T, E> {
// subscribed aggregators.
continue;
};
let (duty_slot, duty_committee_index, _, validator_index, _, _) =
if let Some(tuple) = duty_and_proof.attestation_duties() {
tuple
} else {
crit!(log, "Missing duties when signing aggregate");
continue;
};
let pubkey = &duty_and_proof.duty.validator_pubkey;
let slot = attestation_data.slot;
let committee_index = attestation_data.index;
if duty_slot != slot || duty_committee_index != committee_index {
if duty.slot != slot || duty.committee_index != committee_index {
crit!(log, "Inconsistent validator duties during signing");
continue;
}
if let Some(aggregate) = self.validator_store.produce_signed_aggregate_and_proof(
pubkey,
validator_index,
&duty.pubkey,
duty.validator_index,
aggregated_attestation.clone(),
selection_proof.clone(),
) {

View File

@ -5,13 +5,13 @@ use crate::{
use crate::{http_metrics::metrics, validator_store::ValidatorStore};
use environment::RuntimeContext;
use eth2::types::Graffiti;
use futures::channel::mpsc::Receiver;
use futures::{StreamExt, TryFutureExt};
use futures::TryFutureExt;
use slog::{crit, debug, error, info, trace, warn};
use slot_clock::SlotClock;
use std::ops::Deref;
use std::sync::Arc;
use types::{EthSpec, PublicKey, Slot};
use tokio::sync::mpsc;
use types::{EthSpec, PublicKeyBytes, Slot};
/// Builds a `BlockService`.
pub struct BlockServiceBuilder<T, E: EthSpec> {
@ -121,13 +121,13 @@ impl<T, E: EthSpec> Deref for BlockService<T, E> {
/// Notification from the duties service that we should try to produce a block.
pub struct BlockServiceNotification {
pub slot: Slot,
pub block_proposers: Vec<PublicKey>,
pub block_proposers: Vec<PublicKeyBytes>,
}
impl<T: SlotClock + 'static, E: EthSpec> BlockService<T, E> {
pub fn start_update_service(
self,
notification_rx: Receiver<BlockServiceNotification>,
mut notification_rx: mpsc::Receiver<BlockServiceNotification>,
) -> Result<(), String> {
let log = self.context.log().clone();
@ -135,14 +135,16 @@ impl<T: SlotClock + 'static, E: EthSpec> BlockService<T, E> {
let executor = self.inner.context.executor.clone();
let block_service_fut = notification_rx.for_each(move |notif| {
let service = self.clone();
executor.spawn(
async move {
service.do_update(notif).await.ok();
}
});
executor.spawn(block_service_fut, "block_service");
while let Some(notif) = notification_rx.recv().await {
let service = self.clone();
service.do_update(notif).await.ok();
}
debug!(log, "Block service shutting down");
},
"block_service",
);
Ok(())
}
@ -222,7 +224,11 @@ impl<T: SlotClock + 'static, E: EthSpec> BlockService<T, E> {
}
/// Produce a block at the given slot for validator_pubkey
async fn publish_block(self, slot: Slot, validator_pubkey: PublicKey) -> Result<(), String> {
async fn publish_block(
self,
slot: Slot,
validator_pubkey: PublicKeyBytes,
) -> Result<(), String> {
let log = self.context.log();
let _timer =
metrics::start_timer_vec(&metrics::BLOCK_SERVICE_TIMES, &[metrics::BEACON_BLOCK]);

File diff suppressed because it is too large Load Diff

View File

@ -4,12 +4,12 @@ use environment::RuntimeContext;
use eth2::types::StateId;
use futures::future::FutureExt;
use parking_lot::RwLock;
use slog::Logger;
use slog::{debug, trace};
use slog::{error, Logger};
use slot_clock::SlotClock;
use std::ops::Deref;
use std::sync::Arc;
use tokio::time::{interval_at, Duration, Instant};
use tokio::time::{sleep, Duration};
use types::{EthSpec, Fork};
/// Delay this period of time after the slot starts. This allows the node to process the new slot.
@ -139,33 +139,31 @@ impl<T: SlotClock + 'static, E: EthSpec> ForkService<T, E> {
/// Starts the service that periodically polls for the `Fork`.
pub fn start_update_service(self, context: &RuntimeContext<E>) -> Result<(), String> {
let spec = &context.eth2_config.spec;
let duration_to_next_epoch = self
.slot_clock
.duration_to_next_epoch(E::slots_per_epoch())
.ok_or("Unable to determine duration to next epoch")?;
let mut interval = {
let slot_duration = Duration::from_secs(spec.seconds_per_slot);
// Note: interval_at panics if `slot_duration * E::slots_per_epoch()` = 0
interval_at(
Instant::now() + duration_to_next_epoch + TIME_DELAY_FROM_SLOT,
slot_duration * E::slots_per_epoch() as u32,
)
};
// Run an immediate update before starting the updater service.
context
.executor
.spawn(self.clone().do_update().map(|_| ()), "fork service update");
let executor = context.executor.clone();
let log = context.log().clone();
let spec = E::default_spec();
let interval_fut = async move {
loop {
interval.tick().await;
// Run this poll before the wait, this should hopefully download the fork before the
// other services need them.
self.clone().do_update().await.ok();
if let Some(duration_to_next_epoch) =
self.slot_clock.duration_to_next_epoch(E::slots_per_epoch())
{
sleep(duration_to_next_epoch + TIME_DELAY_FROM_SLOT).await;
} else {
error!(log, "Failed to read slot clock");
// If we can't read the slot clock, just wait another slot.
sleep(Duration::from_secs(spec.seconds_per_slot)).await;
continue;
}
}
};

View File

@ -5,7 +5,7 @@ use std::io::{prelude::*, BufReader};
use std::path::PathBuf;
use std::str::FromStr;
use bls::blst_implementations::PublicKey;
use bls::blst_implementations::PublicKeyBytes;
use types::{graffiti::GraffitiString, Graffiti};
#[derive(Debug)]
@ -26,7 +26,7 @@ pub enum Error {
#[derive(Debug, Clone, Serialize, Deserialize)]
pub struct GraffitiFile {
graffiti_path: PathBuf,
graffitis: HashMap<PublicKey, Graffiti>,
graffitis: HashMap<PublicKeyBytes, Graffiti>,
default: Option<Graffiti>,
}
@ -44,7 +44,10 @@ impl GraffitiFile {
/// default graffiti.
///
/// Returns an error if loading from the graffiti file fails.
pub fn load_graffiti(&mut self, public_key: &PublicKey) -> Result<Option<Graffiti>, Error> {
pub fn load_graffiti(
&mut self,
public_key: &PublicKeyBytes,
) -> Result<Option<Graffiti>, Error> {
self.read_graffiti_file()?;
Ok(self.graffitis.get(public_key).copied().or(self.default))
}
@ -78,7 +81,7 @@ impl GraffitiFile {
/// `Ok((None, graffiti))` represents the graffiti for the default key.
/// `Ok((Some(pk), graffiti))` represents graffiti for the public key `pk`.
/// Returns an error if the line is in the wrong format or does not contain a valid public key or graffiti.
fn read_line(line: &str) -> Result<(Option<PublicKey>, Graffiti), Error> {
fn read_line(line: &str) -> Result<(Option<PublicKeyBytes>, Graffiti), Error> {
if let Some(i) = line.find(':') {
let (key, value) = line.split_at(i);
// Note: `value.len() >=1` so `value[1..]` is safe
@ -88,7 +91,7 @@ fn read_line(line: &str) -> Result<(Option<PublicKey>, Graffiti), Error> {
if key == "default" {
Ok((None, graffiti))
} else {
let pk = PublicKey::from_str(&key).map_err(Error::InvalidPublicKey)?;
let pk = PublicKeyBytes::from_str(&key).map_err(Error::InvalidPublicKey)?;
Ok((Some(pk), graffiti))
}
} else {
@ -114,9 +117,9 @@ mod tests {
// Create a graffiti file in the required format and return a path to the file.
fn create_graffiti_file() -> PathBuf {
let temp = TempDir::new().unwrap();
let pk1 = PublicKey::deserialize(&hex::decode(&PK1[2..]).unwrap()).unwrap();
let pk2 = PublicKey::deserialize(&hex::decode(&PK2[2..]).unwrap()).unwrap();
let pk3 = PublicKey::deserialize(&hex::decode(&PK3[2..]).unwrap()).unwrap();
let pk1 = PublicKeyBytes::deserialize(&hex::decode(&PK1[2..]).unwrap()).unwrap();
let pk2 = PublicKeyBytes::deserialize(&hex::decode(&PK2[2..]).unwrap()).unwrap();
let pk3 = PublicKeyBytes::deserialize(&hex::decode(&PK3[2..]).unwrap()).unwrap();
let file_name = temp.into_path().join("graffiti.txt");
@ -143,9 +146,9 @@ mod tests {
let graffiti_file_path = create_graffiti_file();
let mut gf = GraffitiFile::new(graffiti_file_path);
let pk1 = PublicKey::deserialize(&hex::decode(&PK1[2..]).unwrap()).unwrap();
let pk2 = PublicKey::deserialize(&hex::decode(&PK2[2..]).unwrap()).unwrap();
let pk3 = PublicKey::deserialize(&hex::decode(&PK3[2..]).unwrap()).unwrap();
let pk1 = PublicKeyBytes::deserialize(&hex::decode(&PK1[2..]).unwrap()).unwrap();
let pk2 = PublicKeyBytes::deserialize(&hex::decode(&PK2[2..]).unwrap()).unwrap();
let pk3 = PublicKeyBytes::deserialize(&hex::decode(&PK3[2..]).unwrap()).unwrap();
// Read once
gf.read_graffiti_file().unwrap();
@ -165,7 +168,7 @@ mod tests {
);
// Random pk should return the default graffiti
let random_pk = Keypair::random().pk;
let random_pk = Keypair::random().pk.compress();
assert_eq!(
gf.load_graffiti(&random_pk).unwrap().unwrap(),
GraffitiString::from_str(DEFAULT_GRAFFITI).unwrap().into()

View File

@ -13,6 +13,13 @@ pub const ATTESTATIONS: &str = "attestations";
pub const AGGREGATES: &str = "aggregates";
pub const CURRENT_EPOCH: &str = "current_epoch";
pub const NEXT_EPOCH: &str = "next_epoch";
pub const UPDATE_INDICES: &str = "update_indices";
pub const UPDATE_ATTESTERS_CURRENT_EPOCH: &str = "update_attesters_current_epoch";
pub const UPDATE_ATTESTERS_NEXT_EPOCH: &str = "update_attesters_next_epoch";
pub const UPDATE_ATTESTERS_FETCH: &str = "update_attesters_fetch";
pub const UPDATE_ATTESTERS_STORE: &str = "update_attesters_store";
pub const UPDATE_PROPOSERS: &str = "update_proposers";
pub const SUBSCRIPTIONS: &str = "subscriptions";
pub use lighthouse_metrics::*;
@ -84,6 +91,10 @@ lazy_static::lazy_static! {
"Number of attesters on this host",
&["task"]
);
pub static ref PROPOSAL_CHANGED: Result<IntCounter> = try_create_int_counter(
"vc_beacon_block_proposal_changed",
"A duties update discovered a new block proposer for the current slot",
);
/*
* Endpoint metrics
*/

View File

@ -36,7 +36,7 @@ impl From<String> for Error {
/// Contains objects which have shared access from inside/outside of the metrics server.
pub struct Shared<T: EthSpec> {
pub validator_store: Option<ValidatorStore<SystemTimeSlotClock, T>>,
pub duties_service: Option<DutiesService<SystemTimeSlotClock, T>>,
pub duties_service: Option<Arc<DutiesService<SystemTimeSlotClock, T>>>,
pub genesis_time: Option<u64>,
}

View File

@ -20,7 +20,7 @@ use std::collections::{HashMap, HashSet};
use std::fs::File;
use std::io;
use std::path::PathBuf;
use types::{Graffiti, Keypair, PublicKey};
use types::{Graffiti, Keypair, PublicKey, PublicKeyBytes};
use crate::key_cache;
use crate::key_cache::KeyCache;
@ -284,7 +284,7 @@ pub struct InitializedValidators {
/// The directory that the `self.definitions` will be saved into.
validators_dir: PathBuf,
/// The canonical set of validators.
validators: HashMap<PublicKey, InitializedValidator>,
validators: HashMap<PublicKeyBytes, InitializedValidator>,
/// For logging via `slog`.
log: Logger,
}
@ -317,13 +317,13 @@ impl InitializedValidators {
}
/// Iterate through all **enabled** voting public keys in `self`.
pub fn iter_voting_pubkeys(&self) -> impl Iterator<Item = &PublicKey> {
pub fn iter_voting_pubkeys(&self) -> impl Iterator<Item = &PublicKeyBytes> {
self.validators.iter().map(|(pubkey, _)| pubkey)
}
/// Returns the voting `Keypair` for a given voting `PublicKey`, if that validator is known to
/// `self` **and** the validator is enabled.
pub fn voting_keypair(&self, voting_public_key: &PublicKey) -> Option<&Keypair> {
pub fn voting_keypair(&self, voting_public_key: &PublicKeyBytes) -> Option<&Keypair> {
self.validators
.get(voting_public_key)
.map(|v| v.voting_keypair())
@ -366,7 +366,7 @@ impl InitializedValidators {
}
/// Returns the `graffiti` for a given public key specified in the `ValidatorDefinitions`.
pub fn graffiti(&self, public_key: &PublicKey) -> Option<Graffiti> {
pub fn graffiti(&self, public_key: &PublicKeyBytes) -> Option<Graffiti> {
self.validators.get(public_key).and_then(|v| v.graffiti)
}
@ -513,7 +513,9 @@ impl InitializedValidators {
voting_keystore_path,
..
} => {
if self.validators.contains_key(&def.voting_public_key) {
let pubkey_bytes = def.voting_public_key.compress();
if self.validators.contains_key(&pubkey_bytes) {
continue;
}
@ -536,7 +538,7 @@ impl InitializedValidators {
.map(|l| l.path().to_owned());
self.validators
.insert(init.voting_public_key().clone(), init);
.insert(init.voting_public_key().compress(), init);
info!(
self.log,
"Enabled validator";
@ -569,7 +571,7 @@ impl InitializedValidators {
}
}
} else {
self.validators.remove(&def.voting_public_key);
self.validators.remove(&def.voting_public_key.compress());
match &def.signing_definition {
SigningDefinition::LocalKeystore {
voting_keystore_path,

View File

@ -11,7 +11,6 @@ mod http_metrics;
mod initialized_validators;
mod key_cache;
mod notifier;
mod validator_duty;
mod validator_store;
pub mod http_api;
@ -26,12 +25,11 @@ use account_utils::validator_definitions::ValidatorDefinitions;
use attestation_service::{AttestationService, AttestationServiceBuilder};
use block_service::{BlockService, BlockServiceBuilder};
use clap::ArgMatches;
use duties_service::{DutiesService, DutiesServiceBuilder};
use duties_service::DutiesService;
use environment::RuntimeContext;
use eth2::types::StateId;
use eth2::{reqwest::ClientBuilder, BeaconNodeHttpClient, StatusCode, Url};
use fork_service::{ForkService, ForkServiceBuilder};
use futures::channel::mpsc;
use http_api::ApiSecret;
use initialized_validators::InitializedValidators;
use notifier::spawn_notifier;
@ -44,7 +42,10 @@ use std::marker::PhantomData;
use std::net::SocketAddr;
use std::sync::Arc;
use std::time::{SystemTime, UNIX_EPOCH};
use tokio::time::{sleep, Duration};
use tokio::{
sync::mpsc,
time::{sleep, Duration},
};
use types::{EthSpec, Fork, Hash256};
use validator_store::ValidatorStore;
@ -60,7 +61,7 @@ const HTTP_TIMEOUT: Duration = Duration::from_secs(12);
#[derive(Clone)]
pub struct ProductionValidatorClient<T: EthSpec> {
context: RuntimeContext<T>,
duties_service: DutiesService<SystemTimeSlotClock, T>,
duties_service: Arc<DutiesService<SystemTimeSlotClock, T>>,
fork_service: ForkService<SystemTimeSlotClock, T>,
block_service: BlockService<SystemTimeSlotClock, T>,
attestation_service: AttestationService<SystemTimeSlotClock, T>,
@ -285,13 +286,22 @@ impl<T: EthSpec> ProductionValidatorClient<T> {
validator_store.prune_slashing_protection_db(slot.epoch(T::slots_per_epoch()), true);
}
let duties_service = DutiesServiceBuilder::new()
.slot_clock(slot_clock.clone())
.validator_store(validator_store.clone())
.beacon_nodes(beacon_nodes.clone())
.allow_unsynced_beacon_node(config.allow_unsynced_beacon_node)
.runtime_context(context.service_context("duties".into()))
.build()?;
let duties_context = context.service_context("duties".into());
let duties_service = Arc::new(DutiesService {
attesters: <_>::default(),
proposers: <_>::default(),
indices: <_>::default(),
slot_clock: slot_clock.clone(),
beacon_nodes: beacon_nodes.clone(),
validator_store: validator_store.clone(),
require_synced: if config.allow_unsynced_beacon_node {
RequireSynced::Yes
} else {
RequireSynced::No
},
spec: context.eth2_config.spec.clone(),
context: duties_context,
});
// Update the metrics server.
if let Some(ctx) = &http_metrics_ctx {
@ -343,13 +353,7 @@ impl<T: EthSpec> ProductionValidatorClient<T> {
let (block_service_tx, block_service_rx) = mpsc::channel(channel_capacity);
let log = self.context.log();
self.duties_service
.clone()
.start_update_service(
block_service_tx,
Arc::new(self.context.eth2_config.spec.clone()),
)
.map_err(|e| format!("Unable to start duties service: {}", e))?;
duties_service::start_update_service(self.duties_service.clone(), block_service_tx);
self.fork_service
.clone()

View File

@ -1,7 +1,7 @@
use crate::ProductionValidatorClient;
use slog::{error, info};
use crate::{DutiesService, ProductionValidatorClient};
use slog::{error, info, Logger};
use slot_clock::SlotClock;
use tokio::time::{interval_at, Duration, Instant};
use tokio::time::{sleep, Duration};
use types::EthSpec;
/// Spawns a notifier service which periodically logs information about the node.
@ -11,86 +11,19 @@ pub fn spawn_notifier<T: EthSpec>(client: &ProductionValidatorClient<T>) -> Resu
let duties_service = client.duties_service.clone();
let slot_duration = Duration::from_secs(context.eth2_config.spec.seconds_per_slot);
let duration_to_next_slot = duties_service
.slot_clock
.duration_to_next_slot()
.ok_or("slot_notifier unable to determine time to next slot")?;
// Run the notifier half way through each slot.
let start_instant = Instant::now() + duration_to_next_slot + (slot_duration / 2);
let mut interval = interval_at(start_instant, slot_duration);
let interval_fut = async move {
let log = context.log();
loop {
interval.tick().await;
let num_available = duties_service.beacon_nodes.num_available().await;
let num_synced = duties_service.beacon_nodes.num_synced().await;
let num_total = duties_service.beacon_nodes.num_total().await;
if num_synced > 0 {
info!(
log,
"Connected to beacon node(s)";
"total" => num_total,
"available" => num_available,
"synced" => num_synced,
)
if let Some(duration_to_next_slot) = duties_service.slot_clock.duration_to_next_slot() {
sleep(duration_to_next_slot + slot_duration / 2).await;
notify(&duties_service, &log).await;
} else {
error!(
log,
"No synced beacon nodes";
"total" => num_total,
"available" => num_available,
"synced" => num_synced,
)
}
if let Some(slot) = duties_service.slot_clock.now() {
let epoch = slot.epoch(T::slots_per_epoch());
let total_validators = duties_service.total_validator_count();
let proposing_validators = duties_service.proposer_count(epoch);
let attesting_validators = duties_service.attester_count(epoch);
if total_validators == 0 {
info!(
log,
"No validators present";
"msg" => "see `lighthouse account validator create --help` \
or the HTTP API documentation"
)
} else if total_validators == attesting_validators {
info!(
log,
"All validators active";
"proposers" => proposing_validators,
"active_validators" => attesting_validators,
"total_validators" => total_validators,
"epoch" => format!("{}", epoch),
"slot" => format!("{}", slot),
);
} else if attesting_validators > 0 {
info!(
log,
"Some validators active";
"proposers" => proposing_validators,
"active_validators" => attesting_validators,
"total_validators" => total_validators,
"epoch" => format!("{}", epoch),
"slot" => format!("{}", slot),
);
} else {
info!(
log,
"Awaiting activation";
"validators" => total_validators,
"epoch" => format!("{}", epoch),
"slot" => format!("{}", slot),
);
}
} else {
error!(log, "Unable to read slot clock");
error!(log, "Failed to read slot clock");
// If we can't read the slot clock, just wait another slot.
sleep(slot_duration).await;
continue;
}
}
};
@ -98,3 +31,77 @@ pub fn spawn_notifier<T: EthSpec>(client: &ProductionValidatorClient<T>) -> Resu
executor.spawn(interval_fut, "validator_notifier");
Ok(())
}
/// Performs a single notification routine.
async fn notify<T: SlotClock + 'static, E: EthSpec>(
duties_service: &DutiesService<T, E>,
log: &Logger,
) {
let num_available = duties_service.beacon_nodes.num_available().await;
let num_synced = duties_service.beacon_nodes.num_synced().await;
let num_total = duties_service.beacon_nodes.num_total().await;
if num_synced > 0 {
info!(
log,
"Connected to beacon node(s)";
"total" => num_total,
"available" => num_available,
"synced" => num_synced,
)
} else {
error!(
log,
"No synced beacon nodes";
"total" => num_total,
"available" => num_available,
"synced" => num_synced,
)
}
if let Some(slot) = duties_service.slot_clock.now() {
let epoch = slot.epoch(E::slots_per_epoch());
let total_validators = duties_service.total_validator_count();
let proposing_validators = duties_service.proposer_count(epoch);
let attesting_validators = duties_service.attester_count(epoch);
if total_validators == 0 {
info!(
log,
"No validators present";
"msg" => "see `lighthouse account validator create --help` \
or the HTTP API documentation"
)
} else if total_validators == attesting_validators {
info!(
log,
"All validators active";
"proposers" => proposing_validators,
"active_validators" => attesting_validators,
"total_validators" => total_validators,
"epoch" => format!("{}", epoch),
"slot" => format!("{}", slot),
);
} else if attesting_validators > 0 {
info!(
log,
"Some validators active";
"proposers" => proposing_validators,
"active_validators" => attesting_validators,
"total_validators" => total_validators,
"epoch" => format!("{}", epoch),
"slot" => format!("{}", slot),
);
} else {
info!(
log,
"Awaiting activation";
"validators" => total_validators,
"epoch" => format!("{}", epoch),
"slot" => format!("{}", slot),
);
}
} else {
error!(log, "Unable to read slot clock");
}
}

View File

@ -1,188 +0,0 @@
use eth2::{
types::{BeaconCommitteeSubscription, StateId, ValidatorId},
BeaconNodeHttpClient,
};
use serde::{Deserialize, Serialize};
use slog::{error, Logger};
use std::collections::HashMap;
use types::{CommitteeIndex, Epoch, PublicKey, PublicKeyBytes, Slot};
/// This struct is being used as a shim since we deprecated the `rest_api` in favour of `http_api`.
///
/// Tracking issue: https://github.com/sigp/lighthouse/issues/1643
// NOTE: if you add or remove fields, please adjust `eq_ignoring_proposal_slots`
#[derive(PartialEq, Debug, Serialize, Deserialize, Clone)]
pub struct ValidatorDuty {
/// The validator's BLS public key, uniquely identifying them.
pub validator_pubkey: PublicKey,
/// The validator's index in `state.validators`
pub validator_index: Option<u64>,
/// The slot at which the validator must attest.
pub attestation_slot: Option<Slot>,
/// The index of the committee within `slot` of which the validator is a member.
pub attestation_committee_index: Option<CommitteeIndex>,
/// The position of the validator in the committee.
pub attestation_committee_position: Option<usize>,
/// The committee count at `attestation_slot`.
pub committee_count_at_slot: Option<u64>,
/// The number of validators in the committee.
pub committee_length: Option<u64>,
/// The slots in which a validator must propose a block (can be empty).
///
/// Should be set to `None` when duties are not yet known (before the current epoch).
pub block_proposal_slots: Option<Vec<Slot>>,
}
impl ValidatorDuty {
/// Instantiate `Self` as if there are no known dutes for `validator_pubkey`.
fn no_duties(validator_pubkey: PublicKey, validator_index: Option<u64>) -> Self {
ValidatorDuty {
validator_pubkey,
validator_index,
attestation_slot: None,
attestation_committee_index: None,
attestation_committee_position: None,
committee_count_at_slot: None,
committee_length: None,
block_proposal_slots: None,
}
}
/// Instantiate `Self` by performing requests on the `beacon_node`.
///
/// Will only request proposer duties if `current_epoch == request_epoch`.
pub async fn download(
beacon_node: &BeaconNodeHttpClient,
current_epoch: Epoch,
request_epoch: Epoch,
mut pubkeys: Vec<(PublicKey, Option<u64>)>,
log: &Logger,
) -> Result<Vec<ValidatorDuty>, String> {
for (pubkey, index_opt) in &mut pubkeys {
if index_opt.is_none() {
*index_opt = beacon_node
.get_beacon_states_validator_id(
StateId::Head,
&ValidatorId::PublicKey(PublicKeyBytes::from(&*pubkey)),
)
.await
.map_err(|e| {
error!(
log,
"Failed to obtain validator index";
"pubkey" => ?pubkey,
"error" => ?e
)
})
// Supress the error since we've already logged an error and we don't want to
// stop the rest of the code.
.ok()
.and_then(|body_opt| body_opt.map(|body| body.data.index));
}
}
// Query for all block proposer duties in the current epoch and map the response by index.
let proposal_slots_by_index: HashMap<u64, Vec<Slot>> = if current_epoch == request_epoch {
beacon_node
.get_validator_duties_proposer(current_epoch)
.await
.map(|resp| resp.data)
// Exit early if there's an error.
.map_err(|e| format!("Failed to get proposer indices: {:?}", e))?
.into_iter()
.fold(
HashMap::with_capacity(pubkeys.len()),
|mut map, proposer_data| {
map.entry(proposer_data.validator_index)
.or_insert_with(Vec::new)
.push(proposer_data.slot);
map
},
)
} else {
HashMap::new()
};
let query_indices = pubkeys
.iter()
.filter_map(|(_, index_opt)| *index_opt)
.collect::<Vec<_>>();
let attester_data_map = beacon_node
.post_validator_duties_attester(request_epoch, query_indices.as_slice())
.await
.map(|resp| resp.data)
// Exit early if there's an error.
.map_err(|e| format!("Failed to get attester duties: {:?}", e))?
.into_iter()
.fold(
HashMap::with_capacity(pubkeys.len()),
|mut map, attester_data| {
map.insert(attester_data.validator_index, attester_data);
map
},
);
let duties = pubkeys
.into_iter()
.map(|(pubkey, index_opt)| {
if let Some(index) = index_opt {
if let Some(attester_data) = attester_data_map.get(&index) {
match attester_data.pubkey.decompress() {
Ok(pubkey) => ValidatorDuty {
validator_pubkey: pubkey,
validator_index: Some(attester_data.validator_index),
attestation_slot: Some(attester_data.slot),
attestation_committee_index: Some(attester_data.committee_index),
attestation_committee_position: Some(
attester_data.validator_committee_index as usize,
),
committee_count_at_slot: Some(attester_data.committees_at_slot),
committee_length: Some(attester_data.committee_length),
block_proposal_slots: proposal_slots_by_index
.get(&attester_data.validator_index)
.cloned(),
},
Err(e) => {
error!(
log,
"Could not deserialize validator public key";
"error" => format!("{:?}", e),
"validator_index" => attester_data.validator_index
);
Self::no_duties(pubkey, Some(index))
}
}
} else {
Self::no_duties(pubkey, Some(index))
}
} else {
Self::no_duties(pubkey, None)
}
})
.collect();
Ok(duties)
}
/// Return `true` if these validator duties are equal, ignoring their `block_proposal_slots`.
pub fn eq_ignoring_proposal_slots(&self, other: &Self) -> bool {
self.validator_pubkey == other.validator_pubkey
&& self.validator_index == other.validator_index
&& self.attestation_slot == other.attestation_slot
&& self.attestation_committee_index == other.attestation_committee_index
&& self.attestation_committee_position == other.attestation_committee_position
&& self.committee_count_at_slot == other.committee_count_at_slot
&& self.committee_length == other.committee_length
}
/// Generate a subscription for `self`, if `self` has appropriate attestation duties.
pub fn subscription(&self, is_aggregator: bool) -> Option<BeaconCommitteeSubscription> {
Some(BeaconCommitteeSubscription {
validator_index: self.validator_index?,
committee_index: self.attestation_committee_index?,
committees_at_slot: self.committee_count_at_slot?,
slot: self.attestation_slot?,
is_aggregator,
})
}
}

View File

@ -11,7 +11,7 @@ use std::sync::Arc;
use tempfile::TempDir;
use types::{
graffiti::GraffitiString, Attestation, BeaconBlock, ChainSpec, Domain, Epoch, EthSpec, Fork,
Graffiti, Hash256, Keypair, PublicKey, SelectionProof, Signature, SignedAggregateAndProof,
Graffiti, Hash256, Keypair, PublicKeyBytes, SelectionProof, Signature, SignedAggregateAndProof,
SignedBeaconBlock, SignedRoot, Slot,
};
use validator_dir::ValidatorDir;
@ -106,7 +106,7 @@ impl<T: SlotClock + 'static, E: EthSpec> ValidatorStore<T, E> {
.map_err(|e| format!("failed to create validator definitions: {:?}", e))?;
self.slashing_protection
.register_validator(&validator_def.voting_public_key)
.register_validator(validator_def.voting_public_key.compress())
.map_err(|e| format!("failed to register validator: {:?}", e))?;
validator_def.enabled = enable;
@ -120,7 +120,7 @@ impl<T: SlotClock + 'static, E: EthSpec> ValidatorStore<T, E> {
Ok(validator_def)
}
pub fn voting_pubkeys(&self) -> Vec<PublicKey> {
pub fn voting_pubkeys(&self) -> Vec<PublicKeyBytes> {
self.validators
.read()
.iter_voting_pubkeys()
@ -136,7 +136,11 @@ impl<T: SlotClock + 'static, E: EthSpec> ValidatorStore<T, E> {
self.fork_service.fork()
}
pub fn randao_reveal(&self, validator_pubkey: &PublicKey, epoch: Epoch) -> Option<Signature> {
pub fn randao_reveal(
&self,
validator_pubkey: &PublicKeyBytes,
epoch: Epoch,
) -> Option<Signature> {
self.validators
.read()
.voting_keypair(validator_pubkey)
@ -153,13 +157,13 @@ impl<T: SlotClock + 'static, E: EthSpec> ValidatorStore<T, E> {
})
}
pub fn graffiti(&self, validator_pubkey: &PublicKey) -> Option<Graffiti> {
pub fn graffiti(&self, validator_pubkey: &PublicKeyBytes) -> Option<Graffiti> {
self.validators.read().graffiti(validator_pubkey)
}
pub fn sign_block(
&self,
validator_pubkey: &PublicKey,
validator_pubkey: &PublicKeyBytes,
block: BeaconBlock<E>,
current_slot: Slot,
) -> Option<SignedBeaconBlock<E>> {
@ -236,7 +240,7 @@ impl<T: SlotClock + 'static, E: EthSpec> ValidatorStore<T, E> {
pub fn sign_attestation(
&self,
validator_pubkey: &PublicKey,
validator_pubkey: &PublicKeyBytes,
validator_committee_position: usize,
attestation: &mut Attestation<E>,
current_epoch: Epoch,
@ -334,7 +338,7 @@ impl<T: SlotClock + 'static, E: EthSpec> ValidatorStore<T, E> {
/// modified by actors other than the signing validator.
pub fn produce_signed_aggregate_and_proof(
&self,
validator_pubkey: &PublicKey,
validator_pubkey: &PublicKeyBytes,
validator_index: u64,
aggregate: Attestation<E>,
selection_proof: SelectionProof,
@ -359,7 +363,7 @@ impl<T: SlotClock + 'static, E: EthSpec> ValidatorStore<T, E> {
/// `validator_pubkey`.
pub fn produce_selection_proof(
&self,
validator_pubkey: &PublicKey,
validator_pubkey: &PublicKeyBytes,
slot: Slot,
) -> Option<SelectionProof> {
let validators = self.validators.read();