Use async code when interacting with EL (#3244)

## Overview

This rather extensive PR achieves two primary goals:

1. Uses the finalized/justified checkpoints of fork choice (FC), rather than that of the head state.
2. Refactors fork choice, block production and block processing to `async` functions.

Additionally, it achieves:

- Concurrent forkchoice updates to the EL and cache pruning after a new head is selected.
- Concurrent "block packing" (attestations, etc) and execution payload retrieval during block production.
- Concurrent per-block-processing and execution payload verification during block processing.
- The `Arc`-ification of `SignedBeaconBlock` during block processing (it's never mutated, so why not?):
    - I had to do this to deal with sending blocks into spawned tasks.
    - Previously we were cloning the beacon block at least 2 times during each block processing, these clones are either removed or turned into cheaper `Arc` clones.
    - We were also `Box`-ing and un-`Box`-ing beacon blocks as they moved throughout the networking crate. This is not a big deal, but it's nice to avoid shifting things between the stack and heap.
    - Avoids cloning *all the blocks* in *every chain segment* during sync.
    - It also has the potential to clean up our code where we need to pass an *owned* block around so we can send it back in the case of an error (I didn't do much of this, my PR is already big enough 😅)
- The `BeaconChain::HeadSafetyStatus` struct was removed. It was an old relic from prior merge specs.

For motivation for this change, see https://github.com/sigp/lighthouse/pull/3244#issuecomment-1160963273

## Changes to `canonical_head` and `fork_choice`

Previously, the `BeaconChain` had two separate fields:

```
canonical_head: RwLock<Snapshot>,
fork_choice: RwLock<BeaconForkChoice>
```

Now, we have grouped these values under a single struct:

```
canonical_head: CanonicalHead {
  cached_head: RwLock<Arc<Snapshot>>,
  fork_choice: RwLock<BeaconForkChoice>
} 
```

Apart from ergonomics, the only *actual* change here is wrapping the canonical head snapshot in an `Arc`. This means that we no longer need to hold the `cached_head` (`canonical_head`, in old terms) lock when we want to pull some values from it. This was done to avoid deadlock risks by preventing functions from acquiring (and holding) the `cached_head` and `fork_choice` locks simultaneously.

## Breaking Changes

### The `state` (root) field in the `finalized_checkpoint` SSE event

Consider the scenario where epoch `n` is just finalized, but `start_slot(n)` is skipped. There are two state roots we might in the `finalized_checkpoint` SSE event:

1. The state root of the finalized block, which is `get_block(finalized_checkpoint.root).state_root`.
4. The state root at slot of `start_slot(n)`, which would be the state from (1), but "skipped forward" through any skip slots.

Previously, Lighthouse would choose (2). However, we can see that when [Teku generates that event](de2b2801c8/data/beaconrestapi/src/main/java/tech/pegasys/teku/beaconrestapi/handlers/v1/events/EventSubscriptionManager.java (L171-L182)) it uses [`getStateRootFromBlockRoot`](de2b2801c8/data/provider/src/main/java/tech/pegasys/teku/api/ChainDataProvider.java (L336-L341)) which uses (1).

I have switched Lighthouse from (2) to (1). I think it's a somewhat arbitrary choice between the two, where (1) is easier to compute and is consistent with Teku.

## Notes for Reviewers

I've renamed `BeaconChain::fork_choice` to `BeaconChain::recompute_head`. Doing this helped ensure I broke all previous uses of fork choice and I also find it more descriptive. It describes an action and can't be confused with trying to get a reference to the `ForkChoice` struct.

I've changed the ordering of SSE events when a block is received. It used to be `[block, finalized, head]` and now it's `[block, head, finalized]`. It was easier this way and I don't think we were making any promises about SSE event ordering so it's not "breaking".

I've made it so fork choice will run when it's first constructed. I did this because I wanted to have a cached version of the last call to `get_head`. Ensuring `get_head` has been run *at least once* means that the cached values doesn't need to wrapped in an `Option`. This was fairly simple, it just involved passing a `slot` to the constructor so it knows *when* it's being run. When loading a fork choice from the store and a slot clock isn't handy I've just used the `slot` that was saved in the `fork_choice_store`. That seems like it would be a faithful representation of the slot when we saved it.

I added the `genesis_time: u64` to the `BeaconChain`. It's small, constant and nice to have around.

Since we're using FC for the fin/just checkpoints, we no longer get the `0x00..00` roots at genesis. You can see I had to remove a work-around in `ef-tests` here: b56be3bc2. I can't find any reason why this would be an issue, if anything I think it'll be better since the genesis-alias has caught us out a few times (0x00..00 isn't actually a real root). Edit: I did find a case where the `network` expected the 0x00..00 alias and patched it here: 3f26ac3e2.

You'll notice a lot of changes in tests. Generally, tests should be functionally equivalent. Here are the things creating the most diff-noise in tests:
- Changing tests to be `tokio::async` tests.
- Adding `.await` to fork choice, block processing and block production functions.
- Refactor of the `canonical_head` "API" provided by the `BeaconChain`. E.g., `chain.canonical_head.cached_head()` instead of `chain.canonical_head.read()`.
- Wrapping `SignedBeaconBlock` in an `Arc`.
- In the `beacon_chain/tests/block_verification`, we can't use the `lazy_static` `CHAIN_SEGMENT` variable anymore since it's generated with an async function. We just generate it in each test, not so efficient but hopefully insignificant.

I had to disable `rayon` concurrent tests in the `fork_choice` tests. This is because the use of `rayon` and `block_on` was causing a panic.

Co-authored-by: Mac L <mjladson@pm.me>
This commit is contained in:
Paul Hauner 2022-07-03 05:36:50 +00:00
parent e5212f1320
commit be4e261e74
106 changed files with 6515 additions and 4538 deletions

5
Cargo.lock generated
View File

@ -2062,6 +2062,7 @@ dependencies = [
"eth2_ssz_derive",
"proto_array",
"store",
"tokio",
"types",
]
@ -4173,6 +4174,7 @@ dependencies = [
"serde_derive",
"state_processing",
"store",
"tokio",
"types",
]
@ -5972,6 +5974,7 @@ dependencies = [
"rayon",
"safe_arith",
"smallvec",
"tokio",
"tree_hash",
"types",
]
@ -5984,6 +5987,7 @@ dependencies = [
"eth2_ssz",
"lazy_static",
"state_processing",
"tokio",
"types",
]
@ -6698,6 +6702,7 @@ dependencies = [
"swap_or_not_shuffle",
"tempfile",
"test_random_derive",
"tokio",
"tree_hash",
"tree_hash_derive",
]

View File

@ -976,8 +976,8 @@ fn verify_head_block_is_known<T: BeaconChainTypes>(
max_skip_slots: Option<u64>,
) -> Result<ProtoBlock, Error> {
let block_opt = chain
.fork_choice
.read()
.canonical_head
.fork_choice_read_lock()
.get_block(&attestation.data.beacon_block_root)
.or_else(|| {
chain
@ -1245,7 +1245,10 @@ where
// processing an attestation that does not include our latest finalized block in its chain.
//
// We do not delay consideration for later, we simply drop the attestation.
if !chain.fork_choice.read().contains_block(&target.root)
if !chain
.canonical_head
.fork_choice_read_lock()
.contains_block(&target.root)
&& !chain.early_attester_cache.contains_block(target.root)
{
return Err(Error::UnknownTargetRoot(target.root));

View File

@ -65,7 +65,7 @@ where
.try_read_for(VALIDATOR_PUBKEY_CACHE_LOCK_TIMEOUT)
.ok_or(BeaconChainError::ValidatorPubkeyCacheLockTimeout)?;
let fork = chain.with_head(|head| Ok::<_, BeaconChainError>(head.beacon_state.fork()))?;
let fork = chain.canonical_head.cached_head().head_fork();
let mut signature_sets = Vec::with_capacity(num_indexed * 3);
@ -169,13 +169,13 @@ where
&metrics::ATTESTATION_PROCESSING_BATCH_UNAGG_SIGNATURE_SETUP_TIMES,
);
let fork = chain.canonical_head.cached_head().head_fork();
let pubkey_cache = chain
.validator_pubkey_cache
.try_read_for(VALIDATOR_PUBKEY_CACHE_LOCK_TIMEOUT)
.ok_or(BeaconChainError::ValidatorPubkeyCacheLockTimeout)?;
let fork = chain.with_head(|head| Ok::<_, BeaconChainError>(head.beacon_state.fork()))?;
let mut signature_sets = Vec::with_capacity(num_partially_verified);
// Iterate, flattening to get only the `Ok` values.

File diff suppressed because it is too large Load Diff

View File

@ -13,8 +13,8 @@ use std::sync::Arc;
use store::{Error as StoreError, HotColdDB, ItemStore};
use superstruct::superstruct;
use types::{
BeaconBlock, BeaconState, BeaconStateError, Checkpoint, Epoch, EthSpec, ExecPayload, Hash256,
Slot,
BeaconBlockRef, BeaconState, BeaconStateError, Checkpoint, Epoch, EthSpec, ExecPayload,
Hash256, Slot,
};
#[derive(Debug)]
@ -257,7 +257,7 @@ where
fn on_verified_block<Payload: ExecPayload<E>>(
&mut self,
_block: &BeaconBlock<E, Payload>,
_block: BeaconBlockRef<E, Payload>,
block_root: Hash256,
state: &BeaconState<E>,
) -> Result<(), Self::Error> {

View File

@ -9,12 +9,14 @@
//! values it stores are very small, so this should not be an issue.
use crate::{BeaconChain, BeaconChainError, BeaconChainTypes};
use fork_choice::ExecutionStatus;
use lru::LruCache;
use smallvec::SmallVec;
use state_processing::state_advance::partial_state_advance;
use std::cmp::Ordering;
use types::{
BeaconState, BeaconStateError, ChainSpec, Epoch, EthSpec, Fork, Hash256, Slot, Unsigned,
BeaconState, BeaconStateError, ChainSpec, CloneConfig, Epoch, EthSpec, Fork, Hash256, Slot,
Unsigned,
};
/// The number of sets of proposer indices that should be cached.
@ -135,11 +137,26 @@ impl BeaconProposerCache {
pub fn compute_proposer_duties_from_head<T: BeaconChainTypes>(
current_epoch: Epoch,
chain: &BeaconChain<T>,
) -> Result<(Vec<usize>, Hash256, Fork), BeaconChainError> {
// Take a copy of the head of the chain.
let head = chain.head()?;
let mut state = head.beacon_state;
let head_state_root = head.beacon_block.state_root();
) -> Result<(Vec<usize>, Hash256, ExecutionStatus, Fork), BeaconChainError> {
// Atomically collect information about the head whilst holding the canonical head `Arc` as
// short as possible.
let (mut state, head_state_root, head_block_root) = {
let head = chain.canonical_head.cached_head();
// Take a copy of the head state.
let head_state = head
.snapshot
.beacon_state
.clone_with(CloneConfig::committee_caches_only());
let head_state_root = head.head_state_root();
let head_block_root = head.head_block_root();
(head_state, head_state_root, head_block_root)
};
let execution_status = chain
.canonical_head
.fork_choice_read_lock()
.get_block_execution_status(&head_block_root)
.ok_or(BeaconChainError::HeadMissingFromForkChoice(head_block_root))?;
// Advance the state into the requested epoch.
ensure_state_is_in_epoch(&mut state, head_state_root, current_epoch, &chain.spec)?;
@ -153,7 +170,7 @@ pub fn compute_proposer_duties_from_head<T: BeaconChainTypes>(
.proposer_shuffling_decision_root(chain.genesis_block_root)
.map_err(BeaconChainError::from)?;
Ok((indices, dependent_root, state.fork()))
Ok((indices, dependent_root, execution_status, state.fork()))
}
/// If required, advance `state` to `target_epoch`.

View File

@ -1,4 +1,5 @@
use serde_derive::Serialize;
use std::sync::Arc;
use types::{
beacon_state::CloneConfig, BeaconState, EthSpec, ExecPayload, FullPayload, Hash256,
SignedBeaconBlock,
@ -8,7 +9,7 @@ use types::{
/// head, justified head and finalized head.
#[derive(Clone, Serialize, PartialEq, Debug)]
pub struct BeaconSnapshot<E: EthSpec, Payload: ExecPayload<E> = FullPayload<E>> {
pub beacon_block: SignedBeaconBlock<E, Payload>,
pub beacon_block: Arc<SignedBeaconBlock<E, Payload>>,
pub beacon_block_root: Hash256,
pub beacon_state: BeaconState<E>,
}
@ -16,7 +17,7 @@ pub struct BeaconSnapshot<E: EthSpec, Payload: ExecPayload<E> = FullPayload<E>>
impl<E: EthSpec, Payload: ExecPayload<E>> BeaconSnapshot<E, Payload> {
/// Create a new checkpoint.
pub fn new(
beacon_block: SignedBeaconBlock<E, Payload>,
beacon_block: Arc<SignedBeaconBlock<E, Payload>>,
beacon_block_root: Hash256,
beacon_state: BeaconState<E>,
) -> Self {
@ -39,7 +40,7 @@ impl<E: EthSpec, Payload: ExecPayload<E>> BeaconSnapshot<E, Payload> {
/// Update all fields of the checkpoint.
pub fn update(
&mut self,
beacon_block: SignedBeaconBlock<E, Payload>,
beacon_block: Arc<SignedBeaconBlock<E, Payload>>,
beacon_block_root: Hash256,
beacon_state: BeaconState<E>,
) {

View File

@ -31,24 +31,27 @@
//! |---------------
//! |
//! ▼
//! SignatureVerifiedBlock
//! SignatureVerifiedBlock
//! |
//! ▼
//! FullyVerifiedBlock
//! ExecutionPendingBlock
//! |
//! await
//! |
//! ▼
//! END
//!
//! ```
use crate::execution_payload::{
notify_new_payload, validate_execution_payload_for_gossip, validate_merge_block,
is_optimistic_candidate_block, validate_execution_payload_for_gossip, validate_merge_block,
PayloadNotifier,
};
use crate::snapshot_cache::PreProcessingSnapshot;
use crate::validator_monitor::HISTORIC_EPOCHS as VALIDATOR_MONITOR_HISTORIC_EPOCHS;
use crate::validator_pubkey_cache::ValidatorPubkeyCache;
use crate::{
beacon_chain::{
BLOCK_PROCESSING_CACHE_LOCK_TIMEOUT, MAXIMUM_GOSSIP_CLOCK_DISPARITY,
BeaconForkChoice, BLOCK_PROCESSING_CACHE_LOCK_TIMEOUT, MAXIMUM_GOSSIP_CLOCK_DISPARITY,
VALIDATOR_PUBKEY_CACHE_LOCK_TIMEOUT,
},
metrics, BeaconChain, BeaconChainError, BeaconChainTypes,
@ -56,11 +59,11 @@ use crate::{
use derivative::Derivative;
use eth2::types::EventKind;
use execution_layer::PayloadStatus;
use fork_choice::{ForkChoice, ForkChoiceStore, PayloadVerificationStatus};
use fork_choice::PayloadVerificationStatus;
use parking_lot::RwLockReadGuard;
use proto_array::Block as ProtoBlock;
use safe_arith::ArithError;
use slog::{debug, error, info, Logger};
use slog::{debug, error, warn, Logger};
use slot_clock::SlotClock;
use ssz::Encode;
use state_processing::per_block_processing::is_merge_transition_block;
@ -75,16 +78,16 @@ use std::fs;
use std::io::Write;
use std::sync::Arc;
use std::time::Duration;
use store::{Error as DBError, HotColdDB, HotStateSummary, KeyValueStore, StoreOp};
use store::{Error as DBError, HotStateSummary, KeyValueStore, StoreOp};
use task_executor::JoinHandle;
use tree_hash::TreeHash;
use types::ExecPayload;
use types::{
BeaconBlockRef, BeaconState, BeaconStateError, BlindedPayload, ChainSpec, CloneConfig, Epoch,
EthSpec, ExecutionBlockHash, Hash256, InconsistentFork, PublicKey, PublicKeyBytes,
RelativeEpoch, SignedBeaconBlock, SignedBeaconBlockHeader, Slot,
};
const POS_PANDA_BANNER: &str = r#"
pub const POS_PANDA_BANNER: &str = r#"
,,, ,,, ,,, ,,,
;" ^; ;' ", ;" ^; ;' ",
; s$$$$$$$s ; ; s$$$$$$$s ;
@ -129,7 +132,7 @@ pub enum BlockError<T: EthSpec> {
///
/// It's unclear if this block is valid, but it cannot be processed without already knowing
/// its parent.
ParentUnknown(Box<SignedBeaconBlock<T>>),
ParentUnknown(Arc<SignedBeaconBlock<T>>),
/// The block skips too many slots and is a DoS risk.
TooManySkippedSlots { parent_slot: Slot, block_slot: Slot },
/// The block slot is greater than the present slot.
@ -419,6 +422,12 @@ impl<T: EthSpec> From<ArithError> for BlockError<T> {
}
}
/// Stores information about verifying a payload against an execution engine.
pub struct PayloadVerificationOutcome {
pub payload_verification_status: PayloadVerificationStatus,
pub is_valid_merge_transition_block: bool,
}
/// Information about invalid blocks which might still be slashable despite being invalid.
#[allow(clippy::enum_variant_names)]
pub enum BlockSlashInfo<TErr> {
@ -474,7 +483,7 @@ fn process_block_slash_info<T: BeaconChainTypes>(
/// Verify all signatures (except deposit signatures) on all blocks in the `chain_segment`. If all
/// signatures are valid, the `chain_segment` is mapped to a `Vec<SignatureVerifiedBlock>` that can
/// later be transformed into a `FullyVerifiedBlock` without re-checking the signatures. If any
/// later be transformed into a `ExecutionPendingBlock` without re-checking the signatures. If any
/// signature in the block is invalid, an `Err` is returned (it is not possible to known _which_
/// signature was invalid).
///
@ -483,7 +492,7 @@ fn process_block_slash_info<T: BeaconChainTypes>(
/// The given `chain_segment` must span no more than two epochs, otherwise an error will be
/// returned.
pub fn signature_verify_chain_segment<T: BeaconChainTypes>(
mut chain_segment: Vec<(Hash256, SignedBeaconBlock<T::EthSpec>)>,
mut chain_segment: Vec<(Hash256, Arc<SignedBeaconBlock<T::EthSpec>>)>,
chain: &BeaconChain<T>,
) -> Result<Vec<SignatureVerifiedBlock<T>>, BlockError<T::EthSpec>> {
if chain_segment.is_empty() {
@ -541,7 +550,7 @@ pub fn signature_verify_chain_segment<T: BeaconChainTypes>(
#[derive(Derivative)]
#[derivative(Debug(bound = "T: BeaconChainTypes"))]
pub struct GossipVerifiedBlock<T: BeaconChainTypes> {
pub block: SignedBeaconBlock<T::EthSpec>,
pub block: Arc<SignedBeaconBlock<T::EthSpec>>,
pub block_root: Hash256,
parent: Option<PreProcessingSnapshot<T::EthSpec>>,
}
@ -549,11 +558,15 @@ pub struct GossipVerifiedBlock<T: BeaconChainTypes> {
/// A wrapper around a `SignedBeaconBlock` that indicates that all signatures (except the deposit
/// signatures) have been verified.
pub struct SignatureVerifiedBlock<T: BeaconChainTypes> {
block: SignedBeaconBlock<T::EthSpec>,
block: Arc<SignedBeaconBlock<T::EthSpec>>,
block_root: Hash256,
parent: Option<PreProcessingSnapshot<T::EthSpec>>,
}
/// Used to await the result of executing payload with a remote EE.
type PayloadVerificationHandle<E> =
JoinHandle<Option<Result<PayloadVerificationOutcome, BlockError<E>>>>;
/// A wrapper around a `SignedBeaconBlock` that indicates that this block is fully verified and
/// ready to import into the `BeaconChain`. The validation includes:
///
@ -562,42 +575,42 @@ pub struct SignatureVerifiedBlock<T: BeaconChainTypes> {
/// - State root check
/// - Per block processing
///
/// Note: a `FullyVerifiedBlock` is not _forever_ valid to be imported, it may later become invalid
/// due to finality or some other event. A `FullyVerifiedBlock` should be imported into the
/// Note: a `ExecutionPendingBlock` is not _forever_ valid to be imported, it may later become invalid
/// due to finality or some other event. A `ExecutionPendingBlock` should be imported into the
/// `BeaconChain` immediately after it is instantiated.
pub struct FullyVerifiedBlock<'a, T: BeaconChainTypes> {
pub block: SignedBeaconBlock<T::EthSpec>,
pub struct ExecutionPendingBlock<T: BeaconChainTypes> {
pub block: Arc<SignedBeaconBlock<T::EthSpec>>,
pub block_root: Hash256,
pub state: BeaconState<T::EthSpec>,
pub parent_block: SignedBeaconBlock<T::EthSpec, BlindedPayload<T::EthSpec>>,
pub confirmation_db_batch: Vec<StoreOp<'a, T::EthSpec>>,
pub payload_verification_status: PayloadVerificationStatus,
pub confirmed_state_roots: Vec<Hash256>,
pub payload_verification_handle: PayloadVerificationHandle<T::EthSpec>,
}
/// Implemented on types that can be converted into a `FullyVerifiedBlock`.
/// Implemented on types that can be converted into a `ExecutionPendingBlock`.
///
/// Used to allow functions to accept blocks at various stages of verification.
pub trait IntoFullyVerifiedBlock<T: BeaconChainTypes>: Sized {
fn into_fully_verified_block(
pub trait IntoExecutionPendingBlock<T: BeaconChainTypes>: Sized {
fn into_execution_pending_block(
self,
chain: &Arc<BeaconChain<T>>,
) -> Result<FullyVerifiedBlock<T>, BlockError<T::EthSpec>> {
self.into_fully_verified_block_slashable(chain)
.map(|fully_verified| {
) -> Result<ExecutionPendingBlock<T>, BlockError<T::EthSpec>> {
self.into_execution_pending_block_slashable(chain)
.map(|execution_pending| {
// Supply valid block to slasher.
if let Some(slasher) = chain.slasher.as_ref() {
slasher.accept_block_header(fully_verified.block.signed_block_header());
slasher.accept_block_header(execution_pending.block.signed_block_header());
}
fully_verified
execution_pending
})
.map_err(|slash_info| process_block_slash_info(chain, slash_info))
}
/// Convert the block to fully-verified form while producing data to aid checking slashability.
fn into_fully_verified_block_slashable(
fn into_execution_pending_block_slashable(
self,
chain: &Arc<BeaconChain<T>>,
) -> Result<FullyVerifiedBlock<T>, BlockSlashInfo<BlockError<T::EthSpec>>>;
) -> Result<ExecutionPendingBlock<T>, BlockSlashInfo<BlockError<T::EthSpec>>>;
fn block(&self) -> &SignedBeaconBlock<T::EthSpec>;
}
@ -608,7 +621,7 @@ impl<T: BeaconChainTypes> GossipVerifiedBlock<T> {
///
/// Returns an error if the block is invalid, or if the block was unable to be verified.
pub fn new(
block: SignedBeaconBlock<T::EthSpec>,
block: Arc<SignedBeaconBlock<T::EthSpec>>,
chain: &BeaconChain<T>,
) -> Result<Self, BlockError<T::EthSpec>> {
// If the block is valid for gossip we don't supply it to the slasher here because
@ -623,7 +636,7 @@ impl<T: BeaconChainTypes> GossipVerifiedBlock<T> {
/// As for new, but doesn't pass the block to the slasher.
fn new_without_slasher_checks(
block: SignedBeaconBlock<T::EthSpec>,
block: Arc<SignedBeaconBlock<T::EthSpec>>,
chain: &BeaconChain<T>,
) -> Result<Self, BlockError<T::EthSpec>> {
// Ensure the block is the correct structure for the fork at `block.slot()`.
@ -658,7 +671,11 @@ impl<T: BeaconChainTypes> GossipVerifiedBlock<T> {
// reboot if the `observed_block_producers` cache is empty. In that case, without this
// check, we will load the parent and state from disk only to find out later that we
// already know this block.
if chain.fork_choice.read().contains_block(&block_root) {
if chain
.canonical_head
.fork_choice_read_lock()
.contains_block(&block_root)
{
return Err(BlockError::BlockIsAlreadyKnown);
}
@ -678,10 +695,10 @@ impl<T: BeaconChainTypes> GossipVerifiedBlock<T> {
// Do not process a block that doesn't descend from the finalized root.
//
// We check this *before* we load the parent so that we can return a more detailed error.
let block = check_block_is_finalized_descendant::<T, _>(
block,
&chain.fork_choice.read(),
&chain.store,
check_block_is_finalized_descendant(
chain,
&chain.canonical_head.fork_choice_write_lock(),
&block,
)?;
let block_epoch = block.slot().epoch(T::EthSpec::slots_per_epoch());
@ -827,15 +844,15 @@ impl<T: BeaconChainTypes> GossipVerifiedBlock<T> {
}
}
impl<T: BeaconChainTypes> IntoFullyVerifiedBlock<T> for GossipVerifiedBlock<T> {
impl<T: BeaconChainTypes> IntoExecutionPendingBlock<T> for GossipVerifiedBlock<T> {
/// Completes verification of the wrapped `block`.
fn into_fully_verified_block_slashable(
fn into_execution_pending_block_slashable(
self,
chain: &Arc<BeaconChain<T>>,
) -> Result<FullyVerifiedBlock<T>, BlockSlashInfo<BlockError<T::EthSpec>>> {
let fully_verified =
) -> Result<ExecutionPendingBlock<T>, BlockSlashInfo<BlockError<T::EthSpec>>> {
let execution_pending =
SignatureVerifiedBlock::from_gossip_verified_block_check_slashable(self, chain)?;
fully_verified.into_fully_verified_block_slashable(chain)
execution_pending.into_execution_pending_block_slashable(chain)
}
fn block(&self) -> &SignedBeaconBlock<T::EthSpec> {
@ -849,7 +866,7 @@ impl<T: BeaconChainTypes> SignatureVerifiedBlock<T> {
///
/// Returns an error if the block is invalid, or if the block was unable to be verified.
pub fn new(
block: SignedBeaconBlock<T::EthSpec>,
block: Arc<SignedBeaconBlock<T::EthSpec>>,
block_root: Hash256,
chain: &BeaconChain<T>,
) -> Result<Self, BlockError<T::EthSpec>> {
@ -892,7 +909,7 @@ impl<T: BeaconChainTypes> SignatureVerifiedBlock<T> {
/// As for `new` above but producing `BlockSlashInfo`.
pub fn check_slashable(
block: SignedBeaconBlock<T::EthSpec>,
block: Arc<SignedBeaconBlock<T::EthSpec>>,
block_root: Hash256,
chain: &BeaconChain<T>,
) -> Result<Self, BlockSlashInfo<BlockError<T::EthSpec>>> {
@ -947,12 +964,12 @@ impl<T: BeaconChainTypes> SignatureVerifiedBlock<T> {
}
}
impl<T: BeaconChainTypes> IntoFullyVerifiedBlock<T> for SignatureVerifiedBlock<T> {
impl<T: BeaconChainTypes> IntoExecutionPendingBlock<T> for SignatureVerifiedBlock<T> {
/// Completes verification of the wrapped `block`.
fn into_fully_verified_block_slashable(
fn into_execution_pending_block_slashable(
self,
chain: &Arc<BeaconChain<T>>,
) -> Result<FullyVerifiedBlock<T>, BlockSlashInfo<BlockError<T::EthSpec>>> {
) -> Result<ExecutionPendingBlock<T>, BlockSlashInfo<BlockError<T::EthSpec>>> {
let header = self.block.signed_block_header();
let (parent, block) = if let Some(parent) = self.parent {
(parent, self.block)
@ -961,7 +978,7 @@ impl<T: BeaconChainTypes> IntoFullyVerifiedBlock<T> for SignatureVerifiedBlock<T
.map_err(|e| BlockSlashInfo::SignatureValid(header.clone(), e))?
};
FullyVerifiedBlock::from_signature_verified_components(
ExecutionPendingBlock::from_signature_verified_components(
block,
self.block_root,
parent,
@ -975,19 +992,19 @@ impl<T: BeaconChainTypes> IntoFullyVerifiedBlock<T> for SignatureVerifiedBlock<T
}
}
impl<T: BeaconChainTypes> IntoFullyVerifiedBlock<T> for SignedBeaconBlock<T::EthSpec> {
impl<T: BeaconChainTypes> IntoExecutionPendingBlock<T> for Arc<SignedBeaconBlock<T::EthSpec>> {
/// Verifies the `SignedBeaconBlock` by first transforming it into a `SignatureVerifiedBlock`
/// and then using that implementation of `IntoFullyVerifiedBlock` to complete verification.
fn into_fully_verified_block_slashable(
/// and then using that implementation of `IntoExecutionPendingBlock` to complete verification.
fn into_execution_pending_block_slashable(
self,
chain: &Arc<BeaconChain<T>>,
) -> Result<FullyVerifiedBlock<T>, BlockSlashInfo<BlockError<T::EthSpec>>> {
) -> Result<ExecutionPendingBlock<T>, BlockSlashInfo<BlockError<T::EthSpec>>> {
// Perform an early check to prevent wasting time on irrelevant blocks.
let block_root = check_block_relevancy(&self, None, chain)
.map_err(|e| BlockSlashInfo::SignatureNotChecked(self.signed_block_header(), e))?;
SignatureVerifiedBlock::check_slashable(self, block_root, chain)?
.into_fully_verified_block_slashable(chain)
.into_execution_pending_block_slashable(chain)
}
fn block(&self) -> &SignedBeaconBlock<T::EthSpec> {
@ -995,7 +1012,7 @@ impl<T: BeaconChainTypes> IntoFullyVerifiedBlock<T> for SignedBeaconBlock<T::Eth
}
}
impl<'a, T: BeaconChainTypes> FullyVerifiedBlock<'a, T> {
impl<T: BeaconChainTypes> ExecutionPendingBlock<T> {
/// Instantiates `Self`, a wrapper that indicates that the given `block` is fully valid. See
/// the struct-level documentation for more information.
///
@ -1004,12 +1021,16 @@ impl<'a, T: BeaconChainTypes> FullyVerifiedBlock<'a, T> {
///
/// Returns an error if the block is invalid, or if the block was unable to be verified.
pub fn from_signature_verified_components(
block: SignedBeaconBlock<T::EthSpec>,
block: Arc<SignedBeaconBlock<T::EthSpec>>,
block_root: Hash256,
parent: PreProcessingSnapshot<T::EthSpec>,
chain: &Arc<BeaconChain<T>>,
) -> Result<Self, BlockError<T::EthSpec>> {
if let Some(parent) = chain.fork_choice.read().get_block(&block.parent_root()) {
if let Some(parent) = chain
.canonical_head
.fork_choice_read_lock()
.get_block(&block.parent_root())
{
// Reject any block where the parent has an invalid payload. It's impossible for a valid
// block to descend from an invalid parent.
if parent.execution_status.is_invalid() {
@ -1028,7 +1049,7 @@ impl<'a, T: BeaconChainTypes> FullyVerifiedBlock<'a, T> {
// because it will revert finalization. Note that the finalized block is stored in fork
// choice, so we will not reject any child of the finalized block (this is relevant during
// genesis).
return Err(BlockError::ParentUnknown(Box::new(block)));
return Err(BlockError::ParentUnknown(block));
}
// Reject any block that exceeds our limit on skipped slots.
@ -1048,7 +1069,7 @@ impl<'a, T: BeaconChainTypes> FullyVerifiedBlock<'a, T> {
// Stage a batch of operations to be completed atomically if this block is imported
// successfully.
let mut confirmation_db_batch = vec![];
let mut confirmed_state_roots = vec![];
// The block must have a higher slot than its parent.
if block.slot() <= parent.beacon_block.slot() {
@ -1121,7 +1142,7 @@ impl<'a, T: BeaconChainTypes> FullyVerifiedBlock<'a, T> {
chain.store.do_atomically(state_batch)?;
drop(txn_lock);
confirmation_db_batch.push(StoreOp::DeleteStateTemporaryFlag(state_root));
confirmed_state_roots.push(state_root);
state_root
};
@ -1140,59 +1161,82 @@ impl<'a, T: BeaconChainTypes> FullyVerifiedBlock<'a, T> {
}
}
// If this block triggers the merge, check to ensure that it references valid execution
// blocks.
//
// The specification defines this check inside `on_block` in the fork-choice specification,
// however we perform the check here for two reasons:
//
// - There's no point in importing a block that will fail fork choice, so it's best to fail
// early.
// - Doing the check here means we can keep our fork-choice implementation "pure". I.e., no
// calls to remote servers.
let valid_merge_transition_block =
if is_merge_transition_block(&state, block.message().body()) {
validate_merge_block(chain, block.message())?;
true
} else {
false
let block_slot = block.slot();
let state_current_epoch = state.current_epoch();
// Define a future that will verify the execution payload with an execution engine (but
// don't execute it yet).
let payload_notifier = PayloadNotifier::new(chain.clone(), block.clone(), &state)?;
let is_valid_merge_transition_block =
is_merge_transition_block(&state, block.message().body());
let payload_verification_future = async move {
let chain = payload_notifier.chain.clone();
let block = payload_notifier.block.clone();
// If this block triggers the merge, check to ensure that it references valid execution
// blocks.
//
// The specification defines this check inside `on_block` in the fork-choice specification,
// however we perform the check here for two reasons:
//
// - There's no point in importing a block that will fail fork choice, so it's best to fail
// early.
// - Doing the check here means we can keep our fork-choice implementation "pure". I.e., no
// calls to remote servers.
if is_valid_merge_transition_block {
validate_merge_block(&chain, block.message()).await?;
};
// The specification declares that this should be run *inside* `per_block_processing`,
// however we run it here to keep `per_block_processing` pure (i.e., no calls to external
// servers).
//
// It is important that this function is called *after* `per_slot_processing`, since the
// `randao` may change.
let payload_verification_status = notify_new_payload(chain, &state, block.message())?;
// The specification declares that this should be run *inside* `per_block_processing`,
// however we run it here to keep `per_block_processing` pure (i.e., no calls to external
// servers).
//
// It is important that this function is called *after* `per_slot_processing`, since the
// `randao` may change.
let payload_verification_status = payload_notifier.notify_new_payload().await?;
// If the payload did not validate or invalidate the block, check to see if this block is
// valid for optimistic import.
if payload_verification_status.is_optimistic() {
let current_slot = chain
.slot_clock
.now()
.ok_or(BeaconChainError::UnableToReadSlot)?;
// If the payload did not validate or invalidate the block, check to see if this block is
// valid for optimistic import.
if payload_verification_status.is_optimistic() {
let block_hash_opt = block
.message()
.body()
.execution_payload()
.map(|full_payload| full_payload.execution_payload.block_hash);
if !chain
.fork_choice
.read()
.is_optimistic_candidate_block(
current_slot,
block.slot(),
&block.parent_root(),
&chain.spec,
)
.map_err(BeaconChainError::from)?
{
return Err(ExecutionPayloadError::UnverifiedNonOptimisticCandidate.into());
// Ensure the block is a candidate for optimistic import.
if !is_optimistic_candidate_block(&chain, block.slot(), block.parent_root()).await?
{
warn!(
chain.log,
"Rejecting optimistic block";
"block_hash" => ?block_hash_opt,
"msg" => "the execution engine is not synced"
);
return Err(ExecutionPayloadError::UnverifiedNonOptimisticCandidate.into());
}
}
}
Ok(PayloadVerificationOutcome {
payload_verification_status,
is_valid_merge_transition_block,
})
};
// Spawn the payload verification future as a new task, but don't wait for it to complete.
// The `payload_verification_future` will be awaited later to ensure verification completed
// successfully.
let payload_verification_handle = chain
.task_executor
.spawn_handle(
payload_verification_future,
"execution_payload_verification",
)
.ok_or(BeaconChainError::RuntimeShutdown)?;
// If the block is sufficiently recent, notify the validator monitor.
if let Some(slot) = chain.slot_clock.now() {
let epoch = slot.epoch(T::EthSpec::slots_per_epoch());
if block.slot().epoch(T::EthSpec::slots_per_epoch())
if block_slot.epoch(T::EthSpec::slots_per_epoch())
+ VALIDATOR_MONITOR_HISTORIC_EPOCHS as u64
>= epoch
{
@ -1201,7 +1245,7 @@ impl<'a, T: BeaconChainTypes> FullyVerifiedBlock<'a, T> {
// the `validator_monitor` lock from being bounced or held for a long time whilst
// performing `per_slot_processing`.
for (i, summary) in summaries.iter().enumerate() {
let epoch = state.current_epoch() - Epoch::from(summaries.len() - i);
let epoch = state_current_epoch - Epoch::from(summaries.len() - i);
if let Err(e) =
validator_monitor.process_validator_statuses(epoch, summary, &chain.spec)
{
@ -1300,21 +1344,13 @@ impl<'a, T: BeaconChainTypes> FullyVerifiedBlock<'a, T> {
});
}
if valid_merge_transition_block {
info!(chain.log, "{}", POS_PANDA_BANNER);
info!(chain.log, "Proof of Stake Activated"; "slot" => block.slot());
info!(chain.log, ""; "Terminal POW Block Hash" => ?block.message().execution_payload()?.parent_hash().into_root());
info!(chain.log, ""; "Merge Transition Block Root" => ?block.message().tree_hash_root());
info!(chain.log, ""; "Merge Transition Execution Hash" => ?block.message().execution_payload()?.block_hash().into_root());
}
Ok(Self {
block,
block_root,
state,
parent_block: parent.beacon_block,
confirmation_db_batch,
payload_verification_status,
confirmed_state_roots,
payload_verification_handle,
})
}
}
@ -1366,8 +1402,9 @@ fn check_block_against_finalized_slot<T: BeaconChainTypes>(
chain: &BeaconChain<T>,
) -> Result<(), BlockError<T::EthSpec>> {
let finalized_slot = chain
.head_info()?
.finalized_checkpoint
.canonical_head
.cached_head()
.finalized_checkpoint()
.epoch
.start_slot(T::EthSpec::slots_per_epoch());
@ -1383,13 +1420,17 @@ fn check_block_against_finalized_slot<T: BeaconChainTypes>(
}
/// Returns `Ok(block)` if the block descends from the finalized root.
pub fn check_block_is_finalized_descendant<T: BeaconChainTypes, F: ForkChoiceStore<T::EthSpec>>(
block: SignedBeaconBlock<T::EthSpec>,
fork_choice: &ForkChoice<F, T::EthSpec>,
store: &HotColdDB<T::EthSpec, T::HotStore, T::ColdStore>,
) -> Result<SignedBeaconBlock<T::EthSpec>, BlockError<T::EthSpec>> {
///
/// ## Warning
///
/// Taking a lock on the `chain.canonical_head.fork_choice` might cause a deadlock here.
pub fn check_block_is_finalized_descendant<T: BeaconChainTypes>(
chain: &BeaconChain<T>,
fork_choice: &BeaconForkChoice<T>,
block: &Arc<SignedBeaconBlock<T::EthSpec>>,
) -> Result<(), BlockError<T::EthSpec>> {
if fork_choice.is_descendant_of_finalized(block.parent_root()) {
Ok(block)
Ok(())
} else {
// If fork choice does *not* consider the parent to be a descendant of the finalized block,
// then there are two more cases:
@ -1399,7 +1440,8 @@ pub fn check_block_is_finalized_descendant<T: BeaconChainTypes, F: ForkChoiceSto
// pre-finalization or conflicting with finalization.
// 2. The parent is unknown to us, we probably want to download it since it might actually
// descend from the finalized root.
if store
if chain
.store
.block_exists(&block.parent_root())
.map_err(|e| BlockError::BeaconChainError(e.into()))?
{
@ -1407,7 +1449,7 @@ pub fn check_block_is_finalized_descendant<T: BeaconChainTypes, F: ForkChoiceSto
block_parent_root: block.parent_root(),
})
} else {
Err(BlockError::ParentUnknown(Box::new(block)))
Err(BlockError::ParentUnknown(block.clone()))
}
}
}
@ -1452,7 +1494,11 @@ pub fn check_block_relevancy<T: BeaconChainTypes>(
// Check if the block is already known. We know it is post-finalization, so it is
// sufficient to check the fork choice.
if chain.fork_choice.read().contains_block(&block_root) {
if chain
.canonical_head
.fork_choice_read_lock()
.contains_block(&block_root)
{
return Err(BlockError::BlockIsAlreadyKnown);
}
@ -1477,16 +1523,16 @@ pub fn get_block_root<E: EthSpec>(block: &SignedBeaconBlock<E>) -> Hash256 {
#[allow(clippy::type_complexity)]
fn verify_parent_block_is_known<T: BeaconChainTypes>(
chain: &BeaconChain<T>,
block: SignedBeaconBlock<T::EthSpec>,
) -> Result<(ProtoBlock, SignedBeaconBlock<T::EthSpec>), BlockError<T::EthSpec>> {
block: Arc<SignedBeaconBlock<T::EthSpec>>,
) -> Result<(ProtoBlock, Arc<SignedBeaconBlock<T::EthSpec>>), BlockError<T::EthSpec>> {
if let Some(proto_block) = chain
.fork_choice
.read()
.canonical_head
.fork_choice_read_lock()
.get_block(&block.message().parent_root())
{
Ok((proto_block, block))
} else {
Err(BlockError::ParentUnknown(Box::new(block)))
Err(BlockError::ParentUnknown(block))
}
}
@ -1496,12 +1542,12 @@ fn verify_parent_block_is_known<T: BeaconChainTypes>(
/// whilst attempting the operation.
#[allow(clippy::type_complexity)]
fn load_parent<T: BeaconChainTypes>(
block: SignedBeaconBlock<T::EthSpec>,
block: Arc<SignedBeaconBlock<T::EthSpec>>,
chain: &BeaconChain<T>,
) -> Result<
(
PreProcessingSnapshot<T::EthSpec>,
SignedBeaconBlock<T::EthSpec>,
Arc<SignedBeaconBlock<T::EthSpec>>,
),
BlockError<T::EthSpec>,
> {
@ -1518,11 +1564,11 @@ fn load_parent<T: BeaconChainTypes>(
// choice, so we will not reject any child of the finalized block (this is relevant during
// genesis).
if !chain
.fork_choice
.read()
.canonical_head
.fork_choice_read_lock()
.contains_block(&block.parent_root())
{
return Err(BlockError::ParentUnknown(Box::new(block)));
return Err(BlockError::ParentUnknown(block));
}
let block_delay = chain
@ -1717,18 +1763,12 @@ fn verify_header_signature<T: BeaconChainTypes>(
.get(header.message.proposer_index as usize)
.cloned()
.ok_or(BlockError::UnknownValidator(header.message.proposer_index))?;
let (fork, genesis_validators_root) =
chain.with_head::<_, BlockError<T::EthSpec>, _>(|head| {
Ok((
head.beacon_state.fork(),
head.beacon_state.genesis_validators_root(),
))
})?;
let head_fork = chain.canonical_head.cached_head().head_fork();
if header.verify_signature::<T::EthSpec>(
&proposer_pubkey,
&fork,
genesis_validators_root,
&head_fork,
chain.genesis_validators_root,
&chain.spec,
) {
Ok(())

View File

@ -1,4 +1,4 @@
use crate::beacon_chain::{BEACON_CHAIN_DB_KEY, ETH1_CACHE_DB_KEY, OP_POOL_DB_KEY};
use crate::beacon_chain::{CanonicalHead, BEACON_CHAIN_DB_KEY, ETH1_CACHE_DB_KEY, OP_POOL_DB_KEY};
use crate::eth1_chain::{CachingEth1Backend, SszEth1};
use crate::fork_choice_signal::ForkChoiceSignalTx;
use crate::fork_revert::{reset_fork_choice_to_finalization, revert_to_fork_boundary};
@ -245,6 +245,7 @@ where
let fork_choice =
BeaconChain::<Witness<TSlotClock, TEth1Backend, _, _, _>>::load_fork_choice(
store.clone(),
&self.spec,
)
.map_err(|e| format!("Unable to load fork choice from disk: {:?}", e))?
.ok_or("Fork choice not found in store")?;
@ -337,7 +338,7 @@ where
Ok((
BeaconSnapshot {
beacon_block_root,
beacon_block,
beacon_block: Arc::new(beacon_block),
beacon_state,
},
self,
@ -352,12 +353,15 @@ where
self = updated_builder;
let fc_store = BeaconForkChoiceStore::get_forkchoice_store(store, &genesis);
let current_slot = None;
let fork_choice = ForkChoice::from_anchor(
fc_store,
genesis.beacon_block_root,
&genesis.beacon_block,
&genesis.beacon_state,
current_slot,
&self.spec,
)
.map_err(|e| format!("Unable to initialize ForkChoice: {:?}", e))?;
@ -455,17 +459,20 @@ where
let snapshot = BeaconSnapshot {
beacon_block_root: weak_subj_block_root,
beacon_block: weak_subj_block,
beacon_block: Arc::new(weak_subj_block),
beacon_state: weak_subj_state,
};
let fc_store = BeaconForkChoiceStore::get_forkchoice_store(store, &snapshot);
let current_slot = Some(snapshot.beacon_block.slot());
let fork_choice = ForkChoice::from_anchor(
fc_store,
snapshot.beacon_block_root,
&snapshot.beacon_block,
&snapshot.beacon_state,
current_slot,
&self.spec,
)
.map_err(|e| format!("Unable to initialize ForkChoice: {:?}", e))?;
@ -638,17 +645,18 @@ where
head_block_root,
&head_state,
store.clone(),
Some(current_slot),
&self.spec,
)?;
}
let mut canonical_head = BeaconSnapshot {
let mut head_snapshot = BeaconSnapshot {
beacon_block_root: head_block_root,
beacon_block: head_block,
beacon_block: Arc::new(head_block),
beacon_state: head_state,
};
canonical_head
head_snapshot
.beacon_state
.build_all_caches(&self.spec)
.map_err(|e| format!("Failed to build state caches: {:?}", e))?;
@ -658,25 +666,17 @@ where
//
// This is a sanity check to detect database corruption.
let fc_finalized = fork_choice.finalized_checkpoint();
let head_finalized = canonical_head.beacon_state.finalized_checkpoint();
if fc_finalized != head_finalized {
let is_genesis = head_finalized.root.is_zero()
&& head_finalized.epoch == fc_finalized.epoch
&& fc_finalized.root == genesis_block_root;
let is_wss = store.get_anchor_slot().map_or(false, |anchor_slot| {
fc_finalized.epoch == anchor_slot.epoch(TEthSpec::slots_per_epoch())
});
if !is_genesis && !is_wss {
return Err(format!(
"Database corrupt: fork choice is finalized at {:?} whilst head is finalized at \
let head_finalized = head_snapshot.beacon_state.finalized_checkpoint();
if fc_finalized.epoch < head_finalized.epoch {
return Err(format!(
"Database corrupt: fork choice is finalized at {:?} whilst head is finalized at \
{:?}",
fc_finalized, head_finalized
));
}
fc_finalized, head_finalized
));
}
let validator_pubkey_cache = self.validator_pubkey_cache.map(Ok).unwrap_or_else(|| {
ValidatorPubkeyCache::new(&canonical_head.beacon_state, store.clone())
ValidatorPubkeyCache::new(&head_snapshot.beacon_state, store.clone())
.map_err(|e| format!("Unable to init validator pubkey cache: {:?}", e))
})?;
@ -691,7 +691,7 @@ where
if let Some(slot) = slot_clock.now() {
validator_monitor.process_valid_state(
slot.epoch(TEthSpec::slots_per_epoch()),
&canonical_head.beacon_state,
&head_snapshot.beacon_state,
);
}
@ -725,10 +725,18 @@ where
.do_atomically(self.pending_io_batch)
.map_err(|e| format!("Error writing chain & metadata to disk: {:?}", e))?;
let genesis_validators_root = head_snapshot.beacon_state.genesis_validators_root();
let genesis_time = head_snapshot.beacon_state.genesis_time();
let head_for_snapshot_cache = head_snapshot.clone();
let canonical_head = CanonicalHead::new(fork_choice, Arc::new(head_snapshot));
let beacon_chain = BeaconChain {
spec: self.spec,
config: self.chain_config,
store,
task_executor: self
.task_executor
.ok_or("Cannot build without task executor")?,
store_migrator,
slot_clock,
op_pool: self.op_pool.ok_or("Cannot build without op pool")?,
@ -758,18 +766,18 @@ where
observed_attester_slashings: <_>::default(),
eth1_chain: self.eth1_chain,
execution_layer: self.execution_layer,
genesis_validators_root: canonical_head.beacon_state.genesis_validators_root(),
canonical_head: TimeoutRwLock::new(canonical_head.clone()),
genesis_validators_root,
genesis_time,
canonical_head,
genesis_block_root,
genesis_state_root,
fork_choice: RwLock::new(fork_choice),
fork_choice_signal_tx,
fork_choice_signal_rx,
event_handler: self.event_handler,
head_tracker,
snapshot_cache: TimeoutRwLock::new(SnapshotCache::new(
DEFAULT_SNAPSHOT_CACHE_SIZE,
canonical_head,
head_for_snapshot_cache,
)),
shuffling_cache: TimeoutRwLock::new(ShufflingCache::new()),
beacon_proposer_cache: <_>::default(),
@ -787,9 +795,7 @@ where
validator_monitor: RwLock::new(validator_monitor),
};
let head = beacon_chain
.head()
.map_err(|e| format!("Failed to get head: {:?}", e))?;
let head = beacon_chain.head_snapshot();
// Prime the attester cache with the head state.
beacon_chain
@ -992,10 +998,10 @@ mod test {
.build()
.expect("should build");
let head = chain.head().expect("should get head");
let head = chain.head_snapshot();
let state = head.beacon_state;
let block = head.beacon_block;
let state = &head.beacon_state;
let block = &head.beacon_block;
assert_eq!(state.slot(), Slot::new(0), "should start from genesis");
assert_eq!(
@ -1014,7 +1020,7 @@ mod test {
.get_blinded_block(&Hash256::zero())
.expect("should read db")
.expect("should find genesis block"),
block.clone().into(),
block.clone_as_blinded(),
"should store genesis block under zero hash alias"
);
assert_eq!(

File diff suppressed because it is too large Load Diff

View File

@ -4,6 +4,7 @@ use crate::{
};
use parking_lot::RwLock;
use proto_array::Block as ProtoBlock;
use std::sync::Arc;
use types::*;
pub struct CacheItem<E: EthSpec> {
@ -18,7 +19,7 @@ pub struct CacheItem<E: EthSpec> {
/*
* Values used to make the block available.
*/
block: SignedBeaconBlock<E>,
block: Arc<SignedBeaconBlock<E>>,
proto_block: ProtoBlock,
}
@ -48,7 +49,7 @@ impl<E: EthSpec> EarlyAttesterCache<E> {
pub fn add_head_block(
&self,
beacon_block_root: Hash256,
block: SignedBeaconBlock<E>,
block: Arc<SignedBeaconBlock<E>>,
proto_block: ProtoBlock,
state: &BeaconState<E>,
spec: &ChainSpec,
@ -146,7 +147,7 @@ impl<E: EthSpec> EarlyAttesterCache<E> {
}
/// Returns the block, if `block_root` matches the cached item.
pub fn get_block(&self, block_root: Hash256) -> Option<SignedBeaconBlock<E>> {
pub fn get_block(&self, block_root: Hash256) -> Option<Arc<SignedBeaconBlock<E>>> {
self.item
.read()
.as_ref()

View File

@ -45,8 +45,8 @@ pub enum BeaconChainError {
UnableToReadSlot,
UnableToComputeTimeAtSlot,
RevertedFinalizedEpoch {
previous_epoch: Epoch,
new_epoch: Epoch,
old: Checkpoint,
new: Checkpoint,
},
SlotClockDidNotStart,
NoStateForSlot(Slot),
@ -161,6 +161,7 @@ pub enum BeaconChainError {
BlockRewardSyncError,
HeadMissingFromForkChoice(Hash256),
FinalizedBlockMissingFromForkChoice(Hash256),
HeadBlockMissingFromForkChoice(Hash256),
InvalidFinalizedPayload {
finalized_root: Hash256,
execution_block_hash: ExecutionBlockHash,
@ -184,11 +185,19 @@ pub enum BeaconChainError {
beacon_block_root: Hash256,
},
RuntimeShutdown,
TokioJoin(tokio::task::JoinError),
ProcessInvalidExecutionPayload(JoinError),
ForkChoiceSignalOutOfOrder {
current: Slot,
latest: Slot,
},
ForkchoiceUpdateParamsMissing,
HeadHasInvalidPayload {
block_root: Hash256,
execution_status: ExecutionStatus,
},
AttestationHeadNotInForkChoice(Hash256),
MissingPersistedForkChoice,
}
easy_from_to!(SlotProcessingError, BeaconChainError);
@ -214,7 +223,6 @@ easy_from_to!(BlockReplayError, BeaconChainError);
#[derive(Debug)]
pub enum BlockProductionError {
UnableToGetHeadInfo(BeaconChainError),
UnableToGetBlockRootFromState,
UnableToReadSlot,
UnableToProduceAtSlot(Slot),
@ -239,6 +247,11 @@ pub enum BlockProductionError {
MissingFinalizedBlock(Hash256),
BlockTooLarge(usize),
ForkChoiceError(BeaconChainError),
ShuttingDown,
MissingSyncAggregate,
MissingExecutionPayload,
TokioJoin(tokio::task::JoinError),
BeaconChain(BeaconChainError),
}
easy_from_to!(BlockProcessingError, BlockProductionError);

View File

@ -21,8 +21,59 @@ use state_processing::per_block_processing::{
partially_verify_execution_payload,
};
use std::sync::Arc;
use tokio::task::JoinHandle;
use types::*;
pub type PreparePayloadResult<Payload> = Result<Payload, BlockProductionError>;
pub type PreparePayloadHandle<Payload> = JoinHandle<Option<PreparePayloadResult<Payload>>>;
/// Used to await the result of executing payload with a remote EE.
pub struct PayloadNotifier<T: BeaconChainTypes> {
pub chain: Arc<BeaconChain<T>>,
pub block: Arc<SignedBeaconBlock<T::EthSpec>>,
payload_verification_status: Option<PayloadVerificationStatus>,
}
impl<T: BeaconChainTypes> PayloadNotifier<T> {
pub fn new(
chain: Arc<BeaconChain<T>>,
block: Arc<SignedBeaconBlock<T::EthSpec>>,
state: &BeaconState<T::EthSpec>,
) -> Result<Self, BlockError<T::EthSpec>> {
let payload_verification_status = if is_execution_enabled(state, block.message().body()) {
// Perform the initial stages of payload verification.
//
// We will duplicate these checks again during `per_block_processing`, however these checks
// are cheap and doing them here ensures we protect the execution engine from junk.
partially_verify_execution_payload(
state,
block.message().execution_payload()?,
&chain.spec,
)
.map_err(BlockError::PerBlockProcessingError)?;
None
} else {
Some(PayloadVerificationStatus::Irrelevant)
};
Ok(Self {
chain,
block,
payload_verification_status,
})
}
pub async fn notify_new_payload(
self,
) -> Result<PayloadVerificationStatus, BlockError<T::EthSpec>> {
if let Some(precomputed_status) = self.payload_verification_status {
Ok(precomputed_status)
} else {
notify_new_payload(&self.chain, self.block.message()).await
}
}
}
/// Verify that `execution_payload` contained by `block` is considered valid by an execution
/// engine.
///
@ -32,31 +83,20 @@ use types::*;
/// contains a few extra checks by running `partially_verify_execution_payload` first:
///
/// https://github.com/ethereum/consensus-specs/blob/v1.1.9/specs/bellatrix/beacon-chain.md#notify_new_payload
pub fn notify_new_payload<T: BeaconChainTypes>(
async fn notify_new_payload<'a, T: BeaconChainTypes>(
chain: &Arc<BeaconChain<T>>,
state: &BeaconState<T::EthSpec>,
block: BeaconBlockRef<T::EthSpec>,
block: BeaconBlockRef<'a, T::EthSpec>,
) -> Result<PayloadVerificationStatus, BlockError<T::EthSpec>> {
if !is_execution_enabled(state, block.body()) {
return Ok(PayloadVerificationStatus::Irrelevant);
}
let execution_payload = block.execution_payload()?;
// Perform the initial stages of payload verification.
//
// We will duplicate these checks again during `per_block_processing`, however these checks
// are cheap and doing them here ensures we protect the execution payload from junk.
partially_verify_execution_payload(state, execution_payload, &chain.spec)
.map_err(BlockError::PerBlockProcessingError)?;
let execution_layer = chain
.execution_layer
.as_ref()
.ok_or(ExecutionPayloadError::NoExecutionConnection)?;
let new_payload_response = execution_layer.block_on(|execution_layer| {
execution_layer.notify_new_payload(&execution_payload.execution_payload)
});
let new_payload_response = execution_layer
.notify_new_payload(&execution_payload.execution_payload)
.await;
match new_payload_response {
Ok(status) => match status {
@ -70,13 +110,13 @@ pub fn notify_new_payload<T: BeaconChainTypes>(
// This block has not yet been applied to fork choice, so the latest block that was
// imported to fork choice was the parent.
let latest_root = block.parent_root();
chain.process_invalid_execution_payload(
&InvalidationOperation::InvalidateMany {
chain
.process_invalid_execution_payload(&InvalidationOperation::InvalidateMany {
head_block_root: latest_root,
always_invalidate_head: false,
latest_valid_ancestor: latest_valid_hash,
},
)?;
})
.await?;
Err(ExecutionPayloadError::RejectedByExecutionEngine { status }.into())
}
@ -103,9 +143,9 @@ pub fn notify_new_payload<T: BeaconChainTypes>(
/// Equivalent to the `validate_merge_block` function in the merge Fork Choice Changes:
///
/// https://github.com/ethereum/consensus-specs/blob/v1.1.5/specs/merge/fork-choice.md#validate_merge_block
pub fn validate_merge_block<T: BeaconChainTypes>(
chain: &BeaconChain<T>,
block: BeaconBlockRef<T::EthSpec>,
pub async fn validate_merge_block<'a, T: BeaconChainTypes>(
chain: &Arc<BeaconChain<T>>,
block: BeaconBlockRef<'a, T::EthSpec>,
) -> Result<(), BlockError<T::EthSpec>> {
let spec = &chain.spec;
let block_epoch = block.slot().epoch(T::EthSpec::slots_per_epoch());
@ -137,9 +177,8 @@ pub fn validate_merge_block<T: BeaconChainTypes>(
.ok_or(ExecutionPayloadError::NoExecutionConnection)?;
let is_valid_terminal_pow_block = execution_layer
.block_on(|execution_layer| {
execution_layer.is_valid_terminal_pow_block_hash(execution_payload.parent_hash(), spec)
})
.is_valid_terminal_pow_block_hash(execution_payload.parent_hash(), spec)
.await
.map_err(ExecutionPayloadError::from)?;
match is_valid_terminal_pow_block {
@ -149,23 +188,7 @@ pub fn validate_merge_block<T: BeaconChainTypes>(
}
.into()),
None => {
let current_slot = chain
.slot_clock
.now()
.ok_or(BeaconChainError::UnableToReadSlot)?;
// Ensure the block is a candidate for optimistic import.
if chain
.fork_choice
.read()
.is_optimistic_candidate_block(
current_slot,
block.slot(),
&block.parent_root(),
&chain.spec,
)
.map_err(BeaconChainError::from)?
{
if is_optimistic_candidate_block(chain, block.slot(), block.parent_root()).await? {
debug!(
chain.log,
"Optimistically accepting terminal block";
@ -180,6 +203,36 @@ pub fn validate_merge_block<T: BeaconChainTypes>(
}
}
/// Check to see if a block with the given parameters is valid to be imported optimistically.
pub async fn is_optimistic_candidate_block<T: BeaconChainTypes>(
chain: &Arc<BeaconChain<T>>,
block_slot: Slot,
block_parent_root: Hash256,
) -> Result<bool, BeaconChainError> {
let current_slot = chain.slot()?;
let inner_chain = chain.clone();
// Use a blocking task to check if the block is an optimistic candidate. Interacting
// with the `fork_choice` lock in an async task can block the core executor.
chain
.spawn_blocking_handle(
move || {
inner_chain
.canonical_head
.fork_choice_read_lock()
.is_optimistic_candidate_block(
current_slot,
block_slot,
&block_parent_root,
&inner_chain.spec,
)
},
"validate_merge_block_optimistic_candidate",
)
.await?
.map_err(BeaconChainError::from)
}
/// Validate the gossip block's execution_payload according to the checks described here:
/// https://github.com/ethereum/consensus-specs/blob/dev/specs/merge/p2p-interface.md#beacon_block
pub fn validate_execution_payload_for_gossip<T: BeaconChainTypes>(
@ -243,35 +296,52 @@ pub fn validate_execution_payload_for_gossip<T: BeaconChainTypes>(
/// Equivalent to the `get_execution_payload` function in the Validator Guide:
///
/// https://github.com/ethereum/consensus-specs/blob/v1.1.5/specs/merge/validator.md#block-proposal
pub fn get_execution_payload<T: BeaconChainTypes, Payload: ExecPayload<T::EthSpec>>(
chain: &BeaconChain<T>,
pub fn get_execution_payload<
T: BeaconChainTypes,
Payload: ExecPayload<T::EthSpec> + Default + Send + 'static,
>(
chain: Arc<BeaconChain<T>>,
state: &BeaconState<T::EthSpec>,
finalized_checkpoint: Checkpoint,
proposer_index: u64,
pubkey: Option<PublicKeyBytes>,
) -> Result<Payload, BlockProductionError> {
Ok(
prepare_execution_payload_blocking::<T, Payload>(chain, state, proposer_index, pubkey)?
.unwrap_or_default(),
)
}
) -> Result<PreparePayloadHandle<Payload>, BlockProductionError> {
// Compute all required values from the `state` now to avoid needing to pass it into a spawned
// task.
let spec = &chain.spec;
let slot = state.slot();
let current_epoch = state.current_epoch();
let is_merge_transition_complete = is_merge_transition_complete(state);
let timestamp = compute_timestamp_at_slot(state, spec).map_err(BeaconStateError::from)?;
let random = *state.get_randao_mix(current_epoch)?;
let latest_execution_payload_header_block_hash =
state.latest_execution_payload_header()?.block_hash;
/// Wraps the async `prepare_execution_payload` function as a blocking task.
pub fn prepare_execution_payload_blocking<T: BeaconChainTypes, Payload: ExecPayload<T::EthSpec>>(
chain: &BeaconChain<T>,
state: &BeaconState<T::EthSpec>,
proposer_index: u64,
pubkey: Option<PublicKeyBytes>,
) -> Result<Option<Payload>, BlockProductionError> {
let execution_layer = chain
.execution_layer
.as_ref()
.ok_or(BlockProductionError::ExecutionLayerMissing)?;
// Spawn a task to obtain the execution payload from the EL via a series of async calls. The
// `join_handle` can be used to await the result of the function.
let join_handle = chain
.task_executor
.clone()
.spawn_handle(
async move {
prepare_execution_payload::<T, Payload>(
&chain,
slot,
is_merge_transition_complete,
timestamp,
random,
finalized_checkpoint,
proposer_index,
pubkey,
latest_execution_payload_header_block_hash,
)
.await
},
"get_execution_payload",
)
.ok_or(BlockProductionError::ShuttingDown)?;
execution_layer
.block_on_generic(|_| async {
prepare_execution_payload::<T, Payload>(chain, state, proposer_index, pubkey).await
})
.map_err(BlockProductionError::BlockingFailed)?
Ok(join_handle)
}
/// Prepares an execution payload for inclusion in a block.
@ -288,25 +358,38 @@ pub fn prepare_execution_payload_blocking<T: BeaconChainTypes, Payload: ExecPayl
/// Equivalent to the `prepare_execution_payload` function in the Validator Guide:
///
/// https://github.com/ethereum/consensus-specs/blob/v1.1.5/specs/merge/validator.md#block-proposal
pub async fn prepare_execution_payload<T: BeaconChainTypes, Payload: ExecPayload<T::EthSpec>>(
chain: &BeaconChain<T>,
state: &BeaconState<T::EthSpec>,
#[allow(clippy::too_many_arguments)]
pub async fn prepare_execution_payload<T, Payload>(
chain: &Arc<BeaconChain<T>>,
slot: Slot,
is_merge_transition_complete: bool,
timestamp: u64,
random: Hash256,
finalized_checkpoint: Checkpoint,
proposer_index: u64,
pubkey: Option<PublicKeyBytes>,
) -> Result<Option<Payload>, BlockProductionError> {
latest_execution_payload_header_block_hash: ExecutionBlockHash,
) -> Result<Payload, BlockProductionError>
where
T: BeaconChainTypes,
Payload: ExecPayload<T::EthSpec> + Default,
{
let current_epoch = slot.epoch(T::EthSpec::slots_per_epoch());
let spec = &chain.spec;
let execution_layer = chain
.execution_layer
.as_ref()
.ok_or(BlockProductionError::ExecutionLayerMissing)?;
let parent_hash = if !is_merge_transition_complete(state) {
let parent_hash = if !is_merge_transition_complete {
let is_terminal_block_hash_set = spec.terminal_block_hash != ExecutionBlockHash::zero();
let is_activation_epoch_reached =
state.current_epoch() >= spec.terminal_block_hash_activation_epoch;
current_epoch >= spec.terminal_block_hash_activation_epoch;
if is_terminal_block_hash_set && !is_activation_epoch_reached {
return Ok(None);
// Use the "empty" payload if there's a terminal block hash, but we haven't reached the
// terminal block epoch yet.
return Ok(<_>::default());
}
let terminal_pow_block_hash = execution_layer
@ -317,36 +400,55 @@ pub async fn prepare_execution_payload<T: BeaconChainTypes, Payload: ExecPayload
if let Some(terminal_pow_block_hash) = terminal_pow_block_hash {
terminal_pow_block_hash
} else {
return Ok(None);
// If the merge transition hasn't occurred yet and the EL hasn't found the terminal
// block, return an "empty" payload.
return Ok(<_>::default());
}
} else {
state.latest_execution_payload_header()?.block_hash
latest_execution_payload_header_block_hash
};
let timestamp = compute_timestamp_at_slot(state, spec).map_err(BeaconStateError::from)?;
let random = *state.get_randao_mix(state.current_epoch())?;
let finalized_root = state.finalized_checkpoint().root;
// Try to obtain the finalized proto block from fork choice.
//
// Use a blocking task to interact with the `fork_choice` lock otherwise we risk blocking the
// core `tokio` executor.
let inner_chain = chain.clone();
let finalized_proto_block = chain
.spawn_blocking_handle(
move || {
inner_chain
.canonical_head
.fork_choice_read_lock()
.get_block(&finalized_checkpoint.root)
},
"prepare_execution_payload_finalized_hash",
)
.await
.map_err(BlockProductionError::BeaconChain)?;
// The finalized block hash is not included in the specification, however we provide this
// parameter so that the execution layer can produce a payload id if one is not already known
// (e.g., due to a recent reorg).
let finalized_block_hash =
if let Some(block) = chain.fork_choice.read().get_block(&finalized_root) {
block.execution_status.block_hash()
} else {
chain
.store
.get_blinded_block(&finalized_root)
.map_err(BlockProductionError::FailedToReadFinalizedBlock)?
.ok_or(BlockProductionError::MissingFinalizedBlock(finalized_root))?
.message()
.body()
.execution_payload()
.ok()
.map(|ep| ep.block_hash())
};
let finalized_block_hash = if let Some(block) = finalized_proto_block {
block.execution_status.block_hash()
} else {
chain
.store
.get_blinded_block(&finalized_checkpoint.root)
.map_err(BlockProductionError::FailedToReadFinalizedBlock)?
.ok_or(BlockProductionError::MissingFinalizedBlock(
finalized_checkpoint.root,
))?
.message()
.body()
.execution_payload()
.ok()
.map(|ep| ep.block_hash())
};
// Note: the suggested_fee_recipient is stored in the `execution_layer`, it will add this parameter.
//
// This future is not executed here, it's up to the caller to await it.
let execution_payload = execution_layer
.get_payload::<Payload>(
parent_hash,
@ -355,10 +457,10 @@ pub async fn prepare_execution_payload<T: BeaconChainTypes, Payload: ExecPayload
finalized_block_hash.unwrap_or_else(ExecutionBlockHash::zero),
proposer_index,
pubkey,
state.slot(),
slot,
)
.await
.map_err(BlockProductionError::GetPayloadFailed)?;
Ok(Some(execution_payload))
Ok(execution_payload)
}

View File

@ -97,6 +97,7 @@ pub fn reset_fork_choice_to_finalization<E: EthSpec, Hot: ItemStore<E>, Cold: It
head_block_root: Hash256,
head_state: &BeaconState<E>,
store: Arc<HotColdDB<E, Hot, Cold>>,
current_slot: Option<Slot>,
spec: &ChainSpec,
) -> Result<ForkChoice<BeaconForkChoiceStore<E, Hot, Cold>, E>, String> {
// Fetch finalized block.
@ -138,7 +139,7 @@ pub fn reset_fork_choice_to_finalization<E: EthSpec, Hot: ItemStore<E>, Cold: It
})?;
let finalized_snapshot = BeaconSnapshot {
beacon_block_root: finalized_block_root,
beacon_block: finalized_block,
beacon_block: Arc::new(finalized_block),
beacon_state: finalized_state,
};
@ -149,6 +150,8 @@ pub fn reset_fork_choice_to_finalization<E: EthSpec, Hot: ItemStore<E>, Cold: It
finalized_block_root,
&finalized_snapshot.beacon_block,
&finalized_snapshot.beacon_state,
current_slot,
spec,
)
.map_err(|e| format!("Unable to reset fork choice for revert: {:?}", e))?;
@ -180,11 +183,10 @@ pub fn reset_fork_choice_to_finalization<E: EthSpec, Hot: ItemStore<E>, Cold: It
// This scenario is so rare that it seems OK to double-verify some blocks.
let payload_verification_status = PayloadVerificationStatus::Optimistic;
let (block, _) = block.deconstruct();
fork_choice
.on_block(
block.slot(),
&block,
block.message(),
block.canonical_root(),
// Reward proposer boost. We are reinforcing the canonical chain.
Duration::from_secs(0),

View File

@ -7,6 +7,7 @@ use state_processing::{
};
use std::borrow::Cow;
use std::iter;
use std::sync::Arc;
use std::time::Duration;
use store::{chunked_vector::BlockRoots, AnchorInfo, ChunkWriter, KeyValueStore};
use types::{Hash256, SignedBlindedBeaconBlock, Slot};
@ -58,7 +59,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
/// Return the number of blocks successfully imported.
pub fn import_historical_block_batch(
&self,
blocks: Vec<SignedBlindedBeaconBlock<T::EthSpec>>,
blocks: Vec<Arc<SignedBlindedBeaconBlock<T::EthSpec>>>,
) -> Result<usize, Error> {
let anchor_info = self
.store

View File

@ -9,6 +9,7 @@ pub mod block_reward;
mod block_times_cache;
mod block_verification;
pub mod builder;
pub mod canonical_head;
pub mod chain_config;
mod early_attester_cache;
mod errors;
@ -42,8 +43,8 @@ mod validator_pubkey_cache;
pub use self::beacon_chain::{
AttestationProcessingOutcome, BeaconChain, BeaconChainTypes, BeaconStore, ChainSegmentResult,
ForkChoiceError, HeadInfo, HeadSafetyStatus, ProduceBlockVerification, StateSkipConfig,
WhenSlotSkipped, INVALID_JUSTIFIED_PAYLOAD_SHUTDOWN_REASON, MAXIMUM_GOSSIP_CLOCK_DISPARITY,
ForkChoiceError, ProduceBlockVerification, StateSkipConfig, WhenSlotSkipped,
INVALID_JUSTIFIED_PAYLOAD_SHUTDOWN_REASON, MAXIMUM_GOSSIP_CLOCK_DISPARITY,
};
pub use self::beacon_snapshot::BeaconSnapshot;
pub use self::chain_config::ChainConfig;
@ -52,8 +53,10 @@ pub use self::historical_blocks::HistoricalBlockError;
pub use attestation_verification::Error as AttestationError;
pub use beacon_fork_choice_store::{BeaconForkChoiceStore, Error as ForkChoiceStoreError};
pub use block_verification::{BlockError, ExecutionPayloadError, GossipVerifiedBlock};
pub use canonical_head::{CachedHead, CanonicalHead, CanonicalHeadRwLock};
pub use eth1_chain::{Eth1Chain, Eth1ChainBackend};
pub use events::ServerSentEventHandler;
pub use fork_choice::ExecutionStatus;
pub use metrics::scrape_for_metrics;
pub use parking_lot;
pub use slot_clock;

View File

@ -51,9 +51,7 @@ async fn proposer_prep_service<T: BeaconChainTypes>(
executor.spawn(
async move {
if let Ok(current_slot) = inner_chain.slot() {
if let Err(e) = inner_chain
.prepare_beacon_proposer_async(current_slot)
.await
if let Err(e) = inner_chain.prepare_beacon_proposer(current_slot).await
{
error!(
inner_chain.log,

View File

@ -7,6 +7,7 @@ mod types;
use crate::beacon_chain::{BeaconChainTypes, FORK_CHOICE_DB_KEY};
use crate::persisted_fork_choice::{PersistedForkChoiceV1, PersistedForkChoiceV7};
use crate::types::ChainSpec;
use slog::{warn, Logger};
use std::path::Path;
use std::sync::Arc;
@ -21,6 +22,7 @@ pub fn migrate_schema<T: BeaconChainTypes>(
from: SchemaVersion,
to: SchemaVersion,
log: Logger,
spec: &ChainSpec,
) -> Result<(), StoreError> {
match (from, to) {
// Migrating from the current schema version to iself is always OK, a no-op.
@ -28,8 +30,8 @@ pub fn migrate_schema<T: BeaconChainTypes>(
// Upgrade across multiple versions by recursively migrating one step at a time.
(_, _) if from.as_u64() + 1 < to.as_u64() => {
let next = SchemaVersion(from.as_u64() + 1);
migrate_schema::<T>(db.clone(), datadir, from, next, log.clone())?;
migrate_schema::<T>(db, datadir, next, to, log)
migrate_schema::<T>(db.clone(), datadir, from, next, log.clone(), spec)?;
migrate_schema::<T>(db, datadir, next, to, log, spec)
}
//
@ -89,6 +91,7 @@ pub fn migrate_schema<T: BeaconChainTypes>(
migration_schema_v7::update_with_reinitialized_fork_choice::<T>(
&mut persisted_fork_choice_v7,
db.clone(),
spec,
)
.map_err(StoreError::SchemaMigrationError)?;
}

View File

@ -3,8 +3,7 @@ use crate::beacon_chain::BeaconChainTypes;
use crate::beacon_fork_choice_store::{PersistedForkChoiceStoreV1, PersistedForkChoiceStoreV7};
use crate::persisted_fork_choice::{PersistedForkChoiceV1, PersistedForkChoiceV7};
use crate::schema_change::types::{ProtoNodeV6, SszContainerV6, SszContainerV7};
use crate::types::{Checkpoint, Epoch, Hash256};
use crate::types::{EthSpec, Slot};
use crate::types::{ChainSpec, Checkpoint, Epoch, EthSpec, Hash256, Slot};
use crate::{BeaconForkChoiceStore, BeaconSnapshot};
use fork_choice::ForkChoice;
use proto_array::{core::ProtoNode, core::SszContainer, ProtoArrayForkChoice};
@ -25,6 +24,7 @@ four_byte_option_impl!(four_byte_option_usize, usize);
pub(crate) fn update_with_reinitialized_fork_choice<T: BeaconChainTypes>(
persisted_fork_choice: &mut PersistedForkChoiceV7,
db: Arc<HotColdDB<T::EthSpec, T::HotStore, T::ColdStore>>,
spec: &ChainSpec,
) -> Result<(), String> {
let anchor_block_root = persisted_fork_choice
.fork_choice_store
@ -39,7 +39,7 @@ pub(crate) fn update_with_reinitialized_fork_choice<T: BeaconChainTypes>(
.map_err(|e| format!("{:?}", e))?
.ok_or_else(|| "Missing anchor beacon state".to_string())?;
let snapshot = BeaconSnapshot {
beacon_block: anchor_block,
beacon_block: Arc::new(anchor_block),
beacon_block_root: anchor_block_root,
beacon_state: anchor_state,
};
@ -49,6 +49,10 @@ pub(crate) fn update_with_reinitialized_fork_choice<T: BeaconChainTypes>(
anchor_block_root,
&snapshot.beacon_block,
&snapshot.beacon_state,
// Don't provide the current slot here, just use what's in the store. We don't need to know
// the head here, plus it's nice to avoid mutating fork choice during this process.
None,
spec,
)
.map_err(|e| format!("{:?}", e))?;
persisted_fork_choice.fork_choice = fork_choice.to_persisted();

View File

@ -47,6 +47,12 @@ impl ShufflingCache {
}
}
impl Default for ShufflingCache {
fn default() -> Self {
Self::new()
}
}
/// Contains the shuffling IDs for a beacon block.
pub struct BlockShufflingIds {
pub current: AttestationShufflingId,

View File

@ -1,6 +1,7 @@
use crate::BeaconSnapshot;
use itertools::process_results;
use std::cmp;
use std::sync::Arc;
use std::time::Duration;
use types::{
beacon_state::CloneConfig, BeaconState, BlindedPayload, ChainSpec, Epoch, EthSpec, Hash256,
@ -33,7 +34,7 @@ impl<T: EthSpec> From<BeaconSnapshot<T>> for PreProcessingSnapshot<T> {
Self {
pre_state: snapshot.beacon_state,
beacon_state_root,
beacon_block: snapshot.beacon_block.into(),
beacon_block: snapshot.beacon_block.clone_as_blinded(),
beacon_block_root: snapshot.beacon_block_root,
}
}
@ -63,7 +64,7 @@ impl<T: EthSpec> CacheItem<T> {
Some(self.beacon_block.state_root()).filter(|_| self.pre_state.is_none());
PreProcessingSnapshot {
beacon_block: self.beacon_block.into(),
beacon_block: self.beacon_block.clone_as_blinded(),
beacon_block_root: self.beacon_block_root,
pre_state: self.pre_state.unwrap_or(self.beacon_state),
beacon_state_root,
@ -76,7 +77,7 @@ impl<T: EthSpec> CacheItem<T> {
Some(self.beacon_block.state_root()).filter(|_| self.pre_state.is_none());
PreProcessingSnapshot {
beacon_block: self.beacon_block.clone().into(),
beacon_block: self.beacon_block.clone_as_blinded(),
beacon_block_root: self.beacon_block_root,
pre_state: self
.pre_state
@ -116,7 +117,7 @@ pub enum StateAdvance<T: EthSpec> {
/// The item stored in the `SnapshotCache`.
pub struct CacheItem<T: EthSpec> {
beacon_block: SignedBeaconBlock<T>,
beacon_block: Arc<SignedBeaconBlock<T>>,
beacon_block_root: Hash256,
/// This state is equivalent to `self.beacon_block.state_root()`.
beacon_state: BeaconState<T>,
@ -185,7 +186,7 @@ impl<T: EthSpec> SnapshotCache<T> {
) {
let parent_root = snapshot.beacon_block.message().parent_root();
let item = CacheItem {
beacon_block: snapshot.beacon_block,
beacon_block: snapshot.beacon_block.clone(),
beacon_block_root: snapshot.beacon_block_root,
beacon_state: snapshot.beacon_state,
pre_state,
@ -384,7 +385,7 @@ mod test {
fn get_snapshot(i: u64) -> BeaconSnapshot<MainnetEthSpec> {
let spec = MainnetEthSpec::default_spec();
let beacon_state = get_harness().chain.head_beacon_state().unwrap();
let beacon_state = get_harness().chain.head_beacon_state_cloned();
let signed_beacon_block = SignedBeaconBlock::from_block(
BeaconBlock::empty(&spec),
@ -395,7 +396,7 @@ mod test {
BeaconSnapshot {
beacon_state,
beacon_block: signed_beacon_block,
beacon_block: Arc::new(signed_beacon_block),
beacon_block_root: Hash256::from_low_u64_be(i),
}
}

View File

@ -213,16 +213,14 @@ async fn state_advance_timer<T: BeaconChainTypes>(
let log = log.clone();
let beacon_chain = beacon_chain.clone();
let next_slot = current_slot + 1;
executor.spawn_blocking(
move || {
executor.spawn(
async move {
// Don't run fork choice during sync.
if beacon_chain.best_slot().map_or(true, |head_slot| {
head_slot + MAX_FORK_CHOICE_DISTANCE < current_slot
}) {
if beacon_chain.best_slot() + MAX_FORK_CHOICE_DISTANCE < current_slot {
return;
}
if let Err(e) = beacon_chain.fork_choice_at_slot(next_slot) {
if let Err(e) = beacon_chain.recompute_head_at_slot(next_slot).await {
warn!(
log,
"Error updating fork choice for next slot";
@ -231,17 +229,24 @@ async fn state_advance_timer<T: BeaconChainTypes>(
);
}
// Signal block proposal for the next slot (if it happens to be waiting).
if let Some(tx) = &beacon_chain.fork_choice_signal_tx {
if let Err(e) = tx.notify_fork_choice_complete(next_slot) {
warn!(
log,
"Error signalling fork choice waiter";
"error" => ?e,
"slot" => next_slot,
);
}
}
// Use a blocking task to avoid blocking the core executor whilst waiting for locks
// in `ForkChoiceSignalTx`.
beacon_chain.task_executor.clone().spawn_blocking(
move || {
// Signal block proposal for the next slot (if it happens to be waiting).
if let Some(tx) = &beacon_chain.fork_choice_signal_tx {
if let Err(e) = tx.notify_fork_choice_complete(next_slot) {
warn!(
log,
"Error signalling fork choice waiter";
"error" => ?e,
"slot" => next_slot,
);
}
}
},
"fork_choice_advance_signal_tx",
);
},
"fork_choice_advance",
);
@ -264,7 +269,7 @@ fn advance_head<T: BeaconChainTypes>(
//
// Fork-choice is not run *before* this function to avoid unnecessary calls whilst syncing.
{
let head_slot = beacon_chain.head_info()?.slot;
let head_slot = beacon_chain.best_slot();
// Don't run this when syncing or if lagging too far behind.
if head_slot + MAX_ADVANCE_DISTANCE < current_slot {
@ -275,7 +280,7 @@ fn advance_head<T: BeaconChainTypes>(
}
}
let head_root = beacon_chain.head_info()?.block_root;
let head_root = beacon_chain.head_beacon_block_root();
let (head_slot, head_state_root, mut state) = match beacon_chain
.snapshot_cache

View File

@ -515,13 +515,38 @@ where
}
pub fn get_current_state(&self) -> BeaconState<E> {
self.chain.head().unwrap().beacon_state
self.chain.head_beacon_state_cloned()
}
pub fn get_current_state_and_root(&self) -> (BeaconState<E>, Hash256) {
let head = self.chain.head().unwrap();
let head = self.chain.head_snapshot();
let state_root = head.beacon_state_root();
(head.beacon_state, state_root)
(
head.beacon_state.clone_with_only_committee_caches(),
state_root,
)
}
pub fn head_slot(&self) -> Slot {
self.chain.canonical_head.cached_head().head_slot()
}
pub fn head_block_root(&self) -> Hash256 {
self.chain.canonical_head.cached_head().head_block_root()
}
pub fn finalized_checkpoint(&self) -> Checkpoint {
self.chain
.canonical_head
.cached_head()
.finalized_checkpoint()
}
pub fn justified_checkpoint(&self) -> Checkpoint {
self.chain
.canonical_head
.cached_head()
.justified_checkpoint()
}
pub fn get_current_slot(&self) -> Slot {
@ -565,7 +590,7 @@ where
state.get_block_root(slot).unwrap() == state.get_block_root(slot - 1).unwrap()
}
pub fn make_block(
pub async fn make_block(
&self,
mut state: BeaconState<E>,
slot: Slot,
@ -599,6 +624,7 @@ where
Some(graffiti),
ProduceBlockVerification::VerifyRandao,
)
.await
.unwrap();
let signed_block = block.sign(
@ -613,7 +639,7 @@ where
/// Useful for the `per_block_processing` tests. Creates a block, and returns the state after
/// caches are built but before the generated block is processed.
pub fn make_block_return_pre_state(
pub async fn make_block_return_pre_state(
&self,
mut state: BeaconState<E>,
slot: Slot,
@ -649,6 +675,7 @@ where
Some(graffiti),
ProduceBlockVerification::VerifyRandao,
)
.await
.unwrap();
let signed_block = block.sign(
@ -1098,11 +1125,11 @@ where
let mut attestation_2 = attestation_1.clone();
attestation_2.data.index += 1;
let fork = self.chain.canonical_head.cached_head().head_fork();
for attestation in &mut [&mut attestation_1, &mut attestation_2] {
for &i in &attestation.attesting_indices {
let sk = &self.validator_keypairs[i as usize].sk;
let fork = self.chain.head_info().unwrap().fork;
let genesis_validators_root = self.chain.genesis_validators_root;
let domain = self.chain.spec.get_domain(
@ -1156,11 +1183,11 @@ where
attestation_2.data.index += 1;
let fork = self.chain.canonical_head.cached_head().head_fork();
for attestation in &mut [&mut attestation_1, &mut attestation_2] {
for &i in &attestation.attesting_indices {
let sk = &self.validator_keypairs[i as usize].sk;
let fork = self.chain.head_info().unwrap().fork;
let genesis_validators_root = self.chain.genesis_validators_root;
let domain = self.chain.spec.get_domain(
@ -1182,19 +1209,14 @@ where
}
pub fn make_proposer_slashing(&self, validator_index: u64) -> ProposerSlashing {
let mut block_header_1 = self
.chain
.head_beacon_block()
.unwrap()
.message()
.block_header();
let mut block_header_1 = self.chain.head_beacon_block().message().block_header();
block_header_1.proposer_index = validator_index;
let mut block_header_2 = block_header_1.clone();
block_header_2.state_root = Hash256::zero();
let sk = &self.validator_keypairs[validator_index as usize].sk;
let fork = self.chain.head_info().unwrap().fork;
let fork = self.chain.canonical_head.cached_head().head_fork();
let genesis_validators_root = self.chain.genesis_validators_root;
let mut signed_block_headers = vec![block_header_1, block_header_2]
@ -1212,7 +1234,7 @@ where
pub fn make_voluntary_exit(&self, validator_index: u64, epoch: Epoch) -> SignedVoluntaryExit {
let sk = &self.validator_keypairs[validator_index as usize].sk;
let fork = self.chain.head_info().unwrap().fork;
let fork = self.chain.canonical_head.cached_head().head_fork();
let genesis_validators_root = self.chain.genesis_validators_root;
VoluntaryExit {
@ -1235,7 +1257,7 @@ where
/// Create a new block, apply `block_modifier` to it, sign it and return it.
///
/// The state returned is a pre-block state at the same slot as the produced block.
pub fn make_block_with_modifier(
pub async fn make_block_with_modifier(
&self,
state: BeaconState<E>,
slot: Slot,
@ -1244,7 +1266,7 @@ where
assert_ne!(slot, 0, "can't produce a block at slot 0");
assert!(slot >= state.slot());
let (block, state) = self.make_block_return_pre_state(state, slot);
let (block, state) = self.make_block_return_pre_state(state, slot).await;
let (mut block, _) = block.deconstruct();
block_modifier(&mut block);
@ -1332,23 +1354,25 @@ where
(deposits, state)
}
pub fn process_block(
pub async fn process_block(
&self,
slot: Slot,
block: SignedBeaconBlock<E>,
) -> Result<SignedBeaconBlockHash, BlockError<E>> {
self.set_current_slot(slot);
let block_hash: SignedBeaconBlockHash = self.chain.process_block(block)?.into();
self.chain.fork_choice()?;
let block_hash: SignedBeaconBlockHash =
self.chain.process_block(Arc::new(block)).await?.into();
self.chain.recompute_head_at_current_slot().await?;
Ok(block_hash)
}
pub fn process_block_result(
pub async fn process_block_result(
&self,
block: SignedBeaconBlock<E>,
) -> Result<SignedBeaconBlockHash, BlockError<E>> {
let block_hash: SignedBeaconBlockHash = self.chain.process_block(block)?.into();
self.chain.fork_choice().unwrap();
let block_hash: SignedBeaconBlockHash =
self.chain.process_block(Arc::new(block)).await?.into();
self.chain.recompute_head_at_current_slot().await?;
Ok(block_hash)
}
@ -1403,14 +1427,14 @@ where
self.chain.slot_clock.set_slot(slot.into());
}
pub fn add_block_at_slot(
pub async fn add_block_at_slot(
&self,
slot: Slot,
state: BeaconState<E>,
) -> Result<(SignedBeaconBlockHash, SignedBeaconBlock<E>, BeaconState<E>), BlockError<E>> {
self.set_current_slot(slot);
let (block, new_state) = self.make_block(state, slot);
let block_hash = self.process_block(slot, block.clone())?;
let (block, new_state) = self.make_block(state, slot).await;
let block_hash = self.process_block(slot, block.clone()).await?;
Ok((block_hash, block, new_state))
}
@ -1427,19 +1451,19 @@ where
self.process_attestations(attestations);
}
pub fn add_attested_block_at_slot(
pub async fn add_attested_block_at_slot(
&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)?;
let (block_hash, block, state) = self.add_block_at_slot(slot, state).await?;
self.attest_block(&state, state_root, block_hash, &block, validators);
Ok((block_hash, state))
}
pub fn add_attested_blocks_at_slots(
pub async fn add_attested_blocks_at_slots(
&self,
state: BeaconState<E>,
state_root: Hash256,
@ -1448,9 +1472,10 @@ where
) -> AddBlocksResult<E> {
assert!(!slots.is_empty());
self.add_attested_blocks_at_slots_given_lbh(state, state_root, slots, validators, None)
.await
}
fn add_attested_blocks_at_slots_given_lbh(
async fn add_attested_blocks_at_slots_given_lbh(
&self,
mut state: BeaconState<E>,
state_root: Hash256,
@ -1467,6 +1492,7 @@ where
for slot in slots {
let (block_hash, new_state) = self
.add_attested_block_at_slot(*slot, state, state_root, validators)
.await
.unwrap();
state = new_state;
block_hash_from_slot.insert(*slot, block_hash);
@ -1488,7 +1514,7 @@ where
/// epoch at a time.
///
/// Chains is a vec of `(state, slots, validators)` tuples.
pub fn add_blocks_on_multiple_chains(
pub async fn add_blocks_on_multiple_chains(
&self,
chains: Vec<(BeaconState<E>, Vec<Slot>, Vec<usize>)>,
) -> Vec<AddBlocksResult<E>> {
@ -1547,7 +1573,8 @@ where
&epoch_slots,
&validators,
Some(head_block),
);
)
.await;
block_hashes.extend(new_block_hashes);
state_hashes.extend(new_state_hashes);
@ -1596,18 +1623,18 @@ where
/// Deprecated: Use make_block() instead
///
/// Returns a newly created block, signed by the proposer for the given slot.
pub fn build_block(
pub async fn build_block(
&self,
state: BeaconState<E>,
slot: Slot,
_block_strategy: BlockStrategy,
) -> (SignedBeaconBlock<E>, BeaconState<E>) {
self.make_block(state, slot)
self.make_block(state, slot).await
}
/// Uses `Self::extend_chain` to build the chain out to the `target_slot`.
pub fn extend_to_slot(&self, target_slot: Slot) -> Hash256 {
if self.chain.slot().unwrap() == self.chain.head_info().unwrap().slot {
pub async fn extend_to_slot(&self, target_slot: Slot) -> Hash256 {
if self.chain.slot().unwrap() == self.chain.canonical_head.cached_head().head_slot() {
self.advance_slot();
}
@ -1618,7 +1645,7 @@ where
.checked_add(1)
.unwrap();
self.extend_slots(num_slots)
self.extend_slots(num_slots).await
}
/// Uses `Self::extend_chain` to `num_slots` blocks.
@ -1627,8 +1654,8 @@ where
///
/// - BlockStrategy::OnCanonicalHead,
/// - AttestationStrategy::AllValidators,
pub fn extend_slots(&self, num_slots: usize) -> Hash256 {
if self.chain.slot().unwrap() == self.chain.head_info().unwrap().slot {
pub async fn extend_slots(&self, num_slots: usize) -> Hash256 {
if self.chain.slot().unwrap() == self.chain.canonical_head.cached_head().head_slot() {
self.advance_slot();
}
@ -1637,6 +1664,7 @@ where
BlockStrategy::OnCanonicalHead,
AttestationStrategy::AllValidators,
)
.await
}
/// Deprecated: Use add_attested_blocks_at_slots() instead
@ -1650,7 +1678,7 @@ where
///
/// The `attestation_strategy` dictates which validators will attest to the newly created
/// blocks.
pub fn extend_chain(
pub async fn extend_chain(
&self,
num_blocks: usize,
block_strategy: BlockStrategy,
@ -1685,8 +1713,9 @@ where
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, state_root, &slots, &validators);
let (_, _, last_produced_block_hash, _) = self
.add_attested_blocks_at_slots(state, state_root, &slots, &validators)
.await;
last_produced_block_hash.into()
}
@ -1700,41 +1729,40 @@ where
/// then built `faulty_fork_blocks`.
///
/// Returns `(honest_head, faulty_head)`, the roots of the blocks at the top of each chain.
pub fn generate_two_forks_by_skipping_a_block(
pub async fn generate_two_forks_by_skipping_a_block(
&self,
honest_validators: &[usize],
faulty_validators: &[usize],
honest_fork_blocks: usize,
faulty_fork_blocks: usize,
) -> (Hash256, Hash256) {
let initial_head_slot = self
.chain
.head()
.expect("should get head")
.beacon_block
.slot();
let initial_head_slot = self.chain.head_snapshot().beacon_block.slot();
// Move to the next slot so we may produce some more blocks on the head.
self.advance_slot();
// Extend the chain with blocks where only honest validators agree.
let honest_head = self.extend_chain(
honest_fork_blocks,
BlockStrategy::OnCanonicalHead,
AttestationStrategy::SomeValidators(honest_validators.to_vec()),
);
let honest_head = self
.extend_chain(
honest_fork_blocks,
BlockStrategy::OnCanonicalHead,
AttestationStrategy::SomeValidators(honest_validators.to_vec()),
)
.await;
// Go back to the last block where all agreed, and build blocks upon it where only faulty nodes
// agree.
let faulty_head = self.extend_chain(
faulty_fork_blocks,
BlockStrategy::ForkCanonicalChainAt {
previous_slot: initial_head_slot,
// `initial_head_slot + 2` means one slot is skipped.
first_slot: initial_head_slot + 2,
},
AttestationStrategy::SomeValidators(faulty_validators.to_vec()),
);
let faulty_head = self
.extend_chain(
faulty_fork_blocks,
BlockStrategy::ForkCanonicalChainAt {
previous_slot: initial_head_slot,
// `initial_head_slot + 2` means one slot is skipped.
first_slot: initial_head_slot + 2,
},
AttestationStrategy::SomeValidators(faulty_validators.to_vec()),
)
.await;
assert_ne!(honest_head, faulty_head, "forks should be distinct");

View File

@ -3,6 +3,7 @@
use beacon_chain::test_utils::{AttestationStrategy, BeaconChainHarness, BlockStrategy};
use beacon_chain::{StateSkipConfig, WhenSlotSkipped};
use lazy_static::lazy_static;
use std::sync::Arc;
use tree_hash::TreeHash;
use types::{AggregateSignature, EthSpec, Keypair, MainnetEthSpec, RelativeEpoch, Slot};
@ -17,8 +18,8 @@ lazy_static! {
/// attestation at each slot from genesis through to three epochs past the head.
///
/// It checks the produced attestation against some locally computed values.
#[test]
fn produces_attestations() {
#[tokio::test]
async fn produces_attestations() {
let num_blocks_produced = MainnetEthSpec::slots_per_epoch() * 4;
let additional_slots_tested = MainnetEthSpec::slots_per_epoch() * 3;
@ -37,11 +38,13 @@ fn produces_attestations() {
if slot > 0 && slot <= num_blocks_produced {
harness.advance_slot();
harness.extend_chain(
1,
BlockStrategy::OnCanonicalHead,
AttestationStrategy::AllValidators,
);
harness
.extend_chain(
1,
BlockStrategy::OnCanonicalHead,
AttestationStrategy::AllValidators,
)
.await;
}
let slot = Slot::from(slot);
@ -129,10 +132,20 @@ fn produces_attestations() {
assert_eq!(data.target.root, target_root, "bad target root");
let early_attestation = {
let proto_block = chain.fork_choice.read().get_block(&block_root).unwrap();
let proto_block = chain
.canonical_head
.fork_choice_read_lock()
.get_block(&block_root)
.unwrap();
chain
.early_attester_cache
.add_head_block(block_root, block.clone(), proto_block, &state, &chain.spec)
.add_head_block(
block_root,
Arc::new(block.clone()),
proto_block,
&state,
&chain.spec,
)
.unwrap();
chain
.early_attester_cache
@ -151,8 +164,8 @@ fn produces_attestations() {
/// Ensures that the early attester cache wont create an attestation to a block in a later slot than
/// the one requested.
#[test]
fn early_attester_cache_old_request() {
#[tokio::test]
async fn early_attester_cache_old_request() {
let harness = BeaconChainHarness::builder(MainnetEthSpec)
.default_spec()
.keypairs(KEYPAIRS[..].to_vec())
@ -162,18 +175,20 @@ fn early_attester_cache_old_request() {
harness.advance_slot();
harness.extend_chain(
2,
BlockStrategy::OnCanonicalHead,
AttestationStrategy::AllValidators,
);
harness
.extend_chain(
2,
BlockStrategy::OnCanonicalHead,
AttestationStrategy::AllValidators,
)
.await;
let head = harness.chain.head().unwrap();
let head = harness.chain.head_snapshot();
assert_eq!(head.beacon_block.slot(), 2);
let head_proto_block = harness
.chain
.fork_choice
.read()
.canonical_head
.fork_choice_read_lock()
.get_block(&head.beacon_block_root)
.unwrap();

View File

@ -56,7 +56,7 @@ fn get_harness(validator_count: usize) -> BeaconChainHarness<EphemeralHarnessTyp
fn get_valid_unaggregated_attestation<T: BeaconChainTypes>(
chain: &BeaconChain<T>,
) -> (Attestation<T::EthSpec>, usize, usize, SecretKey, SubnetId) {
let head = chain.head().expect("should get head");
let head = chain.head_snapshot();
let current_slot = chain.slot().expect("should get slot");
let mut valid_attestation = chain
@ -106,7 +106,8 @@ fn get_valid_aggregated_attestation<T: BeaconChainTypes>(
chain: &BeaconChain<T>,
aggregate: Attestation<T::EthSpec>,
) -> (SignedAggregateAndProof<T::EthSpec>, usize, SecretKey) {
let state = &chain.head().expect("should get head").beacon_state;
let head = chain.head_snapshot();
let state = &head.beacon_state;
let current_slot = chain.slot().expect("should get slot");
let committee = state
@ -155,7 +156,8 @@ fn get_non_aggregator<T: BeaconChainTypes>(
chain: &BeaconChain<T>,
aggregate: &Attestation<T::EthSpec>,
) -> (usize, SecretKey) {
let state = &chain.head().expect("should get head").beacon_state;
let head = chain.head_snapshot();
let state = &head.beacon_state;
let current_slot = chain.slot().expect("should get slot");
let committee = state
@ -213,15 +215,17 @@ struct GossipTester {
}
impl GossipTester {
pub fn new() -> Self {
pub async fn new() -> Self {
let harness = get_harness(VALIDATOR_COUNT);
// Extend the chain out a few epochs so we have some chain depth to play with.
harness.extend_chain(
MainnetEthSpec::slots_per_epoch() as usize * 3 - 1,
BlockStrategy::OnCanonicalHead,
AttestationStrategy::AllValidators,
);
harness
.extend_chain(
MainnetEthSpec::slots_per_epoch() as usize * 3 - 1,
BlockStrategy::OnCanonicalHead,
AttestationStrategy::AllValidators,
)
.await;
// Advance into a slot where there have not been blocks or attestations produced.
harness.advance_slot();
@ -395,9 +399,10 @@ impl GossipTester {
}
}
/// Tests verification of `SignedAggregateAndProof` from the gossip network.
#[test]
fn aggregated_gossip_verification() {
#[tokio::test]
async fn aggregated_gossip_verification() {
GossipTester::new()
.await
/*
* The following two tests ensure:
*
@ -511,8 +516,7 @@ fn aggregated_gossip_verification() {
let committee_len = tester
.harness
.chain
.head()
.unwrap()
.head_snapshot()
.beacon_state
.get_beacon_committee(tester.slot(), a.message.aggregate.data.index)
.expect("should get committees")
@ -612,7 +616,7 @@ fn aggregated_gossip_verification() {
tester.valid_aggregate.message.aggregate.clone(),
None,
&sk,
&chain.head_info().unwrap().fork,
&chain.canonical_head.cached_head().head_fork(),
chain.genesis_validators_root,
&chain.spec,
)
@ -669,9 +673,10 @@ fn aggregated_gossip_verification() {
}
/// Tests the verification conditions for an unaggregated attestation on the gossip network.
#[test]
fn unaggregated_gossip_verification() {
#[tokio::test]
async fn unaggregated_gossip_verification() {
GossipTester::new()
.await
/*
* The following test ensures:
*
@ -684,8 +689,7 @@ fn unaggregated_gossip_verification() {
a.data.index = tester
.harness
.chain
.head()
.unwrap()
.head_snapshot()
.beacon_state
.get_committee_count_at_slot(a.data.slot)
.unwrap()
@ -924,16 +928,18 @@ fn unaggregated_gossip_verification() {
/// Ensures that an attestation that skips epochs can still be processed.
///
/// This also checks that we can do a state lookup if we don't get a hit from the shuffling cache.
#[test]
fn attestation_that_skips_epochs() {
#[tokio::test]
async fn attestation_that_skips_epochs() {
let harness = get_harness(VALIDATOR_COUNT);
// Extend the chain out a few epochs so we have some chain depth to play with.
harness.extend_chain(
MainnetEthSpec::slots_per_epoch() as usize * 3 + 1,
BlockStrategy::OnCanonicalHead,
AttestationStrategy::SomeValidators(vec![]),
);
harness
.extend_chain(
MainnetEthSpec::slots_per_epoch() as usize * 3 + 1,
BlockStrategy::OnCanonicalHead,
AttestationStrategy::SomeValidators(vec![]),
)
.await;
let current_slot = harness.chain.slot().expect("should get slot");
let current_epoch = harness.chain.epoch().expect("should get epoch");
@ -992,16 +998,18 @@ fn attestation_that_skips_epochs() {
.expect("should gossip verify attestation that skips slots");
}
#[test]
fn attestation_to_finalized_block() {
#[tokio::test]
async fn attestation_to_finalized_block() {
let harness = get_harness(VALIDATOR_COUNT);
// Extend the chain out a few epochs so we have some chain depth to play with.
harness.extend_chain(
MainnetEthSpec::slots_per_epoch() as usize * 4 + 1,
BlockStrategy::OnCanonicalHead,
AttestationStrategy::AllValidators,
);
harness
.extend_chain(
MainnetEthSpec::slots_per_epoch() as usize * 4 + 1,
BlockStrategy::OnCanonicalHead,
AttestationStrategy::AllValidators,
)
.await;
let finalized_checkpoint = harness
.chain
@ -1067,16 +1075,18 @@ fn attestation_to_finalized_block() {
.contains(earlier_block_root));
}
#[test]
fn verify_aggregate_for_gossip_doppelganger_detection() {
#[tokio::test]
async fn verify_aggregate_for_gossip_doppelganger_detection() {
let harness = get_harness(VALIDATOR_COUNT);
// Extend the chain out a few epochs so we have some chain depth to play with.
harness.extend_chain(
MainnetEthSpec::slots_per_epoch() as usize * 3 - 1,
BlockStrategy::OnCanonicalHead,
AttestationStrategy::AllValidators,
);
harness
.extend_chain(
MainnetEthSpec::slots_per_epoch() as usize * 3 - 1,
BlockStrategy::OnCanonicalHead,
AttestationStrategy::AllValidators,
)
.await;
// Advance into a slot where there have not been blocks or attestations produced.
harness.advance_slot();
@ -1124,16 +1134,18 @@ fn verify_aggregate_for_gossip_doppelganger_detection() {
.expect("should check if gossip aggregator was observed"));
}
#[test]
fn verify_attestation_for_gossip_doppelganger_detection() {
#[tokio::test]
async fn verify_attestation_for_gossip_doppelganger_detection() {
let harness = get_harness(VALIDATOR_COUNT);
// Extend the chain out a few epochs so we have some chain depth to play with.
harness.extend_chain(
MainnetEthSpec::slots_per_epoch() as usize * 3 - 1,
BlockStrategy::OnCanonicalHead,
AttestationStrategy::AllValidators,
);
harness
.extend_chain(
MainnetEthSpec::slots_per_epoch() as usize * 3 - 1,
BlockStrategy::OnCanonicalHead,
AttestationStrategy::AllValidators,
)
.await;
// Advance into a slot where there have not been blocks or attestations produced.
harness.advance_slot();

File diff suppressed because it is too large Load Diff

View File

@ -27,11 +27,11 @@ fn verify_execution_payload_chain<T: EthSpec>(chain: &[FullPayload<T>]) {
}
}
#[test]
#[tokio::test]
// TODO(merge): This isn't working cause the non-zero values in `initialize_beacon_state_from_eth1`
// are causing failed lookups to the execution node. I need to come back to this.
#[should_panic]
fn merge_with_terminal_block_hash_override() {
async fn merge_with_terminal_block_hash_override() {
let altair_fork_epoch = Epoch::new(0);
let bellatrix_fork_epoch = Epoch::new(0);
@ -70,8 +70,7 @@ fn merge_with_terminal_block_hash_override() {
assert!(
harness
.chain
.head()
.unwrap()
.head_snapshot()
.beacon_block
.as_merge()
.is_ok(),
@ -80,9 +79,9 @@ fn merge_with_terminal_block_hash_override() {
let mut execution_payloads = vec![];
for i in 0..E::slots_per_epoch() * 3 {
harness.extend_slots(1);
harness.extend_slots(1).await;
let block = harness.chain.head().unwrap().beacon_block;
let block = &harness.chain.head_snapshot().beacon_block;
let execution_payload = block.message().body().execution_payload().unwrap().clone();
if i == 0 {
@ -94,8 +93,8 @@ fn merge_with_terminal_block_hash_override() {
verify_execution_payload_chain(execution_payloads.as_slice());
}
#[test]
fn base_altair_merge_with_terminal_block_after_fork() {
#[tokio::test]
async fn base_altair_merge_with_terminal_block_after_fork() {
let altair_fork_epoch = Epoch::new(4);
let altair_fork_slot = altair_fork_epoch.start_slot(E::slots_per_epoch());
let bellatrix_fork_epoch = Epoch::new(8);
@ -118,15 +117,15 @@ fn base_altair_merge_with_terminal_block_after_fork() {
* Start with the base fork.
*/
assert!(harness.chain.head().unwrap().beacon_block.as_base().is_ok());
assert!(harness.chain.head_snapshot().beacon_block.as_base().is_ok());
/*
* Do the Altair fork.
*/
harness.extend_to_slot(altair_fork_slot);
harness.extend_to_slot(altair_fork_slot).await;
let altair_head = harness.chain.head().unwrap().beacon_block;
let altair_head = &harness.chain.head_snapshot().beacon_block;
assert!(altair_head.as_altair().is_ok());
assert_eq!(altair_head.slot(), altair_fork_slot);
@ -134,9 +133,9 @@ fn base_altair_merge_with_terminal_block_after_fork() {
* Do the merge fork, without a terminal PoW block.
*/
harness.extend_to_slot(merge_fork_slot);
harness.extend_to_slot(merge_fork_slot).await;
let merge_head = harness.chain.head().unwrap().beacon_block;
let merge_head = &harness.chain.head_snapshot().beacon_block;
assert!(merge_head.as_merge().is_ok());
assert_eq!(merge_head.slot(), merge_fork_slot);
assert_eq!(
@ -148,9 +147,9 @@ fn base_altair_merge_with_terminal_block_after_fork() {
* Next merge block shouldn't include an exec payload.
*/
harness.extend_slots(1);
harness.extend_slots(1).await;
let one_after_merge_head = harness.chain.head().unwrap().beacon_block;
let one_after_merge_head = &harness.chain.head_snapshot().beacon_block;
assert_eq!(
*one_after_merge_head
.message()
@ -175,9 +174,9 @@ fn base_altair_merge_with_terminal_block_after_fork() {
*/
for _ in 0..4 {
harness.extend_slots(1);
harness.extend_slots(1).await;
let block = harness.chain.head().unwrap().beacon_block;
let block = &harness.chain.head_snapshot().beacon_block;
execution_payloads.push(block.message().body().execution_payload().unwrap().clone());
}

View File

@ -46,18 +46,20 @@ fn get_harness(store: Arc<HotColdDB>, validator_count: usize) -> TestHarness {
harness
}
#[test]
fn voluntary_exit() {
#[tokio::test]
async fn voluntary_exit() {
let db_path = tempdir().unwrap();
let store = get_store(&db_path);
let harness = get_harness(store.clone(), VALIDATOR_COUNT);
let spec = &harness.chain.spec.clone();
harness.extend_chain(
(E::slots_per_epoch() * (spec.shard_committee_period + 1)) as usize,
BlockStrategy::OnCanonicalHead,
AttestationStrategy::AllValidators,
);
harness
.extend_chain(
(E::slots_per_epoch() * (spec.shard_committee_period + 1)) as usize,
BlockStrategy::OnCanonicalHead,
AttestationStrategy::AllValidators,
)
.await;
let validator_index1 = VALIDATOR_COUNT - 1;
let validator_index2 = VALIDATOR_COUNT - 2;

View File

@ -2,8 +2,8 @@
use beacon_chain::{
test_utils::{BeaconChainHarness, EphemeralHarnessType},
BeaconChainError, BlockError, ExecutionPayloadError, HeadInfo, StateSkipConfig,
WhenSlotSkipped, INVALID_JUSTIFIED_PAYLOAD_SHUTDOWN_REASON,
BeaconChainError, BlockError, ExecutionPayloadError, StateSkipConfig, WhenSlotSkipped,
INVALID_JUSTIFIED_PAYLOAD_SHUTDOWN_REASON,
};
use execution_layer::{
json_structures::{JsonForkChoiceStateV1, JsonPayloadAttributesV1},
@ -12,6 +12,7 @@ use execution_layer::{
use fork_choice::{Error as ForkChoiceError, InvalidationOperation, PayloadVerificationStatus};
use proto_array::{Error as ProtoArrayError, ExecutionStatus};
use slot_clock::SlotClock;
use std::sync::Arc;
use std::time::Duration;
use task_executor::ShutdownReason;
use tree_hash::TreeHash;
@ -84,19 +85,19 @@ impl InvalidPayloadRig {
fn execution_status(&self, block_root: Hash256) -> ExecutionStatus {
self.harness
.chain
.fork_choice
.read()
.canonical_head
.fork_choice_read_lock()
.get_block(&block_root)
.unwrap()
.execution_status
}
fn fork_choice(&self) {
self.harness.chain.fork_choice().unwrap();
}
fn head_info(&self) -> HeadInfo {
self.harness.chain.head_info().unwrap()
async fn recompute_head(&self) {
self.harness
.chain
.recompute_head_at_current_slot()
.await
.unwrap();
}
fn previous_forkchoice_update_params(&self) -> (ForkChoiceState, PayloadAttributes) {
@ -142,22 +143,24 @@ impl InvalidPayloadRig {
.block_hash
}
fn build_blocks(&mut self, num_blocks: u64, is_valid: Payload) -> Vec<Hash256> {
(0..num_blocks)
.map(|_| self.import_block(is_valid.clone()))
.collect()
async fn build_blocks(&mut self, num_blocks: u64, is_valid: Payload) -> Vec<Hash256> {
let mut roots = Vec::with_capacity(num_blocks as usize);
for _ in 0..num_blocks {
roots.push(self.import_block(is_valid.clone()).await);
}
roots
}
fn move_to_first_justification(&mut self, is_valid: Payload) {
async fn move_to_first_justification(&mut self, is_valid: Payload) {
let slots_till_justification = E::slots_per_epoch() * 3;
self.build_blocks(slots_till_justification, is_valid);
self.build_blocks(slots_till_justification, is_valid).await;
let justified_checkpoint = self.head_info().current_justified_checkpoint;
let justified_checkpoint = self.harness.justified_checkpoint();
assert_eq!(justified_checkpoint.epoch, 2);
}
/// Import a block while setting the newPayload and forkchoiceUpdated responses to `is_valid`.
fn import_block(&mut self, is_valid: Payload) -> Hash256 {
async fn import_block(&mut self, is_valid: Payload) -> Hash256 {
self.import_block_parametric(is_valid, is_valid, |error| {
matches!(
error,
@ -166,6 +169,7 @@ impl InvalidPayloadRig {
)
)
})
.await
}
fn block_root_at_slot(&self, slot: Slot) -> Option<Hash256> {
@ -178,13 +182,13 @@ impl InvalidPayloadRig {
fn validate_manually(&self, block_root: Hash256) {
self.harness
.chain
.fork_choice
.write()
.canonical_head
.fork_choice_write_lock()
.on_valid_execution_payload(block_root)
.unwrap();
}
fn import_block_parametric<F: Fn(&BlockError<E>) -> bool>(
async fn import_block_parametric<F: Fn(&BlockError<E>) -> bool>(
&mut self,
new_payload_response: Payload,
forkchoice_response: Payload,
@ -192,10 +196,10 @@ impl InvalidPayloadRig {
) -> Hash256 {
let mock_execution_layer = self.harness.mock_execution_layer.as_ref().unwrap();
let head = self.harness.chain.head().unwrap();
let state = head.beacon_state;
let head = self.harness.chain.head_snapshot();
let state = head.beacon_state.clone_with_only_committee_caches();
let slot = state.slot() + 1;
let (block, post_state) = self.harness.make_block(state, slot);
let (block, post_state) = self.harness.make_block(state, slot).await;
let block_root = block.canonical_root();
let set_new_payload = |payload: Payload| match payload {
@ -249,7 +253,11 @@ impl InvalidPayloadRig {
} else {
mock_execution_layer.server.full_payload_verification();
}
let root = self.harness.process_block(slot, block.clone()).unwrap();
let root = self
.harness
.process_block(slot, block.clone())
.await
.unwrap();
if self.enable_attestations {
let all_validators: Vec<usize> = (0..VALIDATOR_COUNT).collect();
@ -294,7 +302,7 @@ impl InvalidPayloadRig {
set_new_payload(new_payload_response);
set_forkchoice_updated(forkchoice_response);
match self.harness.process_block(slot, block) {
match self.harness.process_block(slot, block).await {
Err(error) if evaluate_error(&error) => (),
Err(other) => {
panic!("evaluate_error returned false with {:?}", other)
@ -309,8 +317,12 @@ impl InvalidPayloadRig {
}
};
let block_in_forkchoice =
self.harness.chain.fork_choice.read().get_block(&block_root);
let block_in_forkchoice = self
.harness
.chain
.canonical_head
.fork_choice_read_lock()
.get_block(&block_root);
if let Payload::Invalid { .. } = new_payload_response {
// A block found to be immediately invalid should not end up in fork choice.
assert_eq!(block_in_forkchoice, None);
@ -333,106 +345,111 @@ impl InvalidPayloadRig {
block_root
}
fn invalidate_manually(&self, block_root: Hash256) {
async fn invalidate_manually(&self, block_root: Hash256) {
self.harness
.chain
.process_invalid_execution_payload(&InvalidationOperation::InvalidateOne { block_root })
.await
.unwrap();
}
}
/// Simple test of the different import types.
#[test]
fn valid_invalid_syncing() {
#[tokio::test]
async fn valid_invalid_syncing() {
let mut rig = InvalidPayloadRig::new();
rig.move_to_terminal_block();
rig.import_block(Payload::Valid);
rig.import_block(Payload::Valid).await;
rig.import_block(Payload::Invalid {
latest_valid_hash: None,
});
rig.import_block(Payload::Syncing);
})
.await;
rig.import_block(Payload::Syncing).await;
}
/// Ensure that an invalid payload can invalidate its parent too (given the right
/// `latest_valid_hash`.
#[test]
fn invalid_payload_invalidates_parent() {
#[tokio::test]
async fn invalid_payload_invalidates_parent() {
let mut rig = InvalidPayloadRig::new().enable_attestations();
rig.move_to_terminal_block();
rig.import_block(Payload::Valid); // Import a valid transition block.
rig.move_to_first_justification(Payload::Syncing);
rig.import_block(Payload::Valid).await; // Import a valid transition block.
rig.move_to_first_justification(Payload::Syncing).await;
let roots = vec![
rig.import_block(Payload::Syncing),
rig.import_block(Payload::Syncing),
rig.import_block(Payload::Syncing),
rig.import_block(Payload::Syncing).await,
rig.import_block(Payload::Syncing).await,
rig.import_block(Payload::Syncing).await,
];
let latest_valid_hash = rig.block_hash(roots[0]);
rig.import_block(Payload::Invalid {
latest_valid_hash: Some(latest_valid_hash),
});
})
.await;
assert!(rig.execution_status(roots[0]).is_valid_and_post_bellatrix());
assert!(rig.execution_status(roots[1]).is_invalid());
assert!(rig.execution_status(roots[2]).is_invalid());
assert_eq!(rig.head_info().block_root, roots[0]);
assert_eq!(rig.harness.head_block_root(), roots[0]);
}
/// Test invalidation of a payload via the fork choice updated message.
///
/// The `invalid_payload` argument determines the type of invalid payload: `Invalid`,
/// `InvalidBlockHash`, etc, taking the `latest_valid_hash` as an argument.
fn immediate_forkchoice_update_invalid_test(
async fn immediate_forkchoice_update_invalid_test(
invalid_payload: impl FnOnce(Option<ExecutionBlockHash>) -> Payload,
) {
let mut rig = InvalidPayloadRig::new().enable_attestations();
rig.move_to_terminal_block();
rig.import_block(Payload::Valid); // Import a valid transition block.
rig.move_to_first_justification(Payload::Syncing);
rig.import_block(Payload::Valid).await; // Import a valid transition block.
rig.move_to_first_justification(Payload::Syncing).await;
let valid_head_root = rig.import_block(Payload::Valid);
let valid_head_root = rig.import_block(Payload::Valid).await;
let latest_valid_hash = Some(rig.block_hash(valid_head_root));
// Import a block which returns syncing when supplied via newPayload, and then
// invalid when the forkchoice update is sent.
rig.import_block_parametric(Payload::Syncing, invalid_payload(latest_valid_hash), |_| {
false
});
})
.await;
// The head should be the latest valid block.
assert_eq!(rig.head_info().block_root, valid_head_root);
assert_eq!(rig.harness.head_block_root(), valid_head_root);
}
#[test]
fn immediate_forkchoice_update_payload_invalid() {
#[tokio::test]
async fn immediate_forkchoice_update_payload_invalid() {
immediate_forkchoice_update_invalid_test(|latest_valid_hash| Payload::Invalid {
latest_valid_hash,
})
.await
}
#[test]
fn immediate_forkchoice_update_payload_invalid_block_hash() {
immediate_forkchoice_update_invalid_test(|_| Payload::InvalidBlockHash)
#[tokio::test]
async fn immediate_forkchoice_update_payload_invalid_block_hash() {
immediate_forkchoice_update_invalid_test(|_| Payload::InvalidBlockHash).await
}
#[test]
fn immediate_forkchoice_update_payload_invalid_terminal_block() {
immediate_forkchoice_update_invalid_test(|_| Payload::InvalidTerminalBlock)
#[tokio::test]
async fn immediate_forkchoice_update_payload_invalid_terminal_block() {
immediate_forkchoice_update_invalid_test(|_| Payload::InvalidTerminalBlock).await
}
/// Ensure the client tries to exit when the justified checkpoint is invalidated.
#[test]
fn justified_checkpoint_becomes_invalid() {
#[tokio::test]
async fn justified_checkpoint_becomes_invalid() {
let mut rig = InvalidPayloadRig::new().enable_attestations();
rig.move_to_terminal_block();
rig.import_block(Payload::Valid); // Import a valid transition block.
rig.move_to_first_justification(Payload::Syncing);
rig.import_block(Payload::Valid).await; // Import a valid transition block.
rig.move_to_first_justification(Payload::Syncing).await;
let justified_checkpoint = rig.head_info().current_justified_checkpoint;
let justified_checkpoint = rig.harness.justified_checkpoint();
let parent_root_of_justified = rig
.harness
.chain
@ -456,7 +473,8 @@ fn justified_checkpoint_becomes_invalid() {
// is invalid.
BlockError::BeaconChainError(BeaconChainError::JustifiedPayloadInvalid { .. })
)
});
})
.await;
// The beacon chain should have triggered a shutdown.
assert_eq!(
@ -468,18 +486,18 @@ fn justified_checkpoint_becomes_invalid() {
}
/// Ensure that a `latest_valid_hash` for a pre-finality block only reverts a single block.
#[test]
fn pre_finalized_latest_valid_hash() {
#[tokio::test]
async fn pre_finalized_latest_valid_hash() {
let num_blocks = E::slots_per_epoch() * 4;
let finalized_epoch = 2;
let mut rig = InvalidPayloadRig::new().enable_attestations();
rig.move_to_terminal_block();
let mut blocks = vec![];
blocks.push(rig.import_block(Payload::Valid)); // Import a valid transition block.
blocks.extend(rig.build_blocks(num_blocks - 1, Payload::Syncing));
blocks.push(rig.import_block(Payload::Valid).await); // Import a valid transition block.
blocks.extend(rig.build_blocks(num_blocks - 1, Payload::Syncing).await);
assert_eq!(rig.head_info().finalized_checkpoint.epoch, finalized_epoch);
assert_eq!(rig.harness.finalized_checkpoint().epoch, finalized_epoch);
let pre_finalized_block_root = rig.block_root_at_slot(Slot::new(1)).unwrap();
let pre_finalized_block_hash = rig.block_hash(pre_finalized_block_root);
@ -490,10 +508,11 @@ fn pre_finalized_latest_valid_hash() {
// Import a pre-finalized block.
rig.import_block(Payload::Invalid {
latest_valid_hash: Some(pre_finalized_block_hash),
});
})
.await;
// The latest imported block should be the head.
assert_eq!(rig.head_info().block_root, *blocks.last().unwrap());
assert_eq!(rig.harness.head_block_root(), *blocks.last().unwrap());
// The beacon chain should *not* have triggered a shutdown.
assert_eq!(rig.harness.shutdown_reasons(), vec![]);
@ -514,16 +533,16 @@ fn pre_finalized_latest_valid_hash() {
///
/// - Invalidate descendants of `latest_valid_root`.
/// - Validate `latest_valid_root` and its ancestors.
#[test]
fn latest_valid_hash_will_validate() {
#[tokio::test]
async fn latest_valid_hash_will_validate() {
const LATEST_VALID_SLOT: u64 = 3;
let mut rig = InvalidPayloadRig::new().enable_attestations();
rig.move_to_terminal_block();
let mut blocks = vec![];
blocks.push(rig.import_block(Payload::Valid)); // Import a valid transition block.
blocks.extend(rig.build_blocks(4, Payload::Syncing));
blocks.push(rig.import_block(Payload::Valid).await); // Import a valid transition block.
blocks.extend(rig.build_blocks(4, Payload::Syncing).await);
let latest_valid_root = rig
.block_root_at_slot(Slot::new(LATEST_VALID_SLOT))
@ -532,9 +551,10 @@ fn latest_valid_hash_will_validate() {
rig.import_block(Payload::Invalid {
latest_valid_hash: Some(latest_valid_hash),
});
})
.await;
assert_eq!(rig.head_info().slot, LATEST_VALID_SLOT);
assert_eq!(rig.harness.head_slot(), LATEST_VALID_SLOT);
for slot in 0..=5 {
let slot = Slot::new(slot);
@ -558,18 +578,18 @@ fn latest_valid_hash_will_validate() {
}
/// Check behaviour when the `latest_valid_hash` is a junk value.
#[test]
fn latest_valid_hash_is_junk() {
#[tokio::test]
async fn latest_valid_hash_is_junk() {
let num_blocks = E::slots_per_epoch() * 5;
let finalized_epoch = 3;
let mut rig = InvalidPayloadRig::new().enable_attestations();
rig.move_to_terminal_block();
let mut blocks = vec![];
blocks.push(rig.import_block(Payload::Valid)); // Import a valid transition block.
blocks.extend(rig.build_blocks(num_blocks, Payload::Syncing));
blocks.push(rig.import_block(Payload::Valid).await); // Import a valid transition block.
blocks.extend(rig.build_blocks(num_blocks, Payload::Syncing).await);
assert_eq!(rig.head_info().finalized_checkpoint.epoch, finalized_epoch);
assert_eq!(rig.harness.finalized_checkpoint().epoch, finalized_epoch);
// No service should have triggered a shutdown, yet.
assert!(rig.harness.shutdown_reasons().is_empty());
@ -577,10 +597,11 @@ fn latest_valid_hash_is_junk() {
let junk_hash = ExecutionBlockHash::repeat_byte(42);
rig.import_block(Payload::Invalid {
latest_valid_hash: Some(junk_hash),
});
})
.await;
// The latest imported block should be the head.
assert_eq!(rig.head_info().block_root, *blocks.last().unwrap());
assert_eq!(rig.harness.head_block_root(), *blocks.last().unwrap());
// The beacon chain should *not* have triggered a shutdown.
assert_eq!(rig.harness.shutdown_reasons(), vec![]);
@ -598,19 +619,19 @@ fn latest_valid_hash_is_junk() {
}
/// Check that descendants of invalid blocks are also invalidated.
#[test]
fn invalidates_all_descendants() {
#[tokio::test]
async fn invalidates_all_descendants() {
let num_blocks = E::slots_per_epoch() * 4 + E::slots_per_epoch() / 2;
let finalized_epoch = 2;
let finalized_slot = E::slots_per_epoch() * 2;
let mut rig = InvalidPayloadRig::new().enable_attestations();
rig.move_to_terminal_block();
rig.import_block(Payload::Valid); // Import a valid transition block.
let blocks = rig.build_blocks(num_blocks, Payload::Syncing);
rig.import_block(Payload::Valid).await; // Import a valid transition block.
let blocks = rig.build_blocks(num_blocks, Payload::Syncing).await;
assert_eq!(rig.head_info().finalized_checkpoint.epoch, finalized_epoch);
assert_eq!(rig.head_info().block_root, *blocks.last().unwrap());
assert_eq!(rig.harness.finalized_checkpoint().epoch, finalized_epoch);
assert_eq!(rig.harness.head_block_root(), *blocks.last().unwrap());
// Apply a block which conflicts with the canonical chain.
let fork_slot = Slot::new(4 * E::slots_per_epoch() + 3);
@ -621,9 +642,14 @@ fn invalidates_all_descendants() {
.state_at_slot(fork_parent_slot, StateSkipConfig::WithStateRoots)
.unwrap();
assert_eq!(fork_parent_state.slot(), fork_parent_slot);
let (fork_block, _fork_post_state) = rig.harness.make_block(fork_parent_state, fork_slot);
let fork_block_root = rig.harness.chain.process_block(fork_block).unwrap();
rig.fork_choice();
let (fork_block, _fork_post_state) = rig.harness.make_block(fork_parent_state, fork_slot).await;
let fork_block_root = rig
.harness
.chain
.process_block(Arc::new(fork_block))
.await
.unwrap();
rig.recompute_head().await;
// The latest valid hash will be set to the grandparent of the fork block. This means that the
// parent of the fork block will become invalid.
@ -638,14 +664,15 @@ fn invalidates_all_descendants() {
let latest_valid_hash = rig.block_hash(latest_valid_root);
// The new block should not become the head, the old head should remain.
assert_eq!(rig.head_info().block_root, *blocks.last().unwrap());
assert_eq!(rig.harness.head_block_root(), *blocks.last().unwrap());
rig.import_block(Payload::Invalid {
latest_valid_hash: Some(latest_valid_hash),
});
})
.await;
// The block before the fork should become the head.
assert_eq!(rig.head_info().block_root, latest_valid_root);
assert_eq!(rig.harness.head_block_root(), latest_valid_root);
// The fork block should be invalidated, even though it's not an ancestor of the block that
// triggered the INVALID response from the EL.
@ -677,19 +704,19 @@ fn invalidates_all_descendants() {
}
/// Check that the head will switch after the canonical branch is invalidated.
#[test]
fn switches_heads() {
#[tokio::test]
async fn switches_heads() {
let num_blocks = E::slots_per_epoch() * 4 + E::slots_per_epoch() / 2;
let finalized_epoch = 2;
let finalized_slot = E::slots_per_epoch() * 2;
let mut rig = InvalidPayloadRig::new().enable_attestations();
rig.move_to_terminal_block();
rig.import_block(Payload::Valid); // Import a valid transition block.
let blocks = rig.build_blocks(num_blocks, Payload::Syncing);
rig.import_block(Payload::Valid).await; // Import a valid transition block.
let blocks = rig.build_blocks(num_blocks, Payload::Syncing).await;
assert_eq!(rig.head_info().finalized_checkpoint.epoch, finalized_epoch);
assert_eq!(rig.head_info().block_root, *blocks.last().unwrap());
assert_eq!(rig.harness.finalized_checkpoint().epoch, finalized_epoch);
assert_eq!(rig.harness.head_block_root(), *blocks.last().unwrap());
// Apply a block which conflicts with the canonical chain.
let fork_slot = Slot::new(4 * E::slots_per_epoch() + 3);
@ -700,23 +727,29 @@ fn switches_heads() {
.state_at_slot(fork_parent_slot, StateSkipConfig::WithStateRoots)
.unwrap();
assert_eq!(fork_parent_state.slot(), fork_parent_slot);
let (fork_block, _fork_post_state) = rig.harness.make_block(fork_parent_state, fork_slot);
let (fork_block, _fork_post_state) = rig.harness.make_block(fork_parent_state, fork_slot).await;
let fork_parent_root = fork_block.parent_root();
let fork_block_root = rig.harness.chain.process_block(fork_block).unwrap();
rig.fork_choice();
let fork_block_root = rig
.harness
.chain
.process_block(Arc::new(fork_block))
.await
.unwrap();
rig.recompute_head().await;
let latest_valid_slot = fork_parent_slot;
let latest_valid_hash = rig.block_hash(fork_parent_root);
// The new block should not become the head, the old head should remain.
assert_eq!(rig.head_info().block_root, *blocks.last().unwrap());
assert_eq!(rig.harness.head_block_root(), *blocks.last().unwrap());
rig.import_block(Payload::Invalid {
latest_valid_hash: Some(latest_valid_hash),
});
})
.await;
// The fork block should become the head.
assert_eq!(rig.head_info().block_root, fork_block_root);
assert_eq!(rig.harness.head_block_root(), fork_block_root);
// The fork block has not yet been validated.
assert!(rig.execution_status(fork_block_root).is_optimistic());
@ -746,17 +779,18 @@ fn switches_heads() {
}
}
#[test]
fn invalid_during_processing() {
#[tokio::test]
async fn invalid_during_processing() {
let mut rig = InvalidPayloadRig::new();
rig.move_to_terminal_block();
let roots = &[
rig.import_block(Payload::Valid),
rig.import_block(Payload::Valid).await,
rig.import_block(Payload::Invalid {
latest_valid_hash: None,
}),
rig.import_block(Payload::Valid),
})
.await,
rig.import_block(Payload::Valid).await,
];
// 0 should be present in the chain.
@ -772,20 +806,20 @@ fn invalid_during_processing() {
None
);
// 2 should be the head.
let head = rig.harness.chain.head_info().unwrap();
assert_eq!(head.block_root, roots[2]);
let head_block_root = rig.harness.head_block_root();
assert_eq!(head_block_root, roots[2]);
}
#[test]
fn invalid_after_optimistic_sync() {
#[tokio::test]
async fn invalid_after_optimistic_sync() {
let mut rig = InvalidPayloadRig::new().enable_attestations();
rig.move_to_terminal_block();
rig.import_block(Payload::Valid); // Import a valid transition block.
rig.import_block(Payload::Valid).await; // Import a valid transition block.
let mut roots = vec![
rig.import_block(Payload::Syncing),
rig.import_block(Payload::Syncing),
rig.import_block(Payload::Syncing),
rig.import_block(Payload::Syncing).await,
rig.import_block(Payload::Syncing).await,
rig.import_block(Payload::Syncing).await,
];
for root in &roots {
@ -793,29 +827,32 @@ fn invalid_after_optimistic_sync() {
}
// 2 should be the head.
let head = rig.harness.chain.head_info().unwrap();
assert_eq!(head.block_root, roots[2]);
let head = rig.harness.head_block_root();
assert_eq!(head, roots[2]);
roots.push(rig.import_block(Payload::Invalid {
latest_valid_hash: Some(rig.block_hash(roots[1])),
}));
roots.push(
rig.import_block(Payload::Invalid {
latest_valid_hash: Some(rig.block_hash(roots[1])),
})
.await,
);
// Running fork choice is necessary since a block has been invalidated.
rig.fork_choice();
rig.recompute_head().await;
// 1 should be the head, since 2 was invalidated.
let head = rig.harness.chain.head_info().unwrap();
assert_eq!(head.block_root, roots[1]);
let head = rig.harness.head_block_root();
assert_eq!(head, roots[1]);
}
#[test]
fn manually_validate_child() {
#[tokio::test]
async fn manually_validate_child() {
let mut rig = InvalidPayloadRig::new().enable_attestations();
rig.move_to_terminal_block();
rig.import_block(Payload::Valid); // Import a valid transition block.
rig.import_block(Payload::Valid).await; // Import a valid transition block.
let parent = rig.import_block(Payload::Syncing);
let child = rig.import_block(Payload::Syncing);
let parent = rig.import_block(Payload::Syncing).await;
let child = rig.import_block(Payload::Syncing).await;
assert!(rig.execution_status(parent).is_optimistic());
assert!(rig.execution_status(child).is_optimistic());
@ -826,14 +863,14 @@ fn manually_validate_child() {
assert!(rig.execution_status(child).is_valid_and_post_bellatrix());
}
#[test]
fn manually_validate_parent() {
#[tokio::test]
async fn manually_validate_parent() {
let mut rig = InvalidPayloadRig::new().enable_attestations();
rig.move_to_terminal_block();
rig.import_block(Payload::Valid); // Import a valid transition block.
rig.import_block(Payload::Valid).await; // Import a valid transition block.
let parent = rig.import_block(Payload::Syncing);
let child = rig.import_block(Payload::Syncing);
let parent = rig.import_block(Payload::Syncing).await;
let child = rig.import_block(Payload::Syncing).await;
assert!(rig.execution_status(parent).is_optimistic());
assert!(rig.execution_status(child).is_optimistic());
@ -844,14 +881,14 @@ fn manually_validate_parent() {
assert!(rig.execution_status(child).is_optimistic());
}
#[test]
fn payload_preparation() {
#[tokio::test]
async fn payload_preparation() {
let mut rig = InvalidPayloadRig::new();
rig.move_to_terminal_block();
rig.import_block(Payload::Valid);
rig.import_block(Payload::Valid).await;
let el = rig.execution_layer();
let head = rig.harness.chain.head().unwrap();
let head = rig.harness.chain.head_snapshot();
let current_slot = rig.harness.chain.slot().unwrap();
assert_eq!(head.beacon_state.slot(), 1);
assert_eq!(current_slot, 1);
@ -865,18 +902,19 @@ fn payload_preparation() {
let fee_recipient = Address::repeat_byte(99);
// Provide preparation data to the EL for `proposer`.
el.update_proposer_preparation_blocking(
el.update_proposer_preparation(
Epoch::new(1),
&[ProposerPreparationData {
validator_index: proposer as u64,
fee_recipient,
}],
)
.unwrap();
.await;
rig.harness
.chain
.prepare_beacon_proposer_blocking()
.prepare_beacon_proposer(rig.harness.chain.slot().unwrap())
.await
.unwrap();
let payload_attributes = PayloadAttributes {
@ -896,15 +934,15 @@ fn payload_preparation() {
assert_eq!(rig.previous_payload_attributes(), payload_attributes);
}
#[test]
fn invalid_parent() {
#[tokio::test]
async fn invalid_parent() {
let mut rig = InvalidPayloadRig::new();
rig.move_to_terminal_block();
rig.import_block(Payload::Valid); // Import a valid transition block.
rig.import_block(Payload::Valid).await; // Import a valid transition block.
// Import a syncing block atop the transition block (we'll call this the "parent block" since we
// build another block on it later).
let parent_root = rig.import_block(Payload::Syncing);
let parent_root = rig.import_block(Payload::Syncing).await;
let parent_block = rig.harness.get_block(parent_root.into()).unwrap();
let parent_state = rig
.harness
@ -914,34 +952,34 @@ fn invalid_parent() {
// Produce another block atop the parent, but don't import yet.
let slot = parent_block.slot() + 1;
rig.harness.set_current_slot(slot);
let (block, state) = rig.harness.make_block(parent_state, slot);
let (block, state) = rig.harness.make_block(parent_state, slot).await;
let block = Arc::new(block);
let block_root = block.canonical_root();
assert_eq!(block.parent_root(), parent_root);
// Invalidate the parent block.
rig.invalidate_manually(parent_root);
rig.invalidate_manually(parent_root).await;
assert!(rig.execution_status(parent_root).is_invalid());
// Ensure the block built atop an invalid payload is invalid for gossip.
assert!(matches!(
rig.harness.chain.verify_block_for_gossip(block.clone()),
rig.harness.chain.clone().verify_block_for_gossip(block.clone()).await,
Err(BlockError::ParentExecutionPayloadInvalid { parent_root: invalid_root })
if invalid_root == parent_root
));
// Ensure the block built atop an invalid payload is invalid for import.
assert!(matches!(
rig.harness.chain.process_block(block.clone()),
rig.harness.chain.process_block(block.clone()).await,
Err(BlockError::ParentExecutionPayloadInvalid { parent_root: invalid_root })
if invalid_root == parent_root
));
// Ensure the block built atop an invalid payload cannot be imported to fork choice.
let (block, _block_signature) = block.deconstruct();
assert!(matches!(
rig.harness.chain.fork_choice.write().on_block(
rig.harness.chain.canonical_head.fork_choice_write_lock().on_block(
slot,
&block,
block.message(),
block_root,
Duration::from_secs(0),
&state,
@ -960,21 +998,21 @@ fn invalid_parent() {
}
/// Tests to ensure that we will still send a proposer preparation
#[test]
fn payload_preparation_before_transition_block() {
#[tokio::test]
async fn payload_preparation_before_transition_block() {
let rig = InvalidPayloadRig::new();
let el = rig.execution_layer();
let head = rig.harness.chain.head().unwrap();
let head_info = rig.head_info();
assert!(
!head_info.is_merge_transition_complete,
"the head block is pre-transition"
);
let head = rig.harness.chain.head_snapshot();
assert_eq!(
head_info.execution_payload_block_hash,
Some(ExecutionBlockHash::zero()),
"the head block is post-bellatrix"
head.beacon_block
.message()
.body()
.execution_payload()
.unwrap()
.block_hash(),
ExecutionBlockHash::zero(),
"the head block is post-bellatrix but pre-transition"
);
let current_slot = rig.harness.chain.slot().unwrap();
@ -986,24 +1024,32 @@ fn payload_preparation_before_transition_block() {
let fee_recipient = Address::repeat_byte(99);
// Provide preparation data to the EL for `proposer`.
el.update_proposer_preparation_blocking(
el.update_proposer_preparation(
Epoch::new(0),
&[ProposerPreparationData {
validator_index: proposer as u64,
fee_recipient,
}],
)
.unwrap();
.await;
rig.move_to_terminal_block();
rig.harness
.chain
.prepare_beacon_proposer_blocking()
.prepare_beacon_proposer(current_slot)
.await
.unwrap();
let forkchoice_update_params = rig
.harness
.chain
.canonical_head
.fork_choice_read_lock()
.get_forkchoice_update_parameters();
rig.harness
.chain
.update_execution_engine_forkchoice_blocking(current_slot)
.update_execution_engine_forkchoice(current_slot, forkchoice_update_params)
.await
.unwrap();
let (fork_choice_state, payload_attributes) = rig.previous_forkchoice_update_params();
@ -1012,15 +1058,15 @@ fn payload_preparation_before_transition_block() {
assert_eq!(fork_choice_state.head_block_hash, latest_block_hash);
}
#[test]
fn attesting_to_optimistic_head() {
#[tokio::test]
async fn attesting_to_optimistic_head() {
let mut rig = InvalidPayloadRig::new();
rig.move_to_terminal_block();
rig.import_block(Payload::Valid); // Import a valid transition block.
rig.import_block(Payload::Valid).await; // Import a valid transition block.
let root = rig.import_block(Payload::Syncing);
let root = rig.import_block(Payload::Syncing).await;
let head = rig.harness.chain.head().unwrap();
let head = rig.harness.chain.head_snapshot();
let slot = head.beacon_block.slot();
assert_eq!(
head.beacon_block_root, root,

File diff suppressed because it is too large Load Diff

View File

@ -46,15 +46,8 @@ fn get_valid_sync_committee_message(
slot: Slot,
relative_sync_committee: RelativeSyncCommittee,
) -> (SyncCommitteeMessage, usize, SecretKey, SyncSubnetId) {
let head_state = harness
.chain
.head_beacon_state()
.expect("should get head state");
let head_block_root = harness
.chain
.head()
.expect("should get head state")
.beacon_block_root;
let head_state = harness.chain.head_beacon_state_cloned();
let head_block_root = harness.chain.head_snapshot().beacon_block_root;
let (signature, _) = harness
.make_sync_committee_messages(&head_state, head_block_root, slot, relative_sync_committee)
.get(0)
@ -77,16 +70,9 @@ fn get_valid_sync_contribution(
harness: &BeaconChainHarness<EphemeralHarnessType<E>>,
relative_sync_committee: RelativeSyncCommittee,
) -> (SignedContributionAndProof<E>, usize, SecretKey) {
let head_state = harness
.chain
.head_beacon_state()
.expect("should get head state");
let head_state = harness.chain.head_beacon_state_cloned();
let head_block_root = harness
.chain
.head()
.expect("should get head state")
.beacon_block_root;
let head_block_root = harness.chain.head_snapshot().beacon_block_root;
let sync_contributions = harness.make_sync_contributions(
&head_state,
head_block_root,
@ -116,7 +102,7 @@ fn get_non_aggregator(
harness: &BeaconChainHarness<EphemeralHarnessType<E>>,
slot: Slot,
) -> (usize, SecretKey) {
let state = &harness.chain.head().expect("should get head").beacon_state;
let state = &harness.chain.head_snapshot().beacon_state;
let sync_subcommittee_size = E::sync_committee_size()
.safe_div(SYNC_COMMITTEE_SUBNET_COUNT as usize)
.expect("should determine sync subcommittee size");
@ -162,17 +148,19 @@ fn get_non_aggregator(
}
/// Tests verification of `SignedContributionAndProof` from the gossip network.
#[test]
fn aggregated_gossip_verification() {
#[tokio::test]
async fn aggregated_gossip_verification() {
let harness = get_harness(VALIDATOR_COUNT);
let state = harness.get_current_state();
harness.add_attested_blocks_at_slots(
state,
Hash256::zero(),
&[Slot::new(1), Slot::new(2)],
(0..VALIDATOR_COUNT).collect::<Vec<_>>().as_slice(),
);
harness
.add_attested_blocks_at_slots(
state,
Hash256::zero(),
&[Slot::new(1), Slot::new(2)],
(0..VALIDATOR_COUNT).collect::<Vec<_>>().as_slice(),
)
.await;
let current_slot = harness.chain.slot().expect("should get slot");
@ -406,7 +394,7 @@ fn aggregated_gossip_verification() {
valid_aggregate.message.contribution.clone(),
None,
&non_aggregator_sk,
&harness.chain.head_info().expect("should get head info").fork,
&harness.chain.canonical_head.cached_head().head_fork(),
harness.chain.genesis_validators_root,
&harness.chain.spec,
)
@ -474,6 +462,7 @@ fn aggregated_gossip_verification() {
harness
.add_attested_block_at_slot(target_slot, state, Hash256::zero(), &[])
.await
.expect("should add block");
// **Incorrectly** create a sync contribution using the current sync committee
@ -488,17 +477,19 @@ fn aggregated_gossip_verification() {
}
/// Tests the verification conditions for sync committee messages on the gossip network.
#[test]
fn unaggregated_gossip_verification() {
#[tokio::test]
async fn unaggregated_gossip_verification() {
let harness = get_harness(VALIDATOR_COUNT);
let state = harness.get_current_state();
harness.add_attested_blocks_at_slots(
state,
Hash256::zero(),
&[Slot::new(1), Slot::new(2)],
(0..VALIDATOR_COUNT).collect::<Vec<_>>().as_slice(),
);
harness
.add_attested_blocks_at_slots(
state,
Hash256::zero(),
&[Slot::new(1), Slot::new(2)],
(0..VALIDATOR_COUNT).collect::<Vec<_>>().as_slice(),
)
.await;
let current_slot = harness.chain.slot().expect("should get slot");
@ -648,6 +639,7 @@ fn unaggregated_gossip_verification() {
harness
.add_attested_block_at_slot(target_slot, state, Hash256::zero(), &[])
.await
.expect("should add block");
// **Incorrectly** create a sync message using the current sync committee

View File

@ -6,14 +6,16 @@ use beacon_chain::{
AttestationStrategy, BeaconChainHarness, BlockStrategy, EphemeralHarnessType,
OP_POOL_DB_KEY,
},
StateSkipConfig, WhenSlotSkipped,
BeaconChain, StateSkipConfig, WhenSlotSkipped,
};
use lazy_static::lazy_static;
use operation_pool::PersistedOperationPool;
use state_processing::{
per_slot_processing, per_slot_processing::Error as SlotProcessingError, EpochProcessingError,
};
use types::{BeaconStateError, EthSpec, Hash256, Keypair, MinimalEthSpec, RelativeEpoch, Slot};
use types::{
BeaconState, BeaconStateError, EthSpec, Hash256, Keypair, MinimalEthSpec, RelativeEpoch, Slot,
};
// Should ideally be divisible by 3.
pub const VALIDATOR_COUNT: usize = 24;
@ -40,7 +42,7 @@ fn get_harness(validator_count: usize) -> BeaconChainHarness<EphemeralHarnessTyp
fn massive_skips() {
let harness = get_harness(8);
let spec = &harness.chain.spec;
let mut state = harness.chain.head().expect("should get head").beacon_state;
let mut state = harness.chain.head_beacon_state_cloned();
// Run per_slot_processing until it returns an error.
let error = loop {
@ -60,18 +62,20 @@ fn massive_skips() {
)
}
#[test]
fn iterators() {
#[tokio::test]
async fn iterators() {
let num_blocks_produced = MinimalEthSpec::slots_per_epoch() * 2 - 1;
let harness = get_harness(VALIDATOR_COUNT);
harness.extend_chain(
num_blocks_produced as usize,
BlockStrategy::OnCanonicalHead,
// No need to produce attestations for this test.
AttestationStrategy::SomeValidators(vec![]),
);
harness
.extend_chain(
num_blocks_produced as usize,
BlockStrategy::OnCanonicalHead,
// No need to produce attestations for this test.
AttestationStrategy::SomeValidators(vec![]),
)
.await;
let block_roots: Vec<(Hash256, Slot)> = harness
.chain
@ -122,7 +126,7 @@ fn iterators() {
)
});
let head = &harness.chain.head().expect("should get head");
let head = harness.chain.head_snapshot();
assert_eq!(
*block_roots.last().expect("should have some block roots"),
@ -137,20 +141,44 @@ fn iterators() {
);
}
#[test]
fn find_reorgs() {
fn find_reorg_slot(
chain: &BeaconChain<EphemeralHarnessType<MinimalEthSpec>>,
new_state: &BeaconState<MinimalEthSpec>,
new_block_root: Hash256,
) -> Slot {
let (old_state, old_block_root) = {
let head = chain.canonical_head.cached_head();
let old_state = head.snapshot.beacon_state.clone();
let old_block_root = head.head_block_root();
(old_state, old_block_root)
};
beacon_chain::canonical_head::find_reorg_slot(
&old_state,
old_block_root,
new_state,
new_block_root,
&chain.spec,
)
.unwrap()
}
#[tokio::test]
async fn find_reorgs() {
let num_blocks_produced = MinimalEthSpec::slots_per_historical_root() + 1;
let harness = get_harness(VALIDATOR_COUNT);
harness.extend_chain(
num_blocks_produced as usize,
BlockStrategy::OnCanonicalHead,
// No need to produce attestations for this test.
AttestationStrategy::SomeValidators(vec![]),
);
harness
.extend_chain(
num_blocks_produced as usize,
BlockStrategy::OnCanonicalHead,
// No need to produce attestations for this test.
AttestationStrategy::SomeValidators(vec![]),
)
.await;
let head_state = harness.chain.head_beacon_state().unwrap();
let head = harness.chain.head_snapshot();
let head_state = &head.beacon_state;
let head_slot = head_state.slot();
let genesis_state = harness
.chain
@ -160,10 +188,11 @@ fn find_reorgs() {
// because genesis is more than `SLOTS_PER_HISTORICAL_ROOT` away, this should return with the
// finalized slot.
assert_eq!(
harness
.chain
.find_reorg_slot(&genesis_state, harness.chain.genesis_block_root)
.unwrap(),
find_reorg_slot(
&harness.chain,
&genesis_state,
harness.chain.genesis_block_root
),
head_state
.finalized_checkpoint()
.epoch
@ -172,13 +201,11 @@ fn find_reorgs() {
// test head
assert_eq!(
harness
.chain
.find_reorg_slot(
&head_state,
harness.chain.head_beacon_block().unwrap().canonical_root()
)
.unwrap(),
find_reorg_slot(
&harness.chain,
&head_state,
harness.chain.head_beacon_block().canonical_root()
),
head_slot
);
@ -194,16 +221,13 @@ fn find_reorgs() {
.unwrap()
.unwrap();
assert_eq!(
harness
.chain
.find_reorg_slot(&prev_state, prev_block_root)
.unwrap(),
find_reorg_slot(&harness.chain, &prev_state, prev_block_root),
prev_slot
);
}
#[test]
fn chooses_fork() {
#[tokio::test]
async fn chooses_fork() {
let harness = get_harness(VALIDATOR_COUNT);
let two_thirds = (VALIDATOR_COUNT / 3) * 2;
@ -217,22 +241,27 @@ fn chooses_fork() {
let faulty_fork_blocks = delay + 2;
// Build an initial chain where all validators agree.
harness.extend_chain(
initial_blocks,
BlockStrategy::OnCanonicalHead,
AttestationStrategy::AllValidators,
);
harness
.extend_chain(
initial_blocks,
BlockStrategy::OnCanonicalHead,
AttestationStrategy::AllValidators,
)
.await;
let (honest_head, faulty_head) = harness.generate_two_forks_by_skipping_a_block(
&honest_validators,
&faulty_validators,
honest_fork_blocks,
faulty_fork_blocks,
);
let (honest_head, faulty_head) = harness
.generate_two_forks_by_skipping_a_block(
&honest_validators,
&faulty_validators,
honest_fork_blocks,
faulty_fork_blocks,
)
.await;
assert_ne!(honest_head, faulty_head, "forks should be distinct");
let state = &harness.chain.head().expect("should get head").beacon_state;
let head = harness.chain.head_snapshot();
let state = &head.beacon_state;
assert_eq!(
state.slot(),
@ -241,29 +270,28 @@ fn chooses_fork() {
);
assert_eq!(
harness
.chain
.head()
.expect("should get head")
.beacon_block_root,
harness.chain.head_snapshot().beacon_block_root,
honest_head,
"the honest chain should be the canonical chain"
);
}
#[test]
fn finalizes_with_full_participation() {
#[tokio::test]
async fn finalizes_with_full_participation() {
let num_blocks_produced = MinimalEthSpec::slots_per_epoch() * 5;
let harness = get_harness(VALIDATOR_COUNT);
harness.extend_chain(
num_blocks_produced as usize,
BlockStrategy::OnCanonicalHead,
AttestationStrategy::AllValidators,
);
harness
.extend_chain(
num_blocks_produced as usize,
BlockStrategy::OnCanonicalHead,
AttestationStrategy::AllValidators,
)
.await;
let state = &harness.chain.head().expect("should get head").beacon_state;
let head = harness.chain.head_snapshot();
let state = &head.beacon_state;
assert_eq!(
state.slot(),
@ -287,8 +315,8 @@ fn finalizes_with_full_participation() {
);
}
#[test]
fn finalizes_with_two_thirds_participation() {
#[tokio::test]
async fn finalizes_with_two_thirds_participation() {
let num_blocks_produced = MinimalEthSpec::slots_per_epoch() * 5;
let harness = get_harness(VALIDATOR_COUNT);
@ -296,13 +324,16 @@ fn finalizes_with_two_thirds_participation() {
let two_thirds = (VALIDATOR_COUNT / 3) * 2;
let attesters = (0..two_thirds).collect();
harness.extend_chain(
num_blocks_produced as usize,
BlockStrategy::OnCanonicalHead,
AttestationStrategy::SomeValidators(attesters),
);
harness
.extend_chain(
num_blocks_produced as usize,
BlockStrategy::OnCanonicalHead,
AttestationStrategy::SomeValidators(attesters),
)
.await;
let state = &harness.chain.head().expect("should get head").beacon_state;
let head = harness.chain.head_snapshot();
let state = &head.beacon_state;
assert_eq!(
state.slot(),
@ -331,8 +362,8 @@ fn finalizes_with_two_thirds_participation() {
);
}
#[test]
fn does_not_finalize_with_less_than_two_thirds_participation() {
#[tokio::test]
async fn does_not_finalize_with_less_than_two_thirds_participation() {
let num_blocks_produced = MinimalEthSpec::slots_per_epoch() * 5;
let harness = get_harness(VALIDATOR_COUNT);
@ -341,13 +372,16 @@ fn does_not_finalize_with_less_than_two_thirds_participation() {
let less_than_two_thirds = two_thirds - 1;
let attesters = (0..less_than_two_thirds).collect();
harness.extend_chain(
num_blocks_produced as usize,
BlockStrategy::OnCanonicalHead,
AttestationStrategy::SomeValidators(attesters),
);
harness
.extend_chain(
num_blocks_produced as usize,
BlockStrategy::OnCanonicalHead,
AttestationStrategy::SomeValidators(attesters),
)
.await;
let state = &harness.chain.head().expect("should get head").beacon_state;
let head = harness.chain.head_snapshot();
let state = &head.beacon_state;
assert_eq!(
state.slot(),
@ -371,19 +405,22 @@ fn does_not_finalize_with_less_than_two_thirds_participation() {
);
}
#[test]
fn does_not_finalize_without_attestation() {
#[tokio::test]
async fn does_not_finalize_without_attestation() {
let num_blocks_produced = MinimalEthSpec::slots_per_epoch() * 5;
let harness = get_harness(VALIDATOR_COUNT);
harness.extend_chain(
num_blocks_produced as usize,
BlockStrategy::OnCanonicalHead,
AttestationStrategy::SomeValidators(vec![]),
);
harness
.extend_chain(
num_blocks_produced as usize,
BlockStrategy::OnCanonicalHead,
AttestationStrategy::SomeValidators(vec![]),
)
.await;
let state = &harness.chain.head().expect("should get head").beacon_state;
let head = harness.chain.head_snapshot();
let state = &head.beacon_state;
assert_eq!(
state.slot(),
@ -407,18 +444,20 @@ fn does_not_finalize_without_attestation() {
);
}
#[test]
fn roundtrip_operation_pool() {
#[tokio::test]
async fn roundtrip_operation_pool() {
let num_blocks_produced = MinimalEthSpec::slots_per_epoch() * 5;
let harness = get_harness(VALIDATOR_COUNT);
// Add some attestations
harness.extend_chain(
num_blocks_produced as usize,
BlockStrategy::OnCanonicalHead,
AttestationStrategy::AllValidators,
);
harness
.extend_chain(
num_blocks_produced as usize,
BlockStrategy::OnCanonicalHead,
AttestationStrategy::AllValidators,
)
.await;
assert!(harness.chain.op_pool.num_attestations() > 0);
// TODO: could add some other operations
@ -439,20 +478,23 @@ fn roundtrip_operation_pool() {
assert_eq!(harness.chain.op_pool, restored_op_pool);
}
#[test]
fn unaggregated_attestations_added_to_fork_choice_some_none() {
#[tokio::test]
async fn unaggregated_attestations_added_to_fork_choice_some_none() {
let num_blocks_produced = MinimalEthSpec::slots_per_epoch() / 2;
let harness = get_harness(VALIDATOR_COUNT);
harness.extend_chain(
num_blocks_produced as usize,
BlockStrategy::OnCanonicalHead,
AttestationStrategy::AllValidators,
);
harness
.extend_chain(
num_blocks_produced as usize,
BlockStrategy::OnCanonicalHead,
AttestationStrategy::AllValidators,
)
.await;
let state = &harness.chain.head().expect("should get head").beacon_state;
let mut fork_choice = harness.chain.fork_choice.write();
let head = harness.chain.head_snapshot();
let state = &head.beacon_state;
let mut fork_choice = harness.chain.canonical_head.fork_choice_write_lock();
// Move forward a slot so all queued attestations can be processed.
harness.advance_slot();
@ -493,8 +535,8 @@ fn unaggregated_attestations_added_to_fork_choice_some_none() {
}
}
#[test]
fn attestations_with_increasing_slots() {
#[tokio::test]
async fn attestations_with_increasing_slots() {
let num_blocks_produced = MinimalEthSpec::slots_per_epoch() * 5;
let harness = get_harness(VALIDATOR_COUNT);
@ -502,14 +544,16 @@ fn attestations_with_increasing_slots() {
let mut attestations = vec![];
for _ in 0..num_blocks_produced {
harness.extend_chain(
2,
BlockStrategy::OnCanonicalHead,
// Don't produce & include any attestations (we'll collect them later).
AttestationStrategy::SomeValidators(vec![]),
);
harness
.extend_chain(
2,
BlockStrategy::OnCanonicalHead,
// Don't produce & include any attestations (we'll collect them later).
AttestationStrategy::SomeValidators(vec![]),
)
.await;
let head = harness.chain.head().unwrap();
let head = harness.chain.head_snapshot();
let head_state_root = head.beacon_state_root();
attestations.extend(harness.get_unaggregated_attestations(
@ -548,20 +592,23 @@ fn attestations_with_increasing_slots() {
}
}
#[test]
fn unaggregated_attestations_added_to_fork_choice_all_updated() {
#[tokio::test]
async fn unaggregated_attestations_added_to_fork_choice_all_updated() {
let num_blocks_produced = MinimalEthSpec::slots_per_epoch() * 2 - 1;
let harness = get_harness(VALIDATOR_COUNT);
harness.extend_chain(
num_blocks_produced as usize,
BlockStrategy::OnCanonicalHead,
AttestationStrategy::AllValidators,
);
harness
.extend_chain(
num_blocks_produced as usize,
BlockStrategy::OnCanonicalHead,
AttestationStrategy::AllValidators,
)
.await;
let state = &harness.chain.head().expect("should get head").beacon_state;
let mut fork_choice = harness.chain.fork_choice.write();
let head = harness.chain.head_snapshot();
let state = &head.beacon_state;
let mut fork_choice = harness.chain.canonical_head.fork_choice_write_lock();
// Move forward a slot so all queued attestations can be processed.
harness.advance_slot();
@ -605,7 +652,7 @@ fn unaggregated_attestations_added_to_fork_choice_all_updated() {
}
}
fn run_skip_slot_test(skip_slots: u64) {
async fn run_skip_slot_test(skip_slots: u64) {
let num_validators = 8;
let harness_a = get_harness(num_validators);
let harness_b = get_harness(num_validators);
@ -615,83 +662,60 @@ fn run_skip_slot_test(skip_slots: u64) {
harness_b.advance_slot();
}
harness_a.extend_chain(
1,
BlockStrategy::OnCanonicalHead,
// No attestation required for test.
AttestationStrategy::SomeValidators(vec![]),
);
harness_a
.extend_chain(
1,
BlockStrategy::OnCanonicalHead,
// No attestation required for test.
AttestationStrategy::SomeValidators(vec![]),
)
.await;
assert_eq!(
harness_a
.chain
.head()
.expect("should get head")
.beacon_block
.slot(),
harness_a.chain.head_snapshot().beacon_block.slot(),
Slot::new(skip_slots + 1)
);
assert_eq!(
harness_b
.chain
.head()
.expect("should get head")
.beacon_block
.slot(),
harness_b.chain.head_snapshot().beacon_block.slot(),
Slot::new(0)
);
assert_eq!(
harness_b
.chain
.process_block(
harness_a
.chain
.head()
.expect("should get head")
.beacon_block
.clone(),
)
.process_block(harness_a.chain.head_snapshot().beacon_block.clone())
.await
.unwrap(),
harness_a
.chain
.head()
.expect("should get head")
.beacon_block_root
harness_a.chain.head_snapshot().beacon_block_root
);
harness_b
.chain
.fork_choice()
.recompute_head_at_current_slot()
.await
.expect("should run fork choice");
assert_eq!(
harness_b
.chain
.head()
.expect("should get head")
.beacon_block
.slot(),
harness_b.chain.head_snapshot().beacon_block.slot(),
Slot::new(skip_slots + 1)
);
}
#[test]
fn produces_and_processes_with_genesis_skip_slots() {
#[tokio::test]
async fn produces_and_processes_with_genesis_skip_slots() {
for i in 0..MinimalEthSpec::slots_per_epoch() * 4 {
run_skip_slot_test(i)
run_skip_slot_test(i).await
}
}
#[test]
fn block_roots_skip_slot_behaviour() {
#[tokio::test]
async fn block_roots_skip_slot_behaviour() {
let harness = get_harness(VALIDATOR_COUNT);
// Test should be longer than the block roots to ensure a DB lookup is triggered.
let chain_length = harness
.chain
.head()
.unwrap()
.head_snapshot()
.beacon_state
.block_roots()
.len() as u64
@ -708,11 +732,13 @@ fn block_roots_skip_slot_behaviour() {
let slot = harness.chain.slot().unwrap().as_u64();
if !skipped_slots.contains(&slot) {
harness.extend_chain(
1,
BlockStrategy::OnCanonicalHead,
AttestationStrategy::AllValidators,
);
harness
.extend_chain(
1,
BlockStrategy::OnCanonicalHead,
AttestationStrategy::AllValidators,
)
.await;
}
}
@ -820,7 +846,7 @@ fn block_roots_skip_slot_behaviour() {
let future_slot = harness.chain.slot().unwrap() + 1;
assert_eq!(
harness.chain.head().unwrap().beacon_block.slot(),
harness.chain.head_snapshot().beacon_block.slot(),
future_slot - 2,
"test precondition"
);

View File

@ -684,26 +684,20 @@ where
if let Some(execution_layer) = beacon_chain.execution_layer.as_ref() {
// Only send a head update *after* genesis.
if let Ok(current_slot) = beacon_chain.slot() {
let head = beacon_chain
.head_info()
.map_err(|e| format!("Unable to read beacon chain head: {:?}", e))?;
// Issue the head to the execution engine on startup. This ensures it can start
// syncing.
if head
.execution_payload_block_hash
.map_or(false, |h| h != ExecutionBlockHash::zero())
let params = beacon_chain
.canonical_head
.cached_head()
.forkchoice_update_parameters();
if params
.head_hash
.map_or(false, |hash| hash != ExecutionBlockHash::zero())
{
// Spawn a new task using the "async" fork choice update method, rather than
// using the "blocking" method.
//
// Using the blocking method may cause a panic if this code is run inside an
// async context.
// Spawn a new task to update the EE without waiting for it to complete.
let inner_chain = beacon_chain.clone();
runtime_context.executor.spawn(
async move {
let result = inner_chain
.update_execution_engine_forkchoice_async(current_slot)
.update_execution_engine_forkchoice(current_slot, params)
.await;
// No need to exit early if setting the head fails. It will be set again if/when the
@ -811,8 +805,16 @@ where
self.db_path = Some(hot_path.into());
self.freezer_db_path = Some(cold_path.into());
let inner_spec = spec.clone();
let schema_upgrade = |db, from, to| {
migrate_schema::<Witness<TSlotClock, TEth1Backend, _, _, _>>(db, datadir, from, to, log)
migrate_schema::<Witness<TSlotClock, TEth1Backend, _, _, _>>(
db,
datadir,
from,
to,
log,
&inner_spec,
)
};
let store = HotColdDB::open(

View File

@ -1,5 +1,5 @@
use crate::metrics;
use beacon_chain::{BeaconChain, BeaconChainTypes, HeadSafetyStatus};
use beacon_chain::{BeaconChain, BeaconChainTypes, ExecutionStatus};
use lighthouse_network::{types::SyncState, NetworkGlobals};
use parking_lot::Mutex;
use slog::{crit, debug, error, info, warn, Logger};
@ -100,15 +100,10 @@ pub fn spawn_notifier<T: BeaconChainTypes>(
current_sync_state = sync_state;
}
let head_info = match beacon_chain.head_info() {
Ok(head_info) => head_info,
Err(e) => {
error!(log, "Failed to get beacon chain head info"; "error" => format!("{:?}", e));
break;
}
};
let head_slot = head_info.slot;
let cached_head = beacon_chain.canonical_head.cached_head();
let head_slot = cached_head.head_slot();
let head_root = cached_head.head_block_root();
let finalized_checkpoint = cached_head.finalized_checkpoint();
metrics::set_gauge(&metrics::NOTIFIER_HEAD_SLOT, head_slot.as_u64() as i64);
@ -125,9 +120,6 @@ pub fn spawn_notifier<T: BeaconChainTypes>(
};
let current_epoch = current_slot.epoch(T::EthSpec::slots_per_epoch());
let finalized_epoch = head_info.finalized_checkpoint.epoch;
let finalized_root = head_info.finalized_checkpoint.root;
let head_root = head_info.block_root;
// The default is for regular sync but this gets modified if backfill sync is in
// progress.
@ -177,8 +169,8 @@ pub fn spawn_notifier<T: BeaconChainTypes>(
log,
"Slot timer";
"peers" => peer_count_pretty(connected_peer_count),
"finalized_root" => format!("{}", finalized_root),
"finalized_epoch" => finalized_epoch,
"finalized_root" => format!("{}", finalized_checkpoint.root),
"finalized_epoch" => finalized_checkpoint.epoch,
"head_block" => format!("{}", head_root),
"head_slot" => head_slot,
"current_slot" => current_slot,
@ -264,35 +256,29 @@ pub fn spawn_notifier<T: BeaconChainTypes>(
head_root.to_string()
};
let block_hash = match beacon_chain.head_safety_status() {
Ok(HeadSafetyStatus::Safe(hash_opt)) => hash_opt
.map(|hash| format!("{} (verified)", hash))
.unwrap_or_else(|| "n/a".to_string()),
Ok(HeadSafetyStatus::Unsafe(block_hash)) => {
let block_hash = match beacon_chain.canonical_head.head_execution_status() {
Ok(ExecutionStatus::Irrelevant(_)) => "n/a".to_string(),
Ok(ExecutionStatus::Valid(hash)) => format!("{} (verified)", hash),
Ok(ExecutionStatus::Optimistic(hash)) => {
warn!(
log,
"Head execution payload is unverified";
"execution_block_hash" => ?block_hash,
"Head is optimistic";
"info" => "chain not fully verified, \
block and attestation production disabled until execution engine syncs",
"execution_block_hash" => ?hash,
);
format!("{} (unverified)", block_hash)
format!("{} (unverified)", hash)
}
Ok(HeadSafetyStatus::Invalid(block_hash)) => {
Ok(ExecutionStatus::Invalid(hash)) => {
crit!(
log,
"Head execution payload is invalid";
"msg" => "this scenario may be unrecoverable",
"execution_block_hash" => ?block_hash,
"execution_block_hash" => ?hash,
);
format!("{} (invalid)", block_hash)
}
Err(e) => {
error!(
log,
"Failed to read head safety status";
"error" => ?e
);
"n/a".to_string()
format!("{} (invalid)", hash)
}
Err(_) => "unknown".to_string(),
};
info!(
@ -300,8 +286,8 @@ pub fn spawn_notifier<T: BeaconChainTypes>(
"Synced";
"peers" => peer_count_pretty(connected_peer_count),
"exec_hash" => block_hash,
"finalized_root" => format!("{}", finalized_root),
"finalized_epoch" => finalized_epoch,
"finalized_root" => format!("{}", finalized_checkpoint.root),
"finalized_epoch" => finalized_checkpoint.epoch,
"epoch" => current_epoch,
"block" => block_info,
"slot" => current_slot,
@ -312,8 +298,8 @@ pub fn spawn_notifier<T: BeaconChainTypes>(
log,
"Searching for peers";
"peers" => peer_count_pretty(connected_peer_count),
"finalized_root" => format!("{}", finalized_root),
"finalized_epoch" => finalized_epoch,
"finalized_root" => format!("{}", finalized_checkpoint.root),
"finalized_epoch" => finalized_checkpoint.epoch,
"head_slot" => head_slot,
"current_slot" => current_slot,
);
@ -332,57 +318,52 @@ pub fn spawn_notifier<T: BeaconChainTypes>(
fn eth1_logging<T: BeaconChainTypes>(beacon_chain: &BeaconChain<T>, log: &Logger) {
let current_slot_opt = beacon_chain.slot().ok();
if let Ok(head_info) = beacon_chain.head_info() {
// Perform some logging about the eth1 chain
if let Some(eth1_chain) = beacon_chain.eth1_chain.as_ref() {
// No need to do logging if using the dummy backend.
if eth1_chain.is_dummy_backend() {
return;
}
if let Some(status) =
eth1_chain.sync_status(head_info.genesis_time, current_slot_opt, &beacon_chain.spec)
{
debug!(
log,
"Eth1 cache sync status";
"eth1_head_block" => status.head_block_number,
"latest_cached_block_number" => status.latest_cached_block_number,
"latest_cached_timestamp" => status.latest_cached_block_timestamp,
"voting_target_timestamp" => status.voting_target_timestamp,
"ready" => status.lighthouse_is_cached_and_ready
);
if !status.lighthouse_is_cached_and_ready {
let voting_target_timestamp = status.voting_target_timestamp;
let distance = status
.latest_cached_block_timestamp
.map(|latest| {
voting_target_timestamp.saturating_sub(latest)
/ beacon_chain.spec.seconds_per_eth1_block
})
.map(|distance| distance.to_string())
.unwrap_or_else(|| "initializing deposits".to_string());
warn!(
log,
"Syncing eth1 block cache";
"est_blocks_remaining" => distance,
);
}
} else {
error!(
log,
"Unable to determine eth1 sync status";
);
}
// Perform some logging about the eth1 chain
if let Some(eth1_chain) = beacon_chain.eth1_chain.as_ref() {
// No need to do logging if using the dummy backend.
if eth1_chain.is_dummy_backend() {
return;
}
if let Some(status) = eth1_chain.sync_status(
beacon_chain.genesis_time,
current_slot_opt,
&beacon_chain.spec,
) {
debug!(
log,
"Eth1 cache sync status";
"eth1_head_block" => status.head_block_number,
"latest_cached_block_number" => status.latest_cached_block_number,
"latest_cached_timestamp" => status.latest_cached_block_timestamp,
"voting_target_timestamp" => status.voting_target_timestamp,
"ready" => status.lighthouse_is_cached_and_ready
);
if !status.lighthouse_is_cached_and_ready {
let voting_target_timestamp = status.voting_target_timestamp;
let distance = status
.latest_cached_block_timestamp
.map(|latest| {
voting_target_timestamp.saturating_sub(latest)
/ beacon_chain.spec.seconds_per_eth1_block
})
.map(|distance| distance.to_string())
.unwrap_or_else(|| "initializing deposits".to_string());
warn!(
log,
"Syncing eth1 block cache";
"est_blocks_remaining" => distance,
);
}
} else {
error!(
log,
"Unable to determine eth1 sync status";
);
}
} else {
error!(
log,
"Unable to get head info";
);
}
}

View File

@ -274,31 +274,6 @@ impl<T: EthSpec> ExecutionLayer<T> {
self.inner.execution_engine_forkchoice_lock.lock().await
}
/// Convenience function to allow calling async functions in a non-async context.
pub fn block_on<'a, F, U, V>(&'a self, generate_future: F) -> Result<V, Error>
where
F: Fn(&'a Self) -> U,
U: Future<Output = Result<V, Error>>,
{
let runtime = self.executor().handle().ok_or(Error::ShuttingDown)?;
// TODO(merge): respect the shutdown signal.
runtime.block_on(generate_future(self))
}
/// Convenience function to allow calling async functions in a non-async context.
///
/// The function is "generic" since it does not enforce a particular return type on
/// `generate_future`.
pub fn block_on_generic<'a, F, U, V>(&'a self, generate_future: F) -> Result<V, Error>
where
F: Fn(&'a Self) -> U,
U: Future<Output = V>,
{
let runtime = self.executor().handle().ok_or(Error::ShuttingDown)?;
// TODO(merge): respect the shutdown signal.
Ok(runtime.block_on(generate_future(self)))
}
/// Convenience function to allow spawning a task without waiting for the result.
pub fn spawn<F, U>(&self, generate_future: F, name: &'static str)
where
@ -431,18 +406,6 @@ impl<T: EthSpec> ExecutionLayer<T> {
self.engines().is_synced().await
}
/// Updates the proposer preparation data provided by validators
pub fn update_proposer_preparation_blocking(
&self,
update_epoch: Epoch,
preparation_data: &[ProposerPreparationData],
) -> Result<(), Error> {
self.block_on_generic(|_| async move {
self.update_proposer_preparation(update_epoch, preparation_data)
.await
})
}
/// Updates the proposer preparation data provided by validators
pub async fn update_proposer_preparation(
&self,

View File

@ -58,12 +58,10 @@ fn cached_attestation_duties<T: BeaconChainTypes>(
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 head_block_root = chain.canonical_head.cached_head().head_block_root();
let (duties, dependent_root) = chain
.validator_attestation_duties(request_indices, request_epoch, head.block_root)
let (duties, dependent_root, _execution_status) = 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)

View File

@ -1,6 +1,7 @@
use beacon_chain::{BeaconChain, BeaconChainTypes, WhenSlotSkipped};
use eth2::types::BlockId as CoreBlockId;
use std::str::FromStr;
use std::sync::Arc;
use types::{BlindedPayload, Hash256, SignedBeaconBlock, Slot};
/// Wraps `eth2::types::BlockId` and provides a simple way to obtain a block or root for a given
@ -23,19 +24,18 @@ impl BlockId {
chain: &BeaconChain<T>,
) -> Result<Hash256, warp::Rejection> {
match &self.0 {
CoreBlockId::Head => chain
.head_info()
.map(|head| head.block_root)
.map_err(warp_utils::reject::beacon_chain_error),
CoreBlockId::Head => Ok(chain.canonical_head.cached_head().head_block_root()),
CoreBlockId::Genesis => Ok(chain.genesis_block_root),
CoreBlockId::Finalized => chain
.head_info()
.map(|head| head.finalized_checkpoint.root)
.map_err(warp_utils::reject::beacon_chain_error),
CoreBlockId::Justified => chain
.head_info()
.map(|head| head.current_justified_checkpoint.root)
.map_err(warp_utils::reject::beacon_chain_error),
CoreBlockId::Finalized => Ok(chain
.canonical_head
.cached_head()
.finalized_checkpoint()
.root),
CoreBlockId::Justified => Ok(chain
.canonical_head
.cached_head()
.justified_checkpoint()
.root),
CoreBlockId::Slot(slot) => chain
.block_root_at_slot(*slot, WhenSlotSkipped::None)
.map_err(warp_utils::reject::beacon_chain_error)
@ -57,10 +57,7 @@ impl BlockId {
chain: &BeaconChain<T>,
) -> Result<SignedBeaconBlock<T::EthSpec, BlindedPayload<T::EthSpec>>, warp::Rejection> {
match &self.0 {
CoreBlockId::Head => chain
.head_beacon_block()
.map(Into::into)
.map_err(warp_utils::reject::beacon_chain_error),
CoreBlockId::Head => Ok(chain.head_beacon_block().clone_as_blinded()),
CoreBlockId::Slot(slot) => {
let root = self.root(chain)?;
chain
@ -103,11 +100,9 @@ impl BlockId {
pub async fn full_block<T: BeaconChainTypes>(
&self,
chain: &BeaconChain<T>,
) -> Result<SignedBeaconBlock<T::EthSpec>, warp::Rejection> {
) -> Result<Arc<SignedBeaconBlock<T::EthSpec>>, warp::Rejection> {
match &self.0 {
CoreBlockId::Head => chain
.head_beacon_block()
.map_err(warp_utils::reject::beacon_chain_error),
CoreBlockId::Head => Ok(chain.head_beacon_block()),
CoreBlockId::Slot(slot) => {
let root = self.root(chain)?;
chain
@ -122,7 +117,7 @@ impl BlockId {
slot
)));
}
Ok(block)
Ok(Arc::new(block))
}
None => Err(warp_utils::reject::custom_not_found(format!(
"beacon block with root {}",
@ -136,8 +131,8 @@ impl BlockId {
.get_block(&root)
.await
.map_err(warp_utils::reject::beacon_chain_error)
.and_then(|root_opt| {
root_opt.ok_or_else(|| {
.and_then(|block_opt| {
block_opt.map(Arc::new).ok_or_else(|| {
warp_utils::reject::custom_not_found(format!(
"beacon block with root {}",
root

View File

@ -22,7 +22,7 @@ pub fn info<T: BeaconChainTypes>(
pub fn historical_blocks<T: BeaconChainTypes>(
chain: Arc<BeaconChain<T>>,
blocks: Vec<SignedBlindedBeaconBlock<T::EthSpec>>,
blocks: Vec<Arc<SignedBlindedBeaconBlock<T::EthSpec>>>,
) -> Result<AnchorInfo, warp::Rejection> {
chain
.import_historical_block_batch(blocks)

View File

@ -23,7 +23,7 @@ use beacon_chain::{
observed_operations::ObservationOutcome,
validator_monitor::{get_block_delay_ms, timestamp_now},
AttestationError as AttnError, BeaconChain, BeaconChainError, BeaconChainTypes,
HeadSafetyStatus, ProduceBlockVerification, WhenSlotSkipped,
ProduceBlockVerification, WhenSlotSkipped,
};
use block_id::BlockId;
use eth2::types::{self as api_types, EndpointVersion, ValidatorId};
@ -369,9 +369,7 @@ pub fn serve<T: BeaconChainTypes>(
chain: Arc<BeaconChain<T>>| async move {
match *network_globals.sync_state.read() {
SyncState::SyncingFinalized { .. } => {
let head_slot = chain
.best_slot()
.map_err(warp_utils::reject::beacon_chain_error)?;
let head_slot = chain.canonical_head.cached_head().head_slot();
let current_slot =
chain.slot_clock.now_or_genesis().ok_or_else(|| {
@ -404,35 +402,6 @@ pub fn serve<T: BeaconChainTypes>(
)
.untuple_one();
// Create a `warp` filter that rejects requests unless the head has been verified by the
// execution layer.
let only_with_safe_head = warp::any()
.and(chain_filter.clone())
.and_then(move |chain: Arc<BeaconChain<T>>| async move {
let status = chain.head_safety_status().map_err(|e| {
warp_utils::reject::custom_server_error(format!(
"failed to read head safety status: {:?}",
e
))
})?;
match status {
HeadSafetyStatus::Safe(_) => Ok(()),
HeadSafetyStatus::Unsafe(hash) => {
Err(warp_utils::reject::custom_server_error(format!(
"optimistic head hash {:?} has not been verified by the execution layer",
hash
)))
}
HeadSafetyStatus::Invalid(hash) => {
Err(warp_utils::reject::custom_server_error(format!(
"the head block has an invalid payload {:?}, this may be unrecoverable",
hash
)))
}
}
})
.untuple_one();
// Create a `warp` filter that provides access to the logger.
let inner_ctx = ctx.clone();
let log_filter = warp::any().map(move || inner_ctx.log.clone());
@ -451,15 +420,12 @@ pub fn serve<T: BeaconChainTypes>(
.and(chain_filter.clone())
.and_then(|chain: Arc<BeaconChain<T>>| {
blocking_json_task(move || {
chain
.head_info()
.map_err(warp_utils::reject::beacon_chain_error)
.map(|head| api_types::GenesisData {
genesis_time: head.genesis_time,
genesis_validators_root: head.genesis_validators_root,
genesis_fork_version: chain.spec.genesis_fork_version,
})
.map(api_types::GenericResponse::from)
let genesis_data = api_types::GenesisData {
genesis_time: chain.genesis_time,
genesis_validators_root: chain.genesis_validators_root,
genesis_fork_version: chain.spec.genesis_fork_version,
};
Ok(api_types::GenericResponse::from(genesis_data))
})
});
@ -835,10 +801,10 @@ pub fn serve<T: BeaconChainTypes>(
blocking_json_task(move || {
let (root, block) = match (query.slot, query.parent_root) {
// No query parameters, return the canonical head block.
(None, None) => chain
.head_beacon_block()
.map_err(warp_utils::reject::beacon_chain_error)
.map(|block| (block.canonical_root(), block.into()))?,
(None, None) => {
let block = chain.head_beacon_block();
(block.canonical_root(), block.clone_as_blinded())
}
// Only the parent root parameter, do a forwards-iterator lookup.
(None, Some(parent_root)) => {
let parent = BlockId::from_root(parent_root).blinded_block(&chain)?;
@ -945,93 +911,85 @@ pub fn serve<T: BeaconChainTypes>(
.and(network_tx_filter.clone())
.and(log_filter.clone())
.and_then(
|block: SignedBeaconBlock<T::EthSpec>,
|block: Arc<SignedBeaconBlock<T::EthSpec>>,
chain: Arc<BeaconChain<T>>,
network_tx: UnboundedSender<NetworkMessage<T::EthSpec>>,
log: Logger| {
blocking_json_task(move || {
let seen_timestamp = timestamp_now();
log: Logger| async move {
let seen_timestamp = timestamp_now();
// Send the block, regardless of whether or not it is valid. The API
// specification is very clear that this is the desired behaviour.
publish_pubsub_message(
&network_tx,
PubsubMessage::BeaconBlock(Box::new(block.clone())),
)?;
// Send the block, regardless of whether or not it is valid. The API
// specification is very clear that this is the desired behaviour.
publish_pubsub_message(&network_tx, PubsubMessage::BeaconBlock(block.clone()))?;
// Determine the delay after the start of the slot, register it with metrics.
let delay =
get_block_delay_ms(seen_timestamp, block.message(), &chain.slot_clock);
metrics::observe_duration(
&metrics::HTTP_API_BLOCK_BROADCAST_DELAY_TIMES,
delay,
);
// Determine the delay after the start of the slot, register it with metrics.
let delay = get_block_delay_ms(seen_timestamp, block.message(), &chain.slot_clock);
metrics::observe_duration(&metrics::HTTP_API_BLOCK_BROADCAST_DELAY_TIMES, delay);
match chain.process_block(block.clone()) {
Ok(root) => {
info!(
match chain.process_block(block.clone()).await {
Ok(root) => {
info!(
log,
"Valid block from HTTP API";
"block_delay" => ?delay,
"root" => format!("{}", root),
"proposer_index" => block.message().proposer_index(),
"slot" => block.slot(),
);
// Notify the validator monitor.
chain.validator_monitor.read().register_api_block(
seen_timestamp,
block.message(),
root,
&chain.slot_clock,
);
// Update the head since it's likely this block will become the new
// head.
chain
.recompute_head_at_current_slot()
.await
.map_err(warp_utils::reject::beacon_chain_error)?;
// Perform some logging to inform users if their blocks are being produced
// late.
//
// Check to see the thresholds are non-zero to avoid logging errors with small
// slot times (e.g., during testing)
let crit_threshold = chain.slot_clock.unagg_attestation_production_delay();
let error_threshold = crit_threshold / 2;
if delay >= crit_threshold {
crit!(
log,
"Valid block from HTTP API";
"block_delay" => ?delay,
"root" => format!("{}", root),
"proposer_index" => block.message().proposer_index(),
"Block was broadcast too late";
"msg" => "system may be overloaded, block likely to be orphaned",
"delay_ms" => delay.as_millis(),
"slot" => block.slot(),
);
// Notify the validator monitor.
chain.validator_monitor.read().register_api_block(
seen_timestamp,
block.message(),
root,
&chain.slot_clock,
);
// Update the head since it's likely this block will become the new
// head.
chain
.fork_choice()
.map_err(warp_utils::reject::beacon_chain_error)?;
// Perform some logging to inform users if their blocks are being produced
// late.
//
// Check to see the thresholds are non-zero to avoid logging errors with small
// slot times (e.g., during testing)
let crit_threshold = chain.slot_clock.unagg_attestation_production_delay();
let error_threshold = crit_threshold / 2;
if delay >= crit_threshold {
crit!(
log,
"Block was broadcast too late";
"msg" => "system may be overloaded, block likely to be orphaned",
"delay_ms" => delay.as_millis(),
"slot" => block.slot(),
"root" => ?root,
)
} else if delay >= error_threshold {
error!(
log,
"Block broadcast was delayed";
"msg" => "system may be overloaded, block may be orphaned",
"delay_ms" => delay.as_millis(),
"slot" => block.slot(),
"root" => ?root,
)
}
Ok(())
}
Err(e) => {
let msg = format!("{:?}", e);
"root" => ?root,
)
} else if delay >= error_threshold {
error!(
log,
"Invalid block provided to HTTP API";
"reason" => &msg
);
Err(warp_utils::reject::broadcast_without_import(msg))
"Block broadcast was delayed";
"msg" => "system may be overloaded, block may be orphaned",
"delay_ms" => delay.as_millis(),
"slot" => block.slot(),
"root" => ?root,
)
}
Ok(warp::reply::json(&()))
}
})
Err(e) => {
let msg = format!("{:?}", e);
error!(
log,
"Invalid block provided to HTTP API";
"reason" => &msg
);
Err(warp_utils::reject::broadcast_without_import(msg))
}
}
},
);
@ -1049,99 +1007,90 @@ pub fn serve<T: BeaconChainTypes>(
.and(network_tx_filter.clone())
.and(log_filter.clone())
.and_then(
|block: SignedBeaconBlock<T::EthSpec, BlindedPayload<_>>,
|block: Arc<SignedBeaconBlock<T::EthSpec, BlindedPayload<_>>>,
chain: Arc<BeaconChain<T>>,
network_tx: UnboundedSender<NetworkMessage<T::EthSpec>>,
_log: Logger| {
blocking_json_task(move || {
if let Some(el) = chain.execution_layer.as_ref() {
//FIXME(sean): we may not always receive the payload in this response because it
// should be the relay's job to propogate the block. However, since this block is
// already signed and sent this might be ok (so long as the relay validates
// the block before revealing the payload).
_log: Logger| async move {
if let Some(el) = chain.execution_layer.as_ref() {
//FIXME(sean): we may not always receive the payload in this response because it
// should be the relay's job to propogate the block. However, since this block is
// already signed and sent this might be ok (so long as the relay validates
// the block before revealing the payload).
//FIXME(sean) additionally, this endpoint should serve blocks prior to Bellatrix, and should
// be able to support the normal block proposal flow, because at some point full block endpoints
// will be deprecated from the beacon API. This will entail creating full blocks in
// `validator/blinded_blocks`, caching their payloads, and transforming them into blinded
// blocks. We will access the payload of those blocks here. This flow should happen if the
// execution layer has no payload builders or if we have not yet finalized post-merge transition.
let payload = el
.block_on(|el| el.propose_blinded_beacon_block(&block))
.map_err(|e| {
warp_utils::reject::custom_server_error(format!(
"proposal failed: {:?}",
e
))
})?;
let new_block = SignedBeaconBlock::Merge(SignedBeaconBlockMerge {
message: BeaconBlockMerge {
slot: block.message().slot(),
proposer_index: block.message().proposer_index(),
parent_root: block.message().parent_root(),
state_root: block.message().state_root(),
body: BeaconBlockBodyMerge {
randao_reveal: block.message().body().randao_reveal().clone(),
eth1_data: block.message().body().eth1_data().clone(),
graffiti: *block.message().body().graffiti(),
proposer_slashings: block
.message()
.body()
.proposer_slashings()
.clone(),
attester_slashings: block
.message()
.body()
.attester_slashings()
.clone(),
attestations: block.message().body().attestations().clone(),
deposits: block.message().body().deposits().clone(),
voluntary_exits: block
.message()
.body()
.voluntary_exits()
.clone(),
sync_aggregate: block
.message()
.body()
.sync_aggregate()
.unwrap()
.clone(),
execution_payload: payload.into(),
},
//FIXME(sean) additionally, this endpoint should serve blocks prior to Bellatrix, and should
// be able to support the normal block proposal flow, because at some point full block endpoints
// will be deprecated from the beacon API. This will entail creating full blocks in
// `validator/blinded_blocks`, caching their payloads, and transforming them into blinded
// blocks. We will access the payload of those blocks here. This flow should happen if the
// execution layer has no payload builders or if we have not yet finalized post-merge transition.
let payload = el.propose_blinded_beacon_block(&block).await.map_err(|e| {
warp_utils::reject::custom_server_error(format!("proposal failed: {:?}", e))
})?;
let new_block = SignedBeaconBlock::Merge(SignedBeaconBlockMerge {
message: BeaconBlockMerge {
slot: block.message().slot(),
proposer_index: block.message().proposer_index(),
parent_root: block.message().parent_root(),
state_root: block.message().state_root(),
body: BeaconBlockBodyMerge {
randao_reveal: block.message().body().randao_reveal().clone(),
eth1_data: block.message().body().eth1_data().clone(),
graffiti: *block.message().body().graffiti(),
proposer_slashings: block
.message()
.body()
.proposer_slashings()
.clone(),
attester_slashings: block
.message()
.body()
.attester_slashings()
.clone(),
attestations: block.message().body().attestations().clone(),
deposits: block.message().body().deposits().clone(),
voluntary_exits: block.message().body().voluntary_exits().clone(),
sync_aggregate: block
.message()
.body()
.sync_aggregate()
.unwrap()
.clone(),
execution_payload: payload.into(),
},
signature: block.signature().clone(),
});
},
signature: block.signature().clone(),
});
let new_block = Arc::new(new_block);
// Send the block, regardless of whether or not it is valid. The API
// specification is very clear that this is the desired behaviour.
publish_pubsub_message(
&network_tx,
PubsubMessage::BeaconBlock(Box::new(new_block.clone())),
)?;
// Send the block, regardless of whether or not it is valid. The API
// specification is very clear that this is the desired behaviour.
publish_pubsub_message(
&network_tx,
PubsubMessage::BeaconBlock(new_block.clone()),
)?;
match chain.process_block(new_block) {
Ok(_) => {
// Update the head since it's likely this block will become the new
// head.
chain
.fork_choice()
.map_err(warp_utils::reject::beacon_chain_error)?;
match chain.process_block(new_block).await {
Ok(_) => {
// Update the head since it's likely this block will become the new
// head.
chain
.recompute_head_at_current_slot()
.await
.map_err(warp_utils::reject::beacon_chain_error)?;
Ok(())
}
Err(e) => {
let msg = format!("{:?}", e);
Err(warp_utils::reject::broadcast_without_import(msg))
}
Ok(warp::reply::json(&()))
}
Err(e) => {
let msg = format!("{:?}", e);
Err(warp_utils::reject::broadcast_without_import(msg))
}
} else {
Err(warp_utils::reject::custom_server_error(
"no execution layer found".to_string(),
))
}
})
} else {
Err(warp_utils::reject::custom_server_error(
"no execution layer found".to_string(),
))
}
},
);
@ -1401,9 +1350,7 @@ pub fn serve<T: BeaconChainTypes>(
)),
)?;
chain
.import_attester_slashing(slashing)
.map_err(warp_utils::reject::beacon_chain_error)?;
chain.import_attester_slashing(slashing);
}
Ok(())
@ -1744,10 +1691,7 @@ pub fn serve<T: BeaconChainTypes>(
.and_then(
|network_globals: Arc<NetworkGlobals<T::EthSpec>>, chain: Arc<BeaconChain<T>>| {
blocking_json_task(move || {
let head_slot = chain
.head_info()
.map(|info| info.slot)
.map_err(warp_utils::reject::beacon_chain_error)?;
let head_slot = chain.canonical_head.cached_head().head_slot();
let current_slot = chain.slot_clock.now_or_genesis().ok_or_else(|| {
warp_utils::reject::custom_server_error("Unable to read slot clock".into())
})?;
@ -1993,48 +1937,49 @@ pub fn serve<T: BeaconChainTypes>(
|endpoint_version: EndpointVersion,
slot: Slot,
query: api_types::ValidatorBlocksQuery,
chain: Arc<BeaconChain<T>>| {
blocking_json_task(move || {
let randao_reveal = query.randao_reveal.as_ref().map_or_else(
|| {
if query.verify_randao {
Err(warp_utils::reject::custom_bad_request(
"randao_reveal is mandatory unless verify_randao=false".into(),
))
} else {
Ok(Signature::empty())
}
},
|sig_bytes| {
sig_bytes.try_into().map_err(|e| {
warp_utils::reject::custom_bad_request(format!(
"randao reveal is not a valid BLS signature: {:?}",
e
))
})
},
)?;
chain: Arc<BeaconChain<T>>| async move {
let randao_reveal = query.randao_reveal.as_ref().map_or_else(
|| {
if query.verify_randao {
Err(warp_utils::reject::custom_bad_request(
"randao_reveal is mandatory unless verify_randao=false".into(),
))
} else {
Ok(Signature::empty())
}
},
|sig_bytes| {
sig_bytes.try_into().map_err(|e| {
warp_utils::reject::custom_bad_request(format!(
"randao reveal is not a valid BLS signature: {:?}",
e
))
})
},
)?;
let randao_verification = if query.verify_randao {
ProduceBlockVerification::VerifyRandao
} else {
ProduceBlockVerification::NoVerification
};
let randao_verification = if query.verify_randao {
ProduceBlockVerification::VerifyRandao
} else {
ProduceBlockVerification::NoVerification
};
let (block, _) = chain
.produce_block_with_verification::<FullPayload<T::EthSpec>>(
randao_reveal,
slot,
query.graffiti.map(Into::into),
randao_verification,
)
.map_err(warp_utils::reject::block_production_error)?;
let fork_name = block
.to_ref()
.fork_name(&chain.spec)
.map_err(inconsistent_fork_rejection)?;
fork_versioned_response(endpoint_version, fork_name, block)
})
let (block, _) = chain
.produce_block_with_verification::<FullPayload<T::EthSpec>>(
randao_reveal,
slot,
query.graffiti.map(Into::into),
randao_verification,
)
.await
.map_err(warp_utils::reject::block_production_error)?;
let fork_name = block
.to_ref()
.fork_name(&chain.spec)
.map_err(inconsistent_fork_rejection)?;
fork_versioned_response(endpoint_version, fork_name, block)
.map(|response| warp::reply::json(&response))
},
);
@ -2055,48 +2000,48 @@ pub fn serve<T: BeaconChainTypes>(
|endpoint_version: EndpointVersion,
slot: Slot,
query: api_types::ValidatorBlocksQuery,
chain: Arc<BeaconChain<T>>| {
blocking_json_task(move || {
let randao_reveal = query.randao_reveal.as_ref().map_or_else(
|| {
if query.verify_randao {
Err(warp_utils::reject::custom_bad_request(
"randao_reveal is mandatory unless verify_randao=false".into(),
))
} else {
Ok(Signature::empty())
}
},
|sig_bytes| {
sig_bytes.try_into().map_err(|e| {
warp_utils::reject::custom_bad_request(format!(
"randao reveal is not a valid BLS signature: {:?}",
e
))
})
},
)?;
chain: Arc<BeaconChain<T>>| async move {
let randao_reveal = query.randao_reveal.as_ref().map_or_else(
|| {
if query.verify_randao {
Err(warp_utils::reject::custom_bad_request(
"randao_reveal is mandatory unless verify_randao=false".into(),
))
} else {
Ok(Signature::empty())
}
},
|sig_bytes| {
sig_bytes.try_into().map_err(|e| {
warp_utils::reject::custom_bad_request(format!(
"randao reveal is not a valid BLS signature: {:?}",
e
))
})
},
)?;
let randao_verification = if query.verify_randao {
ProduceBlockVerification::VerifyRandao
} else {
ProduceBlockVerification::NoVerification
};
let randao_verification = if query.verify_randao {
ProduceBlockVerification::VerifyRandao
} else {
ProduceBlockVerification::NoVerification
};
let (block, _) = chain
.produce_block_with_verification::<BlindedPayload<T::EthSpec>>(
randao_reveal,
slot,
query.graffiti.map(Into::into),
randao_verification,
)
.map_err(warp_utils::reject::block_production_error)?;
let fork_name = block
.to_ref()
.fork_name(&chain.spec)
.map_err(inconsistent_fork_rejection)?;
fork_versioned_response(endpoint_version, fork_name, block)
})
let (block, _) = chain
.produce_block_with_verification::<BlindedPayload<T::EthSpec>>(
randao_reveal,
slot,
query.graffiti.map(Into::into),
randao_verification,
)
.await
.map_err(warp_utils::reject::block_production_error)?;
let fork_name = block
.to_ref()
.fork_name(&chain.spec)
.map_err(inconsistent_fork_rejection)?;
fork_versioned_response(endpoint_version, fork_name, block)
.map(|response| warp::reply::json(&response))
},
);
@ -2107,7 +2052,6 @@ pub fn serve<T: BeaconChainTypes>(
.and(warp::path::end())
.and(warp::query::<api_types::ValidatorAttestationDataQuery>())
.and(not_while_syncing_filter.clone())
.and(only_with_safe_head.clone())
.and(chain_filter.clone())
.and_then(
|query: api_types::ValidatorAttestationDataQuery, chain: Arc<BeaconChain<T>>| {
@ -2140,7 +2084,6 @@ pub fn serve<T: BeaconChainTypes>(
.and(warp::path::end())
.and(warp::query::<api_types::ValidatorAggregateAttestationQuery>())
.and(not_while_syncing_filter.clone())
.and(only_with_safe_head.clone())
.and(chain_filter.clone())
.and_then(
|query: api_types::ValidatorAggregateAttestationQuery, chain: Arc<BeaconChain<T>>| {
@ -2217,7 +2160,6 @@ pub fn serve<T: BeaconChainTypes>(
.and(warp::path::end())
.and(warp::query::<SyncContributionData>())
.and(not_while_syncing_filter.clone())
.and(only_with_safe_head)
.and(chain_filter.clone())
.and_then(
|sync_committee_data: SyncContributionData, chain: Arc<BeaconChain<T>>| {
@ -2413,40 +2355,39 @@ pub fn serve<T: BeaconChainTypes>(
.and_then(
|chain: Arc<BeaconChain<T>>,
log: Logger,
preparation_data: Vec<ProposerPreparationData>| {
blocking_json_task(move || {
let execution_layer = chain
.execution_layer
.as_ref()
.ok_or(BeaconChainError::ExecutionLayerMissing)
.map_err(warp_utils::reject::beacon_chain_error)?;
let current_epoch = chain
.epoch()
.map_err(warp_utils::reject::beacon_chain_error)?;
preparation_data: Vec<ProposerPreparationData>| async move {
let execution_layer = chain
.execution_layer
.as_ref()
.ok_or(BeaconChainError::ExecutionLayerMissing)
.map_err(warp_utils::reject::beacon_chain_error)?;
debug!(
log,
"Received proposer preparation data";
"count" => preparation_data.len(),
);
let current_slot = chain
.slot()
.map_err(warp_utils::reject::beacon_chain_error)?;
let current_epoch = current_slot.epoch(T::EthSpec::slots_per_epoch());
execution_layer
.update_proposer_preparation_blocking(current_epoch, &preparation_data)
.map_err(|_e| {
warp_utils::reject::custom_bad_request(
"error processing proposer preparations".to_string(),
)
})?;
debug!(
log,
"Received proposer preparation data";
"count" => preparation_data.len(),
);
chain.prepare_beacon_proposer_blocking().map_err(|e| {
execution_layer
.update_proposer_preparation(current_epoch, &preparation_data)
.await;
chain
.prepare_beacon_proposer(current_slot)
.await
.map_err(|e| {
warp_utils::reject::custom_bad_request(format!(
"error updating proposer preparations: {:?}",
e
))
})?;
Ok(())
})
Ok::<_, warp::reject::Rejection>(warp::reply::json(&()))
},
);
@ -2461,69 +2402,66 @@ pub fn serve<T: BeaconChainTypes>(
.and_then(
|chain: Arc<BeaconChain<T>>,
log: Logger,
register_val_data: Vec<SignedValidatorRegistrationData>| {
blocking_json_task(move || {
let execution_layer = chain
.execution_layer
.as_ref()
.ok_or(BeaconChainError::ExecutionLayerMissing)
.map_err(warp_utils::reject::beacon_chain_error)?;
let current_epoch = chain
.slot_clock
.now_or_genesis()
.ok_or(BeaconChainError::UnableToReadSlot)
.map_err(warp_utils::reject::beacon_chain_error)?
.epoch(T::EthSpec::slots_per_epoch());
register_val_data: Vec<SignedValidatorRegistrationData>| async move {
let execution_layer = chain
.execution_layer
.as_ref()
.ok_or(BeaconChainError::ExecutionLayerMissing)
.map_err(warp_utils::reject::beacon_chain_error)?;
let current_slot = chain
.slot_clock
.now_or_genesis()
.ok_or(BeaconChainError::UnableToReadSlot)
.map_err(warp_utils::reject::beacon_chain_error)?;
let current_epoch = current_slot.epoch(T::EthSpec::slots_per_epoch());
debug!(
log,
"Received register validator request";
"count" => register_val_data.len(),
);
debug!(
log,
"Received register validator request";
"count" => register_val_data.len(),
);
let preparation_data = register_val_data
.iter()
.filter_map(|register_data| {
chain
.validator_index(&register_data.message.pubkey)
.ok()
.flatten()
.map(|validator_index| ProposerPreparationData {
validator_index: validator_index as u64,
fee_recipient: register_data.message.fee_recipient,
})
})
.collect::<Vec<_>>();
let preparation_data = register_val_data
.iter()
.filter_map(|register_data| {
chain
.validator_index(&register_data.message.pubkey)
.ok()
.flatten()
.map(|validator_index| ProposerPreparationData {
validator_index: validator_index as u64,
fee_recipient: register_data.message.fee_recipient,
})
})
.collect::<Vec<_>>();
debug!(
log,
"Resolved validator request pubkeys";
"count" => preparation_data.len()
);
debug!(
log,
"Resolved validator request pubkeys";
"count" => preparation_data.len()
);
// Update the prepare beacon proposer cache based on this request.
execution_layer
.update_proposer_preparation_blocking(current_epoch, &preparation_data)
.map_err(|_e| {
warp_utils::reject::custom_bad_request(
"error processing proposer preparations".to_string(),
)
})?;
// Update the prepare beacon proposer cache based on this request.
execution_layer
.update_proposer_preparation(current_epoch, &preparation_data)
.await;
// Call prepare beacon proposer blocking with the latest update in order to make
// sure we have a local payload to fall back to in the event of the blined block
// flow failing.
chain.prepare_beacon_proposer_blocking().map_err(|e| {
// Call prepare beacon proposer blocking with the latest update in order to make
// sure we have a local payload to fall back to in the event of the blined block
// flow failing.
chain
.prepare_beacon_proposer(current_slot)
.await
.map_err(|e| {
warp_utils::reject::custom_bad_request(format!(
"error updating proposer preparations: {:?}",
e
))
})?;
//TODO(sean): In the MEV-boost PR, add a call here to send the update request to the builder
//TODO(sean): In the MEV-boost PR, add a call here to send the update request to the builder
Ok(())
})
Ok::<_, warp::Rejection>(warp::reply::json(&()))
},
);
// POST validator/sync_committee_subscriptions
@ -2689,7 +2627,11 @@ pub fn serve<T: BeaconChainTypes>(
.and_then(|chain: Arc<BeaconChain<T>>| {
blocking_task(move || {
Ok::<_, warp::Rejection>(warp::reply::json(&api_types::GenericResponseRef::from(
chain.fork_choice.read().proto_array().core_proto_array(),
chain
.canonical_head
.fork_choice_read_lock()
.proto_array()
.core_proto_array(),
)))
})
});
@ -2732,9 +2674,6 @@ pub fn serve<T: BeaconChainTypes>(
.and(chain_filter.clone())
.and_then(|chain: Arc<BeaconChain<T>>| {
blocking_json_task(move || {
let head_info = chain
.head_info()
.map_err(warp_utils::reject::beacon_chain_error)?;
let current_slot_opt = chain.slot().ok();
chain
@ -2746,7 +2685,7 @@ pub fn serve<T: BeaconChainTypes>(
)
})
.and_then(|eth1| {
eth1.sync_status(head_info.genesis_time, current_slot_opt, &chain.spec)
eth1.sync_status(chain.genesis_time, current_slot_opt, &chain.spec)
.ok_or_else(|| {
warp_utils::reject::custom_server_error(
"Unable to determine Eth1 sync status".to_string(),
@ -2869,7 +2808,7 @@ pub fn serve<T: BeaconChainTypes>(
.and(chain_filter.clone())
.and(log_filter.clone())
.and_then(
|blocks: Vec<SignedBlindedBeaconBlock<T::EthSpec>>,
|blocks: Vec<Arc<SignedBlindedBeaconBlock<T::EthSpec>>>,
chain: Arc<BeaconChain<T>>,
log: Logger| {
info!(

View File

@ -55,7 +55,7 @@ pub fn proposer_duties<T: BeaconChainTypes>(
.safe_add(1)
.map_err(warp_utils::reject::arith_error)?
{
let (proposers, dependent_root, _) =
let (proposers, dependent_root, _execution_status, _fork) =
compute_proposer_duties_from_head(request_epoch, chain)
.map_err(warp_utils::reject::beacon_chain_error)?;
convert_to_api_response(chain, request_epoch, dependent_root, proposers)
@ -88,16 +88,23 @@ fn try_proposer_duties_from_cache<T: BeaconChainTypes>(
request_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 (head_slot, head_block_root, head_decision_root) = {
let head = chain.canonical_head.cached_head();
let head_block_root = head.head_block_root();
let decision_root = head
.snapshot
.beacon_state
.proposer_shuffling_decision_root(head_block_root)
.map_err(warp_utils::reject::beacon_state_error)?;
(head.head_slot(), head_block_root, decision_root)
};
let head_epoch = head_slot.epoch(T::EthSpec::slots_per_epoch());
let dependent_root = match head_epoch.cmp(&request_epoch) {
// head_epoch == request_epoch
Ordering::Equal => head.proposer_shuffling_decision_root,
Ordering::Equal => head_decision_root,
// head_epoch < request_epoch
Ordering::Less => head.block_root,
Ordering::Less => head_block_root,
// head_epoch > request_epoch
Ordering::Greater => {
return Err(warp_utils::reject::custom_server_error(format!(
@ -132,8 +139,9 @@ fn compute_and_cache_proposer_duties<T: BeaconChainTypes>(
current_epoch: Epoch,
chain: &BeaconChain<T>,
) -> Result<ApiDuties, warp::reject::Rejection> {
let (indices, dependent_root, fork) = compute_proposer_duties_from_head(current_epoch, chain)
.map_err(warp_utils::reject::beacon_chain_error)?;
let (indices, dependent_root, _execution_status, fork) =
compute_proposer_duties_from_head(current_epoch, chain)
.map_err(warp_utils::reject::beacon_chain_error)?;
// Prime the proposer shuffling cache with the newly-learned value.
chain

View File

@ -18,27 +18,23 @@ impl StateId {
chain: &BeaconChain<T>,
) -> Result<Hash256, warp::Rejection> {
let slot = match &self.0 {
CoreStateId::Head => {
return chain
.head_info()
.map(|head| head.state_root)
.map_err(warp_utils::reject::beacon_chain_error)
}
CoreStateId::Head => return Ok(chain.canonical_head.cached_head().head_state_root()),
CoreStateId::Genesis => return Ok(chain.genesis_state_root),
CoreStateId::Finalized => chain.head_info().map(|head| {
head.finalized_checkpoint
.epoch
.start_slot(T::EthSpec::slots_per_epoch())
}),
CoreStateId::Justified => chain.head_info().map(|head| {
head.current_justified_checkpoint
.epoch
.start_slot(T::EthSpec::slots_per_epoch())
}),
CoreStateId::Slot(slot) => Ok(*slot),
CoreStateId::Finalized => chain
.canonical_head
.cached_head()
.finalized_checkpoint()
.epoch
.start_slot(T::EthSpec::slots_per_epoch()),
CoreStateId::Justified => chain
.canonical_head
.cached_head()
.justified_checkpoint()
.epoch
.start_slot(T::EthSpec::slots_per_epoch()),
CoreStateId::Slot(slot) => *slot,
CoreStateId::Root(root) => return Ok(*root),
}
.map_err(warp_utils::reject::beacon_chain_error)?;
};
chain
.state_root_at_slot(slot)
@ -62,11 +58,7 @@ impl StateId {
chain: &BeaconChain<T>,
) -> Result<BeaconState<T::EthSpec>, warp::Rejection> {
let (state_root, slot_opt) = match &self.0 {
CoreStateId::Head => {
return chain
.head_beacon_state()
.map_err(warp_utils::reject::beacon_chain_error)
}
CoreStateId::Head => return Ok(chain.head_beacon_state_cloned()),
CoreStateId::Slot(slot) => (self.root(chain)?, Some(*slot)),
_ => (self.root(chain)?, None),
};

View File

@ -45,6 +45,7 @@ async fn sync_committee_duties_across_fork() {
genesis_state_root,
&all_validators,
)
.await
.unwrap();
harness.advance_slot();
@ -61,6 +62,7 @@ async fn sync_committee_duties_across_fork() {
let state_root = state.canonical_root();
harness
.add_attested_block_at_slot(fork_slot, state, state_root, &all_validators)
.await
.unwrap();
assert_eq!(
@ -244,6 +246,7 @@ async fn sync_committee_indices_across_fork() {
genesis_state_root,
&all_validators,
)
.await
.unwrap();
harness.advance_slot();
@ -277,6 +280,7 @@ async fn sync_committee_indices_across_fork() {
let state_root = state.canonical_root();
harness
.add_attested_block_at_slot(fork_slot + 1, state, state_root, &all_validators)
.await
.unwrap();
let current_period = fork_epoch.sync_committee_period(&spec).unwrap();

View File

@ -47,11 +47,13 @@ pub async fn fork_choice_before_proposal() {
// Create some chain depth.
harness.advance_slot();
harness.extend_chain(
num_initial as usize,
BlockStrategy::OnCanonicalHead,
AttestationStrategy::AllValidators,
);
harness
.extend_chain(
num_initial as usize,
BlockStrategy::OnCanonicalHead,
AttestationStrategy::AllValidators,
)
.await;
// We set up the following block graph, where B is a block that is temporarily orphaned by C,
// but is then reinstated and built upon by D.
@ -64,8 +66,8 @@ pub async fn fork_choice_before_proposal() {
let slot_d = slot_a + 3;
let state_a = harness.get_current_state();
let (block_b, state_b) = harness.make_block(state_a.clone(), slot_b);
let block_root_b = harness.process_block(slot_b, block_b).unwrap();
let (block_b, state_b) = harness.make_block(state_a.clone(), slot_b).await;
let block_root_b = harness.process_block(slot_b, block_b).await.unwrap();
// Create attestations to B but keep them in reserve until after C has been processed.
let attestations_b = harness.make_attestations(
@ -76,8 +78,11 @@ pub async fn fork_choice_before_proposal() {
slot_b,
);
let (block_c, state_c) = harness.make_block(state_a, slot_c);
let block_root_c = harness.process_block(slot_c, block_c.clone()).unwrap();
let (block_c, state_c) = harness.make_block(state_a, slot_c).await;
let block_root_c = harness
.process_block(slot_c, block_c.clone())
.await
.unwrap();
// Create attestations to C from a small number of validators and process them immediately.
let attestations_c = harness.make_attestations(
@ -94,7 +99,7 @@ pub async fn fork_choice_before_proposal() {
// Due to proposer boost, the head should be C during slot C.
assert_eq!(
harness.chain.head_info().unwrap().block_root,
harness.chain.canonical_head.cached_head().head_block_root(),
block_root_c.into()
);
@ -102,7 +107,7 @@ pub async fn fork_choice_before_proposal() {
// Manually prod the per-slot task, because the slot timer doesn't run in the background in
// these tests.
harness.advance_slot();
harness.chain.per_slot_task();
harness.chain.per_slot_task().await;
let proposer_index = state_b
.get_beacon_proposer_index(slot_d, &harness.chain.spec)
@ -119,7 +124,7 @@ pub async fn fork_choice_before_proposal() {
// Head is now B.
assert_eq!(
harness.chain.head_info().unwrap().block_root,
harness.chain.canonical_head.cached_head().head_block_root(),
block_root_b.into()
);
// D's parent is B.

View File

@ -11,7 +11,6 @@ use eth2::{
types::*,
BeaconNodeHttpClient, Error, StatusCode, Timeouts,
};
use execution_layer::test_utils::MockExecutionLayer;
use futures::stream::{Stream, StreamExt};
use futures::FutureExt;
use lighthouse_network::{Enr, EnrExt, PeerId};
@ -21,7 +20,6 @@ use slot_clock::SlotClock;
use state_processing::per_slot_processing;
use std::convert::TryInto;
use std::sync::Arc;
use task_executor::test_utils::TestRuntime;
use tokio::sync::{mpsc, oneshot};
use tokio::time::Duration;
use tree_hash::TreeHash;
@ -52,6 +50,7 @@ const SKIPPED_SLOTS: &[u64] = &[
];
struct ApiTester {
harness: Arc<BeaconChainHarness<EphemeralHarnessType<E>>>,
chain: Arc<BeaconChain<EphemeralHarnessType<E>>>,
client: BeaconNodeHttpClient,
next_block: SignedBeaconBlock<E>,
@ -62,14 +61,9 @@ struct ApiTester {
proposer_slashing: ProposerSlashing,
voluntary_exit: SignedVoluntaryExit,
_server_shutdown: oneshot::Sender<()>,
validator_keypairs: Vec<Keypair>,
network_rx: mpsc::UnboundedReceiver<NetworkMessage<E>>,
local_enr: Enr,
external_peer_id: PeerId,
// This is never directly accessed, but adding it creates a payload cache, which we use in tests here.
#[allow(dead_code)]
mock_el: Option<MockExecutionLayer<E>>,
_runtime: TestRuntime,
}
impl ApiTester {
@ -81,12 +75,14 @@ impl ApiTester {
}
pub async fn new_from_spec(spec: ChainSpec) -> Self {
let harness = BeaconChainHarness::builder(MainnetEthSpec)
.spec(spec.clone())
.deterministic_keypairs(VALIDATOR_COUNT)
.fresh_ephemeral_store()
.mock_execution_layer()
.build();
let harness = Arc::new(
BeaconChainHarness::builder(MainnetEthSpec)
.spec(spec.clone())
.deterministic_keypairs(VALIDATOR_COUNT)
.fresh_ephemeral_store()
.mock_execution_layer()
.build(),
);
harness.advance_slot();
@ -94,17 +90,19 @@ impl ApiTester {
let slot = harness.chain.slot().unwrap().as_u64();
if !SKIPPED_SLOTS.contains(&slot) {
harness.extend_chain(
1,
BlockStrategy::OnCanonicalHead,
AttestationStrategy::AllValidators,
);
harness
.extend_chain(
1,
BlockStrategy::OnCanonicalHead,
AttestationStrategy::AllValidators,
)
.await;
}
harness.advance_slot();
}
let head = harness.chain.head().unwrap();
let head = harness.chain.head_snapshot();
assert_eq!(
harness.chain.slot().unwrap(),
@ -112,12 +110,14 @@ impl ApiTester {
"precondition: current slot is one after head"
);
let (next_block, _next_state) =
harness.make_block(head.beacon_state.clone(), harness.chain.slot().unwrap());
let (next_block, _next_state) = harness
.make_block(head.beacon_state.clone(), harness.chain.slot().unwrap())
.await;
// `make_block` adds random graffiti, so this will produce an alternate block
let (reorg_block, _reorg_state) =
harness.make_block(head.beacon_state.clone(), harness.chain.slot().unwrap());
let (reorg_block, _reorg_state) = harness
.make_block(head.beacon_state.clone(), harness.chain.slot().unwrap())
.await;
let head_state_root = head.beacon_state_root();
let attestations = harness
@ -168,15 +168,19 @@ impl ApiTester {
let chain = harness.chain.clone();
assert_eq!(
chain.head_info().unwrap().finalized_checkpoint.epoch,
chain
.canonical_head
.cached_head()
.finalized_checkpoint()
.epoch,
2,
"precondition: finality"
);
assert_eq!(
chain
.head_info()
.unwrap()
.current_justified_checkpoint
.canonical_head
.cached_head()
.justified_checkpoint()
.epoch,
3,
"precondition: justification"
@ -206,6 +210,7 @@ impl ApiTester {
);
Self {
harness,
chain,
client,
next_block,
@ -216,32 +221,33 @@ impl ApiTester {
proposer_slashing,
voluntary_exit,
_server_shutdown: shutdown_tx,
validator_keypairs: harness.validator_keypairs,
network_rx,
local_enr,
external_peer_id,
mock_el: harness.mock_execution_layer,
_runtime: harness.runtime,
}
}
pub async fn new_from_genesis() -> Self {
let harness = BeaconChainHarness::builder(MainnetEthSpec)
.default_spec()
.deterministic_keypairs(VALIDATOR_COUNT)
.fresh_ephemeral_store()
.build();
let harness = Arc::new(
BeaconChainHarness::builder(MainnetEthSpec)
.default_spec()
.deterministic_keypairs(VALIDATOR_COUNT)
.fresh_ephemeral_store()
.build(),
);
harness.advance_slot();
let head = harness.chain.head().unwrap();
let head = harness.chain.head_snapshot();
let (next_block, _next_state) =
harness.make_block(head.beacon_state.clone(), harness.chain.slot().unwrap());
let (next_block, _next_state) = harness
.make_block(head.beacon_state.clone(), harness.chain.slot().unwrap())
.await;
// `make_block` adds random graffiti, so this will produce an alternate block
let (reorg_block, _reorg_state) =
harness.make_block(head.beacon_state.clone(), harness.chain.slot().unwrap());
let (reorg_block, _reorg_state) = harness
.make_block(head.beacon_state.clone(), harness.chain.slot().unwrap())
.await;
let head_state_root = head.beacon_state_root();
let attestations = harness
@ -286,6 +292,7 @@ impl ApiTester {
);
Self {
harness,
chain,
client,
next_block,
@ -296,15 +303,16 @@ impl ApiTester {
proposer_slashing,
voluntary_exit,
_server_shutdown: shutdown_tx,
validator_keypairs: harness.validator_keypairs,
network_rx,
local_enr,
external_peer_id,
mock_el: None,
_runtime: harness.runtime,
}
}
fn validator_keypairs(&self) -> &[Keypair] {
&self.harness.validator_keypairs
}
fn skip_slots(self, count: u64) -> Self {
for _ in 0..count {
self.chain
@ -329,7 +337,9 @@ impl ApiTester {
StateId::Slot(Slot::from(SKIPPED_SLOTS[3])),
StateId::Root(Hash256::zero()),
];
ids.push(StateId::Root(self.chain.head_info().unwrap().state_root));
ids.push(StateId::Root(
self.chain.canonical_head.cached_head().head_state_root(),
));
ids
}
@ -347,13 +357,20 @@ impl ApiTester {
BlockId::Slot(Slot::from(SKIPPED_SLOTS[3])),
BlockId::Root(Hash256::zero()),
];
ids.push(BlockId::Root(self.chain.head_info().unwrap().block_root));
ids.push(BlockId::Root(
self.chain.canonical_head.cached_head().head_block_root(),
));
ids
}
fn get_state(&self, state_id: StateId) -> Option<BeaconState<E>> {
match state_id {
StateId::Head => Some(self.chain.head().unwrap().beacon_state),
StateId::Head => Some(
self.chain
.head_snapshot()
.beacon_state
.clone_with_only_committee_caches(),
),
StateId::Genesis => self
.chain
.get_state(&self.chain.genesis_state_root, None)
@ -361,9 +378,9 @@ impl ApiTester {
StateId::Finalized => {
let finalized_slot = self
.chain
.head_info()
.unwrap()
.finalized_checkpoint
.canonical_head
.cached_head()
.finalized_checkpoint()
.epoch
.start_slot(E::slots_per_epoch());
@ -378,9 +395,9 @@ impl ApiTester {
StateId::Justified => {
let justified_slot = self
.chain
.head_info()
.unwrap()
.current_justified_checkpoint
.canonical_head
.cached_head()
.justified_checkpoint()
.epoch
.start_slot(E::slots_per_epoch());
@ -404,7 +421,7 @@ impl ApiTester {
pub async fn test_beacon_genesis(self) -> Self {
let result = self.client.get_beacon_genesis().await.unwrap().data;
let state = self.chain.head().unwrap().beacon_state;
let state = &self.chain.head_snapshot().beacon_state;
let expected = GenesisData {
genesis_time: state.genesis_time(),
genesis_validators_root: state.genesis_validators_root(),
@ -426,14 +443,14 @@ impl ApiTester {
.map(|res| res.data.root);
let expected = match state_id {
StateId::Head => Some(self.chain.head_info().unwrap().state_root),
StateId::Head => Some(self.chain.canonical_head.cached_head().head_state_root()),
StateId::Genesis => Some(self.chain.genesis_state_root),
StateId::Finalized => {
let finalized_slot = self
.chain
.head_info()
.unwrap()
.finalized_checkpoint
.canonical_head
.cached_head()
.finalized_checkpoint()
.epoch
.start_slot(E::slots_per_epoch());
@ -442,9 +459,9 @@ impl ApiTester {
StateId::Justified => {
let justified_slot = self
.chain
.head_info()
.unwrap()
.current_justified_checkpoint
.canonical_head
.cached_head()
.justified_checkpoint()
.epoch
.start_slot(E::slots_per_epoch());
@ -754,14 +771,20 @@ impl ApiTester {
fn get_block_root(&self, block_id: BlockId) -> Option<Hash256> {
match block_id {
BlockId::Head => Some(self.chain.head_info().unwrap().block_root),
BlockId::Head => Some(self.chain.canonical_head.cached_head().head_block_root()),
BlockId::Genesis => Some(self.chain.genesis_block_root),
BlockId::Finalized => Some(self.chain.head_info().unwrap().finalized_checkpoint.root),
BlockId::Finalized => Some(
self.chain
.canonical_head
.cached_head()
.finalized_checkpoint()
.root,
),
BlockId::Justified => Some(
self.chain
.head_info()
.unwrap()
.current_justified_checkpoint
.canonical_head
.cached_head()
.justified_checkpoint()
.root,
),
BlockId::Slot(slot) => self
@ -1322,7 +1345,7 @@ impl ApiTester {
pub async fn test_get_node_syncing(self) -> Self {
let result = self.client.get_node_syncing().await.unwrap().data;
let head_slot = self.chain.head_info().unwrap().slot;
let head_slot = self.chain.canonical_head.cached_head().head_slot();
let sync_distance = self.chain.slot().unwrap() - head_slot;
let expected = SyncingData {
@ -1536,7 +1559,7 @@ impl ApiTester {
}
fn validator_count(&self) -> usize {
self.chain.head().unwrap().beacon_state.validators().len()
self.chain.head_snapshot().beacon_state.validators().len()
}
fn interesting_validator_indices(&self) -> Vec<Vec<u64>> {
@ -1621,7 +1644,7 @@ impl ApiTester {
WhenSlotSkipped::Prev,
)
.unwrap()
.unwrap_or(self.chain.head_beacon_block_root().unwrap());
.unwrap_or(self.chain.head_beacon_block_root());
assert_eq!(results.dependent_root, dependent_root);
@ -1696,7 +1719,7 @@ impl ApiTester {
WhenSlotSkipped::Prev,
)
.unwrap()
.unwrap_or(self.chain.head_beacon_block_root().unwrap());
.unwrap_or(self.chain.head_beacon_block_root());
// 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
@ -1824,7 +1847,7 @@ impl ApiTester {
WhenSlotSkipped::Prev,
)
.unwrap()
.unwrap_or(self.chain.head_beacon_block_root().unwrap());
.unwrap_or(self.chain.head_beacon_block_root());
self.client
.get_validator_duties_proposer(current_epoch)
@ -1878,7 +1901,7 @@ impl ApiTester {
}
pub async fn test_block_production(self) -> Self {
let fork = self.chain.head_info().unwrap().fork;
let fork = self.chain.canonical_head.cached_head().head_fork();
let genesis_validators_root = self.chain.genesis_validators_root;
for _ in 0..E::slots_per_epoch() * 3 {
@ -1898,7 +1921,7 @@ impl ApiTester {
let proposer_pubkey = (&proposer_pubkey_bytes).try_into().unwrap();
let sk = self
.validator_keypairs
.validator_keypairs()
.iter()
.find(|kp| kp.pk == proposer_pubkey)
.map(|kp| kp.sk.clone())
@ -1926,7 +1949,7 @@ impl ApiTester {
self.client.post_beacon_blocks(&signed_block).await.unwrap();
assert_eq!(self.chain.head_beacon_block().unwrap(), signed_block);
assert_eq!(self.chain.head_beacon_block().as_ref(), &signed_block);
self.chain.slot_clock.set_slot(slot.as_u64() + 1);
}
@ -1957,7 +1980,7 @@ impl ApiTester {
}
pub async fn test_block_production_verify_randao_invalid(self) -> Self {
let fork = self.chain.head_info().unwrap().fork;
let fork = self.chain.canonical_head.cached_head().head_fork();
let genesis_validators_root = self.chain.genesis_validators_root;
for _ in 0..E::slots_per_epoch() {
@ -1977,7 +2000,7 @@ impl ApiTester {
let proposer_pubkey = (&proposer_pubkey_bytes).try_into().unwrap();
let sk = self
.validator_keypairs
.validator_keypairs()
.iter()
.find(|kp| kp.pk == proposer_pubkey)
.map(|kp| kp.sk.clone())
@ -2040,7 +2063,7 @@ impl ApiTester {
}
pub async fn test_get_validator_attestation_data(self) -> Self {
let mut state = self.chain.head_beacon_state().unwrap();
let mut state = self.chain.head_beacon_state_cloned();
let slot = state.slot();
state
.build_committee_cache(RelativeEpoch::Current, &self.chain.spec)
@ -2070,7 +2093,6 @@ impl ApiTester {
let attestation = self
.chain
.head_beacon_block()
.unwrap()
.message()
.body()
.attestations()[0]
@ -2098,7 +2120,7 @@ impl ApiTester {
let slot = self.chain.slot().unwrap();
let epoch = self.chain.epoch().unwrap();
let mut head = self.chain.head().unwrap();
let mut head = self.chain.head_snapshot().as_ref().clone();
while head.beacon_state.current_epoch() < epoch {
per_slot_processing(&mut head.beacon_state, None, &self.chain.spec).unwrap();
}
@ -2114,7 +2136,7 @@ impl ApiTester {
.client
.post_validator_duties_attester(
epoch,
(0..self.validator_keypairs.len() as u64)
(0..self.validator_keypairs().len() as u64)
.collect::<Vec<u64>>()
.as_slice(),
)
@ -2123,7 +2145,7 @@ impl ApiTester {
.data;
let (i, kp, duty, proof) = self
.validator_keypairs
.validator_keypairs()
.iter()
.enumerate()
.find_map(|(i, kp)| {
@ -2238,9 +2260,9 @@ impl ApiTester {
let mut registrations = vec![];
let mut fee_recipients = vec![];
let fork = self.chain.head().unwrap().beacon_state.fork();
let fork = self.chain.head_snapshot().beacon_state.fork();
for (val_index, keypair) in self.validator_keypairs.iter().enumerate() {
for (val_index, keypair) in self.validator_keypairs().iter().enumerate() {
let pubkey = keypair.pk.compress();
let fee_recipient = Address::from_low_u64_be(val_index as u64);
@ -2273,8 +2295,7 @@ impl ApiTester {
for (val_index, (_, fee_recipient)) in self
.chain
.head()
.unwrap()
.head_snapshot()
.beacon_state
.validators()
.into_iter()
@ -2416,7 +2437,7 @@ impl ApiTester {
pub async fn test_post_lighthouse_liveness(self) -> Self {
let epoch = self.chain.epoch().unwrap();
let head_state = self.chain.head_beacon_state().unwrap();
let head_state = self.chain.head_beacon_state_cloned();
let indices = (0..head_state.validators().len())
.map(|i| i as u64)
.collect::<Vec<_>>();
@ -2533,7 +2554,7 @@ impl ApiTester {
let block_root = self.next_block.canonical_root();
// current_duty_dependent_root = block root because this is the first slot of the epoch
let current_duty_dependent_root = self.chain.head_beacon_block_root().unwrap();
let current_duty_dependent_root = self.chain.head_beacon_block_root();
let current_slot = self.chain.slot().unwrap();
let next_slot = self.next_block.slot();
let finalization_distance = E::slots_per_epoch() * 2;
@ -2556,17 +2577,21 @@ impl ApiTester {
epoch_transition: true,
});
let finalized_block_root = self
.chain
.block_root_at_slot(next_slot - finalization_distance, WhenSlotSkipped::Prev)
.unwrap()
.unwrap();
let finalized_block = self
.chain
.get_blinded_block(&finalized_block_root)
.unwrap()
.unwrap();
let finalized_state_root = finalized_block.state_root();
let expected_finalized = EventKind::FinalizedCheckpoint(SseFinalizedCheckpoint {
block: self
.chain
.block_root_at_slot(next_slot - finalization_distance, WhenSlotSkipped::Prev)
.unwrap()
.unwrap(),
state: self
.chain
.state_root_at_slot(next_slot - finalization_distance)
.unwrap()
.unwrap(),
block: finalized_block_root,
state: finalized_state_root,
epoch: Epoch::new(3),
});
@ -2578,7 +2603,7 @@ impl ApiTester {
let block_events = poll_events(&mut events_future, 3, Duration::from_millis(10000)).await;
assert_eq!(
block_events.as_slice(),
&[expected_block, expected_finalized, expected_head]
&[expected_block, expected_head, expected_finalized]
);
// Test a reorg event

View File

@ -1358,9 +1358,9 @@ pub enum Response<TSpec: EthSpec> {
/// A Status message.
Status(StatusMessage),
/// A response to a get BLOCKS_BY_RANGE request. A None response signals the end of the batch.
BlocksByRange(Option<Box<SignedBeaconBlock<TSpec>>>),
BlocksByRange(Option<Arc<SignedBeaconBlock<TSpec>>>),
/// A response to a get BLOCKS_BY_ROOT request.
BlocksByRoot(Option<Box<SignedBeaconBlock<TSpec>>>),
BlocksByRoot(Option<Arc<SignedBeaconBlock<TSpec>>>),
}
impl<TSpec: EthSpec> std::convert::From<Response<TSpec>> for RPCCodedResponse<TSpec> {

View File

@ -532,10 +532,10 @@ fn handle_v1_response<T: EthSpec>(
Protocol::Goodbye => Err(RPCError::InvalidData(
"Goodbye RPC message has no valid response".to_string(),
)),
Protocol::BlocksByRange => Ok(Some(RPCResponse::BlocksByRange(Box::new(
Protocol::BlocksByRange => Ok(Some(RPCResponse::BlocksByRange(Arc::new(
SignedBeaconBlock::Base(SignedBeaconBlockBase::from_ssz_bytes(decoded_buffer)?),
)))),
Protocol::BlocksByRoot => Ok(Some(RPCResponse::BlocksByRoot(Box::new(
Protocol::BlocksByRoot => Ok(Some(RPCResponse::BlocksByRoot(Arc::new(
SignedBeaconBlock::Base(SignedBeaconBlockBase::from_ssz_bytes(decoded_buffer)?),
)))),
Protocol::Ping => Ok(Some(RPCResponse::Pong(Ping {
@ -572,31 +572,31 @@ fn handle_v2_response<T: EthSpec>(
})?;
match protocol {
Protocol::BlocksByRange => match fork_name {
ForkName::Altair => Ok(Some(RPCResponse::BlocksByRange(Box::new(
ForkName::Altair => Ok(Some(RPCResponse::BlocksByRange(Arc::new(
SignedBeaconBlock::Altair(SignedBeaconBlockAltair::from_ssz_bytes(
decoded_buffer,
)?),
)))),
ForkName::Base => Ok(Some(RPCResponse::BlocksByRange(Box::new(
ForkName::Base => Ok(Some(RPCResponse::BlocksByRange(Arc::new(
SignedBeaconBlock::Base(SignedBeaconBlockBase::from_ssz_bytes(decoded_buffer)?),
)))),
ForkName::Merge => Ok(Some(RPCResponse::BlocksByRange(Box::new(
ForkName::Merge => Ok(Some(RPCResponse::BlocksByRange(Arc::new(
SignedBeaconBlock::Merge(SignedBeaconBlockMerge::from_ssz_bytes(
decoded_buffer,
)?),
)))),
},
Protocol::BlocksByRoot => match fork_name {
ForkName::Altair => Ok(Some(RPCResponse::BlocksByRoot(Box::new(
ForkName::Altair => Ok(Some(RPCResponse::BlocksByRoot(Arc::new(
SignedBeaconBlock::Altair(SignedBeaconBlockAltair::from_ssz_bytes(
decoded_buffer,
)?),
)))),
ForkName::Base => Ok(Some(RPCResponse::BlocksByRoot(Box::new(
ForkName::Base => Ok(Some(RPCResponse::BlocksByRoot(Arc::new(
SignedBeaconBlock::Base(SignedBeaconBlockBase::from_ssz_bytes(decoded_buffer)?),
)))),
ForkName::Merge => Ok(Some(RPCResponse::BlocksByRoot(Box::new(
ForkName::Merge => Ok(Some(RPCResponse::BlocksByRoot(Arc::new(
SignedBeaconBlock::Merge(SignedBeaconBlockMerge::from_ssz_bytes(
decoded_buffer,
)?),
@ -898,10 +898,10 @@ mod tests {
encode_then_decode_response(
Protocol::BlocksByRange,
Version::V1,
RPCCodedResponse::Success(RPCResponse::BlocksByRange(Box::new(empty_base_block()))),
RPCCodedResponse::Success(RPCResponse::BlocksByRange(Arc::new(empty_base_block()))),
ForkName::Base,
),
Ok(Some(RPCResponse::BlocksByRange(Box::new(
Ok(Some(RPCResponse::BlocksByRange(Arc::new(
empty_base_block()
))))
);
@ -911,7 +911,7 @@ mod tests {
encode_then_decode_response(
Protocol::BlocksByRange,
Version::V1,
RPCCodedResponse::Success(RPCResponse::BlocksByRange(Box::new(altair_block()))),
RPCCodedResponse::Success(RPCResponse::BlocksByRange(Arc::new(altair_block()))),
ForkName::Altair,
)
.unwrap_err(),
@ -924,11 +924,11 @@ mod tests {
encode_then_decode_response(
Protocol::BlocksByRoot,
Version::V1,
RPCCodedResponse::Success(RPCResponse::BlocksByRoot(Box::new(empty_base_block()))),
RPCCodedResponse::Success(RPCResponse::BlocksByRoot(Arc::new(empty_base_block()))),
ForkName::Base,
),
Ok(Some(RPCResponse::BlocksByRoot(
Box::new(empty_base_block())
Arc::new(empty_base_block())
)))
);
@ -937,7 +937,7 @@ mod tests {
encode_then_decode_response(
Protocol::BlocksByRoot,
Version::V1,
RPCCodedResponse::Success(RPCResponse::BlocksByRoot(Box::new(altair_block()))),
RPCCodedResponse::Success(RPCResponse::BlocksByRoot(Arc::new(altair_block()))),
ForkName::Altair,
)
.unwrap_err(),
@ -1013,10 +1013,10 @@ mod tests {
encode_then_decode_response(
Protocol::BlocksByRange,
Version::V2,
RPCCodedResponse::Success(RPCResponse::BlocksByRange(Box::new(empty_base_block()))),
RPCCodedResponse::Success(RPCResponse::BlocksByRange(Arc::new(empty_base_block()))),
ForkName::Base,
),
Ok(Some(RPCResponse::BlocksByRange(Box::new(
Ok(Some(RPCResponse::BlocksByRange(Arc::new(
empty_base_block()
))))
);
@ -1028,10 +1028,10 @@ mod tests {
encode_then_decode_response(
Protocol::BlocksByRange,
Version::V2,
RPCCodedResponse::Success(RPCResponse::BlocksByRange(Box::new(empty_base_block()))),
RPCCodedResponse::Success(RPCResponse::BlocksByRange(Arc::new(empty_base_block()))),
ForkName::Altair,
),
Ok(Some(RPCResponse::BlocksByRange(Box::new(
Ok(Some(RPCResponse::BlocksByRange(Arc::new(
empty_base_block()
))))
);
@ -1040,10 +1040,10 @@ mod tests {
encode_then_decode_response(
Protocol::BlocksByRange,
Version::V2,
RPCCodedResponse::Success(RPCResponse::BlocksByRange(Box::new(altair_block()))),
RPCCodedResponse::Success(RPCResponse::BlocksByRange(Arc::new(altair_block()))),
ForkName::Altair,
),
Ok(Some(RPCResponse::BlocksByRange(Box::new(altair_block()))))
Ok(Some(RPCResponse::BlocksByRange(Arc::new(altair_block()))))
);
let merge_block_small = merge_block_small(&fork_context(ForkName::Merge));
@ -1053,12 +1053,12 @@ mod tests {
encode_then_decode_response(
Protocol::BlocksByRange,
Version::V2,
RPCCodedResponse::Success(RPCResponse::BlocksByRange(Box::new(
RPCCodedResponse::Success(RPCResponse::BlocksByRange(Arc::new(
merge_block_small.clone()
))),
ForkName::Merge,
),
Ok(Some(RPCResponse::BlocksByRange(Box::new(
Ok(Some(RPCResponse::BlocksByRange(Arc::new(
merge_block_small.clone()
))))
);
@ -1085,11 +1085,11 @@ mod tests {
encode_then_decode_response(
Protocol::BlocksByRoot,
Version::V2,
RPCCodedResponse::Success(RPCResponse::BlocksByRoot(Box::new(empty_base_block()))),
RPCCodedResponse::Success(RPCResponse::BlocksByRoot(Arc::new(empty_base_block()))),
ForkName::Base,
),
Ok(Some(RPCResponse::BlocksByRoot(
Box::new(empty_base_block())
Arc::new(empty_base_block())
))),
);
@ -1100,11 +1100,11 @@ mod tests {
encode_then_decode_response(
Protocol::BlocksByRoot,
Version::V2,
RPCCodedResponse::Success(RPCResponse::BlocksByRoot(Box::new(empty_base_block()))),
RPCCodedResponse::Success(RPCResponse::BlocksByRoot(Arc::new(empty_base_block()))),
ForkName::Altair,
),
Ok(Some(RPCResponse::BlocksByRoot(
Box::new(empty_base_block())
Arc::new(empty_base_block())
)))
);
@ -1112,22 +1112,22 @@ mod tests {
encode_then_decode_response(
Protocol::BlocksByRoot,
Version::V2,
RPCCodedResponse::Success(RPCResponse::BlocksByRoot(Box::new(altair_block()))),
RPCCodedResponse::Success(RPCResponse::BlocksByRoot(Arc::new(altair_block()))),
ForkName::Altair,
),
Ok(Some(RPCResponse::BlocksByRoot(Box::new(altair_block()))))
Ok(Some(RPCResponse::BlocksByRoot(Arc::new(altair_block()))))
);
assert_eq!(
encode_then_decode_response(
Protocol::BlocksByRoot,
Version::V2,
RPCCodedResponse::Success(RPCResponse::BlocksByRoot(Box::new(
RPCCodedResponse::Success(RPCResponse::BlocksByRoot(Arc::new(
merge_block_small.clone()
))),
ForkName::Merge,
),
Ok(Some(RPCResponse::BlocksByRoot(Box::new(merge_block_small))))
Ok(Some(RPCResponse::BlocksByRoot(Arc::new(merge_block_small))))
);
let mut encoded =
@ -1179,7 +1179,7 @@ mod tests {
let mut encoded_bytes = encode_response(
Protocol::BlocksByRange,
Version::V2,
RPCCodedResponse::Success(RPCResponse::BlocksByRange(Box::new(empty_base_block()))),
RPCCodedResponse::Success(RPCResponse::BlocksByRange(Arc::new(empty_base_block()))),
ForkName::Base,
)
.unwrap();
@ -1200,7 +1200,7 @@ mod tests {
let mut encoded_bytes = encode_response(
Protocol::BlocksByRoot,
Version::V2,
RPCCodedResponse::Success(RPCResponse::BlocksByRoot(Box::new(empty_base_block()))),
RPCCodedResponse::Success(RPCResponse::BlocksByRoot(Arc::new(empty_base_block()))),
ForkName::Base,
)
.unwrap();
@ -1222,7 +1222,7 @@ mod tests {
let mut encoded_bytes = encode_response(
Protocol::BlocksByRange,
Version::V2,
RPCCodedResponse::Success(RPCResponse::BlocksByRange(Box::new(empty_base_block()))),
RPCCodedResponse::Success(RPCResponse::BlocksByRange(Arc::new(empty_base_block()))),
ForkName::Altair,
)
.unwrap();
@ -1247,7 +1247,7 @@ mod tests {
let mut encoded_bytes = encode_response(
Protocol::BlocksByRoot,
Version::V2,
RPCCodedResponse::Success(RPCResponse::BlocksByRoot(Box::new(altair_block()))),
RPCCodedResponse::Success(RPCResponse::BlocksByRoot(Arc::new(altair_block()))),
ForkName::Altair,
)
.unwrap();
@ -1292,7 +1292,7 @@ mod tests {
let mut encoded_bytes = encode_response(
Protocol::BlocksByRoot,
Version::V2,
RPCCodedResponse::Success(RPCResponse::BlocksByRoot(Box::new(empty_base_block()))),
RPCCodedResponse::Success(RPCResponse::BlocksByRoot(Arc::new(empty_base_block()))),
ForkName::Altair,
)
.unwrap();
@ -1316,7 +1316,7 @@ mod tests {
let mut encoded_bytes = encode_response(
Protocol::BlocksByRoot,
Version::V2,
RPCCodedResponse::Success(RPCResponse::BlocksByRoot(Box::new(empty_base_block()))),
RPCCodedResponse::Success(RPCResponse::BlocksByRoot(Arc::new(empty_base_block()))),
ForkName::Altair,
)
.unwrap();

View File

@ -9,6 +9,7 @@ use ssz_types::{
VariableList,
};
use std::ops::Deref;
use std::sync::Arc;
use strum::IntoStaticStr;
use superstruct::superstruct;
use types::{Epoch, EthSpec, Hash256, SignedBeaconBlock, Slot};
@ -237,10 +238,10 @@ pub enum RPCResponse<T: EthSpec> {
/// A response to a get BLOCKS_BY_RANGE request. A None response signifies the end of the
/// batch.
BlocksByRange(Box<SignedBeaconBlock<T>>),
BlocksByRange(Arc<SignedBeaconBlock<T>>),
/// A response to a get BLOCKS_BY_ROOT request.
BlocksByRoot(Box<SignedBeaconBlock<T>>),
BlocksByRoot(Arc<SignedBeaconBlock<T>>),
/// A PONG response to a PING request.
Pong(Ping),

View File

@ -7,6 +7,7 @@ use snap::raw::{decompress_len, Decoder, Encoder};
use ssz::{Decode, Encode};
use std::boxed::Box;
use std::io::{Error, ErrorKind};
use std::sync::Arc;
use types::{
Attestation, AttesterSlashing, EthSpec, ForkContext, ForkName, ProposerSlashing,
SignedAggregateAndProof, SignedBeaconBlock, SignedBeaconBlockAltair, SignedBeaconBlockBase,
@ -17,7 +18,7 @@ use types::{
#[derive(Debug, Clone, PartialEq)]
pub enum PubsubMessage<T: EthSpec> {
/// Gossipsub message providing notification of a new block.
BeaconBlock(Box<SignedBeaconBlock<T>>),
BeaconBlock(Arc<SignedBeaconBlock<T>>),
/// Gossipsub message providing notification of a Aggregate attestation and associated proof.
AggregateAndProofAttestation(Box<SignedAggregateAndProof<T>>),
/// Gossipsub message providing notification of a raw un-aggregated attestation with its shard id.
@ -173,7 +174,7 @@ impl<T: EthSpec> PubsubMessage<T> {
))
}
};
Ok(PubsubMessage::BeaconBlock(Box::new(beacon_block)))
Ok(PubsubMessage::BeaconBlock(Arc::new(beacon_block)))
}
GossipKind::VoluntaryExit => {
let voluntary_exit = SignedVoluntaryExit::from_ssz_bytes(data)

View File

@ -174,15 +174,15 @@ fn test_blocks_by_range_chunked_rpc() {
// BlocksByRange Response
let full_block = BeaconBlock::Base(BeaconBlockBase::<E>::full(&spec));
let signed_full_block = SignedBeaconBlock::from_block(full_block, Signature::empty());
let rpc_response_base = Response::BlocksByRange(Some(Box::new(signed_full_block)));
let rpc_response_base = Response::BlocksByRange(Some(Arc::new(signed_full_block)));
let full_block = BeaconBlock::Altair(BeaconBlockAltair::<E>::full(&spec));
let signed_full_block = SignedBeaconBlock::from_block(full_block, Signature::empty());
let rpc_response_altair = Response::BlocksByRange(Some(Box::new(signed_full_block)));
let rpc_response_altair = Response::BlocksByRange(Some(Arc::new(signed_full_block)));
let full_block = merge_block_small(&common::fork_context(ForkName::Merge));
let signed_full_block = SignedBeaconBlock::from_block(full_block, Signature::empty());
let rpc_response_merge_small = Response::BlocksByRange(Some(Box::new(signed_full_block)));
let rpc_response_merge_small = Response::BlocksByRange(Some(Arc::new(signed_full_block)));
// keep count of the number of messages received
let mut messages_received = 0;
@ -311,7 +311,7 @@ fn test_blocks_by_range_over_limit() {
// BlocksByRange Response
let full_block = merge_block_large(&common::fork_context(ForkName::Merge));
let signed_full_block = SignedBeaconBlock::from_block(full_block, Signature::empty());
let rpc_response_merge_large = Response::BlocksByRange(Some(Box::new(signed_full_block)));
let rpc_response_merge_large = Response::BlocksByRange(Some(Arc::new(signed_full_block)));
let request_id = messages_to_send as usize;
// build the sender future
@ -409,7 +409,7 @@ fn test_blocks_by_range_chunked_rpc_terminates_correctly() {
let spec = E::default_spec();
let empty_block = BeaconBlock::empty(&spec);
let empty_signed = SignedBeaconBlock::from_block(empty_block, Signature::empty());
let rpc_response = Response::BlocksByRange(Some(Box::new(empty_signed)));
let rpc_response = Response::BlocksByRange(Some(Arc::new(empty_signed)));
// keep count of the number of messages received
let mut messages_received: u64 = 0;
@ -540,7 +540,7 @@ fn test_blocks_by_range_single_empty_rpc() {
let spec = E::default_spec();
let empty_block = BeaconBlock::empty(&spec);
let empty_signed = SignedBeaconBlock::from_block(empty_block, Signature::empty());
let rpc_response = Response::BlocksByRange(Some(Box::new(empty_signed)));
let rpc_response = Response::BlocksByRange(Some(Arc::new(empty_signed)));
let messages_to_send = 1;
@ -660,15 +660,15 @@ fn test_blocks_by_root_chunked_rpc() {
// BlocksByRoot Response
let full_block = BeaconBlock::Base(BeaconBlockBase::<E>::full(&spec));
let signed_full_block = SignedBeaconBlock::from_block(full_block, Signature::empty());
let rpc_response_base = Response::BlocksByRoot(Some(Box::new(signed_full_block)));
let rpc_response_base = Response::BlocksByRoot(Some(Arc::new(signed_full_block)));
let full_block = BeaconBlock::Altair(BeaconBlockAltair::<E>::full(&spec));
let signed_full_block = SignedBeaconBlock::from_block(full_block, Signature::empty());
let rpc_response_altair = Response::BlocksByRoot(Some(Box::new(signed_full_block)));
let rpc_response_altair = Response::BlocksByRoot(Some(Arc::new(signed_full_block)));
let full_block = merge_block_small(&common::fork_context(ForkName::Merge));
let signed_full_block = SignedBeaconBlock::from_block(full_block, Signature::empty());
let rpc_response_merge_small = Response::BlocksByRoot(Some(Box::new(signed_full_block)));
let rpc_response_merge_small = Response::BlocksByRoot(Some(Arc::new(signed_full_block)));
// keep count of the number of messages received
let mut messages_received = 0;
@ -803,7 +803,7 @@ fn test_blocks_by_root_chunked_rpc_terminates_correctly() {
// BlocksByRoot Response
let full_block = BeaconBlock::Base(BeaconBlockBase::<E>::full(&spec));
let signed_full_block = SignedBeaconBlock::from_block(full_block, Signature::empty());
let rpc_response = Response::BlocksByRoot(Some(Box::new(signed_full_block)));
let rpc_response = Response::BlocksByRoot(Some(Arc::new(signed_full_block)));
// keep count of the number of messages received
let mut messages_received = 0;

View File

@ -52,6 +52,7 @@ use lighthouse_network::{
use logging::TimeLatch;
use slog::{crit, debug, error, trace, warn, Logger};
use std::collections::VecDeque;
use std::future::Future;
use std::pin::Pin;
use std::sync::{Arc, Weak};
use std::task::Context;
@ -386,7 +387,7 @@ impl<T: BeaconChainTypes> WorkEvent<T> {
message_id: MessageId,
peer_id: PeerId,
peer_client: Client,
block: Box<SignedBeaconBlock<T::EthSpec>>,
block: Arc<SignedBeaconBlock<T::EthSpec>>,
seen_timestamp: Duration,
) -> Self {
Self {
@ -490,7 +491,7 @@ impl<T: BeaconChainTypes> WorkEvent<T> {
/// Create a new `Work` event for some block, where the result from computation (if any) is
/// sent to the other side of `result_tx`.
pub fn rpc_beacon_block(
block: Box<SignedBeaconBlock<T::EthSpec>>,
block: Arc<SignedBeaconBlock<T::EthSpec>>,
seen_timestamp: Duration,
process_type: BlockProcessType,
) -> Self {
@ -507,7 +508,7 @@ impl<T: BeaconChainTypes> WorkEvent<T> {
/// Create a new work event to import `blocks` as a beacon chain segment.
pub fn chain_segment(
process_id: ChainSegmentProcessId,
blocks: Vec<SignedBeaconBlock<T::EthSpec>>,
blocks: Vec<Arc<SignedBeaconBlock<T::EthSpec>>>,
) -> Self {
Self {
drop_during_sync: false,
@ -654,7 +655,7 @@ pub enum Work<T: BeaconChainTypes> {
message_id: MessageId,
peer_id: PeerId,
peer_client: Client,
block: Box<SignedBeaconBlock<T::EthSpec>>,
block: Arc<SignedBeaconBlock<T::EthSpec>>,
seen_timestamp: Duration,
},
DelayedImportBlock {
@ -691,13 +692,13 @@ pub enum Work<T: BeaconChainTypes> {
seen_timestamp: Duration,
},
RpcBlock {
block: Box<SignedBeaconBlock<T::EthSpec>>,
block: Arc<SignedBeaconBlock<T::EthSpec>>,
seen_timestamp: Duration,
process_type: BlockProcessType,
},
ChainSegment {
process_id: ChainSegmentProcessId,
blocks: Vec<SignedBeaconBlock<T::EthSpec>>,
blocks: Vec<Arc<SignedBeaconBlock<T::EthSpec>>>,
},
Status {
peer_id: PeerId,
@ -1307,15 +1308,6 @@ impl<T: BeaconChainTypes> BeaconProcessor<T> {
let idle_tx = toolbox.idle_tx;
let work_reprocessing_tx = toolbox.work_reprocessing_tx;
// Wrap the `idle_tx` in a struct that will fire the idle message whenever it is dropped.
//
// This helps ensure that the worker is always freed in the case of an early exit or panic.
// As such, this instantiation should happen as early in the function as possible.
let send_idle_on_drop = SendOnDrop {
tx: idle_tx,
log: self.log.clone(),
};
let work_id = work.str_id();
let worker_timer =
metrics::start_timer_vec(&metrics::BEACON_PROCESSOR_WORKER_TIME, &[work_id]);
@ -1325,6 +1317,16 @@ impl<T: BeaconChainTypes> BeaconProcessor<T> {
&[work.str_id()],
);
// Wrap the `idle_tx` in a struct that will fire the idle message whenever it is dropped.
//
// This helps ensure that the worker is always freed in the case of an early exit or panic.
// As such, this instantiation should happen as early in the function as possible.
let send_idle_on_drop = SendOnDrop {
tx: idle_tx,
_worker_timer: worker_timer,
log: self.log.clone(),
};
let worker_id = self.current_workers;
self.current_workers = self.current_workers.saturating_add(1);
@ -1338,7 +1340,6 @@ impl<T: BeaconChainTypes> BeaconProcessor<T> {
return;
};
let log = self.log.clone();
let executor = self.executor.clone();
let worker = Worker {
@ -1357,252 +1358,308 @@ impl<T: BeaconChainTypes> BeaconProcessor<T> {
"worker" => worker_id,
);
let sub_executor = executor.clone();
executor.spawn_blocking(
move || {
let _worker_timer = worker_timer;
let task_spawner = TaskSpawner {
executor: executor.clone(),
send_idle_on_drop,
};
match work {
/*
* Individual unaggregated attestation verification.
*/
Work::GossipAttestation {
message_id,
peer_id,
attestation,
subnet_id,
should_import,
seen_timestamp,
} => worker.process_gossip_attestation(
message_id,
peer_id,
attestation,
subnet_id,
should_import,
Some(work_reprocessing_tx),
seen_timestamp,
),
/*
* Batched unaggregated attestation verification.
*/
Work::GossipAttestationBatch { packages } => worker
.process_gossip_attestation_batch(packages, Some(work_reprocessing_tx)),
/*
* Individual aggregated attestation verification.
*/
Work::GossipAggregate {
message_id,
peer_id,
aggregate,
seen_timestamp,
} => worker.process_gossip_aggregate(
message_id,
peer_id,
aggregate,
Some(work_reprocessing_tx),
seen_timestamp,
),
/*
* Batched aggregated attestation verification.
*/
Work::GossipAggregateBatch { packages } => {
worker.process_gossip_aggregate_batch(packages, Some(work_reprocessing_tx))
}
/*
* Verification for beacon blocks received on gossip.
*/
Work::GossipBlock {
let sub_executor = executor;
match work {
/*
* Individual unaggregated attestation verification.
*/
Work::GossipAttestation {
message_id,
peer_id,
attestation,
subnet_id,
should_import,
seen_timestamp,
} => task_spawner.spawn_blocking(move || {
worker.process_gossip_attestation(
message_id,
peer_id,
attestation,
subnet_id,
should_import,
Some(work_reprocessing_tx),
seen_timestamp,
)
}),
/*
* Batched unaggregated attestation verification.
*/
Work::GossipAttestationBatch { packages } => task_spawner.spawn_blocking(|| {
worker.process_gossip_attestation_batch(packages, Some(work_reprocessing_tx))
}),
/*
* Individual aggregated attestation verification.
*/
Work::GossipAggregate {
message_id,
peer_id,
aggregate,
seen_timestamp,
} => task_spawner.spawn_blocking(move || {
worker.process_gossip_aggregate(
message_id,
peer_id,
aggregate,
Some(work_reprocessing_tx),
seen_timestamp,
)
}),
/*
* Batched aggregated attestation verification.
*/
Work::GossipAggregateBatch { packages } => task_spawner.spawn_blocking(|| {
worker.process_gossip_aggregate_batch(packages, Some(work_reprocessing_tx))
}),
/*
* Verification for beacon blocks received on gossip.
*/
Work::GossipBlock {
message_id,
peer_id,
peer_client,
block,
seen_timestamp,
} => task_spawner.spawn_async(async move {
worker
.process_gossip_block(
message_id,
peer_id,
peer_client,
block,
seen_timestamp,
} => worker.process_gossip_block(
message_id,
peer_id,
peer_client,
*block,
work_reprocessing_tx.clone(),
work_reprocessing_tx,
duplicate_cache,
seen_timestamp,
),
/*
* Import for blocks that we received earlier than their intended slot.
*/
Work::DelayedImportBlock {
peer_id,
block,
seen_timestamp,
} => worker.process_gossip_verified_block(
peer_id,
*block,
work_reprocessing_tx,
seen_timestamp,
),
/*
* Voluntary exits received on gossip.
*/
Work::GossipVoluntaryExit {
message_id,
peer_id,
voluntary_exit,
} => worker.process_gossip_voluntary_exit(message_id, peer_id, *voluntary_exit),
/*
* Proposer slashings received on gossip.
*/
Work::GossipProposerSlashing {
message_id,
peer_id,
proposer_slashing,
} => worker.process_gossip_proposer_slashing(
message_id,
peer_id,
*proposer_slashing,
),
/*
* Attester slashings received on gossip.
*/
Work::GossipAttesterSlashing {
message_id,
peer_id,
attester_slashing,
} => worker.process_gossip_attester_slashing(
message_id,
peer_id,
*attester_slashing,
),
/*
* Sync committee message verification.
*/
Work::GossipSyncSignature {
message_id,
peer_id,
sync_signature,
subnet_id,
seen_timestamp,
} => worker.process_gossip_sync_committee_signature(
message_id,
peer_id,
*sync_signature,
subnet_id,
seen_timestamp,
),
/*
* Syn contribution verification.
*/
Work::GossipSyncContribution {
message_id,
peer_id,
sync_contribution,
seen_timestamp,
} => worker.process_sync_committee_contribution(
message_id,
peer_id,
*sync_contribution,
seen_timestamp,
),
/*
* Verification for beacon blocks received during syncing via RPC.
*/
Work::RpcBlock {
block,
seen_timestamp,
process_type,
} => {
worker.process_rpc_block(
*block,
seen_timestamp,
process_type,
work_reprocessing_tx.clone(),
duplicate_cache,
);
}
/*
* Verification for a chain segment (multiple blocks).
*/
Work::ChainSegment { process_id, blocks } => {
worker.process_chain_segment(process_id, blocks)
}
/*
* Processing of Status Messages.
*/
Work::Status { peer_id, message } => worker.process_status(peer_id, message),
/*
* Processing of range syncing requests from other peers.
*/
Work::BlocksByRangeRequest {
peer_id,
request_id,
request,
} => {
return worker.handle_blocks_by_range_request(
sub_executor,
send_idle_on_drop,
peer_id,
request_id,
request,
)
}
/*
* Processing of blocks by roots requests from other peers.
*/
Work::BlocksByRootsRequest {
peer_id,
request_id,
request,
} => {
return worker.handle_blocks_by_root_request(
sub_executor,
send_idle_on_drop,
peer_id,
request_id,
request,
)
}
Work::UnknownBlockAttestation {
message_id,
peer_id,
attestation,
subnet_id,
should_import,
seen_timestamp,
} => worker.process_gossip_attestation(
message_id,
peer_id,
attestation,
subnet_id,
should_import,
None, // Do not allow this attestation to be re-processed beyond this point.
seen_timestamp,
),
Work::UnknownBlockAggregate {
message_id,
peer_id,
aggregate,
seen_timestamp,
} => worker.process_gossip_aggregate(
message_id,
peer_id,
aggregate,
None,
seen_timestamp,
),
};
)
.await
}),
/*
* Import for blocks that we received earlier than their intended slot.
*/
Work::DelayedImportBlock {
peer_id,
block,
seen_timestamp,
} => task_spawner.spawn_async(worker.process_gossip_verified_block(
peer_id,
*block,
work_reprocessing_tx,
seen_timestamp,
)),
/*
* Voluntary exits received on gossip.
*/
Work::GossipVoluntaryExit {
message_id,
peer_id,
voluntary_exit,
} => task_spawner.spawn_blocking(move || {
worker.process_gossip_voluntary_exit(message_id, peer_id, *voluntary_exit)
}),
/*
* Proposer slashings received on gossip.
*/
Work::GossipProposerSlashing {
message_id,
peer_id,
proposer_slashing,
} => task_spawner.spawn_blocking(move || {
worker.process_gossip_proposer_slashing(message_id, peer_id, *proposer_slashing)
}),
/*
* Attester slashings received on gossip.
*/
Work::GossipAttesterSlashing {
message_id,
peer_id,
attester_slashing,
} => task_spawner.spawn_blocking(move || {
worker.process_gossip_attester_slashing(message_id, peer_id, *attester_slashing)
}),
/*
* Sync committee message verification.
*/
Work::GossipSyncSignature {
message_id,
peer_id,
sync_signature,
subnet_id,
seen_timestamp,
} => task_spawner.spawn_blocking(move || {
worker.process_gossip_sync_committee_signature(
message_id,
peer_id,
*sync_signature,
subnet_id,
seen_timestamp,
)
}),
/*
* Syn contribution verification.
*/
Work::GossipSyncContribution {
message_id,
peer_id,
sync_contribution,
seen_timestamp,
} => task_spawner.spawn_blocking(move || {
worker.process_sync_committee_contribution(
message_id,
peer_id,
*sync_contribution,
seen_timestamp,
)
}),
/*
* Verification for beacon blocks received during syncing via RPC.
*/
Work::RpcBlock {
block,
seen_timestamp,
process_type,
} => task_spawner.spawn_async(worker.process_rpc_block(
block,
seen_timestamp,
process_type,
work_reprocessing_tx,
duplicate_cache,
)),
/*
* Verification for a chain segment (multiple blocks).
*/
Work::ChainSegment { process_id, blocks } => task_spawner
.spawn_async(async move { worker.process_chain_segment(process_id, blocks).await }),
/*
* Processing of Status Messages.
*/
Work::Status { peer_id, message } => {
task_spawner.spawn_blocking(move || worker.process_status(peer_id, message))
}
/*
* Processing of range syncing requests from other peers.
*/
Work::BlocksByRangeRequest {
peer_id,
request_id,
request,
} => task_spawner.spawn_blocking_with_manual_send_idle(move |send_idle_on_drop| {
worker.handle_blocks_by_range_request(
sub_executor,
send_idle_on_drop,
peer_id,
request_id,
request,
)
}),
/*
* Processing of blocks by roots requests from other peers.
*/
Work::BlocksByRootsRequest {
peer_id,
request_id,
request,
} => task_spawner.spawn_blocking_with_manual_send_idle(move |send_idle_on_drop| {
worker.handle_blocks_by_root_request(
sub_executor,
send_idle_on_drop,
peer_id,
request_id,
request,
)
}),
Work::UnknownBlockAttestation {
message_id,
peer_id,
attestation,
subnet_id,
should_import,
seen_timestamp,
} => task_spawner.spawn_blocking(move || {
worker.process_gossip_attestation(
message_id,
peer_id,
attestation,
subnet_id,
should_import,
None, // Do not allow this attestation to be re-processed beyond this point.
seen_timestamp,
)
}),
Work::UnknownBlockAggregate {
message_id,
peer_id,
aggregate,
seen_timestamp,
} => task_spawner.spawn_blocking(move || {
worker.process_gossip_aggregate(
message_id,
peer_id,
aggregate,
None,
seen_timestamp,
)
}),
};
}
}
trace!(
log,
"Beacon processor worker done";
"work" => work_id,
"worker" => worker_id,
);
/// Spawns tasks that are either:
///
/// - Blocking (i.e. intensive methods that shouldn't run on the core `tokio` executor)
/// - Async (i.e. `async` methods)
///
/// Takes a `SendOnDrop` and ensures it is dropped after the task completes. This frees the beacon
/// processor worker so a new task can be started.
struct TaskSpawner {
executor: TaskExecutor,
send_idle_on_drop: SendOnDrop,
}
// This explicit `drop` is used to remind the programmer that this variable must
// not be dropped until the worker is complete. Dropping it early will cause the
// worker to be marked as "free" and cause an over-spawning of workers.
drop(send_idle_on_drop);
impl TaskSpawner {
/// Spawn an async task, dropping the `SendOnDrop` after the task has completed.
fn spawn_async(self, task: impl Future<Output = ()> + Send + 'static) {
self.executor.spawn(
async {
task.await;
drop(self.send_idle_on_drop)
},
WORKER_TASK_NAME,
);
)
}
/// Spawn a blocking task, dropping the `SendOnDrop` after the task has completed.
fn spawn_blocking<F>(self, task: F)
where
F: FnOnce() + Send + 'static,
{
self.executor.spawn_blocking(
|| {
task();
drop(self.send_idle_on_drop)
},
WORKER_TASK_NAME,
)
}
/// Spawn a blocking task, passing the `SendOnDrop` into the task.
///
/// ## Notes
///
/// Users must ensure the `SendOnDrop` is dropped at the appropriate time!
pub fn spawn_blocking_with_manual_send_idle<F>(self, task: F)
where
F: FnOnce(SendOnDrop) + Send + 'static,
{
self.executor.spawn_blocking(
|| {
task(self.send_idle_on_drop);
},
WORKER_TASK_NAME,
)
}
}
@ -1618,6 +1675,8 @@ impl<T: BeaconChainTypes> BeaconProcessor<T> {
/// https://doc.rust-lang.org/std/ops/trait.Drop.html#panics
pub struct SendOnDrop {
tx: mpsc::Sender<()>,
// The field is unused, but it's here to ensure the timer is dropped once the task has finished.
_worker_timer: Option<metrics::HistogramTimer>,
log: Logger,
}

View File

@ -8,7 +8,6 @@ use beacon_chain::test_utils::{
AttestationStrategy, BeaconChainHarness, BlockStrategy, EphemeralHarnessType,
};
use beacon_chain::{BeaconChain, MAXIMUM_GOSSIP_CLOCK_DISPARITY};
use environment::{null_logger, Environment, EnvironmentBuilder};
use lighthouse_network::{
discv5::enr::{CombinedKey, EnrBuilder},
rpc::methods::{MetaData, MetaDataV2},
@ -20,7 +19,6 @@ use std::cmp;
use std::iter::Iterator;
use std::sync::Arc;
use std::time::Duration;
use tokio::runtime::Handle;
use tokio::sync::mpsc;
use types::{
Attestation, AttesterSlashing, EthSpec, MainnetEthSpec, ProposerSlashing, SignedBeaconBlock,
@ -45,7 +43,7 @@ const STANDARD_TIMEOUT: Duration = Duration::from_secs(10);
/// Provides utilities for testing the `BeaconProcessor`.
struct TestRig {
chain: Arc<BeaconChain<T>>,
next_block: SignedBeaconBlock<E>,
next_block: Arc<SignedBeaconBlock<E>>,
attestations: Vec<(Attestation<E>, SubnetId)>,
next_block_attestations: Vec<(Attestation<E>, SubnetId)>,
next_block_aggregate_attestations: Vec<SignedAggregateAndProof<E>>,
@ -56,7 +54,7 @@ struct TestRig {
work_journal_rx: mpsc::Receiver<&'static str>,
_network_rx: mpsc::UnboundedReceiver<NetworkMessage<E>>,
_sync_rx: mpsc::UnboundedReceiver<SyncMessage<E>>,
environment: Option<Environment<E>>,
_harness: BeaconChainHarness<T>,
}
/// This custom drop implementation ensures that we shut down the tokio runtime gracefully. Without
@ -65,12 +63,11 @@ impl Drop for TestRig {
fn drop(&mut self) {
// Causes the beacon processor to shutdown.
self.beacon_processor_tx = mpsc::channel(MAX_WORK_EVENT_QUEUE_LEN).0;
self.environment.take().unwrap().shutdown_on_idle();
}
}
impl TestRig {
pub fn new(chain_length: u64) -> Self {
pub async fn new(chain_length: u64) -> Self {
// This allows for testing voluntary exits without building out a massive chain.
let mut spec = E::default_spec();
spec.shard_committee_period = 2;
@ -84,16 +81,18 @@ impl TestRig {
harness.advance_slot();
for _ in 0..chain_length {
harness.extend_chain(
1,
BlockStrategy::OnCanonicalHead,
AttestationStrategy::AllValidators,
);
harness
.extend_chain(
1,
BlockStrategy::OnCanonicalHead,
AttestationStrategy::AllValidators,
)
.await;
harness.advance_slot();
}
let head = harness.chain.head().unwrap();
let head = harness.chain.head_snapshot();
assert_eq!(
harness.chain.slot().unwrap(),
@ -101,8 +100,9 @@ impl TestRig {
"precondition: current slot is one after head"
);
let (next_block, next_state) =
harness.make_block(head.beacon_state.clone(), harness.chain.slot().unwrap());
let (next_block, next_state) = harness
.make_block(head.beacon_state.clone(), harness.chain.slot().unwrap())
.await;
let head_state_root = head.beacon_state_root();
let attestations = harness
@ -155,11 +155,11 @@ impl TestRig {
let proposer_slashing = harness.make_proposer_slashing(2);
let voluntary_exit = harness.make_voluntary_exit(3, harness.chain.epoch().unwrap());
let chain = harness.chain;
let chain = harness.chain.clone();
let (network_tx, _network_rx) = mpsc::unbounded_channel();
let log = null_logger().unwrap();
let log = harness.logger().clone();
let (beacon_processor_tx, beacon_processor_rx) = mpsc::channel(MAX_WORK_EVENT_QUEUE_LEN);
let (sync_tx, _sync_rx) = mpsc::unbounded_channel();
@ -181,15 +181,7 @@ impl TestRig {
&log,
));
let mut environment = EnvironmentBuilder::mainnet()
.null_logger()
.unwrap()
.multi_threaded_tokio_runtime()
.unwrap()
.build()
.unwrap();
let executor = environment.core_context().executor;
let executor = harness.runtime.task_executor.clone();
let (work_journal_tx, work_journal_rx) = mpsc::channel(16_364);
@ -208,7 +200,7 @@ impl TestRig {
Self {
chain,
next_block,
next_block: Arc::new(next_block),
attestations,
next_block_attestations,
next_block_aggregate_attestations,
@ -219,12 +211,16 @@ impl TestRig {
work_journal_rx,
_network_rx,
_sync_rx,
environment: Some(environment),
_harness: harness,
}
}
pub async fn recompute_head(&self) {
self.chain.recompute_head_at_current_slot().await.unwrap()
}
pub fn head_root(&self) -> Hash256 {
self.chain.head().unwrap().beacon_block_root
self.chain.head_snapshot().beacon_block_root
}
pub fn enqueue_gossip_block(&self) {
@ -233,7 +229,7 @@ impl TestRig {
junk_message_id(),
junk_peer_id(),
Client::default(),
Box::new(self.next_block.clone()),
self.next_block.clone(),
Duration::from_secs(0),
))
.unwrap();
@ -241,7 +237,7 @@ impl TestRig {
pub fn enqueue_rpc_block(&self) {
let event = WorkEvent::rpc_beacon_block(
Box::new(self.next_block.clone()),
self.next_block.clone(),
std::time::Duration::default(),
BlockProcessType::ParentLookup {
chain_hash: Hash256::random(),
@ -324,28 +320,16 @@ impl TestRig {
.unwrap();
}
fn handle(&mut self) -> Handle {
self.environment
.as_mut()
.unwrap()
.core_context()
.executor
.handle()
.unwrap()
}
/// Assert that the `BeaconProcessor` doesn't produce any events in the given `duration`.
pub fn assert_no_events_for(&mut self, duration: Duration) {
self.handle().block_on(async {
tokio::select! {
_ = tokio::time::sleep(duration) => (),
event = self.work_journal_rx.recv() => panic!(
"received {:?} within {:?} when expecting no events",
event,
duration
),
}
})
pub async fn assert_no_events_for(&mut self, duration: Duration) {
tokio::select! {
_ = tokio::time::sleep(duration) => (),
event = self.work_journal_rx.recv() => panic!(
"received {:?} within {:?} when expecting no events",
event,
duration
),
}
}
/// Checks that the `BeaconProcessor` event journal contains the `expected` events in the given
@ -354,57 +338,54 @@ impl TestRig {
///
/// Given the described logic, `expected` must not contain `WORKER_FREED` or `NOTHING_TO_DO`
/// events.
pub fn assert_event_journal_contains_ordered(&mut self, expected: &[&str]) {
pub async fn assert_event_journal_contains_ordered(&mut self, expected: &[&str]) {
assert!(expected
.iter()
.all(|ev| ev != &WORKER_FREED && ev != &NOTHING_TO_DO));
let (events, worker_freed_remaining) = self.handle().block_on(async {
let mut events = Vec::with_capacity(expected.len());
let mut worker_freed_remaining = expected.len();
let mut events = Vec::with_capacity(expected.len());
let mut worker_freed_remaining = expected.len();
let drain_future = async {
loop {
match self.work_journal_rx.recv().await {
Some(event) if event == WORKER_FREED => {
worker_freed_remaining -= 1;
if worker_freed_remaining == 0 {
// Break when all expected events are finished.
break;
}
let drain_future = async {
loop {
match self.work_journal_rx.recv().await {
Some(event) if event == WORKER_FREED => {
worker_freed_remaining -= 1;
if worker_freed_remaining == 0 {
// Break when all expected events are finished.
break;
}
Some(event) if event == NOTHING_TO_DO => {
// Ignore these.
}
Some(event) => {
events.push(event);
}
None => break,
}
Some(event) if event == NOTHING_TO_DO => {
// Ignore these.
}
Some(event) => {
events.push(event);
}
None => break,
}
};
// Drain the expected number of events from the channel, or time out and give up.
tokio::select! {
_ = tokio::time::sleep(STANDARD_TIMEOUT) => panic!(
"Timeout ({:?}) expired waiting for events. Expected {:?} but got {:?} waiting for {} `WORKER_FREED` events.",
STANDARD_TIMEOUT,
expected,
events,
worker_freed_remaining,
),
_ = drain_future => {},
}
};
(events, worker_freed_remaining)
});
// Drain the expected number of events from the channel, or time out and give up.
tokio::select! {
_ = tokio::time::sleep(STANDARD_TIMEOUT) => panic!(
"Timeout ({:?}) expired waiting for events. Expected {:?} but got {:?} waiting for {} `WORKER_FREED` events.",
STANDARD_TIMEOUT,
expected,
events,
worker_freed_remaining,
),
_ = drain_future => {},
}
assert_eq!(events, expected);
assert_eq!(worker_freed_remaining, 0);
}
pub fn assert_event_journal(&mut self, expected: &[&str]) {
self.assert_event_journal_with_timeout(expected, STANDARD_TIMEOUT);
pub async fn assert_event_journal(&mut self, expected: &[&str]) {
self.assert_event_journal_with_timeout(expected, STANDARD_TIMEOUT)
.await
}
/// Assert that the `BeaconProcessor` event journal is as `expected`.
@ -413,34 +394,34 @@ impl TestRig {
///
/// We won't attempt to listen for any more than `expected.len()` events. As such, it makes sense
/// to use the `NOTHING_TO_DO` event to ensure that execution has completed.
pub fn assert_event_journal_with_timeout(&mut self, expected: &[&str], timeout: Duration) {
let events = self.handle().block_on(async {
let mut events = Vec::with_capacity(expected.len());
pub async fn assert_event_journal_with_timeout(
&mut self,
expected: &[&str],
timeout: Duration,
) {
let mut events = Vec::with_capacity(expected.len());
let drain_future = async {
while let Some(event) = self.work_journal_rx.recv().await {
events.push(event);
let drain_future = async {
while let Some(event) = self.work_journal_rx.recv().await {
events.push(event);
// Break as soon as we collect the desired number of events.
if events.len() >= expected.len() {
break;
}
// Break as soon as we collect the desired number of events.
if events.len() >= expected.len() {
break;
}
};
// Drain the expected number of events from the channel, or time out and give up.
tokio::select! {
_ = tokio::time::sleep(timeout) => panic!(
"Timeout ({:?}) expired waiting for events. Expected {:?} but got {:?}",
timeout,
expected,
events
),
_ = drain_future => {},
}
};
events
});
// Drain the expected number of events from the channel, or time out and give up.
tokio::select! {
_ = tokio::time::sleep(timeout) => panic!(
"Timeout ({:?}) expired waiting for events. Expected {:?} but got {:?}",
timeout,
expected,
events
),
_ = drain_future => {},
}
assert_eq!(events, expected);
}
@ -455,9 +436,9 @@ fn junk_message_id() -> MessageId {
}
/// Blocks that arrive early should be queued for later processing.
#[test]
fn import_gossip_block_acceptably_early() {
let mut rig = TestRig::new(SMALL_CHAIN);
#[tokio::test]
async fn import_gossip_block_acceptably_early() {
let mut rig = TestRig::new(SMALL_CHAIN).await;
let slot_start = rig
.chain
@ -477,7 +458,8 @@ fn import_gossip_block_acceptably_early() {
rig.enqueue_gossip_block();
rig.assert_event_journal(&[GOSSIP_BLOCK, WORKER_FREED, NOTHING_TO_DO]);
rig.assert_event_journal(&[GOSSIP_BLOCK, WORKER_FREED, NOTHING_TO_DO])
.await;
// Note: this section of the code is a bit race-y. We're assuming that we can set the slot clock
// and check the head in the time between the block arrived early and when its due for
@ -492,7 +474,8 @@ fn import_gossip_block_acceptably_early() {
"block not yet imported"
);
rig.assert_event_journal(&[DELAYED_IMPORT_BLOCK, WORKER_FREED, NOTHING_TO_DO]);
rig.assert_event_journal(&[DELAYED_IMPORT_BLOCK, WORKER_FREED, NOTHING_TO_DO])
.await;
assert_eq!(
rig.head_root(),
@ -502,9 +485,9 @@ fn import_gossip_block_acceptably_early() {
}
/// Blocks that are *too* early shouldn't get into the delay queue.
#[test]
fn import_gossip_block_unacceptably_early() {
let mut rig = TestRig::new(SMALL_CHAIN);
#[tokio::test]
async fn import_gossip_block_unacceptably_early() {
let mut rig = TestRig::new(SMALL_CHAIN).await;
let slot_start = rig
.chain
@ -524,11 +507,12 @@ fn import_gossip_block_unacceptably_early() {
rig.enqueue_gossip_block();
rig.assert_event_journal(&[GOSSIP_BLOCK, WORKER_FREED, NOTHING_TO_DO]);
rig.assert_event_journal(&[GOSSIP_BLOCK, WORKER_FREED, NOTHING_TO_DO])
.await;
// Waiting for 5 seconds is a bit arbitrary, however it *should* be long enough to ensure the
// block isn't imported.
rig.assert_no_events_for(Duration::from_secs(5));
rig.assert_no_events_for(Duration::from_secs(5)).await;
assert!(
rig.head_root() != rig.next_block.canonical_root(),
@ -537,9 +521,9 @@ fn import_gossip_block_unacceptably_early() {
}
/// Blocks that arrive on-time should be processed normally.
#[test]
fn import_gossip_block_at_current_slot() {
let mut rig = TestRig::new(SMALL_CHAIN);
#[tokio::test]
async fn import_gossip_block_at_current_slot() {
let mut rig = TestRig::new(SMALL_CHAIN).await;
assert_eq!(
rig.chain.slot().unwrap(),
@ -549,7 +533,8 @@ fn import_gossip_block_at_current_slot() {
rig.enqueue_gossip_block();
rig.assert_event_journal(&[GOSSIP_BLOCK, WORKER_FREED, NOTHING_TO_DO]);
rig.assert_event_journal(&[GOSSIP_BLOCK, WORKER_FREED, NOTHING_TO_DO])
.await;
assert_eq!(
rig.head_root(),
@ -559,15 +544,16 @@ fn import_gossip_block_at_current_slot() {
}
/// Ensure a valid attestation can be imported.
#[test]
fn import_gossip_attestation() {
let mut rig = TestRig::new(SMALL_CHAIN);
#[tokio::test]
async fn import_gossip_attestation() {
let mut rig = TestRig::new(SMALL_CHAIN).await;
let initial_attns = rig.chain.naive_aggregation_pool.read().num_items();
rig.enqueue_unaggregated_attestation();
rig.assert_event_journal(&[GOSSIP_ATTESTATION, WORKER_FREED, NOTHING_TO_DO]);
rig.assert_event_journal(&[GOSSIP_ATTESTATION, WORKER_FREED, NOTHING_TO_DO])
.await;
assert_eq!(
rig.chain.naive_aggregation_pool.read().num_items(),
@ -583,8 +569,8 @@ enum BlockImportMethod {
/// Ensure that attestations that reference an unknown block get properly re-queued and
/// re-processed upon importing the block.
fn attestation_to_unknown_block_processed(import_method: BlockImportMethod) {
let mut rig = TestRig::new(SMALL_CHAIN);
async fn attestation_to_unknown_block_processed(import_method: BlockImportMethod) {
let mut rig = TestRig::new(SMALL_CHAIN).await;
// Send the attestation but not the block, and check that it was not imported.
@ -592,7 +578,8 @@ fn attestation_to_unknown_block_processed(import_method: BlockImportMethod) {
rig.enqueue_next_block_unaggregated_attestation();
rig.assert_event_journal(&[GOSSIP_ATTESTATION, WORKER_FREED, NOTHING_TO_DO]);
rig.assert_event_journal(&[GOSSIP_ATTESTATION, WORKER_FREED, NOTHING_TO_DO])
.await;
assert_eq!(
rig.chain.naive_aggregation_pool.read().num_items(),
@ -613,11 +600,12 @@ fn attestation_to_unknown_block_processed(import_method: BlockImportMethod) {
}
};
rig.assert_event_journal_contains_ordered(&[block_event, UNKNOWN_BLOCK_ATTESTATION]);
rig.assert_event_journal_contains_ordered(&[block_event, UNKNOWN_BLOCK_ATTESTATION])
.await;
// Run fork choice, since it isn't run when processing an RPC block. At runtime it is the
// responsibility of the sync manager to do this.
rig.chain.fork_choice().unwrap();
rig.recompute_head().await;
assert_eq!(
rig.head_root(),
@ -632,20 +620,20 @@ fn attestation_to_unknown_block_processed(import_method: BlockImportMethod) {
);
}
#[test]
fn attestation_to_unknown_block_processed_after_gossip_block() {
attestation_to_unknown_block_processed(BlockImportMethod::Gossip)
#[tokio::test]
async fn attestation_to_unknown_block_processed_after_gossip_block() {
attestation_to_unknown_block_processed(BlockImportMethod::Gossip).await
}
#[test]
fn attestation_to_unknown_block_processed_after_rpc_block() {
attestation_to_unknown_block_processed(BlockImportMethod::Rpc)
#[tokio::test]
async fn attestation_to_unknown_block_processed_after_rpc_block() {
attestation_to_unknown_block_processed(BlockImportMethod::Rpc).await
}
/// Ensure that attestations that reference an unknown block get properly re-queued and
/// re-processed upon importing the block.
fn aggregate_attestation_to_unknown_block(import_method: BlockImportMethod) {
let mut rig = TestRig::new(SMALL_CHAIN);
async fn aggregate_attestation_to_unknown_block(import_method: BlockImportMethod) {
let mut rig = TestRig::new(SMALL_CHAIN).await;
// Empty the op pool.
rig.chain
@ -659,7 +647,8 @@ fn aggregate_attestation_to_unknown_block(import_method: BlockImportMethod) {
rig.enqueue_next_block_aggregated_attestation();
rig.assert_event_journal(&[GOSSIP_AGGREGATE, WORKER_FREED, NOTHING_TO_DO]);
rig.assert_event_journal(&[GOSSIP_AGGREGATE, WORKER_FREED, NOTHING_TO_DO])
.await;
assert_eq!(
rig.chain.op_pool.num_attestations(),
@ -680,11 +669,12 @@ fn aggregate_attestation_to_unknown_block(import_method: BlockImportMethod) {
}
};
rig.assert_event_journal_contains_ordered(&[block_event, UNKNOWN_BLOCK_AGGREGATE]);
rig.assert_event_journal_contains_ordered(&[block_event, UNKNOWN_BLOCK_AGGREGATE])
.await;
// Run fork choice, since it isn't run when processing an RPC block. At runtime it is the
// responsibility of the sync manager to do this.
rig.chain.fork_choice().unwrap();
rig.recompute_head().await;
assert_eq!(
rig.head_root(),
@ -699,21 +689,21 @@ fn aggregate_attestation_to_unknown_block(import_method: BlockImportMethod) {
);
}
#[test]
fn aggregate_attestation_to_unknown_block_processed_after_gossip_block() {
aggregate_attestation_to_unknown_block(BlockImportMethod::Gossip)
#[tokio::test]
async fn aggregate_attestation_to_unknown_block_processed_after_gossip_block() {
aggregate_attestation_to_unknown_block(BlockImportMethod::Gossip).await
}
#[test]
fn aggregate_attestation_to_unknown_block_processed_after_rpc_block() {
aggregate_attestation_to_unknown_block(BlockImportMethod::Rpc)
#[tokio::test]
async fn aggregate_attestation_to_unknown_block_processed_after_rpc_block() {
aggregate_attestation_to_unknown_block(BlockImportMethod::Rpc).await
}
/// Ensure that attestations that reference an unknown block get properly re-queued and re-processed
/// when the block is not seen.
#[test]
fn requeue_unknown_block_gossip_attestation_without_import() {
let mut rig = TestRig::new(SMALL_CHAIN);
#[tokio::test]
async fn requeue_unknown_block_gossip_attestation_without_import() {
let mut rig = TestRig::new(SMALL_CHAIN).await;
// Send the attestation but not the block, and check that it was not imported.
@ -721,7 +711,8 @@ fn requeue_unknown_block_gossip_attestation_without_import() {
rig.enqueue_next_block_unaggregated_attestation();
rig.assert_event_journal(&[GOSSIP_ATTESTATION, WORKER_FREED, NOTHING_TO_DO]);
rig.assert_event_journal(&[GOSSIP_ATTESTATION, WORKER_FREED, NOTHING_TO_DO])
.await;
assert_eq!(
rig.chain.naive_aggregation_pool.read().num_items(),
@ -734,7 +725,8 @@ fn requeue_unknown_block_gossip_attestation_without_import() {
rig.assert_event_journal_with_timeout(
&[UNKNOWN_BLOCK_ATTESTATION, WORKER_FREED, NOTHING_TO_DO],
Duration::from_secs(1) + QUEUED_ATTESTATION_DELAY,
);
)
.await;
assert_eq!(
rig.chain.naive_aggregation_pool.read().num_items(),
@ -745,9 +737,9 @@ fn requeue_unknown_block_gossip_attestation_without_import() {
/// Ensure that aggregate that reference an unknown block get properly re-queued and re-processed
/// when the block is not seen.
#[test]
fn requeue_unknown_block_gossip_aggregated_attestation_without_import() {
let mut rig = TestRig::new(SMALL_CHAIN);
#[tokio::test]
async fn requeue_unknown_block_gossip_aggregated_attestation_without_import() {
let mut rig = TestRig::new(SMALL_CHAIN).await;
// Send the attestation but not the block, and check that it was not imported.
@ -755,7 +747,8 @@ fn requeue_unknown_block_gossip_aggregated_attestation_without_import() {
rig.enqueue_next_block_aggregated_attestation();
rig.assert_event_journal(&[GOSSIP_AGGREGATE, WORKER_FREED, NOTHING_TO_DO]);
rig.assert_event_journal(&[GOSSIP_AGGREGATE, WORKER_FREED, NOTHING_TO_DO])
.await;
assert_eq!(
rig.chain.naive_aggregation_pool.read().num_items(),
@ -768,7 +761,8 @@ fn requeue_unknown_block_gossip_aggregated_attestation_without_import() {
rig.assert_event_journal_with_timeout(
&[UNKNOWN_BLOCK_AGGREGATE, WORKER_FREED, NOTHING_TO_DO],
Duration::from_secs(1) + QUEUED_ATTESTATION_DELAY,
);
)
.await;
assert_eq!(
rig.chain.op_pool.num_attestations(),
@ -778,10 +772,10 @@ fn requeue_unknown_block_gossip_aggregated_attestation_without_import() {
}
/// Ensure a bunch of valid operations can be imported.
#[test]
fn import_misc_gossip_ops() {
#[tokio::test]
async fn import_misc_gossip_ops() {
// Exits need the long chain so validators aren't too young to exit.
let mut rig = TestRig::new(LONG_CHAIN);
let mut rig = TestRig::new(LONG_CHAIN).await;
/*
* Attester slashing
@ -791,7 +785,8 @@ fn import_misc_gossip_ops() {
rig.enqueue_gossip_attester_slashing();
rig.assert_event_journal(&[GOSSIP_ATTESTER_SLASHING, WORKER_FREED, NOTHING_TO_DO]);
rig.assert_event_journal(&[GOSSIP_ATTESTER_SLASHING, WORKER_FREED, NOTHING_TO_DO])
.await;
assert_eq!(
rig.chain.op_pool.num_attester_slashings(),
@ -807,7 +802,8 @@ fn import_misc_gossip_ops() {
rig.enqueue_gossip_proposer_slashing();
rig.assert_event_journal(&[GOSSIP_PROPOSER_SLASHING, WORKER_FREED, NOTHING_TO_DO]);
rig.assert_event_journal(&[GOSSIP_PROPOSER_SLASHING, WORKER_FREED, NOTHING_TO_DO])
.await;
assert_eq!(
rig.chain.op_pool.num_proposer_slashings(),
@ -823,7 +819,8 @@ fn import_misc_gossip_ops() {
rig.enqueue_gossip_voluntary_exit();
rig.assert_event_journal(&[GOSSIP_VOLUNTARY_EXIT, WORKER_FREED, NOTHING_TO_DO]);
rig.assert_event_journal(&[GOSSIP_VOLUNTARY_EXIT, WORKER_FREED, NOTHING_TO_DO])
.await;
assert_eq!(
rig.chain.op_pool.num_voluntary_exits(),

View File

@ -13,6 +13,7 @@ use lighthouse_network::{Client, MessageAcceptance, MessageId, PeerAction, PeerI
use slog::{crit, debug, error, info, trace, warn};
use slot_clock::SlotClock;
use ssz::Encode;
use std::sync::Arc;
use std::time::{Duration, SystemTime, UNIX_EPOCH};
use store::hot_cold_store::HotColdDBError;
use tokio::sync::mpsc;
@ -636,24 +637,27 @@ impl<T: BeaconChainTypes> Worker<T> {
///
/// Raises a log if there are errors.
#[allow(clippy::too_many_arguments)]
pub fn process_gossip_block(
pub async fn process_gossip_block(
self,
message_id: MessageId,
peer_id: PeerId,
peer_client: Client,
block: SignedBeaconBlock<T::EthSpec>,
block: Arc<SignedBeaconBlock<T::EthSpec>>,
reprocess_tx: mpsc::Sender<ReprocessQueueMessage<T>>,
duplicate_cache: DuplicateCache,
seen_duration: Duration,
) {
if let Some(gossip_verified_block) = self.process_gossip_unverified_block(
message_id,
peer_id,
peer_client,
block,
reprocess_tx.clone(),
seen_duration,
) {
if let Some(gossip_verified_block) = self
.process_gossip_unverified_block(
message_id,
peer_id,
peer_client,
block,
reprocess_tx.clone(),
seen_duration,
)
.await
{
let block_root = gossip_verified_block.block_root;
if let Some(handle) = duplicate_cache.check_and_insert(block_root) {
self.process_gossip_verified_block(
@ -661,7 +665,8 @@ impl<T: BeaconChainTypes> Worker<T> {
gossip_verified_block,
reprocess_tx,
seen_duration,
);
)
.await;
// Drop the handle to remove the entry from the cache
drop(handle);
} else {
@ -678,12 +683,12 @@ impl<T: BeaconChainTypes> Worker<T> {
/// if it passes gossip propagation criteria, tell the network thread to forward it.
///
/// Returns the `GossipVerifiedBlock` if verification passes and raises a log if there are errors.
pub fn process_gossip_unverified_block(
pub async fn process_gossip_unverified_block(
&self,
message_id: MessageId,
peer_id: PeerId,
peer_client: Client,
block: SignedBeaconBlock<T::EthSpec>,
block: Arc<SignedBeaconBlock<T::EthSpec>>,
reprocess_tx: mpsc::Sender<ReprocessQueueMessage<T>>,
seen_duration: Duration,
) -> Option<GossipVerifiedBlock<T>> {
@ -704,7 +709,7 @@ impl<T: BeaconChainTypes> Worker<T> {
Some(peer_client.to_string()),
);
let verified_block = match self.chain.verify_block_for_gossip(block) {
let verified_block = match self.chain.clone().verify_block_for_gossip(block).await {
Ok(verified_block) => {
if block_delay >= self.chain.slot_clock.unagg_attestation_production_delay() {
metrics::inc_counter(&metrics::BEACON_BLOCK_GOSSIP_ARRIVED_LATE_TOTAL);
@ -887,7 +892,7 @@ impl<T: BeaconChainTypes> Worker<T> {
/// Process the beacon block that has already passed gossip verification.
///
/// Raises a log if there are errors.
pub fn process_gossip_verified_block(
pub async fn process_gossip_verified_block(
self,
peer_id: PeerId,
verified_block: GossipVerifiedBlock<T>,
@ -895,9 +900,9 @@ impl<T: BeaconChainTypes> Worker<T> {
// This value is not used presently, but it might come in handy for debugging.
_seen_duration: Duration,
) {
let block = Box::new(verified_block.block.clone());
let block: Arc<_> = verified_block.block.clone();
match self.chain.process_block(verified_block) {
match self.chain.process_block(verified_block).await {
Ok(block_root) => {
metrics::inc_counter(&metrics::BEACON_PROCESSOR_GOSSIP_BLOCK_IMPORTED_TOTAL);
@ -913,24 +918,27 @@ impl<T: BeaconChainTypes> Worker<T> {
)
};
trace!(
debug!(
self.log,
"Gossipsub block processed";
"block" => ?block_root,
"peer_id" => %peer_id
);
match self.chain.fork_choice() {
Ok(()) => trace!(
self.log,
"Fork choice success";
"location" => "block gossip"
),
Err(e) => error!(
if let Err(e) = self.chain.recompute_head_at_current_slot().await {
error!(
self.log,
"Fork choice failed";
"error" => ?e,
"location" => "block gossip"
),
"location" => "block_gossip"
)
} else {
debug!(
self.log,
"Fork choice success";
"block" => ?block_root,
"location" => "block_gossip"
)
}
}
Err(BlockError::ParentUnknown { .. }) => {
@ -1144,13 +1152,9 @@ impl<T: BeaconChainTypes> Worker<T> {
.read()
.register_gossip_attester_slashing(slashing.as_inner());
if let Err(e) = self.chain.import_attester_slashing(slashing) {
debug!(self.log, "Error importing attester slashing"; "error" => ?e);
metrics::inc_counter(&metrics::BEACON_PROCESSOR_ATTESTER_SLASHING_ERROR_TOTAL);
} else {
debug!(self.log, "Successfully imported attester slashing");
metrics::inc_counter(&metrics::BEACON_PROCESSOR_ATTESTER_SLASHING_IMPORTED_TOTAL);
}
self.chain.import_attester_slashing(slashing);
debug!(self.log, "Successfully imported attester slashing");
metrics::inc_counter(&metrics::BEACON_PROCESSOR_ATTESTER_SLASHING_IMPORTED_TOTAL);
}
/// Process the sync committee signature received from the gossip network and:

View File

@ -9,6 +9,7 @@ use lighthouse_network::rpc::*;
use lighthouse_network::{PeerId, PeerRequestId, ReportSource, Response, SyncInfo};
use slog::{debug, error};
use slot_clock::SlotClock;
use std::sync::Arc;
use task_executor::TaskExecutor;
use types::{Epoch, EthSpec, Hash256, Slot};
@ -62,7 +63,7 @@ impl<T: BeaconChainTypes> Worker<T> {
&self,
remote: &StatusMessage,
) -> Result<Option<String>, BeaconChainError> {
let local = self.chain.status_message()?;
let local = self.chain.status_message();
let start_slot = |epoch: Epoch| epoch.start_slot(T::EthSpec::slots_per_epoch());
let irrelevant_reason = if local.fork_digest != remote.fork_digest {
@ -143,7 +144,7 @@ impl<T: BeaconChainTypes> Worker<T> {
Ok(Some(block)) => {
self.send_response(
peer_id,
Response::BlocksByRoot(Some(Box::new(block))),
Response::BlocksByRoot(Some(block)),
request_id,
);
send_block_count += 1;
@ -266,7 +267,7 @@ impl<T: BeaconChainTypes> Worker<T> {
blocks_sent += 1;
self.send_network_message(NetworkMessage::SendResponse {
peer_id,
response: Response::BlocksByRange(Some(Box::new(block))),
response: Response::BlocksByRange(Some(Arc::new(block))),
id: request_id,
});
}

View File

@ -11,7 +11,8 @@ use beacon_chain::{
BeaconChainError, BeaconChainTypes, BlockError, ChainSegmentResult, HistoricalBlockError,
};
use lighthouse_network::PeerAction;
use slog::{debug, error, info, trace, warn};
use slog::{debug, error, info, warn};
use std::sync::Arc;
use tokio::sync::mpsc;
use types::{Epoch, Hash256, SignedBeaconBlock};
@ -45,9 +46,9 @@ pub enum FailureMode {
impl<T: BeaconChainTypes> Worker<T> {
/// Attempt to process a block received from a direct RPC request.
pub fn process_rpc_block(
pub async fn process_rpc_block(
self,
block: SignedBeaconBlock<T::EthSpec>,
block: Arc<SignedBeaconBlock<T::EthSpec>>,
seen_timestamp: Duration,
process_type: BlockProcessType,
reprocess_tx: mpsc::Sender<ReprocessQueueMessage<T>>,
@ -66,7 +67,7 @@ impl<T: BeaconChainTypes> Worker<T> {
}
};
let slot = block.slot();
let result = self.chain.process_block(block);
let result = self.chain.process_block(block).await;
metrics::inc_counter(&metrics::BEACON_PROCESSOR_RPC_BLOCK_IMPORTED_TOTAL);
@ -87,7 +88,8 @@ impl<T: BeaconChainTypes> Worker<T> {
None,
None,
);
self.run_fork_choice()
self.recompute_head("process_rpc_block").await;
}
}
// Sync handles these results
@ -102,10 +104,10 @@ impl<T: BeaconChainTypes> Worker<T> {
/// Attempt to import the chain segment (`blocks`) to the beacon chain, informing the sync
/// thread if more blocks are needed to process it.
pub fn process_chain_segment(
pub async fn process_chain_segment(
&self,
sync_type: ChainSegmentProcessId,
downloaded_blocks: Vec<SignedBeaconBlock<T::EthSpec>>,
downloaded_blocks: Vec<Arc<SignedBeaconBlock<T::EthSpec>>>,
) {
let result = match sync_type {
// this a request from the range sync
@ -114,7 +116,7 @@ impl<T: BeaconChainTypes> Worker<T> {
let end_slot = downloaded_blocks.last().map(|b| b.slot().as_u64());
let sent_blocks = downloaded_blocks.len();
match self.process_blocks(downloaded_blocks.iter()) {
match self.process_blocks(downloaded_blocks.iter()).await {
(_, Ok(_)) => {
debug!(self.log, "Batch processed";
"batch_epoch" => epoch,
@ -183,7 +185,7 @@ impl<T: BeaconChainTypes> Worker<T> {
);
// parent blocks are ordered from highest slot to lowest, so we need to process in
// reverse
match self.process_blocks(downloaded_blocks.iter().rev()) {
match self.process_blocks(downloaded_blocks.iter().rev()).await {
(imported_blocks, Err(e)) => {
debug!(self.log, "Parent lookup failed"; "error" => %e.message);
BatchProcessResult::Failed {
@ -204,19 +206,17 @@ impl<T: BeaconChainTypes> Worker<T> {
}
/// Helper function to process blocks batches which only consumes the chain and blocks to process.
fn process_blocks<'a>(
async fn process_blocks<'a>(
&self,
downloaded_blocks: impl Iterator<Item = &'a SignedBeaconBlock<T::EthSpec>>,
downloaded_blocks: impl Iterator<Item = &'a Arc<SignedBeaconBlock<T::EthSpec>>>,
) -> (usize, Result<(), ChainSegmentFailed>) {
let blocks = downloaded_blocks.cloned().collect::<Vec<_>>();
match self.chain.process_chain_segment(blocks) {
let blocks: Vec<Arc<_>> = downloaded_blocks.cloned().collect();
match self.chain.process_chain_segment(blocks).await {
ChainSegmentResult::Successful { imported_blocks } => {
metrics::inc_counter(&metrics::BEACON_PROCESSOR_CHAIN_SEGMENT_SUCCESS_TOTAL);
if imported_blocks > 0 {
// Batch completed successfully with at least one block, run fork choice.
self.run_fork_choice();
self.recompute_head("process_blocks_ok").await;
}
(imported_blocks, Ok(()))
}
ChainSegmentResult::Failed {
@ -226,7 +226,7 @@ impl<T: BeaconChainTypes> Worker<T> {
metrics::inc_counter(&metrics::BEACON_PROCESSOR_CHAIN_SEGMENT_FAILED_TOTAL);
let r = self.handle_failed_chain_segment(error);
if imported_blocks > 0 {
self.run_fork_choice();
self.recompute_head("process_blocks_err").await;
}
(imported_blocks, r)
}
@ -236,9 +236,13 @@ impl<T: BeaconChainTypes> Worker<T> {
/// Helper function to process backfill block batches which only consumes the chain and blocks to process.
fn process_backfill_blocks(
&self,
blocks: Vec<SignedBeaconBlock<T::EthSpec>>,
blocks: Vec<Arc<SignedBeaconBlock<T::EthSpec>>>,
) -> (usize, Result<(), ChainSegmentFailed>) {
let blinded_blocks = blocks.into_iter().map(Into::into).collect();
let blinded_blocks = blocks
.iter()
.map(|full_block| full_block.clone_as_blinded())
.map(Arc::new)
.collect();
match self.chain.import_historical_block_batch(blinded_blocks) {
Ok(imported_blocks) => {
metrics::inc_counter(
@ -357,18 +361,18 @@ impl<T: BeaconChainTypes> Worker<T> {
/// Runs fork-choice on a given chain. This is used during block processing after one successful
/// block import.
fn run_fork_choice(&self) {
match self.chain.fork_choice() {
Ok(()) => trace!(
async fn recompute_head(&self, location: &str) {
match self.chain.recompute_head_at_current_slot().await {
Ok(()) => debug!(
self.log,
"Fork choice success";
"location" => "batch processing"
"location" => location
),
Err(e) => error!(
self.log,
"Fork choice failed";
"error" => ?e,
"location" => "batch import error"
"location" => location
),
}
}

View File

@ -143,10 +143,6 @@ lazy_static! {
"beacon_processor_attester_slashing_imported_total",
"Total number of attester slashings imported to the op pool."
);
pub static ref BEACON_PROCESSOR_ATTESTER_SLASHING_ERROR_TOTAL: Result<IntCounter> = try_create_int_counter(
"beacon_processor_attester_slashing_error_total",
"Total number of attester slashings that raised an error during processing."
);
// Rpc blocks.
pub static ref BEACON_PROCESSOR_RPC_BLOCK_QUEUE_TOTAL: Result<IntGauge> = try_create_int_gauge(
"beacon_processor_rpc_block_queue_total",

View File

@ -2,9 +2,10 @@ use crate::beacon_processor::{
BeaconProcessor, WorkEvent as BeaconWorkEvent, MAX_WORK_EVENT_QUEUE_LEN,
};
use crate::service::{NetworkMessage, RequestId};
use crate::status::status_message;
use crate::sync::manager::RequestId as SyncId;
use crate::sync::SyncMessage;
use beacon_chain::{BeaconChain, BeaconChainError, BeaconChainTypes};
use beacon_chain::{BeaconChain, BeaconChainTypes};
use lighthouse_network::rpc::*;
use lighthouse_network::{
Client, MessageId, NetworkGlobals, PeerId, PeerRequestId, Request, Response,
@ -114,11 +115,10 @@ impl<T: BeaconChainTypes> Processor<T> {
/// Called when we first connect to a peer, or when the PeerManager determines we need to
/// re-status.
pub fn send_status(&mut self, peer_id: PeerId) {
if let Ok(status_message) = status_message(&self.chain) {
debug!(self.log, "Sending Status Request"; "peer" => %peer_id, &status_message);
self.network
.send_processor_request(peer_id, Request::Status(status_message));
}
let status_message = status_message(&self.chain);
debug!(self.log, "Sending Status Request"; "peer" => %peer_id, &status_message);
self.network
.send_processor_request(peer_id, Request::Status(status_message));
}
/// Handle a `Status` request.
@ -132,12 +132,12 @@ impl<T: BeaconChainTypes> Processor<T> {
) {
debug!(self.log, "Received Status Request"; "peer_id" => %peer_id, &status);
// ignore status responses if we are shutting down
if let Ok(status_message) = status_message(&self.chain) {
// Say status back.
self.network
.send_response(peer_id, Response::Status(status_message), request_id);
}
// Say status back.
self.network.send_response(
peer_id,
Response::Status(status_message(&self.chain)),
request_id,
);
self.send_beacon_processor_work(BeaconWorkEvent::status_message(peer_id, status))
}
@ -178,7 +178,7 @@ impl<T: BeaconChainTypes> Processor<T> {
&mut self,
peer_id: PeerId,
request_id: RequestId,
beacon_block: Option<Box<SignedBeaconBlock<T::EthSpec>>>,
beacon_block: Option<Arc<SignedBeaconBlock<T::EthSpec>>>,
) {
let request_id = match request_id {
RequestId::Sync(sync_id) => match sync_id {
@ -209,7 +209,7 @@ impl<T: BeaconChainTypes> Processor<T> {
&mut self,
peer_id: PeerId,
request_id: RequestId,
beacon_block: Option<Box<SignedBeaconBlock<T::EthSpec>>>,
beacon_block: Option<Arc<SignedBeaconBlock<T::EthSpec>>>,
) {
let request_id = match request_id {
RequestId::Sync(sync_id) => match sync_id {
@ -244,7 +244,7 @@ impl<T: BeaconChainTypes> Processor<T> {
message_id: MessageId,
peer_id: PeerId,
peer_client: Client,
block: Box<SignedBeaconBlock<T::EthSpec>>,
block: Arc<SignedBeaconBlock<T::EthSpec>>,
) {
self.send_beacon_processor_work(BeaconWorkEvent::gossip_beacon_block(
message_id,
@ -370,22 +370,6 @@ impl<T: BeaconChainTypes> Processor<T> {
}
}
/// Build a `StatusMessage` representing the state of the given `beacon_chain`.
pub(crate) fn status_message<T: BeaconChainTypes>(
beacon_chain: &BeaconChain<T>,
) -> Result<StatusMessage, BeaconChainError> {
let head_info = beacon_chain.head_info()?;
let fork_digest = beacon_chain.enr_fork_id().fork_digest;
Ok(StatusMessage {
fork_digest,
finalized_root: head_info.finalized_checkpoint.root,
finalized_epoch: head_info.finalized_checkpoint.epoch,
head_root: head_info.block_root,
head_slot: head_info.slot,
})
}
/// Wraps a Network Channel to employ various RPC related network functionality for the
/// processor.
#[derive(Clone)]

View File

@ -7,7 +7,7 @@ use crate::{
subnet_service::{AttestationService, SubnetServiceMessage},
NetworkConfig,
};
use beacon_chain::{BeaconChain, BeaconChainError, BeaconChainTypes};
use beacon_chain::{BeaconChain, BeaconChainTypes};
use futures::channel::mpsc::Sender;
use futures::future::OptionFuture;
use futures::prelude::*;
@ -30,8 +30,8 @@ use task_executor::ShutdownReason;
use tokio::sync::mpsc;
use tokio::time::Sleep;
use types::{
ChainSpec, EthSpec, ForkContext, RelativeEpoch, Slot, SubnetId, SyncCommitteeSubscription,
SyncSubnetId, Unsigned, ValidatorSubscription,
ChainSpec, EthSpec, ForkContext, Slot, SubnetId, SyncCommitteeSubscription, SyncSubnetId,
Unsigned, ValidatorSubscription,
};
mod tests;
@ -706,29 +706,12 @@ impl<T: BeaconChainTypes> NetworkService<T> {
fn update_gossipsub_parameters(&mut self) {
if let Ok(slot) = self.beacon_chain.slot() {
if let Some(active_validators) = self
let active_validators_opt = self
.beacon_chain
.with_head(|head| {
Ok::<_, BeaconChainError>(
head.beacon_state
.get_cached_active_validator_indices(RelativeEpoch::Current)
.map(|indices| indices.len())
.ok()
.or_else(|| {
// if active validator cached was not build we count the
// active validators
self.beacon_chain.epoch().ok().map(|current_epoch| {
head.beacon_state
.validators()
.iter()
.filter(|validator| validator.is_active_at(current_epoch))
.count()
})
}),
)
})
.unwrap_or(None)
{
.canonical_head
.cached_head()
.active_validator_count();
if let Some(active_validators) = active_validators_opt {
if self
.libp2p
.swarm
@ -742,6 +725,14 @@ impl<T: BeaconChainTypes> NetworkService<T> {
"active_validators" => active_validators
);
}
} else {
// This scenario will only happen if the caches on the cached canonical head aren't
// built. That should never be the case.
error!(
self.log,
"Active validator count unavailable";
"info" => "please report this bug"
);
}
}
}

View File

@ -1,4 +1,5 @@
use beacon_chain::{BeaconChain, BeaconChainError, BeaconChainTypes};
use beacon_chain::{BeaconChain, BeaconChainTypes};
use types::{EthSpec, Hash256};
use lighthouse_network::rpc::StatusMessage;
/// Trait to produce a `StatusMessage` representing the state of the given `beacon_chain`.
@ -6,20 +7,33 @@ use lighthouse_network::rpc::StatusMessage;
/// NOTE: The purpose of this is simply to obtain a `StatusMessage` from the `BeaconChain` without
/// polluting/coupling the type with RPC concepts.
pub trait ToStatusMessage {
fn status_message(&self) -> Result<StatusMessage, BeaconChainError>;
fn status_message(&self) -> StatusMessage;
}
impl<T: BeaconChainTypes> ToStatusMessage for BeaconChain<T> {
fn status_message(&self) -> Result<StatusMessage, BeaconChainError> {
let head_info = self.head_info()?;
let fork_digest = self.enr_fork_id().fork_digest;
Ok(StatusMessage {
fork_digest,
finalized_root: head_info.finalized_checkpoint.root,
finalized_epoch: head_info.finalized_checkpoint.epoch,
head_root: head_info.block_root,
head_slot: head_info.slot,
})
fn status_message(&self) -> StatusMessage {
status_message(self)
}
}
/// Build a `StatusMessage` representing the state of the given `beacon_chain`.
pub(crate) fn status_message<T: BeaconChainTypes>(beacon_chain: &BeaconChain<T>) -> StatusMessage {
let fork_digest = beacon_chain.enr_fork_id().fork_digest;
let cached_head = beacon_chain.canonical_head.cached_head();
let mut finalized_checkpoint = cached_head.finalized_checkpoint();
// Alias the genesis checkpoint root to `0x00`.
let spec = &beacon_chain.spec;
let genesis_epoch = spec.genesis_slot.epoch(T::EthSpec::slots_per_epoch());
if finalized_checkpoint.epoch == genesis_epoch {
finalized_checkpoint.root = Hash256::zero();
}
StatusMessage {
fork_digest,
finalized_root: finalized_checkpoint.root,
finalized_epoch: finalized_checkpoint.epoch,
head_root: cached_head.head_block_root(),
head_slot: cached_head.head_slot(),
}
}

View File

@ -15,6 +15,7 @@ use std::sync::Arc;
use std::time::{Duration, SystemTime};
use store::config::StoreConfig;
use store::{HotColdDB, MemoryStore};
use task_executor::test_utils::TestRuntime;
use types::{
CommitteeIndex, Epoch, EthSpec, Hash256, MainnetEthSpec, Slot, SubnetId,
SyncCommitteeSubscription, SyncSubnetId, ValidatorSubscription,
@ -32,6 +33,7 @@ type TestBeaconChainType = Witness<
pub struct TestBeaconChain {
chain: Arc<BeaconChain<TestBeaconChainType>>,
_test_runtime: TestRuntime,
}
impl TestBeaconChain {
@ -46,11 +48,14 @@ impl TestBeaconChain {
let (shutdown_tx, _) = futures::channel::mpsc::channel(1);
let test_runtime = TestRuntime::default();
let chain = Arc::new(
BeaconChainBuilder::new(MainnetEthSpec)
.logger(log.clone())
.custom_spec(spec.clone())
.store(Arc::new(store))
.task_executor(test_runtime.task_executor.clone())
.genesis_state(
interop_genesis_state::<MainnetEthSpec>(
&keypairs,
@ -74,7 +79,10 @@ impl TestBeaconChain {
.build()
.expect("should build"),
);
Self { chain }
Self {
chain,
_test_runtime: test_runtime,
}
}
}

View File

@ -53,7 +53,7 @@ impl BatchConfig for BackFillBatchConfig {
fn max_batch_processing_attempts() -> u8 {
MAX_BATCH_PROCESSING_ATTEMPTS
}
fn batch_attempt_hash<T: EthSpec>(blocks: &[SignedBeaconBlock<T>]) -> u64 {
fn batch_attempt_hash<T: EthSpec>(blocks: &[Arc<SignedBeaconBlock<T>>]) -> u64 {
use std::collections::hash_map::DefaultHasher;
use std::hash::{Hash, Hasher};
let mut hasher = DefaultHasher::new();
@ -392,7 +392,7 @@ impl<T: BeaconChainTypes> BackFillSync<T> {
batch_id: BatchId,
peer_id: &PeerId,
request_id: Id,
beacon_block: Option<SignedBeaconBlock<T::EthSpec>>,
beacon_block: Option<Arc<SignedBeaconBlock<T::EthSpec>>>,
) -> Result<ProcessResult, BackFillError> {
// check if we have this batch
let batch = match self.batches.get_mut(&batch_id) {

View File

@ -7,6 +7,7 @@ use lighthouse_network::{PeerAction, PeerId};
use lru_cache::LRUTimeCache;
use slog::{crit, debug, error, trace, warn, Logger};
use smallvec::SmallVec;
use std::sync::Arc;
use store::{Hash256, SignedBeaconBlock};
use tokio::sync::mpsc;
@ -105,7 +106,7 @@ impl<T: BeaconChainTypes> BlockLookups<T> {
pub fn search_parent(
&mut self,
block: Box<SignedBeaconBlock<T::EthSpec>>,
block: Arc<SignedBeaconBlock<T::EthSpec>>,
peer_id: PeerId,
cx: &mut SyncNetworkContext<T::EthSpec>,
) {
@ -129,7 +130,7 @@ impl<T: BeaconChainTypes> BlockLookups<T> {
return;
}
let parent_lookup = ParentLookup::new(*block, peer_id);
let parent_lookup = ParentLookup::new(block, peer_id);
self.request_parent(parent_lookup, cx);
}
@ -139,7 +140,7 @@ impl<T: BeaconChainTypes> BlockLookups<T> {
&mut self,
id: Id,
peer_id: PeerId,
block: Option<Box<SignedBeaconBlock<T::EthSpec>>>,
block: Option<Arc<SignedBeaconBlock<T::EthSpec>>>,
seen_timestamp: Duration,
cx: &mut SyncNetworkContext<T::EthSpec>,
) {
@ -203,7 +204,7 @@ impl<T: BeaconChainTypes> BlockLookups<T> {
&mut self,
id: Id,
peer_id: PeerId,
block: Option<Box<SignedBeaconBlock<T::EthSpec>>>,
block: Option<Arc<SignedBeaconBlock<T::EthSpec>>>,
seen_timestamp: Duration,
cx: &mut SyncNetworkContext<T::EthSpec>,
) {
@ -496,7 +497,7 @@ impl<T: BeaconChainTypes> BlockLookups<T> {
Err(BlockError::ParentUnknown(block)) => {
// need to keep looking for parents
// add the block back to the queue and continue the search
parent_lookup.add_block(*block);
parent_lookup.add_block(block);
self.request_parent(parent_lookup, cx);
}
Ok(_) | Err(BlockError::BlockIsAlreadyKnown { .. }) => {
@ -618,7 +619,7 @@ impl<T: BeaconChainTypes> BlockLookups<T> {
fn send_block_for_processing(
&mut self,
block: Box<SignedBeaconBlock<T::EthSpec>>,
block: Arc<SignedBeaconBlock<T::EthSpec>>,
duration: Duration,
process_type: BlockProcessType,
) -> Result<(), ()> {

View File

@ -1,4 +1,5 @@
use lighthouse_network::PeerId;
use std::sync::Arc;
use store::{EthSpec, Hash256, SignedBeaconBlock};
use strum::IntoStaticStr;
@ -21,7 +22,7 @@ pub(crate) struct ParentLookup<T: EthSpec> {
/// The root of the block triggering this parent request.
chain_hash: Hash256,
/// The blocks that have currently been downloaded.
downloaded_blocks: Vec<SignedBeaconBlock<T>>,
downloaded_blocks: Vec<Arc<SignedBeaconBlock<T>>>,
/// Request of the last parent.
current_parent_request: SingleBlockRequest<PARENT_FAIL_TOLERANCE>,
/// Id of the last parent request.
@ -48,10 +49,10 @@ impl<T: EthSpec> ParentLookup<T> {
pub fn contains_block(&self, block: &SignedBeaconBlock<T>) -> bool {
self.downloaded_blocks
.iter()
.any(|d_block| d_block == block)
.any(|d_block| d_block.as_ref() == block)
}
pub fn new(block: SignedBeaconBlock<T>, peer_id: PeerId) -> Self {
pub fn new(block: Arc<SignedBeaconBlock<T>>, peer_id: PeerId) -> Self {
let current_parent_request = SingleBlockRequest::new(block.parent_root(), peer_id);
Self {
@ -86,7 +87,7 @@ impl<T: EthSpec> ParentLookup<T> {
self.current_parent_request.check_peer_disconnected(peer_id)
}
pub fn add_block(&mut self, block: SignedBeaconBlock<T>) {
pub fn add_block(&mut self, block: Arc<SignedBeaconBlock<T>>) {
let next_parent = block.parent_root();
self.downloaded_blocks.push(block);
self.current_parent_request.hash = next_parent;
@ -108,7 +109,7 @@ impl<T: EthSpec> ParentLookup<T> {
self.current_parent_request_id = None;
}
pub fn chain_blocks(&mut self) -> Vec<SignedBeaconBlock<T>> {
pub fn chain_blocks(&mut self) -> Vec<Arc<SignedBeaconBlock<T>>> {
std::mem::take(&mut self.downloaded_blocks)
}
@ -116,9 +117,9 @@ impl<T: EthSpec> ParentLookup<T> {
/// the processing result of the block.
pub fn verify_block(
&mut self,
block: Option<Box<SignedBeaconBlock<T>>>,
block: Option<Arc<SignedBeaconBlock<T>>>,
failed_chains: &mut lru_cache::LRUTimeCache<Hash256>,
) -> Result<Option<Box<SignedBeaconBlock<T>>>, VerifyError> {
) -> Result<Option<Arc<SignedBeaconBlock<T>>>, VerifyError> {
let block = self.current_parent_request.verify_block(block)?;
// check if the parent of this block isn't in the failed cache. If it is, this chain should

View File

@ -1,4 +1,5 @@
use std::collections::HashSet;
use std::sync::Arc;
use lighthouse_network::{rpc::BlocksByRootRequest, PeerId};
use rand::seq::IteratorRandom;
@ -82,8 +83,8 @@ impl<const MAX_ATTEMPTS: u8> SingleBlockRequest<MAX_ATTEMPTS> {
/// Returns the block for processing if the response is what we expected.
pub fn verify_block<T: EthSpec>(
&mut self,
block: Option<Box<SignedBeaconBlock<T>>>,
) -> Result<Option<Box<SignedBeaconBlock<T>>>, VerifyError> {
block: Option<Arc<SignedBeaconBlock<T>>>,
) -> Result<Option<Arc<SignedBeaconBlock<T>>>, VerifyError> {
match self.state {
State::AwaitingDownload => {
self.register_failure();
@ -195,7 +196,7 @@ mod tests {
let mut sl = SingleBlockRequest::<4>::new(block.canonical_root(), peer_id);
sl.request_block().unwrap();
sl.verify_block(Some(Box::new(block))).unwrap().unwrap();
sl.verify_block(Some(Arc::new(block))).unwrap().unwrap();
}
#[test]

View File

@ -158,7 +158,7 @@ fn test_single_block_lookup_happy_path() {
// The peer provides the correct block, should not be penalized. Now the block should be sent
// for processing.
bl.single_block_lookup_response(id, peer_id, Some(Box::new(block)), D, &mut cx);
bl.single_block_lookup_response(id, peer_id, Some(Arc::new(block)), D, &mut cx);
rig.expect_empty_network();
rig.expect_block_process();
@ -204,7 +204,7 @@ fn test_single_block_lookup_wrong_response() {
// Peer sends something else. It should be penalized.
let bad_block = rig.rand_block();
bl.single_block_lookup_response(id, peer_id, Some(Box::new(bad_block)), D, &mut cx);
bl.single_block_lookup_response(id, peer_id, Some(Arc::new(bad_block)), D, &mut cx);
rig.expect_penalty();
rig.expect_block_request(); // should be retried
@ -243,7 +243,7 @@ fn test_single_block_lookup_becomes_parent_request() {
// The peer provides the correct block, should not be penalized. Now the block should be sent
// for processing.
bl.single_block_lookup_response(id, peer_id, Some(Box::new(block.clone())), D, &mut cx);
bl.single_block_lookup_response(id, peer_id, Some(Arc::new(block.clone())), D, &mut cx);
rig.expect_empty_network();
rig.expect_block_process();
@ -252,7 +252,7 @@ fn test_single_block_lookup_becomes_parent_request() {
// Send the stream termination. Peer should have not been penalized, and the request moved to a
// parent request after processing.
bl.single_block_processed(id, Err(BlockError::ParentUnknown(Box::new(block))), &mut cx);
bl.single_block_processed(id, Err(BlockError::ParentUnknown(Arc::new(block))), &mut cx);
assert_eq!(bl.single_block_lookups.len(), 0);
rig.expect_parent_request();
rig.expect_empty_network();
@ -269,11 +269,11 @@ fn test_parent_lookup_happy_path() {
let peer_id = PeerId::random();
// Trigger the request
bl.search_parent(Box::new(block), peer_id, &mut cx);
bl.search_parent(Arc::new(block), peer_id, &mut cx);
let id = rig.expect_parent_request();
// Peer sends the right block, it should be sent for processing. Peer should not be penalized.
bl.parent_lookup_response(id, peer_id, Some(Box::new(parent)), D, &mut cx);
bl.parent_lookup_response(id, peer_id, Some(Arc::new(parent)), D, &mut cx);
rig.expect_block_process();
rig.expect_empty_network();
@ -294,12 +294,12 @@ fn test_parent_lookup_wrong_response() {
let peer_id = PeerId::random();
// Trigger the request
bl.search_parent(Box::new(block), peer_id, &mut cx);
bl.search_parent(Arc::new(block), peer_id, &mut cx);
let id1 = rig.expect_parent_request();
// Peer sends the wrong block, peer should be penalized and the block re-requested.
let bad_block = rig.rand_block();
bl.parent_lookup_response(id1, peer_id, Some(Box::new(bad_block)), D, &mut cx);
bl.parent_lookup_response(id1, peer_id, Some(Arc::new(bad_block)), D, &mut cx);
rig.expect_penalty();
let id2 = rig.expect_parent_request();
@ -308,7 +308,7 @@ fn test_parent_lookup_wrong_response() {
rig.expect_empty_network();
// Send the right block this time.
bl.parent_lookup_response(id2, peer_id, Some(Box::new(parent)), D, &mut cx);
bl.parent_lookup_response(id2, peer_id, Some(Arc::new(parent)), D, &mut cx);
rig.expect_block_process();
// Processing succeeds, now the rest of the chain should be sent for processing.
@ -328,7 +328,7 @@ fn test_parent_lookup_empty_response() {
let peer_id = PeerId::random();
// Trigger the request
bl.search_parent(Box::new(block), peer_id, &mut cx);
bl.search_parent(Arc::new(block), peer_id, &mut cx);
let id1 = rig.expect_parent_request();
// Peer sends an empty response, peer should be penalized and the block re-requested.
@ -337,7 +337,7 @@ fn test_parent_lookup_empty_response() {
let id2 = rig.expect_parent_request();
// Send the right block this time.
bl.parent_lookup_response(id2, peer_id, Some(Box::new(parent)), D, &mut cx);
bl.parent_lookup_response(id2, peer_id, Some(Arc::new(parent)), D, &mut cx);
rig.expect_block_process();
// Processing succeeds, now the rest of the chain should be sent for processing.
@ -357,7 +357,7 @@ fn test_parent_lookup_rpc_failure() {
let peer_id = PeerId::random();
// Trigger the request
bl.search_parent(Box::new(block), peer_id, &mut cx);
bl.search_parent(Arc::new(block), peer_id, &mut cx);
let id1 = rig.expect_parent_request();
// The request fails. It should be tried again.
@ -365,7 +365,7 @@ fn test_parent_lookup_rpc_failure() {
let id2 = rig.expect_parent_request();
// Send the right block this time.
bl.parent_lookup_response(id2, peer_id, Some(Box::new(parent)), D, &mut cx);
bl.parent_lookup_response(id2, peer_id, Some(Arc::new(parent)), D, &mut cx);
rig.expect_block_process();
// Processing succeeds, now the rest of the chain should be sent for processing.
@ -385,7 +385,7 @@ fn test_parent_lookup_too_many_attempts() {
let peer_id = PeerId::random();
// Trigger the request
bl.search_parent(Box::new(block), peer_id, &mut cx);
bl.search_parent(Arc::new(block), peer_id, &mut cx);
for i in 1..=parent_lookup::PARENT_FAIL_TOLERANCE + 1 {
let id = rig.expect_parent_request();
match i % 2 {
@ -397,7 +397,7 @@ fn test_parent_lookup_too_many_attempts() {
_ => {
// Send a bad block this time. It should be tried again.
let bad_block = rig.rand_block();
bl.parent_lookup_response(id, peer_id, Some(Box::new(bad_block)), D, &mut cx);
bl.parent_lookup_response(id, peer_id, Some(Arc::new(bad_block)), D, &mut cx);
rig.expect_penalty();
}
}
@ -427,12 +427,12 @@ fn test_parent_lookup_too_deep() {
let peer_id = PeerId::random();
let trigger_block = blocks.pop().unwrap();
let chain_hash = trigger_block.canonical_root();
bl.search_parent(Box::new(trigger_block), peer_id, &mut cx);
bl.search_parent(Arc::new(trigger_block), peer_id, &mut cx);
for block in blocks.into_iter().rev() {
let id = rig.expect_parent_request();
// the block
bl.parent_lookup_response(id, peer_id, Some(Box::new(block.clone())), D, &mut cx);
bl.parent_lookup_response(id, peer_id, Some(Arc::new(block.clone())), D, &mut cx);
// the stream termination
bl.parent_lookup_response(id, peer_id, None, D, &mut cx);
// the processing request
@ -440,7 +440,7 @@ fn test_parent_lookup_too_deep() {
// the processing result
bl.parent_block_processed(
chain_hash,
Err(BlockError::ParentUnknown(Box::new(block))),
Err(BlockError::ParentUnknown(Arc::new(block))),
&mut cx,
)
}
@ -454,7 +454,7 @@ fn test_parent_lookup_disconnection() {
let (mut bl, mut cx, mut rig) = TestRig::test_setup(None);
let peer_id = PeerId::random();
let trigger_block = rig.rand_block();
bl.search_parent(Box::new(trigger_block), peer_id, &mut cx);
bl.search_parent(Arc::new(trigger_block), peer_id, &mut cx);
bl.peer_disconnected(&peer_id, &mut cx);
assert!(bl.parent_queue.is_empty());
}

View File

@ -88,12 +88,12 @@ pub enum SyncMessage<T: EthSpec> {
RpcBlock {
request_id: RequestId,
peer_id: PeerId,
beacon_block: Option<Box<SignedBeaconBlock<T>>>,
beacon_block: Option<Arc<SignedBeaconBlock<T>>>,
seen_timestamp: Duration,
},
/// A block with an unknown parent has been received.
UnknownBlock(PeerId, Box<SignedBeaconBlock<T>>),
UnknownBlock(PeerId, Arc<SignedBeaconBlock<T>>),
/// A peer has sent an object that references a block that is unknown. This triggers the
/// manager to attempt to find the block matching the unknown hash.
@ -229,17 +229,12 @@ impl<T: BeaconChainTypes> SyncManager<T> {
/// ours that we consider it fully sync'd with respect to our current chain.
fn add_peer(&mut self, peer_id: PeerId, remote: SyncInfo) {
// ensure the beacon chain still exists
let local = match self.chain.status_message() {
Ok(status) => SyncInfo {
head_slot: status.head_slot,
head_root: status.head_root,
finalized_epoch: status.finalized_epoch,
finalized_root: status.finalized_root,
},
Err(e) => {
return error!(self.log, "Failed to get peer sync info";
"msg" => "likely due to head lock contention", "err" => ?e)
}
let status = self.chain.status_message();
let local = SyncInfo {
head_slot: status.head_slot,
head_root: status.head_root,
finalized_epoch: status.finalized_epoch,
finalized_root: status.finalized_root,
};
let sync_type = remote_sync_type(&local, &remote, &self.chain);
@ -379,7 +374,7 @@ impl<T: BeaconChainTypes> SyncManager<T> {
// advanced and will produce a head chain on re-status. Otherwise it will shift
// to being synced
let mut sync_state = {
let head = self.chain.best_slot().unwrap_or_else(|_| Slot::new(0));
let head = self.chain.best_slot();
let current_slot = self.chain.slot().unwrap_or_else(|_| Slot::new(0));
let peers = self.network_globals.peers.read();
@ -482,11 +477,7 @@ impl<T: BeaconChainTypes> SyncManager<T> {
SyncMessage::UnknownBlock(peer_id, block) => {
// If we are not synced or within SLOT_IMPORT_TOLERANCE of the block, ignore
if !self.network_globals.sync_state.read().is_synced() {
let head_slot = self
.chain
.head_info()
.map(|info| info.slot)
.unwrap_or_else(|_| Slot::from(0u64));
let head_slot = self.chain.canonical_head.cached_head().head_slot();
let unknown_block_slot = block.slot();
// if the block is far in the future, ignore it. If its within the slot tolerance of
@ -571,7 +562,7 @@ impl<T: BeaconChainTypes> SyncManager<T> {
&mut self,
request_id: RequestId,
peer_id: PeerId,
beacon_block: Option<Box<SignedBeaconBlock<T::EthSpec>>>,
beacon_block: Option<Arc<SignedBeaconBlock<T::EthSpec>>>,
seen_timestamp: Duration,
) {
match request_id {
@ -599,7 +590,7 @@ impl<T: BeaconChainTypes> SyncManager<T> {
batch_id,
&peer_id,
id,
beacon_block.map(|b| *b),
beacon_block,
) {
Ok(ProcessResult::SyncCompleted) => self.update_sync_state(),
Ok(ProcessResult::Successful) => {}
@ -621,7 +612,7 @@ impl<T: BeaconChainTypes> SyncManager<T> {
chain_id,
batch_id,
id,
beacon_block.map(|b| *b),
beacon_block,
);
self.update_sync_state();
}

View File

@ -65,27 +65,26 @@ impl<T: EthSpec> SyncNetworkContext<T> {
chain: &C,
peers: impl Iterator<Item = PeerId>,
) {
if let Ok(status_message) = chain.status_message() {
for peer_id in peers {
debug!(
self.log,
"Sending Status Request";
"peer" => %peer_id,
"fork_digest" => ?status_message.fork_digest,
"finalized_root" => ?status_message.finalized_root,
"finalized_epoch" => ?status_message.finalized_epoch,
"head_root" => %status_message.head_root,
"head_slot" => %status_message.head_slot,
);
let status_message = chain.status_message();
for peer_id in peers {
debug!(
self.log,
"Sending Status Request";
"peer" => %peer_id,
"fork_digest" => ?status_message.fork_digest,
"finalized_root" => ?status_message.finalized_root,
"finalized_epoch" => ?status_message.finalized_epoch,
"head_root" => %status_message.head_root,
"head_slot" => %status_message.head_slot,
);
let request = Request::Status(status_message.clone());
let request_id = RequestId::Router;
let _ = self.send_network_msg(NetworkMessage::SendRequest {
peer_id,
request,
request_id,
});
}
let request = Request::Status(status_message.clone());
let request_id = RequestId::Router;
let _ = self.send_network_msg(NetworkMessage::SendRequest {
peer_id,
request,
request_id,
});
}
}

View File

@ -59,7 +59,7 @@ pub fn remote_sync_type<T: BeaconChainTypes>(
if remote.head_slot < near_range_start {
PeerSyncType::Behind
} else if remote.head_slot > near_range_end
&& !chain.fork_choice.read().contains_block(&remote.head_root)
&& !chain.block_is_known_to_fork_choice(&remote.head_root)
{
// This peer has a head ahead enough of ours and we have no knowledge of their best
// block.
@ -74,7 +74,7 @@ pub fn remote_sync_type<T: BeaconChainTypes>(
if (local.finalized_epoch + 1 == remote.finalized_epoch
&& near_range_start <= remote.head_slot
&& remote.head_slot <= near_range_end)
|| chain.fork_choice.read().contains_block(&remote.head_root)
|| chain.block_is_known_to_fork_choice(&remote.head_root)
{
// This peer is near enough to us to be considered synced, or
// we have already synced up to this peer's head

View File

@ -4,6 +4,7 @@ use lighthouse_network::PeerId;
use std::collections::HashSet;
use std::hash::{Hash, Hasher};
use std::ops::Sub;
use std::sync::Arc;
use types::{Epoch, EthSpec, SignedBeaconBlock, Slot};
/// The number of times to retry a batch before it is considered failed.
@ -46,7 +47,7 @@ pub trait BatchConfig {
/// Note that simpler hashing functions considered in the past (hash of first block, hash of last
/// block, number of received blocks) are not good enough to differentiate attempts. For this
/// reason, we hash the complete set of blocks both in RangeSync and BackFillSync.
fn batch_attempt_hash<T: EthSpec>(blocks: &[SignedBeaconBlock<T>]) -> u64;
fn batch_attempt_hash<T: EthSpec>(blocks: &[Arc<SignedBeaconBlock<T>>]) -> u64;
}
pub struct RangeSyncBatchConfig {}
@ -58,7 +59,7 @@ impl BatchConfig for RangeSyncBatchConfig {
fn max_batch_processing_attempts() -> u8 {
MAX_BATCH_PROCESSING_ATTEMPTS
}
fn batch_attempt_hash<T: EthSpec>(blocks: &[SignedBeaconBlock<T>]) -> u64 {
fn batch_attempt_hash<T: EthSpec>(blocks: &[Arc<SignedBeaconBlock<T>>]) -> u64 {
let mut hasher = std::collections::hash_map::DefaultHasher::new();
blocks.hash(&mut hasher);
hasher.finish()
@ -100,9 +101,9 @@ pub enum BatchState<T: EthSpec> {
/// The batch has failed either downloading or processing, but can be requested again.
AwaitingDownload,
/// The batch is being downloaded.
Downloading(PeerId, Vec<SignedBeaconBlock<T>>, Id),
Downloading(PeerId, Vec<Arc<SignedBeaconBlock<T>>>, Id),
/// The batch has been completely downloaded and is ready for processing.
AwaitingProcessing(PeerId, Vec<SignedBeaconBlock<T>>),
AwaitingProcessing(PeerId, Vec<Arc<SignedBeaconBlock<T>>>),
/// The batch is being processed.
Processing(Attempt),
/// The batch was successfully processed and is waiting to be validated.
@ -211,7 +212,7 @@ impl<T: EthSpec, B: BatchConfig> BatchInfo<T, B> {
}
/// Adds a block to a downloading batch.
pub fn add_block(&mut self, block: SignedBeaconBlock<T>) -> Result<(), WrongState> {
pub fn add_block(&mut self, block: Arc<SignedBeaconBlock<T>>) -> Result<(), WrongState> {
match self.state.poison() {
BatchState::Downloading(peer, mut blocks, req_id) => {
blocks.push(block);
@ -337,7 +338,7 @@ impl<T: EthSpec, B: BatchConfig> BatchInfo<T, B> {
}
}
pub fn start_processing(&mut self) -> Result<Vec<SignedBeaconBlock<T>>, WrongState> {
pub fn start_processing(&mut self) -> Result<Vec<Arc<SignedBeaconBlock<T>>>, WrongState> {
match self.state.poison() {
BatchState::AwaitingProcessing(peer, blocks) => {
self.state = BatchState::Processing(Attempt::new::<B, T>(peer, &blocks));
@ -436,7 +437,10 @@ pub struct Attempt {
}
impl Attempt {
fn new<B: BatchConfig, T: EthSpec>(peer_id: PeerId, blocks: &[SignedBeaconBlock<T>]) -> Self {
fn new<B: BatchConfig, T: EthSpec>(
peer_id: PeerId,
blocks: &[Arc<SignedBeaconBlock<T>>],
) -> Self {
let hash = B::batch_attempt_hash(blocks);
Attempt { peer_id, hash }
}

View File

@ -8,6 +8,6 @@ pub trait BlockStorage {
impl<T: BeaconChainTypes> BlockStorage for BeaconChain<T> {
fn is_block_known(&self, block_root: &Hash256) -> bool {
self.fork_choice.read().contains_block(block_root)
self.block_is_known_to_fork_choice(block_root)
}
}

View File

@ -9,6 +9,7 @@ use rand::seq::SliceRandom;
use slog::{crit, debug, o, warn};
use std::collections::{btree_map::Entry, BTreeMap, HashSet};
use std::hash::{Hash, Hasher};
use std::sync::Arc;
use tokio::sync::mpsc::Sender;
use types::{Epoch, EthSpec, Hash256, SignedBeaconBlock, Slot};
@ -216,7 +217,7 @@ impl<T: BeaconChainTypes> SyncingChain<T> {
batch_id: BatchId,
peer_id: &PeerId,
request_id: Id,
beacon_block: Option<SignedBeaconBlock<T::EthSpec>>,
beacon_block: Option<Arc<SignedBeaconBlock<T::EthSpec>>>,
) -> ProcessingResult {
// check if we have this batch
let batch = match self.batches.get_mut(&batch_id) {

View File

@ -53,7 +53,7 @@ use lighthouse_network::rpc::GoodbyeReason;
use lighthouse_network::PeerId;
use lighthouse_network::SyncInfo;
use lru_cache::LRUTimeCache;
use slog::{crit, debug, error, trace, warn};
use slog::{crit, debug, trace, warn};
use std::collections::HashMap;
use std::sync::Arc;
use tokio::sync::mpsc;
@ -221,7 +221,7 @@ where
chain_id: ChainId,
batch_id: BatchId,
request_id: Id,
beacon_block: Option<SignedBeaconBlock<T::EthSpec>>,
beacon_block: Option<Arc<SignedBeaconBlock<T::EthSpec>>>,
) {
// check if this chunk removes the chain
match self.chains.call_by_id(chain_id, |chain| {
@ -365,17 +365,12 @@ where
network.status_peers(self.beacon_chain.as_ref(), chain.peers());
let local = match self.beacon_chain.status_message() {
Ok(status) => SyncInfo {
head_slot: status.head_slot,
head_root: status.head_root,
finalized_epoch: status.finalized_epoch,
finalized_root: status.finalized_root,
},
Err(e) => {
return error!(self.log, "Failed to get peer sync info";
"msg" => "likely due to head lock contention", "err" => ?e)
}
let status = self.beacon_chain.status_message();
let local = SyncInfo {
head_slot: status.head_slot,
head_root: status.head_root,
finalized_epoch: status.finalized_epoch,
finalized_root: status.finalized_root,
};
// update the state of the collection
@ -447,8 +442,8 @@ mod tests {
}
impl ToStatusMessage for FakeStorage {
fn status_message(&self) -> Result<StatusMessage, beacon_chain::BeaconChainError> {
Ok(self.status.read().clone())
fn status_message(&self) -> StatusMessage {
self.status.read().clone()
}
}

View File

@ -21,3 +21,4 @@ store = { path = "../store" }
[dev-dependencies]
beacon_chain = { path = "../beacon_chain" }
tokio = { version = "1.14.0", features = ["rt-multi-thread"] }

View File

@ -710,7 +710,7 @@ mod release_tests {
}
/// Test state for sync contribution-related tests.
fn sync_contribution_test_state<E: EthSpec>(
async fn sync_contribution_test_state<E: EthSpec>(
num_committees: usize,
) -> (BeaconChainHarness<EphemeralHarnessType<E>>, ChainSpec) {
let mut spec = E::default_spec();
@ -722,12 +722,14 @@ mod release_tests {
let harness = get_harness::<E>(num_validators, Some(spec.clone()));
let state = harness.get_current_state();
harness.add_attested_blocks_at_slots(
state,
Hash256::zero(),
&[Slot::new(1)],
(0..num_validators).collect::<Vec<_>>().as_slice(),
);
harness
.add_attested_blocks_at_slots(
state,
Hash256::zero(),
&[Slot::new(1)],
(0..num_validators).collect::<Vec<_>>().as_slice(),
)
.await;
(harness, spec)
}
@ -1454,9 +1456,9 @@ mod release_tests {
}
/// End-to-end test of basic sync contribution handling.
#[test]
fn sync_contribution_aggregation_insert_get_prune() {
let (harness, _) = sync_contribution_test_state::<MainnetEthSpec>(1);
#[tokio::test]
async fn sync_contribution_aggregation_insert_get_prune() {
let (harness, _) = sync_contribution_test_state::<MainnetEthSpec>(1).await;
let op_pool = OperationPool::<MainnetEthSpec>::new();
let state = harness.get_current_state();
@ -1514,9 +1516,9 @@ mod release_tests {
}
/// Adding a sync contribution already in the pool should not increase the size of the pool.
#[test]
fn sync_contribution_duplicate() {
let (harness, _) = sync_contribution_test_state::<MainnetEthSpec>(1);
#[tokio::test]
async fn sync_contribution_duplicate() {
let (harness, _) = sync_contribution_test_state::<MainnetEthSpec>(1).await;
let op_pool = OperationPool::<MainnetEthSpec>::new();
let state = harness.get_current_state();
@ -1551,9 +1553,9 @@ mod release_tests {
/// Adding a sync contribution already in the pool with more bits set should increase the
/// number of bits set in the aggregate.
#[test]
fn sync_contribution_with_more_bits() {
let (harness, _) = sync_contribution_test_state::<MainnetEthSpec>(1);
#[tokio::test]
async fn sync_contribution_with_more_bits() {
let (harness, _) = sync_contribution_test_state::<MainnetEthSpec>(1).await;
let op_pool = OperationPool::<MainnetEthSpec>::new();
let state = harness.get_current_state();
@ -1631,9 +1633,9 @@ mod release_tests {
/// Adding a sync contribution already in the pool with fewer bits set should not increase the
/// number of bits set in the aggregate.
#[test]
fn sync_contribution_with_fewer_bits() {
let (harness, _) = sync_contribution_test_state::<MainnetEthSpec>(1);
#[tokio::test]
async fn sync_contribution_with_fewer_bits() {
let (harness, _) = sync_contribution_test_state::<MainnetEthSpec>(1).await;
let op_pool = OperationPool::<MainnetEthSpec>::new();
let state = harness.get_current_state();

View File

@ -669,7 +669,11 @@ impl<E: EthSpec, Hot: ItemStore<E>, Cold: ItemStore<E>> HotColdDB<E, Hot, Cold>
for op in batch {
match op {
StoreOp::PutBlock(block_root, block) => {
self.block_as_kv_store_ops(&block_root, *block, &mut key_value_batch)?;
self.block_as_kv_store_ops(
&block_root,
block.as_ref().clone(),
&mut key_value_batch,
)?;
}
StoreOp::PutState(state_root, state) => {

View File

@ -39,6 +39,7 @@ pub use impls::beacon_state::StorageContainer as BeaconStateStorageContainer;
pub use metadata::AnchorInfo;
pub use metrics::scrape_for_metrics;
use parking_lot::MutexGuard;
use std::sync::Arc;
use strum::{EnumString, IntoStaticStr};
pub use types::*;
@ -152,7 +153,7 @@ pub trait ItemStore<E: EthSpec>: KeyValueStore<E> + Sync + Send + Sized + 'stati
/// Reified key-value storage operation. Helps in modifying the storage atomically.
/// See also https://github.com/sigp/lighthouse/issues/692
pub enum StoreOp<'a, E: EthSpec> {
PutBlock(Hash256, Box<SignedBeaconBlock<E>>),
PutBlock(Hash256, Arc<SignedBeaconBlock<E>>),
PutState(Hash256, &'a BeaconState<E>),
PutStateSummary(Hash256, HotStateSummary),
PutStateTemporaryFlag(Hash256),

View File

@ -3,7 +3,7 @@
//! This service allows task execution on the beacon node for various functionality.
use beacon_chain::{BeaconChain, BeaconChainTypes};
use slog::{debug, info, warn};
use slog::{info, warn};
use slot_clock::SlotClock;
use std::sync::Arc;
use tokio::time::sleep;
@ -13,11 +13,8 @@ pub fn spawn_timer<T: BeaconChainTypes>(
executor: task_executor::TaskExecutor,
beacon_chain: Arc<BeaconChain<T>>,
) -> Result<(), &'static str> {
let log = executor.log();
let per_slot_executor = executor.clone();
let log = executor.log().clone();
let timer_future = async move {
let log = per_slot_executor.log().clone();
loop {
let duration_to_next_slot = match beacon_chain.slot_clock.duration_to_next_slot() {
Some(duration) => duration,
@ -28,31 +25,12 @@ pub fn spawn_timer<T: BeaconChainTypes>(
};
sleep(duration_to_next_slot).await;
let chain = beacon_chain.clone();
if let Some(handle) = per_slot_executor
.spawn_blocking_handle(move || chain.per_slot_task(), "timer_per_slot_task")
{
if let Err(e) = handle.await {
warn!(
log,
"Per slot task failed";
"info" => ?e
);
}
} else {
debug!(
log,
"Per slot task timer stopped";
"info" => "shutting down"
);
break;
}
beacon_chain.per_slot_task().await;
}
};
executor.spawn(timer_future, "timer");
info!(log, "Timer service started");
info!(executor.log(), "Timer service started");
Ok(())
}

View File

@ -5,7 +5,7 @@ authors = ["Sigma Prime <contact@sigmaprime.io>"]
edition = "2021"
[dependencies]
tokio = { version = "1.14.0", features = ["rt-multi-thread"] }
tokio = { version = "1.14.0", features = ["rt-multi-thread", "macros"] }
slog = "2.5.2"
futures = "0.3.7"
exit-future = "0.2.0"

View File

@ -7,6 +7,8 @@ use slog::{crit, debug, o, trace};
use std::sync::Weak;
use tokio::runtime::{Handle, Runtime};
pub use tokio::task::JoinHandle;
/// Provides a reason when Lighthouse is shut down.
#[derive(Copy, Clone, Debug, PartialEq)]
pub enum ShutdownReason {
@ -312,6 +314,61 @@ impl TaskExecutor {
Some(future)
}
/// Block the current (non-async) thread on the completion of some future.
///
/// ## Warning
///
/// This method is "dangerous" since calling it from an async thread will result in a panic! Any
/// use of this outside of testing should be very deeply considered as Lighthouse has been
/// burned by this function in the past.
///
/// Determining what is an "async thread" is rather challenging; just because a function isn't
/// marked as `async` doesn't mean it's not being called from an `async` function or there isn't
/// a `tokio` context present in the thread-local storage due to some `rayon` funkiness. Talk to
/// @paulhauner if you plan to use this function in production. He has put metrics in here to
/// track any use of it, so don't think you can pull a sneaky one on him.
pub fn block_on_dangerous<F: Future>(
&self,
future: F,
name: &'static str,
) -> Option<F::Output> {
let timer = metrics::start_timer_vec(&metrics::BLOCK_ON_TASKS_HISTOGRAM, &[name]);
metrics::inc_gauge_vec(&metrics::BLOCK_ON_TASKS_COUNT, &[name]);
let log = self.log.clone();
let handle = self.handle()?;
let exit = self.exit.clone();
debug!(
log,
"Starting block_on task";
"name" => name
);
handle.block_on(async {
let output = tokio::select! {
output = future => {
debug!(
log,
"Completed block_on task";
"name" => name
);
Some(output)
},
_ = exit => {
debug!(
log,
"Cancelled block_on task";
"name" => name,
);
None
}
};
metrics::dec_gauge_vec(&metrics::BLOCK_ON_TASKS_COUNT, &[name]);
drop(timer);
output
})
}
/// Returns a `Handle` to the current runtime.
pub fn handle(&self) -> Option<Handle> {
self.handle_provider.handle()

View File

@ -18,6 +18,16 @@ lazy_static! {
"Time taken by blocking tasks",
&["blocking_task_hist"]
);
pub static ref BLOCK_ON_TASKS_COUNT: Result<IntGaugeVec> = try_create_int_gauge_vec(
"block_on_tasks_count",
"Total number of block_on_dangerous tasks spawned",
&["name"]
);
pub static ref BLOCK_ON_TASKS_HISTOGRAM: Result<HistogramVec> = try_create_histogram_vec(
"block_on_tasks_histogram",
"Time taken by block_on_dangerous tasks",
&["name"]
);
pub static ref TASKS_HISTOGRAM: Result<HistogramVec> = try_create_histogram_vec(
"async_tasks_time_histogram",
"Time taken by async tasks",

View File

@ -15,3 +15,4 @@ eth2_ssz_derive = "0.3.0"
[dev-dependencies]
beacon_chain = { path = "../../beacon_node/beacon_chain" }
store = { path = "../../beacon_node/store" }
tokio = { version = "1.14.0", features = ["rt-multi-thread"] }

View File

@ -5,7 +5,7 @@ use std::cmp::Ordering;
use std::marker::PhantomData;
use std::time::Duration;
use types::{
consts::merge::INTERVALS_PER_SLOT, AttestationShufflingId, BeaconBlock, BeaconState,
consts::merge::INTERVALS_PER_SLOT, AttestationShufflingId, BeaconBlockRef, BeaconState,
BeaconStateError, ChainSpec, Checkpoint, Epoch, EthSpec, ExecPayload, ExecutionBlockHash,
Hash256, IndexedAttestation, RelativeEpoch, SignedBeaconBlock, Slot,
};
@ -248,6 +248,7 @@ fn dequeue_attestations(
/// Equivalent to the `is_from_block` `bool` in:
///
/// https://github.com/ethereum/consensus-specs/blob/dev/specs/phase0/fork-choice.md#validate_on_attestation
#[derive(Clone, Copy)]
pub enum AttestationFromBlock {
True,
False,
@ -261,6 +262,13 @@ pub struct ForkchoiceUpdateParameters {
pub finalized_hash: Option<ExecutionBlockHash>,
}
#[derive(Clone, Copy, Debug, PartialEq)]
pub struct ForkChoiceView {
pub head_block_root: Hash256,
pub justified_checkpoint: Checkpoint,
pub finalized_checkpoint: Checkpoint,
}
/// Provides an implementation of "Ethereum 2.0 Phase 0 -- Beacon Chain Fork Choice":
///
/// https://github.com/ethereum/eth2.0-specs/blob/v0.12.1/specs/phase0/fork-choice.md#ethereum-20-phase-0----beacon-chain-fork-choice
@ -279,7 +287,9 @@ pub struct ForkChoice<T, E> {
/// Attestations that arrived at the current slot and must be queued for later processing.
queued_attestations: Vec<QueuedAttestation>,
/// Stores a cache of the values required to be sent to the execution layer.
forkchoice_update_parameters: Option<ForkchoiceUpdateParameters>,
forkchoice_update_parameters: ForkchoiceUpdateParameters,
/// The most recent result of running `Self::get_head`.
head_block_root: Hash256,
_phantom: PhantomData<E>,
}
@ -306,6 +316,8 @@ where
anchor_block_root: Hash256,
anchor_block: &SignedBeaconBlock<E>,
anchor_state: &BeaconState<E>,
current_slot: Option<Slot>,
spec: &ChainSpec,
) -> Result<Self, Error<T::Error>> {
// Sanity check: the anchor must lie on an epoch boundary.
if anchor_block.slot() % E::slots_per_epoch() != 0 {
@ -340,6 +352,9 @@ where
},
);
// If the current slot is not provided, use the value that was last provided to the store.
let current_slot = current_slot.unwrap_or_else(|| fc_store.get_current_slot());
let proto_array = ProtoArrayForkChoice::new(
finalized_block_slot,
finalized_block_state_root,
@ -350,15 +365,28 @@ where
execution_status,
)?;
Ok(Self {
let mut fork_choice = Self {
fc_store,
proto_array,
queued_attestations: vec![],
forkchoice_update_parameters: None,
// This will be updated during the next call to `Self::get_head`.
forkchoice_update_parameters: ForkchoiceUpdateParameters {
head_hash: None,
finalized_hash: None,
head_root: Hash256::zero(),
},
// This will be updated during the next call to `Self::get_head`.
head_block_root: Hash256::zero(),
_phantom: PhantomData,
})
};
// Ensure that `fork_choice.head_block_root` is updated.
fork_choice.get_head(current_slot, spec)?;
Ok(fork_choice)
}
/*
/// Instantiates `Self` from some existing components.
///
/// This is useful if the existing components have been loaded from disk after a process
@ -376,13 +404,13 @@ where
_phantom: PhantomData,
}
}
*/
/// Returns cached information that can be used to issue a `forkchoiceUpdated` message to an
/// execution engine.
///
/// These values are updated each time `Self::get_head` is called. May return `None` if
/// `Self::get_head` has not yet been called.
pub fn get_forkchoice_update_parameters(&self) -> Option<ForkchoiceUpdateParameters> {
/// These values are updated each time `Self::get_head` is called.
pub fn get_forkchoice_update_parameters(&self) -> ForkchoiceUpdateParameters {
self.forkchoice_update_parameters
}
@ -455,6 +483,8 @@ where
spec,
)?;
self.head_block_root = head_root;
// Cache some values for the next forkchoiceUpdate call to the execution layer.
let head_hash = self
.get_block(&head_root)
@ -463,15 +493,35 @@ where
let finalized_hash = self
.get_block(&finalized_root)
.and_then(|b| b.execution_status.block_hash());
self.forkchoice_update_parameters = Some(ForkchoiceUpdateParameters {
self.forkchoice_update_parameters = ForkchoiceUpdateParameters {
head_root,
head_hash,
finalized_hash,
});
};
Ok(head_root)
}
/// Return information about:
///
/// - The LMD head of the chain.
/// - The FFG checkpoints.
///
/// The information is "cached" since the last call to `Self::get_head`.
///
/// ## Notes
///
/// The finalized/justified checkpoints are determined from the fork choice store. Therefore,
/// it's possible that the state corresponding to `get_state(get_block(head_block_root))` will
/// have *differing* finalized and justified information.
pub fn cached_fork_choice_view(&self) -> ForkChoiceView {
ForkChoiceView {
head_block_root: self.head_block_root,
justified_checkpoint: self.justified_checkpoint(),
finalized_checkpoint: self.finalized_checkpoint(),
}
}
/// Returns `true` if the given `store` should be updated to set
/// `state.current_justified_checkpoint` its `justified_checkpoint`.
///
@ -566,7 +616,7 @@ where
pub fn on_block<Payload: ExecPayload<E>>(
&mut self,
current_slot: Slot,
block: &BeaconBlock<E, Payload>,
block: BeaconBlockRef<E, Payload>,
block_root: Hash256,
block_delay: Duration,
state: &BeaconState<E>,
@ -966,6 +1016,11 @@ where
}
}
/// Returns the weight for the given block root.
pub fn get_block_weight(&self, block_root: &Hash256) -> Option<u64> {
self.proto_array.get_weight(block_root)
}
/// Returns the `ProtoBlock` for the justified checkpoint.
///
/// ## Notes
@ -995,6 +1050,39 @@ where
.is_descendant(self.fc_store.finalized_checkpoint().root, block_root)
}
/// Returns `Ok(true)` if `block_root` has been imported optimistically. That is, the
/// execution payload has not been verified.
///
/// Returns `Ok(false)` if `block_root`'s execution payload has been verfied, if it is a
/// pre-Bellatrix block or if it is before the PoW terminal block.
///
/// In the case where the block could not be found in fork-choice, it returns the
/// `execution_status` of the current finalized block.
///
/// This function assumes the `block_root` exists.
pub fn is_optimistic_block(&self, block_root: &Hash256) -> Result<bool, Error<T::Error>> {
if let Some(status) = self.get_block_execution_status(block_root) {
Ok(status.is_optimistic())
} else {
Ok(self.get_finalized_block()?.execution_status.is_optimistic())
}
}
/// The same as `is_optimistic_block` but does not fallback to `self.get_finalized_block`
/// when the block cannot be found.
///
/// Intended to be used when checking if the head has been imported optimistically.
pub fn is_optimistic_block_no_fallback(
&self,
block_root: &Hash256,
) -> Result<bool, Error<T::Error>> {
if let Some(status) = self.get_block_execution_status(block_root) {
Ok(status.is_optimistic())
} else {
Err(Error::MissingProtoArrayBlock(*block_root))
}
}
/// Returns `Ok(false)` if a block is not viable to be imported optimistically.
///
/// ## Notes
@ -1109,17 +1197,31 @@ where
pub fn from_persisted(
persisted: PersistedForkChoice,
fc_store: T,
spec: &ChainSpec,
) -> Result<Self, Error<T::Error>> {
let proto_array = ProtoArrayForkChoice::from_bytes(&persisted.proto_array_bytes)
.map_err(Error::InvalidProtoArrayBytes)?;
Ok(Self {
let current_slot = fc_store.get_current_slot();
let mut fork_choice = Self {
fc_store,
proto_array,
queued_attestations: persisted.queued_attestations,
forkchoice_update_parameters: None,
// Will be updated in the following call to `Self::get_head`.
forkchoice_update_parameters: ForkchoiceUpdateParameters {
head_hash: None,
finalized_hash: None,
head_root: Hash256::zero(),
},
// Will be updated in the following call to `Self::get_head`.
head_block_root: Hash256::zero(),
_phantom: PhantomData,
})
};
fork_choice.get_head(current_slot, spec)?;
Ok(fork_choice)
}
/// Takes a snapshot of `Self` and stores it in `PersistedForkChoice`, allowing this struct to

View File

@ -1,4 +1,4 @@
use types::{BeaconBlock, BeaconState, Checkpoint, EthSpec, ExecPayload, Hash256, Slot};
use types::{BeaconBlockRef, BeaconState, Checkpoint, EthSpec, ExecPayload, Hash256, Slot};
/// Approximates the `Store` in "Ethereum 2.0 Phase 0 -- Beacon Chain Fork Choice":
///
@ -33,7 +33,7 @@ pub trait ForkChoiceStore<T: EthSpec>: Sized {
/// choice. Allows the implementer to performing caching or other housekeeping duties.
fn on_verified_block<Payload: ExecPayload<T>>(
&mut self,
block: &BeaconBlock<T, Payload>,
block: BeaconBlockRef<T, Payload>,
block_root: Hash256,
state: &BeaconState<T>,
) -> Result<(), Self::Error>;

View File

@ -2,8 +2,9 @@ mod fork_choice;
mod fork_choice_store;
pub use crate::fork_choice::{
AttestationFromBlock, Error, ForkChoice, InvalidAttestation, InvalidBlock,
PayloadVerificationStatus, PersistedForkChoice, QueuedAttestation,
AttestationFromBlock, Error, ForkChoice, ForkChoiceView, ForkchoiceUpdateParameters,
InvalidAttestation, InvalidBlock, PayloadVerificationStatus, PersistedForkChoice,
QueuedAttestation,
};
pub use fork_choice_store::ForkChoiceStore;
pub use proto_array::{Block as ProtoBlock, ExecutionStatus, InvalidationOperation};

View File

@ -16,9 +16,8 @@ use fork_choice::{
};
use store::MemoryStore;
use types::{
test_utils::generate_deterministic_keypair, BeaconBlock, BeaconBlockRef, BeaconState,
ChainSpec, Checkpoint, Epoch, EthSpec, Hash256, IndexedAttestation, MainnetEthSpec, Slot,
SubnetId,
test_utils::generate_deterministic_keypair, BeaconBlockRef, BeaconState, ChainSpec, Checkpoint,
Epoch, EthSpec, Hash256, IndexedAttestation, MainnetEthSpec, SignedBeaconBlock, Slot, SubnetId,
};
pub type E = MainnetEthSpec;
@ -74,7 +73,14 @@ impl ForkChoiceTest {
where
T: Fn(&BeaconForkChoiceStore<E, MemoryStore<E>, MemoryStore<E>>) -> U,
{
func(&self.harness.chain.fork_choice.read().fc_store())
func(
&self
.harness
.chain
.canonical_head
.fork_choice_read_lock()
.fc_store(),
)
}
/// Assert the epochs match.
@ -109,15 +115,7 @@ impl ForkChoiceTest {
/// Assert the given slot is greater than the head slot.
pub fn assert_finalized_epoch_is_less_than(self, epoch: Epoch) -> Self {
assert!(
self.harness
.chain
.head_info()
.unwrap()
.finalized_checkpoint
.epoch
< epoch
);
assert!(self.harness.finalized_checkpoint().epoch < epoch);
self
}
@ -150,11 +148,17 @@ impl ForkChoiceTest {
{
self.harness
.chain
.fork_choice
.write()
.canonical_head
.fork_choice_write_lock()
.update_time(self.harness.chain.slot().unwrap())
.unwrap();
func(self.harness.chain.fork_choice.read().queued_attestations());
func(
self.harness
.chain
.canonical_head
.fork_choice_read_lock()
.queued_attestations(),
);
self
}
@ -173,7 +177,7 @@ impl ForkChoiceTest {
}
/// Build the chain whilst `predicate` returns `true` and `process_block_result` does not error.
pub fn apply_blocks_while<F>(self, mut predicate: F) -> Result<Self, Self>
pub async fn apply_blocks_while<F>(self, mut predicate: F) -> Result<Self, Self>
where
F: FnMut(BeaconBlockRef<'_, E>, &BeaconState<E>) -> bool,
{
@ -182,12 +186,12 @@ impl ForkChoiceTest {
let validators = self.harness.get_all_validators();
loop {
let slot = self.harness.get_current_slot();
let (block, state_) = self.harness.make_block(state, slot);
let (block, state_) = self.harness.make_block(state, slot).await;
state = state_;
if !predicate(block.message(), &state) {
break;
}
if let Ok(block_hash) = self.harness.process_block_result(block.clone()) {
if let Ok(block_hash) = self.harness.process_block_result(block.clone()).await {
self.harness.attest_block(
&state,
block.state_root(),
@ -205,25 +209,29 @@ impl ForkChoiceTest {
}
/// Apply `count` blocks to the chain (with attestations).
pub fn apply_blocks(self, count: usize) -> Self {
pub async fn apply_blocks(self, count: usize) -> Self {
self.harness.advance_slot();
self.harness.extend_chain(
count,
BlockStrategy::OnCanonicalHead,
AttestationStrategy::AllValidators,
);
self.harness
.extend_chain(
count,
BlockStrategy::OnCanonicalHead,
AttestationStrategy::AllValidators,
)
.await;
self
}
/// Apply `count` blocks to the chain (without attestations).
pub fn apply_blocks_without_new_attestations(self, count: usize) -> Self {
pub async fn apply_blocks_without_new_attestations(self, count: usize) -> Self {
self.harness.advance_slot();
self.harness.extend_chain(
count,
BlockStrategy::OnCanonicalHead,
AttestationStrategy::SomeValidators(vec![]),
);
self.harness
.extend_chain(
count,
BlockStrategy::OnCanonicalHead,
AttestationStrategy::SomeValidators(vec![]),
)
.await;
self
}
@ -256,9 +264,9 @@ impl ForkChoiceTest {
/// Applies a block directly to fork choice, bypassing the beacon chain.
///
/// Asserts the block was applied successfully.
pub fn apply_block_directly_to_fork_choice<F>(self, mut func: F) -> Self
pub async fn apply_block_directly_to_fork_choice<F>(self, mut func: F) -> Self
where
F: FnMut(&mut BeaconBlock<E>, &mut BeaconState<E>),
F: FnMut(&mut SignedBeaconBlock<E>, &mut BeaconState<E>),
{
let state = self
.harness
@ -269,18 +277,17 @@ impl ForkChoiceTest {
)
.unwrap();
let slot = self.harness.get_current_slot();
let (signed_block, mut state) = self.harness.make_block(state, slot);
let (mut block, _) = signed_block.deconstruct();
func(&mut block, &mut state);
let (mut signed_block, mut state) = self.harness.make_block(state, slot).await;
func(&mut signed_block, &mut state);
let current_slot = self.harness.get_current_slot();
self.harness
.chain
.fork_choice
.write()
.canonical_head
.fork_choice_write_lock()
.on_block(
current_slot,
&block,
block.canonical_root(),
signed_block.message(),
signed_block.canonical_root(),
Duration::from_secs(0),
&state,
PayloadVerificationStatus::Verified,
@ -293,13 +300,13 @@ impl ForkChoiceTest {
/// Applies a block directly to fork choice, bypassing the beacon chain.
///
/// Asserts that an error occurred and allows inspecting it via `comparison_func`.
pub fn apply_invalid_block_directly_to_fork_choice<F, G>(
pub async fn apply_invalid_block_directly_to_fork_choice<F, G>(
self,
mut mutation_func: F,
mut comparison_func: G,
) -> Self
where
F: FnMut(&mut BeaconBlock<E>, &mut BeaconState<E>),
F: FnMut(&mut SignedBeaconBlock<E>, &mut BeaconState<E>),
G: FnMut(ForkChoiceError),
{
let state = self
@ -311,19 +318,18 @@ impl ForkChoiceTest {
)
.unwrap();
let slot = self.harness.get_current_slot();
let (signed_block, mut state) = self.harness.make_block(state, slot);
let (mut block, _) = signed_block.deconstruct();
mutation_func(&mut block, &mut state);
let (mut signed_block, mut state) = self.harness.make_block(state, slot).await;
mutation_func(&mut signed_block, &mut state);
let current_slot = self.harness.get_current_slot();
let err = self
.harness
.chain
.fork_choice
.write()
.canonical_head
.fork_choice_write_lock()
.on_block(
current_slot,
&block,
block.canonical_root(),
signed_block.message(),
signed_block.canonical_root(),
Duration::from_secs(0),
&state,
PayloadVerificationStatus::Verified,
@ -339,7 +345,7 @@ impl ForkChoiceTest {
/// database.
fn check_justified_balances(&self) {
let harness = &self.harness;
let fc = self.harness.chain.fork_choice.read();
let fc = self.harness.chain.canonical_head.fork_choice_read_lock();
let state_root = harness
.chain
@ -377,7 +383,7 @@ impl ForkChoiceTest {
/// Returns an attestation that is valid for some slot in the given `chain`.
///
/// Also returns some info about who created it.
fn apply_attestation_to_chain<F, G>(
async fn apply_attestation_to_chain<F, G>(
self,
delay: MutationDelay,
mut mutation_func: F,
@ -387,7 +393,7 @@ impl ForkChoiceTest {
F: FnMut(&mut IndexedAttestation<E>, &BeaconChain<EphemeralHarnessType<E>>),
G: FnMut(Result<(), BeaconChainError>),
{
let head = self.harness.chain.head().expect("should get head");
let head = self.harness.chain.head_snapshot();
let current_slot = self.harness.chain.slot().expect("should get slot");
let mut attestation = self
@ -438,11 +444,13 @@ impl ForkChoiceTest {
if let MutationDelay::Blocks(slots) = delay {
self.harness.advance_slot();
self.harness.extend_chain(
slots,
BlockStrategy::OnCanonicalHead,
AttestationStrategy::SomeValidators(vec![]),
);
self.harness
.extend_chain(
slots,
BlockStrategy::OnCanonicalHead,
AttestationStrategy::SomeValidators(vec![]),
)
.await;
}
mutation_func(
@ -464,17 +472,9 @@ impl ForkChoiceTest {
pub fn check_finalized_block_is_accessible(self) -> Self {
self.harness
.chain
.fork_choice
.write()
.get_block(
&self
.harness
.chain
.head_info()
.unwrap()
.finalized_checkpoint
.root,
)
.canonical_head
.fork_choice_read_lock()
.get_block(&self.harness.finalized_checkpoint().root)
.unwrap();
self
@ -488,7 +488,7 @@ fn is_safe_to_update(slot: Slot, spec: &ChainSpec) -> bool {
#[test]
fn justified_and_finalized_blocks() {
let tester = ForkChoiceTest::new();
let fork_choice = tester.harness.chain.fork_choice.read();
let fork_choice = tester.harness.chain.canonical_head.fork_choice_read_lock();
let justified_checkpoint = fork_choice.justified_checkpoint();
assert_eq!(justified_checkpoint.epoch, 0);
@ -503,44 +503,50 @@ fn justified_and_finalized_blocks() {
/// - The new justified checkpoint descends from the current.
/// - Current slot is within `SAFE_SLOTS_TO_UPDATE_JUSTIFIED`
#[test]
fn justified_checkpoint_updates_with_descendent_inside_safe_slots() {
#[tokio::test]
async fn justified_checkpoint_updates_with_descendent_inside_safe_slots() {
ForkChoiceTest::new()
.apply_blocks_while(|_, state| state.current_justified_checkpoint().epoch == 0)
.await
.unwrap()
.move_inside_safe_to_update()
.assert_justified_epoch(0)
.apply_blocks(1)
.await
.assert_justified_epoch(2);
}
/// - The new justified checkpoint descends from the current.
/// - Current slot is **not** within `SAFE_SLOTS_TO_UPDATE_JUSTIFIED`
/// - This is **not** the first justification since genesis
#[test]
fn justified_checkpoint_updates_with_descendent_outside_safe_slots() {
#[tokio::test]
async fn justified_checkpoint_updates_with_descendent_outside_safe_slots() {
ForkChoiceTest::new()
.apply_blocks_while(|_, state| state.current_justified_checkpoint().epoch <= 2)
.await
.unwrap()
.move_outside_safe_to_update()
.assert_justified_epoch(2)
.assert_best_justified_epoch(2)
.apply_blocks(1)
.await
.assert_justified_epoch(3);
}
/// - The new justified checkpoint descends from the current.
/// - Current slot is **not** within `SAFE_SLOTS_TO_UPDATE_JUSTIFIED`
/// - This is the first justification since genesis
#[test]
fn justified_checkpoint_updates_first_justification_outside_safe_to_update() {
#[tokio::test]
async fn justified_checkpoint_updates_first_justification_outside_safe_to_update() {
ForkChoiceTest::new()
.apply_blocks_while(|_, state| state.current_justified_checkpoint().epoch == 0)
.await
.unwrap()
.move_to_next_unsafe_period()
.assert_justified_epoch(0)
.assert_best_justified_epoch(0)
.apply_blocks(1)
.await
.assert_justified_epoch(2)
.assert_best_justified_epoch(2);
}
@ -548,12 +554,14 @@ fn justified_checkpoint_updates_first_justification_outside_safe_to_update() {
/// - The new justified checkpoint **does not** descend from the current.
/// - Current slot is within `SAFE_SLOTS_TO_UPDATE_JUSTIFIED`
/// - Finalized epoch has **not** increased.
#[test]
fn justified_checkpoint_updates_with_non_descendent_inside_safe_slots_without_finality() {
#[tokio::test]
async fn justified_checkpoint_updates_with_non_descendent_inside_safe_slots_without_finality() {
ForkChoiceTest::new()
.apply_blocks_while(|_, state| state.current_justified_checkpoint().epoch == 0)
.await
.unwrap()
.apply_blocks(1)
.await
.move_inside_safe_to_update()
.assert_justified_epoch(2)
.apply_block_directly_to_fork_choice(|_, state| {
@ -567,6 +575,7 @@ fn justified_checkpoint_updates_with_non_descendent_inside_safe_slots_without_fi
.get_block_root(Epoch::new(1).start_slot(E::slots_per_epoch()))
.unwrap();
})
.await
.assert_justified_epoch(3)
.assert_best_justified_epoch(3);
}
@ -574,12 +583,14 @@ fn justified_checkpoint_updates_with_non_descendent_inside_safe_slots_without_fi
/// - The new justified checkpoint **does not** descend from the current.
/// - Current slot is **not** within `SAFE_SLOTS_TO_UPDATE_JUSTIFIED`.
/// - Finalized epoch has **not** increased.
#[test]
fn justified_checkpoint_updates_with_non_descendent_outside_safe_slots_without_finality() {
#[tokio::test]
async fn justified_checkpoint_updates_with_non_descendent_outside_safe_slots_without_finality() {
ForkChoiceTest::new()
.apply_blocks_while(|_, state| state.current_justified_checkpoint().epoch == 0)
.await
.unwrap()
.apply_blocks(1)
.await
.move_to_next_unsafe_period()
.assert_justified_epoch(2)
.apply_block_directly_to_fork_choice(|_, state| {
@ -593,6 +604,7 @@ fn justified_checkpoint_updates_with_non_descendent_outside_safe_slots_without_f
.get_block_root(Epoch::new(1).start_slot(E::slots_per_epoch()))
.unwrap();
})
.await
.assert_justified_epoch(2)
.assert_best_justified_epoch(3);
}
@ -600,12 +612,14 @@ fn justified_checkpoint_updates_with_non_descendent_outside_safe_slots_without_f
/// - The new justified checkpoint **does not** descend from the current.
/// - Current slot is **not** within `SAFE_SLOTS_TO_UPDATE_JUSTIFIED`
/// - Finalized epoch has increased.
#[test]
fn justified_checkpoint_updates_with_non_descendent_outside_safe_slots_with_finality() {
#[tokio::test]
async fn justified_checkpoint_updates_with_non_descendent_outside_safe_slots_with_finality() {
ForkChoiceTest::new()
.apply_blocks_while(|_, state| state.current_justified_checkpoint().epoch == 0)
.await
.unwrap()
.apply_blocks(1)
.await
.move_to_next_unsafe_period()
.assert_justified_epoch(2)
.apply_block_directly_to_fork_choice(|_, state| {
@ -619,17 +633,20 @@ fn justified_checkpoint_updates_with_non_descendent_outside_safe_slots_with_fina
.get_block_root(Epoch::new(1).start_slot(E::slots_per_epoch()))
.unwrap();
})
.await
.assert_justified_epoch(3)
.assert_best_justified_epoch(3);
}
/// Check that the balances are obtained correctly.
#[test]
fn justified_balances() {
#[tokio::test]
async fn justified_balances() {
ForkChoiceTest::new()
.apply_blocks_while(|_, state| state.current_justified_checkpoint().epoch == 0)
.await
.unwrap()
.apply_blocks(1)
.await
.assert_justified_epoch(2)
.check_justified_balances()
}
@ -648,15 +665,16 @@ macro_rules! assert_invalid_block {
/// Specification v0.12.1
///
/// assert block.parent_root in store.block_states
#[test]
fn invalid_block_unknown_parent() {
#[tokio::test]
async fn invalid_block_unknown_parent() {
let junk = Hash256::from_low_u64_be(42);
ForkChoiceTest::new()
.apply_blocks(2)
.await
.apply_invalid_block_directly_to_fork_choice(
|block, _| {
*block.parent_root_mut() = junk;
*block.message_mut().parent_root_mut() = junk;
},
|err| {
assert_invalid_block!(
@ -665,36 +683,42 @@ fn invalid_block_unknown_parent() {
if parent == junk
)
},
);
)
.await;
}
/// Specification v0.12.1
///
/// assert get_current_slot(store) >= block.slot
#[test]
fn invalid_block_future_slot() {
#[tokio::test]
async fn invalid_block_future_slot() {
ForkChoiceTest::new()
.apply_blocks(2)
.await
.apply_invalid_block_directly_to_fork_choice(
|block, _| {
*block.slot_mut() += 1;
*block.message_mut().slot_mut() += 1;
},
|err| assert_invalid_block!(err, InvalidBlock::FutureSlot { .. }),
);
)
.await;
}
/// Specification v0.12.1
///
/// assert block.slot > finalized_slot
#[test]
fn invalid_block_finalized_slot() {
#[tokio::test]
async fn invalid_block_finalized_slot() {
ForkChoiceTest::new()
.apply_blocks_while(|_, state| state.finalized_checkpoint().epoch == 0)
.await
.unwrap()
.apply_blocks(1)
.await
.apply_invalid_block_directly_to_fork_choice(
|block, _| {
*block.slot_mut() = Epoch::new(2).start_slot(E::slots_per_epoch()) - 1;
*block.message_mut().slot_mut() =
Epoch::new(2).start_slot(E::slots_per_epoch()) - 1;
},
|err| {
assert_invalid_block!(
@ -703,7 +727,8 @@ fn invalid_block_finalized_slot() {
if finalized_slot == Epoch::new(2).start_slot(E::slots_per_epoch())
)
},
);
)
.await;
}
/// Specification v0.12.1
@ -714,18 +739,20 @@ fn invalid_block_finalized_slot() {
/// Note: we technically don't do this exact check, but an equivalent check. Reference:
///
/// https://github.com/ethereum/eth2.0-specs/pull/1884
#[test]
fn invalid_block_finalized_descendant() {
#[tokio::test]
async fn invalid_block_finalized_descendant() {
let invalid_ancestor = Mutex::new(Hash256::zero());
ForkChoiceTest::new()
.apply_blocks_while(|_, state| state.finalized_checkpoint().epoch == 0)
.await
.unwrap()
.apply_blocks(1)
.await
.assert_finalized_epoch(2)
.apply_invalid_block_directly_to_fork_choice(
|block, state| {
*block.parent_root_mut() = *state
*block.message_mut().parent_root_mut() = *state
.get_block_root(Epoch::new(1).start_slot(E::slots_per_epoch()))
.unwrap();
*invalid_ancestor.lock().unwrap() = block.parent_root();
@ -737,7 +764,8 @@ fn invalid_block_finalized_descendant() {
if block_ancestor == Some(*invalid_ancestor.lock().unwrap())
)
},
);
)
.await;
}
macro_rules! assert_invalid_attestation {
@ -754,23 +782,26 @@ macro_rules! assert_invalid_attestation {
}
/// Ensure we can process a valid attestation.
#[test]
fn valid_attestation() {
#[tokio::test]
async fn valid_attestation() {
ForkChoiceTest::new()
.apply_blocks_without_new_attestations(1)
.await
.apply_attestation_to_chain(
MutationDelay::NoDelay,
|_, _| {},
|result| assert_eq!(result.unwrap(), ()),
);
)
.await;
}
/// This test is not in the specification, however we reject an attestation with an empty
/// aggregation bitfield since it has no purpose beyond wasting our time.
#[test]
fn invalid_attestation_empty_bitfield() {
#[tokio::test]
async fn invalid_attestation_empty_bitfield() {
ForkChoiceTest::new()
.apply_blocks_without_new_attestations(1)
.await
.apply_attestation_to_chain(
MutationDelay::NoDelay,
|attestation, _| {
@ -779,7 +810,8 @@ fn invalid_attestation_empty_bitfield() {
|result| {
assert_invalid_attestation!(result, InvalidAttestation::EmptyAggregationBitfield)
},
);
)
.await;
}
/// Specification v0.12.1:
@ -787,10 +819,11 @@ fn invalid_attestation_empty_bitfield() {
/// assert target.epoch in [expected_current_epoch, previous_epoch]
///
/// (tests epoch after current epoch)
#[test]
fn invalid_attestation_future_epoch() {
#[tokio::test]
async fn invalid_attestation_future_epoch() {
ForkChoiceTest::new()
.apply_blocks_without_new_attestations(1)
.await
.apply_attestation_to_chain(
MutationDelay::NoDelay,
|attestation, _| {
@ -803,7 +836,8 @@ fn invalid_attestation_future_epoch() {
if attestation_epoch == Epoch::new(2) && current_epoch == Epoch::new(0)
)
},
);
)
.await;
}
/// Specification v0.12.1:
@ -811,10 +845,11 @@ fn invalid_attestation_future_epoch() {
/// assert target.epoch in [expected_current_epoch, previous_epoch]
///
/// (tests epoch prior to previous epoch)
#[test]
fn invalid_attestation_past_epoch() {
#[tokio::test]
async fn invalid_attestation_past_epoch() {
ForkChoiceTest::new()
.apply_blocks_without_new_attestations(E::slots_per_epoch() as usize * 3 + 1)
.await
.apply_attestation_to_chain(
MutationDelay::NoDelay,
|attestation, _| {
@ -827,16 +862,18 @@ fn invalid_attestation_past_epoch() {
if attestation_epoch == Epoch::new(0) && current_epoch == Epoch::new(3)
)
},
);
)
.await;
}
/// Specification v0.12.1:
///
/// assert target.epoch == compute_epoch_at_slot(attestation.data.slot)
#[test]
fn invalid_attestation_target_epoch() {
#[tokio::test]
async fn invalid_attestation_target_epoch() {
ForkChoiceTest::new()
.apply_blocks_without_new_attestations(E::slots_per_epoch() as usize + 1)
.await
.apply_attestation_to_chain(
MutationDelay::NoDelay,
|attestation, _| {
@ -849,18 +886,20 @@ fn invalid_attestation_target_epoch() {
if target == Epoch::new(1) && slot == Slot::new(1)
)
},
);
)
.await;
}
/// Specification v0.12.1:
///
/// assert target.root in store.blocks
#[test]
fn invalid_attestation_unknown_target_root() {
#[tokio::test]
async fn invalid_attestation_unknown_target_root() {
let junk = Hash256::from_low_u64_be(42);
ForkChoiceTest::new()
.apply_blocks_without_new_attestations(1)
.await
.apply_attestation_to_chain(
MutationDelay::NoDelay,
|attestation, _| {
@ -873,18 +912,20 @@ fn invalid_attestation_unknown_target_root() {
if root == junk
)
},
);
)
.await;
}
/// Specification v0.12.1:
///
/// assert attestation.data.beacon_block_root in store.blocks
#[test]
fn invalid_attestation_unknown_beacon_block_root() {
#[tokio::test]
async fn invalid_attestation_unknown_beacon_block_root() {
let junk = Hash256::from_low_u64_be(42);
ForkChoiceTest::new()
.apply_blocks_without_new_attestations(1)
.await
.apply_attestation_to_chain(
MutationDelay::NoDelay,
|attestation, _| {
@ -897,16 +938,18 @@ fn invalid_attestation_unknown_beacon_block_root() {
if beacon_block_root == junk
)
},
);
)
.await;
}
/// Specification v0.12.1:
///
/// assert store.blocks[attestation.data.beacon_block_root].slot <= attestation.data.slot
#[test]
fn invalid_attestation_future_block() {
#[tokio::test]
async fn invalid_attestation_future_block() {
ForkChoiceTest::new()
.apply_blocks_without_new_attestations(1)
.await
.apply_attestation_to_chain(
MutationDelay::Blocks(1),
|attestation, chain| {
@ -923,19 +966,21 @@ fn invalid_attestation_future_block() {
if block == 2 && attestation == 1
)
},
);
)
.await;
}
/// Specification v0.12.1:
///
/// assert target.root == get_ancestor(store, attestation.data.beacon_block_root, target_slot)
#[test]
fn invalid_attestation_inconsistent_ffg_vote() {
#[tokio::test]
async fn invalid_attestation_inconsistent_ffg_vote() {
let local_opt = Mutex::new(None);
let attestation_opt = Mutex::new(None);
ForkChoiceTest::new()
.apply_blocks_without_new_attestations(1)
.await
.apply_attestation_to_chain(
MutationDelay::NoDelay,
|attestation, chain| {
@ -962,22 +1007,25 @@ fn invalid_attestation_inconsistent_ffg_vote() {
&& local == local_opt.lock().unwrap().unwrap()
)
},
);
)
.await;
}
/// Specification v0.12.1:
///
/// assert get_current_slot(store) >= attestation.data.slot + 1
#[test]
fn invalid_attestation_delayed_slot() {
#[tokio::test]
async fn invalid_attestation_delayed_slot() {
ForkChoiceTest::new()
.apply_blocks_without_new_attestations(1)
.await
.inspect_queued_attestations(|queue| assert_eq!(queue.len(), 0))
.apply_attestation_to_chain(
MutationDelay::NoDelay,
|_, _| {},
|result| assert_eq!(result.unwrap(), ()),
)
.await
.inspect_queued_attestations(|queue| assert_eq!(queue.len(), 1))
.skip_slot()
.inspect_queued_attestations(|queue| assert_eq!(queue.len(), 0));
@ -985,10 +1033,11 @@ fn invalid_attestation_delayed_slot() {
/// Tests that the correct target root is used when the attested-to block is in a prior epoch to
/// the attestation.
#[test]
fn valid_attestation_skip_across_epoch() {
#[tokio::test]
async fn valid_attestation_skip_across_epoch() {
ForkChoiceTest::new()
.apply_blocks(E::slots_per_epoch() as usize - 1)
.await
.skip_slots(2)
.apply_attestation_to_chain(
MutationDelay::NoDelay,
@ -999,15 +1048,18 @@ fn valid_attestation_skip_across_epoch() {
)
},
|result| result.unwrap(),
);
)
.await;
}
#[test]
fn can_read_finalized_block() {
#[tokio::test]
async fn can_read_finalized_block() {
ForkChoiceTest::new()
.apply_blocks_while(|_, state| state.finalized_checkpoint().epoch == 0)
.await
.unwrap()
.apply_blocks(1)
.await
.check_finalized_block_is_accessible();
}
@ -1025,8 +1077,8 @@ fn weak_subjectivity_fail_on_startup() {
ForkChoiceTest::new_with_chain_config(chain_config);
}
#[test]
fn weak_subjectivity_pass_on_startup() {
#[tokio::test]
async fn weak_subjectivity_pass_on_startup() {
let epoch = Epoch::new(0);
let root = Hash256::zero();
@ -1037,23 +1089,21 @@ fn weak_subjectivity_pass_on_startup() {
ForkChoiceTest::new_with_chain_config(chain_config)
.apply_blocks(E::slots_per_epoch() as usize)
.await
.assert_shutdown_signal_not_sent();
}
#[test]
fn weak_subjectivity_check_passes() {
#[tokio::test]
async fn weak_subjectivity_check_passes() {
let setup_harness = ForkChoiceTest::new()
.apply_blocks_while(|_, state| state.finalized_checkpoint().epoch == 0)
.await
.unwrap()
.apply_blocks(1)
.await
.assert_finalized_epoch(2);
let checkpoint = setup_harness
.harness
.chain
.head_info()
.unwrap()
.finalized_checkpoint;
let checkpoint = setup_harness.harness.finalized_checkpoint();
let chain_config = ChainConfig {
weak_subjectivity_checkpoint: Some(checkpoint),
@ -1062,26 +1112,25 @@ fn weak_subjectivity_check_passes() {
ForkChoiceTest::new_with_chain_config(chain_config.clone())
.apply_blocks_while(|_, state| state.finalized_checkpoint().epoch == 0)
.await
.unwrap()
.apply_blocks(1)
.await
.assert_finalized_epoch(2)
.assert_shutdown_signal_not_sent();
}
#[test]
fn weak_subjectivity_check_fails_early_epoch() {
#[tokio::test]
async fn weak_subjectivity_check_fails_early_epoch() {
let setup_harness = ForkChoiceTest::new()
.apply_blocks_while(|_, state| state.finalized_checkpoint().epoch == 0)
.await
.unwrap()
.apply_blocks(1)
.await
.assert_finalized_epoch(2);
let mut checkpoint = setup_harness
.harness
.chain
.head_info()
.unwrap()
.finalized_checkpoint;
let mut checkpoint = setup_harness.harness.finalized_checkpoint();
checkpoint.epoch = checkpoint.epoch - 1;
@ -1092,25 +1141,23 @@ fn weak_subjectivity_check_fails_early_epoch() {
ForkChoiceTest::new_with_chain_config(chain_config.clone())
.apply_blocks_while(|_, state| state.finalized_checkpoint().epoch < 3)
.await
.unwrap_err()
.assert_finalized_epoch_is_less_than(checkpoint.epoch)
.assert_shutdown_signal_sent();
}
#[test]
fn weak_subjectivity_check_fails_late_epoch() {
#[tokio::test]
async fn weak_subjectivity_check_fails_late_epoch() {
let setup_harness = ForkChoiceTest::new()
.apply_blocks_while(|_, state| state.finalized_checkpoint().epoch == 0)
.await
.unwrap()
.apply_blocks(1)
.await
.assert_finalized_epoch(2);
let mut checkpoint = setup_harness
.harness
.chain
.head_info()
.unwrap()
.finalized_checkpoint;
let mut checkpoint = setup_harness.harness.finalized_checkpoint();
checkpoint.epoch = checkpoint.epoch + 1;
@ -1121,25 +1168,23 @@ fn weak_subjectivity_check_fails_late_epoch() {
ForkChoiceTest::new_with_chain_config(chain_config.clone())
.apply_blocks_while(|_, state| state.finalized_checkpoint().epoch < 4)
.await
.unwrap_err()
.assert_finalized_epoch_is_less_than(checkpoint.epoch)
.assert_shutdown_signal_sent();
}
#[test]
fn weak_subjectivity_check_fails_incorrect_root() {
#[tokio::test]
async fn weak_subjectivity_check_fails_incorrect_root() {
let setup_harness = ForkChoiceTest::new()
.apply_blocks_while(|_, state| state.finalized_checkpoint().epoch == 0)
.await
.unwrap()
.apply_blocks(1)
.await
.assert_finalized_epoch(2);
let mut checkpoint = setup_harness
.harness
.chain
.head_info()
.unwrap()
.finalized_checkpoint;
let mut checkpoint = setup_harness.harness.finalized_checkpoint();
checkpoint.root = Hash256::zero();
@ -1150,27 +1195,31 @@ fn weak_subjectivity_check_fails_incorrect_root() {
ForkChoiceTest::new_with_chain_config(chain_config.clone())
.apply_blocks_while(|_, state| state.finalized_checkpoint().epoch < 3)
.await
.unwrap_err()
.assert_finalized_epoch_is_less_than(checkpoint.epoch)
.assert_shutdown_signal_sent();
}
#[test]
fn weak_subjectivity_check_epoch_boundary_is_skip_slot() {
#[tokio::test]
async fn weak_subjectivity_check_epoch_boundary_is_skip_slot() {
let setup_harness = ForkChoiceTest::new()
// first two epochs
.apply_blocks_while(|_, state| state.finalized_checkpoint().epoch == 0)
.await
.unwrap();
// get the head, it will become the finalized root of epoch 4
let checkpoint_root = setup_harness.harness.chain.head_info().unwrap().block_root;
let checkpoint_root = setup_harness.harness.head_block_root();
setup_harness
// epoch 3 will be entirely skip slots
.skip_slots(E::slots_per_epoch() as usize)
.apply_blocks_while(|_, state| state.finalized_checkpoint().epoch < 5)
.await
.unwrap()
.apply_blocks(1)
.await
.assert_finalized_epoch(5);
// the checkpoint at epoch 4 should become the root of last block of epoch 2
@ -1187,31 +1236,37 @@ fn weak_subjectivity_check_epoch_boundary_is_skip_slot() {
// recreate the chain exactly
ForkChoiceTest::new_with_chain_config(chain_config.clone())
.apply_blocks_while(|_, state| state.finalized_checkpoint().epoch == 0)
.await
.unwrap()
.skip_slots(E::slots_per_epoch() as usize)
.apply_blocks_while(|_, state| state.finalized_checkpoint().epoch < 5)
.await
.unwrap()
.apply_blocks(1)
.await
.assert_finalized_epoch(5)
.assert_shutdown_signal_not_sent();
}
#[test]
fn weak_subjectivity_check_epoch_boundary_is_skip_slot_failure() {
#[tokio::test]
async fn weak_subjectivity_check_epoch_boundary_is_skip_slot_failure() {
let setup_harness = ForkChoiceTest::new()
// first two epochs
.apply_blocks_while(|_, state| state.finalized_checkpoint().epoch == 0)
.await
.unwrap();
// get the head, it will become the finalized root of epoch 4
let checkpoint_root = setup_harness.harness.chain.head_info().unwrap().block_root;
let checkpoint_root = setup_harness.harness.head_block_root();
setup_harness
// epoch 3 will be entirely skip slots
.skip_slots(E::slots_per_epoch() as usize)
.apply_blocks_while(|_, state| state.finalized_checkpoint().epoch < 5)
.await
.unwrap()
.apply_blocks(1)
.await
.assert_finalized_epoch(5);
// Invalid checkpoint (epoch too early)
@ -1228,9 +1283,11 @@ fn weak_subjectivity_check_epoch_boundary_is_skip_slot_failure() {
// recreate the chain exactly
ForkChoiceTest::new_with_chain_config(chain_config.clone())
.apply_blocks_while(|_, state| state.finalized_checkpoint().epoch == 0)
.await
.unwrap()
.skip_slots(E::slots_per_epoch() as usize)
.apply_blocks_while(|_, state| state.finalized_checkpoint().epoch < 6)
.await
.unwrap_err()
.assert_finalized_epoch_is_less_than(checkpoint.epoch)
.assert_shutdown_signal_sent();

View File

@ -16,6 +16,7 @@ four_byte_option_impl!(four_byte_option_usize, usize);
four_byte_option_impl!(four_byte_option_checkpoint, Checkpoint);
/// Defines an operation which may invalidate the `execution_status` of some nodes.
#[derive(Clone)]
pub enum InvalidationOperation {
/// Invalidate only `block_root` and it's descendants. Don't invalidate any ancestors.
InvalidateOne { block_root: Hash256 },

View File

@ -7,6 +7,7 @@ edition = "2021"
[dev-dependencies]
env_logger = "0.9.0"
beacon_chain = { path = "../../beacon_node/beacon_chain" }
tokio = { version = "1.14.0", features = ["rt-multi-thread"] }
[dependencies]
bls = { path = "../../crypto/bls" }

View File

@ -27,7 +27,7 @@ lazy_static! {
static ref KEYPAIRS: Vec<Keypair> = generate_deterministic_keypairs(MAX_VALIDATOR_COUNT);
}
fn get_harness<E: EthSpec>(
async fn get_harness<E: EthSpec>(
epoch_offset: u64,
num_validators: usize,
) -> BeaconChainHarness<EphemeralHarnessType<E>> {
@ -41,27 +41,31 @@ fn get_harness<E: EthSpec>(
.build();
let state = harness.get_current_state();
if last_slot_of_epoch > Slot::new(0) {
harness.add_attested_blocks_at_slots(
state,
Hash256::zero(),
(1..last_slot_of_epoch.as_u64())
.map(Slot::new)
.collect::<Vec<_>>()
.as_slice(),
(0..num_validators).collect::<Vec<_>>().as_slice(),
);
harness
.add_attested_blocks_at_slots(
state,
Hash256::zero(),
(1..last_slot_of_epoch.as_u64())
.map(Slot::new)
.collect::<Vec<_>>()
.as_slice(),
(0..num_validators).collect::<Vec<_>>().as_slice(),
)
.await;
}
harness
}
#[test]
fn valid_block_ok() {
#[tokio::test]
async fn valid_block_ok() {
let spec = MainnetEthSpec::default_spec();
let harness = get_harness::<MainnetEthSpec>(EPOCH_OFFSET, VALIDATOR_COUNT);
let harness = get_harness::<MainnetEthSpec>(EPOCH_OFFSET, VALIDATOR_COUNT).await;
let state = harness.get_current_state();
let slot = state.slot();
let (block, mut state) = harness.make_block_return_pre_state(state, slot + Slot::new(1));
let (block, mut state) = harness
.make_block_return_pre_state(state, slot + Slot::new(1))
.await;
let result = per_block_processing(
&mut state,
@ -75,15 +79,15 @@ fn valid_block_ok() {
assert!(result.is_ok());
}
#[test]
fn invalid_block_header_state_slot() {
#[tokio::test]
async fn invalid_block_header_state_slot() {
let spec = MainnetEthSpec::default_spec();
let harness = get_harness::<MainnetEthSpec>(EPOCH_OFFSET, VALIDATOR_COUNT);
let harness = get_harness::<MainnetEthSpec>(EPOCH_OFFSET, VALIDATOR_COUNT).await;
let state = harness.get_current_state();
let slot = state.slot() + Slot::new(1);
let (signed_block, mut state) = harness.make_block_return_pre_state(state, slot);
let (signed_block, mut state) = harness.make_block_return_pre_state(state, slot).await;
let (mut block, signature) = signed_block.deconstruct();
*block.slot_mut() = slot + Slot::new(1);
@ -104,15 +108,17 @@ fn invalid_block_header_state_slot() {
);
}
#[test]
fn invalid_parent_block_root() {
#[tokio::test]
async fn invalid_parent_block_root() {
let spec = MainnetEthSpec::default_spec();
let harness = get_harness::<MainnetEthSpec>(EPOCH_OFFSET, VALIDATOR_COUNT);
let harness = get_harness::<MainnetEthSpec>(EPOCH_OFFSET, VALIDATOR_COUNT).await;
let state = harness.get_current_state();
let slot = state.slot();
let (signed_block, mut state) = harness.make_block_return_pre_state(state, slot + Slot::new(1));
let (signed_block, mut state) = harness
.make_block_return_pre_state(state, slot + Slot::new(1))
.await;
let (mut block, signature) = signed_block.deconstruct();
*block.parent_root_mut() = Hash256::from([0xAA; 32]);
@ -136,14 +142,16 @@ fn invalid_parent_block_root() {
);
}
#[test]
fn invalid_block_signature() {
#[tokio::test]
async fn invalid_block_signature() {
let spec = MainnetEthSpec::default_spec();
let harness = get_harness::<MainnetEthSpec>(EPOCH_OFFSET, VALIDATOR_COUNT);
let harness = get_harness::<MainnetEthSpec>(EPOCH_OFFSET, VALIDATOR_COUNT).await;
let state = harness.get_current_state();
let slot = state.slot();
let (signed_block, mut state) = harness.make_block_return_pre_state(state, slot + Slot::new(1));
let (signed_block, mut state) = harness
.make_block_return_pre_state(state, slot + Slot::new(1))
.await;
let (block, _) = signed_block.deconstruct();
let result = per_block_processing(
@ -164,17 +172,19 @@ fn invalid_block_signature() {
);
}
#[test]
fn invalid_randao_reveal_signature() {
#[tokio::test]
async fn invalid_randao_reveal_signature() {
let spec = MainnetEthSpec::default_spec();
let harness = get_harness::<MainnetEthSpec>(EPOCH_OFFSET, VALIDATOR_COUNT);
let harness = get_harness::<MainnetEthSpec>(EPOCH_OFFSET, VALIDATOR_COUNT).await;
let state = harness.get_current_state();
let slot = state.slot();
let (signed_block, mut state) = harness.make_block_with_modifier(state, slot + 1, |block| {
*block.body_mut().randao_reveal_mut() = Signature::empty();
});
let (signed_block, mut state) = harness
.make_block_with_modifier(state, slot + 1, |block| {
*block.body_mut().randao_reveal_mut() = Signature::empty();
})
.await;
let result = per_block_processing(
&mut state,
@ -189,16 +199,22 @@ fn invalid_randao_reveal_signature() {
assert_eq!(result, Err(BlockProcessingError::RandaoSignatureInvalid));
}
#[test]
fn valid_4_deposits() {
#[tokio::test]
async fn valid_4_deposits() {
let spec = MainnetEthSpec::default_spec();
let harness = get_harness::<MainnetEthSpec>(EPOCH_OFFSET, VALIDATOR_COUNT);
let harness = get_harness::<MainnetEthSpec>(EPOCH_OFFSET, VALIDATOR_COUNT).await;
let mut state = harness.get_current_state();
let (deposits, state) = harness.make_deposits(&mut state, 4, None, None);
let deposits = VariableList::from(deposits);
let mut head_block = harness.chain.head_beacon_block().unwrap().deconstruct().0;
let mut head_block = harness
.chain
.head_beacon_block()
.as_ref()
.clone()
.deconstruct()
.0;
*head_block.to_mut().body_mut().deposits_mut() = deposits;
let result = process_operations::process_deposits(state, head_block.body().deposits(), &spec);
@ -207,16 +223,22 @@ fn valid_4_deposits() {
assert_eq!(result, Ok(()));
}
#[test]
fn invalid_deposit_deposit_count_too_big() {
#[tokio::test]
async fn invalid_deposit_deposit_count_too_big() {
let spec = MainnetEthSpec::default_spec();
let harness = get_harness::<MainnetEthSpec>(EPOCH_OFFSET, VALIDATOR_COUNT);
let harness = get_harness::<MainnetEthSpec>(EPOCH_OFFSET, VALIDATOR_COUNT).await;
let mut state = harness.get_current_state();
let (deposits, state) = harness.make_deposits(&mut state, 1, None, None);
let deposits = VariableList::from(deposits);
let mut head_block = harness.chain.head_beacon_block().unwrap().deconstruct().0;
let mut head_block = harness
.chain
.head_beacon_block()
.as_ref()
.clone()
.deconstruct()
.0;
*head_block.to_mut().body_mut().deposits_mut() = deposits;
let big_deposit_count = NUM_DEPOSITS + 1;
@ -233,16 +255,22 @@ fn invalid_deposit_deposit_count_too_big() {
);
}
#[test]
fn invalid_deposit_count_too_small() {
#[tokio::test]
async fn invalid_deposit_count_too_small() {
let spec = MainnetEthSpec::default_spec();
let harness = get_harness::<MainnetEthSpec>(EPOCH_OFFSET, VALIDATOR_COUNT);
let harness = get_harness::<MainnetEthSpec>(EPOCH_OFFSET, VALIDATOR_COUNT).await;
let mut state = harness.get_current_state();
let (deposits, state) = harness.make_deposits(&mut state, 1, None, None);
let deposits = VariableList::from(deposits);
let mut head_block = harness.chain.head_beacon_block().unwrap().deconstruct().0;
let mut head_block = harness
.chain
.head_beacon_block()
.as_ref()
.clone()
.deconstruct()
.0;
*head_block.to_mut().body_mut().deposits_mut() = deposits;
let small_deposit_count = NUM_DEPOSITS - 1;
@ -259,16 +287,22 @@ fn invalid_deposit_count_too_small() {
);
}
#[test]
fn invalid_deposit_bad_merkle_proof() {
#[tokio::test]
async fn invalid_deposit_bad_merkle_proof() {
let spec = MainnetEthSpec::default_spec();
let harness = get_harness::<MainnetEthSpec>(EPOCH_OFFSET, VALIDATOR_COUNT);
let harness = get_harness::<MainnetEthSpec>(EPOCH_OFFSET, VALIDATOR_COUNT).await;
let mut state = harness.get_current_state();
let (deposits, state) = harness.make_deposits(&mut state, 1, None, None);
let deposits = VariableList::from(deposits);
let mut head_block = harness.chain.head_beacon_block().unwrap().deconstruct().0;
let mut head_block = harness
.chain
.head_beacon_block()
.as_ref()
.clone()
.deconstruct()
.0;
*head_block.to_mut().body_mut().deposits_mut() = deposits;
let bad_index = state.eth1_deposit_index() as usize;
@ -287,17 +321,23 @@ fn invalid_deposit_bad_merkle_proof() {
);
}
#[test]
fn invalid_deposit_wrong_sig() {
#[tokio::test]
async fn invalid_deposit_wrong_sig() {
let spec = MainnetEthSpec::default_spec();
let harness = get_harness::<MainnetEthSpec>(EPOCH_OFFSET, VALIDATOR_COUNT);
let harness = get_harness::<MainnetEthSpec>(EPOCH_OFFSET, VALIDATOR_COUNT).await;
let mut state = harness.get_current_state();
let (deposits, state) =
harness.make_deposits(&mut state, 1, None, Some(SignatureBytes::empty()));
let deposits = VariableList::from(deposits);
let mut head_block = harness.chain.head_beacon_block().unwrap().deconstruct().0;
let mut head_block = harness
.chain
.head_beacon_block()
.as_ref()
.clone()
.deconstruct()
.0;
*head_block.to_mut().body_mut().deposits_mut() = deposits;
let result = process_operations::process_deposits(state, head_block.body().deposits(), &spec);
@ -305,17 +345,23 @@ fn invalid_deposit_wrong_sig() {
assert_eq!(result, Ok(()));
}
#[test]
fn invalid_deposit_invalid_pub_key() {
#[tokio::test]
async fn invalid_deposit_invalid_pub_key() {
let spec = MainnetEthSpec::default_spec();
let harness = get_harness::<MainnetEthSpec>(EPOCH_OFFSET, VALIDATOR_COUNT);
let harness = get_harness::<MainnetEthSpec>(EPOCH_OFFSET, VALIDATOR_COUNT).await;
let mut state = harness.get_current_state();
let (deposits, state) =
harness.make_deposits(&mut state, 1, Some(PublicKeyBytes::empty()), None);
let deposits = VariableList::from(deposits);
let mut head_block = harness.chain.head_beacon_block().unwrap().deconstruct().0;
let mut head_block = harness
.chain
.head_beacon_block()
.as_ref()
.clone()
.deconstruct()
.0;
*head_block.to_mut().body_mut().deposits_mut() = deposits;
let result = process_operations::process_deposits(state, head_block.body().deposits(), &spec);
@ -324,13 +370,19 @@ fn invalid_deposit_invalid_pub_key() {
assert_eq!(result, Ok(()));
}
#[test]
fn invalid_attestation_no_committee_for_index() {
#[tokio::test]
async fn invalid_attestation_no_committee_for_index() {
let spec = MainnetEthSpec::default_spec();
let harness = get_harness::<MainnetEthSpec>(EPOCH_OFFSET, VALIDATOR_COUNT);
let harness = get_harness::<MainnetEthSpec>(EPOCH_OFFSET, VALIDATOR_COUNT).await;
let mut state = harness.get_current_state();
let mut head_block = harness.chain.head_beacon_block().unwrap().deconstruct().0;
let mut head_block = harness
.chain
.head_beacon_block()
.as_ref()
.clone()
.deconstruct()
.0;
head_block.to_mut().body_mut().attestations_mut()[0]
.data
.index += 1;
@ -352,13 +404,19 @@ fn invalid_attestation_no_committee_for_index() {
);
}
#[test]
fn invalid_attestation_wrong_justified_checkpoint() {
#[tokio::test]
async fn invalid_attestation_wrong_justified_checkpoint() {
let spec = MainnetEthSpec::default_spec();
let harness = get_harness::<MainnetEthSpec>(EPOCH_OFFSET, VALIDATOR_COUNT);
let harness = get_harness::<MainnetEthSpec>(EPOCH_OFFSET, VALIDATOR_COUNT).await;
let mut state = harness.get_current_state();
let mut head_block = harness.chain.head_beacon_block().unwrap().deconstruct().0;
let mut head_block = harness
.chain
.head_beacon_block()
.as_ref()
.clone()
.deconstruct()
.0;
let old_justified_checkpoint = head_block.body().attestations()[0].data.source;
let mut new_justified_checkpoint = old_justified_checkpoint;
new_justified_checkpoint.epoch += Epoch::new(1);
@ -389,13 +447,19 @@ fn invalid_attestation_wrong_justified_checkpoint() {
);
}
#[test]
fn invalid_attestation_bad_aggregation_bitfield_len() {
#[tokio::test]
async fn invalid_attestation_bad_aggregation_bitfield_len() {
let spec = MainnetEthSpec::default_spec();
let harness = get_harness::<MainnetEthSpec>(EPOCH_OFFSET, VALIDATOR_COUNT);
let harness = get_harness::<MainnetEthSpec>(EPOCH_OFFSET, VALIDATOR_COUNT).await;
let mut state = harness.get_current_state();
let mut head_block = harness.chain.head_beacon_block().unwrap().deconstruct().0;
let mut head_block = harness
.chain
.head_beacon_block()
.as_ref()
.clone()
.deconstruct()
.0;
head_block.to_mut().body_mut().attestations_mut()[0].aggregation_bits =
Bitfield::with_capacity(spec.target_committee_size).unwrap();
@ -416,13 +480,19 @@ fn invalid_attestation_bad_aggregation_bitfield_len() {
);
}
#[test]
fn invalid_attestation_bad_signature() {
#[tokio::test]
async fn invalid_attestation_bad_signature() {
let spec = MainnetEthSpec::default_spec();
let harness = get_harness::<MainnetEthSpec>(EPOCH_OFFSET, 97); // minimal number of required validators for this test
let harness = get_harness::<MainnetEthSpec>(EPOCH_OFFSET, 97).await; // minimal number of required validators for this test
let mut state = harness.get_current_state();
let mut head_block = harness.chain.head_beacon_block().unwrap().deconstruct().0;
let mut head_block = harness
.chain
.head_beacon_block()
.as_ref()
.clone()
.deconstruct()
.0;
head_block.to_mut().body_mut().attestations_mut()[0].signature = AggregateSignature::empty();
let result = process_operations::process_attestations(
@ -444,13 +514,19 @@ fn invalid_attestation_bad_signature() {
);
}
#[test]
fn invalid_attestation_included_too_early() {
#[tokio::test]
async fn invalid_attestation_included_too_early() {
let spec = MainnetEthSpec::default_spec();
let harness = get_harness::<MainnetEthSpec>(EPOCH_OFFSET, VALIDATOR_COUNT);
let harness = get_harness::<MainnetEthSpec>(EPOCH_OFFSET, VALIDATOR_COUNT).await;
let mut state = harness.get_current_state();
let mut head_block = harness.chain.head_beacon_block().unwrap().deconstruct().0;
let mut head_block = harness
.chain
.head_beacon_block()
.as_ref()
.clone()
.deconstruct()
.0;
let new_attesation_slot = head_block.body().attestations()[0].data.slot
+ Slot::new(MainnetEthSpec::slots_per_epoch());
head_block.to_mut().body_mut().attestations_mut()[0]
@ -479,14 +555,20 @@ fn invalid_attestation_included_too_early() {
);
}
#[test]
fn invalid_attestation_included_too_late() {
#[tokio::test]
async fn invalid_attestation_included_too_late() {
let spec = MainnetEthSpec::default_spec();
// note to maintainer: might need to increase validator count if we get NoCommittee
let harness = get_harness::<MainnetEthSpec>(EPOCH_OFFSET, VALIDATOR_COUNT);
let harness = get_harness::<MainnetEthSpec>(EPOCH_OFFSET, VALIDATOR_COUNT).await;
let mut state = harness.get_current_state();
let mut head_block = harness.chain.head_beacon_block().unwrap().deconstruct().0;
let mut head_block = harness
.chain
.head_beacon_block()
.as_ref()
.clone()
.deconstruct()
.0;
let new_attesation_slot = head_block.body().attestations()[0].data.slot
- Slot::new(MainnetEthSpec::slots_per_epoch());
head_block.to_mut().body_mut().attestations_mut()[0]
@ -512,14 +594,20 @@ fn invalid_attestation_included_too_late() {
);
}
#[test]
fn invalid_attestation_target_epoch_slot_mismatch() {
#[tokio::test]
async fn invalid_attestation_target_epoch_slot_mismatch() {
let spec = MainnetEthSpec::default_spec();
// note to maintainer: might need to increase validator count if we get NoCommittee
let harness = get_harness::<MainnetEthSpec>(EPOCH_OFFSET, VALIDATOR_COUNT);
let harness = get_harness::<MainnetEthSpec>(EPOCH_OFFSET, VALIDATOR_COUNT).await;
let mut state = harness.get_current_state();
let mut head_block = harness.chain.head_beacon_block().unwrap().deconstruct().0;
let mut head_block = harness
.chain
.head_beacon_block()
.as_ref()
.clone()
.deconstruct()
.0;
head_block.to_mut().body_mut().attestations_mut()[0]
.data
.target
@ -544,10 +632,10 @@ fn invalid_attestation_target_epoch_slot_mismatch() {
);
}
#[test]
fn valid_insert_attester_slashing() {
#[tokio::test]
async fn valid_insert_attester_slashing() {
let spec = MainnetEthSpec::default_spec();
let harness = get_harness::<MainnetEthSpec>(EPOCH_OFFSET, VALIDATOR_COUNT);
let harness = get_harness::<MainnetEthSpec>(EPOCH_OFFSET, VALIDATOR_COUNT).await;
let attester_slashing = harness.make_attester_slashing(vec![1, 2]);
@ -563,10 +651,10 @@ fn valid_insert_attester_slashing() {
assert_eq!(result, Ok(()));
}
#[test]
fn invalid_attester_slashing_not_slashable() {
#[tokio::test]
async fn invalid_attester_slashing_not_slashable() {
let spec = MainnetEthSpec::default_spec();
let harness = get_harness::<MainnetEthSpec>(EPOCH_OFFSET, VALIDATOR_COUNT);
let harness = get_harness::<MainnetEthSpec>(EPOCH_OFFSET, VALIDATOR_COUNT).await;
let mut attester_slashing = harness.make_attester_slashing(vec![1, 2]);
attester_slashing.attestation_1 = attester_slashing.attestation_2.clone();
@ -589,10 +677,10 @@ fn invalid_attester_slashing_not_slashable() {
);
}
#[test]
fn invalid_attester_slashing_1_invalid() {
#[tokio::test]
async fn invalid_attester_slashing_1_invalid() {
let spec = MainnetEthSpec::default_spec();
let harness = get_harness::<MainnetEthSpec>(EPOCH_OFFSET, VALIDATOR_COUNT);
let harness = get_harness::<MainnetEthSpec>(EPOCH_OFFSET, VALIDATOR_COUNT).await;
let mut attester_slashing = harness.make_attester_slashing(vec![1, 2]);
attester_slashing.attestation_1.attesting_indices = VariableList::from(vec![2, 1]);
@ -618,10 +706,10 @@ fn invalid_attester_slashing_1_invalid() {
);
}
#[test]
fn invalid_attester_slashing_2_invalid() {
#[tokio::test]
async fn invalid_attester_slashing_2_invalid() {
let spec = MainnetEthSpec::default_spec();
let harness = get_harness::<MainnetEthSpec>(EPOCH_OFFSET, VALIDATOR_COUNT);
let harness = get_harness::<MainnetEthSpec>(EPOCH_OFFSET, VALIDATOR_COUNT).await;
let mut attester_slashing = harness.make_attester_slashing(vec![1, 2]);
attester_slashing.attestation_2.attesting_indices = VariableList::from(vec![2, 1]);
@ -647,10 +735,10 @@ fn invalid_attester_slashing_2_invalid() {
);
}
#[test]
fn valid_insert_proposer_slashing() {
#[tokio::test]
async fn valid_insert_proposer_slashing() {
let spec = MainnetEthSpec::default_spec();
let harness = get_harness::<MainnetEthSpec>(EPOCH_OFFSET, VALIDATOR_COUNT);
let harness = get_harness::<MainnetEthSpec>(EPOCH_OFFSET, VALIDATOR_COUNT).await;
let proposer_slashing = harness.make_proposer_slashing(1);
let mut state = harness.get_current_state();
let result = process_operations::process_proposer_slashings(
@ -663,10 +751,10 @@ fn valid_insert_proposer_slashing() {
assert!(result.is_ok());
}
#[test]
fn invalid_proposer_slashing_proposals_identical() {
#[tokio::test]
async fn invalid_proposer_slashing_proposals_identical() {
let spec = MainnetEthSpec::default_spec();
let harness = get_harness::<MainnetEthSpec>(EPOCH_OFFSET, VALIDATOR_COUNT);
let harness = get_harness::<MainnetEthSpec>(EPOCH_OFFSET, VALIDATOR_COUNT).await;
let mut proposer_slashing = harness.make_proposer_slashing(1);
proposer_slashing.signed_header_1.message = proposer_slashing.signed_header_2.message.clone();
@ -689,10 +777,10 @@ fn invalid_proposer_slashing_proposals_identical() {
);
}
#[test]
fn invalid_proposer_slashing_proposer_unknown() {
#[tokio::test]
async fn invalid_proposer_slashing_proposer_unknown() {
let spec = MainnetEthSpec::default_spec();
let harness = get_harness::<MainnetEthSpec>(EPOCH_OFFSET, VALIDATOR_COUNT);
let harness = get_harness::<MainnetEthSpec>(EPOCH_OFFSET, VALIDATOR_COUNT).await;
let mut proposer_slashing = harness.make_proposer_slashing(1);
proposer_slashing.signed_header_1.message.proposer_index = 3_141_592;
@ -716,10 +804,10 @@ fn invalid_proposer_slashing_proposer_unknown() {
);
}
#[test]
fn invalid_proposer_slashing_duplicate_slashing() {
#[tokio::test]
async fn invalid_proposer_slashing_duplicate_slashing() {
let spec = MainnetEthSpec::default_spec();
let harness = get_harness::<MainnetEthSpec>(EPOCH_OFFSET, VALIDATOR_COUNT);
let harness = get_harness::<MainnetEthSpec>(EPOCH_OFFSET, VALIDATOR_COUNT).await;
let proposer_slashing = harness.make_proposer_slashing(1);
let mut state = harness.get_current_state();
@ -747,10 +835,10 @@ fn invalid_proposer_slashing_duplicate_slashing() {
);
}
#[test]
fn invalid_bad_proposal_1_signature() {
#[tokio::test]
async fn invalid_bad_proposal_1_signature() {
let spec = MainnetEthSpec::default_spec();
let harness = get_harness::<MainnetEthSpec>(EPOCH_OFFSET, VALIDATOR_COUNT);
let harness = get_harness::<MainnetEthSpec>(EPOCH_OFFSET, VALIDATOR_COUNT).await;
let mut proposer_slashing = harness.make_proposer_slashing(1);
proposer_slashing.signed_header_1.signature = Signature::empty();
let mut state = harness.get_current_state();
@ -771,10 +859,10 @@ fn invalid_bad_proposal_1_signature() {
);
}
#[test]
fn invalid_bad_proposal_2_signature() {
#[tokio::test]
async fn invalid_bad_proposal_2_signature() {
let spec = MainnetEthSpec::default_spec();
let harness = get_harness::<MainnetEthSpec>(EPOCH_OFFSET, VALIDATOR_COUNT);
let harness = get_harness::<MainnetEthSpec>(EPOCH_OFFSET, VALIDATOR_COUNT).await;
let mut proposer_slashing = harness.make_proposer_slashing(1);
proposer_slashing.signed_header_2.signature = Signature::empty();
let mut state = harness.get_current_state();
@ -795,10 +883,10 @@ fn invalid_bad_proposal_2_signature() {
);
}
#[test]
fn invalid_proposer_slashing_proposal_epoch_mismatch() {
#[tokio::test]
async fn invalid_proposer_slashing_proposal_epoch_mismatch() {
let spec = MainnetEthSpec::default_spec();
let harness = get_harness::<MainnetEthSpec>(EPOCH_OFFSET, VALIDATOR_COUNT);
let harness = get_harness::<MainnetEthSpec>(EPOCH_OFFSET, VALIDATOR_COUNT).await;
let mut proposer_slashing = harness.make_proposer_slashing(1);
proposer_slashing.signed_header_1.message.slot = Slot::new(0);
proposer_slashing.signed_header_2.message.slot = Slot::new(128);

View File

@ -6,8 +6,8 @@ use bls::Hash256;
use env_logger::{Builder, Env};
use types::Slot;
#[test]
fn runs_without_error() {
#[tokio::test]
async fn runs_without_error() {
Builder::from_env(Env::default().default_filter_or("error")).init();
let harness = BeaconChainHarness::builder(MinimalEthSpec)
@ -22,15 +22,17 @@ fn runs_without_error() {
(MinimalEthSpec::genesis_epoch() + 4).end_slot(MinimalEthSpec::slots_per_epoch());
let state = harness.get_current_state();
harness.add_attested_blocks_at_slots(
state,
Hash256::zero(),
(1..target_slot.as_u64())
.map(Slot::new)
.collect::<Vec<_>>()
.as_slice(),
(0..8).collect::<Vec<_>>().as_slice(),
);
harness
.add_attested_blocks_at_slots(
state,
Hash256::zero(),
(1..target_slot.as_u64())
.map(Slot::new)
.collect::<Vec<_>>()
.as_slice(),
(0..8).collect::<Vec<_>>().as_slice(),
)
.await;
let mut new_head_state = harness.get_current_state();
process_epoch(&mut new_head_state, &spec).unwrap();
@ -45,8 +47,8 @@ mod release_tests {
use beacon_chain::test_utils::{AttestationStrategy, BlockStrategy};
use types::{Epoch, ForkName, InconsistentFork, MainnetEthSpec};
#[test]
fn altair_state_on_base_fork() {
#[tokio::test]
async fn altair_state_on_base_fork() {
let mut spec = MainnetEthSpec::default_spec();
let slots_per_epoch = MainnetEthSpec::slots_per_epoch();
// The Altair fork happens at epoch 1.
@ -61,12 +63,14 @@ mod release_tests {
harness.advance_slot();
harness.extend_chain(
// Build out enough blocks so we get an Altair block at the very end of an epoch.
(slots_per_epoch * 2 - 1) as usize,
BlockStrategy::OnCanonicalHead,
AttestationStrategy::AllValidators,
);
harness
.extend_chain(
// Build out enough blocks so we get an Altair block at the very end of an epoch.
(slots_per_epoch * 2 - 1) as usize,
BlockStrategy::OnCanonicalHead,
AttestationStrategy::AllValidators,
)
.await;
harness.get_current_state()
};
@ -103,8 +107,8 @@ mod release_tests {
);
}
#[test]
fn base_state_on_altair_fork() {
#[tokio::test]
async fn base_state_on_altair_fork() {
let mut spec = MainnetEthSpec::default_spec();
let slots_per_epoch = MainnetEthSpec::slots_per_epoch();
// The Altair fork never happens.
@ -119,12 +123,14 @@ mod release_tests {
harness.advance_slot();
harness.extend_chain(
// Build out enough blocks so we get a block at the very end of an epoch.
(slots_per_epoch * 2 - 1) as usize,
BlockStrategy::OnCanonicalHead,
AttestationStrategy::AllValidators,
);
harness
.extend_chain(
// Build out enough blocks so we get a block at the very end of an epoch.
(slots_per_epoch * 2 - 1) as usize,
BlockStrategy::OnCanonicalHead,
AttestationStrategy::AllValidators,
)
.await;
harness.get_current_state()
};

View File

@ -17,7 +17,7 @@ fn get_harness<T: EthSpec>() -> BeaconChainHarness<EphemeralHarnessType<T>> {
}
fn build_state<T: EthSpec>() -> BeaconState<T> {
let state = get_harness::<T>().chain.head_beacon_state().unwrap();
let state = get_harness::<T>().chain.head_beacon_state_cloned();
assert_eq!(state.as_base().unwrap().validators.len(), VALIDATOR_COUNT);
assert_eq!(state.as_base().unwrap().balances.len(), VALIDATOR_COUNT);

View File

@ -53,6 +53,7 @@ criterion = "0.3.3"
beacon_chain = { path = "../../beacon_node/beacon_chain" }
eth2_interop_keypairs = { path = "../../common/eth2_interop_keypairs" }
state_processing = { path = "../state_processing" }
tokio = "1.14.0"
[features]
default = ["sqlite", "legacy-arith"]

View File

@ -38,7 +38,7 @@ use tree_hash_derive::TreeHash;
derive(Debug, PartialEq, TreeHash),
tree_hash(enum_behaviour = "transparent")
),
map_ref_into(BeaconBlockBodyRef),
map_ref_into(BeaconBlockBodyRef, BeaconBlock),
map_ref_mut_into(BeaconBlockBodyRefMut)
)]
#[derive(Debug, Clone, Serialize, Deserialize, Encode, TreeHash, Derivative)]
@ -541,6 +541,50 @@ impl_from!(BeaconBlockBase, <E, FullPayload<E>>, <E, BlindedPayload<E>>, |body:
impl_from!(BeaconBlockAltair, <E, FullPayload<E>>, <E, BlindedPayload<E>>, |body: BeaconBlockBodyAltair<_, _>| body.into());
impl_from!(BeaconBlockMerge, <E, FullPayload<E>>, <E, BlindedPayload<E>>, |body: BeaconBlockBodyMerge<_, _>| body.into());
// We can clone blocks with payloads to blocks without payloads, without cloning the payload.
macro_rules! impl_clone_as_blinded {
($ty_name:ident, <$($from_params:ty),*>, <$($to_params:ty),*>) => {
impl<E: EthSpec> $ty_name<$($from_params),*>
{
pub fn clone_as_blinded(&self) -> $ty_name<$($to_params),*> {
let $ty_name {
slot,
proposer_index,
parent_root,
state_root,
body,
} = self;
$ty_name {
slot: *slot,
proposer_index: *proposer_index,
parent_root: *parent_root,
state_root: *state_root,
body: body.clone_as_blinded(),
}
}
}
}
}
impl_clone_as_blinded!(BeaconBlockBase, <E, FullPayload<E>>, <E, BlindedPayload<E>>);
impl_clone_as_blinded!(BeaconBlockAltair, <E, FullPayload<E>>, <E, BlindedPayload<E>>);
impl_clone_as_blinded!(BeaconBlockMerge, <E, FullPayload<E>>, <E, BlindedPayload<E>>);
// A reference to a full beacon block can be cloned into a blinded beacon block, without cloning the
// execution payload.
impl<'a, E: EthSpec> From<BeaconBlockRef<'a, E, FullPayload<E>>>
for BeaconBlock<E, BlindedPayload<E>>
{
fn from(
full_block: BeaconBlockRef<'a, E, FullPayload<E>>,
) -> BeaconBlock<E, BlindedPayload<E>> {
map_beacon_block_ref_into_beacon_block!(&'a _, full_block, |inner, cons| {
cons(inner.clone_as_blinded())
})
}
}
impl<E: EthSpec> From<BeaconBlock<E, FullPayload<E>>>
for (
BeaconBlock<E, BlindedPayload<E>>,

View File

@ -251,6 +251,53 @@ impl<E: EthSpec> From<BeaconBlockBodyMerge<E, FullPayload<E>>>
}
}
// We can clone a full block into a blinded block, without cloning the payload.
impl<E: EthSpec> BeaconBlockBodyBase<E, FullPayload<E>> {
pub fn clone_as_blinded(&self) -> BeaconBlockBodyBase<E, BlindedPayload<E>> {
let (block_body, _payload) = self.clone().into();
block_body
}
}
impl<E: EthSpec> BeaconBlockBodyAltair<E, FullPayload<E>> {
pub fn clone_as_blinded(&self) -> BeaconBlockBodyAltair<E, BlindedPayload<E>> {
let (block_body, _payload) = self.clone().into();
block_body
}
}
impl<E: EthSpec> BeaconBlockBodyMerge<E, FullPayload<E>> {
pub fn clone_as_blinded(&self) -> BeaconBlockBodyMerge<E, BlindedPayload<E>> {
let BeaconBlockBodyMerge {
randao_reveal,
eth1_data,
graffiti,
proposer_slashings,
attester_slashings,
attestations,
deposits,
voluntary_exits,
sync_aggregate,
execution_payload: FullPayload { execution_payload },
} = self;
BeaconBlockBodyMerge {
randao_reveal: randao_reveal.clone(),
eth1_data: eth1_data.clone(),
graffiti: *graffiti,
proposer_slashings: proposer_slashings.clone(),
attester_slashings: attester_slashings.clone(),
attestations: attestations.clone(),
deposits: deposits.clone(),
voluntary_exits: voluntary_exits.clone(),
sync_aggregate: sync_aggregate.clone(),
execution_payload: BlindedPayload {
execution_payload_header: From::from(execution_payload),
},
}
}
}
impl<E: EthSpec> From<BeaconBlockBody<E, FullPayload<E>>>
for (
BeaconBlockBody<E, BlindedPayload<E>>,

View File

@ -34,32 +34,34 @@ fn default_values() {
assert!(cache.get_beacon_committees_at_slot(Slot::new(0)).is_err());
}
fn new_state<T: EthSpec>(validator_count: usize, slot: Slot) -> BeaconState<T> {
async fn new_state<T: EthSpec>(validator_count: usize, slot: Slot) -> BeaconState<T> {
let harness = get_harness(validator_count);
let head_state = harness.get_current_state();
if slot > Slot::new(0) {
harness.add_attested_blocks_at_slots(
head_state,
Hash256::zero(),
(1..slot.as_u64())
.map(Slot::new)
.collect::<Vec<_>>()
.as_slice(),
(0..validator_count).collect::<Vec<_>>().as_slice(),
);
harness
.add_attested_blocks_at_slots(
head_state,
Hash256::zero(),
(1..slot.as_u64())
.map(Slot::new)
.collect::<Vec<_>>()
.as_slice(),
(0..validator_count).collect::<Vec<_>>().as_slice(),
)
.await;
}
harness.get_current_state()
}
#[test]
#[tokio::test]
#[should_panic]
fn fails_without_validators() {
new_state::<MinimalEthSpec>(0, Slot::new(0));
async fn fails_without_validators() {
new_state::<MinimalEthSpec>(0, Slot::new(0)).await;
}
#[test]
fn initializes_with_the_right_epoch() {
let state = new_state::<MinimalEthSpec>(16, Slot::new(0));
#[tokio::test]
async fn initializes_with_the_right_epoch() {
let state = new_state::<MinimalEthSpec>(16, Slot::new(0)).await;
let spec = &MinimalEthSpec::default_spec();
let cache = CommitteeCache::default();
@ -75,13 +77,13 @@ fn initializes_with_the_right_epoch() {
assert!(cache.is_initialized_at(state.next_epoch().unwrap()));
}
#[test]
fn shuffles_for_the_right_epoch() {
#[tokio::test]
async fn shuffles_for_the_right_epoch() {
let num_validators = MinimalEthSpec::minimum_validator_count() * 2;
let epoch = Epoch::new(6);
let slot = epoch.start_slot(MinimalEthSpec::slots_per_epoch());
let mut state = new_state::<MinimalEthSpec>(num_validators, slot);
let mut state = new_state::<MinimalEthSpec>(num_validators, slot).await;
let spec = &MinimalEthSpec::default_spec();
let distinct_hashes: Vec<Hash256> = (0..MinimalEthSpec::epochs_per_historical_vector())

View File

@ -25,7 +25,7 @@ lazy_static! {
static ref KEYPAIRS: Vec<Keypair> = generate_deterministic_keypairs(MAX_VALIDATOR_COUNT);
}
fn get_harness<E: EthSpec>(
async fn get_harness<E: EthSpec>(
validator_count: usize,
slot: Slot,
) -> BeaconChainHarness<EphemeralHarnessType<E>> {
@ -41,24 +41,26 @@ fn get_harness<E: EthSpec>(
.map(Slot::new)
.collect::<Vec<_>>();
let state = harness.get_current_state();
harness.add_attested_blocks_at_slots(
state,
Hash256::zero(),
slots.as_slice(),
(0..validator_count).collect::<Vec<_>>().as_slice(),
);
harness
.add_attested_blocks_at_slots(
state,
Hash256::zero(),
slots.as_slice(),
(0..validator_count).collect::<Vec<_>>().as_slice(),
)
.await;
}
harness
}
fn build_state<E: EthSpec>(validator_count: usize) -> BeaconState<E> {
async fn build_state<E: EthSpec>(validator_count: usize) -> BeaconState<E> {
get_harness(validator_count, Slot::new(0))
.await
.chain
.head_beacon_state()
.unwrap()
.head_beacon_state_cloned()
}
fn test_beacon_proposer_index<T: EthSpec>() {
async fn test_beacon_proposer_index<T: EthSpec>() {
let spec = T::default_spec();
// Get the i'th candidate proposer for the given state and slot
@ -85,20 +87,20 @@ fn test_beacon_proposer_index<T: EthSpec>() {
// Test where we have one validator per slot.
// 0th candidate should be chosen every time.
let state = build_state(T::slots_per_epoch() as usize);
let state = build_state(T::slots_per_epoch() as usize).await;
for i in 0..T::slots_per_epoch() {
test(&state, Slot::from(i), 0);
}
// Test where we have two validators per slot.
// 0th candidate should be chosen every time.
let state = build_state((T::slots_per_epoch() as usize).mul(2));
let state = build_state((T::slots_per_epoch() as usize).mul(2)).await;
for i in 0..T::slots_per_epoch() {
test(&state, Slot::from(i), 0);
}
// Test with two validators per slot, first validator has zero balance.
let mut state = build_state::<T>((T::slots_per_epoch() as usize).mul(2));
let mut state = build_state::<T>((T::slots_per_epoch() as usize).mul(2)).await;
let slot0_candidate0 = ith_candidate(&state, Slot::new(0), 0, &spec);
state.validators_mut()[slot0_candidate0].effective_balance = 0;
test(&state, Slot::new(0), 1);
@ -107,9 +109,9 @@ fn test_beacon_proposer_index<T: EthSpec>() {
}
}
#[test]
fn beacon_proposer_index() {
test_beacon_proposer_index::<MinimalEthSpec>();
#[tokio::test]
async fn beacon_proposer_index() {
test_beacon_proposer_index::<MinimalEthSpec>().await;
}
/// Test that
@ -144,11 +146,11 @@ fn test_cache_initialization<T: EthSpec>(
);
}
#[test]
fn cache_initialization() {
#[tokio::test]
async fn cache_initialization() {
let spec = MinimalEthSpec::default_spec();
let mut state = build_state::<MinimalEthSpec>(16);
let mut state = build_state::<MinimalEthSpec>(16).await;
*state.slot_mut() =
(MinimalEthSpec::genesis_epoch() + 1).start_slot(MinimalEthSpec::slots_per_epoch());
@ -211,11 +213,11 @@ fn test_clone_config<E: EthSpec>(base_state: &BeaconState<E>, clone_config: Clon
}
}
#[test]
fn clone_config() {
#[tokio::test]
async fn clone_config() {
let spec = MinimalEthSpec::default_spec();
let mut state = build_state::<MinimalEthSpec>(16);
let mut state = build_state::<MinimalEthSpec>(16).await;
state.build_all_caches(&spec).unwrap();
state
@ -314,7 +316,7 @@ mod committees {
assert!(expected_indices_iter.next().is_none());
}
fn committee_consistency_test<T: EthSpec>(
async fn committee_consistency_test<T: EthSpec>(
validator_count: usize,
state_epoch: Epoch,
cache_epoch: RelativeEpoch,
@ -322,7 +324,7 @@ mod committees {
let spec = &T::default_spec();
let slot = state_epoch.start_slot(T::slots_per_epoch());
let harness = get_harness::<T>(validator_count, slot);
let harness = get_harness::<T>(validator_count, slot).await;
let mut new_head_state = harness.get_current_state();
let distinct_hashes: Vec<Hash256> = (0..T::epochs_per_historical_vector())
@ -350,7 +352,7 @@ mod committees {
);
}
fn committee_consistency_test_suite<T: EthSpec>(cached_epoch: RelativeEpoch) {
async fn committee_consistency_test_suite<T: EthSpec>(cached_epoch: RelativeEpoch) {
let spec = T::default_spec();
let validator_count = spec
@ -359,13 +361,15 @@ mod committees {
.mul(spec.target_committee_size)
.add(1);
committee_consistency_test::<T>(validator_count as usize, Epoch::new(0), cached_epoch);
committee_consistency_test::<T>(validator_count as usize, Epoch::new(0), cached_epoch)
.await;
committee_consistency_test::<T>(
validator_count as usize,
T::genesis_epoch() + 4,
cached_epoch,
);
)
.await;
committee_consistency_test::<T>(
validator_count as usize,
@ -374,38 +378,39 @@ mod committees {
.mul(T::slots_per_epoch())
.mul(4),
cached_epoch,
);
)
.await;
}
#[test]
fn current_epoch_committee_consistency() {
committee_consistency_test_suite::<MinimalEthSpec>(RelativeEpoch::Current);
#[tokio::test]
async fn current_epoch_committee_consistency() {
committee_consistency_test_suite::<MinimalEthSpec>(RelativeEpoch::Current).await;
}
#[test]
fn previous_epoch_committee_consistency() {
committee_consistency_test_suite::<MinimalEthSpec>(RelativeEpoch::Previous);
#[tokio::test]
async fn previous_epoch_committee_consistency() {
committee_consistency_test_suite::<MinimalEthSpec>(RelativeEpoch::Previous).await;
}
#[test]
fn next_epoch_committee_consistency() {
committee_consistency_test_suite::<MinimalEthSpec>(RelativeEpoch::Next);
#[tokio::test]
async fn next_epoch_committee_consistency() {
committee_consistency_test_suite::<MinimalEthSpec>(RelativeEpoch::Next).await;
}
}
mod get_outstanding_deposit_len {
use super::*;
fn state() -> BeaconState<MinimalEthSpec> {
async fn state() -> BeaconState<MinimalEthSpec> {
get_harness(16, Slot::new(0))
.await
.chain
.head_beacon_state()
.unwrap()
.head_beacon_state_cloned()
}
#[test]
fn returns_ok() {
let mut state = state();
#[tokio::test]
async fn returns_ok() {
let mut state = state().await;
assert_eq!(state.get_outstanding_deposit_len(), Ok(0));
state.eth1_data_mut().deposit_count = 17;
@ -413,9 +418,9 @@ mod get_outstanding_deposit_len {
assert_eq!(state.get_outstanding_deposit_len(), Ok(1));
}
#[test]
fn returns_err_if_the_state_is_invalid() {
let mut state = state();
#[tokio::test]
async fn returns_err_if_the_state_is_invalid() {
let mut state = state().await;
// The state is invalid, deposit count is lower than deposit index.
state.eth1_data_mut().deposit_count = 16;
*state.eth1_deposit_index_mut() = 17;

View File

@ -28,6 +28,8 @@ pub trait ExecPayload<T: EthSpec>:
+ Hash
+ TryFrom<ExecutionPayloadHeader<T>>
+ From<ExecutionPayload<T>>
+ Send
+ 'static
{
fn block_type() -> BlockType;

View File

@ -346,6 +346,14 @@ impl<E: EthSpec> From<SignedBeaconBlock<E>> for SignedBlindedBeaconBlock<E> {
}
}
// We can blind borrowed blocks with payloads by converting the payload into a header (without
// cloning the payload contents).
impl<E: EthSpec> SignedBeaconBlock<E> {
pub fn clone_as_blinded(&self) -> SignedBlindedBeaconBlock<E> {
SignedBeaconBlock::from_block(self.message().into(), self.signature().clone())
}
}
#[cfg(test)]
mod test {
use super::*;

View File

@ -222,7 +222,7 @@ pub fn migrate_db<E: EthSpec>(
runtime_context: &RuntimeContext<E>,
log: Logger,
) -> Result<(), Error> {
let spec = runtime_context.eth2_config.spec.clone();
let spec = &runtime_context.eth2_config.spec;
let hot_path = client_config.get_db_path();
let cold_path = client_config.get_freezer_db_path();
@ -236,7 +236,7 @@ pub fn migrate_db<E: EthSpec>(
Ok(())
},
client_config.store.clone(),
spec,
spec.clone(),
log.clone(),
)?;
@ -253,6 +253,7 @@ pub fn migrate_db<E: EthSpec>(
from,
to,
log,
spec,
)
}

View File

@ -216,14 +216,7 @@ impl<T: BeaconChainTypes> SlasherService<T> {
};
// Add to local op pool.
if let Err(e) = beacon_chain.import_attester_slashing(verified_slashing) {
error!(
log,
"Beacon chain refused attester slashing";
"error" => ?e,
"slashing" => ?slashing,
);
}
beacon_chain.import_attester_slashing(verified_slashing);
// Publish to the network if broadcast is enabled.
if slasher.config().broadcast {

View File

@ -81,11 +81,23 @@ pub struct Cases<T> {
}
impl<T: Case> Cases<T> {
pub fn test_results(&self, fork_name: ForkName) -> Vec<CaseResult> {
self.test_cases
.into_par_iter()
.enumerate()
.map(|(i, (ref path, ref tc))| CaseResult::new(i, path, tc, tc.result(i, fork_name)))
.collect()
pub fn test_results(&self, fork_name: ForkName, use_rayon: bool) -> Vec<CaseResult> {
if use_rayon {
self.test_cases
.into_par_iter()
.enumerate()
.map(|(i, (ref path, ref tc))| {
CaseResult::new(i, path, tc, tc.result(i, fork_name))
})
.collect()
} else {
self.test_cases
.iter()
.enumerate()
.map(|(i, (ref path, ref tc))| {
CaseResult::new(i, path, tc, tc.result(i, fork_name))
})
.collect()
}
}
}

View File

@ -7,15 +7,17 @@ use beacon_chain::{
obtain_indexed_attestation_and_committees_per_slot, VerifiedAttestation,
},
test_utils::{BeaconChainHarness, EphemeralHarnessType},
BeaconChainTypes, HeadInfo,
BeaconChainTypes, CachedHead,
};
use serde_derive::Deserialize;
use ssz_derive::Decode;
use state_processing::state_advance::complete_state_advance;
use std::future::Future;
use std::sync::Arc;
use std::time::Duration;
use types::{
Attestation, BeaconBlock, BeaconState, Checkpoint, Epoch, EthSpec, ExecutionBlockHash,
ForkName, Hash256, IndexedAttestation, SignedBeaconBlock, Slot, Uint256,
Attestation, BeaconBlock, BeaconState, Checkpoint, EthSpec, ExecutionBlockHash, ForkName,
Hash256, IndexedAttestation, SignedBeaconBlock, Slot, Uint256,
};
#[derive(Default, Debug, PartialEq, Clone, Deserialize, Decode)]
@ -287,19 +289,20 @@ impl<E: EthSpec> Tester<E> {
Ok(self.spec.genesis_slot + slots_since_genesis)
}
fn find_head(&self) -> Result<HeadInfo, Error> {
fn block_on_dangerous<F: Future>(&self, future: F) -> Result<F::Output, Error> {
self.harness
.chain
.fork_choice()
.map_err(|e| Error::InternalError(format!("failed to find head with {:?}", e)))?;
self.harness
.chain
.head_info()
.map_err(|e| Error::InternalError(format!("failed to read head with {:?}", e)))
.task_executor
.clone()
.block_on_dangerous(future, "ef_tests_block_on")
.ok_or_else(|| Error::InternalError("runtime shutdown".into()))
}
fn genesis_epoch(&self) -> Epoch {
self.spec.genesis_slot.epoch(E::slots_per_epoch())
fn find_head(&self) -> Result<CachedHead<E>, Error> {
let chain = self.harness.chain.clone();
self.block_on_dangerous(chain.recompute_head_at_current_slot())?
.map_err(|e| Error::InternalError(format!("failed to find head with {:?}", e)))?;
Ok(self.harness.chain.canonical_head.cached_head())
}
pub fn set_tick(&self, tick: u64) {
@ -314,15 +317,16 @@ impl<E: EthSpec> Tester<E> {
self.harness
.chain
.fork_choice
.write()
.canonical_head
.fork_choice_write_lock()
.update_time(slot)
.unwrap();
}
pub fn process_block(&self, block: SignedBeaconBlock<E>, valid: bool) -> Result<(), Error> {
let result = self.harness.chain.process_block(block.clone());
let block_root = block.canonical_root();
let block = Arc::new(block);
let result = self.block_on_dangerous(self.harness.chain.process_block(block.clone()))?;
if result.is_ok() != valid {
return Err(Error::DidntFail(format!(
"block with root {} was valid={} whilst test expects valid={}. result: {:?}",
@ -367,16 +371,20 @@ impl<E: EthSpec> Tester<E> {
.seconds_from_current_slot_start(self.spec.seconds_per_slot)
.unwrap();
let (block, _) = block.deconstruct();
let result = self.harness.chain.fork_choice.write().on_block(
self.harness.chain.slot().unwrap(),
&block,
block_root,
block_delay,
&state,
PayloadVerificationStatus::Irrelevant,
&self.harness.chain.spec,
);
let result = self
.harness
.chain
.canonical_head
.fork_choice_write_lock()
.on_block(
self.harness.chain.slot().unwrap(),
block.message(),
block_root,
block_delay,
&state,
PayloadVerificationStatus::Irrelevant,
&self.harness.chain.spec,
);
if result.is_ok() {
return Err(Error::DidntFail(format!(
@ -424,10 +432,11 @@ impl<E: EthSpec> Tester<E> {
}
pub fn check_head(&self, expected_head: Head) -> Result<(), Error> {
let chain_head = self.find_head().map(|head| Head {
slot: head.slot,
root: head.block_root,
})?;
let head = self.find_head()?;
let chain_head = Head {
slot: head.head_slot(),
root: head.head_block_root(),
};
check_equal("head", chain_head, expected_head)
}
@ -446,15 +455,15 @@ impl<E: EthSpec> Tester<E> {
}
pub fn check_justified_checkpoint(&self, expected_checkpoint: Checkpoint) -> Result<(), Error> {
let head_checkpoint = self.find_head()?.current_justified_checkpoint;
let fc_checkpoint = self.harness.chain.fork_choice.read().justified_checkpoint();
let head_checkpoint = self.find_head()?.justified_checkpoint();
let fc_checkpoint = self
.harness
.chain
.canonical_head
.fork_choice_read_lock()
.justified_checkpoint();
assert_checkpoints_eq(
"justified_checkpoint",
self.genesis_epoch(),
head_checkpoint,
fc_checkpoint,
);
assert_checkpoints_eq("justified_checkpoint", head_checkpoint, fc_checkpoint);
check_equal("justified_checkpoint", fc_checkpoint, expected_checkpoint)
}
@ -463,15 +472,15 @@ impl<E: EthSpec> Tester<E> {
&self,
expected_checkpoint_root: Hash256,
) -> Result<(), Error> {
let head_checkpoint = self.find_head()?.current_justified_checkpoint;
let fc_checkpoint = self.harness.chain.fork_choice.read().justified_checkpoint();
let head_checkpoint = self.find_head()?.justified_checkpoint();
let fc_checkpoint = self
.harness
.chain
.canonical_head
.fork_choice_read_lock()
.justified_checkpoint();
assert_checkpoints_eq(
"justified_checkpoint_root",
self.genesis_epoch(),
head_checkpoint,
fc_checkpoint,
);
assert_checkpoints_eq("justified_checkpoint_root", head_checkpoint, fc_checkpoint);
check_equal(
"justified_checkpoint_root",
@ -481,15 +490,15 @@ impl<E: EthSpec> Tester<E> {
}
pub fn check_finalized_checkpoint(&self, expected_checkpoint: Checkpoint) -> Result<(), Error> {
let head_checkpoint = self.find_head()?.finalized_checkpoint;
let fc_checkpoint = self.harness.chain.fork_choice.read().finalized_checkpoint();
let head_checkpoint = self.find_head()?.finalized_checkpoint();
let fc_checkpoint = self
.harness
.chain
.canonical_head
.fork_choice_read_lock()
.finalized_checkpoint();
assert_checkpoints_eq(
"finalized_checkpoint",
self.genesis_epoch(),
head_checkpoint,
fc_checkpoint,
);
assert_checkpoints_eq("finalized_checkpoint", head_checkpoint, fc_checkpoint);
check_equal("finalized_checkpoint", fc_checkpoint, expected_checkpoint)
}
@ -501,8 +510,8 @@ impl<E: EthSpec> Tester<E> {
let best_justified_checkpoint = self
.harness
.chain
.fork_choice
.read()
.canonical_head
.fork_choice_read_lock()
.best_justified_checkpoint();
check_equal(
"best_justified_checkpoint",
@ -515,7 +524,12 @@ impl<E: EthSpec> Tester<E> {
&self,
expected_proposer_boost_root: Hash256,
) -> Result<(), Error> {
let proposer_boost_root = self.harness.chain.fork_choice.read().proposer_boost_root();
let proposer_boost_root = self
.harness
.chain
.canonical_head
.fork_choice_read_lock()
.proposer_boost_root();
check_equal(
"proposer_boost_root",
proposer_boost_root,
@ -530,20 +544,8 @@ impl<E: EthSpec> Tester<E> {
/// This function is necessary due to a quirk documented in this issue:
///
/// https://github.com/ethereum/consensus-specs/issues/2566
fn assert_checkpoints_eq(name: &str, genesis_epoch: Epoch, head: Checkpoint, fc: Checkpoint) {
if fc.epoch == genesis_epoch {
assert_eq!(
head,
Checkpoint {
epoch: genesis_epoch,
root: Hash256::zero()
},
"{} (genesis)",
name
)
} else {
assert_eq!(head, fc, "{} (non-genesis)", name)
}
fn assert_checkpoints_eq(name: &str, head: Checkpoint, fc: Checkpoint) {
assert_eq!(head, fc, "{}", name)
}
/// Convenience function to create `Error` messages.

Some files were not shown because too many files have changed in this diff Show More