Kintsugi on_merge_block tests (#2811)

* Start v1.1.5 updates

* Implement new payload creation logic

* Tidy, add comments

* Remove unused error enums

* Add validate payload for gossip

* Refactor validate_merge_block

* Split payload verification in per block processing

* Add execute_payload

* Tidy

* Tidy

* Start working on new fork choice tests

* Fix failing merge block test

* Skip block_lookup_failed test

* Fix failing terminal block test

* Fixes from self-review

* Address review comments
This commit is contained in:
Paul Hauner 2021-11-17 11:45:30 +11:00
parent 44a7b37ce3
commit 5f0fef2d1e
No known key found for this signature in database
GPG Key ID: 5E2CFF9B75FA63DF
14 changed files with 585 additions and 271 deletions

View File

@ -15,6 +15,7 @@ use crate::chain_config::ChainConfig;
use crate::errors::{BeaconChainError as Error, BlockProductionError};
use crate::eth1_chain::{Eth1Chain, Eth1ChainBackend};
use crate::events::ServerSentEventHandler;
use crate::execution_payload::get_execution_payload;
use crate::head_tracker::HeadTracker;
use crate::historical_blocks::HistoricalBlockError;
use crate::migrate::BackgroundMigrator;
@ -65,9 +66,7 @@ use ssz::Encode;
use state_processing::{
common::get_indexed_attestation,
per_block_processing,
per_block_processing::{
compute_timestamp_at_slot, errors::AttestationValidationError, is_merge_complete,
},
per_block_processing::{errors::AttestationValidationError, is_merge_complete},
per_slot_processing,
state_advance::{complete_state_advance, partial_state_advance},
BlockSignatureStrategy, SigVerifiedOp,
@ -2881,63 +2880,6 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
SyncAggregate::new()
}))
};
// Closure to fetch a sync aggregate in cases where it is required.
let get_execution_payload = |latest_execution_payload_header: &ExecutionPayloadHeader<
T::EthSpec,
>|
-> Result<ExecutionPayload<_>, BlockProductionError> {
let execution_layer = self
.execution_layer
.as_ref()
.ok_or(BlockProductionError::ExecutionLayerMissing)?;
let parent_hash;
if !is_merge_complete(&state) {
let terminal_pow_block_hash = execution_layer
.block_on(|execution_layer| {
execution_layer.get_terminal_pow_block_hash(&self.spec)
})
.map_err(BlockProductionError::TerminalPoWBlockLookupFailed)?;
if let Some(terminal_pow_block_hash) = terminal_pow_block_hash {
parent_hash = terminal_pow_block_hash;
} else {
return Ok(<_>::default());
}
} else {
parent_hash = latest_execution_payload_header.block_hash;
}
let timestamp =
compute_timestamp_at_slot(&state, &self.spec).map_err(BeaconStateError::from)?;
let random = *state.get_randao_mix(state.current_epoch())?;
let finalized_root = state.finalized_checkpoint().root;
let finalized_block_hash =
if let Some(block) = self.fork_choice.read().get_block(&finalized_root) {
block.execution_status.block_hash()
} else {
self.store
.get_block(&finalized_root)
.map_err(BlockProductionError::FailedToReadFinalizedBlock)?
.ok_or(BlockProductionError::MissingFinalizedBlock(finalized_root))?
.message()
.body()
.execution_payload()
.map(|ep| ep.block_hash)
};
execution_layer
.block_on(|execution_layer| {
execution_layer.get_payload(
parent_hash,
timestamp,
random,
finalized_block_hash.unwrap_or_else(Hash256::zero),
)
})
.map_err(BlockProductionError::GetPayloadFailed)
};
let inner_block = match &state {
BeaconState::Base(_) => BeaconBlock::Base(BeaconBlockBase {
@ -2976,10 +2918,9 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
},
})
}
BeaconState::Merge(state) => {
BeaconState::Merge(_) => {
let sync_aggregate = get_sync_aggregate()?;
let execution_payload =
get_execution_payload(&state.latest_execution_payload_header)?;
let execution_payload = get_execution_payload(self, &state)?;
BeaconBlock::Merge(BeaconBlockMerge {
slot,
proposer_index,

View File

@ -40,6 +40,9 @@
//! END
//!
//! ```
use crate::execution_payload::{
execute_payload, validate_execution_payload_for_gossip, validate_merge_block,
};
use crate::snapshot_cache::PreProcessingSnapshot;
use crate::validator_monitor::HISTORIC_EPOCHS as VALIDATOR_MONITOR_HISTORIC_EPOCHS;
use crate::validator_pubkey_cache::ValidatorPubkeyCache;
@ -50,15 +53,14 @@ use crate::{
},
metrics, BeaconChain, BeaconChainError, BeaconChainTypes,
};
use execution_layer::ExecutePayloadResponseStatus;
use fork_choice::{ForkChoice, ForkChoiceStore, PayloadVerificationStatus};
use parking_lot::RwLockReadGuard;
use proto_array::{Block as ProtoBlock, ExecutionStatus};
use proto_array::Block as ProtoBlock;
use safe_arith::ArithError;
use slog::{debug, error, info, Logger};
use slog::{debug, error, Logger};
use slot_clock::SlotClock;
use ssz::Encode;
use state_processing::per_block_processing::{is_execution_enabled, is_merge_block};
use state_processing::per_block_processing::is_merge_block;
use state_processing::{
block_signature_verifier::{BlockSignatureVerifier, Error as BlockSignatureVerifierError},
per_block_processing, per_slot_processing,
@ -71,9 +73,9 @@ use std::io::Write;
use store::{Error as DBError, HotColdDB, HotStateSummary, KeyValueStore, StoreOp};
use tree_hash::TreeHash;
use types::{
BeaconBlockRef, BeaconState, BeaconStateError, ChainSpec, CloneConfig, Epoch, EthSpec,
ExecutionPayload, Hash256, InconsistentFork, PublicKey, PublicKeyBytes, RelativeEpoch,
SignedBeaconBlock, SignedBeaconBlockHeader, Slot,
BeaconBlockRef, BeaconState, BeaconStateError, ChainSpec, CloneConfig, Epoch, EthSpec, Hash256,
InconsistentFork, PublicKey, PublicKeyBytes, RelativeEpoch, SignedBeaconBlock,
SignedBeaconBlockHeader, Slot,
};
/// Maximum block slot number. Block with slots bigger than this constant will NOT be processed.
@ -266,38 +268,47 @@ pub enum ExecutionPayloadError {
///
/// The block is invalid and the peer is faulty
RejectedByExecutionEngine,
/// The execution engine returned SYNCING for the payload
///
/// ## Peer scoring
///
/// It is not known if the block is valid or invalid.
ExecutionEngineIsSyncing,
/// The execution payload timestamp does not match the slot
///
/// ## Peer scoring
///
/// The block is invalid and the peer is faulty
InvalidPayloadTimestamp { expected: u64, found: u64 },
/// The execution payload transaction list data exceeds size limits
///
/// ## Peer scoring
///
/// The block is invalid and the peer is faulty
TransactionDataExceedsSizeLimit,
/// The execution payload references an execution block that cannot trigger the merge.
///
/// ## Peer scoring
///
/// The block is invalid and the peer sent us a block that passes gossip propagation conditions,
/// but is invalid upon further verification.
InvalidTerminalPoWBlock,
/// The execution payload references execution blocks that are unavailable on our execution
/// nodes.
InvalidTerminalPoWBlock { parent_hash: Hash256 },
/// The `TERMINAL_BLOCK_HASH` is set, but the block has not reached the
/// `TERMINAL_BLOCK_HASH_ACTIVATION_EPOCH`.
///
/// ## Peer scoring
///
/// It's not clear if the peer is invalid or if it's on a different execution fork to us.
TerminalPoWBlockNotFound,
/// The block is invalid and the peer sent us a block that passes gossip propagation conditions,
/// but is invalid upon further verification.
InvalidActivationEpoch {
activation_epoch: Epoch,
epoch: Epoch,
},
/// The `TERMINAL_BLOCK_HASH` is set, but does not match the value specified by the block.
///
/// ## Peer scoring
///
/// 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,
},
/// The execution node failed to provide a parent block to a known block. This indicates an
/// issue with the execution node.
///
/// ## Peer scoring
///
/// The peer is not necessarily invalid.
PoWParentMissing(Hash256),
}
impl From<execution_layer::Error> for ExecutionPayloadError {
@ -768,8 +779,8 @@ impl<T: BeaconChainTypes> GossipVerifiedBlock<T> {
});
}
// validate the block's execution_payload
validate_execution_payload(&parent_block, block.message(), chain)?;
// Validate the block's execution_payload (if any).
validate_execution_payload_for_gossip(&parent_block, block.message(), chain)?;
Ok(Self {
block,
@ -1103,83 +1114,16 @@ impl<'a, T: BeaconChainTypes> FullyVerifiedBlock<'a, T> {
// - Doing the check here means we can keep our fork-choice implementation "pure". I.e., no
// calls to remote servers.
if is_merge_block(&state, block.message().body()) {
let execution_layer = chain
.execution_layer
.as_ref()
.ok_or(ExecutionPayloadError::NoExecutionConnection)?;
let execution_payload =
block
.message()
.body()
.execution_payload()
.ok_or_else(|| InconsistentFork {
fork_at_slot: eth2::types::ForkName::Merge,
object_fork: block.message().body().fork_name(),
})?;
let is_valid_terminal_pow_block = execution_layer
.block_on(|execution_layer| {
execution_layer.is_valid_terminal_pow_block_hash(
execution_payload.parent_hash,
&chain.spec,
)
})
.map_err(ExecutionPayloadError::from)?;
match is_valid_terminal_pow_block {
Some(true) => Ok(()),
Some(false) => Err(ExecutionPayloadError::InvalidTerminalPoWBlock),
None => {
info!(
chain.log,
"Optimistically accepting terminal block";
"block_hash" => ?execution_payload.parent_hash,
"msg" => "the terminal block/parent was unavailable"
);
Ok(())
}
}?;
validate_merge_block(chain, block.message())?
}
// This is the soonest we can run these checks as they must be called AFTER per_slot_processing
// The specification declares that this should be run *inside* `per_block_processing`,
// however we run it here to keep `per_block_processing` pure (i.e., no calls to external
// servers).
//
// TODO(merge): handle the latest_valid_hash of an invalid payload.
let (_latest_valid_hash, payload_verification_status) =
if is_execution_enabled(&state, block.message().body()) {
let execution_layer = chain
.execution_layer
.as_ref()
.ok_or(ExecutionPayloadError::NoExecutionConnection)?;
let execution_payload =
block
.message()
.body()
.execution_payload()
.ok_or_else(|| InconsistentFork {
fork_at_slot: eth2::types::ForkName::Merge,
object_fork: block.message().body().fork_name(),
})?;
let execute_payload_response = execution_layer
.block_on(|execution_layer| execution_layer.execute_payload(execution_payload));
match execute_payload_response {
Ok((status, latest_valid_hash)) => match status {
ExecutePayloadResponseStatus::Valid => {
(latest_valid_hash, PayloadVerificationStatus::Verified)
}
ExecutePayloadResponseStatus::Invalid => {
return Err(ExecutionPayloadError::RejectedByExecutionEngine.into());
}
ExecutePayloadResponseStatus::Syncing => {
(latest_valid_hash, PayloadVerificationStatus::NotVerified)
}
},
Err(_) => (None, PayloadVerificationStatus::NotVerified),
}
} else {
(None, PayloadVerificationStatus::Irrelevant)
};
// It is important that this function is called *after* `per_slot_processing`, since the
// `randao` may change.
let payload_verification_status = execute_payload(chain, &state, block.message())?;
// If the block is sufficiently recent, notify the validator monitor.
if let Some(slot) = chain.slot_clock.now() {
@ -1290,64 +1234,6 @@ impl<'a, T: BeaconChainTypes> FullyVerifiedBlock<'a, T> {
}
}
/// Validate the gossip block's execution_payload according to the checks described here:
/// https://github.com/ethereum/consensus-specs/blob/dev/specs/merge/p2p-interface.md#beacon_block
fn validate_execution_payload<T: BeaconChainTypes>(
parent_block: &ProtoBlock,
block: BeaconBlockRef<'_, T::EthSpec>,
chain: &BeaconChain<T>,
) -> Result<(), BlockError<T::EthSpec>> {
// Only apply this validation if this is a merge beacon block.
if let Some(execution_payload) = block.body().execution_payload() {
// This logic should match `is_execution_enabled`. We use only the execution block hash of
// the parent here in order to avoid loading the parent state during gossip verification.
let is_merge_complete = match parent_block.execution_status {
// Optimistically declare that an "unknown" status block has completed the merge.
ExecutionStatus::Valid(_) | ExecutionStatus::Unknown(_) => true,
// It's impossible for an irrelevant block to have completed the merge. It is pre-merge
// by definition.
ExecutionStatus::Irrelevant(_) => false,
// If the parent has an invalid payload then it's impossible to build a valid block upon
// it. Reject the block.
ExecutionStatus::Invalid(_) => {
return Err(BlockError::ParentExecutionPayloadInvalid {
parent_root: parent_block.root,
})
}
};
let is_merge_block =
!is_merge_complete && *execution_payload != <ExecutionPayload<T::EthSpec>>::default();
if !is_merge_block && !is_merge_complete {
return Ok(());
}
let expected_timestamp = chain
.slot_clock
.compute_timestamp_at_slot(block.slot())
.ok_or(BlockError::BeaconChainError(
BeaconChainError::UnableToComputeTimeAtSlot,
))?;
// The block's execution payload timestamp is correct with respect to the slot
if execution_payload.timestamp != expected_timestamp {
return Err(BlockError::ExecutionPayloadError(
ExecutionPayloadError::InvalidPayloadTimestamp {
expected: expected_timestamp,
found: execution_payload.timestamp,
},
));
}
// The execution payload transaction list data is within expected size limits
if execution_payload.transactions.len() > T::EthSpec::max_transactions_per_payload() {
return Err(BlockError::ExecutionPayloadError(
ExecutionPayloadError::TransactionDataExceedsSizeLimit,
));
}
}
Ok(())
}
/// Check that the count of skip slots between the block and its parent does not exceed our maximum
/// value.
///

View File

@ -181,6 +181,7 @@ pub enum BlockProductionError {
state_slot: Slot,
},
ExecutionLayerMissing,
BlockingFailed(execution_layer::Error),
TerminalPoWBlockLookupFailed(execution_layer::Error),
GetPayloadFailed(execution_layer::Error),
FailedToReadFinalizedBlock(store::Error),

View File

@ -0,0 +1,306 @@
//! This module contains various functions for producing and verifying `ExecutionPayloads`.
//!
//! Lighthouse tends to do payload tasks in *slightly* different locations to the specification.
//! This is because some tasks involve calling out to external servers and it's nice to keep those
//! away from our pure `state_processing` and `fork_choice` crates.
//!
//! So, this module contains functions that one might expect to find in other crates, but they live
//! here for good reason.
use crate::{
BeaconChain, BeaconChainError, BeaconChainTypes, BlockError, BlockProductionError,
ExecutionPayloadError,
};
use execution_layer::ExecutePayloadResponseStatus;
use fork_choice::PayloadVerificationStatus;
use proto_array::{Block as ProtoBlock, ExecutionStatus};
use slog::debug;
use slot_clock::SlotClock;
use state_processing::per_block_processing::{
compute_timestamp_at_slot, is_execution_enabled, is_merge_complete,
partially_verify_execution_payload,
};
use types::*;
/// Verify that `execution_payload` contained by `block` is considered valid by an execution
/// engine.
///
/// ## Specification
///
/// Equivalent to the `execute_payload` function in the merge Beacon Chain Changes, although it
/// contains a few extra checks by running `partially_verify_execution_payload` first:
///
/// https://github.com/ethereum/consensus-specs/blob/v1.1.5/specs/merge/beacon-chain.md#execute_payload
pub fn execute_payload<T: BeaconChainTypes>(
chain: &BeaconChain<T>,
state: &BeaconState<T::EthSpec>,
block: BeaconBlockRef<T::EthSpec>,
) -> Result<PayloadVerificationStatus, BlockError<T::EthSpec>> {
if !is_execution_enabled(state, block.body()) {
return Ok(PayloadVerificationStatus::Irrelevant);
}
let execution_payload = block.execution_payload()?;
// Perform the initial stages of payload verification.
//
// We will duplicate these checks again during `per_block_processing`, however these checks
// are cheap and doing them here ensures we protect the execution payload from junk.
partially_verify_execution_payload(state, execution_payload, &chain.spec)
.map_err(BlockError::PerBlockProcessingError)?;
let execution_layer = chain
.execution_layer
.as_ref()
.ok_or(ExecutionPayloadError::NoExecutionConnection)?;
let execute_payload_response = execution_layer
.block_on(|execution_layer| execution_layer.execute_payload(execution_payload));
match execute_payload_response {
Ok((status, _latest_valid_hash)) => match status {
ExecutePayloadResponseStatus::Valid => Ok(PayloadVerificationStatus::Verified),
// TODO(merge): invalidate any invalid ancestors of this block in fork choice.
ExecutePayloadResponseStatus::Invalid => {
Err(ExecutionPayloadError::RejectedByExecutionEngine.into())
}
ExecutePayloadResponseStatus::Syncing => Ok(PayloadVerificationStatus::NotVerified),
},
Err(_) => Ok(PayloadVerificationStatus::NotVerified),
}
}
/// Verify that the block which triggers the merge is valid to be imported to fork choice.
///
/// ## Errors
///
/// Will return an error when using a pre-merge fork `state`. Ensure to only run this function
/// after the merge fork.
///
/// ## Specification
///
/// Equivalent to the `validate_merge_block` function in the merge Fork Choice Changes:
///
/// https://github.com/ethereum/consensus-specs/blob/v1.1.5/specs/merge/fork-choice.md#validate_merge_block
pub fn validate_merge_block<T: BeaconChainTypes>(
chain: &BeaconChain<T>,
block: BeaconBlockRef<T::EthSpec>,
) -> Result<(), BlockError<T::EthSpec>> {
let spec = &chain.spec;
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 block_epoch < spec.terminal_block_hash_activation_epoch {
return Err(ExecutionPayloadError::InvalidActivationEpoch {
activation_epoch: spec.terminal_block_hash_activation_epoch,
epoch: block_epoch,
}
.into());
}
if execution_payload.parent_hash != spec.terminal_block_hash {
return Err(ExecutionPayloadError::InvalidTerminalBlockHash {
terminal_block_hash: spec.terminal_block_hash,
payload_parent_hash: execution_payload.parent_hash,
}
.into());
}
return Ok(());
}
let execution_layer = chain
.execution_layer
.as_ref()
.ok_or(ExecutionPayloadError::NoExecutionConnection)?;
let is_valid_terminal_pow_block = execution_layer
.block_on(|execution_layer| {
execution_layer.is_valid_terminal_pow_block_hash(execution_payload.parent_hash, spec)
})
.map_err(ExecutionPayloadError::from)?;
match is_valid_terminal_pow_block {
Some(true) => Ok(()),
Some(false) => Err(ExecutionPayloadError::InvalidTerminalPoWBlock {
parent_hash: execution_payload.parent_hash,
}
.into()),
None => {
debug!(
chain.log,
"Optimistically accepting terminal block";
"block_hash" => ?execution_payload.parent_hash,
"msg" => "the terminal block/parent was unavailable"
);
Ok(())
}
}
}
/// Validate the gossip block's execution_payload according to the checks described here:
/// https://github.com/ethereum/consensus-specs/blob/dev/specs/merge/p2p-interface.md#beacon_block
pub fn validate_execution_payload_for_gossip<T: BeaconChainTypes>(
parent_block: &ProtoBlock,
block: BeaconBlockRef<'_, T::EthSpec>,
chain: &BeaconChain<T>,
) -> Result<(), BlockError<T::EthSpec>> {
// Only apply this validation if this is a merge beacon block.
if let Some(execution_payload) = block.body().execution_payload() {
// This logic should match `is_execution_enabled`. We use only the execution block hash of
// the parent here in order to avoid loading the parent state during gossip verification.
let is_merge_complete = match parent_block.execution_status {
// Optimistically declare that an "unknown" status block has completed the merge.
ExecutionStatus::Valid(_) | ExecutionStatus::Unknown(_) => true,
// It's impossible for an irrelevant block to have completed the merge. It is pre-merge
// by definition.
ExecutionStatus::Irrelevant(_) => false,
// If the parent has an invalid payload then it's impossible to build a valid block upon
// it. Reject the block.
ExecutionStatus::Invalid(_) => {
return Err(BlockError::ParentExecutionPayloadInvalid {
parent_root: parent_block.root,
})
}
};
if is_merge_complete || execution_payload != &<_>::default() {
let expected_timestamp = chain
.slot_clock
.compute_timestamp_at_slot(block.slot())
.ok_or(BlockError::BeaconChainError(
BeaconChainError::UnableToComputeTimeAtSlot,
))?;
// The block's execution payload timestamp is correct with respect to the slot
if execution_payload.timestamp != expected_timestamp {
return Err(BlockError::ExecutionPayloadError(
ExecutionPayloadError::InvalidPayloadTimestamp {
expected: expected_timestamp,
found: execution_payload.timestamp,
},
));
}
}
}
Ok(())
}
/// Gets an execution payload for inclusion in a block.
///
/// ## Errors
///
/// Will return an error when using a pre-merge fork `state`. Ensure to only run this function
/// after the merge fork.
///
/// ## Specification
///
/// Equivalent to the `get_execution_payload` function in the Validator Guide:
///
/// https://github.com/ethereum/consensus-specs/blob/v1.1.5/specs/merge/validator.md#block-proposal
pub fn get_execution_payload<T: BeaconChainTypes>(
chain: &BeaconChain<T>,
state: &BeaconState<T::EthSpec>,
) -> Result<ExecutionPayload<T::EthSpec>, BlockProductionError> {
Ok(prepare_execution_payload_blocking(chain, state)?.unwrap_or_default())
}
/// Wraps the async `prepare_execution_payload` function as a blocking task.
pub fn prepare_execution_payload_blocking<T: BeaconChainTypes>(
chain: &BeaconChain<T>,
state: &BeaconState<T::EthSpec>,
) -> Result<Option<ExecutionPayload<T::EthSpec>>, BlockProductionError> {
let execution_layer = chain
.execution_layer
.as_ref()
.ok_or(BlockProductionError::ExecutionLayerMissing)?;
execution_layer
.block_on_generic(|_| async { prepare_execution_payload(chain, state).await })
.map_err(BlockProductionError::BlockingFailed)?
}
/// Prepares an execution payload for inclusion in a block.
///
/// Will return `Ok(None)` if the merge fork has occurred, but a terminal block has not been found.
///
/// ## Errors
///
/// Will return an error when using a pre-merge fork `state`. Ensure to only run this function
/// after the merge fork.
///
/// ## Specification
///
/// Equivalent to the `prepare_execution_payload` function in the Validator Guide:
///
/// https://github.com/ethereum/consensus-specs/blob/v1.1.5/specs/merge/validator.md#block-proposal
pub async fn prepare_execution_payload<T: BeaconChainTypes>(
chain: &BeaconChain<T>,
state: &BeaconState<T::EthSpec>,
) -> Result<Option<ExecutionPayload<T::EthSpec>>, BlockProductionError> {
let spec = &chain.spec;
let execution_layer = chain
.execution_layer
.as_ref()
.ok_or(BlockProductionError::ExecutionLayerMissing)?;
let parent_hash = if !is_merge_complete(state) {
let is_terminal_block_hash_set = spec.terminal_block_hash != Hash256::zero();
let is_activation_epoch_reached =
state.current_epoch() >= spec.terminal_block_hash_activation_epoch;
if is_terminal_block_hash_set && !is_activation_epoch_reached {
return Ok(None);
}
let terminal_pow_block_hash = execution_layer
.get_terminal_pow_block_hash(spec)
.await
.map_err(BlockProductionError::TerminalPoWBlockLookupFailed)?;
if let Some(terminal_pow_block_hash) = terminal_pow_block_hash {
terminal_pow_block_hash
} else {
return Ok(None);
}
} else {
state.latest_execution_payload_header()?.block_hash
};
let timestamp = compute_timestamp_at_slot(state, spec).map_err(BeaconStateError::from)?;
let random = *state.get_randao_mix(state.current_epoch())?;
let finalized_root = state.finalized_checkpoint().root;
// The finalized block hash is not included in the specification, however we provide this
// parameter so that the execution layer can produce a payload id if one is not already known
// (e.g., due to a recent reorg).
let finalized_block_hash =
if let Some(block) = chain.fork_choice.read().get_block(&finalized_root) {
block.execution_status.block_hash()
} else {
chain
.store
.get_block(&finalized_root)
.map_err(BlockProductionError::FailedToReadFinalizedBlock)?
.ok_or(BlockProductionError::MissingFinalizedBlock(finalized_root))?
.message()
.body()
.execution_payload()
.map(|ep| ep.block_hash)
};
// Note: the fee_recipient is stored in the `execution_layer`, it will add this parameter.
let execution_payload = execution_layer
.get_payload(
parent_hash,
timestamp,
random,
finalized_block_hash.unwrap_or_else(Hash256::zero),
)
.await
.map_err(BlockProductionError::GetPayloadFailed)?;
Ok(Some(execution_payload))
}

View File

@ -12,6 +12,7 @@ pub mod chain_config;
mod errors;
pub mod eth1_chain;
pub mod events;
mod execution_payload;
pub mod fork_revert;
mod head_tracker;
pub mod historical_blocks;

View File

@ -142,10 +142,28 @@ impl ExecutionLayer {
.runtime()
.upgrade()
.ok_or(Error::ShuttingDown)?;
// TODO(paul): respect the shutdown signal.
// TODO(merge): respect the shutdown signal.
runtime.block_on(generate_future(self))
}
/// Convenience function to allow calling async functions in a non-async context.
///
/// The function is "generic" since it does not enforce a particular return type on
/// `generate_future`.
pub fn block_on_generic<'a, T, U, V>(&'a self, generate_future: T) -> Result<V, Error>
where
T: Fn(&'a Self) -> U,
U: Future<Output = V>,
{
let runtime = self
.executor()
.runtime()
.upgrade()
.ok_or(Error::ShuttingDown)?;
// TODO(merge): respect the shutdown signal.
Ok(runtime.block_on(generate_future(self)))
}
/// Convenience function to allow spawning a task without waiting for the result.
pub fn spawn<T, U>(&self, generate_future: T, name: &'static str)
where
@ -441,7 +459,7 @@ impl ExecutionLayer {
///
/// `get_terminal_pow_block_hash`
///
/// https://github.com/ethereum/consensus-specs/blob/v1.1.0/specs/merge/validator.md
/// https://github.com/ethereum/consensus-specs/blob/v1.1.5/specs/merge/validator.md
pub async fn get_terminal_pow_block_hash(
&self,
spec: &ChainSpec,
@ -449,22 +467,21 @@ impl ExecutionLayer {
let hash_opt = self
.engines()
.first_success(|engine| async move {
if spec.terminal_block_hash != Hash256::zero() {
// Note: the specification is written such that if there are multiple blocks in
// the PoW chain with the terminal block hash, then to select 0'th one.
//
// Whilst it's not clear what the 0'th block is, we ignore this completely and
// make the assumption that there are no two blocks in the chain with the same
// hash. Such a scenario would be a devestating hash collision with external
// implications far outweighing those here.
Ok(self
.get_pow_block(engine, spec.terminal_block_hash)
let terminal_block_hash = spec.terminal_block_hash;
if terminal_block_hash != Hash256::zero() {
if self
.get_pow_block(engine, terminal_block_hash)
.await?
.map(|block| block.block_hash))
} else {
self.get_pow_block_hash_at_total_difficulty(engine, spec)
.await
.is_some()
{
return Ok(Some(terminal_block_hash));
} else {
return Ok(None);
}
}
self.get_pow_block_hash_at_total_difficulty(engine, spec)
.await
})
.await
.map_err(Error::EngineErrors)?;
@ -490,13 +507,12 @@ impl ExecutionLayer {
///
/// `get_pow_block_at_terminal_total_difficulty`
///
/// https://github.com/ethereum/consensus-specs/blob/v1.1.0/specs/merge/validator.md
/// https://github.com/ethereum/consensus-specs/blob/v1.1.5/specs/merge/validator.md
async fn get_pow_block_hash_at_total_difficulty(
&self,
engine: &Engine<HttpJsonRpc>,
spec: &ChainSpec,
) -> Result<Option<Hash256>, ApiError> {
let mut ttd_exceeding_block = None;
let mut block = engine
.api
.get_block_by_number(BlockByNumberQuery::Tag(LATEST_TAG))
@ -505,25 +521,33 @@ impl ExecutionLayer {
self.execution_blocks().await.put(block.block_hash, block);
// TODO(merge): This function can theoretically loop indefinitely, as per the
// specification. We should consider how to fix this. See discussion:
// TODO(merge): This implementation adheres to the following PR in the `dev` branch:
//
// https://github.com/ethereum/consensus-specs/issues/2636
// https://github.com/ethereum/consensus-specs/pull/2719
//
// Therefore this implementation is not strictly v1.1.5, it is more lenient to some
// edge-cases during EL genesis. We should revisit this prior to the merge to ensure that
// this implementation becomes canonical.
loop {
if block.total_difficulty >= spec.terminal_total_difficulty {
ttd_exceeding_block = Some(block.block_hash);
let block_reached_ttd = block.total_difficulty >= spec.terminal_total_difficulty;
if block_reached_ttd && block.parent_hash == Hash256::zero() {
return Ok(Some(block.block_hash));
} else if block.parent_hash == Hash256::zero() {
// The end of the chain has been reached without finding the TTD, there is no
// terminal block.
return Ok(None);
}
// Try to prevent infinite loops.
if block.block_hash == block.parent_hash {
return Err(ApiError::ParentHashEqualsBlockHash(block.block_hash));
}
let parent = self
.get_pow_block(engine, block.parent_hash)
.await?
.ok_or(ApiError::ExecutionBlockNotFound(block.parent_hash))?;
let parent_reached_ttd = parent.total_difficulty >= spec.terminal_total_difficulty;
block = self
.get_pow_block(engine, block.parent_hash)
.await?
.ok_or(ApiError::ExecutionBlockNotFound(block.parent_hash))?;
if block_reached_ttd && !parent_reached_ttd {
return Ok(Some(block.block_hash));
} else {
return Ok(ttd_exceeding_block);
block = parent;
}
}
}
@ -617,10 +641,6 @@ impl ExecutionLayer {
parent: ExecutionBlock,
spec: &ChainSpec,
) -> bool {
if block.block_hash == spec.terminal_block_hash {
return true;
}
let is_total_difficulty_reached = block.total_difficulty >= spec.terminal_total_difficulty;
let is_parent_total_difficulty_valid =
parent.total_difficulty < spec.terminal_total_difficulty;

View File

@ -170,6 +170,11 @@ impl<T: EthSpec> ExecutionBlockGenerator<T> {
self.insert_pow_blocks(next_block..=target_block)
}
pub fn drop_all_blocks(&mut self) {
self.blocks = <_>::default();
self.block_hashes = <_>::default();
}
pub fn insert_pow_blocks(
&mut self,
block_numbers: impl Iterator<Item = u64>,
@ -211,12 +216,14 @@ impl<T: EthSpec> ExecutionBlockGenerator<T> {
"block {} is already known, forking is not supported",
block.block_number()
));
} else if block.parent_hash() != Hash256::zero()
&& !self.blocks.contains_key(&block.parent_hash())
{
} else if block.block_number() != 0 && !self.blocks.contains_key(&block.parent_hash()) {
return Err(format!("parent block {:?} is unknown", block.parent_hash()));
}
self.insert_block_without_checks(block)
}
pub fn insert_block_without_checks(&mut self, block: Block<T>) -> Result<(), String> {
self.block_hashes
.insert(block.block_number(), block.block_hash());
self.blocks.insert(block.block_hash(), block);

View File

@ -4,6 +4,7 @@ use crate::engine_api::http::JSONRPC_VERSION;
use crate::engine_api::ExecutePayloadResponseStatus;
use bytes::Bytes;
use environment::null_logger;
use execution_block_generator::{Block, PoWBlock};
use handle_rpc::handle_rpc;
use parking_lot::{Mutex, RwLock, RwLockWriteGuard};
use serde::{Deserialize, Serialize};
@ -118,6 +119,40 @@ impl<T: EthSpec> MockServer<T> {
pub fn all_payloads_valid(&self) {
*self.ctx.static_execute_payload_response.lock() = Some(ExecutePayloadResponseStatus::Valid)
}
pub fn insert_pow_block(
&self,
block_number: u64,
block_hash: Hash256,
parent_hash: Hash256,
total_difficulty: Uint256,
) {
let block = Block::PoW(PoWBlock {
block_number,
block_hash,
parent_hash,
total_difficulty,
});
self.ctx
.execution_block_generator
.write()
// The EF tests supply blocks out of order, so we must import them "without checks" and
// trust they form valid chains.
.insert_block_without_checks(block)
.unwrap()
}
pub fn get_block(&self, block_hash: Hash256) -> Option<Block<T>> {
self.ctx
.execution_block_generator
.read()
.block_by_hash(block_hash)
}
pub fn drop_all_blocks(&self) {
self.ctx.execution_block_generator.write().drop_all_blocks()
}
}
#[derive(Debug)]

View File

@ -295,9 +295,18 @@ pub fn get_new_eth1_data<T: EthSpec>(
}
}
/// https://github.com/ethereum/consensus-specs/blob/dev/specs/merge/beacon-chain.md#process_execution_payload
pub fn process_execution_payload<T: EthSpec>(
state: &mut BeaconState<T>,
/// Performs *partial* verification of the `payload`.
///
/// The verification is partial, since the execution payload is not verified against an execution
/// engine. That is expected to be performed by an upstream function.
///
/// ## Specification
///
/// Contains a partial set of checks from the `process_execution_payload` function:
///
/// https://github.com/ethereum/consensus-specs/blob/v1.1.5/specs/merge/beacon-chain.md#process_execution_payload
pub fn partially_verify_execution_payload<T: EthSpec>(
state: &BeaconState<T>,
payload: &ExecutionPayload<T>,
spec: &ChainSpec,
) -> Result<(), BlockProcessingError> {
@ -327,6 +336,23 @@ pub fn process_execution_payload<T: EthSpec>(
}
);
Ok(())
}
/// Calls `partially_verify_execution_payload` and then updates the payload header in the `state`.
///
/// ## Specification
///
/// Partially equivalent to the `process_execution_payload` function:
///
/// https://github.com/ethereum/consensus-specs/blob/v1.1.5/specs/merge/beacon-chain.md#process_execution_payload
pub fn process_execution_payload<T: EthSpec>(
state: &mut BeaconState<T>,
payload: &ExecutionPayload<T>,
spec: &ChainSpec,
) -> Result<(), BlockProcessingError> {
partially_verify_execution_payload(state, payload, spec)?;
*state.latest_execution_payload_header_mut()? = ExecutionPayloadHeader {
parent_hash: payload.parent_hash,
coinbase: payload.coinbase,

View File

@ -234,6 +234,17 @@ impl<'a, T: EthSpec> BeaconBlockRef<'a, T> {
..self.block_header()
}
}
/// Extracts a reference to an execution payload from a block, returning an error if the block
/// is pre-merge.
pub fn execution_payload(&self) -> Result<&ExecutionPayload<T>, InconsistentFork> {
self.body()
.execution_payload()
.ok_or_else(|| InconsistentFork {
fork_at_slot: ForkName::Merge,
object_fork: self.body().fork_name(),
})
}
}
impl<'a, T: EthSpec> BeaconBlockRefMut<'a, T> {

View File

@ -39,9 +39,6 @@ excluded_paths = [
"tests/minimal/altair/merkle/single_proof",
"tests/mainnet/merge/merkle/single_proof",
"tests/minimal/merge/merkle/single_proof",
# Fork choice tests featuring PoW blocks
"tests/minimal/merge/fork_choice/on_merge_block/",
"tests/mainnet/merge/fork_choice/on_merge_block/"
]
def normalize_path(path):

View File

@ -9,13 +9,26 @@ use beacon_chain::{
BeaconChainTypes, HeadInfo,
};
use serde_derive::Deserialize;
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,
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 total_difficulty: Uint256,
// This field is not used and I expect it to be removed. See:
//
// https://github.com/ethereum/consensus-specs/pull/2720
pub difficulty: Uint256,
}
#[derive(Debug, Clone, Copy, PartialEq, Deserialize)]
#[serde(deny_unknown_fields)]
pub struct Head {
@ -37,11 +50,12 @@ pub struct Checks {
#[derive(Debug, Clone, Deserialize)]
#[serde(untagged, deny_unknown_fields)]
pub enum Step<B, A> {
pub enum Step<B, A, P> {
Tick { tick: u64 },
ValidBlock { block: B },
MaybeValidBlock { block: B, valid: bool },
Attestation { attestation: A },
PowBlock { pow_block: P },
Checks { checks: Box<Checks> },
}
@ -56,7 +70,7 @@ pub struct ForkChoiceTest<E: EthSpec> {
pub description: String,
pub anchor_state: BeaconState<E>,
pub anchor_block: BeaconBlock<E>,
pub steps: Vec<Step<SignedBeaconBlock<E>, Attestation<E>>>,
pub steps: Vec<Step<SignedBeaconBlock<E>, Attestation<E>, PowBlock>>,
}
impl<E: EthSpec> LoadCase for ForkChoiceTest<E> {
@ -69,7 +83,7 @@ impl<E: EthSpec> LoadCase for ForkChoiceTest<E> {
.expect("path must be valid OsStr")
.to_string();
let spec = &testing_spec::<E>(fork_name);
let steps: Vec<Step<String, String>> = yaml_decode_file(&path.join("steps.yaml"))?;
let steps: Vec<Step<String, String, String>> = yaml_decode_file(&path.join("steps.yaml"))?;
// Resolve the object names in `steps.yaml` into actual decoded block/attestation objects.
let steps = steps
.into_iter()
@ -91,6 +105,10 @@ impl<E: EthSpec> LoadCase for ForkChoiceTest<E> {
ssz_decode_file(&path.join(format!("{}.ssz_snappy", attestation)))
.map(|attestation| Step::Attestation { attestation })
}
Step::PowBlock { pow_block } => {
ssz_decode_file(&path.join(format!("{}.ssz_snappy", pow_block)))
.map(|pow_block| Step::PowBlock { pow_block })
}
Step::Checks { checks } => Ok(Step::Checks { checks }),
})
.collect::<Result<_, _>>()?;
@ -133,7 +151,13 @@ impl<E: EthSpec> Case for ForkChoiceTest<E> {
// https://github.com/sigp/lighthouse/issues/2741
//
// We should eventually solve the above issue and remove this `SkippedKnownFailure`.
if self.description == "new_finalized_slot_is_justified_checkpoint_ancestor" {
if self.description == "new_finalized_slot_is_justified_checkpoint_ancestor"
// This test is skipped until we can do retrospective confirmations of the terminal
// block after an optimistic sync.
//
// TODO(merge): enable this test before production.
|| self.description == "block_lookup_failed"
{
return Err(Error::SkippedKnownFailure);
};
@ -145,6 +169,7 @@ impl<E: EthSpec> Case for ForkChoiceTest<E> {
tester.process_block(block.clone(), *valid)?
}
Step::Attestation { attestation } => tester.process_attestation(attestation)?,
Step::PowBlock { pow_block } => tester.process_pow_block(pow_block),
Step::Checks { checks } => {
let Checks {
head,
@ -231,6 +256,15 @@ impl<E: EthSpec> Tester<E> {
));
}
// Drop any blocks that might be loaded in the mock execution layer. Some of these tests
// will provide their own blocks and we want to start from a clean state.
harness
.mock_execution_layer
.as_ref()
.unwrap()
.server
.drop_all_blocks();
assert_eq!(
harness.chain.slot_clock.genesis_duration().as_secs(),
genesis_time
@ -357,6 +391,21 @@ impl<E: EthSpec> Tester<E> {
.map_err(|e| Error::InternalError(format!("attestation import failed with {:?}", e)))
}
pub fn process_pow_block(&self, pow_block: &PowBlock) {
let el = self.harness.mock_execution_layer.as_ref().unwrap();
// The EF tests don't supply a block number. Our mock execution layer is fine with duplicate
// block numbers for the purposes of this test.
let block_number = 0;
el.server.insert_pow_block(
block_number,
pow_block.block_hash,
pow_block.parent_hash,
pow_block.total_difficulty,
);
}
pub fn check_head(&self, expected_head: Head) -> Result<(), Error> {
let chain_head = self.find_head().map(|head| Head {
slot: head.slot,

View File

@ -491,6 +491,34 @@ impl<E: EthSpec + TypeName> Handler for ForkChoiceOnBlockHandler<E> {
}
}
#[derive(Derivative)]
#[derivative(Default(bound = ""))]
pub struct ForkChoiceOnMergeBlockHandler<E>(PhantomData<E>);
impl<E: EthSpec + TypeName> Handler for ForkChoiceOnMergeBlockHandler<E> {
type Case = cases::ForkChoiceTest<E>;
fn config_name() -> &'static str {
E::name()
}
fn runner_name() -> &'static str {
"fork_choice"
}
fn handler_name(&self) -> String {
"on_merge_block".into()
}
fn is_enabled_for_fork(&self, fork_name: ForkName) -> bool {
// These tests check block validity (which may include signatures) and there is no need to
// run them with fake crypto.
cfg!(not(feature = "fake_crypto"))
// These tests only exist for the merge.
&& fork_name == ForkName::Merge
}
}
#[derive(Derivative)]
#[derivative(Default(bound = ""))]
pub struct GenesisValidityHandler<E>(PhantomData<E>);

View File

@ -423,6 +423,12 @@ fn fork_choice_on_block() {
ForkChoiceOnBlockHandler::<MainnetEthSpec>::default().run();
}
#[test]
fn fork_choice_on_merge_block() {
ForkChoiceOnMergeBlockHandler::<MinimalEthSpec>::default().run();
ForkChoiceOnMergeBlockHandler::<MainnetEthSpec>::default().run();
}
#[test]
fn genesis_initialization() {
GenesisInitializationHandler::<MinimalEthSpec>::default().run();