Retrospective invalidation of exec. payloads for opt. sync (#2837)

## Issue Addressed

NA

## Proposed Changes

Adds the functionality to allow blocks to be validated/invalidated after their import as per the [optimistic sync spec](https://github.com/ethereum/consensus-specs/blob/dev/sync/optimistic.md#how-to-optimistically-import-blocks). This means:

- Updating `ProtoArray` to allow flipping the `execution_status` of ancestors/descendants based on payload validity updates.
- Creating separation between `execution_layer` and the `beacon_chain` by creating a `PayloadStatus` struct.
- Refactoring how the `execution_layer` selects a `PayloadStatus` from the multiple statuses returned from multiple EEs.
- Adding testing framework for optimistic imports.
- Add `ExecutionBlockHash(Hash256)` new-type struct to avoid confusion between *beacon block roots* and *execution payload hashes*.
- Add `merge` to [`FORKS`](c3a793fd73/Makefile (L17)) in the `Makefile` to ensure we test the beacon chain with merge settings.
    - Fix some tests here that were failing due to a missing execution layer.

## TODO

- [ ] Balance tests

Co-authored-by: Mark Mackey <mark@sigmaprime.io>
This commit is contained in:
Paul Hauner 2022-02-28 22:07:48 +00:00
parent 5e1f8a8480
commit 27e83b888c
50 changed files with 3358 additions and 768 deletions

View File

@ -15,7 +15,7 @@ PINNED_NIGHTLY ?= nightly
# List of all hard forks. This list is used to set env variables for several tests so that
# they run for different forks.
FORKS=phase0 altair
FORKS=phase0 altair merge
# Builds the Lighthouse binary in release (optimized).
#

View File

@ -52,7 +52,7 @@ use crate::{metrics, BeaconChainError};
use eth2::types::{
EventKind, SseBlock, SseChainReorg, SseFinalizedCheckpoint, SseHead, SseLateHead, SyncDuty,
};
use execution_layer::{ExecutionLayer, PayloadStatusV1Status};
use execution_layer::{ExecutionLayer, PayloadStatus};
use fork_choice::{AttestationFromBlock, ForkChoice};
use futures::channel::mpsc::Sender;
use itertools::process_results;
@ -112,6 +112,10 @@ pub const FORK_CHOICE_DB_KEY: Hash256 = Hash256::zero();
/// Defines how old a block can be before it's no longer a candidate for the early attester cache.
const EARLY_ATTESTER_CACHE_HISTORIC_SLOTS: u64 = 4;
/// Reported to the user when the justified block has an invalid execution payload.
pub const INVALID_JUSTIFIED_PAYLOAD_SHUTDOWN_REASON: &str =
"Justified block has an invalid execution payload.";
/// Defines the behaviour when a block/block-root for a skipped slot is requested.
pub enum WhenSlotSkipped {
/// If the slot is a skip slot, return `None`.
@ -201,7 +205,7 @@ pub struct HeadInfo {
pub genesis_validators_root: Hash256,
pub proposer_shuffling_decision_root: Hash256,
pub is_merge_transition_complete: bool,
pub execution_payload_block_hash: Option<Hash256>,
pub execution_payload_block_hash: Option<ExecutionBlockHash>,
}
pub trait BeaconChainTypes: Send + Sync + 'static {
@ -220,15 +224,15 @@ pub enum HeadSafetyStatus {
///
/// If the block is post-terminal-block, `Some(execution_payload.block_hash)` is included with
/// the variant.
Safe(Option<Hash256>),
Safe(Option<ExecutionBlockHash>),
/// The head block execution payload has not yet been verified by an EL.
///
/// The `execution_payload.block_hash` of the head block is returned.
Unsafe(Hash256),
Unsafe(ExecutionBlockHash),
/// The head block execution payload was deemed to be invalid by an EL.
///
/// The `execution_payload.block_hash` of the head block is returned.
Invalid(Hash256),
Invalid(ExecutionBlockHash),
}
pub type BeaconForkChoice<T> = ForkChoice<
@ -3173,6 +3177,101 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
Ok((block, state))
}
/// This method must be called whenever an execution engine indicates that a payload is
/// invalid.
///
/// If the `latest_root` is known to fork-choice it will be invalidated. If it is not known, an
/// error will be returned.
///
/// If `latest_valid_hash` is `None` or references a block unknown to fork choice, no other
/// blocks will be invalidated. If `latest_valid_hash` is a block known to fork choice, all
/// blocks between the `latest_root` and the `latest_valid_hash` will be invalidated (which may
/// cause further, second-order invalidations).
///
/// ## Notes
///
/// Use these rules to set `latest_root`:
///
/// - When `forkchoiceUpdated` indicates an invalid block, set `latest_root` to be the
/// block root that was the head of the chain when `forkchoiceUpdated` was called.
/// - When `executePayload` returns an invalid block *during* block import, set
/// `latest_root` to be the parent of the beacon block containing the invalid
/// payload (because the block containing the payload is not present in fork choice).
/// - When `executePayload` returns an invalid block *after* block import, set
/// `latest_root` to be root of the beacon block containing the invalid payload.
pub fn process_invalid_execution_payload(
&self,
latest_root: Hash256,
latest_valid_hash: Option<ExecutionBlockHash>,
) -> Result<(), Error> {
debug!(
self.log,
"Invalid execution payload in block";
"latest_valid_hash" => ?latest_valid_hash,
"latest_root" => ?latest_root,
);
// Update fork choice.
if let Err(e) = self
.fork_choice
.write()
.on_invalid_execution_payload(latest_root, latest_valid_hash)
{
crit!(
self.log,
"Failed to process invalid payload";
"error" => ?e,
"latest_valid_hash" => ?latest_valid_hash,
"latest_root" => ?latest_root,
);
}
// Run fork choice since it's possible that the payload invalidation might result in a new
// head.
//
// Don't return early though, since invalidating the justified checkpoint might cause an
// error here.
if let Err(e) = self.fork_choice() {
crit!(
self.log,
"Failed to run fork choice routine";
"error" => ?e,
);
}
// Atomically obtain the justified root from fork choice.
let justified_block = self.fork_choice.read().get_justified_block()?;
if justified_block.execution_status.is_invalid() {
crit!(
self.log,
"The justified checkpoint is invalid";
"msg" => "ensure you are not connected to a malicious network. This error is not \
recoverable, please reach out to the lighthouse developers for assistance."
);
let mut shutdown_sender = self.shutdown_sender();
if let Err(e) = shutdown_sender.try_send(ShutdownReason::Failure(
INVALID_JUSTIFIED_PAYLOAD_SHUTDOWN_REASON,
)) {
crit!(
self.log,
"Unable to trigger client shut down";
"msg" => "shut down may already be under way",
"error" => ?e
);
}
// Return an error here to try and prevent progression by upstream functions.
return Err(Error::JustifiedPayloadInvalid {
justified_root: justified_block.root,
execution_block_hash: justified_block.execution_status.block_hash(),
});
}
Ok(())
}
/// Execute the fork choice algorithm and enthrone the result as the canonical head.
pub fn fork_choice(&self) -> Result<(), Error> {
metrics::inc_counter(&metrics::FORK_CHOICE_REQUESTS);
@ -3188,19 +3287,47 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
}
fn fork_choice_internal(&self) -> Result<(), Error> {
// Determine the root of the block that is the head of the chain.
let beacon_block_root = self
.fork_choice
.write()
.get_head(self.slot()?, &self.spec)?;
// Atomically obtain the head block root and the finalized block.
let (beacon_block_root, finalized_block) = {
let mut fork_choice = self.fork_choice.write();
// Determine the root of the block that is the head of the chain.
let beacon_block_root = fork_choice.get_head(self.slot()?, &self.spec)?;
(beacon_block_root, fork_choice.get_finalized_block()?)
};
let current_head = self.head_info()?;
let old_finalized_checkpoint = current_head.finalized_checkpoint;
// Exit early if the head hasn't changed.
if beacon_block_root == current_head.block_root {
return Ok(());
}
// Check to ensure that this finalized block hasn't been marked as invalid.
if let ExecutionStatus::Invalid(block_hash) = finalized_block.execution_status {
crit!(
self.log,
"Finalized block has an invalid payload";
"msg" => "You must use the `--purge-db` flag to clear the database and restart sync. \
You may be on a hostile network.",
"block_hash" => ?block_hash
);
let mut shutdown_sender = self.shutdown_sender();
shutdown_sender
.try_send(ShutdownReason::Failure(
"Finalized block has an invalid execution payload.",
))
.map_err(BeaconChainError::InvalidFinalizedPayloadShutdownError)?;
// Exit now, the node is in an invalid state.
return Err(Error::InvalidFinalizedPayload {
finalized_root: finalized_block.root,
execution_block_hash: block_hash,
});
}
let lag_timer = metrics::start_timer(&metrics::FORK_CHOICE_SET_HEAD_LAG_TIMES);
// At this point we know that the new head block is not the same as the previous one
@ -3448,33 +3575,6 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
}
if new_finalized_checkpoint.epoch != old_finalized_checkpoint.epoch {
// Check to ensure that this finalized block hasn't been marked as invalid.
let finalized_block = self
.fork_choice
.read()
.get_block(&new_finalized_checkpoint.root)
.ok_or(BeaconChainError::FinalizedBlockMissingFromForkChoice(
new_finalized_checkpoint.root,
))?;
if let ExecutionStatus::Invalid(block_hash) = finalized_block.execution_status {
crit!(
self.log,
"Finalized block has an invalid payload";
"msg" => "You must use the `--purge-db` flag to clear the database and restart sync. \
You may be on a hostile network.",
"block_hash" => ?block_hash
);
let mut shutdown_sender = self.shutdown_sender();
shutdown_sender
.try_send(ShutdownReason::Failure(
"Finalized block has an invalid execution payload.",
))
.map_err(BeaconChainError::InvalidFinalizedPayloadShutdownError)?;
// Exit now, the node is in an invalid state.
return Ok(());
}
// Due to race conditions, it's technically possible that the head we load here is
// different to the one earlier in this function.
//
@ -3575,64 +3675,59 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
// If this is a post-merge block, update the execution layer.
if let Some(new_head_execution_block_hash) = new_head_execution_block_hash_opt {
if is_merge_transition_complete {
let execution_layer = self
.execution_layer
.clone()
.ok_or(Error::ExecutionLayerMissing)?;
let store = self.store.clone();
let log = self.log.clone();
// Spawn the update task, without waiting for it to complete.
execution_layer.spawn(
move |execution_layer| async move {
if let Err(e) = Self::update_execution_engine_forkchoice(
execution_layer,
store,
new_finalized_checkpoint.root,
new_head_execution_block_hash,
&log,
)
.await
{
crit!(
log,
"Failed to update execution head";
"error" => ?e
);
}
},
"update_execution_engine_forkchoice",
)
let finalized_execution_block_hash = finalized_block
.execution_status
.block_hash()
.unwrap_or_else(ExecutionBlockHash::zero);
if let Err(e) = self.update_execution_engine_forkchoice_blocking(
finalized_execution_block_hash,
beacon_block_root,
new_head_execution_block_hash,
) {
crit!(
self.log,
"Failed to update execution head";
"error" => ?e
);
}
}
}
Ok(())
}
pub async fn update_execution_engine_forkchoice(
execution_layer: ExecutionLayer,
store: BeaconStore<T>,
finalized_beacon_block_root: Hash256,
head_execution_block_hash: Hash256,
log: &Logger,
pub fn update_execution_engine_forkchoice_blocking(
&self,
finalized_execution_block_hash: ExecutionBlockHash,
head_block_root: Hash256,
head_execution_block_hash: ExecutionBlockHash,
) -> Result<(), Error> {
// Loading the finalized block from the store is not ideal. Perhaps it would be better to
// store it on fork-choice so we can do a lookup without hitting the database.
//
// See: https://github.com/sigp/lighthouse/pull/2627#issuecomment-927537245
let finalized_block = store
.get_block(&finalized_beacon_block_root)?
.ok_or(Error::MissingBeaconBlock(finalized_beacon_block_root))?;
let execution_layer = self
.execution_layer
.as_ref()
.ok_or(Error::ExecutionLayerMissing)?;
let finalized_execution_block_hash = finalized_block
.message()
.body()
.execution_payload()
.ok()
.map(|ep| ep.block_hash)
.unwrap_or_else(Hash256::zero);
execution_layer
.block_on_generic(|_| {
self.update_execution_engine_forkchoice_async(
finalized_execution_block_hash,
head_block_root,
head_execution_block_hash,
)
})
.map_err(Error::ForkchoiceUpdate)?
}
let forkchoice_updated_response = execution_layer
pub async fn update_execution_engine_forkchoice_async(
&self,
finalized_execution_block_hash: ExecutionBlockHash,
head_block_root: Hash256,
head_execution_block_hash: ExecutionBlockHash,
) -> Result<(), Error> {
let forkchoice_updated_response = self
.execution_layer
.as_ref()
.ok_or(Error::ExecutionLayerMissing)?
.notify_forkchoice_updated(
head_execution_block_hash,
finalized_execution_block_hash,
@ -3642,14 +3737,14 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
.map_err(Error::ExecutionForkChoiceUpdateFailed);
match forkchoice_updated_response {
Ok((status, latest_valid_hash)) => match status {
PayloadStatusV1Status::Valid | PayloadStatusV1Status::Syncing => Ok(()),
Ok(status) => match &status {
PayloadStatus::Valid | PayloadStatus::Syncing => Ok(()),
// The specification doesn't list `ACCEPTED` as a valid response to a fork choice
// update. This response *seems* innocent enough, so we won't return early with an
// error. However, we create a log to bring attention to the issue.
PayloadStatusV1Status::Accepted => {
PayloadStatus::Accepted => {
warn!(
log,
self.log,
"Fork choice update received ACCEPTED";
"msg" => "execution engine provided an unexpected response to a fork \
choice update. although this is not a serious issue, please raise \
@ -3657,16 +3752,38 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
);
Ok(())
}
PayloadStatusV1Status::Invalid
| PayloadStatusV1Status::InvalidTerminalBlock
| PayloadStatusV1Status::InvalidBlockHash => {
// TODO(bellatrix): process the invalid payload.
PayloadStatus::Invalid {
latest_valid_hash, ..
} => {
warn!(
self.log,
"Fork choice update invalidated payload";
"status" => ?status
);
// The execution engine has stated that all blocks between the
// `head_execution_block_hash` and `latest_valid_hash` are invalid.
self.process_invalid_execution_payload(
head_block_root,
Some(*latest_valid_hash),
)?;
Err(BeaconChainError::ExecutionForkChoiceUpdateInvalid { status })
}
PayloadStatus::InvalidTerminalBlock { .. }
| PayloadStatus::InvalidBlockHash { .. } => {
warn!(
self.log,
"Fork choice update invalidated payload";
"status" => ?status
);
// The execution engine has stated that the head block is invalid, however it
// hasn't returned a latest valid ancestor.
//
// See: https://github.com/sigp/lighthouse/pull/2837
Err(BeaconChainError::ExecutionForkChoiceUpdateInvalid {
status,
latest_valid_hash,
})
// Using a `None` latest valid ancestor will result in only the head block
// being invalidated (no ancestors).
self.process_invalid_execution_payload(head_block_root, None)?;
Err(BeaconChainError::ExecutionForkChoiceUpdateInvalid { status })
}
},
Err(e) => Err(e),

View File

@ -54,7 +54,7 @@ use crate::{
metrics, BeaconChain, BeaconChainError, BeaconChainTypes,
};
use eth2::types::EventKind;
use execution_layer::PayloadStatusV1Status;
use execution_layer::PayloadStatus;
use fork_choice::{ForkChoice, ForkChoiceStore, PayloadVerificationStatus};
use parking_lot::RwLockReadGuard;
use proto_array::Block as ProtoBlock;
@ -76,9 +76,9 @@ use std::time::Duration;
use store::{Error as DBError, HotColdDB, HotStateSummary, KeyValueStore, StoreOp};
use tree_hash::TreeHash;
use types::{
BeaconBlockRef, BeaconState, BeaconStateError, ChainSpec, CloneConfig, Epoch, EthSpec, Hash256,
InconsistentFork, PublicKey, PublicKeyBytes, RelativeEpoch, SignedBeaconBlock,
SignedBeaconBlockHeader, Slot,
BeaconBlockRef, BeaconState, BeaconStateError, ChainSpec, CloneConfig, Epoch, EthSpec,
ExecutionBlockHash, Hash256, InconsistentFork, PublicKey, PublicKeyBytes, RelativeEpoch,
SignedBeaconBlock, SignedBeaconBlockHeader, Slot,
};
/// Maximum block slot number. Block with slots bigger than this constant will NOT be processed.
@ -270,10 +270,7 @@ pub enum ExecutionPayloadError {
/// ## Peer scoring
///
/// The block is invalid and the peer is faulty
RejectedByExecutionEngine {
status: PayloadStatusV1Status,
latest_valid_hash: Option<Vec<Hash256>>,
},
RejectedByExecutionEngine { status: PayloadStatus },
/// The execution payload timestamp does not match the slot
///
/// ## Peer scoring
@ -286,7 +283,7 @@ pub enum ExecutionPayloadError {
///
/// The block is invalid and the peer sent us a block that passes gossip propagation conditions,
/// but is invalid upon further verification.
InvalidTerminalPoWBlock { parent_hash: Hash256 },
InvalidTerminalPoWBlock { parent_hash: ExecutionBlockHash },
/// The `TERMINAL_BLOCK_HASH` is set, but the block has not reached the
/// `TERMINAL_BLOCK_HASH_ACTIVATION_EPOCH`.
///
@ -305,8 +302,8 @@ pub enum ExecutionPayloadError {
/// The block is invalid and the peer sent us a block that passes gossip propagation conditions,
/// but is invalid upon further verification.
InvalidTerminalBlockHash {
terminal_block_hash: Hash256,
payload_parent_hash: Hash256,
terminal_block_hash: ExecutionBlockHash,
payload_parent_hash: ExecutionBlockHash,
},
/// The execution node failed to provide a parent block to a known block. This indicates an
/// issue with the execution node.
@ -314,7 +311,7 @@ pub enum ExecutionPayloadError {
/// ## Peer scoring
///
/// The peer is not necessarily invalid.
PoWParentMissing(Hash256),
PoWParentMissing(ExecutionBlockHash),
}
impl From<execution_layer::Error> for ExecutionPayloadError {

View File

@ -8,7 +8,7 @@ use crate::naive_aggregation_pool::Error as NaiveAggregationError;
use crate::observed_aggregates::Error as ObservedAttestationsError;
use crate::observed_attesters::Error as ObservedAttestersError;
use crate::observed_block_producers::Error as ObservedBlockProducersError;
use execution_layer::PayloadStatusV1Status;
use execution_layer::PayloadStatus;
use futures::channel::mpsc::TrySendError;
use operation_pool::OpPoolError;
use safe_arith::ArithError;
@ -139,15 +139,27 @@ pub enum BeaconChainError {
ExecutionLayerMissing,
ExecutionForkChoiceUpdateFailed(execution_layer::Error),
ExecutionForkChoiceUpdateInvalid {
status: PayloadStatusV1Status,
latest_valid_hash: Option<Vec<Hash256>>,
status: PayloadStatus,
},
BlockRewardSlotError,
BlockRewardAttestationError,
BlockRewardSyncError,
HeadMissingFromForkChoice(Hash256),
FinalizedBlockMissingFromForkChoice(Hash256),
InvalidFinalizedPayload {
finalized_root: Hash256,
execution_block_hash: ExecutionBlockHash,
},
InvalidFinalizedPayloadShutdownError(TrySendError<ShutdownReason>),
JustifiedPayloadInvalid {
justified_root: Hash256,
execution_block_hash: Option<ExecutionBlockHash>,
},
ForkchoiceUpdate(execution_layer::Error),
FinalizedCheckpointMismatch {
head_state: Checkpoint,
fork_choice: Hash256,
},
}
easy_from_to!(SlotProcessingError, BeaconChainError);

View File

@ -11,7 +11,7 @@ use crate::{
BeaconChain, BeaconChainError, BeaconChainTypes, BlockError, BlockProductionError,
ExecutionPayloadError,
};
use execution_layer::PayloadStatusV1Status;
use execution_layer::PayloadStatus;
use fork_choice::PayloadVerificationStatus;
use proto_array::{Block as ProtoBlock, ExecutionStatus};
use slog::debug;
@ -57,22 +57,26 @@ pub fn notify_new_payload<T: BeaconChainTypes>(
.block_on(|execution_layer| execution_layer.notify_new_payload(execution_payload));
match new_payload_response {
Ok((status, latest_valid_hash)) => match status {
PayloadStatusV1Status::Valid => Ok(PayloadVerificationStatus::Verified),
PayloadStatusV1Status::Syncing | PayloadStatusV1Status::Accepted => {
Ok(status) => match status {
PayloadStatus::Valid => Ok(PayloadVerificationStatus::Verified),
PayloadStatus::Syncing | PayloadStatus::Accepted => {
Ok(PayloadVerificationStatus::NotVerified)
}
PayloadStatusV1Status::Invalid
| PayloadStatusV1Status::InvalidTerminalBlock
| PayloadStatusV1Status::InvalidBlockHash => {
// TODO(bellatrix): process the invalid payload.
//
// See: https://github.com/sigp/lighthouse/pull/2837
Err(ExecutionPayloadError::RejectedByExecutionEngine {
status,
latest_valid_hash,
}
.into())
PayloadStatus::Invalid {
latest_valid_hash, ..
} => {
// 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(latest_root, Some(latest_valid_hash))?;
Err(ExecutionPayloadError::RejectedByExecutionEngine { status }.into())
}
PayloadStatus::InvalidTerminalBlock { .. } | PayloadStatus::InvalidBlockHash { .. } => {
// Returning an error here should be sufficient to invalidate the block. We have no
// information to indicate its parent is invalid, so no need to run
// `BeaconChain::process_invalid_execution_payload`.
Err(ExecutionPayloadError::RejectedByExecutionEngine { status }.into())
}
},
Err(e) => Err(ExecutionPayloadError::RequestFailed(e).into()),
@ -99,7 +103,7 @@ pub fn validate_merge_block<T: BeaconChainTypes>(
let block_epoch = block.slot().epoch(T::EthSpec::slots_per_epoch());
let execution_payload = block.execution_payload()?;
if spec.terminal_block_hash != Hash256::zero() {
if spec.terminal_block_hash != ExecutionBlockHash::zero() {
if block_epoch < spec.terminal_block_hash_activation_epoch {
return Err(ExecutionPayloadError::InvalidActivationEpoch {
activation_epoch: spec.terminal_block_hash_activation_epoch,
@ -263,7 +267,7 @@ pub async fn prepare_execution_payload<T: BeaconChainTypes>(
.ok_or(BlockProductionError::ExecutionLayerMissing)?;
let parent_hash = if !is_merge_transition_complete(state) {
let is_terminal_block_hash_set = spec.terminal_block_hash != Hash256::zero();
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;
@ -314,7 +318,7 @@ pub async fn prepare_execution_payload<T: BeaconChainTypes>(
parent_hash,
timestamp,
random,
finalized_block_hash.unwrap_or_else(Hash256::zero),
finalized_block_hash.unwrap_or_else(ExecutionBlockHash::zero),
proposer_index,
)
.await

View File

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

View File

@ -432,7 +432,7 @@ where
spec: chain.spec.clone(),
chain: Arc::new(chain),
validator_keypairs,
shutdown_receiver,
shutdown_receiver: Arc::new(Mutex::new(shutdown_receiver)),
mock_execution_layer: self.mock_execution_layer,
execution_layer_runtime: self.execution_layer_runtime,
rng: make_rng(),
@ -449,7 +449,7 @@ pub struct BeaconChainHarness<T: BeaconChainTypes> {
pub chain: Arc<BeaconChain<T>>,
pub spec: ChainSpec,
pub shutdown_receiver: Receiver<ShutdownReason>,
pub shutdown_receiver: Arc<Mutex<Receiver<ShutdownReason>>>,
pub mock_execution_layer: Option<MockExecutionLayer<T::EthSpec>>,
pub execution_layer_runtime: Option<ExecutionLayerRuntime>,
@ -502,6 +502,17 @@ where
epoch.start_slot(E::slots_per_epoch()).into()
}
pub fn shutdown_reasons(&self) -> Vec<ShutdownReason> {
let mutex = self.shutdown_receiver.clone();
let mut receiver = mutex.lock();
std::iter::from_fn(move || match receiver.try_next() {
Ok(Some(s)) => Some(s),
Ok(None) => panic!("shutdown sender dropped"),
Err(_) => None,
})
.collect()
}
pub fn get_current_state(&self) -> BeaconState<E> {
self.chain.head().unwrap().beacon_state
}

View File

@ -26,6 +26,7 @@ fn produces_attestations() {
.default_spec()
.keypairs(KEYPAIRS[..].to_vec())
.fresh_ephemeral_store()
.mock_execution_layer()
.build();
let chain = &harness.chain;

View File

@ -42,6 +42,7 @@ fn get_harness(validator_count: usize) -> BeaconChainHarness<EphemeralHarnessTyp
.spec(spec)
.keypairs(KEYPAIRS[0..validator_count].to_vec())
.fresh_ephemeral_store()
.mock_execution_layer()
.build();
harness.advance_slot();

View File

@ -54,6 +54,7 @@ fn get_harness(validator_count: usize) -> BeaconChainHarness<EphemeralHarnessTyp
.default_spec()
.keypairs(KEYPAIRS[0..validator_count].to_vec())
.fresh_ephemeral_store()
.mock_execution_layer()
.build();
harness.advance_slot();
@ -839,6 +840,7 @@ fn verify_block_for_gossip_slashing_detection() {
.keypairs(KEYPAIRS.to_vec())
.fresh_ephemeral_store()
.initial_mutator(Box::new(move |builder| builder.slasher(inner_slasher)))
.mock_execution_layer()
.build();
harness.advance_slot();
@ -918,6 +920,7 @@ fn add_base_block_to_altair_chain() {
.spec(spec)
.keypairs(KEYPAIRS[..].to_vec())
.fresh_ephemeral_store()
.mock_execution_layer()
.build();
// Move out of the genesis slot.
@ -1036,6 +1039,7 @@ fn add_altair_block_to_base_chain() {
.spec(spec)
.keypairs(KEYPAIRS[..].to_vec())
.fresh_ephemeral_store()
.mock_execution_layer()
.build();
// Move out of the genesis slot.

View File

@ -3,6 +3,7 @@ mod attestation_verification;
mod block_verification;
mod merge;
mod op_verification;
mod payload_invalidation;
mod store_tests;
mod sync_committee_verification;
mod tests;

View File

@ -13,7 +13,7 @@ fn verify_execution_payload_chain<T: EthSpec>(chain: &[ExecutionPayload<T>]) {
for ep in chain {
assert!(*ep != ExecutionPayload::default());
assert!(ep.block_hash != Hash256::zero());
assert!(ep.block_hash != ExecutionBlockHash::zero());
// Check against previous `ExecutionPayload`.
if let Some(prev_ep) = prev_ep {
@ -40,7 +40,7 @@ fn merge_with_terminal_block_hash_override() {
spec.terminal_total_difficulty,
DEFAULT_TERMINAL_BLOCK,
0,
Hash256::zero(),
ExecutionBlockHash::zero(),
)
.unwrap()
.block_hash;

View File

@ -40,6 +40,7 @@ fn get_harness(store: Arc<HotColdDB>, validator_count: usize) -> TestHarness {
.default_spec()
.keypairs(KEYPAIRS[0..validator_count].to_vec())
.fresh_disk_store(store)
.mock_execution_layer()
.build();
harness.advance_slot();
harness

View File

@ -0,0 +1,601 @@
#![cfg(not(debug_assertions))]
use beacon_chain::{
test_utils::{BeaconChainHarness, EphemeralHarnessType},
BeaconChainError, BlockError, ExecutionPayloadError, HeadInfo, StateSkipConfig,
WhenSlotSkipped, INVALID_JUSTIFIED_PAYLOAD_SHUTDOWN_REASON,
};
use proto_array::ExecutionStatus;
use task_executor::ShutdownReason;
use types::*;
const VALIDATOR_COUNT: usize = 32;
type E = MainnetEthSpec;
#[derive(PartialEq, Clone)]
enum Payload {
Valid,
Invalid {
latest_valid_hash: Option<ExecutionBlockHash>,
},
Syncing,
}
struct InvalidPayloadRig {
harness: BeaconChainHarness<EphemeralHarnessType<E>>,
enable_attestations: bool,
}
impl InvalidPayloadRig {
fn new() -> Self {
let mut spec = E::default_spec();
spec.altair_fork_epoch = Some(Epoch::new(0));
spec.bellatrix_fork_epoch = Some(Epoch::new(0));
let harness = BeaconChainHarness::builder(MainnetEthSpec)
.spec(spec)
.deterministic_keypairs(VALIDATOR_COUNT)
.mock_execution_layer()
.fresh_ephemeral_store()
.build();
// Move to slot 1.
harness.advance_slot();
Self {
harness,
enable_attestations: false,
}
}
fn enable_attestations(mut self) -> Self {
self.enable_attestations = true;
self
}
fn block_hash(&self, block_root: Hash256) -> ExecutionBlockHash {
self.harness
.chain
.get_block(&block_root)
.unwrap()
.unwrap()
.message()
.body()
.execution_payload()
.unwrap()
.block_hash
}
fn execution_status(&self, block_root: Hash256) -> ExecutionStatus {
self.harness
.chain
.fork_choice
.read()
.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()
}
fn move_to_terminal_block(&self) {
let mock_execution_layer = self.harness.mock_execution_layer.as_ref().unwrap();
mock_execution_layer
.server
.execution_block_generator()
.move_to_terminal_block()
.unwrap();
}
fn build_blocks(&mut self, num_blocks: u64, is_valid: Payload) -> Vec<Hash256> {
(0..num_blocks)
.map(|_| self.import_block(is_valid.clone()))
.collect()
}
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);
let justified_checkpoint = self.head_info().current_justified_checkpoint;
assert_eq!(justified_checkpoint.epoch, 2);
}
fn import_block(&mut self, is_valid: Payload) -> Hash256 {
self.import_block_parametric(is_valid, |error| {
matches!(
error,
BlockError::ExecutionPayloadError(
ExecutionPayloadError::RejectedByExecutionEngine { .. }
)
)
})
}
fn block_root_at_slot(&self, slot: Slot) -> Option<Hash256> {
self.harness
.chain
.block_root_at_slot(slot, WhenSlotSkipped::None)
.unwrap()
}
fn import_block_parametric<F: Fn(&BlockError<E>) -> bool>(
&mut self,
is_valid: Payload,
evaluate_error: F,
) -> 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 slot = state.slot() + 1;
let (block, post_state) = self.harness.make_block(state, slot);
let block_root = block.canonical_root();
match is_valid {
Payload::Valid | Payload::Syncing => {
if is_valid == Payload::Syncing {
// Importing a payload whilst returning `SYNCING` simulates an EE that obtains
// the block via it's own means (e.g., devp2p).
let should_import_payload = true;
mock_execution_layer
.server
.all_payloads_syncing(should_import_payload);
} else {
mock_execution_layer.server.full_payload_verification();
}
let root = self.harness.process_block(slot, block.clone()).unwrap();
if self.enable_attestations {
let all_validators: Vec<usize> = (0..VALIDATOR_COUNT).collect();
self.harness.attest_block(
&post_state,
block.state_root(),
block_root.into(),
&block,
&all_validators,
);
}
let execution_status = self.execution_status(root.into());
match is_valid {
Payload::Syncing => assert!(execution_status.is_not_verified()),
Payload::Valid => assert!(execution_status.is_valid()),
Payload::Invalid { .. } => unreachable!(),
}
assert_eq!(
self.harness.chain.get_block(&block_root).unwrap().unwrap(),
block,
"block from db must match block imported"
);
}
Payload::Invalid { latest_valid_hash } => {
let latest_valid_hash = latest_valid_hash
.unwrap_or_else(|| self.block_hash(block.message().parent_root()));
mock_execution_layer
.server
.all_payloads_invalid(latest_valid_hash);
match self.harness.process_block(slot, block) {
Err(error) if evaluate_error(&error) => (),
Err(other) => {
panic!("evaluate_error returned false with {:?}", other)
}
Ok(_) => panic!("block with invalid payload was imported"),
};
assert!(
self.harness
.chain
.fork_choice
.read()
.get_block(&block_root)
.is_none(),
"invalid block must not exist in fork choice"
);
assert!(
self.harness.chain.get_block(&block_root).unwrap().is_none(),
"invalid block cannot be accessed via get_block"
);
}
}
block_root
}
}
/// Simple test of the different import types.
#[test]
fn valid_invalid_syncing() {
let mut rig = InvalidPayloadRig::new();
rig.move_to_terminal_block();
rig.import_block(Payload::Valid);
rig.import_block(Payload::Invalid {
latest_valid_hash: None,
});
rig.import_block(Payload::Syncing);
}
/// Ensure that an invalid payload can invalidate its parent too (given the right
/// `latest_valid_hash`.
#[test]
fn invalid_payload_invalidates_parent() {
let mut rig = InvalidPayloadRig::new();
rig.move_to_terminal_block();
let roots = vec![
rig.import_block(Payload::Syncing),
rig.import_block(Payload::Syncing),
rig.import_block(Payload::Syncing),
];
let latest_valid_hash = rig.block_hash(roots[0]);
rig.import_block(Payload::Invalid {
latest_valid_hash: Some(latest_valid_hash),
});
assert!(rig.execution_status(roots[0]).is_valid());
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]);
}
/// Ensure the client tries to exit when the justified checkpoint is invalidated.
#[test]
fn justified_checkpoint_becomes_invalid() {
let mut rig = InvalidPayloadRig::new().enable_attestations();
rig.move_to_terminal_block();
rig.move_to_first_justification(Payload::Syncing);
let justified_checkpoint = rig.head_info().current_justified_checkpoint;
let parent_root_of_justified = rig
.harness
.chain
.get_block(&justified_checkpoint.root)
.unwrap()
.unwrap()
.parent_root();
let parent_hash_of_justified = rig.block_hash(parent_root_of_justified);
// No service should have triggered a shutdown, yet.
assert!(rig.harness.shutdown_reasons().is_empty());
// Import a block that will invalidate the justified checkpoint.
rig.import_block_parametric(
Payload::Invalid {
latest_valid_hash: Some(parent_hash_of_justified),
},
|error| {
matches!(
error,
// The block import should fail since the beacon chain knows the justified payload
// is invalid.
BlockError::BeaconChainError(BeaconChainError::JustifiedPayloadInvalid { .. })
)
},
);
// The beacon chain should have triggered a shutdown.
assert_eq!(
rig.harness.shutdown_reasons(),
vec![ShutdownReason::Failure(
INVALID_JUSTIFIED_PAYLOAD_SHUTDOWN_REASON
)]
);
}
/// Ensure that a `latest_valid_hash` for a pre-finality block only reverts a single block.
#[test]
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 blocks = rig.build_blocks(num_blocks, Payload::Syncing);
assert_eq!(rig.head_info().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);
// No service should have triggered a shutdown, yet.
assert!(rig.harness.shutdown_reasons().is_empty());
// Import a pre-finalized block.
rig.import_block(Payload::Invalid {
latest_valid_hash: Some(pre_finalized_block_hash),
});
// The latest imported block should be the head.
assert_eq!(rig.head_info().block_root, *blocks.last().unwrap());
// The beacon chain should *not* have triggered a shutdown.
assert_eq!(rig.harness.shutdown_reasons(), vec![]);
// All blocks should still be unverified.
for i in E::slots_per_epoch() * finalized_epoch..num_blocks {
let slot = Slot::new(i);
let root = rig.block_root_at_slot(slot).unwrap();
assert!(rig.execution_status(root).is_not_verified());
}
}
/// Ensure that a `latest_valid_hash` will:
///
/// - Invalidate descendants of `latest_valid_root`.
/// - Validate `latest_valid_root` and its ancestors.
#[test]
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 blocks = rig.build_blocks(4, Payload::Syncing);
let latest_valid_root = rig
.block_root_at_slot(Slot::new(LATEST_VALID_SLOT))
.unwrap();
let latest_valid_hash = rig.block_hash(latest_valid_root);
rig.import_block(Payload::Invalid {
latest_valid_hash: Some(latest_valid_hash),
});
assert_eq!(rig.head_info().slot, LATEST_VALID_SLOT);
for slot in 0..=4 {
let slot = Slot::new(slot);
let root = if slot > 0 {
// If not the genesis slot, check the blocks we just produced.
blocks[slot.as_usize() - 1]
} else {
// Genesis slot
rig.block_root_at_slot(slot).unwrap()
};
let execution_status = rig.execution_status(root);
if slot > LATEST_VALID_SLOT {
assert!(execution_status.is_invalid())
} else if slot == 0 {
assert!(execution_status.is_irrelevant())
} else {
assert!(execution_status.is_valid())
}
}
}
/// Check behaviour when the `latest_valid_hash` is a junk value.
#[test]
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 blocks = rig.build_blocks(num_blocks, Payload::Syncing);
assert_eq!(rig.head_info().finalized_checkpoint.epoch, finalized_epoch);
// No service should have triggered a shutdown, yet.
assert!(rig.harness.shutdown_reasons().is_empty());
let junk_hash = ExecutionBlockHash::repeat_byte(42);
rig.import_block(Payload::Invalid {
latest_valid_hash: Some(junk_hash),
});
// The latest imported block should be the head.
assert_eq!(rig.head_info().block_root, *blocks.last().unwrap());
// The beacon chain should *not* have triggered a shutdown.
assert_eq!(rig.harness.shutdown_reasons(), vec![]);
// All blocks should still be unverified.
for i in E::slots_per_epoch() * finalized_epoch..num_blocks {
let slot = Slot::new(i);
let root = rig.block_root_at_slot(slot).unwrap();
assert!(rig.execution_status(root).is_not_verified());
}
}
/// Check that descendants of invalid blocks are also invalidated.
#[test]
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();
let blocks = rig.build_blocks(num_blocks, Payload::Syncing);
assert_eq!(rig.head_info().finalized_checkpoint.epoch, finalized_epoch);
assert_eq!(rig.head_info().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);
let fork_parent_slot = fork_slot - 1;
let fork_parent_state = rig
.harness
.chain
.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();
// 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.
let latest_valid_slot = fork_parent_slot - 1;
let latest_valid_root = rig
.harness
.chain
.block_root_at_slot(latest_valid_slot, WhenSlotSkipped::None)
.unwrap()
.unwrap();
assert!(blocks.contains(&latest_valid_root));
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());
rig.import_block(Payload::Invalid {
latest_valid_hash: Some(latest_valid_hash),
});
// The block before the fork should become the head.
assert_eq!(rig.head_info().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.
assert!(rig.execution_status(fork_block_root).is_invalid());
for root in blocks {
let slot = rig.harness.chain.get_block(&root).unwrap().unwrap().slot();
// Fork choice doesn't have info about pre-finalization, nothing to check here.
if slot < finalized_slot {
continue;
}
let execution_status = rig.execution_status(root);
if slot <= latest_valid_slot {
// Blocks prior to the latest valid hash are valid.
assert!(execution_status.is_valid());
} else {
// Blocks after the latest valid hash are invalid.
assert!(execution_status.is_invalid());
}
}
}
/// Check that the head will switch after the canonical branch is invalidated.
#[test]
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();
let blocks = rig.build_blocks(num_blocks, Payload::Syncing);
assert_eq!(rig.head_info().finalized_checkpoint.epoch, finalized_epoch);
assert_eq!(rig.head_info().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);
let fork_parent_slot = fork_slot - 1;
let fork_parent_state = rig
.harness
.chain
.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_parent_root = fork_block.parent_root();
let fork_block_root = rig.harness.chain.process_block(fork_block).unwrap();
rig.fork_choice();
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());
rig.import_block(Payload::Invalid {
latest_valid_hash: Some(latest_valid_hash),
});
// The fork block should become the head.
assert_eq!(rig.head_info().block_root, fork_block_root);
// The fork block has not yet been validated.
assert!(rig.execution_status(fork_block_root).is_not_verified());
for root in blocks {
let slot = rig.harness.chain.get_block(&root).unwrap().unwrap().slot();
// Fork choice doesn't have info about pre-finalization, nothing to check here.
if slot < finalized_slot {
continue;
}
let execution_status = rig.execution_status(root);
if slot <= latest_valid_slot {
// Blocks prior to the latest valid hash are valid.
assert!(execution_status.is_valid());
} else {
// Blocks after the latest valid hash are invalid.
assert!(execution_status.is_invalid());
}
}
}
#[test]
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::Invalid {
latest_valid_hash: None,
}),
rig.import_block(Payload::Valid),
];
// 0 should be present in the chain.
assert!(rig.harness.chain.get_block(&roots[0]).unwrap().is_some());
// 1 should *not* be present in the chain.
assert_eq!(rig.harness.chain.get_block(&roots[1]).unwrap(), None);
// 2 should be the head.
let head = rig.harness.chain.head_info().unwrap();
assert_eq!(head.block_root, roots[2]);
}
#[test]
fn invalid_after_optimistic_sync() {
let mut rig = InvalidPayloadRig::new();
rig.move_to_terminal_block();
let mut roots = vec![
rig.import_block(Payload::Syncing),
rig.import_block(Payload::Syncing),
rig.import_block(Payload::Syncing),
];
for root in &roots {
assert!(rig.harness.chain.get_block(root).unwrap().is_some());
}
// 2 should be the head.
let head = rig.harness.chain.head_info().unwrap();
assert_eq!(head.block_root, roots[2]);
roots.push(rig.import_block(Payload::Invalid {
latest_valid_hash: Some(rig.block_hash(roots[1])),
}));
// Running fork choice is necessary since a block has been invalidated.
rig.fork_choice();
// 1 should be the head, since 2 was invalidated.
let head = rig.harness.chain.head_info().unwrap();
assert_eq!(head.block_root, roots[1]);
}

View File

@ -66,6 +66,7 @@ fn get_harness(
.default_spec()
.keypairs(KEYPAIRS[0..validator_count].to_vec())
.fresh_disk_store(store)
.mock_execution_layer()
.build();
harness.advance_slot();
harness
@ -554,6 +555,7 @@ fn delete_blocks_and_states() {
.default_spec()
.keypairs(validators_keypairs)
.fresh_disk_store(store.clone())
.mock_execution_layer()
.build();
let unforked_blocks: u64 = 4 * E::slots_per_epoch();
@ -680,6 +682,7 @@ fn multi_epoch_fork_valid_blocks_test(
.default_spec()
.keypairs(validators_keypairs)
.fresh_disk_store(store)
.mock_execution_layer()
.build();
let num_fork1_blocks: u64 = num_fork1_blocks_.try_into().unwrap();
@ -974,6 +977,7 @@ fn prunes_abandoned_fork_between_two_finalized_checkpoints() {
.default_spec()
.keypairs(validators_keypairs)
.fresh_ephemeral_store()
.mock_execution_layer()
.build();
let slots_per_epoch = rig.slots_per_epoch();
let (mut state, state_root) = rig.get_current_state_and_root();
@ -1083,6 +1087,7 @@ fn pruning_does_not_touch_abandoned_block_shared_with_canonical_chain() {
.default_spec()
.keypairs(validators_keypairs)
.fresh_ephemeral_store()
.mock_execution_layer()
.build();
let slots_per_epoch = rig.slots_per_epoch();
let (state, state_root) = rig.get_current_state_and_root();
@ -1212,6 +1217,7 @@ fn pruning_does_not_touch_blocks_prior_to_finalization() {
.default_spec()
.keypairs(validators_keypairs)
.fresh_ephemeral_store()
.mock_execution_layer()
.build();
let slots_per_epoch = rig.slots_per_epoch();
let (mut state, state_root) = rig.get_current_state_and_root();
@ -1306,6 +1312,7 @@ fn prunes_fork_growing_past_youngest_finalized_checkpoint() {
.default_spec()
.keypairs(validators_keypairs)
.fresh_ephemeral_store()
.mock_execution_layer()
.build();
let (state, state_root) = rig.get_current_state_and_root();
@ -1448,6 +1455,7 @@ fn prunes_skipped_slots_states() {
.default_spec()
.keypairs(validators_keypairs)
.fresh_ephemeral_store()
.mock_execution_layer()
.build();
let (state, state_root) = rig.get_current_state_and_root();
@ -1571,6 +1579,7 @@ fn finalizes_non_epoch_start_slot() {
.default_spec()
.keypairs(validators_keypairs)
.fresh_ephemeral_store()
.mock_execution_layer()
.build();
let (state, state_root) = rig.get_current_state_and_root();
@ -2140,6 +2149,7 @@ fn finalizes_after_resuming_from_db() {
.default_spec()
.keypairs(KEYPAIRS[0..validator_count].to_vec())
.fresh_disk_store(store.clone())
.mock_execution_layer()
.build();
harness.advance_slot();
@ -2183,6 +2193,7 @@ fn finalizes_after_resuming_from_db() {
.default_spec()
.keypairs(KEYPAIRS[0..validator_count].to_vec())
.resumed_disk_store(store)
.mock_execution_layer()
.build();
assert_chains_pretty_much_the_same(&original_chain, &resumed_harness.chain);
@ -2254,6 +2265,7 @@ fn revert_minority_fork_on_resume() {
.spec(spec1)
.keypairs(KEYPAIRS[0..validator_count].to_vec())
.fresh_disk_store(store1)
.mock_execution_layer()
.build();
// Chain with fork epoch configured.
@ -2263,6 +2275,7 @@ fn revert_minority_fork_on_resume() {
.spec(spec2.clone())
.keypairs(KEYPAIRS[0..validator_count].to_vec())
.fresh_disk_store(store2)
.mock_execution_layer()
.build();
// Apply the same blocks to both chains initially.
@ -2358,6 +2371,7 @@ fn revert_minority_fork_on_resume() {
.set_slot(end_slot.as_u64());
builder
}))
.mock_execution_layer()
.build();
// Head should now be just before the fork.

View File

@ -30,6 +30,7 @@ fn get_harness(validator_count: usize) -> BeaconChainHarness<EphemeralHarnessTyp
.spec(spec)
.keypairs(KEYPAIRS[0..validator_count].to_vec())
.fresh_ephemeral_store()
.mock_execution_layer()
.build();
harness.advance_slot();

View File

@ -28,6 +28,7 @@ fn get_harness(validator_count: usize) -> BeaconChainHarness<EphemeralHarnessTyp
.default_spec()
.keypairs(KEYPAIRS[0..validator_count].to_vec())
.fresh_ephemeral_store()
.mock_execution_layer()
.build();
harness.advance_slot();

View File

@ -31,8 +31,8 @@ use std::time::Duration;
use timer::spawn_timer;
use tokio::sync::{mpsc::UnboundedSender, oneshot};
use types::{
test_utils::generate_deterministic_keypairs, BeaconState, ChainSpec, EthSpec, Hash256,
SignedBeaconBlock,
test_utils::generate_deterministic_keypairs, BeaconState, ChainSpec, EthSpec,
ExecutionBlockHash, Hash256, SignedBeaconBlock,
};
/// Interval between polling the eth1 node for genesis information.
@ -662,9 +662,6 @@ where
);
if let Some(execution_layer) = beacon_chain.execution_layer.as_ref() {
let store = beacon_chain.store.clone();
let inner_execution_layer = execution_layer.clone();
let head = beacon_chain
.head_info()
.map_err(|e| format!("Unable to read beacon chain head: {:?}", e))?;
@ -672,18 +669,38 @@ where
// Issue the head to the execution engine on startup. This ensures it can start
// syncing.
if let Some(block_hash) = head.execution_payload_block_hash {
let finalized_root = head.finalized_checkpoint.root;
let finalized_block = beacon_chain
.store
.get_block(&finalized_root)
.map_err(|e| format!("Failed to read finalized block from DB: {:?}", e))?
.ok_or(format!(
"Finalized block missing from store: {:?}",
finalized_root
))?;
let finalized_execution_block_hash = finalized_block
.message()
.body()
.execution_payload()
.ok()
.map(|ep| ep.block_hash)
.unwrap_or_else(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.
let inner_chain = beacon_chain.clone();
runtime_context.executor.spawn(
async move {
let result = BeaconChain::<
Witness<TSlotClock, TEth1Backend, TEthSpec, THotStore, TColdStore>,
>::update_execution_engine_forkchoice(
inner_execution_layer,
store,
head.finalized_checkpoint.root,
block_hash,
&log,
)
.await;
let result = inner_chain
.update_execution_engine_forkchoice_async(
finalized_execution_block_hash,
head.block_root,
block_hash,
)
.await;
// No need to exit early if setting the head fails. It will be set again if/when the
// node comes online.

View File

@ -5,7 +5,7 @@ use serde::{Deserialize, Serialize};
pub const LATEST_TAG: &str = "latest";
use crate::engines::ForkChoiceState;
pub use types::{Address, EthSpec, ExecutionPayload, Hash256, Uint256};
pub use types::{Address, EthSpec, ExecutionBlockHash, ExecutionPayload, Hash256, Uint256};
pub mod http;
pub mod json_structures;
@ -17,14 +17,15 @@ pub enum Error {
Reqwest(reqwest::Error),
BadResponse(String),
RequestFailed(String),
InvalidExecutePayloadResponse(&'static str),
JsonRpc(RpcError),
Json(serde_json::Error),
ServerMessage { code: i64, message: String },
Eip155Failure,
IsSyncing,
ExecutionBlockNotFound(Hash256),
ExecutionBlockNotFound(ExecutionBlockHash),
ExecutionHeadBlockNotFound,
ParentHashEqualsBlockHash(Hash256),
ParentHashEqualsBlockHash(ExecutionBlockHash),
PayloadIdUnavailable,
}
@ -52,7 +53,7 @@ pub trait EngineApi {
async fn get_block_by_hash<'a>(
&self,
block_hash: Hash256,
block_hash: ExecutionBlockHash,
) -> Result<Option<ExecutionBlock>, Error>;
async fn new_payload_v1<T: EthSpec>(
@ -85,7 +86,7 @@ pub enum PayloadStatusV1Status {
#[derive(Clone, Debug, PartialEq)]
pub struct PayloadStatusV1 {
pub status: PayloadStatusV1Status,
pub latest_valid_hash: Option<Hash256>,
pub latest_valid_hash: Option<ExecutionBlockHash>,
pub validation_error: Option<String>,
}
@ -99,10 +100,10 @@ pub enum BlockByNumberQuery<'a> {
#[serde(rename_all = "camelCase")]
pub struct ExecutionBlock {
#[serde(rename = "hash")]
pub block_hash: Hash256,
pub block_hash: ExecutionBlockHash,
#[serde(rename = "number", with = "eth2_serde_utils::u64_hex_be")]
pub block_number: u64,
pub parent_hash: Hash256,
pub parent_hash: ExecutionBlockHash,
pub total_difficulty: Uint256,
}

View File

@ -125,7 +125,7 @@ impl EngineApi for HttpJsonRpc {
async fn get_block_by_hash<'a>(
&self,
block_hash: Hash256,
block_hash: ExecutionBlockHash,
) -> Result<Option<ExecutionBlock>, Error> {
let params = json!([block_hash, RETURN_FULL_TRANSACTION_OBJECTS]);
@ -413,7 +413,9 @@ mod test {
Tester::new()
.assert_request_equals(
|client| async move {
let _ = client.get_block_by_hash(Hash256::repeat_byte(1)).await;
let _ = client
.get_block_by_hash(ExecutionBlockHash::repeat_byte(1))
.await;
},
json!({
"id": STATIC_ID,
@ -433,9 +435,9 @@ mod test {
let _ = client
.forkchoice_updated_v1(
ForkChoiceState {
head_block_hash: Hash256::repeat_byte(1),
safe_block_hash: Hash256::repeat_byte(1),
finalized_block_hash: Hash256::zero(),
head_block_hash: ExecutionBlockHash::repeat_byte(1),
safe_block_hash: ExecutionBlockHash::repeat_byte(1),
finalized_block_hash: ExecutionBlockHash::zero(),
},
Some(PayloadAttributes {
timestamp: 5,
@ -488,7 +490,7 @@ mod test {
|client| async move {
let _ = client
.new_payload_v1::<MainnetEthSpec>(ExecutionPayload {
parent_hash: Hash256::repeat_byte(0),
parent_hash: ExecutionBlockHash::repeat_byte(0),
fee_recipient: Address::repeat_byte(1),
state_root: Hash256::repeat_byte(1),
receipts_root: Hash256::repeat_byte(0),
@ -500,7 +502,7 @@ mod test {
timestamp: 42,
extra_data: vec![].into(),
base_fee_per_gas: Uint256::from(1),
block_hash: Hash256::repeat_byte(1),
block_hash: ExecutionBlockHash::repeat_byte(1),
transactions: vec![].into(),
})
.await;
@ -538,9 +540,9 @@ mod test {
let _ = client
.forkchoice_updated_v1(
ForkChoiceState {
head_block_hash: Hash256::repeat_byte(0),
safe_block_hash: Hash256::repeat_byte(0),
finalized_block_hash: Hash256::repeat_byte(1),
head_block_hash: ExecutionBlockHash::repeat_byte(0),
safe_block_hash: ExecutionBlockHash::repeat_byte(0),
finalized_block_hash: ExecutionBlockHash::repeat_byte(1),
},
None,
)
@ -588,9 +590,9 @@ mod test {
let _ = client
.forkchoice_updated_v1(
ForkChoiceState {
head_block_hash: Hash256::from_str("0x3b8fb240d288781d4aac94d3fd16809ee413bc99294a085798a589dae51ddd4a").unwrap(),
safe_block_hash: Hash256::from_str("0x3b8fb240d288781d4aac94d3fd16809ee413bc99294a085798a589dae51ddd4a").unwrap(),
finalized_block_hash: Hash256::zero(),
head_block_hash: ExecutionBlockHash::from_str("0x3b8fb240d288781d4aac94d3fd16809ee413bc99294a085798a589dae51ddd4a").unwrap(),
safe_block_hash: ExecutionBlockHash::from_str("0x3b8fb240d288781d4aac94d3fd16809ee413bc99294a085798a589dae51ddd4a").unwrap(),
finalized_block_hash: ExecutionBlockHash::zero(),
},
Some(PayloadAttributes {
timestamp: 5,
@ -635,9 +637,9 @@ mod test {
let response = client
.forkchoice_updated_v1(
ForkChoiceState {
head_block_hash: Hash256::from_str("0x3b8fb240d288781d4aac94d3fd16809ee413bc99294a085798a589dae51ddd4a").unwrap(),
safe_block_hash: Hash256::from_str("0x3b8fb240d288781d4aac94d3fd16809ee413bc99294a085798a589dae51ddd4a").unwrap(),
finalized_block_hash: Hash256::zero(),
head_block_hash: ExecutionBlockHash::from_str("0x3b8fb240d288781d4aac94d3fd16809ee413bc99294a085798a589dae51ddd4a").unwrap(),
safe_block_hash: ExecutionBlockHash::from_str("0x3b8fb240d288781d4aac94d3fd16809ee413bc99294a085798a589dae51ddd4a").unwrap(),
finalized_block_hash: ExecutionBlockHash::zero(),
},
Some(PayloadAttributes {
timestamp: 5,
@ -650,7 +652,7 @@ mod test {
assert_eq!(response, ForkchoiceUpdatedResponse {
payload_status: PayloadStatusV1 {
status: PayloadStatusV1Status::Valid,
latest_valid_hash: Some(Hash256::zero()),
latest_valid_hash: Some(ExecutionBlockHash::zero()),
validation_error: Some(String::new()),
},
payload_id:
@ -703,7 +705,7 @@ mod test {
.unwrap();
let expected = ExecutionPayload {
parent_hash: Hash256::from_str("0x3b8fb240d288781d4aac94d3fd16809ee413bc99294a085798a589dae51ddd4a").unwrap(),
parent_hash: ExecutionBlockHash::from_str("0x3b8fb240d288781d4aac94d3fd16809ee413bc99294a085798a589dae51ddd4a").unwrap(),
fee_recipient: Address::from_str("0xa94f5374fce5edbc8e2a8697c15331677e6ebf0b").unwrap(),
state_root: Hash256::from_str("0xca3149fa9e37db08d1cd49c9061db1002ef1cd58db2210f2115c8c989b2bdf45").unwrap(),
receipts_root: Hash256::from_str("0x56e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421").unwrap(),
@ -715,7 +717,7 @@ mod test {
timestamp: 5,
extra_data: vec![].into(),
base_fee_per_gas: Uint256::from(7),
block_hash: Hash256::from_str("0x6359b8381a370e2f54072a5784ddd78b6ed024991558c511d4452eb4f6ac898c").unwrap(),
block_hash: ExecutionBlockHash::from_str("0x6359b8381a370e2f54072a5784ddd78b6ed024991558c511d4452eb4f6ac898c").unwrap(),
transactions: vec![].into(),
};
@ -728,7 +730,7 @@ mod test {
|client| async move {
let _ = client
.new_payload_v1::<MainnetEthSpec>(ExecutionPayload {
parent_hash: Hash256::from_str("0x3b8fb240d288781d4aac94d3fd16809ee413bc99294a085798a589dae51ddd4a").unwrap(),
parent_hash: ExecutionBlockHash::from_str("0x3b8fb240d288781d4aac94d3fd16809ee413bc99294a085798a589dae51ddd4a").unwrap(),
fee_recipient: Address::from_str("0xa94f5374fce5edbc8e2a8697c15331677e6ebf0b").unwrap(),
state_root: Hash256::from_str("0xca3149fa9e37db08d1cd49c9061db1002ef1cd58db2210f2115c8c989b2bdf45").unwrap(),
receipts_root: Hash256::from_str("0x56e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421").unwrap(),
@ -740,7 +742,7 @@ mod test {
timestamp: 5,
extra_data: vec![].into(),
base_fee_per_gas: Uint256::from(7),
block_hash: Hash256::from_str("0x3559e851470f6e7bbed1db474980683e8c315bfce99b2a6ef47c057c04de7858").unwrap(),
block_hash: ExecutionBlockHash::from_str("0x3559e851470f6e7bbed1db474980683e8c315bfce99b2a6ef47c057c04de7858").unwrap(),
transactions: vec![].into(),
})
.await;
@ -788,7 +790,7 @@ mod test {
assert_eq!(response,
PayloadStatusV1 {
status: PayloadStatusV1Status::Valid,
latest_valid_hash: Some(Hash256::from_str("0x3559e851470f6e7bbed1db474980683e8c315bfce99b2a6ef47c057c04de7858").unwrap()),
latest_valid_hash: Some(ExecutionBlockHash::from_str("0x3559e851470f6e7bbed1db474980683e8c315bfce99b2a6ef47c057c04de7858").unwrap()),
validation_error: Some(String::new()),
}
);
@ -801,9 +803,9 @@ mod test {
let _ = client
.forkchoice_updated_v1(
ForkChoiceState {
head_block_hash: Hash256::from_str("0x3559e851470f6e7bbed1db474980683e8c315bfce99b2a6ef47c057c04de7858").unwrap(),
safe_block_hash: Hash256::from_str("0x3559e851470f6e7bbed1db474980683e8c315bfce99b2a6ef47c057c04de7858").unwrap(),
finalized_block_hash: Hash256::from_str("0x3b8fb240d288781d4aac94d3fd16809ee413bc99294a085798a589dae51ddd4a").unwrap(),
head_block_hash: ExecutionBlockHash::from_str("0x3559e851470f6e7bbed1db474980683e8c315bfce99b2a6ef47c057c04de7858").unwrap(),
safe_block_hash: ExecutionBlockHash::from_str("0x3559e851470f6e7bbed1db474980683e8c315bfce99b2a6ef47c057c04de7858").unwrap(),
finalized_block_hash: ExecutionBlockHash::from_str("0x3b8fb240d288781d4aac94d3fd16809ee413bc99294a085798a589dae51ddd4a").unwrap(),
},
None,
)
@ -840,9 +842,9 @@ mod test {
let response = client
.forkchoice_updated_v1(
ForkChoiceState {
head_block_hash: Hash256::from_str("0x3559e851470f6e7bbed1db474980683e8c315bfce99b2a6ef47c057c04de7858").unwrap(),
safe_block_hash: Hash256::from_str("0x3559e851470f6e7bbed1db474980683e8c315bfce99b2a6ef47c057c04de7858").unwrap(),
finalized_block_hash: Hash256::from_str("0x3b8fb240d288781d4aac94d3fd16809ee413bc99294a085798a589dae51ddd4a").unwrap(),
head_block_hash: ExecutionBlockHash::from_str("0x3559e851470f6e7bbed1db474980683e8c315bfce99b2a6ef47c057c04de7858").unwrap(),
safe_block_hash: ExecutionBlockHash::from_str("0x3559e851470f6e7bbed1db474980683e8c315bfce99b2a6ef47c057c04de7858").unwrap(),
finalized_block_hash: ExecutionBlockHash::from_str("0x3b8fb240d288781d4aac94d3fd16809ee413bc99294a085798a589dae51ddd4a").unwrap(),
},
None,
)
@ -851,7 +853,7 @@ mod test {
assert_eq!(response, ForkchoiceUpdatedResponse {
payload_status: PayloadStatusV1 {
status: PayloadStatusV1Status::Valid,
latest_valid_hash: Some(Hash256::zero()),
latest_valid_hash: Some(ExecutionBlockHash::zero()),
validation_error: Some(String::new()),
},
payload_id: None,

View File

@ -1,6 +1,6 @@
use super::*;
use serde::{Deserialize, Serialize};
use types::{EthSpec, FixedVector, Transaction, Unsigned, VariableList};
use types::{EthSpec, ExecutionBlockHash, FixedVector, Transaction, Unsigned, VariableList};
#[derive(Debug, PartialEq, Serialize, Deserialize)]
#[serde(rename_all = "camelCase")]
@ -58,7 +58,7 @@ pub struct JsonPayloadIdResponse {
#[derive(Debug, PartialEq, Default, Serialize, Deserialize)]
#[serde(bound = "T: EthSpec", rename_all = "camelCase")]
pub struct JsonExecutionPayloadV1<T: EthSpec> {
pub parent_hash: Hash256,
pub parent_hash: ExecutionBlockHash,
pub fee_recipient: Address,
pub state_root: Hash256,
pub receipts_root: Hash256,
@ -76,7 +76,7 @@ pub struct JsonExecutionPayloadV1<T: EthSpec> {
#[serde(with = "ssz_types::serde_utils::hex_var_list")]
pub extra_data: VariableList<u8, T::MaxExtraDataBytes>,
pub base_fee_per_gas: Uint256,
pub block_hash: Hash256,
pub block_hash: ExecutionBlockHash,
#[serde(with = "serde_transactions")]
pub transactions:
VariableList<Transaction<T::MaxBytesPerTransaction>, T::MaxTransactionsPerPayload>,
@ -206,9 +206,9 @@ impl From<JsonPayloadAttributesV1> for PayloadAttributes {
#[derive(Debug, PartialEq, Serialize, Deserialize)]
#[serde(rename_all = "camelCase")]
pub struct JsonForkChoiceStateV1 {
pub head_block_hash: Hash256,
pub safe_block_hash: Hash256,
pub finalized_block_hash: Hash256,
pub head_block_hash: ExecutionBlockHash,
pub safe_block_hash: ExecutionBlockHash,
pub finalized_block_hash: ExecutionBlockHash,
}
impl From<ForkChoiceState> for JsonForkChoiceStateV1 {
@ -260,7 +260,7 @@ pub enum JsonPayloadStatusV1Status {
#[serde(rename_all = "camelCase")]
pub struct JsonPayloadStatusV1 {
pub status: JsonPayloadStatusV1Status,
pub latest_valid_hash: Option<Hash256>,
pub latest_valid_hash: Option<ExecutionBlockHash>,
pub validation_error: Option<String>,
}

View File

@ -8,7 +8,7 @@ use lru::LruCache;
use slog::{crit, debug, info, warn, Logger};
use std::future::Future;
use tokio::sync::{Mutex, RwLock};
use types::{Address, Hash256};
use types::{Address, ExecutionBlockHash, Hash256};
/// The number of payload IDs that will be stored for each `Engine`.
///
@ -25,9 +25,9 @@ enum EngineState {
#[derive(Copy, Clone, PartialEq, Debug)]
pub struct ForkChoiceState {
pub head_block_hash: Hash256,
pub safe_block_hash: Hash256,
pub finalized_block_hash: Hash256,
pub head_block_hash: ExecutionBlockHash,
pub safe_block_hash: ExecutionBlockHash,
pub finalized_block_hash: ExecutionBlockHash,
}
/// Used to enable/disable logging on some tasks.
@ -48,7 +48,7 @@ impl Logging {
#[derive(Hash, PartialEq, std::cmp::Eq)]
struct PayloadIdCacheKey {
pub head_block_hash: Hash256,
pub head_block_hash: ExecutionBlockHash,
pub timestamp: u64,
pub random: Hash256,
pub suggested_fee_recipient: Address,
@ -75,7 +75,7 @@ impl<T> Engine<T> {
pub async fn get_payload_id(
&self,
head_block_hash: Hash256,
head_block_hash: ExecutionBlockHash,
timestamp: u64,
random: Hash256,
suggested_fee_recipient: Address,

View File

@ -7,10 +7,11 @@
use engine_api::{Error as ApiError, *};
use engines::{Engine, EngineError, Engines, ForkChoiceState, Logging};
use lru::LruCache;
use payload_status::process_multiple_payload_statuses;
use sensitive_url::SensitiveUrl;
use slog::{crit, debug, error, info, Logger};
use slot_clock::SlotClock;
use std::collections::{HashMap, HashSet};
use std::collections::HashMap;
use std::future::Future;
use std::sync::Arc;
use std::time::Duration;
@ -19,12 +20,14 @@ use tokio::{
sync::{Mutex, MutexGuard},
time::{sleep, sleep_until, Instant},
};
use types::{ChainSpec, Epoch, ProposerPreparationData};
use types::{ChainSpec, Epoch, ExecutionBlockHash, ProposerPreparationData};
pub use engine_api::{http::HttpJsonRpc, PayloadAttributes, PayloadStatusV1Status};
pub use payload_status::PayloadStatus;
mod engine_api;
mod engines;
mod payload_status;
pub mod test_utils;
/// Each time the `ExecutionLayer` retrieves a block from an execution node, it stores that block
@ -50,6 +53,7 @@ pub enum Error {
ShuttingDown,
FeeRecipientUnspecified,
ConsensusFailure,
MissingLatestValidHash,
}
impl From<ApiError> for Error {
@ -68,7 +72,7 @@ struct Inner {
engines: Engines<HttpJsonRpc>,
suggested_fee_recipient: Option<Address>,
proposer_preparation_data: Mutex<HashMap<u64, ProposerPreparationDataEntry>>,
execution_blocks: Mutex<LruCache<Hash256, ExecutionBlock>>,
execution_blocks: Mutex<LruCache<ExecutionBlockHash, ExecutionBlock>>,
executor: TaskExecutor,
log: Logger,
}
@ -137,7 +141,9 @@ impl ExecutionLayer {
}
/// Note: this function returns a mutex guard, be careful to avoid deadlocks.
async fn execution_blocks(&self) -> MutexGuard<'_, LruCache<Hash256, ExecutionBlock>> {
async fn execution_blocks(
&self,
) -> MutexGuard<'_, LruCache<ExecutionBlockHash, ExecutionBlock>> {
self.inner.execution_blocks.lock().await
}
@ -384,10 +390,10 @@ impl ExecutionLayer {
/// will be contacted.
pub async fn get_payload<T: EthSpec>(
&self,
parent_hash: Hash256,
parent_hash: ExecutionBlockHash,
timestamp: u64,
random: Hash256,
finalized_block_hash: Hash256,
finalized_block_hash: ExecutionBlockHash,
proposer_index: u64,
) -> Result<ExecutionPayload<T>, Error> {
let suggested_fee_recipient = self.get_suggested_fee_recipient(proposer_index).await;
@ -434,7 +440,16 @@ impl ExecutionLayer {
)
.await
.map(|response| response.payload_id)?
.ok_or(ApiError::PayloadIdUnavailable)?
.ok_or_else(|| {
error!(
self.log(),
"Exec engine unable to produce payload";
"msg" => "No payload ID, the engine is likely syncing. \
This has the potential to cause a missed block proposal.",
);
ApiError::PayloadIdUnavailable
})?
};
engine.api.get_payload_v1(payload_id).await
@ -459,7 +474,7 @@ impl ExecutionLayer {
pub async fn notify_new_payload<T: EthSpec>(
&self,
execution_payload: &ExecutionPayload<T>,
) -> Result<(PayloadStatusV1Status, Option<Vec<Hash256>>), Error> {
) -> Result<PayloadStatus, Error> {
debug!(
self.log(),
"Issuing engine_newPayload";
@ -473,81 +488,11 @@ impl ExecutionLayer {
.broadcast(|engine| engine.api.new_payload_v1(execution_payload.clone()))
.await;
let mut errors = vec![];
let mut valid = 0;
let mut invalid = 0;
let mut syncing = 0;
let mut invalid_latest_valid_hash = HashSet::new();
for result in broadcast_results {
match result {
Ok(response) => match (&response.latest_valid_hash, &response.status) {
(Some(latest_hash), &PayloadStatusV1Status::Valid) => {
// According to a strict interpretation of the spec, the EE should never
// respond with `VALID` *and* a `latest_valid_hash`.
//
// For the sake of being liberal with what we accept, we will accept a
// `latest_valid_hash` *only if* it matches the submitted payload.
// Otherwise, register an error.
if latest_hash == &execution_payload.block_hash {
valid += 1;
} else {
errors.push(EngineError::Api {
id: "unknown".to_string(),
error: engine_api::Error::BadResponse(
format!(
"new_payload: response.status = Valid but invalid latest_valid_hash. Expected({:?}) Found({:?})",
execution_payload.block_hash,
latest_hash,
)
),
});
}
}
(Some(latest_hash), &PayloadStatusV1Status::Invalid) => {
invalid += 1;
invalid_latest_valid_hash.insert(*latest_hash);
}
(None, &PayloadStatusV1Status::InvalidBlockHash)
| (None, &PayloadStatusV1Status::InvalidTerminalBlock) => invalid += 1,
(None, &PayloadStatusV1Status::Syncing)
| (None, &PayloadStatusV1Status::Accepted) => syncing += 1,
_ => errors.push(EngineError::Api {
id: "unknown".to_string(),
error: engine_api::Error::BadResponse(format!(
"new_payload: response does not conform to engine API spec: {:?}",
response,
)),
}),
},
Err(e) => errors.push(e),
}
}
if valid > 0 && invalid > 0 {
crit!(
self.log(),
"Consensus failure between execution nodes";
"method" => "new_payload"
);
// In this situation, better to have a failure of liveness than vote on a potentially invalid chain
return Err(Error::ConsensusFailure);
}
if valid > 0 {
Ok((
PayloadStatusV1Status::Valid,
Some(vec![execution_payload.block_hash]),
))
} else if invalid > 0 {
Ok((
PayloadStatusV1Status::Invalid,
Some(invalid_latest_valid_hash.into_iter().collect()),
))
} else if syncing > 0 {
Ok((PayloadStatusV1Status::Syncing, None))
} else {
Err(Error::EngineErrors(errors))
}
process_multiple_payload_statuses(
execution_payload.block_hash,
broadcast_results.into_iter(),
self.log(),
)
}
/// Maps to the `engine_consensusValidated` JSON-RPC call.
@ -565,10 +510,10 @@ impl ExecutionLayer {
/// - An error, if all nodes return an error.
pub async fn notify_forkchoice_updated(
&self,
head_block_hash: Hash256,
finalized_block_hash: Hash256,
head_block_hash: ExecutionBlockHash,
finalized_block_hash: ExecutionBlockHash,
payload_attributes: Option<PayloadAttributes>,
) -> Result<(PayloadStatusV1Status, Option<Vec<Hash256>>), Error> {
) -> Result<PayloadStatus, Error> {
debug!(
self.log(),
"Issuing engine_forkchoiceUpdated";
@ -597,78 +542,13 @@ impl ExecutionLayer {
})
.await;
let mut errors = vec![];
let mut valid = 0;
let mut invalid = 0;
let mut syncing = 0;
let mut invalid_latest_valid_hash = HashSet::new();
for result in broadcast_results {
match result {
Ok(response) => match (&response.payload_status.latest_valid_hash, &response.payload_status.status) {
// TODO(bellatrix) a strict interpretation of the v1.0.0.alpha.6 spec says that
// `latest_valid_hash` *cannot* be `None`. However, we accept it to maintain
// Geth compatibility for the short term. See:
//
// https://github.com/ethereum/go-ethereum/issues/24404
(None, &PayloadStatusV1Status::Valid) => valid += 1,
(Some(latest_hash), &PayloadStatusV1Status::Valid) => {
if latest_hash == &head_block_hash {
valid += 1;
} else {
errors.push(EngineError::Api {
id: "unknown".to_string(),
error: engine_api::Error::BadResponse(
format!(
"forkchoice_updated: payload_status = Valid but invalid latest_valid_hash. Expected({:?}) Found({:?})",
head_block_hash,
*latest_hash,
)
),
});
}
}
(Some(latest_hash), &PayloadStatusV1Status::Invalid) => {
invalid += 1;
invalid_latest_valid_hash.insert(*latest_hash);
}
(None, &PayloadStatusV1Status::InvalidTerminalBlock) => invalid += 1,
(None, &PayloadStatusV1Status::Syncing) => syncing += 1,
_ => {
errors.push(EngineError::Api {
id: "unknown".to_string(),
error: engine_api::Error::BadResponse(format!(
"forkchoice_updated: response does not conform to engine API spec: {:?}",
response
)),
})
}
}
Err(e) => errors.push(e),
}
}
if valid > 0 && invalid > 0 {
crit!(
self.log(),
"Consensus failure between execution nodes";
"method" => "forkchoice_updated"
);
// In this situation, better to have a failure of liveness than vote on a potentially invalid chain
return Err(Error::ConsensusFailure);
}
if valid > 0 {
Ok((PayloadStatusV1Status::Valid, Some(vec![head_block_hash])))
} else if invalid > 0 {
Ok((
PayloadStatusV1Status::Invalid,
Some(invalid_latest_valid_hash.into_iter().collect()),
))
} else if syncing > 0 {
Ok((PayloadStatusV1Status::Syncing, None))
} else {
Err(Error::EngineErrors(errors))
}
process_multiple_payload_statuses(
head_block_hash,
broadcast_results
.into_iter()
.map(|result| result.map(|response| response.payload_status)),
self.log(),
)
}
/// Used during block production to determine if the merge has been triggered.
@ -681,12 +561,12 @@ impl ExecutionLayer {
pub async fn get_terminal_pow_block_hash(
&self,
spec: &ChainSpec,
) -> Result<Option<Hash256>, Error> {
) -> Result<Option<ExecutionBlockHash>, Error> {
let hash_opt = self
.engines()
.first_success(|engine| async move {
let terminal_block_hash = spec.terminal_block_hash;
if terminal_block_hash != Hash256::zero() {
if terminal_block_hash != ExecutionBlockHash::zero() {
if self
.get_pow_block(engine, terminal_block_hash)
.await?
@ -730,7 +610,7 @@ impl ExecutionLayer {
&self,
engine: &Engine<HttpJsonRpc>,
spec: &ChainSpec,
) -> Result<Option<Hash256>, ApiError> {
) -> Result<Option<ExecutionBlockHash>, ApiError> {
let mut block = engine
.api
.get_block_by_number(BlockByNumberQuery::Tag(LATEST_TAG))
@ -742,7 +622,7 @@ impl ExecutionLayer {
loop {
let block_reached_ttd = block.total_difficulty >= spec.terminal_total_difficulty;
if block_reached_ttd {
if block.parent_hash == Hash256::zero() {
if block.parent_hash == ExecutionBlockHash::zero() {
return Ok(Some(block.block_hash));
}
let parent = self
@ -790,7 +670,7 @@ impl ExecutionLayer {
/// https://github.com/ethereum/consensus-specs/blob/v1.1.0/specs/merge/fork-choice.md
pub async fn is_valid_terminal_pow_block_hash(
&self,
block_hash: Hash256,
block_hash: ExecutionBlockHash,
spec: &ChainSpec,
) -> Result<Option<bool>, Error> {
let broadcast_results = self
@ -869,7 +749,7 @@ impl ExecutionLayer {
async fn get_pow_block(
&self,
engine: &Engine<HttpJsonRpc>,
hash: Hash256,
hash: ExecutionBlockHash,
) -> Result<Option<ExecutionBlock>, ApiError> {
if let Some(cached) = self.execution_blocks().await.get(&hash).copied() {
// The block was in the cache, no need to request it from the execution
@ -963,7 +843,7 @@ mod test {
MockExecutionLayer::default_params()
.move_to_terminal_block()
.with_terminal_block(|spec, el, _| async move {
let missing_terminal_block = Hash256::repeat_byte(42);
let missing_terminal_block = ExecutionBlockHash::repeat_byte(42);
assert_eq!(
el.is_valid_terminal_pow_block_hash(missing_terminal_block, &spec)

View File

@ -0,0 +1,191 @@
use crate::engine_api::{Error as ApiError, PayloadStatusV1, PayloadStatusV1Status};
use crate::engines::EngineError;
use crate::Error;
use slog::{crit, warn, Logger};
use types::ExecutionBlockHash;
/// Provides a simpler, easier to parse version of `PayloadStatusV1` for upstream users.
///
/// It primarily ensures that the `latest_valid_hash` is always present when relevant.
#[derive(Debug, Clone, PartialEq)]
pub enum PayloadStatus {
Valid,
Invalid {
latest_valid_hash: ExecutionBlockHash,
validation_error: Option<String>,
},
Syncing,
Accepted,
InvalidBlockHash {
validation_error: Option<String>,
},
InvalidTerminalBlock {
validation_error: Option<String>,
},
}
/// Processes the responses from multiple execution engines, finding the "best" status and returning
/// it (if any).
///
/// This function has the following basic goals:
///
/// - Detect a consensus failure between nodes.
/// - Find the most-synced node by preferring a definite response (valid/invalid) over a
/// syncing/accepted response or error.
///
/// # Details
///
/// - If there are conflicting valid/invalid responses, always return an error.
/// - If there are syncing/accepted responses but valid/invalid responses exist, return the
/// valid/invalid responses since they're definite.
/// - If there are multiple valid responses, return the first one processed.
/// - If there are multiple invalid responses, return the first one processed.
/// - Syncing/accepted responses are grouped, if there are multiple of them, return the first one
/// processed.
/// - If there are no responses (only errors or nothing), return an error.
pub fn process_multiple_payload_statuses(
head_block_hash: ExecutionBlockHash,
statuses: impl Iterator<Item = Result<PayloadStatusV1, EngineError>>,
log: &Logger,
) -> Result<PayloadStatus, Error> {
let mut errors = vec![];
let mut valid_statuses = vec![];
let mut invalid_statuses = vec![];
let mut other_statuses = vec![];
for status in statuses {
match status {
Err(e) => errors.push(e),
Ok(response) => match &response.status {
PayloadStatusV1Status::Valid => {
if response
.latest_valid_hash
.map_or(false, |h| h == head_block_hash)
{
// The response is only valid if `latest_valid_hash` is not `null` and
// equal to the provided `block_hash`.
valid_statuses.push(PayloadStatus::Valid)
} else {
errors.push(EngineError::Api {
id: "unknown".to_string(),
error: ApiError::BadResponse(
format!(
"new_payload: response.status = VALID but invalid latest_valid_hash. Expected({:?}) Found({:?})",
head_block_hash,
response.latest_valid_hash,
)
),
});
}
}
PayloadStatusV1Status::Invalid => {
if let Some(latest_valid_hash) = response.latest_valid_hash {
// The response is only valid if `latest_valid_hash` is not `null`.
invalid_statuses.push(PayloadStatus::Invalid {
latest_valid_hash,
validation_error: response.validation_error.clone(),
})
} else {
errors.push(EngineError::Api {
id: "unknown".to_string(),
error: ApiError::BadResponse(
"new_payload: response.status = INVALID but null latest_valid_hash"
.to_string(),
),
});
}
}
PayloadStatusV1Status::InvalidBlockHash => {
// In the interests of being liberal with what we accept, only raise a
// warning here.
if response.latest_valid_hash.is_some() {
warn!(
log,
"Malformed response from execution engine";
"msg" => "expected a null latest_valid_hash",
"status" => ?response.status
)
}
invalid_statuses.push(PayloadStatus::InvalidBlockHash {
validation_error: response.validation_error.clone(),
});
}
PayloadStatusV1Status::InvalidTerminalBlock => {
// In the interests of being liberal with what we accept, only raise a
// warning here.
if response.latest_valid_hash.is_some() {
warn!(
log,
"Malformed response from execution engine";
"msg" => "expected a null latest_valid_hash",
"status" => ?response.status
)
}
invalid_statuses.push(PayloadStatus::InvalidTerminalBlock {
validation_error: response.validation_error.clone(),
});
}
PayloadStatusV1Status::Syncing => {
// In the interests of being liberal with what we accept, only raise a
// warning here.
if response.latest_valid_hash.is_some() {
warn!(
log,
"Malformed response from execution engine";
"msg" => "expected a null latest_valid_hash",
"status" => ?response.status
)
}
other_statuses.push(PayloadStatus::Syncing)
}
PayloadStatusV1Status::Accepted => {
// In the interests of being liberal with what we accept, only raise a
// warning here.
if response.latest_valid_hash.is_some() {
warn!(
log,
"Malformed response from execution engine";
"msg" => "expected a null latest_valid_hash",
"status" => ?response.status
)
}
other_statuses.push(PayloadStatus::Accepted)
}
},
}
}
if !valid_statuses.is_empty() && !invalid_statuses.is_empty() {
crit!(
log,
"Consensus failure between execution nodes";
"invalid_statuses" => ?invalid_statuses,
"valid_statuses" => ?valid_statuses,
);
// Choose to exit and ignore the valid response. This preferences correctness over
// liveness.
return Err(Error::ConsensusFailure);
}
// Log any errors to assist with troubleshooting.
for error in &errors {
warn!(
log,
"Error whilst processing payload status";
"error" => ?error,
);
}
valid_statuses
.first()
.or_else(|| invalid_statuses.first())
.or_else(|| other_statuses.first())
.cloned()
.map(Result::Ok)
.unwrap_or_else(|| Err(Error::EngineErrors(errors)))
}

View File

@ -1,4 +1,7 @@
use crate::engine_api::{
json_structures::{
JsonForkchoiceUpdatedV1Response, JsonPayloadStatusV1, JsonPayloadStatusV1Status,
},
ExecutionBlock, PayloadAttributes, PayloadId, PayloadStatusV1, PayloadStatusV1Status,
};
use crate::engines::ForkChoiceState;
@ -6,7 +9,7 @@ use serde::{Deserialize, Serialize};
use std::collections::HashMap;
use tree_hash::TreeHash;
use tree_hash_derive::TreeHash;
use types::{EthSpec, ExecutionPayload, Hash256, Uint256};
use types::{EthSpec, ExecutionBlockHash, ExecutionPayload, Hash256, Uint256};
const GAS_LIMIT: u64 = 16384;
const GAS_USED: u64 = GAS_LIMIT - 1;
@ -26,14 +29,14 @@ impl<T: EthSpec> Block<T> {
}
}
pub fn parent_hash(&self) -> Hash256 {
pub fn parent_hash(&self) -> ExecutionBlockHash {
match self {
Block::PoW(block) => block.parent_hash,
Block::PoS(payload) => payload.parent_hash,
}
}
pub fn block_hash(&self) -> Hash256 {
pub fn block_hash(&self) -> ExecutionBlockHash {
match self {
Block::PoW(block) => block.block_hash,
Block::PoS(payload) => payload.block_hash,
@ -69,8 +72,8 @@ impl<T: EthSpec> Block<T> {
#[serde(rename_all = "camelCase")]
pub struct PoWBlock {
pub block_number: u64,
pub block_hash: Hash256,
pub parent_hash: Hash256,
pub block_hash: ExecutionBlockHash,
pub parent_hash: ExecutionBlockHash,
pub total_difficulty: Uint256,
}
@ -78,18 +81,18 @@ pub struct ExecutionBlockGenerator<T: EthSpec> {
/*
* Common database
*/
blocks: HashMap<Hash256, Block<T>>,
block_hashes: HashMap<u64, Hash256>,
blocks: HashMap<ExecutionBlockHash, Block<T>>,
block_hashes: HashMap<u64, ExecutionBlockHash>,
/*
* PoW block parameters
*/
pub terminal_total_difficulty: Uint256,
pub terminal_block_number: u64,
pub terminal_block_hash: Hash256,
pub terminal_block_hash: ExecutionBlockHash,
/*
* PoS block parameters
*/
pub pending_payloads: HashMap<Hash256, ExecutionPayload<T>>,
pub pending_payloads: HashMap<ExecutionBlockHash, ExecutionPayload<T>>,
pub next_payload_id: u64,
pub payload_ids: HashMap<PayloadId, ExecutionPayload<T>>,
}
@ -98,7 +101,7 @@ impl<T: EthSpec> ExecutionBlockGenerator<T> {
pub fn new(
terminal_total_difficulty: Uint256,
terminal_block_number: u64,
terminal_block_hash: Hash256,
terminal_block_hash: ExecutionBlockHash,
) -> Self {
let mut gen = Self {
blocks: <_>::default(),
@ -141,11 +144,11 @@ impl<T: EthSpec> ExecutionBlockGenerator<T> {
.map(|block| block.as_execution_block(self.terminal_total_difficulty))
}
pub fn block_by_hash(&self, hash: Hash256) -> Option<Block<T>> {
pub fn block_by_hash(&self, hash: ExecutionBlockHash) -> Option<Block<T>> {
self.blocks.get(&hash).cloned()
}
pub fn execution_block_by_hash(&self, hash: Hash256) -> Option<ExecutionBlock> {
pub fn execution_block_by_hash(&self, hash: ExecutionBlockHash) -> Option<ExecutionBlock> {
self.block_by_hash(hash)
.map(|block| block.as_execution_block(self.terminal_total_difficulty))
}
@ -187,7 +190,7 @@ impl<T: EthSpec> ExecutionBlockGenerator<T> {
pub fn insert_pow_block(&mut self, block_number: u64) -> Result<(), String> {
let parent_hash = if block_number == 0 {
Hash256::zero()
ExecutionBlockHash::zero()
} else if let Some(hash) = self.block_hashes.get(&(block_number - 1)) {
*hash
} else {
@ -231,7 +234,7 @@ impl<T: EthSpec> ExecutionBlockGenerator<T> {
}
pub fn get_payload(&mut self, id: &PayloadId) -> Option<ExecutionPayload<T>> {
self.payload_ids.remove(id)
self.payload_ids.get(id).cloned()
}
pub fn new_payload(&mut self, payload: ExecutionPayload<T>) -> PayloadStatusV1 {
@ -267,39 +270,35 @@ impl<T: EthSpec> ExecutionBlockGenerator<T> {
&mut self,
forkchoice_state: ForkChoiceState,
payload_attributes: Option<PayloadAttributes>,
) -> Result<Option<PayloadId>, String> {
) -> Result<JsonForkchoiceUpdatedV1Response, String> {
if let Some(payload) = self
.pending_payloads
.remove(&forkchoice_state.head_block_hash)
{
self.insert_block(Block::PoS(payload))?;
}
if !self.blocks.contains_key(&forkchoice_state.head_block_hash) {
return Err(format!(
"block hash {:?} unknown",
forkchoice_state.head_block_hash
));
}
if !self.blocks.contains_key(&forkchoice_state.safe_block_hash) {
return Err(format!(
"block hash {:?} unknown",
forkchoice_state.head_block_hash
));
}
if forkchoice_state.finalized_block_hash != Hash256::zero()
let unknown_head_block_hash = !self.blocks.contains_key(&forkchoice_state.head_block_hash);
let unknown_safe_block_hash = !self.blocks.contains_key(&forkchoice_state.safe_block_hash);
let unknown_finalized_block_hash = forkchoice_state.finalized_block_hash
!= ExecutionBlockHash::zero()
&& !self
.blocks
.contains_key(&forkchoice_state.finalized_block_hash)
{
return Err(format!(
"finalized block hash {:?} is unknown",
forkchoice_state.finalized_block_hash
));
.contains_key(&forkchoice_state.finalized_block_hash);
if unknown_head_block_hash || unknown_safe_block_hash || unknown_finalized_block_hash {
return Ok(JsonForkchoiceUpdatedV1Response {
payload_status: JsonPayloadStatusV1 {
status: JsonPayloadStatusV1Status::Syncing,
latest_valid_hash: None,
validation_error: None,
},
payload_id: None,
});
}
match payload_attributes {
None => Ok(None),
let id = match payload_attributes {
None => None,
Some(attributes) => {
if !self.blocks.iter().any(|(_, block)| {
block.block_hash() == self.terminal_block_hash
@ -334,17 +333,27 @@ impl<T: EthSpec> ExecutionBlockGenerator<T> {
timestamp: attributes.timestamp,
extra_data: "block gen was here".as_bytes().to_vec().into(),
base_fee_per_gas: Uint256::one(),
block_hash: Hash256::zero(),
block_hash: ExecutionBlockHash::zero(),
transactions: vec![].into(),
};
execution_payload.block_hash = execution_payload.tree_hash_root();
execution_payload.block_hash =
ExecutionBlockHash::from_root(execution_payload.tree_hash_root());
self.payload_ids.insert(id, execution_payload);
Ok(Some(id))
Some(id)
}
}
};
Ok(JsonForkchoiceUpdatedV1Response {
payload_status: JsonPayloadStatusV1 {
status: JsonPayloadStatusV1Status::Valid,
latest_valid_hash: Some(forkchoice_state.head_block_hash),
validation_error: None,
},
payload_id: id.map(Into::into),
})
}
}
@ -356,7 +365,7 @@ pub fn generate_pow_block(
terminal_total_difficulty: Uint256,
terminal_block_number: u64,
block_number: u64,
parent_hash: Hash256,
parent_hash: ExecutionBlockHash,
) -> Result<PoWBlock, String> {
if block_number > terminal_block_number {
return Err(format!(
@ -378,12 +387,12 @@ pub fn generate_pow_block(
let mut block = PoWBlock {
block_number,
block_hash: Hash256::zero(),
block_hash: ExecutionBlockHash::zero(),
parent_hash,
total_difficulty,
};
block.block_hash = block.tree_hash_root();
block.block_hash = ExecutionBlockHash::from_root(block.tree_hash_root());
Ok(block)
}
@ -402,7 +411,7 @@ mod test {
let mut generator: ExecutionBlockGenerator<MainnetEthSpec> = ExecutionBlockGenerator::new(
TERMINAL_DIFFICULTY.into(),
TERMINAL_BLOCK,
Hash256::zero(),
ExecutionBlockHash::zero(),
);
for i in 0..=TERMINAL_BLOCK {
@ -420,7 +429,7 @@ mod test {
let expected_parent = i
.checked_sub(1)
.map(|i| generator.block_by_number(i).unwrap().block_hash())
.unwrap_or_else(Hash256::zero);
.unwrap_or_else(ExecutionBlockHash::zero);
assert_eq!(block.parent_hash(), expected_parent);
assert_eq!(

View File

@ -1,5 +1,5 @@
use super::Context;
use crate::engine_api::{http::*, PayloadStatusV1, PayloadStatusV1Status};
use crate::engine_api::{http::*, *};
use crate::json_structures::*;
use serde::de::DeserializeOwned;
use serde_json::Value as JsonValue;
@ -57,26 +57,29 @@ pub async fn handle_rpc<T: EthSpec>(
ENGINE_NEW_PAYLOAD_V1 => {
let request: JsonExecutionPayloadV1<T> = get_param(params, 0)?;
let response = if let Some(status) = *ctx.static_new_payload_response.lock() {
match status {
PayloadStatusV1Status::Valid => PayloadStatusV1 {
status,
latest_valid_hash: Some(request.block_hash),
validation_error: None,
},
PayloadStatusV1Status::Syncing => PayloadStatusV1 {
status,
latest_valid_hash: None,
validation_error: None,
},
_ => unimplemented!("invalid static newPayloadResponse"),
}
let (static_response, should_import) =
if let Some(mut response) = ctx.static_new_payload_response.lock().clone() {
if response.status.status == PayloadStatusV1Status::Valid {
response.status.latest_valid_hash = Some(request.block_hash)
}
(Some(response.status), response.should_import)
} else {
(None, true)
};
let dynamic_response = if should_import {
Some(
ctx.execution_block_generator
.write()
.new_payload(request.into()),
)
} else {
ctx.execution_block_generator
.write()
.new_payload(request.into())
None
};
let response = static_response.or(dynamic_response).unwrap();
Ok(serde_json::to_value(JsonPayloadStatusV1::from(response)).unwrap())
}
ENGINE_GET_PAYLOAD_V1 => {
@ -95,8 +98,7 @@ pub async fn handle_rpc<T: EthSpec>(
let forkchoice_state: JsonForkChoiceStateV1 = get_param(params, 0)?;
let payload_attributes: Option<JsonPayloadAttributesV1> = get_param(params, 1)?;
let head_block_hash = forkchoice_state.head_block_hash;
let id = ctx
let response = ctx
.execution_block_generator
.write()
.forkchoice_updated_v1(
@ -104,15 +106,7 @@ pub async fn handle_rpc<T: EthSpec>(
payload_attributes.map(|json| json.into()),
)?;
Ok(serde_json::to_value(JsonForkchoiceUpdatedV1Response {
payload_status: JsonPayloadStatusV1 {
status: JsonPayloadStatusV1Status::Valid,
latest_valid_hash: Some(head_block_hash),
validation_error: None,
},
payload_id: id.map(Into::into),
})
.unwrap())
Ok(serde_json::to_value(response).unwrap())
}
other => Err(format!(
"The method {} does not exist/is not available",

View File

@ -58,7 +58,7 @@ impl<T: EthSpec> MockExecutionLayer<T> {
Self::new(
DEFAULT_TERMINAL_DIFFICULTY.into(),
DEFAULT_TERMINAL_BLOCK,
Hash256::zero(),
ExecutionBlockHash::zero(),
Epoch::new(0),
)
}
@ -66,7 +66,7 @@ impl<T: EthSpec> MockExecutionLayer<T> {
pub fn new(
terminal_total_difficulty: Uint256,
terminal_block: u64,
terminal_block_hash: Hash256,
terminal_block_hash: ExecutionBlockHash,
terminal_block_hash_activation_epoch: Epoch,
) -> Self {
let el_runtime = ExecutionLayerRuntime::default();
@ -117,7 +117,7 @@ impl<T: EthSpec> MockExecutionLayer<T> {
self.el
.notify_forkchoice_updated(
parent_hash,
Hash256::zero(),
ExecutionBlockHash::zero(),
Some(PayloadAttributes {
timestamp,
random,
@ -145,13 +145,11 @@ impl<T: EthSpec> MockExecutionLayer<T> {
assert_eq!(payload.timestamp, timestamp);
assert_eq!(payload.random, random);
let (payload_response, latest_valid_hash) =
self.el.notify_new_payload(&payload).await.unwrap();
assert_eq!(payload_response, PayloadStatusV1Status::Valid);
assert_eq!(latest_valid_hash, Some(vec![payload.block_hash]));
let status = self.el.notify_new_payload(&payload).await.unwrap();
assert_eq!(status, PayloadStatus::Valid);
self.el
.notify_forkchoice_updated(block_hash, Hash256::zero(), None)
.notify_forkchoice_updated(block_hash, ExecutionBlockHash::zero(), None)
.await
.unwrap();

View File

@ -1,7 +1,6 @@
//! Provides a mock execution engine HTTP JSON-RPC API for use in testing.
use crate::engine_api::http::JSONRPC_VERSION;
use crate::engine_api::PayloadStatusV1Status;
use crate::engine_api::{http::JSONRPC_VERSION, PayloadStatusV1, PayloadStatusV1Status};
use bytes::Bytes;
use environment::null_logger;
use execution_block_generator::{Block, PoWBlock};
@ -15,7 +14,7 @@ use std::marker::PhantomData;
use std::net::{Ipv4Addr, SocketAddr, SocketAddrV4};
use std::sync::Arc;
use tokio::{runtime, sync::oneshot};
use types::{EthSpec, Hash256, Uint256};
use types::{EthSpec, ExecutionBlockHash, Uint256};
use warp::Filter;
pub use execution_block_generator::{generate_pow_block, ExecutionBlockGenerator};
@ -41,7 +40,7 @@ impl<T: EthSpec> MockServer<T> {
&runtime::Handle::current(),
DEFAULT_TERMINAL_DIFFICULTY.into(),
DEFAULT_TERMINAL_BLOCK,
Hash256::zero(),
ExecutionBlockHash::zero(),
)
}
@ -49,7 +48,7 @@ impl<T: EthSpec> MockServer<T> {
handle: &runtime::Handle,
terminal_difficulty: Uint256,
terminal_block: u64,
terminal_block_hash: Hash256,
terminal_block_hash: ExecutionBlockHash,
) -> Self {
let last_echo_request = Arc::new(RwLock::new(None));
let preloaded_responses = Arc::new(Mutex::new(vec![]));
@ -117,14 +116,54 @@ impl<T: EthSpec> MockServer<T> {
}
pub fn all_payloads_valid(&self) {
*self.ctx.static_new_payload_response.lock() = Some(PayloadStatusV1Status::Valid)
let response = StaticNewPayloadResponse {
status: PayloadStatusV1 {
status: PayloadStatusV1Status::Valid,
latest_valid_hash: None,
validation_error: None,
},
should_import: true,
};
*self.ctx.static_new_payload_response.lock() = Some(response)
}
/// Setting `should_import = true` simulates an EE that initially returns `SYNCING` but obtains
/// the block via it's own means (e.g., devp2p).
pub fn all_payloads_syncing(&self, should_import: bool) {
let response = StaticNewPayloadResponse {
status: PayloadStatusV1 {
status: PayloadStatusV1Status::Syncing,
latest_valid_hash: None,
validation_error: None,
},
should_import,
};
*self.ctx.static_new_payload_response.lock() = Some(response)
}
pub fn all_payloads_invalid(&self, latest_valid_hash: ExecutionBlockHash) {
let response = StaticNewPayloadResponse {
status: PayloadStatusV1 {
status: PayloadStatusV1Status::Invalid,
latest_valid_hash: Some(latest_valid_hash),
validation_error: Some("static response".into()),
},
should_import: true,
};
*self.ctx.static_new_payload_response.lock() = Some(response)
}
/// Disables any static payload response so the execution block generator will do its own
/// verification.
pub fn full_payload_verification(&self) {
*self.ctx.static_new_payload_response.lock() = None
}
pub fn insert_pow_block(
&self,
block_number: u64,
block_hash: Hash256,
parent_hash: Hash256,
block_hash: ExecutionBlockHash,
parent_hash: ExecutionBlockHash,
total_difficulty: Uint256,
) {
let block = Block::PoW(PoWBlock {
@ -143,7 +182,7 @@ impl<T: EthSpec> MockServer<T> {
.unwrap()
}
pub fn get_block(&self, block_hash: Hash256) -> Option<Block<T>> {
pub fn get_block(&self, block_hash: ExecutionBlockHash) -> Option<Block<T>> {
self.ctx
.execution_block_generator
.read()
@ -178,6 +217,12 @@ struct MissingIdField;
impl warp::reject::Reject for MissingIdField {}
#[derive(Debug, Clone, PartialEq)]
pub struct StaticNewPayloadResponse {
status: PayloadStatusV1,
should_import: bool,
}
/// A wrapper around all the items required to spawn the HTTP server.
///
/// The server will gracefully handle the case where any fields are `None`.
@ -187,7 +232,7 @@ pub struct Context<T: EthSpec> {
pub last_echo_request: Arc<RwLock<Option<Bytes>>>,
pub execution_block_generator: RwLock<ExecutionBlockGenerator<T>>,
pub preloaded_responses: Arc<Mutex<Vec<serde_json::Value>>>,
pub static_new_payload_response: Arc<Mutex<Option<PayloadStatusV1Status>>>,
pub static_new_payload_response: Arc<Mutex<Option<StaticNewPayloadResponse>>>,
pub _phantom: PhantomData<T>,
}

View File

@ -7,7 +7,7 @@ use std::sync::Weak;
use tokio::runtime::Runtime;
/// Provides a reason when Lighthouse is shut down.
#[derive(Copy, Clone, Debug)]
#[derive(Copy, Clone, Debug, PartialEq)]
pub enum ShutdownReason {
/// The node shut down successfully.
Success(&'static str),

View File

@ -6,8 +6,8 @@ use std::marker::PhantomData;
use std::time::Duration;
use types::{
consts::merge::INTERVALS_PER_SLOT, AttestationShufflingId, BeaconBlock, BeaconState,
BeaconStateError, ChainSpec, Checkpoint, Epoch, EthSpec, Hash256, IndexedAttestation,
RelativeEpoch, SignedBeaconBlock, Slot,
BeaconStateError, ChainSpec, Checkpoint, Epoch, EthSpec, ExecutionBlockHash, Hash256,
IndexedAttestation, RelativeEpoch, SignedBeaconBlock, Slot,
};
#[derive(Debug)]
@ -17,6 +17,7 @@ pub enum Error<T> {
ProtoArrayError(String),
InvalidProtoArrayBytes(String),
InvalidLegacyProtoArrayBytes(String),
FailedToProcessInvalidExecutionPayload(String),
MissingProtoArrayBlock(Hash256),
UnknownAncestor {
ancestor_slot: Slot,
@ -43,6 +44,12 @@ pub enum Error<T> {
block_root: Hash256,
payload_verification_status: PayloadVerificationStatus,
},
MissingJustifiedBlock {
justified_checkpoint: Checkpoint,
},
MissingFinalizedBlock {
finalized_checkpoint: Checkpoint,
},
}
impl<T> From<InvalidAttestation> for Error<T> {
@ -299,9 +306,15 @@ where
let execution_status = anchor_block.message_merge().map_or_else(
|()| ExecutionStatus::irrelevant(),
|message| {
// Assume that this payload is valid, since the anchor should be a trusted block and
// state.
ExecutionStatus::Valid(message.body.execution_payload.block_hash)
let execution_payload = &message.body.execution_payload;
if execution_payload == &<_>::default() {
// A default payload does not have execution enabled.
ExecutionStatus::irrelevant()
} else {
// Assume that this payload is valid, since the anchor should be a trusted block and
// state.
ExecutionStatus::Valid(message.body.execution_payload.block_hash)
}
},
);
@ -464,6 +477,17 @@ where
Ok(true)
}
/// See `ProtoArrayForkChoice::process_execution_payload_invalidation` for documentation.
pub fn on_invalid_execution_payload(
&mut self,
head_block_root: Hash256,
latest_valid_ancestor_root: Option<ExecutionBlockHash>,
) -> Result<(), Error<T::Error>> {
self.proto_array
.process_execution_payload_invalidation(head_block_root, latest_valid_ancestor_root)
.map_err(Error::FailedToProcessInvalidExecutionPayload)
}
/// Add `block` to the fork choice DAG.
///
/// - `block_root` is the root of `block.
@ -592,7 +616,7 @@ where
let execution_status = if let Ok(execution_payload) = block.body().execution_payload() {
let block_hash = execution_payload.block_hash;
if block_hash == Hash256::zero() {
if block_hash == ExecutionBlockHash::zero() {
// The block is post-merge-fork, but pre-terminal-PoW block. We don't need to verify
// the payload.
ExecutionStatus::irrelevant()
@ -875,6 +899,29 @@ where
}
}
/// Returns the `ProtoBlock` for the justified checkpoint.
///
/// ## Notes
///
/// This does *not* return the "best justified checkpoint". It returns the justified checkpoint
/// that is used for computing balances.
pub fn get_justified_block(&self) -> Result<ProtoBlock, Error<T::Error>> {
let justified_checkpoint = self.justified_checkpoint();
self.get_block(&justified_checkpoint.root)
.ok_or(Error::MissingJustifiedBlock {
justified_checkpoint,
})
}
/// Returns the `ProtoBlock` for the finalized checkpoint.
pub fn get_finalized_block(&self) -> Result<ProtoBlock, Error<T::Error>> {
let finalized_checkpoint = self.finalized_checkpoint();
self.get_block(&finalized_checkpoint.root)
.ok_or(Error::MissingFinalizedBlock {
finalized_checkpoint,
})
}
/// Return `true` if `block_root` is equal to the finalized root, or a known descendant of it.
pub fn is_descendant_of_finalized(&self, block_root: Hash256) -> bool {
self.proto_array

View File

@ -122,18 +122,24 @@ impl ForkChoiceTest {
}
/// Assert there was a shutdown signal sent by the beacon chain.
pub fn assert_shutdown_signal_sent(mut self) -> Self {
self.harness.shutdown_receiver.close();
let msg = self.harness.shutdown_receiver.try_next().unwrap();
assert!(msg.is_some());
pub fn shutdown_signal_sent(&self) -> bool {
let mutex = self.harness.shutdown_receiver.clone();
let mut shutdown_receiver = mutex.lock();
shutdown_receiver.close();
let msg = shutdown_receiver.try_next().unwrap();
msg.is_some()
}
/// Assert there was a shutdown signal sent by the beacon chain.
pub fn assert_shutdown_signal_sent(self) -> Self {
assert!(self.shutdown_signal_sent());
self
}
/// Assert no shutdown was signal sent by the beacon chain.
pub fn assert_shutdown_signal_not_sent(mut self) -> Self {
self.harness.shutdown_receiver.close();
let msg = self.harness.shutdown_receiver.try_next().unwrap();
assert!(msg.is_none());
pub fn assert_shutdown_signal_not_sent(self) -> Self {
assert!(!self.shutdown_signal_sent());
self
}
@ -479,6 +485,22 @@ fn is_safe_to_update(slot: Slot, spec: &ChainSpec) -> bool {
slot % E::slots_per_epoch() < spec.safe_slots_to_update_justified
}
#[test]
fn justified_and_finalized_blocks() {
let tester = ForkChoiceTest::new();
let fork_choice = tester.harness.chain.fork_choice.read();
let justified_checkpoint = fork_choice.justified_checkpoint();
assert_eq!(justified_checkpoint.epoch, 0);
assert!(justified_checkpoint.root != Hash256::zero());
assert!(fork_choice.get_justified_block().is_ok());
let finalized_checkpoint = fork_choice.finalized_checkpoint();
assert_eq!(finalized_checkpoint.epoch, 0);
assert!(finalized_checkpoint.root != Hash256::zero());
assert!(fork_choice.get_finalized_block().is_ok());
}
/// - The new justified checkpoint descends from the current.
/// - Current slot is within `SAFE_SLOTS_TO_UPDATE_JUSTIFIED`
#[test]

View File

@ -6,6 +6,18 @@ fn main() {
write_test_def_to_yaml("no_votes.yaml", get_no_votes_test_definition());
write_test_def_to_yaml("ffg_01.yaml", get_ffg_case_01_test_definition());
write_test_def_to_yaml("ffg_02.yaml", get_ffg_case_02_test_definition());
write_test_def_to_yaml(
"execution_status_01.yaml",
get_execution_status_test_definition_01(),
);
write_test_def_to_yaml(
"execution_status_02.yaml",
get_execution_status_test_definition_02(),
);
write_test_def_to_yaml(
"execution_status_03.yaml",
get_execution_status_test_definition_03(),
);
}
fn write_test_def_to_yaml(filename: &str, def: ForkChoiceTestDefinition) {

View File

@ -1,9 +1,10 @@
use types::{Checkpoint, Epoch, Hash256};
use types::{Checkpoint, Epoch, ExecutionBlockHash, Hash256};
#[derive(Clone, PartialEq, Debug)]
pub enum Error {
FinalizedNodeUnknown(Hash256),
JustifiedNodeUnknown(Hash256),
NodeUnknown(Hash256),
InvalidFinalizedRootChange,
InvalidNodeIndex(usize),
InvalidParentIndex(usize),
@ -15,6 +16,7 @@ pub enum Error {
DeltaOverflow(usize),
ProposerBoostOverflow(usize),
IndexOverflow(&'static str),
InvalidExecutionDeltaOverflow(usize),
InvalidDeltaLen {
deltas: usize,
indices: usize,
@ -26,7 +28,21 @@ pub enum Error {
InvalidBestNode(Box<InvalidBestNodeInfo>),
InvalidAncestorOfValidPayload {
ancestor_block_root: Hash256,
ancestor_payload_block_hash: Hash256,
ancestor_payload_block_hash: ExecutionBlockHash,
},
ValidExecutionStatusBecameInvalid {
block_root: Hash256,
payload_block_hash: ExecutionBlockHash,
},
InvalidJustifiedCheckpointExecutionStatus {
justified_root: Hash256,
},
UnknownLatestValidAncestorHash {
block_root: Hash256,
latest_valid_ancestor_hash: Option<ExecutionBlockHash>,
},
IrrelevantDescendant {
block_root: Hash256,
},
}

View File

@ -1,11 +1,16 @@
mod execution_status;
mod ffg_updates;
mod no_votes;
mod votes;
use crate::proto_array_fork_choice::{Block, ExecutionStatus, ProtoArrayForkChoice};
use serde_derive::{Deserialize, Serialize};
use types::{AttestationShufflingId, Checkpoint, Epoch, EthSpec, Hash256, MainnetEthSpec, Slot};
use types::{
AttestationShufflingId, Checkpoint, Epoch, EthSpec, ExecutionBlockHash, Hash256,
MainnetEthSpec, Slot,
};
pub use execution_status::*;
pub use ffg_updates::*;
pub use no_votes::*;
pub use votes::*;
@ -18,6 +23,13 @@ pub enum Operation {
justified_state_balances: Vec<u64>,
expected_head: Hash256,
},
ProposerBoostFindHead {
justified_checkpoint: Checkpoint,
finalized_checkpoint: Checkpoint,
justified_state_balances: Vec<u64>,
expected_head: Hash256,
proposer_boost_root: Hash256,
},
InvalidFindHead {
justified_checkpoint: Checkpoint,
finalized_checkpoint: Checkpoint,
@ -40,6 +52,14 @@ pub enum Operation {
prune_threshold: usize,
expected_len: usize,
},
InvalidatePayload {
head_block_root: Hash256,
latest_valid_ancestor_root: Option<ExecutionBlockHash>,
},
AssertWeight {
block_root: Hash256,
weight: u64,
},
}
#[derive(Debug, Clone, Serialize, Deserialize)]
@ -52,9 +72,11 @@ pub struct ForkChoiceTestDefinition {
impl ForkChoiceTestDefinition {
pub fn run(self) {
let mut spec = MainnetEthSpec::default_spec();
spec.proposer_score_boost = Some(50);
let junk_shuffling_id =
AttestationShufflingId::from_components(Epoch::new(0), Hash256::zero());
let execution_status = ExecutionStatus::irrelevant();
let mut fork_choice = ProtoArrayForkChoice::new(
self.finalized_block_slot,
Hash256::zero(),
@ -62,7 +84,7 @@ impl ForkChoiceTestDefinition {
self.finalized_checkpoint,
junk_shuffling_id.clone(),
junk_shuffling_id,
execution_status,
ExecutionStatus::Unknown(ExecutionBlockHash::zero()),
)
.expect("should create fork choice struct");
@ -80,7 +102,7 @@ impl ForkChoiceTestDefinition {
finalized_checkpoint,
&justified_state_balances,
Hash256::zero(),
&MainnetEthSpec::default_spec(),
&spec,
)
.map_err(|e| e)
.unwrap_or_else(|e| {
@ -89,7 +111,34 @@ impl ForkChoiceTestDefinition {
assert_eq!(
head, expected_head,
"Operation at index {} failed checks. Operation: {:?}",
"Operation at index {} failed head check. Operation: {:?}",
op_index, op
);
check_bytes_round_trip(&fork_choice);
}
Operation::ProposerBoostFindHead {
justified_checkpoint,
finalized_checkpoint,
justified_state_balances,
expected_head,
proposer_boost_root,
} => {
let head = fork_choice
.find_head::<MainnetEthSpec>(
justified_checkpoint,
finalized_checkpoint,
&justified_state_balances,
proposer_boost_root,
&spec,
)
.map_err(|e| e)
.unwrap_or_else(|e| {
panic!("find_head op at index {} returned error {}", op_index, e)
});
assert_eq!(
head, expected_head,
"Operation at index {} failed head check. Operation: {:?}",
op_index, op
);
check_bytes_round_trip(&fork_choice);
@ -104,7 +153,7 @@ impl ForkChoiceTestDefinition {
finalized_checkpoint,
&justified_state_balances,
Hash256::zero(),
&MainnetEthSpec::default_spec(),
&spec,
);
assert!(
@ -138,7 +187,10 @@ impl ForkChoiceTestDefinition {
),
justified_checkpoint,
finalized_checkpoint,
execution_status,
// All blocks are imported optimistically.
execution_status: ExecutionStatus::Unknown(ExecutionBlockHash::from_root(
root,
)),
};
fork_choice.process_block(block).unwrap_or_else(|e| {
panic!(
@ -183,22 +235,41 @@ impl ForkChoiceTestDefinition {
expected_len
);
}
Operation::InvalidatePayload {
head_block_root,
latest_valid_ancestor_root,
} => fork_choice
.process_execution_payload_invalidation(
head_block_root,
latest_valid_ancestor_root,
)
.unwrap(),
Operation::AssertWeight { block_root, weight } => assert_eq!(
fork_choice.get_weight(&block_root).unwrap(),
weight,
"block weight"
),
}
}
}
}
/// Gives a hash that is not the zero hash (unless i is `usize::max_value)`.
fn get_hash(i: u64) -> Hash256 {
/// Gives a root that is not the zero hash (unless i is `usize::max_value)`.
fn get_root(i: u64) -> Hash256 {
Hash256::from_low_u64_be(i + 1)
}
/// Gives a hash that is not the zero hash (unless i is `usize::max_value)`.
fn get_hash(i: u64) -> ExecutionBlockHash {
ExecutionBlockHash::from_root(get_root(i))
}
/// Gives a checkpoint with a root that is not the zero hash (unless i is `usize::max_value)`.
/// `Epoch` will always equal `i`.
fn get_checkpoint(i: u64) -> Checkpoint {
Checkpoint {
epoch: Epoch::new(i),
root: get_hash(i),
root: get_root(i),
}
}

File diff suppressed because it is too large Load Diff

View File

@ -9,7 +9,7 @@ pub fn get_ffg_case_01_test_definition() -> ForkChoiceTestDefinition {
justified_checkpoint: get_checkpoint(0),
finalized_checkpoint: get_checkpoint(0),
justified_state_balances: balances.clone(),
expected_head: get_hash(0),
expected_head: get_root(0),
});
// Build the following tree (stick? lol).
@ -23,22 +23,22 @@ pub fn get_ffg_case_01_test_definition() -> ForkChoiceTestDefinition {
// 3 <- just: 2, fin: 1
ops.push(Operation::ProcessBlock {
slot: Slot::new(1),
root: get_hash(1),
parent_root: get_hash(0),
root: get_root(1),
parent_root: get_root(0),
justified_checkpoint: get_checkpoint(0),
finalized_checkpoint: get_checkpoint(0),
});
ops.push(Operation::ProcessBlock {
slot: Slot::new(2),
root: get_hash(2),
parent_root: get_hash(1),
root: get_root(2),
parent_root: get_root(1),
justified_checkpoint: get_checkpoint(1),
finalized_checkpoint: get_checkpoint(0),
});
ops.push(Operation::ProcessBlock {
slot: Slot::new(3),
root: get_hash(3),
parent_root: get_hash(2),
root: get_root(3),
parent_root: get_root(2),
justified_checkpoint: get_checkpoint(2),
finalized_checkpoint: get_checkpoint(1),
});
@ -56,7 +56,7 @@ pub fn get_ffg_case_01_test_definition() -> ForkChoiceTestDefinition {
justified_checkpoint: get_checkpoint(0),
finalized_checkpoint: get_checkpoint(0),
justified_state_balances: balances.clone(),
expected_head: get_hash(3),
expected_head: get_root(3),
});
// Ensure that with justified epoch 1 we find 2
@ -72,7 +72,7 @@ pub fn get_ffg_case_01_test_definition() -> ForkChoiceTestDefinition {
justified_checkpoint: get_checkpoint(1),
finalized_checkpoint: get_checkpoint(0),
justified_state_balances: balances.clone(),
expected_head: get_hash(2),
expected_head: get_root(2),
});
// Ensure that with justified epoch 2 we find 3
@ -88,7 +88,7 @@ pub fn get_ffg_case_01_test_definition() -> ForkChoiceTestDefinition {
justified_checkpoint: get_checkpoint(2),
finalized_checkpoint: get_checkpoint(1),
justified_state_balances: balances,
expected_head: get_hash(3),
expected_head: get_root(3),
});
// END OF TESTS
@ -109,7 +109,7 @@ pub fn get_ffg_case_02_test_definition() -> ForkChoiceTestDefinition {
justified_checkpoint: get_checkpoint(0),
finalized_checkpoint: get_checkpoint(0),
justified_state_balances: balances.clone(),
expected_head: get_hash(0),
expected_head: get_root(0),
});
// Build the following tree.
@ -129,48 +129,48 @@ pub fn get_ffg_case_02_test_definition() -> ForkChoiceTestDefinition {
// Left branch
ops.push(Operation::ProcessBlock {
slot: Slot::new(1),
root: get_hash(1),
parent_root: get_hash(0),
root: get_root(1),
parent_root: get_root(0),
justified_checkpoint: get_checkpoint(0),
finalized_checkpoint: get_checkpoint(0),
});
ops.push(Operation::ProcessBlock {
slot: Slot::new(2),
root: get_hash(3),
parent_root: get_hash(1),
root: get_root(3),
parent_root: get_root(1),
justified_checkpoint: Checkpoint {
epoch: Epoch::new(1),
root: get_hash(1),
root: get_root(1),
},
finalized_checkpoint: get_checkpoint(0),
});
ops.push(Operation::ProcessBlock {
slot: Slot::new(3),
root: get_hash(5),
parent_root: get_hash(3),
root: get_root(5),
parent_root: get_root(3),
justified_checkpoint: Checkpoint {
epoch: Epoch::new(1),
root: get_hash(1),
root: get_root(1),
},
finalized_checkpoint: get_checkpoint(0),
});
ops.push(Operation::ProcessBlock {
slot: Slot::new(4),
root: get_hash(7),
parent_root: get_hash(5),
root: get_root(7),
parent_root: get_root(5),
justified_checkpoint: Checkpoint {
epoch: Epoch::new(1),
root: get_hash(1),
root: get_root(1),
},
finalized_checkpoint: get_checkpoint(0),
});
ops.push(Operation::ProcessBlock {
slot: Slot::new(5),
root: get_hash(9),
parent_root: get_hash(7),
root: get_root(9),
parent_root: get_root(7),
justified_checkpoint: Checkpoint {
epoch: Epoch::new(2),
root: get_hash(3),
root: get_root(3),
},
finalized_checkpoint: get_checkpoint(0),
});
@ -178,42 +178,42 @@ pub fn get_ffg_case_02_test_definition() -> ForkChoiceTestDefinition {
// Right branch
ops.push(Operation::ProcessBlock {
slot: Slot::new(1),
root: get_hash(2),
parent_root: get_hash(0),
root: get_root(2),
parent_root: get_root(0),
justified_checkpoint: get_checkpoint(0),
finalized_checkpoint: get_checkpoint(0),
});
ops.push(Operation::ProcessBlock {
slot: Slot::new(2),
root: get_hash(4),
parent_root: get_hash(2),
root: get_root(4),
parent_root: get_root(2),
justified_checkpoint: get_checkpoint(0),
finalized_checkpoint: get_checkpoint(0),
});
ops.push(Operation::ProcessBlock {
slot: Slot::new(3),
root: get_hash(6),
parent_root: get_hash(4),
root: get_root(6),
parent_root: get_root(4),
justified_checkpoint: get_checkpoint(0),
finalized_checkpoint: get_checkpoint(0),
});
ops.push(Operation::ProcessBlock {
slot: Slot::new(4),
root: get_hash(8),
parent_root: get_hash(6),
root: get_root(8),
parent_root: get_root(6),
justified_checkpoint: Checkpoint {
epoch: Epoch::new(1),
root: get_hash(2),
root: get_root(2),
},
finalized_checkpoint: get_checkpoint(0),
});
ops.push(Operation::ProcessBlock {
slot: Slot::new(5),
root: get_hash(10),
parent_root: get_hash(8),
root: get_root(10),
parent_root: get_root(8),
justified_checkpoint: Checkpoint {
epoch: Epoch::new(2),
root: get_hash(4),
root: get_root(4),
},
finalized_checkpoint: get_checkpoint(0),
});
@ -235,23 +235,23 @@ pub fn get_ffg_case_02_test_definition() -> ForkChoiceTestDefinition {
justified_checkpoint: get_checkpoint(0),
finalized_checkpoint: get_checkpoint(0),
justified_state_balances: balances.clone(),
expected_head: get_hash(10),
expected_head: get_root(10),
});
// Same as above, but with justified epoch 2.
ops.push(Operation::FindHead {
justified_checkpoint: Checkpoint {
epoch: Epoch::new(2),
root: get_hash(4),
root: get_root(4),
},
finalized_checkpoint: get_checkpoint(0),
justified_state_balances: balances.clone(),
expected_head: get_hash(10),
expected_head: get_root(10),
});
// Same as above, but with justified epoch 3 (should be invalid).
ops.push(Operation::InvalidFindHead {
justified_checkpoint: Checkpoint {
epoch: Epoch::new(3),
root: get_hash(6),
root: get_root(6),
},
finalized_checkpoint: get_checkpoint(0),
justified_state_balances: balances.clone(),
@ -272,7 +272,7 @@ pub fn get_ffg_case_02_test_definition() -> ForkChoiceTestDefinition {
// 9 10
ops.push(Operation::ProcessAttestation {
validator_index: 0,
block_root: get_hash(1),
block_root: get_root(1),
target_epoch: Epoch::new(0),
});
@ -293,23 +293,23 @@ pub fn get_ffg_case_02_test_definition() -> ForkChoiceTestDefinition {
justified_checkpoint: get_checkpoint(0),
finalized_checkpoint: get_checkpoint(0),
justified_state_balances: balances.clone(),
expected_head: get_hash(9),
expected_head: get_root(9),
});
// Save as above but justified epoch 2.
ops.push(Operation::FindHead {
justified_checkpoint: Checkpoint {
epoch: Epoch::new(2),
root: get_hash(3),
root: get_root(3),
},
finalized_checkpoint: get_checkpoint(0),
justified_state_balances: balances.clone(),
expected_head: get_hash(9),
expected_head: get_root(9),
});
// Save as above but justified epoch 3 (should fail).
ops.push(Operation::InvalidFindHead {
justified_checkpoint: Checkpoint {
epoch: Epoch::new(3),
root: get_hash(5),
root: get_root(5),
},
finalized_checkpoint: get_checkpoint(0),
justified_state_balances: balances.clone(),
@ -330,7 +330,7 @@ pub fn get_ffg_case_02_test_definition() -> ForkChoiceTestDefinition {
// 9 10
ops.push(Operation::ProcessAttestation {
validator_index: 1,
block_root: get_hash(2),
block_root: get_root(2),
target_epoch: Epoch::new(0),
});
@ -351,23 +351,23 @@ pub fn get_ffg_case_02_test_definition() -> ForkChoiceTestDefinition {
justified_checkpoint: get_checkpoint(0),
finalized_checkpoint: get_checkpoint(0),
justified_state_balances: balances.clone(),
expected_head: get_hash(10),
expected_head: get_root(10),
});
// Same as above but justified epoch 2.
ops.push(Operation::FindHead {
justified_checkpoint: Checkpoint {
epoch: Epoch::new(2),
root: get_hash(4),
root: get_root(4),
},
finalized_checkpoint: get_checkpoint(0),
justified_state_balances: balances.clone(),
expected_head: get_hash(10),
expected_head: get_root(10),
});
// Same as above but justified epoch 3 (should fail).
ops.push(Operation::InvalidFindHead {
justified_checkpoint: Checkpoint {
epoch: Epoch::new(3),
root: get_hash(6),
root: get_root(6),
},
finalized_checkpoint: get_checkpoint(0),
justified_state_balances: balances.clone(),
@ -389,27 +389,27 @@ pub fn get_ffg_case_02_test_definition() -> ForkChoiceTestDefinition {
ops.push(Operation::FindHead {
justified_checkpoint: Checkpoint {
epoch: Epoch::new(0),
root: get_hash(1),
root: get_root(1),
},
finalized_checkpoint: get_checkpoint(0),
justified_state_balances: balances.clone(),
expected_head: get_hash(9),
expected_head: get_root(9),
});
// Same as above but justified epoch 2.
ops.push(Operation::FindHead {
justified_checkpoint: Checkpoint {
epoch: Epoch::new(2),
root: get_hash(3),
root: get_root(3),
},
finalized_checkpoint: get_checkpoint(0),
justified_state_balances: balances.clone(),
expected_head: get_hash(9),
expected_head: get_root(9),
});
// Same as above but justified epoch 3 (should fail).
ops.push(Operation::InvalidFindHead {
justified_checkpoint: Checkpoint {
epoch: Epoch::new(3),
root: get_hash(5),
root: get_root(5),
},
finalized_checkpoint: get_checkpoint(0),
justified_state_balances: balances.clone(),
@ -432,23 +432,23 @@ pub fn get_ffg_case_02_test_definition() -> ForkChoiceTestDefinition {
justified_checkpoint: get_checkpoint(0),
finalized_checkpoint: get_checkpoint(0),
justified_state_balances: balances.clone(),
expected_head: get_hash(10),
expected_head: get_root(10),
});
// Same as above but justified epoch 2.
ops.push(Operation::FindHead {
justified_checkpoint: Checkpoint {
epoch: Epoch::new(2),
root: get_hash(4),
root: get_root(4),
},
finalized_checkpoint: get_checkpoint(0),
justified_state_balances: balances.clone(),
expected_head: get_hash(10),
expected_head: get_root(10),
});
// Same as above but justified epoch 3 (should fail).
ops.push(Operation::InvalidFindHead {
justified_checkpoint: Checkpoint {
epoch: Epoch::new(3),
root: get_hash(6),
root: get_root(6),
},
finalized_checkpoint: get_checkpoint(0),
justified_state_balances: balances,

View File

@ -24,7 +24,7 @@ pub fn get_no_votes_test_definition() -> ForkChoiceTestDefinition {
// 2
Operation::ProcessBlock {
slot: Slot::new(1),
root: get_hash(2),
root: get_root(2),
parent_root: Hash256::zero(),
justified_checkpoint: Checkpoint {
epoch: Epoch::new(1),
@ -50,7 +50,7 @@ pub fn get_no_votes_test_definition() -> ForkChoiceTestDefinition {
root: Hash256::zero(),
},
justified_state_balances: balances.clone(),
expected_head: get_hash(2),
expected_head: get_root(2),
},
// Add block 1
//
@ -59,8 +59,8 @@ pub fn get_no_votes_test_definition() -> ForkChoiceTestDefinition {
// 2 1
Operation::ProcessBlock {
slot: Slot::new(1),
root: get_hash(1),
parent_root: get_hash(0),
root: get_root(1),
parent_root: get_root(0),
justified_checkpoint: Checkpoint {
epoch: Epoch::new(1),
root: Hash256::zero(),
@ -85,7 +85,7 @@ pub fn get_no_votes_test_definition() -> ForkChoiceTestDefinition {
root: Hash256::zero(),
},
justified_state_balances: balances.clone(),
expected_head: get_hash(2),
expected_head: get_root(2),
},
// Add block 3
//
@ -96,8 +96,8 @@ pub fn get_no_votes_test_definition() -> ForkChoiceTestDefinition {
// 3
Operation::ProcessBlock {
slot: Slot::new(2),
root: get_hash(3),
parent_root: get_hash(1),
root: get_root(3),
parent_root: get_root(1),
justified_checkpoint: Checkpoint {
epoch: Epoch::new(1),
root: Hash256::zero(),
@ -124,7 +124,7 @@ pub fn get_no_votes_test_definition() -> ForkChoiceTestDefinition {
root: Hash256::zero(),
},
justified_state_balances: balances.clone(),
expected_head: get_hash(2),
expected_head: get_root(2),
},
// Add block 4
//
@ -135,8 +135,8 @@ pub fn get_no_votes_test_definition() -> ForkChoiceTestDefinition {
// 4 3
Operation::ProcessBlock {
slot: Slot::new(2),
root: get_hash(4),
parent_root: get_hash(2),
root: get_root(4),
parent_root: get_root(2),
justified_checkpoint: Checkpoint {
epoch: Epoch::new(1),
root: Hash256::zero(),
@ -163,7 +163,7 @@ pub fn get_no_votes_test_definition() -> ForkChoiceTestDefinition {
root: Hash256::zero(),
},
justified_state_balances: balances.clone(),
expected_head: get_hash(4),
expected_head: get_root(4),
},
// Add block 5 with a justified epoch of 2
//
@ -176,8 +176,8 @@ pub fn get_no_votes_test_definition() -> ForkChoiceTestDefinition {
// 5 <- justified epoch = 2
Operation::ProcessBlock {
slot: Slot::new(3),
root: get_hash(5),
parent_root: get_hash(4),
root: get_root(5),
parent_root: get_root(4),
justified_checkpoint: get_checkpoint(2),
finalized_checkpoint: Checkpoint {
epoch: Epoch::new(1),
@ -203,7 +203,7 @@ pub fn get_no_votes_test_definition() -> ForkChoiceTestDefinition {
root: Hash256::zero(),
},
justified_state_balances: balances.clone(),
expected_head: get_hash(4),
expected_head: get_root(4),
},
// Ensure there is an error when starting from a block that has the wrong justified epoch.
//
@ -217,7 +217,7 @@ pub fn get_no_votes_test_definition() -> ForkChoiceTestDefinition {
Operation::InvalidFindHead {
justified_checkpoint: Checkpoint {
epoch: Epoch::new(1),
root: get_hash(5),
root: get_root(5),
},
finalized_checkpoint: Checkpoint {
epoch: Epoch::new(1),
@ -241,7 +241,7 @@ pub fn get_no_votes_test_definition() -> ForkChoiceTestDefinition {
root: Hash256::zero(),
},
justified_state_balances: balances.clone(),
expected_head: get_hash(5),
expected_head: get_root(5),
},
// Add block 6
//
@ -256,8 +256,8 @@ pub fn get_no_votes_test_definition() -> ForkChoiceTestDefinition {
// 6
Operation::ProcessBlock {
slot: Slot::new(4),
root: get_hash(6),
parent_root: get_hash(5),
root: get_root(6),
parent_root: get_root(5),
justified_checkpoint: get_checkpoint(2),
finalized_checkpoint: Checkpoint {
epoch: Epoch::new(1),
@ -282,7 +282,7 @@ pub fn get_no_votes_test_definition() -> ForkChoiceTestDefinition {
root: Hash256::zero(),
},
justified_state_balances: balances,
expected_head: get_hash(6),
expected_head: get_root(6),
},
];

View File

@ -8,14 +8,14 @@ pub fn get_votes_test_definition() -> ForkChoiceTestDefinition {
ops.push(Operation::FindHead {
justified_checkpoint: Checkpoint {
epoch: Epoch::new(1),
root: get_hash(0),
root: get_root(0),
},
finalized_checkpoint: Checkpoint {
epoch: Epoch::new(1),
root: get_hash(0),
root: get_root(0),
},
justified_state_balances: balances.clone(),
expected_head: get_hash(0),
expected_head: get_root(0),
});
// Add a block with a hash of 2.
@ -25,15 +25,15 @@ pub fn get_votes_test_definition() -> ForkChoiceTestDefinition {
// 2
ops.push(Operation::ProcessBlock {
slot: Slot::new(1),
root: get_hash(2),
parent_root: get_hash(0),
root: get_root(2),
parent_root: get_root(0),
justified_checkpoint: Checkpoint {
epoch: Epoch::new(1),
root: get_hash(0),
root: get_root(0),
},
finalized_checkpoint: Checkpoint {
epoch: Epoch::new(1),
root: get_hash(0),
root: get_root(0),
},
});
@ -45,14 +45,14 @@ pub fn get_votes_test_definition() -> ForkChoiceTestDefinition {
ops.push(Operation::FindHead {
justified_checkpoint: Checkpoint {
epoch: Epoch::new(1),
root: get_hash(0),
root: get_root(0),
},
finalized_checkpoint: Checkpoint {
epoch: Epoch::new(1),
root: get_hash(0),
root: get_root(0),
},
justified_state_balances: balances.clone(),
expected_head: get_hash(2),
expected_head: get_root(2),
});
// Add a block with a hash of 1 that comes off the genesis block (this is a fork compared
@ -63,15 +63,15 @@ pub fn get_votes_test_definition() -> ForkChoiceTestDefinition {
// 2 1
ops.push(Operation::ProcessBlock {
slot: Slot::new(1),
root: get_hash(1),
parent_root: get_hash(0),
root: get_root(1),
parent_root: get_root(0),
justified_checkpoint: Checkpoint {
epoch: Epoch::new(1),
root: get_hash(0),
root: get_root(0),
},
finalized_checkpoint: Checkpoint {
epoch: Epoch::new(1),
root: get_hash(0),
root: get_root(0),
},
});
@ -83,14 +83,14 @@ pub fn get_votes_test_definition() -> ForkChoiceTestDefinition {
ops.push(Operation::FindHead {
justified_checkpoint: Checkpoint {
epoch: Epoch::new(1),
root: get_hash(0),
root: get_root(0),
},
finalized_checkpoint: Checkpoint {
epoch: Epoch::new(1),
root: get_hash(0),
root: get_root(0),
},
justified_state_balances: balances.clone(),
expected_head: get_hash(2),
expected_head: get_root(2),
});
// Add a vote to block 1
@ -100,7 +100,7 @@ pub fn get_votes_test_definition() -> ForkChoiceTestDefinition {
// 2 1 <- +vote
ops.push(Operation::ProcessAttestation {
validator_index: 0,
block_root: get_hash(1),
block_root: get_root(1),
target_epoch: Epoch::new(2),
});
@ -112,14 +112,14 @@ pub fn get_votes_test_definition() -> ForkChoiceTestDefinition {
ops.push(Operation::FindHead {
justified_checkpoint: Checkpoint {
epoch: Epoch::new(1),
root: get_hash(0),
root: get_root(0),
},
finalized_checkpoint: Checkpoint {
epoch: Epoch::new(1),
root: get_hash(0),
root: get_root(0),
},
justified_state_balances: balances.clone(),
expected_head: get_hash(1),
expected_head: get_root(1),
});
// Add a vote to block 2
@ -129,7 +129,7 @@ pub fn get_votes_test_definition() -> ForkChoiceTestDefinition {
// +vote-> 2 1
ops.push(Operation::ProcessAttestation {
validator_index: 1,
block_root: get_hash(2),
block_root: get_root(2),
target_epoch: Epoch::new(2),
});
@ -141,14 +141,14 @@ pub fn get_votes_test_definition() -> ForkChoiceTestDefinition {
ops.push(Operation::FindHead {
justified_checkpoint: Checkpoint {
epoch: Epoch::new(1),
root: get_hash(0),
root: get_root(0),
},
finalized_checkpoint: Checkpoint {
epoch: Epoch::new(1),
root: get_hash(0),
root: get_root(0),
},
justified_state_balances: balances.clone(),
expected_head: get_hash(2),
expected_head: get_root(2),
});
// Add block 3.
@ -160,15 +160,15 @@ pub fn get_votes_test_definition() -> ForkChoiceTestDefinition {
// 3
ops.push(Operation::ProcessBlock {
slot: Slot::new(2),
root: get_hash(3),
parent_root: get_hash(1),
root: get_root(3),
parent_root: get_root(1),
justified_checkpoint: Checkpoint {
epoch: Epoch::new(1),
root: get_hash(0),
root: get_root(0),
},
finalized_checkpoint: Checkpoint {
epoch: Epoch::new(1),
root: get_hash(0),
root: get_root(0),
},
});
@ -182,14 +182,14 @@ pub fn get_votes_test_definition() -> ForkChoiceTestDefinition {
ops.push(Operation::FindHead {
justified_checkpoint: Checkpoint {
epoch: Epoch::new(1),
root: get_hash(0),
root: get_root(0),
},
finalized_checkpoint: Checkpoint {
epoch: Epoch::new(1),
root: get_hash(0),
root: get_root(0),
},
justified_state_balances: balances.clone(),
expected_head: get_hash(2),
expected_head: get_root(2),
});
// Move validator #0 vote from 1 to 3
@ -201,7 +201,7 @@ pub fn get_votes_test_definition() -> ForkChoiceTestDefinition {
// 3 <- +vote
ops.push(Operation::ProcessAttestation {
validator_index: 0,
block_root: get_hash(3),
block_root: get_root(3),
target_epoch: Epoch::new(3),
});
@ -215,14 +215,14 @@ pub fn get_votes_test_definition() -> ForkChoiceTestDefinition {
ops.push(Operation::FindHead {
justified_checkpoint: Checkpoint {
epoch: Epoch::new(1),
root: get_hash(0),
root: get_root(0),
},
finalized_checkpoint: Checkpoint {
epoch: Epoch::new(1),
root: get_hash(0),
root: get_root(0),
},
justified_state_balances: balances.clone(),
expected_head: get_hash(2),
expected_head: get_root(2),
});
// Move validator #1 vote from 2 to 1 (this is an equivocation, but fork choice doesn't
@ -235,7 +235,7 @@ pub fn get_votes_test_definition() -> ForkChoiceTestDefinition {
// 3
ops.push(Operation::ProcessAttestation {
validator_index: 1,
block_root: get_hash(1),
block_root: get_root(1),
target_epoch: Epoch::new(3),
});
@ -249,14 +249,14 @@ pub fn get_votes_test_definition() -> ForkChoiceTestDefinition {
ops.push(Operation::FindHead {
justified_checkpoint: Checkpoint {
epoch: Epoch::new(1),
root: get_hash(0),
root: get_root(0),
},
finalized_checkpoint: Checkpoint {
epoch: Epoch::new(1),
root: get_hash(0),
root: get_root(0),
},
justified_state_balances: balances.clone(),
expected_head: get_hash(3),
expected_head: get_root(3),
});
// Add block 4.
@ -270,15 +270,15 @@ pub fn get_votes_test_definition() -> ForkChoiceTestDefinition {
// 4
ops.push(Operation::ProcessBlock {
slot: Slot::new(3),
root: get_hash(4),
parent_root: get_hash(3),
root: get_root(4),
parent_root: get_root(3),
justified_checkpoint: Checkpoint {
epoch: Epoch::new(1),
root: get_hash(0),
root: get_root(0),
},
finalized_checkpoint: Checkpoint {
epoch: Epoch::new(1),
root: get_hash(0),
root: get_root(0),
},
});
@ -294,14 +294,14 @@ pub fn get_votes_test_definition() -> ForkChoiceTestDefinition {
ops.push(Operation::FindHead {
justified_checkpoint: Checkpoint {
epoch: Epoch::new(1),
root: get_hash(0),
root: get_root(0),
},
finalized_checkpoint: Checkpoint {
epoch: Epoch::new(1),
root: get_hash(0),
root: get_root(0),
},
justified_state_balances: balances.clone(),
expected_head: get_hash(4),
expected_head: get_root(4),
});
// Add block 5, which has a justified epoch of 2.
@ -317,15 +317,15 @@ pub fn get_votes_test_definition() -> ForkChoiceTestDefinition {
// 5 <- justified epoch = 2
ops.push(Operation::ProcessBlock {
slot: Slot::new(4),
root: get_hash(5),
parent_root: get_hash(4),
root: get_root(5),
parent_root: get_root(4),
justified_checkpoint: Checkpoint {
epoch: Epoch::new(2),
root: get_hash(1),
root: get_root(1),
},
finalized_checkpoint: Checkpoint {
epoch: Epoch::new(2),
root: get_hash(1),
root: get_root(1),
},
});
@ -343,14 +343,14 @@ pub fn get_votes_test_definition() -> ForkChoiceTestDefinition {
ops.push(Operation::FindHead {
justified_checkpoint: Checkpoint {
epoch: Epoch::new(1),
root: get_hash(0),
root: get_root(0),
},
finalized_checkpoint: Checkpoint {
epoch: Epoch::new(1),
root: get_hash(0),
root: get_root(0),
},
justified_state_balances: balances.clone(),
expected_head: get_hash(4),
expected_head: get_root(4),
});
// Add block 6, which has a justified epoch of 0.
@ -366,15 +366,15 @@ pub fn get_votes_test_definition() -> ForkChoiceTestDefinition {
// 5 6 <- justified epoch = 0
ops.push(Operation::ProcessBlock {
slot: Slot::new(0),
root: get_hash(6),
parent_root: get_hash(4),
root: get_root(6),
parent_root: get_root(4),
justified_checkpoint: Checkpoint {
epoch: Epoch::new(1),
root: get_hash(0),
root: get_root(0),
},
finalized_checkpoint: Checkpoint {
epoch: Epoch::new(1),
root: get_hash(0),
root: get_root(0),
},
});
@ -391,12 +391,12 @@ pub fn get_votes_test_definition() -> ForkChoiceTestDefinition {
// +2 vote-> 5 6
ops.push(Operation::ProcessAttestation {
validator_index: 0,
block_root: get_hash(5),
block_root: get_root(5),
target_epoch: Epoch::new(4),
});
ops.push(Operation::ProcessAttestation {
validator_index: 1,
block_root: get_hash(5),
block_root: get_root(5),
target_epoch: Epoch::new(4),
});
@ -420,41 +420,41 @@ pub fn get_votes_test_definition() -> ForkChoiceTestDefinition {
// 9
ops.push(Operation::ProcessBlock {
slot: Slot::new(0),
root: get_hash(7),
parent_root: get_hash(5),
root: get_root(7),
parent_root: get_root(5),
justified_checkpoint: Checkpoint {
epoch: Epoch::new(2),
root: get_hash(5),
root: get_root(5),
},
finalized_checkpoint: Checkpoint {
epoch: Epoch::new(2),
root: get_hash(5),
root: get_root(5),
},
});
ops.push(Operation::ProcessBlock {
slot: Slot::new(0),
root: get_hash(8),
parent_root: get_hash(7),
root: get_root(8),
parent_root: get_root(7),
justified_checkpoint: Checkpoint {
epoch: Epoch::new(2),
root: get_hash(5),
root: get_root(5),
},
finalized_checkpoint: Checkpoint {
epoch: Epoch::new(2),
root: get_hash(5),
root: get_root(5),
},
});
ops.push(Operation::ProcessBlock {
slot: Slot::new(0),
root: get_hash(9),
parent_root: get_hash(8),
root: get_root(9),
parent_root: get_root(8),
justified_checkpoint: Checkpoint {
epoch: Epoch::new(2),
root: get_hash(5),
root: get_root(5),
},
finalized_checkpoint: Checkpoint {
epoch: Epoch::new(2),
root: get_hash(5),
root: get_root(5),
},
});
@ -479,14 +479,14 @@ pub fn get_votes_test_definition() -> ForkChoiceTestDefinition {
ops.push(Operation::FindHead {
justified_checkpoint: Checkpoint {
epoch: Epoch::new(1),
root: get_hash(0),
root: get_root(0),
},
finalized_checkpoint: Checkpoint {
epoch: Epoch::new(1),
root: get_hash(0),
root: get_root(0),
},
justified_state_balances: balances.clone(),
expected_head: get_hash(6),
expected_head: get_root(6),
});
// Change fork-choice justified epoch to 1, and the start block to 5 and ensure that 9 is
@ -512,14 +512,14 @@ pub fn get_votes_test_definition() -> ForkChoiceTestDefinition {
ops.push(Operation::FindHead {
justified_checkpoint: Checkpoint {
epoch: Epoch::new(2),
root: get_hash(5),
root: get_root(5),
},
finalized_checkpoint: Checkpoint {
epoch: Epoch::new(2),
root: get_hash(5),
root: get_root(5),
},
justified_state_balances: balances.clone(),
expected_head: get_hash(9),
expected_head: get_root(9),
});
// Change fork-choice justified epoch to 1, and the start block to 5 and ensure that 9 is
@ -544,12 +544,12 @@ pub fn get_votes_test_definition() -> ForkChoiceTestDefinition {
// 9 <- +2 votes
ops.push(Operation::ProcessAttestation {
validator_index: 0,
block_root: get_hash(9),
block_root: get_root(9),
target_epoch: Epoch::new(5),
});
ops.push(Operation::ProcessAttestation {
validator_index: 1,
block_root: get_hash(9),
block_root: get_root(9),
target_epoch: Epoch::new(5),
});
@ -572,15 +572,15 @@ pub fn get_votes_test_definition() -> ForkChoiceTestDefinition {
// 9 10
ops.push(Operation::ProcessBlock {
slot: Slot::new(0),
root: get_hash(10),
parent_root: get_hash(8),
root: get_root(10),
parent_root: get_root(8),
justified_checkpoint: Checkpoint {
epoch: Epoch::new(2),
root: get_hash(5),
root: get_root(5),
},
finalized_checkpoint: Checkpoint {
epoch: Epoch::new(2),
root: get_hash(5),
root: get_root(5),
},
});
@ -588,14 +588,14 @@ pub fn get_votes_test_definition() -> ForkChoiceTestDefinition {
ops.push(Operation::FindHead {
justified_checkpoint: Checkpoint {
epoch: Epoch::new(2),
root: get_hash(5),
root: get_root(5),
},
finalized_checkpoint: Checkpoint {
epoch: Epoch::new(2),
root: get_hash(5),
root: get_root(5),
},
justified_state_balances: balances.clone(),
expected_head: get_hash(9),
expected_head: get_root(9),
});
// Introduce 2 more validators into the system
@ -620,12 +620,12 @@ pub fn get_votes_test_definition() -> ForkChoiceTestDefinition {
// 9 10 <- +2 votes
ops.push(Operation::ProcessAttestation {
validator_index: 2,
block_root: get_hash(10),
block_root: get_root(10),
target_epoch: Epoch::new(5),
});
ops.push(Operation::ProcessAttestation {
validator_index: 3,
block_root: get_hash(10),
block_root: get_root(10),
target_epoch: Epoch::new(5),
});
@ -649,14 +649,14 @@ pub fn get_votes_test_definition() -> ForkChoiceTestDefinition {
ops.push(Operation::FindHead {
justified_checkpoint: Checkpoint {
epoch: Epoch::new(2),
root: get_hash(5),
root: get_root(5),
},
finalized_checkpoint: Checkpoint {
epoch: Epoch::new(2),
root: get_hash(5),
root: get_root(5),
},
justified_state_balances: balances.clone(),
expected_head: get_hash(10),
expected_head: get_root(10),
});
// Set the balances of the last two validators to zero
@ -674,14 +674,14 @@ pub fn get_votes_test_definition() -> ForkChoiceTestDefinition {
ops.push(Operation::FindHead {
justified_checkpoint: Checkpoint {
epoch: Epoch::new(2),
root: get_hash(5),
root: get_root(5),
},
finalized_checkpoint: Checkpoint {
epoch: Epoch::new(2),
root: get_hash(5),
root: get_root(5),
},
justified_state_balances: balances.clone(),
expected_head: get_hash(9),
expected_head: get_root(9),
});
// Set the balances of the last two validators back to 1
@ -699,14 +699,14 @@ pub fn get_votes_test_definition() -> ForkChoiceTestDefinition {
ops.push(Operation::FindHead {
justified_checkpoint: Checkpoint {
epoch: Epoch::new(2),
root: get_hash(5),
root: get_root(5),
},
finalized_checkpoint: Checkpoint {
epoch: Epoch::new(2),
root: get_hash(5),
root: get_root(5),
},
justified_state_balances: balances.clone(),
expected_head: get_hash(10),
expected_head: get_root(10),
});
// Remove the last two validators
@ -725,19 +725,19 @@ pub fn get_votes_test_definition() -> ForkChoiceTestDefinition {
ops.push(Operation::FindHead {
justified_checkpoint: Checkpoint {
epoch: Epoch::new(2),
root: get_hash(5),
root: get_root(5),
},
finalized_checkpoint: Checkpoint {
epoch: Epoch::new(2),
root: get_hash(5),
root: get_root(5),
},
justified_state_balances: balances.clone(),
expected_head: get_hash(9),
expected_head: get_root(9),
});
// Ensure that pruning below the prune threshold does not prune.
ops.push(Operation::Prune {
finalized_root: get_hash(5),
finalized_root: get_root(5),
prune_threshold: usize::max_value(),
expected_len: 11,
});
@ -746,14 +746,14 @@ pub fn get_votes_test_definition() -> ForkChoiceTestDefinition {
ops.push(Operation::FindHead {
justified_checkpoint: Checkpoint {
epoch: Epoch::new(2),
root: get_hash(5),
root: get_root(5),
},
finalized_checkpoint: Checkpoint {
epoch: Epoch::new(2),
root: get_hash(5),
root: get_root(5),
},
justified_state_balances: balances.clone(),
expected_head: get_hash(9),
expected_head: get_root(9),
});
// Ensure that pruning above the prune threshold does prune.
@ -775,7 +775,7 @@ pub fn get_votes_test_definition() -> ForkChoiceTestDefinition {
// / \
// 9 10
ops.push(Operation::Prune {
finalized_root: get_hash(5),
finalized_root: get_root(5),
prune_threshold: 1,
expected_len: 6,
});
@ -784,14 +784,14 @@ pub fn get_votes_test_definition() -> ForkChoiceTestDefinition {
ops.push(Operation::FindHead {
justified_checkpoint: Checkpoint {
epoch: Epoch::new(2),
root: get_hash(5),
root: get_root(5),
},
finalized_checkpoint: Checkpoint {
epoch: Epoch::new(2),
root: get_hash(5),
root: get_root(5),
},
justified_state_balances: balances.clone(),
expected_head: get_hash(9),
expected_head: get_root(9),
});
// Add block 11
@ -807,15 +807,15 @@ pub fn get_votes_test_definition() -> ForkChoiceTestDefinition {
// 11
ops.push(Operation::ProcessBlock {
slot: Slot::new(0),
root: get_hash(11),
parent_root: get_hash(9),
root: get_root(11),
parent_root: get_root(9),
justified_checkpoint: Checkpoint {
epoch: Epoch::new(2),
root: get_hash(5),
root: get_root(5),
},
finalized_checkpoint: Checkpoint {
epoch: Epoch::new(2),
root: get_hash(5),
root: get_root(5),
},
});
@ -833,25 +833,25 @@ pub fn get_votes_test_definition() -> ForkChoiceTestDefinition {
ops.push(Operation::FindHead {
justified_checkpoint: Checkpoint {
epoch: Epoch::new(2),
root: get_hash(5),
root: get_root(5),
},
finalized_checkpoint: Checkpoint {
epoch: Epoch::new(2),
root: get_hash(5),
root: get_root(5),
},
justified_state_balances: balances,
expected_head: get_hash(11),
expected_head: get_root(11),
});
ForkChoiceTestDefinition {
finalized_block_slot: Slot::new(0),
justified_checkpoint: Checkpoint {
epoch: Epoch::new(1),
root: get_hash(0),
root: get_root(0),
},
finalized_checkpoint: Checkpoint {
epoch: Epoch::new(1),
root: get_hash(0),
root: get_root(0),
},
operations: ops,
}

View File

@ -4,8 +4,11 @@ use serde_derive::{Deserialize, Serialize};
use ssz::four_byte_option_impl;
use ssz::Encode;
use ssz_derive::{Decode, Encode};
use std::collections::HashMap;
use types::{AttestationShufflingId, ChainSpec, Checkpoint, Epoch, EthSpec, Hash256, Slot};
use std::collections::{HashMap, HashSet};
use types::{
AttestationShufflingId, ChainSpec, Checkpoint, Epoch, EthSpec, ExecutionBlockHash, Hash256,
Slot,
};
// Define a "legacy" implementation of `Option<usize>` which uses four bytes for encoding the union
// selector.
@ -126,26 +129,42 @@ impl ProtoArray {
continue;
}
let mut node_delta = deltas
.get(node_index)
.copied()
.ok_or(Error::InvalidNodeDelta(node_index))?;
let execution_status_is_invalid = node.execution_status.is_invalid();
let mut node_delta = if execution_status_is_invalid {
// If the node has an invalid execution payload, reduce its weight to zero.
0_i64
.checked_sub(node.weight as i64)
.ok_or(Error::InvalidExecutionDeltaOverflow(node_index))?
} else {
deltas
.get(node_index)
.copied()
.ok_or(Error::InvalidNodeDelta(node_index))?
};
// If we find the node for which the proposer boost was previously applied, decrease
// the delta by the previous score amount.
if self.previous_proposer_boost.root != Hash256::zero()
&& self.previous_proposer_boost.root == node.root
// Invalid nodes will always have a weight of zero so there's no need to subtract
// the proposer boost delta.
&& !execution_status_is_invalid
{
node_delta = node_delta
.checked_sub(self.previous_proposer_boost.score as i64)
.ok_or(Error::DeltaOverflow(node_index))?;
}
// If we find the node matching the current proposer boost root, increase
// the delta by the new score amount.
// the delta by the new score amount (unless the block has an invalid execution status).
//
// https://github.com/ethereum/consensus-specs/blob/dev/specs/phase0/fork-choice.md#get_latest_attesting_balance
if let Some(proposer_score_boost) = spec.proposer_score_boost {
if proposer_boost_root != Hash256::zero() && proposer_boost_root == node.root {
if proposer_boost_root != Hash256::zero()
&& proposer_boost_root == node.root
// Invalid nodes (or their ancestors) should not receive a proposer boost.
&& !execution_status_is_invalid
{
proposer_score =
calculate_proposer_boost::<E>(new_balances, proposer_score_boost)
.ok_or(Error::ProposerBoostOverflow(node_index))?;
@ -156,7 +175,10 @@ impl ProtoArray {
}
// Apply the delta to the node.
if node_delta < 0 {
if execution_status_is_invalid {
// Invalid nodes always have a weight of 0.
node.weight = 0
} else if node_delta < 0 {
// Note: I am conflicted about whether to use `saturating_sub` or `checked_sub`
// here.
//
@ -250,14 +272,20 @@ impl ProtoArray {
self.maybe_update_best_child_and_descendant(parent_index, node_index)?;
if matches!(block.execution_status, ExecutionStatus::Valid(_)) {
self.propagate_execution_payload_verification(parent_index)?;
self.propagate_execution_payload_validation(parent_index)?;
}
}
Ok(())
}
pub fn propagate_execution_payload_verification(
/// Updates the `verified_node_index` and all ancestors to have validated execution payloads.
///
/// Returns an error if:
///
/// - The `verified_node_index` is unknown.
/// - Any of the to-be-validated payloads are already invalid.
pub fn propagate_execution_payload_validation(
&mut self,
verified_node_index: usize,
) -> Result<(), Error> {
@ -300,6 +328,213 @@ impl ProtoArray {
}
}
/// Invalidate the relevant ancestors and descendants of a block with an invalid execution
/// payload.
///
/// The `head_block_root` should be the beacon block root of the block with the invalid
/// execution payload, _or_ its parent where the block with the invalid payload has not yet
/// been applied to `self`.
///
/// The `latest_valid_hash` should be the hash of most recent *valid* execution payload
/// contained in an ancestor block of `head_block_root`.
///
/// This function will invalidate:
///
/// * The block matching `head_block_root` _unless_ that block has a payload matching `latest_valid_hash`.
/// * All ancestors of `head_block_root` back to the block with payload matching
/// `latest_valid_hash` (endpoint > exclusive). In the case where the `head_block_root` is the parent
/// of the invalid block and itself matches `latest_valid_hash`, no ancestors will be invalidated.
/// * All descendants of `latest_valid_hash` if supplied and consistent with `head_block_root`,
/// or else all descendants of `head_block_root`.
///
/// ## Details
///
/// If `head_block_root` is not known to fork choice, an error is returned.
///
/// If `latest_valid_hash` is `Some(hash)` where `hash` is either not known to fork choice
/// (perhaps it's junk or pre-finalization), then only the `head_block_root` block will be
/// invalidated (no ancestors). No error will be returned in this case.
///
/// If `latest_valid_hash` is `Some(hash)` where `hash` is a known ancestor of
/// `head_block_root`, then all blocks between `head_block_root` and `latest_valid_hash` will
/// be invalidated. Additionally, all blocks that descend from a newly-invalidated block will
/// also be invalidated.
pub fn propagate_execution_payload_invalidation(
&mut self,
head_block_root: Hash256,
latest_valid_ancestor_hash: Option<ExecutionBlockHash>,
) -> Result<(), Error> {
let mut invalidated_indices: HashSet<usize> = <_>::default();
/*
* Step 1:
*
* Find the `head_block_root` and maybe iterate backwards and invalidate ancestors. Record
* all invalidated block indices in `invalidated_indices`.
*/
let mut index = *self
.indices
.get(&head_block_root)
.ok_or(Error::NodeUnknown(head_block_root))?;
// Try to map the ancestor payload *hash* to an ancestor beacon block *root*.
let latest_valid_ancestor_root = latest_valid_ancestor_hash
.and_then(|hash| self.execution_block_hash_to_beacon_block_root(&hash));
// Set to `true` if both conditions are satisfied:
//
// 1. The `head_block_root` is a descendant of `latest_valid_ancestor_hash`
// 2. The `latest_valid_ancestor_hash` is equal to or a descendant of the finalized block.
let latest_valid_ancestor_is_descendant =
latest_valid_ancestor_root.map_or(false, |ancestor_root| {
self.is_descendant(ancestor_root, head_block_root)
&& self.is_descendant(self.finalized_checkpoint.root, ancestor_root)
});
// Collect all *ancestors* which were declared invalid since they reside between the
// `head_block_root` and the `latest_valid_ancestor_root`.
loop {
let node = self
.nodes
.get_mut(index)
.ok_or(Error::InvalidNodeIndex(index))?;
match node.execution_status {
ExecutionStatus::Valid(hash)
| ExecutionStatus::Invalid(hash)
| ExecutionStatus::Unknown(hash) => {
// If we're no longer processing the `head_block_root` and the last valid
// ancestor is unknown, exit this loop and proceed to invalidate and
// descendants of `head_block_root`/`latest_valid_ancestor_root`.
//
// In effect, this means that if an unknown hash (junk or pre-finalization) is
// supplied, don't validate any ancestors. The alternative is to invalidate
// *all* ancestors, which would likely involve shutting down the client due to
// an invalid justified checkpoint.
if !latest_valid_ancestor_is_descendant && node.root != head_block_root {
break;
} else if Some(hash) == latest_valid_ancestor_hash {
// If the `best_child` or `best_descendant` of the latest valid hash was
// invalidated, set those fields to `None`.
//
// In theory, an invalid `best_child` necessarily infers an invalid
// `best_descendant`. However, we check each variable independently to
// defend against errors which might result in an invalid block being set as
// head.
if node
.best_child
.map_or(false, |i| invalidated_indices.contains(&i))
{
node.best_child = None
}
if node
.best_descendant
.map_or(false, |i| invalidated_indices.contains(&i))
{
node.best_descendant = None
}
// It might be new knowledge that this block is valid, ensure that it and all
// ancestors are marked as valid.
self.propagate_execution_payload_validation(index)?;
break;
}
}
ExecutionStatus::Irrelevant(_) => break,
}
match &node.execution_status {
// It's illegal for an execution client to declare that some previously-valid block
// is now invalid. This is a consensus failure on their behalf.
ExecutionStatus::Valid(hash) => {
return Err(Error::ValidExecutionStatusBecameInvalid {
block_root: node.root,
payload_block_hash: *hash,
})
}
ExecutionStatus::Unknown(hash) => {
node.execution_status = ExecutionStatus::Invalid(*hash);
// It's impossible for an invalid block to lead to a "best" block, so set these
// fields to `None`.
//
// Failing to set these values will result in `Self::node_leads_to_viable_head`
// returning `false` for *valid* ancestors of invalid blocks.
node.best_child = None;
node.best_descendant = None;
}
// The block is already invalid, but keep going backwards to ensure all ancestors
// are updated.
ExecutionStatus::Invalid(_) => (),
// This block is pre-merge, therefore it has no execution status. Nor do its
// ancestors.
ExecutionStatus::Irrelevant(_) => break,
}
invalidated_indices.insert(index);
if let Some(parent_index) = node.parent {
index = parent_index
} else {
// The root of the block tree has been reached (aka the finalized block), without
// matching `latest_valid_ancestor_hash`. It's not possible or useful to go any
// further back: the finalized checkpoint is invalid so all is lost!
break;
}
}
/*
* Step 2:
*
* Start at either the `latest_valid_ancestor` or the `head_block_root` and iterate
* *forwards* to invalidate all descendants of all blocks in `invalidated_indices`.
*/
let starting_block_root = latest_valid_ancestor_root
.filter(|_| latest_valid_ancestor_is_descendant)
.unwrap_or(head_block_root);
let latest_valid_ancestor_index = *self
.indices
.get(&starting_block_root)
.ok_or(Error::NodeUnknown(starting_block_root))?;
let first_potential_descendant = latest_valid_ancestor_index + 1;
// Collect all *descendants* which have been declared invalid since they're the descendant of a block
// with an invalid execution payload.
for index in first_potential_descendant..self.nodes.len() {
let node = self
.nodes
.get_mut(index)
.ok_or(Error::InvalidNodeIndex(index))?;
if let Some(parent_index) = node.parent {
if invalidated_indices.contains(&parent_index) {
match &node.execution_status {
ExecutionStatus::Valid(hash) => {
return Err(Error::ValidExecutionStatusBecameInvalid {
block_root: node.root,
payload_block_hash: *hash,
})
}
ExecutionStatus::Unknown(hash) | ExecutionStatus::Invalid(hash) => {
node.execution_status = ExecutionStatus::Invalid(*hash)
}
ExecutionStatus::Irrelevant(_) => {
return Err(Error::IrrelevantDescendant {
block_root: node.root,
})
}
}
invalidated_indices.insert(index);
}
}
}
Ok(())
}
/// Follows the best-descendant links to find the best-block (i.e., head-block).
///
/// ## Notes
@ -320,6 +555,19 @@ impl ProtoArray {
.get(justified_index)
.ok_or(Error::InvalidJustifiedIndex(justified_index))?;
// Since there are no valid descendants of a justified block with an invalid execution
// payload, there would be no head to choose from.
//
// Fork choice is effectively broken until a new justified root is set. It might not be
// practically possible to set a new justified root if we are unable to find a new head.
//
// This scenario is *unsupported*. It represents a serious consensus failure.
if justified_node.execution_status.is_invalid() {
return Err(Error::InvalidJustifiedCheckpointExecutionStatus {
justified_root: *justified_root,
});
}
let best_descendant_index = justified_node.best_descendant.unwrap_or(justified_index);
let best_node = self
@ -537,6 +785,10 @@ impl ProtoArray {
/// Any node that has a different finalized or justified epoch should not be viable for the
/// head.
fn node_is_viable_for_head(&self, node: &ProtoNode) -> bool {
if node.execution_status.is_invalid() {
return false;
}
if let (Some(node_justified_checkpoint), Some(node_finalized_checkpoint)) =
(node.justified_checkpoint, node.finalized_checkpoint)
{
@ -568,6 +820,42 @@ impl ProtoArray {
self.iter_nodes(block_root)
.map(|node| (node.root, node.slot))
}
/// Returns `true` if the `descendant_root` has an ancestor with `ancestor_root`. Always
/// returns `false` if either input root is unknown.
///
/// ## Notes
///
/// Still returns `true` if `ancestor_root` is known and `ancestor_root == descendant_root`.
pub fn is_descendant(&self, ancestor_root: Hash256, descendant_root: Hash256) -> bool {
self.indices
.get(&ancestor_root)
.and_then(|ancestor_index| self.nodes.get(*ancestor_index))
.and_then(|ancestor| {
self.iter_block_roots(&descendant_root)
.take_while(|(_root, slot)| *slot >= ancestor.slot)
.find(|(_root, slot)| *slot == ancestor.slot)
.map(|(root, _slot)| root == ancestor_root)
})
.unwrap_or(false)
}
/// Returns the first *beacon block root* which contains an execution payload with the given
/// `block_hash`, if any.
pub fn execution_block_hash_to_beacon_block_root(
&self,
block_hash: &ExecutionBlockHash,
) -> Option<Hash256> {
self.nodes
.iter()
.rev()
.find(|node| {
node.execution_status
.block_hash()
.map_or(false, |node_block_hash| node_block_hash == *block_hash)
})
.map(|node| node.root)
}
}
/// A helper method to calculate the proposer boost based on the given `validator_balances`.

View File

@ -5,7 +5,10 @@ use serde_derive::{Deserialize, Serialize};
use ssz::{Decode, Encode};
use ssz_derive::{Decode, Encode};
use std::collections::HashMap;
use types::{AttestationShufflingId, ChainSpec, Checkpoint, Epoch, EthSpec, Hash256, Slot};
use types::{
AttestationShufflingId, ChainSpec, Checkpoint, Epoch, EthSpec, ExecutionBlockHash, Hash256,
Slot,
};
pub const DEFAULT_PRUNE_THRESHOLD: usize = 256;
@ -21,11 +24,11 @@ pub struct VoteTracker {
#[ssz(enum_behaviour = "union")]
pub enum ExecutionStatus {
/// An EL has determined that the payload is valid.
Valid(Hash256),
Valid(ExecutionBlockHash),
/// An EL has determined that the payload is invalid.
Invalid(Hash256),
Invalid(ExecutionBlockHash),
/// An EL has not yet verified the execution payload.
Unknown(Hash256),
Unknown(ExecutionBlockHash),
/// The block is either prior to the merge fork, or after the merge fork but before the terminal
/// PoW block has been found.
///
@ -41,7 +44,7 @@ impl ExecutionStatus {
ExecutionStatus::Irrelevant(false)
}
pub fn block_hash(&self) -> Option<Hash256> {
pub fn block_hash(&self) -> Option<ExecutionBlockHash> {
match self {
ExecutionStatus::Valid(hash)
| ExecutionStatus::Invalid(hash)
@ -49,6 +52,37 @@ impl ExecutionStatus {
ExecutionStatus::Irrelevant(_) => None,
}
}
/// Returns `true` if the block:
///
/// - Has execution enabled
/// - Has a valid payload
pub fn is_valid(&self) -> bool {
matches!(self, ExecutionStatus::Valid(_))
}
/// Returns `true` if the block:
///
/// - Has execution enabled
/// - Has a payload that has not yet been verified by an EL.
pub fn is_not_verified(&self) -> bool {
matches!(self, ExecutionStatus::Unknown(_))
}
/// Returns `true` if the block:
///
/// - Has execution enabled
/// - Has an invalid payload.
pub fn is_invalid(&self) -> bool {
matches!(self, ExecutionStatus::Invalid(_))
}
/// Returns `true` if the block:
///
/// - Does not have execution enabled (before or after Bellatrix fork)
pub fn is_irrelevant(&self) -> bool {
matches!(self, ExecutionStatus::Irrelevant(_))
}
}
/// A block that is to be applied to the fork choice.
@ -150,6 +184,17 @@ impl ProtoArrayForkChoice {
})
}
/// See `ProtoArray::propagate_execution_payload_invalidation` for documentation.
pub fn process_execution_payload_invalidation(
&mut self,
head_block_root: Hash256,
latest_valid_ancestor_root: Option<ExecutionBlockHash>,
) -> Result<(), String> {
self.proto_array
.propagate_execution_payload_invalidation(head_block_root, latest_valid_ancestor_root)
.map_err(|e| format!("Failed to process invalid payload: {:?}", e))
}
pub fn process_attestation(
&mut self,
validator_index: usize,
@ -267,25 +312,19 @@ impl ProtoArrayForkChoice {
}
}
/// Returns `true` if the `descendant_root` has an ancestor with `ancestor_root`. Always
/// returns `false` if either input roots are unknown.
///
/// ## Notes
///
/// Still returns `true` if `ancestor_root` is known and `ancestor_root == descendant_root`.
/// Returns the weight of a given block.
pub fn get_weight(&self, block_root: &Hash256) -> Option<u64> {
let block_index = self.proto_array.indices.get(block_root)?;
self.proto_array
.nodes
.get(*block_index)
.map(|node| node.weight)
}
/// See `ProtoArray` documentation.
pub fn is_descendant(&self, ancestor_root: Hash256, descendant_root: Hash256) -> bool {
self.proto_array
.indices
.get(&ancestor_root)
.and_then(|ancestor_index| self.proto_array.nodes.get(*ancestor_index))
.and_then(|ancestor| {
self.proto_array
.iter_block_roots(&descendant_root)
.take_while(|(_root, slot)| *slot >= ancestor.slot)
.find(|(_root, slot)| *slot == ancestor.slot)
.map(|(root, _slot)| root == ancestor_root)
})
.unwrap_or(false)
.is_descendant(ancestor_root, descendant_root)
}
pub fn latest_message(&self, validator_index: usize) -> Option<(Hash256, Epoch)> {

View File

@ -58,8 +58,8 @@ pub enum BlockProcessingError {
InconsistentBlockFork(InconsistentFork),
InconsistentStateFork(InconsistentFork),
ExecutionHashChainIncontiguous {
expected: Hash256,
found: Hash256,
expected: ExecutionBlockHash,
found: ExecutionBlockHash,
},
ExecutionRandaoMismatch {
expected: Hash256,

View File

@ -144,7 +144,7 @@ pub struct ChainSpec {
/// The Merge fork epoch is optional, with `None` representing "Merge never happens".
pub bellatrix_fork_epoch: Option<Epoch>,
pub terminal_total_difficulty: Uint256,
pub terminal_block_hash: Hash256,
pub terminal_block_hash: ExecutionBlockHash,
pub terminal_block_hash_activation_epoch: Epoch,
/*
@ -549,7 +549,7 @@ impl ChainSpec {
// `Uint256::MAX` which is `2*256- 1`.
.checked_add(Uint256::one())
.expect("addition does not overflow"),
terminal_block_hash: Hash256::zero(),
terminal_block_hash: ExecutionBlockHash::zero(),
terminal_block_hash_activation_epoch: Epoch::new(u64::MAX),
/*
@ -746,7 +746,7 @@ impl ChainSpec {
// `Uint256::MAX` which is `2*256- 1`.
.checked_add(Uint256::one())
.expect("addition does not overflow"),
terminal_block_hash: Hash256::zero(),
terminal_block_hash: ExecutionBlockHash::zero(),
terminal_block_hash_activation_epoch: Epoch::new(u64::MAX),
/*
@ -787,7 +787,7 @@ pub struct Config {
pub terminal_total_difficulty: Uint256,
// TODO(merge): remove this default
#[serde(default = "default_terminal_block_hash")]
pub terminal_block_hash: Hash256,
pub terminal_block_hash: ExecutionBlockHash,
// TODO(merge): remove this default
#[serde(default = "default_terminal_block_hash_activation_epoch")]
pub terminal_block_hash_activation_epoch: Epoch,
@ -870,8 +870,8 @@ const fn default_terminal_total_difficulty() -> Uint256 {
])
}
fn default_terminal_block_hash() -> Hash256 {
Hash256::zero()
fn default_terminal_block_hash() -> ExecutionBlockHash {
ExecutionBlockHash::zero()
}
fn default_terminal_block_hash_activation_epoch() -> Epoch {

View File

@ -0,0 +1,101 @@
use crate::test_utils::TestRandom;
use crate::Hash256;
use rand::RngCore;
use serde_derive::{Deserialize, Serialize};
use ssz::{Decode, DecodeError, Encode};
use std::fmt;
#[cfg_attr(feature = "arbitrary-fuzz", derive(arbitrary::Arbitrary))]
#[derive(Default, Debug, Clone, Copy, Serialize, Deserialize, Eq, PartialEq, Hash)]
#[serde(transparent)]
pub struct ExecutionBlockHash(Hash256);
impl ExecutionBlockHash {
pub fn zero() -> Self {
Self(Hash256::zero())
}
pub fn repeat_byte(b: u8) -> Self {
Self(Hash256::repeat_byte(b))
}
pub fn from_root(root: Hash256) -> Self {
Self(root)
}
pub fn into_root(self) -> Hash256 {
self.0
}
}
impl Encode for ExecutionBlockHash {
fn is_ssz_fixed_len() -> bool {
<Hash256 as Encode>::is_ssz_fixed_len()
}
fn ssz_fixed_len() -> usize {
<Hash256 as Encode>::ssz_fixed_len()
}
fn ssz_bytes_len(&self) -> usize {
self.0.ssz_bytes_len()
}
fn ssz_append(&self, buf: &mut Vec<u8>) {
self.0.ssz_append(buf)
}
}
impl Decode for ExecutionBlockHash {
fn is_ssz_fixed_len() -> bool {
<Hash256 as Decode>::is_ssz_fixed_len()
}
fn ssz_fixed_len() -> usize {
<Hash256 as Decode>::ssz_fixed_len()
}
fn from_ssz_bytes(bytes: &[u8]) -> Result<Self, DecodeError> {
Hash256::from_ssz_bytes(bytes).map(Self)
}
}
impl tree_hash::TreeHash for ExecutionBlockHash {
fn tree_hash_type() -> tree_hash::TreeHashType {
Hash256::tree_hash_type()
}
fn tree_hash_packed_encoding(&self) -> Vec<u8> {
self.0.tree_hash_packed_encoding()
}
fn tree_hash_packing_factor() -> usize {
Hash256::tree_hash_packing_factor()
}
fn tree_hash_root(&self) -> tree_hash::Hash256 {
self.0.tree_hash_root()
}
}
impl TestRandom for ExecutionBlockHash {
fn random_for_test(rng: &mut impl RngCore) -> Self {
Self(Hash256::random_for_test(rng))
}
}
impl std::str::FromStr for ExecutionBlockHash {
type Err = String;
fn from_str(s: &str) -> Result<Self, Self::Err> {
Hash256::from_str(s)
.map(Self)
.map_err(|e| format!("{:?}", e))
}
}
impl fmt::Display for ExecutionBlockHash {
fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
write!(f, "{}", self.0)
}
}

View File

@ -15,7 +15,7 @@ pub type Transaction<T> = VariableList<u8, T>;
#[derivative(PartialEq, Hash(bound = "T: EthSpec"))]
#[serde(bound = "T: EthSpec")]
pub struct ExecutionPayload<T: EthSpec> {
pub parent_hash: Hash256,
pub parent_hash: ExecutionBlockHash,
pub fee_recipient: Address,
pub state_root: Hash256,
pub receipts_root: Hash256,
@ -34,7 +34,7 @@ pub struct ExecutionPayload<T: EthSpec> {
pub extra_data: VariableList<u8, T::MaxExtraDataBytes>,
#[serde(with = "eth2_serde_utils::quoted_u256")]
pub base_fee_per_gas: Uint256,
pub block_hash: Hash256,
pub block_hash: ExecutionBlockHash,
#[serde(with = "ssz_types::serde_utils::list_of_hex_var_list")]
pub transactions:
VariableList<Transaction<T::MaxBytesPerTransaction>, T::MaxTransactionsPerPayload>,

View File

@ -9,7 +9,7 @@ use tree_hash_derive::TreeHash;
Default, Debug, Clone, PartialEq, Serialize, Deserialize, Encode, Decode, TreeHash, TestRandom,
)]
pub struct ExecutionPayloadHeader<T: EthSpec> {
pub parent_hash: Hash256,
pub parent_hash: ExecutionBlockHash,
pub fee_recipient: Address,
pub state_root: Hash256,
pub receipts_root: Hash256,
@ -28,7 +28,7 @@ pub struct ExecutionPayloadHeader<T: EthSpec> {
pub extra_data: VariableList<u8, T::MaxExtraDataBytes>,
#[serde(with = "eth2_serde_utils::quoted_u256")]
pub base_fee_per_gas: Uint256,
pub block_hash: Hash256,
pub block_hash: ExecutionBlockHash,
pub transactions_root: Hash256,
}

View File

@ -37,6 +37,7 @@ pub mod deposit_message;
pub mod enr_fork_id;
pub mod eth1_data;
pub mod eth_spec;
pub mod execution_block_hash;
pub mod execution_payload;
pub mod execution_payload_header;
pub mod fork;
@ -113,6 +114,7 @@ pub use crate::deposit_message::DepositMessage;
pub use crate::enr_fork_id::EnrForkId;
pub use crate::eth1_data::Eth1Data;
pub use crate::eth_spec::EthSpecId;
pub use crate::execution_block_hash::ExecutionBlockHash;
pub use crate::execution_payload::{ExecutionPayload, Transaction};
pub use crate::execution_payload_header::ExecutionPayloadHeader;
pub use crate::fork::Fork;

View File

@ -23,7 +23,7 @@ pub fn run<T: EthSpec>(matches: &ArgMatches) -> Result<(), String> {
base_fee_per_gas,
timestamp: genesis_time,
block_hash: eth1_block_hash,
random: eth1_block_hash,
random: eth1_block_hash.into_root(),
..ExecutionPayloadHeader::default()
};
let mut file = File::create(file_name).map_err(|_| "Unable to create file".to_string())?;

View File

@ -108,7 +108,7 @@ pub fn run<T: EthSpec>(testnet_dir_path: PathBuf, matches: &ArgMatches) -> Resul
let genesis_state = interop_genesis_state::<T>(
&keypairs,
genesis_time,
eth1_block_hash,
eth1_block_hash.into_root(),
execution_payload_header,
&spec,
)?;

View File

@ -14,15 +14,15 @@ use ssz_derive::Decode;
use state_processing::state_advance::complete_state_advance;
use std::time::Duration;
use types::{
Attestation, BeaconBlock, BeaconState, Checkpoint, Epoch, EthSpec, ForkName, Hash256,
IndexedAttestation, SignedBeaconBlock, Slot, Uint256,
Attestation, BeaconBlock, BeaconState, Checkpoint, Epoch, EthSpec, ExecutionBlockHash,
ForkName, Hash256, IndexedAttestation, SignedBeaconBlock, Slot, Uint256,
};
#[derive(Default, Debug, PartialEq, Clone, Deserialize, Decode)]
#[serde(deny_unknown_fields)]
pub struct PowBlock {
pub block_hash: Hash256,
pub parent_hash: Hash256,
pub block_hash: ExecutionBlockHash,
pub parent_hash: ExecutionBlockHash,
pub total_difficulty: Uint256,
}

View File

@ -1,10 +1,10 @@
use crate::execution_engine::{ExecutionEngine, GenericExecutionEngine};
use execution_layer::{ExecutionLayer, PayloadAttributes, PayloadStatusV1Status};
use execution_layer::{ExecutionLayer, PayloadAttributes, PayloadStatus};
use std::sync::Arc;
use std::time::{Duration, Instant, SystemTime, UNIX_EPOCH};
use task_executor::TaskExecutor;
use tokio::time::sleep;
use types::{Address, ChainSpec, EthSpec, Hash256, MainnetEthSpec, Uint256};
use types::{Address, ChainSpec, EthSpec, ExecutionBlockHash, Hash256, MainnetEthSpec, Uint256};
const EXECUTION_ENGINE_START_TIMEOUT: Duration = Duration::from_secs(10);
@ -139,7 +139,7 @@ impl<E: GenericExecutionEngine> TestRig<E> {
let parent_hash = terminal_pow_block_hash;
let timestamp = timestamp_now();
let random = Hash256::zero();
let finalized_block_hash = Hash256::zero();
let finalized_block_hash = ExecutionBlockHash::zero();
let proposer_index = 0;
let valid_payload = self
.ee_a
@ -161,15 +161,15 @@ impl<E: GenericExecutionEngine> TestRig<E> {
* `notify_new_payload`.
*/
let head_block_hash = valid_payload.block_hash;
let finalized_block_hash = Hash256::zero();
let finalized_block_hash = ExecutionBlockHash::zero();
let payload_attributes = None;
let (status, _) = self
let status = self
.ee_a
.execution_layer
.notify_forkchoice_updated(head_block_hash, finalized_block_hash, payload_attributes)
.await
.unwrap();
assert_eq!(status, PayloadStatusV1Status::Syncing);
assert_eq!(status, PayloadStatus::Syncing);
/*
* Execution Engine A:
@ -177,13 +177,13 @@ impl<E: GenericExecutionEngine> TestRig<E> {
* Provide the valid payload back to the EE again.
*/
let (status, _) = self
let status = self
.ee_a
.execution_layer
.notify_new_payload(&valid_payload)
.await
.unwrap();
assert_eq!(status, PayloadStatusV1Status::Valid);
assert_eq!(status, PayloadStatus::Valid);
/*
* Execution Engine A:
@ -193,15 +193,15 @@ impl<E: GenericExecutionEngine> TestRig<E> {
* Do not provide payload attributes (we'll test that later).
*/
let head_block_hash = valid_payload.block_hash;
let finalized_block_hash = Hash256::zero();
let finalized_block_hash = ExecutionBlockHash::zero();
let payload_attributes = None;
let (status, _) = self
let status = self
.ee_a
.execution_layer
.notify_forkchoice_updated(head_block_hash, finalized_block_hash, payload_attributes)
.await
.unwrap();
assert_eq!(status, PayloadStatusV1Status::Valid);
assert_eq!(status, PayloadStatus::Valid);
/*
* Execution Engine A:
@ -211,16 +211,13 @@ impl<E: GenericExecutionEngine> TestRig<E> {
let mut invalid_payload = valid_payload.clone();
invalid_payload.random = Hash256::from_low_u64_be(42);
let (status, _) = self
let status = self
.ee_a
.execution_layer
.notify_new_payload(&invalid_payload)
.await
.unwrap();
assert!(matches!(
status,
PayloadStatusV1Status::Invalid | PayloadStatusV1Status::InvalidBlockHash
));
assert!(matches!(status, PayloadStatus::InvalidBlockHash { .. }));
/*
* Execution Engine A:
@ -231,7 +228,7 @@ impl<E: GenericExecutionEngine> TestRig<E> {
let parent_hash = valid_payload.block_hash;
let timestamp = valid_payload.timestamp + 1;
let random = Hash256::zero();
let finalized_block_hash = Hash256::zero();
let finalized_block_hash = ExecutionBlockHash::zero();
let proposer_index = 0;
let second_payload = self
.ee_a
@ -252,13 +249,13 @@ impl<E: GenericExecutionEngine> TestRig<E> {
* Provide the second payload back to the EE again.
*/
let (status, _) = self
let status = self
.ee_a
.execution_layer
.notify_new_payload(&second_payload)
.await
.unwrap();
assert_eq!(status, PayloadStatusV1Status::Valid);
assert_eq!(status, PayloadStatus::Valid);
/*
* Execution Engine A:
@ -266,32 +263,32 @@ impl<E: GenericExecutionEngine> TestRig<E> {
* Indicate that the payload is the head of the chain, providing payload attributes.
*/
let head_block_hash = valid_payload.block_hash;
let finalized_block_hash = Hash256::zero();
let finalized_block_hash = ExecutionBlockHash::zero();
let payload_attributes = Some(PayloadAttributes {
timestamp: second_payload.timestamp + 1,
random: Hash256::zero(),
suggested_fee_recipient: Address::zero(),
});
let (status, _) = self
let status = self
.ee_a
.execution_layer
.notify_forkchoice_updated(head_block_hash, finalized_block_hash, payload_attributes)
.await
.unwrap();
assert_eq!(status, PayloadStatusV1Status::Valid);
assert_eq!(status, PayloadStatus::Valid);
/*
* Execution Engine B:
*
* Provide the second payload, without providing the first.
*/
let (status, _) = self
let status = self
.ee_b
.execution_layer
.notify_new_payload(&second_payload)
.await
.unwrap();
assert_eq!(status, PayloadStatusV1Status::Syncing);
assert_eq!(status, PayloadStatus::Accepted);
/*
* Execution Engine B:
@ -299,15 +296,15 @@ impl<E: GenericExecutionEngine> TestRig<E> {
* Set the second payload as the head, without providing payload attributes.
*/
let head_block_hash = second_payload.block_hash;
let finalized_block_hash = Hash256::zero();
let finalized_block_hash = ExecutionBlockHash::zero();
let payload_attributes = None;
let (status, _) = self
let status = self
.ee_b
.execution_layer
.notify_forkchoice_updated(head_block_hash, finalized_block_hash, payload_attributes)
.await
.unwrap();
assert_eq!(status, PayloadStatusV1Status::Syncing);
assert_eq!(status, PayloadStatus::Syncing);
/*
* Execution Engine B:
@ -315,26 +312,26 @@ impl<E: GenericExecutionEngine> TestRig<E> {
* Provide the first payload to the EE.
*/
let (status, _) = self
let status = self
.ee_b
.execution_layer
.notify_new_payload(&valid_payload)
.await
.unwrap();
assert_eq!(status, PayloadStatusV1Status::Valid);
assert_eq!(status, PayloadStatus::Valid);
/*
* Execution Engine B:
*
* Provide the second payload, now the first has been provided.
*/
let (status, _) = self
let status = self
.ee_b
.execution_layer
.notify_new_payload(&second_payload)
.await
.unwrap();
assert_eq!(status, PayloadStatusV1Status::Valid);
assert_eq!(status, PayloadStatus::Valid);
/*
* Execution Engine B:
@ -342,15 +339,15 @@ impl<E: GenericExecutionEngine> TestRig<E> {
* Set the second payload as the head, without providing payload attributes.
*/
let head_block_hash = second_payload.block_hash;
let finalized_block_hash = Hash256::zero();
let finalized_block_hash = ExecutionBlockHash::zero();
let payload_attributes = None;
let (status, _) = self
let status = self
.ee_b
.execution_layer
.notify_forkchoice_updated(head_block_hash, finalized_block_hash, payload_attributes)
.await
.unwrap();
assert_eq!(status, PayloadStatusV1Status::Valid);
assert_eq!(status, PayloadStatus::Valid);
}
}