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:
parent
44a7b37ce3
commit
5f0fef2d1e
@ -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,
|
||||
|
@ -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.
|
||||
///
|
||||
|
@ -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),
|
||||
|
306
beacon_node/beacon_chain/src/execution_payload.rs
Normal file
306
beacon_node/beacon_chain/src/execution_payload.rs
Normal 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))
|
||||
}
|
@ -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;
|
||||
|
@ -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;
|
||||
|
@ -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);
|
||||
|
@ -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)]
|
||||
|
@ -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,
|
||||
|
@ -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> {
|
||||
|
@ -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):
|
||||
|
@ -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,
|
||||
|
@ -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>);
|
||||
|
@ -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();
|
||||
|
Loading…
Reference in New Issue
Block a user