Don't panic in forkchoiceUpdated handler (#3165)

## Issue Addressed

Fix a panic due to misuse of the Tokio executor when processing a forkchoiceUpdated response. We were previously calling `process_invalid_execution_payload` from the async function `update_execution_engine_forkchoice_async`, which resulted in a panic because `process_invalid_execution_payload` contains a call to fork choice, which ultimately calls `block_on`.

An example backtrace can be found here: https://gist.github.com/michaelsproul/ac5da03e203d6ffac672423eaf52fb20

## Proposed Changes

Wrap the call to `process_invalid_execution_payload` in a `spawn_blocking` so that `block_on` is no longer called from an async context.

## Additional Info

- I've been thinking about how to catch bugs like this with static analysis (a new Clippy lint).
- The payload validation tests have been re-worked to support distinct responses from the mock EE for newPayload and forkchoiceUpdated. Three new tests have been added covering the `Invalid`, `InvalidBlockHash` and `InvalidTerminalBlock` cases.
- I think we need a bunch more tests of different legal and illegal variations
This commit is contained in:
Michael Sproul 2022-05-04 23:30:34 +00:00
parent 10795f1c86
commit ae47a93c42
9 changed files with 365 additions and 132 deletions

View File

@ -2208,7 +2208,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
/// This method is generally much more efficient than importing each block using
/// `Self::process_block`.
pub fn process_chain_segment(
&self,
self: &Arc<Self>,
chain_segment: Vec<SignedBeaconBlock<T::EthSpec>>,
) -> ChainSegmentResult<T::EthSpec> {
let mut filtered_chain_segment = Vec::with_capacity(chain_segment.len());
@ -2402,7 +2402,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
/// Returns an `Err` if the given block was invalid, or an error was encountered during
/// verification.
pub fn process_block<B: IntoFullyVerifiedBlock<T>>(
&self,
self: &Arc<Self>,
unverified_block: B,
) -> Result<Hash256, BlockError<T::EthSpec>> {
// Start the Prometheus timer.
@ -3234,7 +3234,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
///
/// See the documentation of `InvalidationOperation` for information about defining `op`.
pub fn process_invalid_execution_payload(
&self,
self: &Arc<Self>,
op: &InvalidationOperation,
) -> Result<(), Error> {
debug!(
@ -3302,7 +3302,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
}
/// Execute the fork choice algorithm and enthrone the result as the canonical head.
pub fn fork_choice(&self) -> Result<(), Error> {
pub fn fork_choice(self: &Arc<Self>) -> Result<(), Error> {
metrics::inc_counter(&metrics::FORK_CHOICE_REQUESTS);
let _timer = metrics::start_timer(&metrics::FORK_CHOICE_TIMES);
@ -3315,7 +3315,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
result
}
fn fork_choice_internal(&self) -> Result<(), Error> {
fn fork_choice_internal(self: &Arc<Self>) -> Result<(), Error> {
// Atomically obtain the head block root and the finalized block.
let (beacon_block_root, finalized_block) = {
let mut fork_choice = self.fork_choice.write();
@ -3718,7 +3718,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
Ok(())
}
pub fn prepare_beacon_proposer_blocking(&self) -> Result<(), Error> {
pub fn prepare_beacon_proposer_blocking(self: &Arc<Self>) -> Result<(), Error> {
let current_slot = self.slot()?;
// Avoids raising an error before Bellatrix.
@ -3750,7 +3750,10 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
/// 1. We're in the tail-end of the slot (as defined by PAYLOAD_PREPARATION_LOOKAHEAD_FACTOR)
/// 2. The head block is one slot (or less) behind the prepare slot (e.g., we're preparing for
/// the next slot and the block at the current slot is already known).
pub async fn prepare_beacon_proposer_async(&self, current_slot: Slot) -> Result<(), Error> {
pub async fn prepare_beacon_proposer_async(
self: &Arc<Self>,
current_slot: Slot,
) -> Result<(), Error> {
let prepare_slot = current_slot + 1;
let prepare_epoch = prepare_slot.epoch(T::EthSpec::slots_per_epoch());
@ -3952,7 +3955,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
}
pub fn update_execution_engine_forkchoice_blocking(
&self,
self: &Arc<Self>,
current_slot: Slot,
) -> Result<(), Error> {
// Avoids raising an error before Bellatrix.
@ -3973,7 +3976,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
}
pub async fn update_execution_engine_forkchoice_async(
&self,
self: &Arc<Self>,
current_slot: Slot,
) -> Result<(), Error> {
let next_slot = current_slot + 1;
@ -4091,7 +4094,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
drop(forkchoice_lock);
match forkchoice_updated_response {
Ok(status) => match &status {
Ok(status) => match status {
PayloadStatus::Valid => {
// Ensure that fork choice knows that the block is no longer optimistic.
if let Err(e) = self
@ -4134,13 +4137,24 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
);
// 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(
&InvalidationOperation::InvalidateMany {
head_block_root,
always_invalidate_head: true,
latest_valid_ancestor: *latest_valid_hash,
},
)?;
let chain = self.clone();
execution_layer
.executor()
.spawn_blocking_handle(
move || {
chain.process_invalid_execution_payload(
&InvalidationOperation::InvalidateMany {
head_block_root,
always_invalidate_head: true,
latest_valid_ancestor: latest_valid_hash,
},
)
},
"process_invalid_execution_payload_many",
)
.ok_or(BeaconChainError::RuntimeShutdown)?
.await
.map_err(BeaconChainError::ProcessInvalidExecutionPayload)??;
Err(BeaconChainError::ExecutionForkChoiceUpdateInvalid { status })
}
@ -4156,11 +4170,22 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
//
// Using a `None` latest valid ancestor will result in only the head block
// being invalidated (no ancestors).
self.process_invalid_execution_payload(
&InvalidationOperation::InvalidateOne {
block_root: head_block_root,
},
)?;
let chain = self.clone();
execution_layer
.executor()
.spawn_blocking_handle(
move || {
chain.process_invalid_execution_payload(
&InvalidationOperation::InvalidateOne {
block_root: head_block_root,
},
)
},
"process_invalid_execution_payload_single",
)
.ok_or(BeaconChainError::RuntimeShutdown)?
.await
.map_err(BeaconChainError::ProcessInvalidExecutionPayload)??;
Err(BeaconChainError::ExecutionForkChoiceUpdateInvalid { status })
}

View File

@ -72,6 +72,7 @@ use state_processing::{
use std::borrow::Cow;
use std::fs;
use std::io::Write;
use std::sync::Arc;
use std::time::Duration;
use store::{Error as DBError, HotColdDB, HotStateSummary, KeyValueStore, StoreOp};
use tree_hash::TreeHash;
@ -577,7 +578,7 @@ pub struct FullyVerifiedBlock<'a, T: BeaconChainTypes> {
pub trait IntoFullyVerifiedBlock<T: BeaconChainTypes>: Sized {
fn into_fully_verified_block(
self,
chain: &BeaconChain<T>,
chain: &Arc<BeaconChain<T>>,
) -> Result<FullyVerifiedBlock<T>, BlockError<T::EthSpec>> {
self.into_fully_verified_block_slashable(chain)
.map(|fully_verified| {
@ -593,7 +594,7 @@ pub trait IntoFullyVerifiedBlock<T: BeaconChainTypes>: Sized {
/// Convert the block to fully-verified form while producing data to aid checking slashability.
fn into_fully_verified_block_slashable(
self,
chain: &BeaconChain<T>,
chain: &Arc<BeaconChain<T>>,
) -> Result<FullyVerifiedBlock<T>, BlockSlashInfo<BlockError<T::EthSpec>>>;
fn block(&self) -> &SignedBeaconBlock<T::EthSpec>;
@ -828,7 +829,7 @@ impl<T: BeaconChainTypes> IntoFullyVerifiedBlock<T> for GossipVerifiedBlock<T> {
/// Completes verification of the wrapped `block`.
fn into_fully_verified_block_slashable(
self,
chain: &BeaconChain<T>,
chain: &Arc<BeaconChain<T>>,
) -> Result<FullyVerifiedBlock<T>, BlockSlashInfo<BlockError<T::EthSpec>>> {
let fully_verified =
SignatureVerifiedBlock::from_gossip_verified_block_check_slashable(self, chain)?;
@ -948,7 +949,7 @@ impl<T: BeaconChainTypes> IntoFullyVerifiedBlock<T> for SignatureVerifiedBlock<T
/// Completes verification of the wrapped `block`.
fn into_fully_verified_block_slashable(
self,
chain: &BeaconChain<T>,
chain: &Arc<BeaconChain<T>>,
) -> Result<FullyVerifiedBlock<T>, BlockSlashInfo<BlockError<T::EthSpec>>> {
let header = self.block.signed_block_header();
let (parent, block) = if let Some(parent) = self.parent {
@ -977,7 +978,7 @@ impl<T: BeaconChainTypes> IntoFullyVerifiedBlock<T> for SignedBeaconBlock<T::Eth
/// and then using that implementation of `IntoFullyVerifiedBlock` to complete verification.
fn into_fully_verified_block_slashable(
self,
chain: &BeaconChain<T>,
chain: &Arc<BeaconChain<T>>,
) -> Result<FullyVerifiedBlock<T>, BlockSlashInfo<BlockError<T::EthSpec>>> {
// Perform an early check to prevent wasting time on irrelevant blocks.
let block_root = check_block_relevancy(&self, None, chain)
@ -1004,7 +1005,7 @@ impl<'a, T: BeaconChainTypes> FullyVerifiedBlock<'a, T> {
block: SignedBeaconBlock<T::EthSpec>,
block_root: Hash256,
parent: PreProcessingSnapshot<T::EthSpec>,
chain: &BeaconChain<T>,
chain: &Arc<BeaconChain<T>>,
) -> Result<Self, BlockError<T::EthSpec>> {
if let Some(parent) = chain.fork_choice.read().get_block(&block.parent_root()) {
// Reject any block where the parent has an invalid payload. It's impossible for a valid

View File

@ -26,6 +26,7 @@ use state_processing::{
};
use std::time::Duration;
use task_executor::ShutdownReason;
use tokio::task::JoinError;
use types::*;
macro_rules! easy_from_to {
@ -170,6 +171,8 @@ pub enum BeaconChainError {
CannotAttestToFinalizedBlock {
beacon_block_root: Hash256,
},
RuntimeShutdown,
ProcessInvalidExecutionPayload(JoinError),
}
easy_from_to!(SlotProcessingError, BeaconChainError);

View File

@ -20,6 +20,7 @@ use state_processing::per_block_processing::{
compute_timestamp_at_slot, is_execution_enabled, is_merge_transition_complete,
partially_verify_execution_payload,
};
use std::sync::Arc;
use types::*;
/// Verify that `execution_payload` contained by `block` is considered valid by an execution
@ -32,7 +33,7 @@ use types::*;
///
/// https://github.com/ethereum/consensus-specs/blob/v1.1.9/specs/bellatrix/beacon-chain.md#notify_new_payload
pub fn notify_new_payload<T: BeaconChainTypes>(
chain: &BeaconChain<T>,
chain: &Arc<BeaconChain<T>>,
state: &BeaconState<T::EthSpec>,
block: BeaconBlockRef<T::EthSpec>,
) -> Result<PayloadVerificationStatus, BlockError<T::EthSpec>> {

View File

@ -172,7 +172,7 @@ async fn state_advance_timer<T: BeaconChainTypes>(
///
/// See the module-level documentation for rationale.
fn advance_head<T: BeaconChainTypes>(
beacon_chain: &BeaconChain<T>,
beacon_chain: &Arc<BeaconChain<T>>,
log: &Logger,
) -> Result<(), Error> {
let current_slot = beacon_chain.slot()?;

View File

@ -21,13 +21,15 @@ const VALIDATOR_COUNT: usize = 32;
type E = MainnetEthSpec;
#[derive(PartialEq, Clone)]
#[derive(PartialEq, Clone, Copy)]
enum Payload {
Valid,
Invalid {
latest_valid_hash: Option<ExecutionBlockHash>,
},
Syncing,
InvalidBlockHash,
InvalidTerminalBlock,
}
struct InvalidPayloadRig {
@ -154,8 +156,9 @@ impl InvalidPayloadRig {
assert_eq!(justified_checkpoint.epoch, 2);
}
/// Import a block while setting the newPayload and forkchoiceUpdated responses to `is_valid`.
fn import_block(&mut self, is_valid: Payload) -> Hash256 {
self.import_block_parametric(is_valid, |error| {
self.import_block_parametric(is_valid, is_valid, |error| {
matches!(
error,
BlockError::ExecutionPayloadError(
@ -183,7 +186,8 @@ impl InvalidPayloadRig {
fn import_block_parametric<F: Fn(&BlockError<E>) -> bool>(
&mut self,
is_valid: Payload,
new_payload_response: Payload,
forkchoice_response: Payload,
evaluate_error: F,
) -> Hash256 {
let mock_execution_layer = self.harness.mock_execution_layer.as_ref().unwrap();
@ -194,15 +198,54 @@ impl InvalidPayloadRig {
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);
let set_new_payload = |payload: Payload| match payload {
Payload::Valid => mock_execution_layer
.server
.all_payloads_valid_on_new_payload(),
Payload::Syncing => mock_execution_layer
.server
.all_payloads_syncing_on_new_payload(true),
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_on_new_payload(latest_valid_hash)
}
Payload::InvalidBlockHash => mock_execution_layer
.server
.all_payloads_invalid_block_hash_on_new_payload(),
Payload::InvalidTerminalBlock => mock_execution_layer
.server
.all_payloads_invalid_terminal_block_on_new_payload(),
};
let set_forkchoice_updated = |payload: Payload| match payload {
Payload::Valid => mock_execution_layer
.server
.all_payloads_valid_on_forkchoice_updated(),
Payload::Syncing => mock_execution_layer
.server
.all_payloads_syncing_on_forkchoice_updated(),
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_on_forkchoice_updated(latest_valid_hash)
}
Payload::InvalidBlockHash => mock_execution_layer
.server
.all_payloads_invalid_block_hash_on_forkchoice_updated(),
Payload::InvalidTerminalBlock => mock_execution_layer
.server
.all_payloads_invalid_terminal_block_on_forkchoice_updated(),
};
match (new_payload_response, forkchoice_response) {
(Payload::Valid | Payload::Syncing, Payload::Valid | Payload::Syncing) => {
if new_payload_response == Payload::Syncing {
set_new_payload(new_payload_response);
set_forkchoice_updated(forkchoice_response);
} else {
mock_execution_layer.server.full_payload_verification();
}
@ -221,10 +264,12 @@ impl InvalidPayloadRig {
let execution_status = self.execution_status(root.into());
match is_valid {
match forkchoice_response {
Payload::Syncing => assert!(execution_status.is_optimistic()),
Payload::Valid => assert!(execution_status.is_valid_and_post_bellatrix()),
Payload::Invalid { .. } => unreachable!(),
Payload::Invalid { .. }
| Payload::InvalidBlockHash
| Payload::InvalidTerminalBlock => unreachable!(),
}
assert_eq!(
@ -233,35 +278,46 @@ impl InvalidPayloadRig {
"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);
(
Payload::Invalid { .. } | Payload::InvalidBlockHash | Payload::InvalidTerminalBlock,
_,
)
| (
_,
Payload::Invalid { .. } | Payload::InvalidBlockHash | Payload::InvalidTerminalBlock,
) => {
set_new_payload(new_payload_response);
set_forkchoice_updated(forkchoice_response);
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"),
Ok(_) => {
// An invalid payload should only be imported initially if its status when
// initially supplied to the EE is Valid or Syncing.
assert!(matches!(
new_payload_response,
Payload::Valid | Payload::Syncing
));
}
};
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"
);
let block_in_forkchoice =
self.harness.chain.fork_choice.read().get_block(&block_root);
if let Payload::Invalid { .. } = new_payload_response {
// A block found to be immediately invalid should not end up in fork choice.
assert_eq!(block_in_forkchoice, None);
assert!(
self.harness.chain.get_block(&block_root).unwrap().is_none(),
"invalid block cannot be accessed via get_block"
);
} else {
// A block imported and then found invalid should have an invalid status.
assert!(block_in_forkchoice.unwrap().execution_status.is_invalid());
}
}
}
@ -317,6 +373,48 @@ fn invalid_payload_invalidates_parent() {
assert_eq!(rig.head_info().block_root, roots[0]);
}
/// Test invalidation of a payload via the fork choice updated message.
///
/// The `invalid_payload` argument determines the type of invalid payload: `Invalid`,
/// `InvalidBlockHash`, etc, taking the `latest_valid_hash` as an argument.
fn immediate_forkchoice_update_invalid_test(
invalid_payload: impl FnOnce(Option<ExecutionBlockHash>) -> Payload,
) {
let mut rig = InvalidPayloadRig::new().enable_attestations();
rig.move_to_terminal_block();
rig.import_block(Payload::Valid); // Import a valid transition block.
rig.move_to_first_justification(Payload::Syncing);
let valid_head_root = rig.import_block(Payload::Valid);
let latest_valid_hash = Some(rig.block_hash(valid_head_root));
// Import a block which returns syncing when supplied via newPayload, and then
// invalid when the forkchoice update is sent.
rig.import_block_parametric(Payload::Syncing, invalid_payload(latest_valid_hash), |_| {
false
});
// The head should be the latest valid block.
assert_eq!(rig.head_info().block_root, valid_head_root);
}
#[test]
fn immediate_forkchoice_update_payload_invalid() {
immediate_forkchoice_update_invalid_test(|latest_valid_hash| Payload::Invalid {
latest_valid_hash,
})
}
#[test]
fn immediate_forkchoice_update_payload_invalid_block_hash() {
immediate_forkchoice_update_invalid_test(|_| Payload::InvalidBlockHash)
}
#[test]
fn immediate_forkchoice_update_payload_invalid_terminal_block() {
immediate_forkchoice_update_invalid_test(|_| Payload::InvalidTerminalBlock)
}
/// Ensure the client tries to exit when the justified checkpoint is invalidated.
#[test]
fn justified_checkpoint_becomes_invalid() {
@ -339,19 +437,17 @@ fn justified_checkpoint_becomes_invalid() {
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 { .. })
)
},
);
let is_valid = Payload::Invalid {
latest_valid_hash: Some(parent_hash_of_justified),
};
rig.import_block_parametric(is_valid, is_valid, |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!(

View File

@ -2011,26 +2011,28 @@ fn weak_subjectivity_sync() {
let seconds_per_slot = spec.seconds_per_slot;
// Initialise a new beacon chain from the finalized checkpoint
let beacon_chain = BeaconChainBuilder::new(MinimalEthSpec)
.store(store.clone())
.custom_spec(test_spec::<E>())
.weak_subjectivity_state(wss_state, wss_block.clone(), genesis_state)
.unwrap()
.logger(log.clone())
.store_migrator_config(MigratorConfig::default().blocking())
.dummy_eth1_backend()
.expect("should build dummy backend")
.testing_slot_clock(Duration::from_secs(seconds_per_slot))
.expect("should configure testing slot clock")
.shutdown_sender(shutdown_tx)
.chain_config(ChainConfig::default())
.event_handler(Some(ServerSentEventHandler::new_with_capacity(
log.clone(),
1,
)))
.monitor_validators(true, vec![], log)
.build()
.expect("should build");
let beacon_chain = Arc::new(
BeaconChainBuilder::new(MinimalEthSpec)
.store(store.clone())
.custom_spec(test_spec::<E>())
.weak_subjectivity_state(wss_state, wss_block.clone(), genesis_state)
.unwrap()
.logger(log.clone())
.store_migrator_config(MigratorConfig::default().blocking())
.dummy_eth1_backend()
.expect("should build dummy backend")
.testing_slot_clock(Duration::from_secs(seconds_per_slot))
.expect("should configure testing slot clock")
.shutdown_sender(shutdown_tx)
.chain_config(ChainConfig::default())
.event_handler(Some(ServerSentEventHandler::new_with_capacity(
log.clone(),
1,
)))
.monitor_validators(true, vec![], log)
.build()
.expect("should build"),
);
// Apply blocks forward to reach head.
let chain_dump = harness.chain.chain_dump().unwrap();

View File

@ -268,7 +268,7 @@ impl ExecutionLayer {
&self.inner.builders
}
fn executor(&self) -> &TaskExecutor {
pub fn executor(&self) -> &TaskExecutor {
&self.inner.executor
}

View File

@ -126,51 +126,156 @@ impl<T: EthSpec> MockServer<T> {
self.ctx.previous_request.lock().take()
}
pub fn all_payloads_valid(&self) {
let response = StaticNewPayloadResponse {
status: PayloadStatusV1 {
status: PayloadStatusV1Status::Valid,
latest_valid_hash: None,
validation_error: None,
},
should_import: true,
};
*self.ctx.static_forkchoice_updated_response.lock() = Some(response.status.clone());
pub fn set_new_payload_response(&self, response: StaticNewPayloadResponse) {
*self.ctx.static_new_payload_response.lock() = Some(response)
}
pub fn set_forkchoice_updated_response(&self, status: PayloadStatusV1) {
*self.ctx.static_forkchoice_updated_response.lock() = Some(status);
}
fn valid_status() -> PayloadStatusV1 {
PayloadStatusV1 {
status: PayloadStatusV1Status::Valid,
latest_valid_hash: None,
validation_error: None,
}
}
fn valid_new_payload_response() -> StaticNewPayloadResponse {
StaticNewPayloadResponse {
status: Self::valid_status(),
should_import: true,
}
}
fn syncing_status() -> PayloadStatusV1 {
PayloadStatusV1 {
status: PayloadStatusV1Status::Syncing,
latest_valid_hash: None,
validation_error: None,
}
}
fn syncing_new_payload_response(should_import: bool) -> StaticNewPayloadResponse {
StaticNewPayloadResponse {
status: Self::syncing_status(),
should_import,
}
}
fn invalid_status(latest_valid_hash: ExecutionBlockHash) -> PayloadStatusV1 {
PayloadStatusV1 {
status: PayloadStatusV1Status::Invalid,
latest_valid_hash: Some(latest_valid_hash),
validation_error: Some("static response".into()),
}
}
fn invalid_new_payload_response(
latest_valid_hash: ExecutionBlockHash,
) -> StaticNewPayloadResponse {
StaticNewPayloadResponse {
status: Self::invalid_status(latest_valid_hash),
should_import: true,
}
}
fn invalid_block_hash_status() -> PayloadStatusV1 {
PayloadStatusV1 {
status: PayloadStatusV1Status::InvalidBlockHash,
latest_valid_hash: None,
validation_error: Some("static response".into()),
}
}
fn invalid_block_hash_new_payload_response() -> StaticNewPayloadResponse {
StaticNewPayloadResponse {
status: Self::invalid_block_hash_status(),
should_import: true,
}
}
fn invalid_terminal_block_status() -> PayloadStatusV1 {
PayloadStatusV1 {
status: PayloadStatusV1Status::InvalidTerminalBlock,
latest_valid_hash: None,
validation_error: Some("static response".into()),
}
}
fn invalid_terminal_block_new_payload_response() -> StaticNewPayloadResponse {
StaticNewPayloadResponse {
status: Self::invalid_terminal_block_status(),
should_import: true,
}
}
pub fn all_payloads_valid(&self) {
self.all_payloads_valid_on_new_payload();
self.all_payloads_valid_on_forkchoice_updated();
}
pub fn all_payloads_valid_on_new_payload(&self) {
self.set_new_payload_response(Self::valid_new_payload_response());
}
pub fn all_payloads_valid_on_forkchoice_updated(&self) {
self.set_forkchoice_updated_response(Self::valid_status());
}
/// Setting `should_import = true` simulates an EE that initially returns `SYNCING` but obtains
/// the block via it's own means (e.g., devp2p).
/// the block via its 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_forkchoice_updated_response.lock() = Some(response.status.clone());
*self.ctx.static_new_payload_response.lock() = Some(response)
self.all_payloads_syncing_on_new_payload(should_import);
self.all_payloads_syncing_on_forkchoice_updated();
}
pub fn all_payloads_syncing_on_new_payload(&self, should_import: bool) {
self.set_new_payload_response(Self::syncing_new_payload_response(should_import));
}
pub fn all_payloads_syncing_on_forkchoice_updated(&self) {
self.set_forkchoice_updated_response(Self::syncing_status());
}
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_forkchoice_updated_response.lock() = Some(response.status.clone());
*self.ctx.static_new_payload_response.lock() = Some(response)
self.all_payloads_invalid_on_new_payload(latest_valid_hash);
self.all_payloads_invalid_on_forkchoice_updated(latest_valid_hash);
}
/// Disables any static payload response so the execution block generator will do its own
pub fn all_payloads_invalid_on_new_payload(&self, latest_valid_hash: ExecutionBlockHash) {
self.set_new_payload_response(Self::invalid_new_payload_response(latest_valid_hash));
}
pub fn all_payloads_invalid_on_forkchoice_updated(
&self,
latest_valid_hash: ExecutionBlockHash,
) {
self.set_forkchoice_updated_response(Self::invalid_status(latest_valid_hash));
}
pub fn all_payloads_invalid_block_hash_on_new_payload(&self) {
self.set_new_payload_response(Self::invalid_block_hash_new_payload_response());
}
pub fn all_payloads_invalid_block_hash_on_forkchoice_updated(&self) {
self.set_forkchoice_updated_response(Self::invalid_block_hash_status());
}
pub fn all_payloads_invalid_terminal_block_on_new_payload(&self) {
self.set_new_payload_response(Self::invalid_terminal_block_new_payload_response());
}
pub fn all_payloads_invalid_terminal_block_on_forkchoice_updated(&self) {
self.set_forkchoice_updated_response(Self::invalid_terminal_block_status());
}
/// Disables any static payload responses so the execution block generator will do its own
/// verification.
pub fn full_payload_verification(&self) {
*self.ctx.static_new_payload_response.lock() = None
*self.ctx.static_new_payload_response.lock() = None;
*self.ctx.static_forkchoice_updated_response.lock() = None;
}
pub fn insert_pow_block(