Merge branch 'capella' of https://github.com/sigp/lighthouse into eip4844

# Conflicts:
#	beacon_node/beacon_chain/src/beacon_chain.rs
#	beacon_node/beacon_chain/src/block_verification.rs
#	beacon_node/beacon_chain/src/test_utils.rs
#	beacon_node/execution_layer/src/engine_api.rs
#	beacon_node/execution_layer/src/engine_api/http.rs
#	beacon_node/execution_layer/src/lib.rs
#	beacon_node/execution_layer/src/test_utils/handle_rpc.rs
#	beacon_node/http_api/src/lib.rs
#	beacon_node/http_api/tests/fork_tests.rs
#	beacon_node/network/src/beacon_processor/mod.rs
#	beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs
#	beacon_node/network/src/beacon_processor/worker/sync_methods.rs
#	beacon_node/operation_pool/src/bls_to_execution_changes.rs
#	beacon_node/operation_pool/src/lib.rs
#	beacon_node/operation_pool/src/persistence.rs
#	consensus/serde_utils/src/u256_hex_be_opt.rs
#	testing/antithesis/Dockerfile.libvoidstar
This commit is contained in:
realbigsean 2023-02-07 12:12:56 -05:00
commit 26a296246d
No known key found for this signature in database
GPG Key ID: BE1B3DB104F6C788
53 changed files with 1680 additions and 560 deletions

6
Cargo.lock generated
View File

@ -1074,8 +1074,10 @@ dependencies = [
"lazy_static", "lazy_static",
"lighthouse_metrics", "lighthouse_metrics",
"lighthouse_network", "lighthouse_network",
"logging",
"monitoring_api", "monitoring_api",
"network", "network",
"operation_pool",
"parking_lot 0.12.1", "parking_lot 0.12.1",
"sensitive_url", "sensitive_url",
"serde", "serde",
@ -1085,6 +1087,7 @@ dependencies = [
"slasher_service", "slasher_service",
"slog", "slog",
"slot_clock", "slot_clock",
"state_processing",
"store", "store",
"task_executor", "task_executor",
"time 0.3.17", "time 0.3.17",
@ -3238,6 +3241,7 @@ dependencies = [
"logging", "logging",
"lru 0.7.8", "lru 0.7.8",
"network", "network",
"operation_pool",
"parking_lot 0.12.1", "parking_lot 0.12.1",
"proto_array", "proto_array",
"safe_arith", "safe_arith",
@ -5042,6 +5046,7 @@ dependencies = [
"lru_cache", "lru_cache",
"matches", "matches",
"num_cpus", "num_cpus",
"operation_pool",
"rand 0.8.5", "rand 0.8.5",
"rlp", "rlp",
"slog", "slog",
@ -5377,6 +5382,7 @@ dependencies = [
"lighthouse_metrics", "lighthouse_metrics",
"maplit", "maplit",
"parking_lot 0.12.1", "parking_lot 0.12.1",
"rand 0.8.5",
"rayon", "rayon",
"serde", "serde",
"serde_derive", "serde_derive",

View File

@ -167,7 +167,8 @@ lint:
-A clippy::from-over-into \ -A clippy::from-over-into \
-A clippy::upper-case-acronyms \ -A clippy::upper-case-acronyms \
-A clippy::vec-init-then-push \ -A clippy::vec-init-then-push \
-A clippy::question-mark -A clippy::question-mark \
-A clippy::uninlined-format-args
nightly-lint: nightly-lint:
cp .github/custom/clippy.toml . cp .github/custom/clippy.toml .

View File

@ -27,6 +27,11 @@
//! ▼ //! ▼
//! impl VerifiedAttestation //! impl VerifiedAttestation
//! ``` //! ```
// Ignore this lint for `AttestationSlashInfo` which is of comparable size to the non-error types it
// is returned alongside.
#![allow(clippy::result_large_err)]
mod batch; mod batch;
use crate::{ use crate::{

View File

@ -73,7 +73,7 @@ use fork_choice::{
use futures::channel::mpsc::Sender; use futures::channel::mpsc::Sender;
use itertools::process_results; use itertools::process_results;
use itertools::Itertools; use itertools::Itertools;
use operation_pool::{AttestationRef, OperationPool, PersistedOperationPool}; use operation_pool::{AttestationRef, OperationPool, PersistedOperationPool, ReceivedPreCapella};
use parking_lot::{Mutex, RwLock}; use parking_lot::{Mutex, RwLock};
use proto_array::{CountUnrealizedFull, DoNotReOrg, ProposerHeadError}; use proto_array::{CountUnrealizedFull, DoNotReOrg, ProposerHeadError};
use safe_arith::SafeArith; use safe_arith::SafeArith;
@ -1020,7 +1020,9 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
.ok_or(Error::ExecutionLayerMissing)? .ok_or(Error::ExecutionLayerMissing)?
.get_payload_by_block_hash(exec_block_hash, fork) .get_payload_by_block_hash(exec_block_hash, fork)
.await .await
.map_err(|e| Error::ExecutionLayerErrorPayloadReconstruction(exec_block_hash, e))? .map_err(|e| {
Error::ExecutionLayerErrorPayloadReconstruction(exec_block_hash, Box::new(e))
})?
.ok_or(Error::BlockHashMissingFromExecutionLayer(exec_block_hash))?; .ok_or(Error::BlockHashMissingFromExecutionLayer(exec_block_hash))?;
//FIXME(sean) avoid the clone by comparing refs to headers (`as_execution_payload_header` method ?) //FIXME(sean) avoid the clone by comparing refs to headers (`as_execution_payload_header` method ?)
@ -1040,8 +1042,6 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
return Err(Error::InconsistentPayloadReconstructed { return Err(Error::InconsistentPayloadReconstructed {
slot: blinded_block.slot(), slot: blinded_block.slot(),
exec_block_hash, exec_block_hash,
canonical_payload_root: execution_payload_header.tree_hash_root(),
reconstructed_payload_root: header_from_payload.tree_hash_root(),
canonical_transactions_root: execution_payload_header.transactions_root(), canonical_transactions_root: execution_payload_header.transactions_root(),
reconstructed_transactions_root: header_from_payload.transactions_root(), reconstructed_transactions_root: header_from_payload.transactions_root(),
}); });
@ -2375,10 +2375,11 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
pub fn import_bls_to_execution_change( pub fn import_bls_to_execution_change(
&self, &self,
bls_to_execution_change: SigVerifiedOp<SignedBlsToExecutionChange, T::EthSpec>, bls_to_execution_change: SigVerifiedOp<SignedBlsToExecutionChange, T::EthSpec>,
received_pre_capella: ReceivedPreCapella,
) -> bool { ) -> bool {
if self.eth1_chain.is_some() { if self.eth1_chain.is_some() {
self.op_pool self.op_pool
.insert_bls_to_execution_change(bls_to_execution_change) .insert_bls_to_execution_change(bls_to_execution_change, received_pre_capella)
} else { } else {
false false
} }

View File

@ -46,6 +46,11 @@
//! END //! END
//! //!
//! ``` //! ```
// Ignore this lint for `BlockSlashInfo` which is of comparable size to the non-error types it is
// returned alongside.
#![allow(clippy::result_large_err)]
use crate::blob_verification::{ use crate::blob_verification::{
validate_blob_for_gossip, AsBlock, AvailableBlock, BlobError, BlockWrapper, IntoAvailableBlock, validate_blob_for_gossip, AsBlock, AvailableBlock, BlobError, BlockWrapper, IntoAvailableBlock,
IntoBlockWrapper, IntoBlockWrapper,

View File

@ -0,0 +1,135 @@
//! Provides tools for checking if a node is ready for the Capella upgrade and following merge
//! transition.
use crate::{BeaconChain, BeaconChainTypes};
use execution_layer::http::{
ENGINE_FORKCHOICE_UPDATED_V2, ENGINE_GET_PAYLOAD_V2, ENGINE_NEW_PAYLOAD_V2,
};
use serde::{Deserialize, Serialize};
use std::fmt;
use std::time::Duration;
use types::*;
/// The time before the Capella fork when we will start issuing warnings about preparation.
use super::merge_readiness::SECONDS_IN_A_WEEK;
pub const CAPELLA_READINESS_PREPARATION_SECONDS: u64 = SECONDS_IN_A_WEEK * 2;
pub const ENGINE_CAPABILITIES_REFRESH_INTERVAL: u64 = 300;
#[derive(Debug, Serialize, Deserialize)]
#[serde(rename_all = "snake_case")]
#[serde(tag = "type")]
pub enum CapellaReadiness {
/// The execution engine is capella-enabled (as far as we can tell)
Ready,
/// The EL can be reached and has the correct configuration, however it's not yet synced.
NotSynced,
/// We are connected to an execution engine which doesn't support the V2 engine api methods
V2MethodsNotSupported { error: String },
/// The transition configuration with the EL failed, there might be a problem with
/// connectivity, authentication or a difference in configuration.
ExchangeCapabilitiesFailed { error: String },
/// The user has not configured an execution endpoint
NoExecutionEndpoint,
}
impl fmt::Display for CapellaReadiness {
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
match self {
CapellaReadiness::Ready => {
write!(f, "This node appears ready for Capella.")
}
CapellaReadiness::ExchangeCapabilitiesFailed { error } => write!(
f,
"Could not exchange capabilities with the \
execution endpoint: {}",
error
),
CapellaReadiness::NotSynced => write!(
f,
"The execution endpoint is connected and configured, \
however it is not yet synced"
),
CapellaReadiness::NoExecutionEndpoint => write!(
f,
"The --execution-endpoint flag is not specified, this is a \
requirement post-merge"
),
CapellaReadiness::V2MethodsNotSupported { error } => write!(
f,
"The execution endpoint does not appear to support \
the required engine api methods for Capella: {}",
error
),
}
}
}
impl<T: BeaconChainTypes> BeaconChain<T> {
/// Returns `true` if capella epoch is set and Capella fork has occurred or will
/// occur within `CAPELLA_READINESS_PREPARATION_SECONDS`
pub fn is_time_to_prepare_for_capella(&self, current_slot: Slot) -> bool {
if let Some(capella_epoch) = self.spec.capella_fork_epoch {
let capella_slot = capella_epoch.start_slot(T::EthSpec::slots_per_epoch());
let capella_readiness_preparation_slots =
CAPELLA_READINESS_PREPARATION_SECONDS / self.spec.seconds_per_slot;
// Return `true` if Capella has happened or is within the preparation time.
current_slot + capella_readiness_preparation_slots > capella_slot
} else {
// The Capella fork epoch has not been defined yet, no need to prepare.
false
}
}
/// Attempts to connect to the EL and confirm that it is ready for capella.
pub async fn check_capella_readiness(&self) -> CapellaReadiness {
if let Some(el) = self.execution_layer.as_ref() {
match el
.get_engine_capabilities(Some(Duration::from_secs(
ENGINE_CAPABILITIES_REFRESH_INTERVAL,
)))
.await
{
Err(e) => {
// The EL was either unreachable or responded with an error
CapellaReadiness::ExchangeCapabilitiesFailed {
error: format!("{:?}", e),
}
}
Ok(capabilities) => {
let mut missing_methods = String::from("Required Methods Unsupported:");
let mut all_good = true;
if !capabilities.get_payload_v2 {
missing_methods.push(' ');
missing_methods.push_str(ENGINE_GET_PAYLOAD_V2);
all_good = false;
}
if !capabilities.forkchoice_updated_v2 {
missing_methods.push(' ');
missing_methods.push_str(ENGINE_FORKCHOICE_UPDATED_V2);
all_good = false;
}
if !capabilities.new_payload_v2 {
missing_methods.push(' ');
missing_methods.push_str(ENGINE_NEW_PAYLOAD_V2);
all_good = false;
}
if all_good {
if !el.is_synced_for_notifier().await {
// The EL is not synced.
CapellaReadiness::NotSynced
} else {
CapellaReadiness::Ready
}
} else {
CapellaReadiness::V2MethodsNotSupported {
error: missing_methods,
}
}
}
}
} else {
CapellaReadiness::NoExecutionEndpoint
}
}
}

View File

@ -143,13 +143,11 @@ pub enum BeaconChainError {
BuilderMissing, BuilderMissing,
ExecutionLayerMissing, ExecutionLayerMissing,
BlockVariantLacksExecutionPayload(Hash256), BlockVariantLacksExecutionPayload(Hash256),
ExecutionLayerErrorPayloadReconstruction(ExecutionBlockHash, execution_layer::Error), ExecutionLayerErrorPayloadReconstruction(ExecutionBlockHash, Box<execution_layer::Error>),
BlockHashMissingFromExecutionLayer(ExecutionBlockHash), BlockHashMissingFromExecutionLayer(ExecutionBlockHash),
InconsistentPayloadReconstructed { InconsistentPayloadReconstructed {
slot: Slot, slot: Slot,
exec_block_hash: ExecutionBlockHash, exec_block_hash: ExecutionBlockHash,
canonical_payload_root: Hash256,
reconstructed_payload_root: Hash256,
canonical_transactions_root: Hash256, canonical_transactions_root: Hash256,
reconstructed_transactions_root: Hash256, reconstructed_transactions_root: Hash256,
}, },

View File

@ -43,7 +43,7 @@ impl ForkChoiceSignalTx {
/// ///
/// Return an error if the provided `slot` is strictly less than any previously provided slot. /// Return an error if the provided `slot` is strictly less than any previously provided slot.
pub fn notify_fork_choice_complete(&self, slot: Slot) -> Result<(), BeaconChainError> { pub fn notify_fork_choice_complete(&self, slot: Slot) -> Result<(), BeaconChainError> {
let &(ref lock, ref condvar) = &*self.pair; let (lock, condvar) = &*self.pair;
let mut current_slot = lock.lock(); let mut current_slot = lock.lock();
@ -72,7 +72,7 @@ impl Default for ForkChoiceSignalTx {
impl ForkChoiceSignalRx { impl ForkChoiceSignalRx {
pub fn wait_for_fork_choice(&self, slot: Slot, timeout: Duration) -> ForkChoiceWaitResult { pub fn wait_for_fork_choice(&self, slot: Slot, timeout: Duration) -> ForkChoiceWaitResult {
let &(ref lock, ref condvar) = &*self.pair; let (lock, condvar) = &*self.pair;
let mut current_slot = lock.lock(); let mut current_slot = lock.lock();

View File

@ -12,6 +12,7 @@ mod block_times_cache;
mod block_verification; mod block_verification;
pub mod builder; pub mod builder;
pub mod canonical_head; pub mod canonical_head;
pub mod capella_readiness;
pub mod chain_config; pub mod chain_config;
mod early_attester_cache; mod early_attester_cache;
mod errors; mod errors;

View File

@ -8,7 +8,7 @@ use std::fmt::Write;
use types::*; use types::*;
/// The time before the Bellatrix fork when we will start issuing warnings about preparation. /// The time before the Bellatrix fork when we will start issuing warnings about preparation.
const SECONDS_IN_A_WEEK: u64 = 604800; pub const SECONDS_IN_A_WEEK: u64 = 604800;
pub const MERGE_READINESS_PREPARATION_SECONDS: u64 = SECONDS_IN_A_WEEK * 2; pub const MERGE_READINESS_PREPARATION_SECONDS: u64 = SECONDS_IN_A_WEEK * 2;
#[derive(Default, Debug, Serialize, Deserialize)] #[derive(Default, Debug, Serialize, Deserialize)]

View File

@ -2,6 +2,7 @@
mod migration_schema_v12; mod migration_schema_v12;
mod migration_schema_v13; mod migration_schema_v13;
mod migration_schema_v14; mod migration_schema_v14;
mod migration_schema_v15;
use crate::beacon_chain::{BeaconChainTypes, ETH1_CACHE_DB_KEY}; use crate::beacon_chain::{BeaconChainTypes, ETH1_CACHE_DB_KEY};
use crate::eth1_chain::SszEth1; use crate::eth1_chain::SszEth1;
@ -123,6 +124,14 @@ pub fn migrate_schema<T: BeaconChainTypes>(
let ops = migration_schema_v14::downgrade_from_v14::<T>(db.clone(), log)?; let ops = migration_schema_v14::downgrade_from_v14::<T>(db.clone(), log)?;
db.store_schema_version_atomically(to, ops) db.store_schema_version_atomically(to, ops)
} }
(SchemaVersion(14), SchemaVersion(15)) => {
let ops = migration_schema_v15::upgrade_to_v15::<T>(db.clone(), log)?;
db.store_schema_version_atomically(to, ops)
}
(SchemaVersion(15), SchemaVersion(14)) => {
let ops = migration_schema_v15::downgrade_from_v15::<T>(db.clone(), log)?;
db.store_schema_version_atomically(to, ops)
}
// Anything else is an error. // Anything else is an error.
(_, _) => Err(HotColdDBError::UnsupportedSchemaVersion { (_, _) => Err(HotColdDBError::UnsupportedSchemaVersion {
target_version: to, target_version: to,

View File

@ -0,0 +1,78 @@
use crate::beacon_chain::{BeaconChainTypes, OP_POOL_DB_KEY};
use operation_pool::{
PersistedOperationPool, PersistedOperationPoolV14, PersistedOperationPoolV15,
};
use slog::{debug, info, Logger};
use std::sync::Arc;
use store::{Error, HotColdDB, KeyValueStoreOp, StoreItem};
pub fn upgrade_to_v15<T: BeaconChainTypes>(
db: Arc<HotColdDB<T::EthSpec, T::HotStore, T::ColdStore>>,
log: Logger,
) -> Result<Vec<KeyValueStoreOp>, Error> {
// Load a V14 op pool and transform it to V15.
let PersistedOperationPoolV14::<T::EthSpec> {
attestations,
sync_contributions,
attester_slashings,
proposer_slashings,
voluntary_exits,
bls_to_execution_changes,
} = if let Some(op_pool_v14) = db.get_item(&OP_POOL_DB_KEY)? {
op_pool_v14
} else {
debug!(log, "Nothing to do, no operation pool stored");
return Ok(vec![]);
};
let v15 = PersistedOperationPool::V15(PersistedOperationPoolV15 {
attestations,
sync_contributions,
attester_slashings,
proposer_slashings,
voluntary_exits,
bls_to_execution_changes,
// Initialize with empty set
capella_bls_change_broadcast_indices: <_>::default(),
});
Ok(vec![v15.as_kv_store_op(OP_POOL_DB_KEY)])
}
pub fn downgrade_from_v15<T: BeaconChainTypes>(
db: Arc<HotColdDB<T::EthSpec, T::HotStore, T::ColdStore>>,
log: Logger,
) -> Result<Vec<KeyValueStoreOp>, Error> {
// Load a V15 op pool and transform it to V14.
let PersistedOperationPoolV15 {
attestations,
sync_contributions,
attester_slashings,
proposer_slashings,
voluntary_exits,
bls_to_execution_changes,
capella_bls_change_broadcast_indices,
} = if let Some(PersistedOperationPool::<T::EthSpec>::V15(op_pool)) =
db.get_item(&OP_POOL_DB_KEY)?
{
op_pool
} else {
debug!(log, "Nothing to do, no operation pool stored");
return Ok(vec![]);
};
info!(
log,
"Forgetting address changes for Capella broadcast";
"count" => capella_bls_change_broadcast_indices.len(),
);
let v14 = PersistedOperationPoolV14 {
attestations,
sync_contributions,
attester_slashings,
proposer_slashings,
voluntary_exits,
bls_to_execution_changes,
};
Ok(vec![v14.as_kv_store_op(OP_POOL_DB_KEY)])
}

View File

@ -316,6 +316,26 @@ where
self self
} }
/// Initializes the BLS withdrawal keypairs for `num_keypairs` validators to
/// the "determistic" values, regardless of wether or not the validator has
/// a BLS or execution address in the genesis deposits.
///
/// This aligns with the withdrawal commitments used in the "interop"
/// genesis states.
pub fn deterministic_withdrawal_keypairs(self, num_keypairs: usize) -> Self {
self.withdrawal_keypairs(
types::test_utils::generate_deterministic_keypairs(num_keypairs)
.into_iter()
.map(Option::Some)
.collect(),
)
}
pub fn withdrawal_keypairs(mut self, withdrawal_keypairs: Vec<Option<Keypair>>) -> Self {
self.withdrawal_keypairs = withdrawal_keypairs;
self
}
pub fn default_spec(self) -> Self { pub fn default_spec(self) -> Self {
self.spec_or_default(None) self.spec_or_default(None)
} }
@ -376,7 +396,6 @@ where
.collect::<Result<_, _>>() .collect::<Result<_, _>>()
.unwrap(); .unwrap();
let spec = MainnetEthSpec::default_spec();
let config = execution_layer::Config { let config = execution_layer::Config {
execution_endpoints: urls, execution_endpoints: urls,
secret_files: vec![], secret_files: vec![],
@ -387,7 +406,6 @@ where
config, config,
self.runtime.task_executor.clone(), self.runtime.task_executor.clone(),
self.log.clone(), self.log.clone(),
&spec,
) )
.unwrap(); .unwrap();
@ -429,6 +447,7 @@ where
DEFAULT_TERMINAL_BLOCK, DEFAULT_TERMINAL_BLOCK,
shanghai_time, shanghai_time,
eip4844_time, eip4844_time,
None,
Some(JwtKey::from_slice(&DEFAULT_JWT_SECRET).unwrap()), Some(JwtKey::from_slice(&DEFAULT_JWT_SECRET).unwrap()),
spec, spec,
None, None,
@ -438,7 +457,11 @@ where
self self
} }
pub fn mock_execution_layer_with_builder(mut self, beacon_url: SensitiveUrl) -> Self { pub fn mock_execution_layer_with_builder(
mut self,
beacon_url: SensitiveUrl,
builder_threshold: Option<u128>,
) -> Self {
// Get a random unused port // Get a random unused port
let port = unused_port::unused_tcp_port().unwrap(); let port = unused_port::unused_tcp_port().unwrap();
let builder_url = SensitiveUrl::parse(format!("http://127.0.0.1:{port}").as_str()).unwrap(); let builder_url = SensitiveUrl::parse(format!("http://127.0.0.1:{port}").as_str()).unwrap();
@ -455,6 +478,7 @@ where
DEFAULT_TERMINAL_BLOCK, DEFAULT_TERMINAL_BLOCK,
shanghai_time, shanghai_time,
eip4844_time, eip4844_time,
builder_threshold,
Some(JwtKey::from_slice(&DEFAULT_JWT_SECRET).unwrap()), Some(JwtKey::from_slice(&DEFAULT_JWT_SECRET).unwrap()),
spec.clone(), spec.clone(),
Some(builder_url.clone()), Some(builder_url.clone()),

View File

@ -6,6 +6,10 @@ edition = "2021"
[dev-dependencies] [dev-dependencies]
serde_yaml = "0.8.13" serde_yaml = "0.8.13"
logging = { path = "../../common/logging" }
state_processing = { path = "../../consensus/state_processing" }
operation_pool = { path = "../operation_pool" }
tokio = "1.14.0"
[dependencies] [dependencies]
beacon_chain = { path = "../beacon_chain" } beacon_chain = { path = "../beacon_chain" }

View File

@ -0,0 +1,322 @@
use crate::*;
use lighthouse_network::PubsubMessage;
use network::NetworkMessage;
use slog::{debug, info, warn, Logger};
use slot_clock::SlotClock;
use std::cmp;
use std::collections::HashSet;
use std::mem;
use std::time::Duration;
use tokio::sync::mpsc::UnboundedSender;
use tokio::time::sleep;
use types::EthSpec;
/// The size of each chunk of addresses changes to be broadcast at the Capella
/// fork.
const BROADCAST_CHUNK_SIZE: usize = 128;
/// The delay between broadcasting each chunk.
const BROADCAST_CHUNK_DELAY: Duration = Duration::from_millis(500);
/// If the Capella fork has already been reached, `broadcast_address_changes` is
/// called immediately.
///
/// If the Capella fork has not been reached, waits until the start of the fork
/// epoch and then calls `broadcast_address_changes`.
pub async fn broadcast_address_changes_at_capella<T: BeaconChainTypes>(
chain: &BeaconChain<T>,
network_send: UnboundedSender<NetworkMessage<T::EthSpec>>,
log: &Logger,
) {
let spec = &chain.spec;
let slot_clock = &chain.slot_clock;
let capella_fork_slot = if let Some(epoch) = spec.capella_fork_epoch {
epoch.start_slot(T::EthSpec::slots_per_epoch())
} else {
// Exit now if Capella is not defined.
return;
};
// Wait until the Capella fork epoch.
while chain.slot().map_or(true, |slot| slot < capella_fork_slot) {
match slot_clock.duration_to_slot(capella_fork_slot) {
Some(duration) => {
// Sleep until the Capella fork.
sleep(duration).await;
break;
}
None => {
// We were unable to read the slot clock wait another slot
// and then try again.
sleep(slot_clock.slot_duration()).await;
}
}
}
// The following function will be called in two scenarios:
//
// 1. The node has been running for some time and the Capella fork has just
// been reached.
// 2. The node has just started and it is *after* the Capella fork.
broadcast_address_changes(chain, network_send, log).await
}
/// Broadcasts any address changes that are flagged for broadcasting at the
/// Capella fork epoch.
///
/// Address changes are published in chunks, with a delay between each chunk.
/// This helps reduce the load on the P2P network and also helps prevent us from
/// clogging our `network_send` channel and being late to publish
/// blocks, attestations, etc.
pub async fn broadcast_address_changes<T: BeaconChainTypes>(
chain: &BeaconChain<T>,
network_send: UnboundedSender<NetworkMessage<T::EthSpec>>,
log: &Logger,
) {
let head = chain.head_snapshot();
let mut changes = chain
.op_pool
.get_bls_to_execution_changes_received_pre_capella(&head.beacon_state, &chain.spec);
while !changes.is_empty() {
// This `split_off` approach is to allow us to have owned chunks of the
// `changes` vec. The `std::slice::Chunks` method uses references and
// the `itertools` iterator that achives this isn't `Send` so it doesn't
// work well with the `sleep` at the end of the loop.
let tail = changes.split_off(cmp::min(BROADCAST_CHUNK_SIZE, changes.len()));
let chunk = mem::replace(&mut changes, tail);
let mut published_indices = HashSet::with_capacity(BROADCAST_CHUNK_SIZE);
let mut num_ok = 0;
let mut num_err = 0;
// Publish each individual address change.
for address_change in chunk {
let validator_index = address_change.message.validator_index;
let pubsub_message = PubsubMessage::BlsToExecutionChange(Box::new(address_change));
let message = NetworkMessage::Publish {
messages: vec![pubsub_message],
};
// It seems highly unlikely that this unbounded send will fail, but
// we handle the result nontheless.
if let Err(e) = network_send.send(message) {
debug!(
log,
"Failed to publish change message";
"error" => ?e,
"validator_index" => validator_index
);
num_err += 1;
} else {
debug!(
log,
"Published address change message";
"validator_index" => validator_index
);
num_ok += 1;
published_indices.insert(validator_index);
}
}
// Remove any published indices from the list of indices that need to be
// published.
chain
.op_pool
.register_indices_broadcasted_at_capella(&published_indices);
info!(
log,
"Published address change messages";
"num_published" => num_ok,
);
if num_err > 0 {
warn!(
log,
"Failed to publish address changes";
"info" => "failed messages will be retried",
"num_unable_to_publish" => num_err,
);
}
sleep(BROADCAST_CHUNK_DELAY).await;
}
debug!(
log,
"Address change routine complete";
);
}
#[cfg(not(debug_assertions))] // Tests run too slow in debug.
#[cfg(test)]
mod tests {
use super::*;
use beacon_chain::test_utils::{BeaconChainHarness, EphemeralHarnessType};
use operation_pool::ReceivedPreCapella;
use state_processing::{SigVerifiedOp, VerifyOperation};
use std::collections::HashSet;
use tokio::sync::mpsc;
use types::*;
type E = MainnetEthSpec;
pub const VALIDATOR_COUNT: usize = BROADCAST_CHUNK_SIZE * 3;
pub const EXECUTION_ADDRESS: Address = Address::repeat_byte(42);
struct Tester {
harness: BeaconChainHarness<EphemeralHarnessType<E>>,
/// Changes which should be broadcast at the Capella fork.
received_pre_capella_changes: Vec<SigVerifiedOp<SignedBlsToExecutionChange, E>>,
/// Changes which should *not* be broadcast at the Capella fork.
not_received_pre_capella_changes: Vec<SigVerifiedOp<SignedBlsToExecutionChange, E>>,
}
impl Tester {
fn new() -> Self {
let altair_fork_epoch = Epoch::new(0);
let bellatrix_fork_epoch = Epoch::new(0);
let capella_fork_epoch = Epoch::new(2);
let mut spec = E::default_spec();
spec.altair_fork_epoch = Some(altair_fork_epoch);
spec.bellatrix_fork_epoch = Some(bellatrix_fork_epoch);
spec.capella_fork_epoch = Some(capella_fork_epoch);
let harness = BeaconChainHarness::builder(E::default())
.spec(spec)
.logger(logging::test_logger())
.deterministic_keypairs(VALIDATOR_COUNT)
.deterministic_withdrawal_keypairs(VALIDATOR_COUNT)
.fresh_ephemeral_store()
.mock_execution_layer()
.build();
Self {
harness,
received_pre_capella_changes: <_>::default(),
not_received_pre_capella_changes: <_>::default(),
}
}
fn produce_verified_address_change(
&self,
validator_index: u64,
) -> SigVerifiedOp<SignedBlsToExecutionChange, E> {
let change = self
.harness
.make_bls_to_execution_change(validator_index, EXECUTION_ADDRESS);
let head = self.harness.chain.head_snapshot();
change
.validate(&head.beacon_state, &self.harness.spec)
.unwrap()
}
fn produce_received_pre_capella_changes(mut self, indices: Vec<u64>) -> Self {
for validator_index in indices {
self.received_pre_capella_changes
.push(self.produce_verified_address_change(validator_index));
}
self
}
fn produce_not_received_pre_capella_changes(mut self, indices: Vec<u64>) -> Self {
for validator_index in indices {
self.not_received_pre_capella_changes
.push(self.produce_verified_address_change(validator_index));
}
self
}
async fn run(self) {
let harness = self.harness;
let chain = harness.chain.clone();
let mut broadcast_indices = HashSet::new();
for change in self.received_pre_capella_changes {
broadcast_indices.insert(change.as_inner().message.validator_index);
chain
.op_pool
.insert_bls_to_execution_change(change, ReceivedPreCapella::Yes);
}
let mut non_broadcast_indices = HashSet::new();
for change in self.not_received_pre_capella_changes {
non_broadcast_indices.insert(change.as_inner().message.validator_index);
chain
.op_pool
.insert_bls_to_execution_change(change, ReceivedPreCapella::No);
}
harness.set_current_slot(
chain
.spec
.capella_fork_epoch
.unwrap()
.start_slot(E::slots_per_epoch()),
);
let (sender, mut receiver) = mpsc::unbounded_channel();
broadcast_address_changes_at_capella(&chain, sender, &logging::test_logger()).await;
let mut broadcasted_changes = vec![];
while let Some(NetworkMessage::Publish { mut messages }) = receiver.recv().await {
match messages.pop().unwrap() {
PubsubMessage::BlsToExecutionChange(change) => broadcasted_changes.push(change),
_ => panic!("unexpected message"),
}
}
assert_eq!(
broadcasted_changes.len(),
broadcast_indices.len(),
"all expected changes should have been broadcast"
);
for broadcasted in &broadcasted_changes {
assert!(
!non_broadcast_indices.contains(&broadcasted.message.validator_index),
"messages not flagged for broadcast should not have been broadcast"
);
}
let head = chain.head_snapshot();
assert!(
chain
.op_pool
.get_bls_to_execution_changes_received_pre_capella(
&head.beacon_state,
&chain.spec,
)
.is_empty(),
"there shouldn't be any capella broadcast changes left in the op pool"
);
}
}
// Useful for generating even-numbered indices. Required since only even
// numbered genesis validators have BLS credentials.
fn even_indices(start: u64, count: usize) -> Vec<u64> {
(start..).filter(|i| i % 2 == 0).take(count).collect()
}
#[tokio::test]
async fn one_chunk() {
Tester::new()
.produce_received_pre_capella_changes(even_indices(0, 4))
.produce_not_received_pre_capella_changes(even_indices(10, 4))
.run()
.await;
}
#[tokio::test]
async fn multiple_chunks() {
Tester::new()
.produce_received_pre_capella_changes(even_indices(0, BROADCAST_CHUNK_SIZE * 3 / 2))
.run()
.await;
}
}

View File

@ -1,3 +1,4 @@
use crate::address_change_broadcast::broadcast_address_changes_at_capella;
use crate::config::{ClientGenesis, Config as ClientConfig}; use crate::config::{ClientGenesis, Config as ClientConfig};
use crate::notifier::spawn_notifier; use crate::notifier::spawn_notifier;
use crate::Client; use crate::Client;
@ -154,7 +155,6 @@ where
config, config,
context.executor.clone(), context.executor.clone(),
context.log().clone(), context.log().clone(),
&spec,
) )
.map_err(|e| format!("unable to start execution layer endpoints: {:?}", e))?; .map_err(|e| format!("unable to start execution layer endpoints: {:?}", e))?;
Some(execution_layer) Some(execution_layer)
@ -809,6 +809,25 @@ where
// Spawns a routine that polls the `exchange_transition_configuration` endpoint. // Spawns a routine that polls the `exchange_transition_configuration` endpoint.
execution_layer.spawn_transition_configuration_poll(beacon_chain.spec.clone()); execution_layer.spawn_transition_configuration_poll(beacon_chain.spec.clone());
} }
// Spawn a service to publish BLS to execution changes at the Capella fork.
if let Some(network_senders) = self.network_senders {
let inner_chain = beacon_chain.clone();
let broadcast_context =
runtime_context.service_context("addr_bcast".to_string());
let log = broadcast_context.log().clone();
broadcast_context.executor.spawn(
async move {
broadcast_address_changes_at_capella(
&inner_chain,
network_senders.network_send(),
&log,
)
.await
},
"addr_broadcast",
);
}
} }
start_proposer_prep_service(runtime_context.executor.clone(), beacon_chain.clone()); start_proposer_prep_service(runtime_context.executor.clone(), beacon_chain.clone());

View File

@ -1,5 +1,6 @@
extern crate slog; extern crate slog;
mod address_change_broadcast;
pub mod config; pub mod config;
mod metrics; mod metrics;
mod notifier; mod notifier;

View File

@ -1,5 +1,6 @@
use crate::metrics; use crate::metrics;
use beacon_chain::{ use beacon_chain::{
capella_readiness::CapellaReadiness,
merge_readiness::{MergeConfig, MergeReadiness}, merge_readiness::{MergeConfig, MergeReadiness},
BeaconChain, BeaconChainTypes, ExecutionStatus, BeaconChain, BeaconChainTypes, ExecutionStatus,
}; };
@ -313,6 +314,7 @@ pub fn spawn_notifier<T: BeaconChainTypes>(
eth1_logging(&beacon_chain, &log); eth1_logging(&beacon_chain, &log);
merge_readiness_logging(current_slot, &beacon_chain, &log).await; merge_readiness_logging(current_slot, &beacon_chain, &log).await;
capella_readiness_logging(current_slot, &beacon_chain, &log).await;
} }
}; };
@ -350,12 +352,15 @@ async fn merge_readiness_logging<T: BeaconChainTypes>(
} }
if merge_completed && !has_execution_layer { if merge_completed && !has_execution_layer {
error!( if !beacon_chain.is_time_to_prepare_for_capella(current_slot) {
log, // logging of the EE being offline is handled in `capella_readiness_logging()`
"Execution endpoint required"; error!(
"info" => "you need an execution engine to validate blocks, see: \ log,
https://lighthouse-book.sigmaprime.io/merge-migration.html" "Execution endpoint required";
); "info" => "you need an execution engine to validate blocks, see: \
https://lighthouse-book.sigmaprime.io/merge-migration.html"
);
}
return; return;
} }
@ -419,6 +424,60 @@ async fn merge_readiness_logging<T: BeaconChainTypes>(
} }
} }
/// Provides some helpful logging to users to indicate if their node is ready for Capella
async fn capella_readiness_logging<T: BeaconChainTypes>(
current_slot: Slot,
beacon_chain: &BeaconChain<T>,
log: &Logger,
) {
let capella_completed = beacon_chain
.canonical_head
.cached_head()
.snapshot
.beacon_block
.message()
.body()
.execution_payload()
.map_or(false, |payload| payload.withdrawals_root().is_ok());
let has_execution_layer = beacon_chain.execution_layer.is_some();
if capella_completed && has_execution_layer
|| !beacon_chain.is_time_to_prepare_for_capella(current_slot)
{
return;
}
if capella_completed && !has_execution_layer {
error!(
log,
"Execution endpoint required";
"info" => "you need a Capella enabled execution engine to validate blocks, see: \
https://lighthouse-book.sigmaprime.io/merge-migration.html"
);
return;
}
match beacon_chain.check_capella_readiness().await {
CapellaReadiness::Ready => {
info!(log, "Ready for Capella")
}
readiness @ CapellaReadiness::ExchangeCapabilitiesFailed { error: _ } => {
error!(
log,
"Not ready for Capella";
"info" => %readiness,
"hint" => "try updating Lighthouse and/or the execution layer",
)
}
readiness => warn!(
log,
"Not ready for Capella";
"info" => %readiness,
),
}
}
fn eth1_logging<T: BeaconChainTypes>(beacon_chain: &BeaconChain<T>, log: &Logger) { fn eth1_logging<T: BeaconChainTypes>(beacon_chain: &BeaconChain<T>, log: &Logger) {
let current_slot_opt = beacon_chain.slot().ok(); let current_slot_opt = beacon_chain.slot().ok();

View File

@ -122,7 +122,7 @@ impl SszEth1Cache {
cache: self.deposit_cache.to_deposit_cache()?, cache: self.deposit_cache.to_deposit_cache()?,
last_processed_block: self.last_processed_block, last_processed_block: self.last_processed_block,
}), }),
endpoint: endpoint_from_config(&config, &spec) endpoint: endpoint_from_config(&config)
.map_err(|e| format!("Failed to create endpoint: {:?}", e))?, .map_err(|e| format!("Failed to create endpoint: {:?}", e))?,
to_finalize: RwLock::new(None), to_finalize: RwLock::new(None),
// Set the remote head_block zero when creating a new instance. We only care about // Set the remote head_block zero when creating a new instance. We only care about

View File

@ -363,7 +363,7 @@ impl Default for Config {
} }
} }
pub fn endpoint_from_config(config: &Config, spec: &ChainSpec) -> Result<HttpJsonRpc, String> { pub fn endpoint_from_config(config: &Config) -> Result<HttpJsonRpc, String> {
match config.endpoint.clone() { match config.endpoint.clone() {
Eth1Endpoint::Auth { Eth1Endpoint::Auth {
endpoint, endpoint,
@ -373,16 +373,11 @@ pub fn endpoint_from_config(config: &Config, spec: &ChainSpec) -> Result<HttpJso
} => { } => {
let auth = Auth::new_with_path(jwt_path, jwt_id, jwt_version) let auth = Auth::new_with_path(jwt_path, jwt_id, jwt_version)
.map_err(|e| format!("Failed to initialize jwt auth: {:?}", e))?; .map_err(|e| format!("Failed to initialize jwt auth: {:?}", e))?;
HttpJsonRpc::new_with_auth( HttpJsonRpc::new_with_auth(endpoint, auth, Some(config.execution_timeout_multiplier))
endpoint, .map_err(|e| format!("Failed to create eth1 json rpc client: {:?}", e))
auth,
Some(config.execution_timeout_multiplier),
spec,
)
.map_err(|e| format!("Failed to create eth1 json rpc client: {:?}", e))
} }
Eth1Endpoint::NoAuth(endpoint) => { Eth1Endpoint::NoAuth(endpoint) => {
HttpJsonRpc::new(endpoint, Some(config.execution_timeout_multiplier), spec) HttpJsonRpc::new(endpoint, Some(config.execution_timeout_multiplier))
.map_err(|e| format!("Failed to create eth1 json rpc client: {:?}", e)) .map_err(|e| format!("Failed to create eth1 json rpc client: {:?}", e))
} }
} }
@ -409,7 +404,7 @@ impl Service {
deposit_cache: RwLock::new(DepositUpdater::new( deposit_cache: RwLock::new(DepositUpdater::new(
config.deposit_contract_deploy_block, config.deposit_contract_deploy_block,
)), )),
endpoint: endpoint_from_config(&config, &spec)?, endpoint: endpoint_from_config(&config)?,
to_finalize: RwLock::new(None), to_finalize: RwLock::new(None),
remote_head_block: RwLock::new(None), remote_head_block: RwLock::new(None),
config: RwLock::new(config), config: RwLock::new(config),
@ -438,7 +433,7 @@ impl Service {
inner: Arc::new(Inner { inner: Arc::new(Inner {
block_cache: <_>::default(), block_cache: <_>::default(),
deposit_cache: RwLock::new(deposit_cache), deposit_cache: RwLock::new(deposit_cache),
endpoint: endpoint_from_config(&config, &spec) endpoint: endpoint_from_config(&config)
.map_err(Error::FailedToInitializeFromSnapshot)?, .map_err(Error::FailedToInitializeFromSnapshot)?,
to_finalize: RwLock::new(None), to_finalize: RwLock::new(None),
remote_head_block: RwLock::new(None), remote_head_block: RwLock::new(None),

View File

@ -494,8 +494,7 @@ mod deposit_tree {
let mut deposit_counts = vec![]; let mut deposit_counts = vec![];
let client = let client =
HttpJsonRpc::new(SensitiveUrl::parse(&eth1.endpoint()).unwrap(), None, spec) HttpJsonRpc::new(SensitiveUrl::parse(&eth1.endpoint()).unwrap(), None).unwrap();
.unwrap();
// Perform deposits to the smart contract, recording it's state along the way. // Perform deposits to the smart contract, recording it's state along the way.
for deposit in &deposits { for deposit in &deposits {
@ -599,12 +598,8 @@ mod http {
.expect("should start eth1 environment"); .expect("should start eth1 environment");
let deposit_contract = &eth1.deposit_contract; let deposit_contract = &eth1.deposit_contract;
let web3 = eth1.web3(); let web3 = eth1.web3();
let client = HttpJsonRpc::new( let client =
SensitiveUrl::parse(&eth1.endpoint()).unwrap(), HttpJsonRpc::new(SensitiveUrl::parse(&eth1.endpoint()).unwrap(), None).unwrap();
None,
&MainnetEthSpec::default_spec(),
)
.unwrap();
let block_number = get_block_number(&web3).await; let block_number = get_block_number(&web3).await;
let logs = blocking_deposit_logs(&client, &eth1, 0..block_number).await; let logs = blocking_deposit_logs(&client, &eth1, 0..block_number).await;
@ -720,8 +715,7 @@ mod fast {
) )
.unwrap(); .unwrap();
let client = let client =
HttpJsonRpc::new(SensitiveUrl::parse(&eth1.endpoint()).unwrap(), None, &spec) HttpJsonRpc::new(SensitiveUrl::parse(&eth1.endpoint()).unwrap(), None).unwrap();
.unwrap();
let n = 10; let n = 10;
let deposits: Vec<_> = (0..n).map(|_| random_deposit_data()).collect(); let deposits: Vec<_> = (0..n).map(|_| random_deposit_data()).collect();
for deposit in &deposits { for deposit in &deposits {

View File

@ -1,4 +1,9 @@
use crate::engines::ForkchoiceState; use crate::engines::ForkchoiceState;
use crate::http::{
ENGINE_EXCHANGE_TRANSITION_CONFIGURATION_V1, ENGINE_FORKCHOICE_UPDATED_V1,
ENGINE_FORKCHOICE_UPDATED_V2, ENGINE_GET_PAYLOAD_V1, ENGINE_GET_PAYLOAD_V2,
ENGINE_NEW_PAYLOAD_V1, ENGINE_NEW_PAYLOAD_V2,
};
use crate::BlobTxConversionError; use crate::BlobTxConversionError;
pub use ethers_core::types::Transaction; pub use ethers_core::types::Transaction;
use ethers_core::utils::rlp::{self, Decodable, Rlp}; use ethers_core::utils::rlp::{self, Decodable, Rlp};
@ -10,8 +15,8 @@ use std::convert::TryFrom;
use strum::IntoStaticStr; use strum::IntoStaticStr;
use superstruct::superstruct; use superstruct::superstruct;
pub use types::{ pub use types::{
Address, EthSpec, ExecutionBlockHash, ExecutionPayload, ExecutionPayloadHeader, FixedVector, Address, EthSpec, ExecutionBlockHash, ExecutionPayload, ExecutionPayloadHeader,
ForkName, Hash256, Uint256, VariableList, Withdrawal, ExecutionPayloadRef, FixedVector, ForkName, Hash256, Uint256, VariableList, Withdrawal,
}; };
use types::{ExecutionPayloadCapella, ExecutionPayloadEip4844, ExecutionPayloadMerge}; use types::{ExecutionPayloadCapella, ExecutionPayloadEip4844, ExecutionPayloadMerge};
@ -330,6 +335,8 @@ pub struct ProposeBlindedBlockResponse {
#[superstruct( #[superstruct(
variants(Merge, Capella, Eip4844), variants(Merge, Capella, Eip4844),
variant_attributes(derive(Clone, Debug, PartialEq),), variant_attributes(derive(Clone, Debug, PartialEq),),
map_into(ExecutionPayload),
map_ref_into(ExecutionPayloadRef),
cast_error(ty = "Error", expr = "Error::IncorrectStateVariant"), cast_error(ty = "Error", expr = "Error::IncorrectStateVariant"),
partial_getter_error(ty = "Error", expr = "Error::IncorrectStateVariant") partial_getter_error(ty = "Error", expr = "Error::IncorrectStateVariant")
)] )]
@ -344,27 +351,49 @@ pub struct GetPayloadResponse<T: EthSpec> {
pub block_value: Uint256, pub block_value: Uint256,
} }
impl<T: EthSpec> GetPayloadResponse<T> { impl<'a, T: EthSpec> From<GetPayloadResponseRef<'a, T>> for ExecutionPayloadRef<'a, T> {
pub fn execution_payload(self) -> ExecutionPayload<T> { fn from(response: GetPayloadResponseRef<'a, T>) -> Self {
match self { map_get_payload_response_ref_into_execution_payload_ref!(&'a _, response, |inner, cons| {
GetPayloadResponse::Merge(response) => { cons(&inner.execution_payload)
ExecutionPayload::Merge(response.execution_payload) })
} }
GetPayloadResponse::Capella(response) => { }
ExecutionPayload::Capella(response.execution_payload)
} impl<T: EthSpec> From<GetPayloadResponse<T>> for ExecutionPayload<T> {
GetPayloadResponse::Eip4844(response) => { fn from(response: GetPayloadResponse<T>) -> Self {
ExecutionPayload::Eip4844(response.execution_payload) map_get_payload_response_into_execution_payload!(response, |inner, cons| {
} cons(inner.execution_payload)
})
}
}
impl<T: EthSpec> From<GetPayloadResponse<T>> for (ExecutionPayload<T>, Uint256) {
fn from(response: GetPayloadResponse<T>) -> Self {
match response {
GetPayloadResponse::Merge(inner) => (
ExecutionPayload::Merge(inner.execution_payload),
inner.block_value,
),
GetPayloadResponse::Capella(inner) => (
ExecutionPayload::Capella(inner.execution_payload),
inner.block_value,
),
GetPayloadResponse::Eip4844(inner) => (
ExecutionPayload::Eip4844(inner.execution_payload),
inner.block_value,
),
} }
} }
} }
// This name is work in progress, it could impl<T: EthSpec> GetPayloadResponse<T> {
// change when this method is actually proposed pub fn execution_payload_ref(&self) -> ExecutionPayloadRef<T> {
// but I'm writing this as it has been described self.to_ref().into()
}
}
#[derive(Clone, Copy, Debug)] #[derive(Clone, Copy, Debug)]
pub struct SupportedApis { pub struct EngineCapabilities {
pub new_payload_v1: bool, pub new_payload_v1: bool,
pub new_payload_v2: bool, pub new_payload_v2: bool,
pub new_payload_v3: bool, pub new_payload_v3: bool,
@ -375,3 +404,32 @@ pub struct SupportedApis {
pub get_payload_v3: bool, pub get_payload_v3: bool,
pub exchange_transition_configuration_v1: bool, pub exchange_transition_configuration_v1: bool,
} }
impl EngineCapabilities {
pub fn to_response(&self) -> Vec<&str> {
let mut response = Vec::new();
if self.new_payload_v1 {
response.push(ENGINE_NEW_PAYLOAD_V1);
}
if self.new_payload_v2 {
response.push(ENGINE_NEW_PAYLOAD_V2);
}
if self.forkchoice_updated_v1 {
response.push(ENGINE_FORKCHOICE_UPDATED_V1);
}
if self.forkchoice_updated_v2 {
response.push(ENGINE_FORKCHOICE_UPDATED_V2);
}
if self.get_payload_v1 {
response.push(ENGINE_GET_PAYLOAD_V1);
}
if self.get_payload_v2 {
response.push(ENGINE_GET_PAYLOAD_V2);
}
if self.exchange_transition_configuration_v1 {
response.push(ENGINE_EXCHANGE_TRANSITION_CONFIGURATION_V1);
}
response
}
}

View File

@ -7,10 +7,11 @@ use reqwest::header::CONTENT_TYPE;
use sensitive_url::SensitiveUrl; use sensitive_url::SensitiveUrl;
use serde::de::DeserializeOwned; use serde::de::DeserializeOwned;
use serde_json::json; use serde_json::json;
use tokio::sync::RwLock; use std::collections::HashSet;
use tokio::sync::Mutex;
use std::time::Duration; use std::time::{Duration, SystemTime};
use types::{ChainSpec, EthSpec}; use types::EthSpec;
pub use deposit_log::{DepositLog, Log}; pub use deposit_log::{DepositLog, Log};
pub use reqwest::Client; pub use reqwest::Client;
@ -50,8 +51,37 @@ pub const ENGINE_EXCHANGE_TRANSITION_CONFIGURATION_V1: &str =
"engine_exchangeTransitionConfigurationV1"; "engine_exchangeTransitionConfigurationV1";
pub const ENGINE_EXCHANGE_TRANSITION_CONFIGURATION_V1_TIMEOUT: Duration = Duration::from_secs(1); pub const ENGINE_EXCHANGE_TRANSITION_CONFIGURATION_V1_TIMEOUT: Duration = Duration::from_secs(1);
pub const ENGINE_EXCHANGE_CAPABILITIES: &str = "engine_exchangeCapabilities";
pub const ENGINE_EXCHANGE_CAPABILITIES_TIMEOUT: Duration = Duration::from_secs(1);
/// This error is returned during a `chainId` call by Geth. /// This error is returned during a `chainId` call by Geth.
pub const EIP155_ERROR_STR: &str = "chain not synced beyond EIP-155 replay-protection fork block"; pub const EIP155_ERROR_STR: &str = "chain not synced beyond EIP-155 replay-protection fork block";
/// This code is returned by all clients when a method is not supported
/// (verified geth, nethermind, erigon, besu)
pub const METHOD_NOT_FOUND_CODE: i64 = -32601;
pub static LIGHTHOUSE_CAPABILITIES: &[&str] = &[
ENGINE_NEW_PAYLOAD_V1,
ENGINE_NEW_PAYLOAD_V2,
ENGINE_GET_PAYLOAD_V1,
ENGINE_GET_PAYLOAD_V2,
ENGINE_FORKCHOICE_UPDATED_V1,
ENGINE_FORKCHOICE_UPDATED_V2,
ENGINE_EXCHANGE_TRANSITION_CONFIGURATION_V1,
];
/// This is necessary because a user might run a capella-enabled version of
/// lighthouse before they update to a capella-enabled execution engine.
// TODO (mark): rip this out once we are post-capella on mainnet
pub static PRE_CAPELLA_ENGINE_CAPABILITIES: EngineCapabilities = EngineCapabilities {
new_payload_v1: true,
new_payload_v2: false,
forkchoice_updated_v1: true,
forkchoice_updated_v2: false,
get_payload_v1: true,
get_payload_v2: false,
exchange_transition_configuration_v1: true,
};
/// Contains methods to convert arbitrary bytes to an ETH2 deposit contract object. /// Contains methods to convert arbitrary bytes to an ETH2 deposit contract object.
pub mod deposit_log { pub mod deposit_log {
@ -528,11 +558,47 @@ pub mod deposit_methods {
} }
} }
#[derive(Clone, Debug)]
pub struct CapabilitiesCacheEntry {
engine_capabilities: EngineCapabilities,
fetch_time: SystemTime,
}
impl CapabilitiesCacheEntry {
pub fn new(engine_capabilities: EngineCapabilities) -> Self {
Self {
engine_capabilities,
fetch_time: SystemTime::now(),
}
}
pub fn engine_capabilities(&self) -> &EngineCapabilities {
&self.engine_capabilities
}
pub fn age(&self) -> Duration {
// duration_since() may fail because measurements taken earlier
// are not guaranteed to always be before later measurements
// due to anomalies such as the system clock being adjusted
// either forwards or backwards
//
// In such cases, we'll just say the age is zero
SystemTime::now()
.duration_since(self.fetch_time)
.unwrap_or(Duration::ZERO)
}
/// returns `true` if the entry's age is >= age_limit
pub fn older_than(&self, age_limit: Option<Duration>) -> bool {
age_limit.map_or(false, |limit| self.age() >= limit)
}
}
pub struct HttpJsonRpc { pub struct HttpJsonRpc {
pub client: Client, pub client: Client,
pub url: SensitiveUrl, pub url: SensitiveUrl,
pub execution_timeout_multiplier: u32, pub execution_timeout_multiplier: u32,
pub cached_supported_apis: RwLock<Option<SupportedApis>>, pub engine_capabilities_cache: Mutex<Option<CapabilitiesCacheEntry>>,
auth: Option<Auth>, auth: Option<Auth>,
} }
@ -540,29 +606,12 @@ impl HttpJsonRpc {
pub fn new( pub fn new(
url: SensitiveUrl, url: SensitiveUrl,
execution_timeout_multiplier: Option<u32>, execution_timeout_multiplier: Option<u32>,
spec: &ChainSpec,
) -> Result<Self, Error> { ) -> Result<Self, Error> {
// FIXME: remove this `cached_supported_apis` spec hack once the `engine_getCapabilities`
// method is implemented in all execution clients:
// https://github.com/ethereum/execution-apis/issues/321
let cached_supported_apis = RwLock::new(Some(SupportedApis {
new_payload_v1: true,
new_payload_v2: spec.capella_fork_epoch.is_some() || spec.eip4844_fork_epoch.is_some(),
new_payload_v3: spec.eip4844_fork_epoch.is_some(),
forkchoice_updated_v1: true,
forkchoice_updated_v2: spec.capella_fork_epoch.is_some()
|| spec.eip4844_fork_epoch.is_some(),
get_payload_v1: true,
get_payload_v2: spec.capella_fork_epoch.is_some() || spec.eip4844_fork_epoch.is_some(),
get_payload_v3: spec.eip4844_fork_epoch.is_some(),
exchange_transition_configuration_v1: true,
}));
Ok(Self { Ok(Self {
client: Client::builder().build()?, client: Client::builder().build()?,
url, url,
execution_timeout_multiplier: execution_timeout_multiplier.unwrap_or(1), execution_timeout_multiplier: execution_timeout_multiplier.unwrap_or(1),
cached_supported_apis, engine_capabilities_cache: Mutex::new(None),
auth: None, auth: None,
}) })
} }
@ -571,29 +620,12 @@ impl HttpJsonRpc {
url: SensitiveUrl, url: SensitiveUrl,
auth: Auth, auth: Auth,
execution_timeout_multiplier: Option<u32>, execution_timeout_multiplier: Option<u32>,
spec: &ChainSpec,
) -> Result<Self, Error> { ) -> Result<Self, Error> {
// FIXME: remove this `cached_supported_apis` spec hack once the `engine_getCapabilities`
// method is implemented in all execution clients:
// https://github.com/ethereum/execution-apis/issues/321
let cached_supported_apis = RwLock::new(Some(SupportedApis {
new_payload_v1: true,
new_payload_v2: spec.capella_fork_epoch.is_some() || spec.eip4844_fork_epoch.is_some(),
new_payload_v3: spec.eip4844_fork_epoch.is_some(),
forkchoice_updated_v1: true,
forkchoice_updated_v2: spec.capella_fork_epoch.is_some()
|| spec.eip4844_fork_epoch.is_some(),
get_payload_v1: true,
get_payload_v2: spec.capella_fork_epoch.is_some() || spec.eip4844_fork_epoch.is_some(),
get_payload_v3: spec.eip4844_fork_epoch.is_some(),
exchange_transition_configuration_v1: true,
}));
Ok(Self { Ok(Self {
client: Client::builder().build()?, client: Client::builder().build()?,
url, url,
execution_timeout_multiplier: execution_timeout_multiplier.unwrap_or(1), execution_timeout_multiplier: execution_timeout_multiplier.unwrap_or(1),
cached_supported_apis, engine_capabilities_cache: Mutex::new(None),
auth: Some(auth), auth: Some(auth),
}) })
} }
@ -791,7 +823,7 @@ impl HttpJsonRpc {
pub async fn get_payload_v1<T: EthSpec>( pub async fn get_payload_v1<T: EthSpec>(
&self, &self,
payload_id: PayloadId, payload_id: PayloadId,
) -> Result<ExecutionPayload<T>, Error> { ) -> Result<GetPayloadResponse<T>, Error> {
let params = json!([JsonPayloadIdRequest::from(payload_id)]); let params = json!([JsonPayloadIdRequest::from(payload_id)]);
let payload_v1: JsonExecutionPayloadV1<T> = self let payload_v1: JsonExecutionPayloadV1<T> = self
@ -802,7 +834,11 @@ impl HttpJsonRpc {
) )
.await?; .await?;
Ok(JsonExecutionPayload::V1(payload_v1).into()) Ok(GetPayloadResponse::Merge(GetPayloadResponseMerge {
execution_payload: payload_v1.into(),
// Have to guess zero here as we don't know the value
block_value: Uint256::zero(),
}))
} }
pub async fn get_payload_v2<T: EthSpec>( pub async fn get_payload_v2<T: EthSpec>(
@ -961,37 +997,67 @@ impl HttpJsonRpc {
Ok(response) Ok(response)
} }
// TODO: This is currently a stub for the `engine_getCapabilities` pub async fn exchange_capabilities(&self) -> Result<EngineCapabilities, Error> {
// method. This stub is unused because we set cached_supported_apis let params = json!([LIGHTHOUSE_CAPABILITIES]);
// in the constructor based on the `spec`
// Implement this once the execution clients support it let response: Result<HashSet<String>, _> = self
// https://github.com/ethereum/execution-apis/issues/321 .rpc_request(
pub async fn get_capabilities(&self) -> Result<SupportedApis, Error> { ENGINE_EXCHANGE_CAPABILITIES,
Ok(SupportedApis { params,
new_payload_v1: true, ENGINE_EXCHANGE_CAPABILITIES_TIMEOUT * self.execution_timeout_multiplier,
new_payload_v2: true, )
new_payload_v3: true, .await;
forkchoice_updated_v1: true,
forkchoice_updated_v2: true, match response {
get_payload_v1: true, // TODO (mark): rip this out once we are post capella on mainnet
get_payload_v2: true, Err(error) => match error {
get_payload_v3: true, Error::ServerMessage { code, message: _ } if code == METHOD_NOT_FOUND_CODE => {
exchange_transition_configuration_v1: true, Ok(PRE_CAPELLA_ENGINE_CAPABILITIES)
}) }
_ => Err(error),
},
Ok(capabilities) => Ok(EngineCapabilities {
new_payload_v1: capabilities.contains(ENGINE_NEW_PAYLOAD_V1),
new_payload_v2: capabilities.contains(ENGINE_NEW_PAYLOAD_V2),
forkchoice_updated_v1: capabilities.contains(ENGINE_FORKCHOICE_UPDATED_V1),
forkchoice_updated_v2: capabilities.contains(ENGINE_FORKCHOICE_UPDATED_V2),
get_payload_v1: capabilities.contains(ENGINE_GET_PAYLOAD_V1),
get_payload_v2: capabilities.contains(ENGINE_GET_PAYLOAD_V2),
exchange_transition_configuration_v1: capabilities
.contains(ENGINE_EXCHANGE_TRANSITION_CONFIGURATION_V1),
}),
}
} }
pub async fn set_cached_supported_apis(&self, supported_apis: Option<SupportedApis>) { pub async fn clear_exchange_capabilties_cache(&self) {
*self.cached_supported_apis.write().await = supported_apis; *self.engine_capabilities_cache.lock().await = None;
} }
pub async fn get_cached_supported_apis(&self) -> Result<SupportedApis, Error> { /// Returns the execution engine capabilities resulting from a call to
let cached_opt = *self.cached_supported_apis.read().await; /// engine_exchangeCapabilities. If the capabilities cache is not populated,
if let Some(supported_apis) = cached_opt { /// or if it is populated with a cached result of age >= `age_limit`, this
Ok(supported_apis) /// method will fetch the result from the execution engine and populate the
/// cache before returning it. Otherwise it will return a cached result from
/// a previous call.
///
/// Set `age_limit` to `None` to always return the cached result
/// Set `age_limit` to `Some(Duration::ZERO)` to force fetching from EE
pub async fn get_engine_capabilities(
&self,
age_limit: Option<Duration>,
) -> Result<EngineCapabilities, Error> {
let mut lock = self.engine_capabilities_cache.lock().await;
if lock
.as_ref()
.map_or(true, |entry| entry.older_than(age_limit))
{
let engine_capabilities = self.exchange_capabilities().await?;
*lock = Some(CapabilitiesCacheEntry::new(engine_capabilities));
Ok(engine_capabilities)
} else { } else {
let supported_apis = self.get_capabilities().await?; // here entry is guaranteed to exist so unwrap() is safe
self.set_cached_supported_apis(Some(supported_apis)).await; Ok(*lock.as_ref().unwrap().engine_capabilities())
Ok(supported_apis)
} }
} }
@ -1001,12 +1067,12 @@ impl HttpJsonRpc {
&self, &self,
execution_payload: ExecutionPayload<T>, execution_payload: ExecutionPayload<T>,
) -> Result<PayloadStatusV1, Error> { ) -> Result<PayloadStatusV1, Error> {
let supported_apis = self.get_cached_supported_apis().await?; let engine_capabilities = self.get_engine_capabilities(None).await?;
if supported_apis.new_payload_v3 { if engine_capabilities.new_payload_v3 {
self.new_payload_v3(execution_payload).await self.new_payload_v3(execution_payload).await
} else if supported_apis.new_payload_v2 { }else if engine_capabilities.new_payload_v2 {
self.new_payload_v2(execution_payload).await self.new_payload_v2(execution_payload).await
} else if supported_apis.new_payload_v1 { } else if engine_capabilities.new_payload_v1 {
self.new_payload_v1(execution_payload).await self.new_payload_v1(execution_payload).await
} else { } else {
Err(Error::RequiredMethodUnsupported("engine_newPayload")) Err(Error::RequiredMethodUnsupported("engine_newPayload"))
@ -1019,22 +1085,13 @@ impl HttpJsonRpc {
&self, &self,
fork_name: ForkName, fork_name: ForkName,
payload_id: PayloadId, payload_id: PayloadId,
) -> Result<ExecutionPayload<T>, Error> { ) -> Result<GetPayloadResponse<T>, Error> {
let supported_apis = self.get_cached_supported_apis().await?; let engine_capabilities = self.get_engine_capabilities(None).await?;
if supported_apis.get_payload_v3 { if engine_capabilities.get_payload_v3 {
Ok(self self.get_payload_v3(fork_name, payload_id).await
.get_payload_v3(fork_name, payload_id) } else if engine_capabilities.get_payload_v2 {
.await? self.get_payload_v2(fork_name, payload_id).await
.execution_payload()) } else if engine_capabilities.new_payload_v1 {
} else if supported_apis.get_payload_v2 {
// TODO: modify this method to return GetPayloadResponse instead
// of throwing away the `block_value` and returning only the
// ExecutionPayload
Ok(self
.get_payload_v2(fork_name, payload_id)
.await?
.execution_payload())
} else if supported_apis.get_payload_v1 {
self.get_payload_v1(payload_id).await self.get_payload_v1(payload_id).await
} else { } else {
Err(Error::RequiredMethodUnsupported("engine_getPayload")) Err(Error::RequiredMethodUnsupported("engine_getPayload"))
@ -1048,11 +1105,11 @@ impl HttpJsonRpc {
forkchoice_state: ForkchoiceState, forkchoice_state: ForkchoiceState,
payload_attributes: Option<PayloadAttributes>, payload_attributes: Option<PayloadAttributes>,
) -> Result<ForkchoiceUpdatedResponse, Error> { ) -> Result<ForkchoiceUpdatedResponse, Error> {
let supported_apis = self.get_cached_supported_apis().await?; let engine_capabilities = self.get_engine_capabilities(None).await?;
if supported_apis.forkchoice_updated_v2 { if engine_capabilities.forkchoice_updated_v2 {
self.forkchoice_updated_v2(forkchoice_state, payload_attributes) self.forkchoice_updated_v2(forkchoice_state, payload_attributes)
.await .await
} else if supported_apis.forkchoice_updated_v1 { } else if engine_capabilities.forkchoice_updated_v1 {
self.forkchoice_updated_v1(forkchoice_state, payload_attributes) self.forkchoice_updated_v1(forkchoice_state, payload_attributes)
.await .await
} else { } else {
@ -1080,7 +1137,6 @@ mod test {
impl Tester { impl Tester {
pub fn new(with_auth: bool) -> Self { pub fn new(with_auth: bool) -> Self {
let server = MockServer::unit_testing(); let server = MockServer::unit_testing();
let spec = MainnetEthSpec::default_spec();
let rpc_url = SensitiveUrl::parse(&server.url()).unwrap(); let rpc_url = SensitiveUrl::parse(&server.url()).unwrap();
let echo_url = SensitiveUrl::parse(&format!("{}/echo", server.url())).unwrap(); let echo_url = SensitiveUrl::parse(&format!("{}/echo", server.url())).unwrap();
@ -1091,13 +1147,13 @@ mod test {
let echo_auth = let echo_auth =
Auth::new(JwtKey::from_slice(&DEFAULT_JWT_SECRET).unwrap(), None, None); Auth::new(JwtKey::from_slice(&DEFAULT_JWT_SECRET).unwrap(), None, None);
( (
Arc::new(HttpJsonRpc::new_with_auth(rpc_url, rpc_auth, None, &spec).unwrap()), Arc::new(HttpJsonRpc::new_with_auth(rpc_url, rpc_auth, None).unwrap()),
Arc::new(HttpJsonRpc::new_with_auth(echo_url, echo_auth, None, &spec).unwrap()), Arc::new(HttpJsonRpc::new_with_auth(echo_url, echo_auth, None).unwrap()),
) )
} else { } else {
( (
Arc::new(HttpJsonRpc::new(rpc_url, None, &spec).unwrap()), Arc::new(HttpJsonRpc::new(rpc_url, None).unwrap()),
Arc::new(HttpJsonRpc::new(echo_url, None, &spec).unwrap()), Arc::new(HttpJsonRpc::new(echo_url, None).unwrap()),
) )
}; };
@ -1685,10 +1741,11 @@ mod test {
} }
})], })],
|client| async move { |client| async move {
let payload = client let payload: ExecutionPayload<_> = client
.get_payload_v1::<MainnetEthSpec>(str_to_payload_id("0xa247243752eb10b4")) .get_payload_v1::<MainnetEthSpec>(str_to_payload_id("0xa247243752eb10b4"))
.await .await
.unwrap(); .unwrap()
.into();
let expected = ExecutionPayload::Merge(ExecutionPayloadMerge { let expected = ExecutionPayload::Merge(ExecutionPayloadMerge {
parent_hash: ExecutionBlockHash::from_str("0x3b8fb240d288781d4aac94d3fd16809ee413bc99294a085798a589dae51ddd4a").unwrap(), parent_hash: ExecutionBlockHash::from_str("0x3b8fb240d288781d4aac94d3fd16809ee413bc99294a085798a589dae51ddd4a").unwrap(),

View File

@ -1,13 +1,15 @@
//! Provides generic behaviour for multiple execution engines, specifically fallback behaviour. //! Provides generic behaviour for multiple execution engines, specifically fallback behaviour.
use crate::engine_api::{ use crate::engine_api::{
Error as EngineApiError, ForkchoiceUpdatedResponse, PayloadAttributes, PayloadId, EngineCapabilities, Error as EngineApiError, ForkchoiceUpdatedResponse, PayloadAttributes,
PayloadId,
}; };
use crate::HttpJsonRpc; use crate::HttpJsonRpc;
use lru::LruCache; use lru::LruCache;
use slog::{debug, error, info, Logger}; use slog::{debug, error, info, warn, Logger};
use std::future::Future; use std::future::Future;
use std::sync::Arc; use std::sync::Arc;
use std::time::Duration;
use task_executor::TaskExecutor; use task_executor::TaskExecutor;
use tokio::sync::{watch, Mutex, RwLock}; use tokio::sync::{watch, Mutex, RwLock};
use tokio_stream::wrappers::WatchStream; use tokio_stream::wrappers::WatchStream;
@ -18,6 +20,7 @@ use types::ExecutionBlockHash;
/// Since the size of each value is small (~100 bytes) a large number is used for safety. /// Since the size of each value is small (~100 bytes) a large number is used for safety.
/// FIXME: check this assumption now that the key includes entire payload attributes which now includes withdrawals /// FIXME: check this assumption now that the key includes entire payload attributes which now includes withdrawals
const PAYLOAD_ID_LRU_CACHE_SIZE: usize = 512; const PAYLOAD_ID_LRU_CACHE_SIZE: usize = 512;
const CACHED_ENGINE_CAPABILITIES_AGE_LIMIT: Duration = Duration::from_secs(900); // 15 minutes
/// Stores the remembered state of a engine. /// Stores the remembered state of a engine.
#[derive(Copy, Clone, PartialEq, Debug, Eq, Default)] #[derive(Copy, Clone, PartialEq, Debug, Eq, Default)]
@ -29,6 +32,14 @@ enum EngineStateInternal {
AuthFailed, AuthFailed,
} }
#[derive(Copy, Clone, Debug, Default, Eq, PartialEq)]
enum CapabilitiesCacheAction {
#[default]
None,
Update,
Clear,
}
/// A subset of the engine state to inform other services if the engine is online or offline. /// A subset of the engine state to inform other services if the engine is online or offline.
#[derive(Debug, Clone, PartialEq, Eq, Copy)] #[derive(Debug, Clone, PartialEq, Eq, Copy)]
pub enum EngineState { pub enum EngineState {
@ -231,7 +242,7 @@ impl Engine {
/// Run the `EngineApi::upcheck` function if the node's last known state is not synced. This /// Run the `EngineApi::upcheck` function if the node's last known state is not synced. This
/// might be used to recover the node if offline. /// might be used to recover the node if offline.
pub async fn upcheck(&self) { pub async fn upcheck(&self) {
let state: EngineStateInternal = match self.api.upcheck().await { let (state, cache_action) = match self.api.upcheck().await {
Ok(()) => { Ok(()) => {
let mut state = self.state.write().await; let mut state = self.state.write().await;
if **state != EngineStateInternal::Synced { if **state != EngineStateInternal::Synced {
@ -249,12 +260,12 @@ impl Engine {
); );
} }
state.update(EngineStateInternal::Synced); state.update(EngineStateInternal::Synced);
**state (**state, CapabilitiesCacheAction::Update)
} }
Err(EngineApiError::IsSyncing) => { Err(EngineApiError::IsSyncing) => {
let mut state = self.state.write().await; let mut state = self.state.write().await;
state.update(EngineStateInternal::Syncing); state.update(EngineStateInternal::Syncing);
**state (**state, CapabilitiesCacheAction::Update)
} }
Err(EngineApiError::Auth(err)) => { Err(EngineApiError::Auth(err)) => {
error!( error!(
@ -265,7 +276,7 @@ impl Engine {
let mut state = self.state.write().await; let mut state = self.state.write().await;
state.update(EngineStateInternal::AuthFailed); state.update(EngineStateInternal::AuthFailed);
**state (**state, CapabilitiesCacheAction::None)
} }
Err(e) => { Err(e) => {
error!( error!(
@ -276,10 +287,30 @@ impl Engine {
let mut state = self.state.write().await; let mut state = self.state.write().await;
state.update(EngineStateInternal::Offline); state.update(EngineStateInternal::Offline);
**state // need to clear the engine capabilities cache if we detect the
// execution engine is offline as it is likely the engine is being
// updated to a newer version with new capabilities
(**state, CapabilitiesCacheAction::Clear)
} }
}; };
// do this after dropping state lock guard to avoid holding two locks at once
match cache_action {
CapabilitiesCacheAction::None => {}
CapabilitiesCacheAction::Update => {
if let Err(e) = self
.get_engine_capabilities(Some(CACHED_ENGINE_CAPABILITIES_AGE_LIMIT))
.await
{
warn!(self.log,
"Error during exchange capabilities";
"error" => ?e,
)
}
}
CapabilitiesCacheAction::Clear => self.api.clear_exchange_capabilties_cache().await,
}
debug!( debug!(
self.log, self.log,
"Execution engine upcheck complete"; "Execution engine upcheck complete";
@ -287,6 +318,22 @@ impl Engine {
); );
} }
/// Returns the execution engine capabilities resulting from a call to
/// engine_exchangeCapabilities. If the capabilities cache is not populated,
/// or if it is populated with a cached result of age >= `age_limit`, this
/// method will fetch the result from the execution engine and populate the
/// cache before returning it. Otherwise it will return a cached result from
/// a previous call.
///
/// Set `age_limit` to `None` to always return the cached result
/// Set `age_limit` to `Some(Duration::ZERO)` to force fetching from EE
pub async fn get_engine_capabilities(
&self,
age_limit: Option<Duration>,
) -> Result<EngineCapabilities, EngineApiError> {
self.api.get_engine_capabilities(age_limit).await
}
/// Run `func` on the node regardless of the node's current state. /// Run `func` on the node regardless of the node's current state.
/// ///
/// ## Note /// ## Note

View File

@ -7,6 +7,7 @@
use crate::payload_cache::PayloadCache; use crate::payload_cache::PayloadCache;
use auth::{strip_prefix, Auth, JwtKey}; use auth::{strip_prefix, Auth, JwtKey};
use builder_client::BuilderHttpClient; use builder_client::BuilderHttpClient;
pub use engine_api::EngineCapabilities;
use engine_api::Error as ApiError; use engine_api::Error as ApiError;
pub use engine_api::*; pub use engine_api::*;
pub use engine_api::{http, http::deposit_methods, http::HttpJsonRpc}; pub use engine_api::{http, http::deposit_methods, http::HttpJsonRpc};
@ -123,9 +124,13 @@ impl From<ApiError> for Error {
} }
pub enum BlockProposalContents<T: EthSpec, Payload: AbstractExecPayload<T>> { pub enum BlockProposalContents<T: EthSpec, Payload: AbstractExecPayload<T>> {
Payload(Payload), Payload {
payload: Payload,
block_value: Uint256,
},
PayloadAndBlobs { PayloadAndBlobs {
payload: Payload, payload: Payload,
block_value: Uint256,
kzg_commitments: VariableList<KzgCommitment, T::MaxBlobsPerBlock>, kzg_commitments: VariableList<KzgCommitment, T::MaxBlobsPerBlock>,
blobs: VariableList<Blob<T>, T::MaxBlobsPerBlock>, blobs: VariableList<Blob<T>, T::MaxBlobsPerBlock>,
}, },
@ -151,9 +156,13 @@ impl<T: EthSpec, Payload: AbstractExecPayload<T>> BlockProposalContents<T, Paylo
pub fn payload(&self) -> &Payload { pub fn payload(&self) -> &Payload {
match self { match self {
Self::Payload(payload) => payload, Self::Payload {
payload,
block_value: _,
} => payload,
Self::PayloadAndBlobs { Self::PayloadAndBlobs {
payload, payload,
block_value: _,
kzg_commitments: _, kzg_commitments: _,
blobs: _, blobs: _,
} => payload, } => payload,
@ -161,21 +170,43 @@ impl<T: EthSpec, Payload: AbstractExecPayload<T>> BlockProposalContents<T, Paylo
} }
pub fn to_payload(self) -> Payload { pub fn to_payload(self) -> Payload {
match self { match self {
Self::Payload(payload) => payload, Self::Payload {
payload,
block_value: _,
} => payload,
Self::PayloadAndBlobs { Self::PayloadAndBlobs {
payload, payload,
block_value: _,
kzg_commitments: _, kzg_commitments: _,
blobs: _, blobs: _,
} => payload, } => payload,
} }
} }
pub fn block_value(&self) -> &Uint256 {
match self {
Self::Payload {
payload: _,
block_value,
} => block_value,
Self::PayloadAndBlobs {
payload: _,
block_value,
kzg_commitments: _,
blobs: _,
} => block_value,
}
}
pub fn default_at_fork(fork_name: ForkName) -> Result<Self, BeaconStateError> { pub fn default_at_fork(fork_name: ForkName) -> Result<Self, BeaconStateError> {
Ok(match fork_name { Ok(match fork_name {
ForkName::Base | ForkName::Altair | ForkName::Merge | ForkName::Capella => { ForkName::Base | ForkName::Altair | ForkName::Merge | ForkName::Capella => {
BlockProposalContents::Payload(Payload::default_at_fork(fork_name)?) BlockProposalContents::Payload {
payload: Payload::default_at_fork(fork_name)?,
block_value: Uint256::zero(),
}
} }
ForkName::Eip4844 => BlockProposalContents::PayloadAndBlobs { ForkName::Eip4844 => BlockProposalContents::PayloadAndBlobs {
payload: Payload::default_at_fork(fork_name)?, payload: Payload::default_at_fork(fork_name)?,
block_value: Uint256::zero(),
blobs: VariableList::default(), blobs: VariableList::default(),
kzg_commitments: VariableList::default(), kzg_commitments: VariableList::default(),
}, },
@ -267,12 +298,7 @@ pub struct ExecutionLayer<T: EthSpec> {
impl<T: EthSpec> ExecutionLayer<T> { impl<T: EthSpec> ExecutionLayer<T> {
/// Instantiate `Self` with an Execution engine specified in `Config`, using JSON-RPC via HTTP. /// Instantiate `Self` with an Execution engine specified in `Config`, using JSON-RPC via HTTP.
pub fn from_config( pub fn from_config(config: Config, executor: TaskExecutor, log: Logger) -> Result<Self, Error> {
config: Config,
executor: TaskExecutor,
log: Logger,
spec: &ChainSpec,
) -> Result<Self, Error> {
let Config { let Config {
execution_endpoints: urls, execution_endpoints: urls,
builder_url, builder_url,
@ -327,13 +353,8 @@ impl<T: EthSpec> ExecutionLayer<T> {
let engine: Engine = { let engine: Engine = {
let auth = Auth::new(jwt_key, jwt_id, jwt_version); let auth = Auth::new(jwt_key, jwt_id, jwt_version);
debug!(log, "Loaded execution endpoint"; "endpoint" => %execution_url, "jwt_path" => ?secret_file.as_path()); debug!(log, "Loaded execution endpoint"; "endpoint" => %execution_url, "jwt_path" => ?secret_file.as_path());
let api = HttpJsonRpc::new_with_auth( let api = HttpJsonRpc::new_with_auth(execution_url, auth, execution_timeout_multiplier)
execution_url, .map_err(Error::ApiError)?;
auth,
execution_timeout_multiplier,
&spec,
)
.map_err(Error::ApiError)?;
Engine::new(api, executor.clone(), &log) Engine::new(api, executor.clone(), &log)
}; };
@ -377,12 +398,12 @@ impl<T: EthSpec> ExecutionLayer<T> {
&self.inner.builder &self.inner.builder
} }
/// Cache a full payload, keyed on the `tree_hash_root` of its `transactions` field. /// Cache a full payload, keyed on the `tree_hash_root` of the payload
fn cache_payload(&self, payload: &ExecutionPayload<T>) -> Option<ExecutionPayload<T>> { fn cache_payload(&self, payload: ExecutionPayloadRef<T>) -> Option<ExecutionPayload<T>> {
self.inner.payload_cache.put(payload.clone()) self.inner.payload_cache.put(payload.clone_from_ref())
} }
/// Attempt to retrieve a full payload from the payload cache by the `transactions_root`. /// Attempt to retrieve a full payload from the payload cache by the payload root
pub fn get_payload_by_root(&self, root: &Hash256) -> Option<ExecutionPayload<T>> { pub fn get_payload_by_root(&self, root: &Hash256) -> Option<ExecutionPayload<T>> {
self.inner.payload_cache.pop(root) self.inner.payload_cache.pop(root)
} }
@ -819,17 +840,32 @@ impl<T: EthSpec> ExecutionLayer<T> {
"parent_hash" => ?parent_hash, "parent_hash" => ?parent_hash,
); );
let relay_value = relay.data.message.value;
let local_value = *local.block_value();
if local_value >= relay_value {
info!(
self.log(),
"Local block is more profitable than relay block";
"local_block_value" => %local_value,
"relay_value" => %relay_value
);
return Ok(ProvenancedPayload::Local(local));
}
match verify_builder_bid( match verify_builder_bid(
&relay, &relay,
parent_hash, parent_hash,
payload_attributes.prev_randao(), payload_attributes,
payload_attributes.timestamp(),
Some(local.payload().block_number()), Some(local.payload().block_number()),
self.inner.builder_profit_threshold, self.inner.builder_profit_threshold,
current_fork,
spec, spec,
) { ) {
Ok(()) => Ok(ProvenancedPayload::Builder( Ok(()) => Ok(ProvenancedPayload::Builder(
BlockProposalContents::Payload(relay.data.message.header), BlockProposalContents::Payload {
payload: relay.data.message.header,
block_value: relay.data.message.value,
},
)), )),
Err(reason) if !reason.payload_invalid() => { Err(reason) if !reason.payload_invalid() => {
info!( info!(
@ -873,19 +909,25 @@ impl<T: EthSpec> ExecutionLayer<T> {
match verify_builder_bid( match verify_builder_bid(
&relay, &relay,
parent_hash, parent_hash,
payload_attributes.prev_randao(), payload_attributes,
payload_attributes.timestamp(),
None, None,
self.inner.builder_profit_threshold, self.inner.builder_profit_threshold,
current_fork,
spec, spec,
) { ) {
Ok(()) => Ok(ProvenancedPayload::Builder( Ok(()) => Ok(ProvenancedPayload::Builder(
BlockProposalContents::Payload(relay.data.message.header), BlockProposalContents::Payload {
payload: relay.data.message.header,
block_value: relay.data.message.value,
},
)), )),
// If the payload is valid then use it. The local EE failed // If the payload is valid then use it. The local EE failed
// to produce a payload so we have no alternative. // to produce a payload so we have no alternative.
Err(e) if !e.payload_invalid() => Ok(ProvenancedPayload::Builder( Err(e) if !e.payload_invalid() => Ok(ProvenancedPayload::Builder(
BlockProposalContents::Payload(relay.data.message.header), BlockProposalContents::Payload {
payload: relay.data.message.header,
block_value: relay.data.message.value,
},
)), )),
Err(reason) => { Err(reason) => {
metrics::inc_counter_vec( metrics::inc_counter_vec(
@ -999,7 +1041,7 @@ impl<T: EthSpec> ExecutionLayer<T> {
payload_attributes: &PayloadAttributes, payload_attributes: &PayloadAttributes,
forkchoice_update_params: ForkchoiceUpdateParameters, forkchoice_update_params: ForkchoiceUpdateParameters,
current_fork: ForkName, current_fork: ForkName,
f: fn(&ExecutionLayer<T>, &ExecutionPayload<T>) -> Option<ExecutionPayload<T>>, f: fn(&ExecutionLayer<T>, ExecutionPayloadRef<T>) -> Option<ExecutionPayload<T>>,
) -> Result<BlockProposalContents<T, Payload>, Error> { ) -> Result<BlockProposalContents<T, Payload>, Error> {
self.engine() self.engine()
.request(move |engine| async move { .request(move |engine| async move {
@ -1082,9 +1124,9 @@ impl<T: EthSpec> ExecutionLayer<T> {
); );
engine.api.get_payload::<T>(current_fork, payload_id).await engine.api.get_payload::<T>(current_fork, payload_id).await
}; };
let (blob, payload) = tokio::join!(blob_fut, payload_fut); let (blob, payload_response) = tokio::join!(blob_fut, payload_fut);
let payload = payload.map(|full_payload| { let (execution_payload, block_value) = payload_response.map(|payload_response| {
if full_payload.fee_recipient() != payload_attributes.suggested_fee_recipient() { if payload_response.execution_payload_ref().fee_recipient() != payload_attributes.suggested_fee_recipient() {
error!( error!(
self.log(), self.log(),
"Inconsistent fee recipient"; "Inconsistent fee recipient";
@ -1093,28 +1135,32 @@ impl<T: EthSpec> ExecutionLayer<T> {
indicate that fees are being diverted to another address. Please \ indicate that fees are being diverted to another address. Please \
ensure that the value of suggested_fee_recipient is set correctly and \ ensure that the value of suggested_fee_recipient is set correctly and \
that the Execution Engine is trusted.", that the Execution Engine is trusted.",
"fee_recipient" => ?full_payload.fee_recipient(), "fee_recipient" => ?payload_response.execution_payload_ref().fee_recipient(),
"suggested_fee_recipient" => ?payload_attributes.suggested_fee_recipient(), "suggested_fee_recipient" => ?payload_attributes.suggested_fee_recipient(),
); );
} }
if f(self, &full_payload).is_some() { if f(self, payload_response.execution_payload_ref()).is_some() {
warn!( warn!(
self.log(), self.log(),
"Duplicate payload cached, this might indicate redundant proposal \ "Duplicate payload cached, this might indicate redundant proposal \
attempts." attempts."
); );
} }
full_payload.into() payload_response.into()
})?; })?;
if let Some(blob) = blob.transpose()? { if let Some(blob) = blob.transpose()? {
// FIXME(sean) cache blobs // FIXME(sean) cache blobs
Ok(BlockProposalContents::PayloadAndBlobs { Ok(BlockProposalContents::PayloadAndBlobs {
payload, payload: execution_payload.into(),
block_value,
blobs: blob.blobs, blobs: blob.blobs,
kzg_commitments: blob.kzgs, kzg_commitments: blob.kzgs,
}) })
} else { } else {
Ok(BlockProposalContents::Payload(payload)) Ok(BlockProposalContents::Payload {
payload: execution_payload.into(),
block_value,
})
} }
}) })
.await .await
@ -1373,6 +1419,26 @@ impl<T: EthSpec> ExecutionLayer<T> {
} }
} }
/// Returns the execution engine capabilities resulting from a call to
/// engine_exchangeCapabilities. If the capabilities cache is not populated,
/// or if it is populated with a cached result of age >= `age_limit`, this
/// method will fetch the result from the execution engine and populate the
/// cache before returning it. Otherwise it will return a cached result from
/// a previous call.
///
/// Set `age_limit` to `None` to always return the cached result
/// Set `age_limit` to `Some(Duration::ZERO)` to force fetching from EE
pub async fn get_engine_capabilities(
&self,
age_limit: Option<Duration>,
) -> Result<EngineCapabilities, Error> {
self.engine()
.request(|engine| engine.get_engine_capabilities(age_limit))
.await
.map_err(Box::new)
.map_err(Error::EngineError)
}
/// Used during block production to determine if the merge has been triggered. /// Used during block production to determine if the merge has been triggered.
/// ///
/// ## Specification /// ## Specification
@ -1801,6 +1867,11 @@ enum InvalidBuilderPayload {
signature: Signature, signature: Signature,
pubkey: PublicKeyBytes, pubkey: PublicKeyBytes,
}, },
#[allow(dead_code)]
WithdrawalsRoot {
payload: Hash256,
expected: Hash256,
},
} }
impl InvalidBuilderPayload { impl InvalidBuilderPayload {
@ -1815,6 +1886,7 @@ impl InvalidBuilderPayload {
InvalidBuilderPayload::BlockNumber { .. } => true, InvalidBuilderPayload::BlockNumber { .. } => true,
InvalidBuilderPayload::Fork { .. } => true, InvalidBuilderPayload::Fork { .. } => true,
InvalidBuilderPayload::Signature { .. } => true, InvalidBuilderPayload::Signature { .. } => true,
InvalidBuilderPayload::WithdrawalsRoot { .. } => true,
} }
} }
} }
@ -1850,6 +1922,13 @@ impl fmt::Display for InvalidBuilderPayload {
"invalid payload signature {} for pubkey {}", "invalid payload signature {} for pubkey {}",
signature, pubkey signature, pubkey
), ),
InvalidBuilderPayload::WithdrawalsRoot { payload, expected } => {
write!(
f,
"payload withdrawals root was {} not {}",
payload, expected
)
}
} }
} }
} }
@ -1858,10 +1937,10 @@ impl fmt::Display for InvalidBuilderPayload {
fn verify_builder_bid<T: EthSpec, Payload: AbstractExecPayload<T>>( fn verify_builder_bid<T: EthSpec, Payload: AbstractExecPayload<T>>(
bid: &ForkVersionedResponse<SignedBuilderBid<T, Payload>>, bid: &ForkVersionedResponse<SignedBuilderBid<T, Payload>>,
parent_hash: ExecutionBlockHash, parent_hash: ExecutionBlockHash,
prev_randao: Hash256, payload_attributes: &PayloadAttributes,
timestamp: u64,
block_number: Option<u64>, block_number: Option<u64>,
profit_threshold: Uint256, profit_threshold: Uint256,
current_fork: ForkName,
spec: &ChainSpec, spec: &ChainSpec,
) -> Result<(), Box<InvalidBuilderPayload>> { ) -> Result<(), Box<InvalidBuilderPayload>> {
let is_signature_valid = bid.data.verify_signature(spec); let is_signature_valid = bid.data.verify_signature(spec);
@ -1888,29 +1967,25 @@ fn verify_builder_bid<T: EthSpec, Payload: AbstractExecPayload<T>>(
payload: header.parent_hash(), payload: header.parent_hash(),
expected: parent_hash, expected: parent_hash,
})) }))
} else if header.prev_randao() != prev_randao { } else if header.prev_randao() != payload_attributes.prev_randao() {
Err(Box::new(InvalidBuilderPayload::PrevRandao { Err(Box::new(InvalidBuilderPayload::PrevRandao {
payload: header.prev_randao(), payload: header.prev_randao(),
expected: prev_randao, expected: payload_attributes.prev_randao(),
})) }))
} else if header.timestamp() != timestamp { } else if header.timestamp() != payload_attributes.timestamp() {
Err(Box::new(InvalidBuilderPayload::Timestamp { Err(Box::new(InvalidBuilderPayload::Timestamp {
payload: header.timestamp(), payload: header.timestamp(),
expected: timestamp, expected: payload_attributes.timestamp(),
})) }))
} else if block_number.map_or(false, |n| n != header.block_number()) { } else if block_number.map_or(false, |n| n != header.block_number()) {
Err(Box::new(InvalidBuilderPayload::BlockNumber { Err(Box::new(InvalidBuilderPayload::BlockNumber {
payload: header.block_number(), payload: header.block_number(),
expected: block_number, expected: block_number,
})) }))
} else if !matches!(bid.version, Some(ForkName::Merge)) { } else if bid.version != Some(current_fork) {
// Once fork information is added to the payload, we will need to
// check that the local and relay payloads match. At this point, if
// we are requesting a payload at all, we have to assume this is
// the Bellatrix fork.
Err(Box::new(InvalidBuilderPayload::Fork { Err(Box::new(InvalidBuilderPayload::Fork {
payload: bid.version, payload: bid.version,
expected: ForkName::Merge, expected: current_fork,
})) }))
} else if !is_signature_valid { } else if !is_signature_valid {
Err(Box::new(InvalidBuilderPayload::Signature { Err(Box::new(InvalidBuilderPayload::Signature {

View File

@ -1,25 +1,33 @@
use super::Context; use super::Context;
use crate::engine_api::{http::*, *}; use crate::engine_api::{http::*, *};
use crate::json_structures::*; use crate::json_structures::*;
use crate::test_utils::DEFAULT_MOCK_EL_PAYLOAD_VALUE_WEI;
use serde::de::DeserializeOwned; use serde::de::DeserializeOwned;
use serde_json::Value as JsonValue; use serde_json::Value as JsonValue;
use std::sync::Arc; use std::sync::Arc;
use types::{EthSpec, ForkName}; use types::{EthSpec, ForkName};
pub const GENERIC_ERROR_CODE: i64 = -1234;
pub const BAD_PARAMS_ERROR_CODE: i64 = -32602;
pub const UNKNOWN_PAYLOAD_ERROR_CODE: i64 = -38001;
pub const FORK_REQUEST_MISMATCH_ERROR_CODE: i64 = -32000;
pub async fn handle_rpc<T: EthSpec>( pub async fn handle_rpc<T: EthSpec>(
body: JsonValue, body: JsonValue,
ctx: Arc<Context<T>>, ctx: Arc<Context<T>>,
) -> Result<JsonValue, String> { ) -> Result<JsonValue, (String, i64)> {
*ctx.previous_request.lock() = Some(body.clone()); *ctx.previous_request.lock() = Some(body.clone());
let method = body let method = body
.get("method") .get("method")
.and_then(JsonValue::as_str) .and_then(JsonValue::as_str)
.ok_or_else(|| "missing/invalid method field".to_string())?; .ok_or_else(|| "missing/invalid method field".to_string())
.map_err(|s| (s, GENERIC_ERROR_CODE))?;
let params = body let params = body
.get("params") .get("params")
.ok_or_else(|| "missing/invalid params field".to_string())?; .ok_or_else(|| "missing/invalid params field".to_string())
.map_err(|s| (s, GENERIC_ERROR_CODE))?;
match method { match method {
ETH_SYNCING => Ok(JsonValue::Bool(false)), ETH_SYNCING => Ok(JsonValue::Bool(false)),
@ -27,7 +35,8 @@ pub async fn handle_rpc<T: EthSpec>(
let tag = params let tag = params
.get(0) .get(0)
.and_then(JsonValue::as_str) .and_then(JsonValue::as_str)
.ok_or_else(|| "missing/invalid params[0] value".to_string())?; .ok_or_else(|| "missing/invalid params[0] value".to_string())
.map_err(|s| (s, BAD_PARAMS_ERROR_CODE))?;
match tag { match tag {
"latest" => Ok(serde_json::to_value( "latest" => Ok(serde_json::to_value(
@ -36,7 +45,10 @@ pub async fn handle_rpc<T: EthSpec>(
.latest_execution_block(), .latest_execution_block(),
) )
.unwrap()), .unwrap()),
other => Err(format!("The tag {} is not supported", other)), other => Err((
format!("The tag {} is not supported", other),
BAD_PARAMS_ERROR_CODE,
)),
} }
} }
ETH_GET_BLOCK_BY_HASH => { ETH_GET_BLOCK_BY_HASH => {
@ -47,7 +59,8 @@ pub async fn handle_rpc<T: EthSpec>(
.and_then(|s| { .and_then(|s| {
s.parse() s.parse()
.map_err(|e| format!("unable to parse hash: {:?}", e)) .map_err(|e| format!("unable to parse hash: {:?}", e))
})?; })
.map_err(|s| (s, BAD_PARAMS_ERROR_CODE))?;
// If we have a static response set, just return that. // If we have a static response set, just return that.
if let Some(response) = *ctx.static_get_block_by_hash_response.lock() { if let Some(response) = *ctx.static_get_block_by_hash_response.lock() {
@ -57,7 +70,8 @@ pub async fn handle_rpc<T: EthSpec>(
let full_tx = params let full_tx = params
.get(1) .get(1)
.and_then(JsonValue::as_bool) .and_then(JsonValue::as_bool)
.ok_or_else(|| "missing/invalid params[1] value".to_string())?; .ok_or_else(|| "missing/invalid params[1] value".to_string())
.map_err(|s| (s, BAD_PARAMS_ERROR_CODE))?;
if full_tx { if full_tx {
Ok(serde_json::to_value( Ok(serde_json::to_value(
ctx.execution_block_generator ctx.execution_block_generator
@ -76,15 +90,17 @@ pub async fn handle_rpc<T: EthSpec>(
} }
ENGINE_NEW_PAYLOAD_V1 | ENGINE_NEW_PAYLOAD_V2 | ENGINE_NEW_PAYLOAD_V3 => { ENGINE_NEW_PAYLOAD_V1 | ENGINE_NEW_PAYLOAD_V2 | ENGINE_NEW_PAYLOAD_V3 => {
let request = match method { let request = match method {
ENGINE_NEW_PAYLOAD_V1 => { ENGINE_NEW_PAYLOAD_V1 => JsonExecutionPayload::V1(
JsonExecutionPayload::V1(get_param::<JsonExecutionPayloadV1<T>>(params, 0)?) get_param::<JsonExecutionPayloadV1<T>>(params, 0)
} .map_err(|s| (s, BAD_PARAMS_ERROR_CODE))?,
),
ENGINE_NEW_PAYLOAD_V2 => get_param::<JsonExecutionPayloadV2<T>>(params, 0) ENGINE_NEW_PAYLOAD_V2 => get_param::<JsonExecutionPayloadV2<T>>(params, 0)
.map(|jep| JsonExecutionPayload::V2(jep)) .map(|jep| JsonExecutionPayload::V2(jep))
.or_else(|_| { .or_else(|_| {
get_param::<JsonExecutionPayloadV1<T>>(params, 0) get_param::<JsonExecutionPayloadV1<T>>(params, 0)
.map(|jep| JsonExecutionPayload::V1(jep)) .map(|jep| JsonExecutionPayload::V1(jep))
})?, })
.map_err(|s| (s, BAD_PARAMS_ERROR_CODE))?,
ENGINE_NEW_PAYLOAD_V3 => get_param::<JsonExecutionPayloadV3<T>>(params, 0) ENGINE_NEW_PAYLOAD_V3 => get_param::<JsonExecutionPayloadV3<T>>(params, 0)
.map(|jep| JsonExecutionPayload::V3(jep)) .map(|jep| JsonExecutionPayload::V3(jep))
.or_else(|_| { .or_else(|_| {
@ -106,37 +122,46 @@ pub async fn handle_rpc<T: EthSpec>(
match fork { match fork {
ForkName::Merge => { ForkName::Merge => {
if matches!(request, JsonExecutionPayload::V2(_)) { if matches!(request, JsonExecutionPayload::V2(_)) {
return Err(format!( return Err((
"{} called with `ExecutionPayloadV2` before capella fork!", format!(
method "{} called with `ExecutionPayloadV2` before Capella fork!",
method
),
GENERIC_ERROR_CODE,
)); ));
} }
} }
ForkName::Capella => { ForkName::Capella => {
if method == ENGINE_NEW_PAYLOAD_V1 { if method == ENGINE_NEW_PAYLOAD_V1 {
return Err(format!("{} called after capella fork!", method)); return Err((
format!("{} called after Capella fork!", method),
GENERIC_ERROR_CODE,
));
} }
if matches!(request, JsonExecutionPayload::V1(_)) { if matches!(request, JsonExecutionPayload::V1(_)) {
return Err(format!( return Err((
"{} called with `ExecutionPayloadV1` after capella fork!", format!(
method "{} called with `ExecutionPayloadV1` after Capella fork!",
method
),
GENERIC_ERROR_CODE,
)); ));
} }
} }
ForkName::Eip4844 => { ForkName::Eip4844 => {
if method == ENGINE_NEW_PAYLOAD_V1 || method == ENGINE_NEW_PAYLOAD_V2 { if method == ENGINE_NEW_PAYLOAD_V1 || method == ENGINE_NEW_PAYLOAD_V2 {
return Err(format!("{} called after capella fork!", method)); return Err((format!("{} called after capella fork!", method), GENERIC_ERROR_CODE));
} }
if matches!(request, JsonExecutionPayload::V1(_)) { if matches!(request, JsonExecutionPayload::V1(_)) {
return Err(format!( return Err((format!(
"{} called with `ExecutionPayloadV1` after eip4844 fork!", "{} called with `ExecutionPayloadV1` after eip4844 fork!",
method method
)); ), GENERIC_ERROR_CODE));
} }
if matches!(request, JsonExecutionPayload::V2(_)) { if matches!(request, JsonExecutionPayload::V2(_)) {
return Err(format!( return Err((format!(
"{} called with `ExecutionPayloadV2` after eip4844 fork!", "{} called with `ExecutionPayloadV2` after eip4844 fork!",
method method), GENERIC_ERROR_CODE
)); ));
} }
} }
@ -174,14 +199,20 @@ pub async fn handle_rpc<T: EthSpec>(
Ok(serde_json::to_value(JsonPayloadStatusV1::from(response)).unwrap()) Ok(serde_json::to_value(JsonPayloadStatusV1::from(response)).unwrap())
} }
ENGINE_GET_PAYLOAD_V1 | ENGINE_GET_PAYLOAD_V2 | ENGINE_GET_PAYLOAD_V3 => { ENGINE_GET_PAYLOAD_V1 | ENGINE_GET_PAYLOAD_V2 | ENGINE_GET_PAYLOAD_V3 => {
let request: JsonPayloadIdRequest = get_param(params, 0)?; let request: JsonPayloadIdRequest =
get_param(params, 0).map_err(|s| (s, BAD_PARAMS_ERROR_CODE))?;
let id = request.into(); let id = request.into();
let response = ctx let response = ctx
.execution_block_generator .execution_block_generator
.write() .write()
.get_payload(&id) .get_payload(&id)
.ok_or_else(|| format!("no payload for id {:?}", id))?; .ok_or_else(|| {
(
format!("no payload for id {:?}", id),
UNKNOWN_PAYLOAD_ERROR_CODE,
)
})?;
// validate method called correctly according to shanghai fork time // validate method called correctly according to shanghai fork time
if ctx if ctx
@ -191,7 +222,10 @@ pub async fn handle_rpc<T: EthSpec>(
== ForkName::Capella == ForkName::Capella
&& method == ENGINE_GET_PAYLOAD_V1 && method == ENGINE_GET_PAYLOAD_V1
{ {
return Err(format!("{} called after capella fork!", method)); return Err((
format!("{} called after Capella fork!", method),
FORK_REQUEST_MISMATCH_ERROR_CODE,
));
} }
// validate method called correctly according to eip4844 fork time // validate method called correctly according to eip4844 fork time
if ctx if ctx
@ -201,7 +235,7 @@ pub async fn handle_rpc<T: EthSpec>(
== ForkName::Eip4844 == ForkName::Eip4844
&& (method == ENGINE_GET_PAYLOAD_V1 || method == ENGINE_GET_PAYLOAD_V2) && (method == ENGINE_GET_PAYLOAD_V1 || method == ENGINE_GET_PAYLOAD_V2)
{ {
return Err(format!("{} called after eip4844 fork!", method)); return Err((format!("{} called after eip4844 fork!", method), FORK_REQUEST_MISMATCH_ERROR_CODE));
} }
match method { match method {
@ -212,14 +246,14 @@ pub async fn handle_rpc<T: EthSpec>(
JsonExecutionPayload::V1(execution_payload) => { JsonExecutionPayload::V1(execution_payload) => {
serde_json::to_value(JsonGetPayloadResponseV1 { serde_json::to_value(JsonGetPayloadResponseV1 {
execution_payload, execution_payload,
block_value: 0.into(), block_value: DEFAULT_MOCK_EL_PAYLOAD_VALUE_WEI.into(),
}) })
.unwrap() .unwrap()
} }
JsonExecutionPayload::V2(execution_payload) => { JsonExecutionPayload::V2(execution_payload) => {
serde_json::to_value(JsonGetPayloadResponseV2 { serde_json::to_value(JsonGetPayloadResponseV2 {
execution_payload, execution_payload,
block_value: 0.into(), block_value: DEFAULT_MOCK_EL_PAYLOAD_VALUE_WEI.into(),
}) })
.unwrap() .unwrap()
} }
@ -229,61 +263,65 @@ pub async fn handle_rpc<T: EthSpec>(
JsonExecutionPayload::V1(execution_payload) => { JsonExecutionPayload::V1(execution_payload) => {
serde_json::to_value(JsonGetPayloadResponseV1 { serde_json::to_value(JsonGetPayloadResponseV1 {
execution_payload, execution_payload,
block_value: 0.into(), block_value: DEFAULT_MOCK_EL_PAYLOAD_VALUE_WEI.into()
}) })
.unwrap() .unwrap()
} }
JsonExecutionPayload::V2(execution_payload) => { JsonExecutionPayload::V2(execution_payload) => {
serde_json::to_value(JsonGetPayloadResponseV2 { serde_json::to_value(JsonGetPayloadResponseV2 {
execution_payload, execution_payload,
block_value: 0.into(), block_value: DEFAULT_MOCK_EL_PAYLOAD_VALUE_WEI.into()
}) })
.unwrap() .unwrap()
} }
JsonExecutionPayload::V3(execution_payload) => { JsonExecutionPayload::V3(execution_payload) => {
serde_json::to_value(JsonGetPayloadResponseV3 { serde_json::to_value(JsonGetPayloadResponseV3 {
execution_payload, execution_payload,
block_value: 0.into(), block_value: DEFAULT_MOCK_EL_PAYLOAD_VALUE_WEI.into()
}) })
.unwrap() .unwrap()
} }
}), }),
_ => unreachable!(), _ => unreachable!(),
} }
} }
ENGINE_FORKCHOICE_UPDATED_V1 | ENGINE_FORKCHOICE_UPDATED_V2 => { ENGINE_FORKCHOICE_UPDATED_V1 | ENGINE_FORKCHOICE_UPDATED_V2 => {
let forkchoice_state: JsonForkchoiceStateV1 = get_param(params, 0)?; let forkchoice_state: JsonForkchoiceStateV1 =
get_param(params, 0).map_err(|s| (s, BAD_PARAMS_ERROR_CODE))?;
let payload_attributes = match method { let payload_attributes = match method {
ENGINE_FORKCHOICE_UPDATED_V1 => { ENGINE_FORKCHOICE_UPDATED_V1 => {
let jpa1: Option<JsonPayloadAttributesV1> = get_param(params, 1)?; let jpa1: Option<JsonPayloadAttributesV1> =
get_param(params, 1).map_err(|s| (s, BAD_PARAMS_ERROR_CODE))?;
jpa1.map(JsonPayloadAttributes::V1) jpa1.map(JsonPayloadAttributes::V1)
} }
ENGINE_FORKCHOICE_UPDATED_V2 => { ENGINE_FORKCHOICE_UPDATED_V2 => {
// we can't use `deny_unknown_fields` without breaking compatibility with some // we can't use `deny_unknown_fields` without breaking compatibility with some
// clients that haven't updated to the latest engine_api spec. So instead we'll // clients that haven't updated to the latest engine_api spec. So instead we'll
// need to deserialize based on timestamp // need to deserialize based on timestamp
get_param::<Option<JsonPayloadAttributes>>(params, 1).and_then(|pa| { get_param::<Option<JsonPayloadAttributes>>(params, 1)
pa.and_then(|pa| { .and_then(|pa| {
match ctx pa.and_then(|pa| {
.execution_block_generator match ctx
.read() .execution_block_generator
.get_fork_at_timestamp(*pa.timestamp()) .read()
{ .get_fork_at_timestamp(*pa.timestamp())
ForkName::Merge => { {
get_param::<Option<JsonPayloadAttributesV1>>(params, 1) ForkName::Merge => {
.map(|opt| opt.map(JsonPayloadAttributes::V1)) get_param::<Option<JsonPayloadAttributesV1>>(params, 1)
.transpose() .map(|opt| opt.map(JsonPayloadAttributes::V1))
.transpose()
}
ForkName::Capella => {
get_param::<Option<JsonPayloadAttributesV2>>(params, 1)
.map(|opt| opt.map(JsonPayloadAttributes::V2))
.transpose()
}
_ => unreachable!(),
} }
ForkName::Capella | ForkName::Eip4844 => { })
get_param::<Option<JsonPayloadAttributesV2>>(params, 1) .transpose()
.map(|opt| opt.map(JsonPayloadAttributes::V2))
.transpose()
}
_ => unreachable!(),
}
}) })
.transpose() .map_err(|s| (s, BAD_PARAMS_ERROR_CODE))?
})?
} }
_ => unreachable!(), _ => unreachable!(),
}; };
@ -297,20 +335,29 @@ pub async fn handle_rpc<T: EthSpec>(
{ {
ForkName::Merge => { ForkName::Merge => {
if matches!(pa, JsonPayloadAttributes::V2(_)) { if matches!(pa, JsonPayloadAttributes::V2(_)) {
return Err(format!( return Err((
"{} called with `JsonPayloadAttributesV2` before capella fork!", format!(
method "{} called with `JsonPayloadAttributesV2` before Capella fork!",
method
),
GENERIC_ERROR_CODE,
)); ));
} }
} }
ForkName::Capella | ForkName::Eip4844 => { ForkName::Capella | ForkName::Eip4844 => {
if method == ENGINE_FORKCHOICE_UPDATED_V1 { if method == ENGINE_FORKCHOICE_UPDATED_V1 {
return Err(format!("{} called after capella fork!", method)); return Err((
format!("{} called after Capella fork!", method),
FORK_REQUEST_MISMATCH_ERROR_CODE,
));
} }
if matches!(pa, JsonPayloadAttributes::V1(_)) { if matches!(pa, JsonPayloadAttributes::V1(_)) {
return Err(format!( return Err((
"{} called with `JsonPayloadAttributesV1` after capella fork!", format!(
method "{} called with `JsonPayloadAttributesV1` after Capella fork!",
method
),
FORK_REQUEST_MISMATCH_ERROR_CODE,
)); ));
} }
} }
@ -337,10 +384,14 @@ pub async fn handle_rpc<T: EthSpec>(
return Ok(serde_json::to_value(response).unwrap()); return Ok(serde_json::to_value(response).unwrap());
} }
let mut response = ctx.execution_block_generator.write().forkchoice_updated( let mut response = ctx
forkchoice_state.into(), .execution_block_generator
payload_attributes.map(|json| json.into()), .write()
)?; .forkchoice_updated(
forkchoice_state.into(),
payload_attributes.map(|json| json.into()),
)
.map_err(|s| (s, GENERIC_ERROR_CODE))?;
if let Some(mut status) = ctx.static_forkchoice_updated_response.lock().clone() { if let Some(mut status) = ctx.static_forkchoice_updated_response.lock().clone() {
if status.status == PayloadStatusV1Status::Valid { if status.status == PayloadStatusV1Status::Valid {
@ -361,9 +412,13 @@ pub async fn handle_rpc<T: EthSpec>(
}; };
Ok(serde_json::to_value(transition_config).unwrap()) Ok(serde_json::to_value(transition_config).unwrap())
} }
other => Err(format!( ENGINE_EXCHANGE_CAPABILITIES => {
"The method {} does not exist/is not available", let engine_capabilities = ctx.engine_capabilities.read();
other Ok(serde_json::to_value(engine_capabilities.to_response()).unwrap())
}
other => Err((
format!("The method {} does not exist/is not available", other),
METHOD_NOT_FOUND_CODE,
)), )),
} }
} }

View File

@ -1,4 +1,4 @@
use crate::test_utils::DEFAULT_JWT_SECRET; use crate::test_utils::{DEFAULT_BUILDER_PAYLOAD_VALUE_WEI, DEFAULT_JWT_SECRET};
use crate::{Config, ExecutionLayer, PayloadAttributes}; use crate::{Config, ExecutionLayer, PayloadAttributes};
use async_trait::async_trait; use async_trait::async_trait;
use eth2::types::{BlockId, StateId, ValidatorId}; use eth2::types::{BlockId, StateId, ValidatorId};
@ -84,8 +84,7 @@ impl<E: EthSpec> TestingBuilder<E> {
}; };
let el = let el =
ExecutionLayer::from_config(config, executor.clone(), executor.log().clone(), &spec) ExecutionLayer::from_config(config, executor.clone(), executor.log().clone()).unwrap();
.unwrap();
// This should probably be done for all fields, we only update ones we are testing with so far. // This should probably be done for all fields, we only update ones we are testing with so far.
let mut context = Context::for_mainnet(); let mut context = Context::for_mainnet();
@ -329,7 +328,7 @@ impl<E: EthSpec> mev_build_rs::BlindedBlockProvider for MockBuilder<E> {
let mut message = BuilderBid { let mut message = BuilderBid {
header, header,
value: ssz_rs::U256::default(), value: to_ssz_rs(&Uint256::from(DEFAULT_BUILDER_PAYLOAD_VALUE_WEI))?,
public_key: self.builder_sk.public_key(), public_key: self.builder_sk.public_key(),
}; };

View File

@ -29,6 +29,7 @@ impl<T: EthSpec> MockExecutionLayer<T> {
DEFAULT_TERMINAL_BLOCK, DEFAULT_TERMINAL_BLOCK,
None, None,
None, None,
None,
Some(JwtKey::from_slice(&DEFAULT_JWT_SECRET).unwrap()), Some(JwtKey::from_slice(&DEFAULT_JWT_SECRET).unwrap()),
spec, spec,
None, None,
@ -41,6 +42,7 @@ impl<T: EthSpec> MockExecutionLayer<T> {
terminal_block: u64, terminal_block: u64,
shanghai_time: Option<u64>, shanghai_time: Option<u64>,
eip4844_time: Option<u64>, eip4844_time: Option<u64>,
builder_threshold: Option<u128>,
jwt_key: Option<JwtKey>, jwt_key: Option<JwtKey>,
spec: ChainSpec, spec: ChainSpec,
builder_url: Option<SensitiveUrl>, builder_url: Option<SensitiveUrl>,
@ -69,12 +71,11 @@ impl<T: EthSpec> MockExecutionLayer<T> {
builder_url, builder_url,
secret_files: vec![path], secret_files: vec![path],
suggested_fee_recipient: Some(Address::repeat_byte(42)), suggested_fee_recipient: Some(Address::repeat_byte(42)),
builder_profit_threshold: DEFAULT_BUILDER_THRESHOLD_WEI, builder_profit_threshold: builder_threshold.unwrap_or(DEFAULT_BUILDER_THRESHOLD_WEI),
..Default::default() ..Default::default()
}; };
let el = let el =
ExecutionLayer::from_config(config, executor.clone(), executor.log().clone(), &spec) ExecutionLayer::from_config(config, executor.clone(), executor.log().clone()).unwrap();
.unwrap();
Self { Self {
server, server,
@ -106,7 +107,7 @@ impl<T: EthSpec> MockExecutionLayer<T> {
prev_randao, prev_randao,
Address::repeat_byte(42), Address::repeat_byte(42),
// FIXME: think about how to handle different forks / withdrawals here.. // FIXME: think about how to handle different forks / withdrawals here..
Some(vec![]), None,
); );
// Insert a proposer to ensure the fork choice updated command works. // Insert a proposer to ensure the fork choice updated command works.

View File

@ -22,6 +22,7 @@ use tokio::{runtime, sync::oneshot};
use types::{EthSpec, ExecutionBlockHash, Uint256}; use types::{EthSpec, ExecutionBlockHash, Uint256};
use warp::{http::StatusCode, Filter, Rejection}; use warp::{http::StatusCode, Filter, Rejection};
use crate::EngineCapabilities;
pub use execution_block_generator::{generate_pow_block, Block, ExecutionBlockGenerator}; pub use execution_block_generator::{generate_pow_block, Block, ExecutionBlockGenerator};
pub use hook::Hook; pub use hook::Hook;
pub use mock_builder::{Context as MockBuilderContext, MockBuilder, Operation, TestingBuilder}; pub use mock_builder::{Context as MockBuilderContext, MockBuilder, Operation, TestingBuilder};
@ -31,6 +32,17 @@ pub const DEFAULT_TERMINAL_DIFFICULTY: u64 = 6400;
pub const DEFAULT_TERMINAL_BLOCK: u64 = 64; pub const DEFAULT_TERMINAL_BLOCK: u64 = 64;
pub const DEFAULT_JWT_SECRET: [u8; 32] = [42; 32]; pub const DEFAULT_JWT_SECRET: [u8; 32] = [42; 32];
pub const DEFAULT_BUILDER_THRESHOLD_WEI: u128 = 1_000_000_000_000_000_000; pub const DEFAULT_BUILDER_THRESHOLD_WEI: u128 = 1_000_000_000_000_000_000;
pub const DEFAULT_MOCK_EL_PAYLOAD_VALUE_WEI: u128 = 10_000_000_000_000_000;
pub const DEFAULT_BUILDER_PAYLOAD_VALUE_WEI: u128 = 20_000_000_000_000_000;
pub const DEFAULT_ENGINE_CAPABILITIES: EngineCapabilities = EngineCapabilities {
new_payload_v1: true,
new_payload_v2: true,
forkchoice_updated_v1: true,
forkchoice_updated_v2: true,
get_payload_v1: true,
get_payload_v2: true,
exchange_transition_configuration_v1: true,
};
mod execution_block_generator; mod execution_block_generator;
mod handle_rpc; mod handle_rpc;
@ -117,6 +129,7 @@ impl<T: EthSpec> MockServer<T> {
hook: <_>::default(), hook: <_>::default(),
new_payload_statuses: <_>::default(), new_payload_statuses: <_>::default(),
fcu_payload_statuses: <_>::default(), fcu_payload_statuses: <_>::default(),
engine_capabilities: Arc::new(RwLock::new(DEFAULT_ENGINE_CAPABILITIES)),
_phantom: PhantomData, _phantom: PhantomData,
}); });
@ -147,6 +160,10 @@ impl<T: EthSpec> MockServer<T> {
} }
} }
pub fn set_engine_capabilities(&self, engine_capabilities: EngineCapabilities) {
*self.ctx.engine_capabilities.write() = engine_capabilities;
}
pub fn new( pub fn new(
handle: &runtime::Handle, handle: &runtime::Handle,
jwt_key: JwtKey, jwt_key: JwtKey,
@ -469,6 +486,7 @@ pub struct Context<T: EthSpec> {
pub new_payload_statuses: Arc<Mutex<HashMap<ExecutionBlockHash, PayloadStatusV1>>>, pub new_payload_statuses: Arc<Mutex<HashMap<ExecutionBlockHash, PayloadStatusV1>>>,
pub fcu_payload_statuses: Arc<Mutex<HashMap<ExecutionBlockHash, PayloadStatusV1>>>, pub fcu_payload_statuses: Arc<Mutex<HashMap<ExecutionBlockHash, PayloadStatusV1>>>,
pub engine_capabilities: Arc<RwLock<EngineCapabilities>>,
pub _phantom: PhantomData<T>, pub _phantom: PhantomData<T>,
} }
@ -620,11 +638,11 @@ pub fn serve<T: EthSpec>(
"jsonrpc": JSONRPC_VERSION, "jsonrpc": JSONRPC_VERSION,
"result": result "result": result
}), }),
Err(message) => json!({ Err((message, code)) => json!({
"id": id, "id": id,
"jsonrpc": JSONRPC_VERSION, "jsonrpc": JSONRPC_VERSION,
"error": { "error": {
"code": -1234, // Junk error code. "code": code,
"message": message "message": message
} }
}), }),

View File

@ -36,6 +36,7 @@ tree_hash = "0.4.1"
sysinfo = "0.26.5" sysinfo = "0.26.5"
system_health = { path = "../../common/system_health" } system_health = { path = "../../common/system_health" }
directory = { path = "../../common/directory" } directory = { path = "../../common/directory" }
operation_pool = { path = "../operation_pool" }
[dev-dependencies] [dev-dependencies]
store = { path = "../store" } store = { path = "../store" }

View File

@ -35,6 +35,7 @@ use eth2::types::{
use lighthouse_network::{types::SyncState, EnrExt, NetworkGlobals, PeerId, PubsubMessage}; use lighthouse_network::{types::SyncState, EnrExt, NetworkGlobals, PeerId, PubsubMessage};
use lighthouse_version::version_with_platform; use lighthouse_version::version_with_platform;
use network::{NetworkMessage, NetworkSenders, ValidatorSubscriptionMessage}; use network::{NetworkMessage, NetworkSenders, ValidatorSubscriptionMessage};
use operation_pool::ReceivedPreCapella;
use parking_lot::RwLock; use parking_lot::RwLock;
use serde::{Deserialize, Serialize}; use serde::{Deserialize, Serialize};
use slog::{crit, debug, error, info, warn, Logger}; use slog::{crit, debug, error, info, warn, Logger};
@ -1690,8 +1691,12 @@ pub fn serve<T: BeaconChainTypes>(
.to_execution_address; .to_execution_address;
// New to P2P *and* op pool, gossip immediately if post-Capella. // New to P2P *and* op pool, gossip immediately if post-Capella.
let publish = chain.current_slot_is_post_capella().unwrap_or(false); let received_pre_capella = if chain.current_slot_is_post_capella().unwrap_or(false) {
if publish { ReceivedPreCapella::No
} else {
ReceivedPreCapella::Yes
};
if matches!(received_pre_capella, ReceivedPreCapella::No) {
publish_pubsub_message( publish_pubsub_message(
&network_tx, &network_tx,
PubsubMessage::BlsToExecutionChange(Box::new( PubsubMessage::BlsToExecutionChange(Box::new(
@ -1702,14 +1707,14 @@ pub fn serve<T: BeaconChainTypes>(
// Import to op pool (may return `false` if there's a race). // Import to op pool (may return `false` if there's a race).
let imported = let imported =
chain.import_bls_to_execution_change(verified_address_change); chain.import_bls_to_execution_change(verified_address_change, received_pre_capella);
info!( info!(
log, log,
"Processed BLS to execution change"; "Processed BLS to execution change";
"validator_index" => validator_index, "validator_index" => validator_index,
"address" => ?address, "address" => ?address,
"published" => publish, "published" => matches!(received_pre_capella, ReceivedPreCapella::No),
"imported" => imported, "imported" => imported,
); );
} }
@ -3566,6 +3571,7 @@ pub fn serve<T: BeaconChainTypes>(
.or(post_beacon_pool_sync_committees.boxed()) .or(post_beacon_pool_sync_committees.boxed())
.or(post_beacon_rewards_sync_committee.boxed()) .or(post_beacon_rewards_sync_committee.boxed())
.or(post_beacon_pool_bls_to_execution_changes.boxed()) .or(post_beacon_pool_bls_to_execution_changes.boxed())
.or(post_beacon_rewards_sync_committee.boxed())
.or(post_validator_duties_attester.boxed()) .or(post_validator_duties_attester.boxed())
.or(post_validator_duties_sync.boxed()) .or(post_validator_duties_sync.boxed())
.or(post_validator_aggregate_and_proofs.boxed()) .or(post_validator_aggregate_and_proofs.boxed())

View File

@ -6,6 +6,7 @@ use beacon_chain::{
}; };
use eth2::types::{IndexedErrorMessage, StateId, SyncSubcommittee}; use eth2::types::{IndexedErrorMessage, StateId, SyncSubcommittee};
use genesis::{bls_withdrawal_credentials, interop_genesis_state_with_withdrawal_credentials}; use genesis::{bls_withdrawal_credentials, interop_genesis_state_with_withdrawal_credentials};
use std::collections::HashSet;
use types::{ use types::{
test_utils::{generate_deterministic_keypair, generate_deterministic_keypairs}, test_utils::{generate_deterministic_keypair, generate_deterministic_keypairs},
Address, ChainSpec, Epoch, EthSpec, Hash256, MinimalEthSpec, Slot, Address, ChainSpec, Epoch, EthSpec, Hash256, MinimalEthSpec, Slot,
@ -438,6 +439,8 @@ async fn bls_to_execution_changes_update_all_around_capella_fork() {
.await .await
.unwrap(); .unwrap();
let expected_received_pre_capella_messages = valid_address_changes[..num_pre_capella].to_vec();
// Conflicting changes for the same validators should all fail. // Conflicting changes for the same validators should all fail.
let error = client let error = client
.post_beacon_pool_bls_to_execution_changes(&conflicting_address_changes[..num_pre_capella]) .post_beacon_pool_bls_to_execution_changes(&conflicting_address_changes[..num_pre_capella])
@ -464,6 +467,20 @@ async fn bls_to_execution_changes_update_all_around_capella_fork() {
harness.extend_to_slot(capella_slot - 1).await; harness.extend_to_slot(capella_slot - 1).await;
assert_eq!(harness.head_slot(), capella_slot - 1); assert_eq!(harness.head_slot(), capella_slot - 1);
assert_eq!(
harness
.chain
.op_pool
.get_bls_to_execution_changes_received_pre_capella(
&harness.chain.head_snapshot().beacon_state,
&spec,
)
.into_iter()
.collect::<HashSet<_>>(),
HashSet::from_iter(expected_received_pre_capella_messages.into_iter()),
"all pre-capella messages should be queued for capella broadcast"
);
// Add Capella blocks which should be full of BLS to execution changes. // Add Capella blocks which should be full of BLS to execution changes.
for i in 0..validator_count / max_bls_to_execution_changes { for i in 0..validator_count / max_bls_to_execution_changes {
let head_block_root = harness.extend_slots(1).await; let head_block_root = harness.extend_slots(1).await;

View File

@ -11,9 +11,11 @@ use eth2::{
types::{BlockId as CoreBlockId, StateId as CoreStateId, *}, types::{BlockId as CoreBlockId, StateId as CoreStateId, *},
BeaconNodeHttpClient, Error, StatusCode, Timeouts, BeaconNodeHttpClient, Error, StatusCode, Timeouts,
}; };
use execution_layer::test_utils::Operation;
use execution_layer::test_utils::TestingBuilder; use execution_layer::test_utils::TestingBuilder;
use execution_layer::test_utils::DEFAULT_BUILDER_THRESHOLD_WEI; use execution_layer::test_utils::DEFAULT_BUILDER_THRESHOLD_WEI;
use execution_layer::test_utils::{
Operation, DEFAULT_BUILDER_PAYLOAD_VALUE_WEI, DEFAULT_MOCK_EL_PAYLOAD_VALUE_WEI,
};
use futures::stream::{Stream, StreamExt}; use futures::stream::{Stream, StreamExt};
use futures::FutureExt; use futures::FutureExt;
use http_api::{BlockId, StateId}; use http_api::{BlockId, StateId};
@ -72,38 +74,53 @@ struct ApiTester {
mock_builder: Option<Arc<TestingBuilder<E>>>, mock_builder: Option<Arc<TestingBuilder<E>>>,
} }
struct ApiTesterConfig {
spec: ChainSpec,
builder_threshold: Option<u128>,
}
impl Default for ApiTesterConfig {
fn default() -> Self {
let mut spec = E::default_spec();
spec.shard_committee_period = 2;
Self {
spec,
builder_threshold: None,
}
}
}
impl ApiTester { impl ApiTester {
pub async fn new() -> Self { pub async fn new() -> Self {
// This allows for testing voluntary exits without building out a massive chain. // This allows for testing voluntary exits without building out a massive chain.
let mut spec = E::default_spec(); Self::new_from_config(ApiTesterConfig::default()).await
spec.shard_committee_period = 2;
Self::new_from_spec(spec).await
} }
pub async fn new_with_hard_forks(altair: bool, bellatrix: bool) -> Self { pub async fn new_with_hard_forks(altair: bool, bellatrix: bool) -> Self {
let mut spec = E::default_spec(); let mut config = ApiTesterConfig::default();
spec.shard_committee_period = 2;
// Set whether the chain has undergone each hard fork. // Set whether the chain has undergone each hard fork.
if altair { if altair {
spec.altair_fork_epoch = Some(Epoch::new(0)); config.spec.altair_fork_epoch = Some(Epoch::new(0));
} }
if bellatrix { if bellatrix {
spec.bellatrix_fork_epoch = Some(Epoch::new(0)); config.spec.bellatrix_fork_epoch = Some(Epoch::new(0));
} }
Self::new_from_spec(spec).await Self::new_from_config(config).await
} }
pub async fn new_from_spec(spec: ChainSpec) -> Self { pub async fn new_from_config(config: ApiTesterConfig) -> Self {
// Get a random unused port // Get a random unused port
let spec = config.spec;
let port = unused_port::unused_tcp_port().unwrap(); let port = unused_port::unused_tcp_port().unwrap();
let beacon_url = SensitiveUrl::parse(format!("http://127.0.0.1:{port}").as_str()).unwrap(); let beacon_url = SensitiveUrl::parse(format!("http://127.0.0.1:{port}").as_str()).unwrap();
let harness = Arc::new( let harness = Arc::new(
BeaconChainHarness::builder(MainnetEthSpec) BeaconChainHarness::builder(MainnetEthSpec)
.spec(spec.clone()) .spec(spec.clone())
.logger(logging::test_logger())
.deterministic_keypairs(VALIDATOR_COUNT) .deterministic_keypairs(VALIDATOR_COUNT)
.fresh_ephemeral_store() .fresh_ephemeral_store()
.mock_execution_layer_with_builder(beacon_url.clone()) .mock_execution_layer_with_builder(beacon_url.clone(), config.builder_threshold)
.build(), .build(),
); );
@ -358,6 +375,28 @@ impl ApiTester {
tester tester
} }
pub async fn new_mev_tester_no_builder_threshold() -> Self {
let mut config = ApiTesterConfig {
builder_threshold: Some(0),
spec: E::default_spec(),
};
config.spec.altair_fork_epoch = Some(Epoch::new(0));
config.spec.bellatrix_fork_epoch = Some(Epoch::new(0));
let tester = Self::new_from_config(config)
.await
.test_post_validator_register_validator()
.await;
tester
.mock_builder
.as_ref()
.unwrap()
.builder
.add_operation(Operation::Value(Uint256::from(
DEFAULT_BUILDER_PAYLOAD_VALUE_WEI,
)));
tester
}
fn skip_slots(self, count: u64) -> Self { fn skip_slots(self, count: u64) -> Self {
for _ in 0..count { for _ in 0..count {
self.chain self.chain
@ -3278,6 +3317,117 @@ impl ApiTester {
self self
} }
pub async fn test_builder_payload_chosen_when_more_profitable(self) -> Self {
// Mutate value.
self.mock_builder
.as_ref()
.unwrap()
.builder
.add_operation(Operation::Value(Uint256::from(
DEFAULT_MOCK_EL_PAYLOAD_VALUE_WEI + 1,
)));
let slot = self.chain.slot().unwrap();
let epoch = self.chain.epoch().unwrap();
let (_, randao_reveal) = self.get_test_randao(slot, epoch).await;
let payload: BlindedPayload<E> = self
.client
.get_validator_blinded_blocks::<E, BlindedPayload<E>>(slot, &randao_reveal, None)
.await
.unwrap()
.data
.body()
.execution_payload()
.unwrap()
.into();
// The builder's payload should've been chosen, so this cache should not be populated
assert!(self
.chain
.execution_layer
.as_ref()
.unwrap()
.get_payload_by_root(&payload.tree_hash_root())
.is_none());
self
}
pub async fn test_local_payload_chosen_when_equally_profitable(self) -> Self {
// Mutate value.
self.mock_builder
.as_ref()
.unwrap()
.builder
.add_operation(Operation::Value(Uint256::from(
DEFAULT_MOCK_EL_PAYLOAD_VALUE_WEI,
)));
let slot = self.chain.slot().unwrap();
let epoch = self.chain.epoch().unwrap();
let (_, randao_reveal) = self.get_test_randao(slot, epoch).await;
let payload: BlindedPayload<E> = self
.client
.get_validator_blinded_blocks::<E, BlindedPayload<E>>(slot, &randao_reveal, None)
.await
.unwrap()
.data
.body()
.execution_payload()
.unwrap()
.into();
// The local payload should've been chosen, so this cache should be populated
assert!(self
.chain
.execution_layer
.as_ref()
.unwrap()
.get_payload_by_root(&payload.tree_hash_root())
.is_some());
self
}
pub async fn test_local_payload_chosen_when_more_profitable(self) -> Self {
// Mutate value.
self.mock_builder
.as_ref()
.unwrap()
.builder
.add_operation(Operation::Value(Uint256::from(
DEFAULT_MOCK_EL_PAYLOAD_VALUE_WEI - 1,
)));
let slot = self.chain.slot().unwrap();
let epoch = self.chain.epoch().unwrap();
let (_, randao_reveal) = self.get_test_randao(slot, epoch).await;
let payload: BlindedPayload<E> = self
.client
.get_validator_blinded_blocks::<E, BlindedPayload<E>>(slot, &randao_reveal, None)
.await
.unwrap()
.data
.body()
.execution_payload()
.unwrap()
.into();
// The local payload should've been chosen, so this cache should be populated
assert!(self
.chain
.execution_layer
.as_ref()
.unwrap()
.get_payload_by_root(&payload.tree_hash_root())
.is_some());
self
}
#[cfg(target_os = "linux")] #[cfg(target_os = "linux")]
pub async fn test_get_lighthouse_health(self) -> Self { pub async fn test_get_lighthouse_health(self) -> Self {
self.client.get_lighthouse_health().await.unwrap(); self.client.get_lighthouse_health().await.unwrap();
@ -3747,9 +3897,9 @@ async fn get_events() {
#[tokio::test(flavor = "multi_thread", worker_threads = 2)] #[tokio::test(flavor = "multi_thread", worker_threads = 2)]
async fn get_events_altair() { async fn get_events_altair() {
let mut spec = E::default_spec(); let mut config = ApiTesterConfig::default();
spec.altair_fork_epoch = Some(Epoch::new(0)); config.spec.altair_fork_epoch = Some(Epoch::new(0));
ApiTester::new_from_spec(spec) ApiTester::new_from_config(config)
.await .await
.test_get_events_altair() .test_get_events_altair()
.await; .await;
@ -4262,6 +4412,18 @@ async fn builder_inadequate_builder_threshold() {
.await; .await;
} }
#[tokio::test(flavor = "multi_thread", worker_threads = 2)]
async fn builder_payload_chosen_by_profit() {
ApiTester::new_mev_tester_no_builder_threshold()
.await
.test_builder_payload_chosen_when_more_profitable()
.await
.test_local_payload_chosen_when_equally_profitable()
.await
.test_local_payload_chosen_when_more_profitable()
.await;
}
#[tokio::test(flavor = "multi_thread", worker_threads = 2)] #[tokio::test(flavor = "multi_thread", worker_threads = 2)]
async fn lighthouse_endpoints() { async fn lighthouse_endpoints() {
ApiTester::new() ApiTester::new()

View File

@ -45,6 +45,7 @@ tokio-util = { version = "0.6.3", features = ["time"] }
derivative = "2.2.0" derivative = "2.2.0"
delay_map = "0.1.1" delay_map = "0.1.1"
ethereum-types = { version = "0.14.1", optional = true } ethereum-types = { version = "0.14.1", optional = true }
operation_pool = { path = "../operation_pool" }
[features] [features]
deterministic_long_lived_attnets = [ "ethereum-types" ] deterministic_long_lived_attnets = [ "ethereum-types" ]

View File

@ -802,6 +802,21 @@ impl<T: BeaconChainTypes> std::convert::From<ReadyWork<T>> for WorkEvent<T> {
seen_timestamp, seen_timestamp,
}, },
}, },
ReadyWork::LightClientUpdate(QueuedLightClientUpdate {
peer_id,
message_id,
light_client_optimistic_update,
seen_timestamp,
..
}) => Self {
drop_during_sync: true,
work: Work::UnknownLightClientOptimisticUpdate {
message_id,
peer_id,
light_client_optimistic_update,
seen_timestamp,
},
},
} }
} }
} }
@ -986,6 +1001,7 @@ impl<T: BeaconChainTypes> Work<T> {
Work::UnknownBlockAggregate { .. } => UNKNOWN_BLOCK_AGGREGATE, Work::UnknownBlockAggregate { .. } => UNKNOWN_BLOCK_AGGREGATE,
Work::UnknownLightClientOptimisticUpdate { .. } => UNKNOWN_LIGHT_CLIENT_UPDATE, Work::UnknownLightClientOptimisticUpdate { .. } => UNKNOWN_LIGHT_CLIENT_UPDATE,
Work::GossipBlsToExecutionChange { .. } => GOSSIP_BLS_TO_EXECUTION_CHANGE, Work::GossipBlsToExecutionChange { .. } => GOSSIP_BLS_TO_EXECUTION_CHANGE,
Work::UnknownLightClientOptimisticUpdate { .. } => UNKNOWN_LIGHT_CLIENT_UPDATE,
} }
} }
} }
@ -1522,6 +1538,9 @@ impl<T: BeaconChainTypes> BeaconProcessor<T> {
Work::BlobsByRootsRequest { .. } => { Work::BlobsByRootsRequest { .. } => {
blbroots_queue.push(work, work_id, &self.log) blbroots_queue.push(work, work_id, &self.log)
} }
Work::UnknownLightClientOptimisticUpdate { .. } => {
unknown_light_client_update_queue.push(work, work_id, &self.log)
}
} }
} }
} }

View File

@ -31,7 +31,8 @@ use tokio::sync::mpsc::{self, Receiver, Sender};
use tokio::time::error::Error as TimeError; use tokio::time::error::Error as TimeError;
use tokio_util::time::delay_queue::{DelayQueue, Key as DelayKey}; use tokio_util::time::delay_queue::{DelayQueue, Key as DelayKey};
use types::{ use types::{
Attestation, EthSpec, Hash256, LightClientOptimisticUpdate, SignedAggregateAndProof, SubnetId, Attestation, EthSpec, Hash256, LightClientOptimisticUpdate, SignedAggregateAndProof,
SignedBeaconBlock, SubnetId,
}; };
const TASK_NAME: &str = "beacon_processor_reprocess_queue"; const TASK_NAME: &str = "beacon_processor_reprocess_queue";

View File

@ -13,6 +13,7 @@ use beacon_chain::{
GossipVerifiedBlock, NotifyExecutionLayer, GossipVerifiedBlock, NotifyExecutionLayer,
}; };
use lighthouse_network::{Client, MessageAcceptance, MessageId, PeerAction, PeerId, ReportSource}; use lighthouse_network::{Client, MessageAcceptance, MessageId, PeerAction, PeerId, ReportSource};
use operation_pool::ReceivedPreCapella;
use slog::{crit, debug, error, info, trace, warn}; use slog::{crit, debug, error, info, trace, warn};
use slot_clock::SlotClock; use slot_clock::SlotClock;
use ssz::Encode; use ssz::Encode;
@ -1262,7 +1263,12 @@ impl<T: BeaconChainTypes> Worker<T> {
self.propagate_validation_result(message_id, peer_id, MessageAcceptance::Accept); self.propagate_validation_result(message_id, peer_id, MessageAcceptance::Accept);
self.chain.import_bls_to_execution_change(change); // Address change messages from gossip are only processed *after* the
// Capella fork epoch.
let received_pre_capella = ReceivedPreCapella::No;
self.chain
.import_bls_to_execution_change(change, received_pre_capella);
debug!( debug!(
self.log, self.log,

View File

@ -19,6 +19,7 @@ serde = "1.0.116"
serde_derive = "1.0.116" serde_derive = "1.0.116"
store = { path = "../store" } store = { path = "../store" }
bitvec = "1" bitvec = "1"
rand = "0.8.5"
[dev-dependencies] [dev-dependencies]
beacon_chain = { path = "../beacon_chain" } beacon_chain = { path = "../beacon_chain" }

View File

@ -1,11 +1,20 @@
use state_processing::SigVerifiedOp; use state_processing::SigVerifiedOp;
use std::collections::{hash_map::Entry, HashMap}; use std::collections::{hash_map::Entry, HashMap, HashSet};
use std::sync::Arc; use std::sync::Arc;
use types::{ use types::{
AbstractExecPayload, BeaconState, ChainSpec, EthSpec, SignedBeaconBlock, AbstractExecPayload, BeaconState, ChainSpec, EthSpec, SignedBeaconBlock,
SignedBlsToExecutionChange, SignedBlsToExecutionChange,
}; };
/// Indicates if a `BlsToExecutionChange` was received before or after the
/// Capella fork. This is used to know which messages we should broadcast at the
/// Capella fork epoch.
#[derive(Copy, Clone)]
pub enum ReceivedPreCapella {
Yes,
No,
}
/// Pool of BLS to execution changes that maintains a LIFO queue and an index by validator. /// Pool of BLS to execution changes that maintains a LIFO queue and an index by validator.
/// ///
/// Using the LIFO queue for block production disincentivises spam on P2P at the Capella fork, /// Using the LIFO queue for block production disincentivises spam on P2P at the Capella fork,
@ -16,6 +25,9 @@ pub struct BlsToExecutionChanges<T: EthSpec> {
by_validator_index: HashMap<u64, Arc<SigVerifiedOp<SignedBlsToExecutionChange, T>>>, by_validator_index: HashMap<u64, Arc<SigVerifiedOp<SignedBlsToExecutionChange, T>>>,
/// Last-in-first-out (LIFO) queue of verified messages. /// Last-in-first-out (LIFO) queue of verified messages.
queue: Vec<Arc<SigVerifiedOp<SignedBlsToExecutionChange, T>>>, queue: Vec<Arc<SigVerifiedOp<SignedBlsToExecutionChange, T>>>,
/// Contains a set of validator indices which need to have their changes
/// broadcast at the capella epoch.
received_pre_capella_indices: HashSet<u64>,
} }
impl<T: EthSpec> BlsToExecutionChanges<T> { impl<T: EthSpec> BlsToExecutionChanges<T> {
@ -31,16 +43,18 @@ impl<T: EthSpec> BlsToExecutionChanges<T> {
pub fn insert( pub fn insert(
&mut self, &mut self,
verified_change: SigVerifiedOp<SignedBlsToExecutionChange, T>, verified_change: SigVerifiedOp<SignedBlsToExecutionChange, T>,
received_pre_capella: ReceivedPreCapella,
) -> bool { ) -> bool {
let validator_index = verified_change.as_inner().message.validator_index;
// Wrap in an `Arc` once on insert. // Wrap in an `Arc` once on insert.
let verified_change = Arc::new(verified_change); let verified_change = Arc::new(verified_change);
match self match self.by_validator_index.entry(validator_index) {
.by_validator_index
.entry(verified_change.as_inner().message.validator_index)
{
Entry::Vacant(entry) => { Entry::Vacant(entry) => {
self.queue.push(verified_change.clone()); self.queue.push(verified_change.clone());
entry.insert(verified_change); entry.insert(verified_change);
if matches!(received_pre_capella, ReceivedPreCapella::Yes) {
self.received_pre_capella_indices.insert(validator_index);
}
true true
} }
Entry::Occupied(_) => false, Entry::Occupied(_) => false,
@ -61,6 +75,24 @@ impl<T: EthSpec> BlsToExecutionChanges<T> {
self.queue.iter().rev() self.queue.iter().rev()
} }
/// Returns only those which are flagged for broadcasting at the Capella
/// fork. Uses FIFO ordering, although we expect this list to be shuffled by
/// the caller.
pub fn iter_received_pre_capella(
&self,
) -> impl Iterator<Item = &Arc<SigVerifiedOp<SignedBlsToExecutionChange, T>>> {
self.queue.iter().filter(|address_change| {
self.received_pre_capella_indices
.contains(&address_change.as_inner().message.validator_index)
})
}
/// Returns the set of indicies which should have their address changes
/// broadcast at the Capella fork.
pub fn iter_pre_capella_indices(&self) -> impl Iterator<Item = &u64> {
self.received_pre_capella_indices.iter()
}
/// Prune BLS to execution changes that have been applied to the state more than 1 block ago. /// Prune BLS to execution changes that have been applied to the state more than 1 block ago.
/// ///
/// The block check is necessary to avoid pruning too eagerly and losing the ability to include /// The block check is necessary to avoid pruning too eagerly and losing the ability to include
@ -102,4 +134,14 @@ impl<T: EthSpec> BlsToExecutionChanges<T> {
self.by_validator_index.remove(&validator_index); self.by_validator_index.remove(&validator_index);
} }
} }
/// Removes `broadcasted` validators from the set of validators that should
/// have their BLS changes broadcast at the Capella fork boundary.
pub fn register_indices_broadcasted_at_capella(&mut self, broadcasted: &HashSet<u64>) {
self.received_pre_capella_indices = self
.received_pre_capella_indices
.difference(broadcasted)
.copied()
.collect();
}
} }

View File

@ -9,12 +9,13 @@ mod persistence;
mod reward_cache; mod reward_cache;
mod sync_aggregate_id; mod sync_aggregate_id;
pub use crate::bls_to_execution_changes::ReceivedPreCapella;
pub use attestation::AttMaxCover; pub use attestation::AttMaxCover;
pub use attestation_storage::{AttestationRef, SplitAttestation}; pub use attestation_storage::{AttestationRef, SplitAttestation};
pub use max_cover::MaxCover; pub use max_cover::MaxCover;
pub use persistence::{ pub use persistence::{
PersistedOperationPool, PersistedOperationPoolV12, PersistedOperationPoolV14, PersistedOperationPool, PersistedOperationPoolV12, PersistedOperationPoolV14,
PersistedOperationPoolV5, PersistedOperationPoolV15, PersistedOperationPoolV5,
}; };
pub use reward_cache::RewardCache; pub use reward_cache::RewardCache;
@ -24,6 +25,8 @@ use crate::sync_aggregate_id::SyncAggregateId;
use attester_slashing::AttesterSlashingMaxCover; use attester_slashing::AttesterSlashingMaxCover;
use max_cover::maximum_cover; use max_cover::maximum_cover;
use parking_lot::{RwLock, RwLockWriteGuard}; use parking_lot::{RwLock, RwLockWriteGuard};
use rand::seq::SliceRandom;
use rand::thread_rng;
use state_processing::per_block_processing::errors::AttestationValidationError; use state_processing::per_block_processing::errors::AttestationValidationError;
use state_processing::per_block_processing::{ use state_processing::per_block_processing::{
get_slashable_indices_modular, verify_exit, VerifySignatures, get_slashable_indices_modular, verify_exit, VerifySignatures,
@ -533,10 +536,11 @@ impl<T: EthSpec> OperationPool<T> {
pub fn insert_bls_to_execution_change( pub fn insert_bls_to_execution_change(
&self, &self,
verified_change: SigVerifiedOp<SignedBlsToExecutionChange, T>, verified_change: SigVerifiedOp<SignedBlsToExecutionChange, T>,
received_pre_capella: ReceivedPreCapella,
) -> bool { ) -> bool {
self.bls_to_execution_changes self.bls_to_execution_changes
.write() .write()
.insert(verified_change) .insert(verified_change, received_pre_capella)
} }
/// Get a list of execution changes for inclusion in a block. /// Get a list of execution changes for inclusion in a block.
@ -562,6 +566,42 @@ impl<T: EthSpec> OperationPool<T> {
) )
} }
/// Get a list of execution changes to be broadcast at the Capella fork.
///
/// The list that is returned will be shuffled to help provide a fair
/// broadcast of messages.
pub fn get_bls_to_execution_changes_received_pre_capella(
&self,
state: &BeaconState<T>,
spec: &ChainSpec,
) -> Vec<SignedBlsToExecutionChange> {
let mut changes = filter_limit_operations(
self.bls_to_execution_changes
.read()
.iter_received_pre_capella(),
|address_change| {
address_change.signature_is_still_valid(&state.fork())
&& state
.get_validator(address_change.as_inner().message.validator_index as usize)
.map_or(false, |validator| {
!validator.has_eth1_withdrawal_credential(spec)
})
},
|address_change| address_change.as_inner().clone(),
usize::max_value(),
);
changes.shuffle(&mut thread_rng());
changes
}
/// Removes `broadcasted` validators from the set of validators that should
/// have their BLS changes broadcast at the Capella fork boundary.
pub fn register_indices_broadcasted_at_capella(&self, broadcasted: &HashSet<u64>) {
self.bls_to_execution_changes
.write()
.register_indices_broadcasted_at_capella(broadcasted);
}
/// Prune BLS to execution changes that have been applied to the state more than 1 block ago. /// Prune BLS to execution changes that have been applied to the state more than 1 block ago.
pub fn prune_bls_to_execution_changes<Payload: AbstractExecPayload<T>>( pub fn prune_bls_to_execution_changes<Payload: AbstractExecPayload<T>>(
&self, &self,

View File

@ -1,6 +1,6 @@
use crate::attestation_id::AttestationId; use crate::attestation_id::AttestationId;
use crate::attestation_storage::AttestationMap; use crate::attestation_storage::AttestationMap;
use crate::bls_to_execution_changes::BlsToExecutionChanges; use crate::bls_to_execution_changes::{BlsToExecutionChanges, ReceivedPreCapella};
use crate::sync_aggregate_id::SyncAggregateId; use crate::sync_aggregate_id::SyncAggregateId;
use crate::OpPoolError; use crate::OpPoolError;
use crate::OperationPool; use crate::OperationPool;
@ -9,6 +9,8 @@ use parking_lot::RwLock;
use ssz::{Decode, Encode}; use ssz::{Decode, Encode};
use ssz_derive::{Decode, Encode}; use ssz_derive::{Decode, Encode};
use state_processing::SigVerifiedOp; use state_processing::SigVerifiedOp;
use std::collections::HashSet;
use std::mem;
use store::{DBColumn, Error as StoreError, StoreItem}; use store::{DBColumn, Error as StoreError, StoreItem};
use types::*; use types::*;
@ -19,7 +21,7 @@ type PersistedSyncContributions<T> = Vec<(SyncAggregateId, Vec<SyncCommitteeCont
/// Operations are stored in arbitrary order, so it's not a good idea to compare instances /// Operations are stored in arbitrary order, so it's not a good idea to compare instances
/// of this type (or its encoded form) for equality. Convert back to an `OperationPool` first. /// of this type (or its encoded form) for equality. Convert back to an `OperationPool` first.
#[superstruct( #[superstruct(
variants(V5, V12, V14), variants(V5, V12, V14, V15),
variant_attributes( variant_attributes(
derive(Derivative, PartialEq, Debug, Encode, Decode), derive(Derivative, PartialEq, Debug, Encode, Decode),
derivative(Clone), derivative(Clone),
@ -33,7 +35,7 @@ pub struct PersistedOperationPool<T: EthSpec> {
#[superstruct(only(V5))] #[superstruct(only(V5))]
pub attestations_v5: Vec<(AttestationId, Vec<Attestation<T>>)>, pub attestations_v5: Vec<(AttestationId, Vec<Attestation<T>>)>,
/// Attestations and their attesting indices. /// Attestations and their attesting indices.
#[superstruct(only(V12, V14))] #[superstruct(only(V12, V14, V15))]
pub attestations: Vec<(Attestation<T>, Vec<u64>)>, pub attestations: Vec<(Attestation<T>, Vec<u64>)>,
/// Mapping from sync contribution ID to sync contributions and aggregate. /// Mapping from sync contribution ID to sync contributions and aggregate.
pub sync_contributions: PersistedSyncContributions<T>, pub sync_contributions: PersistedSyncContributions<T>,
@ -41,23 +43,27 @@ pub struct PersistedOperationPool<T: EthSpec> {
#[superstruct(only(V5))] #[superstruct(only(V5))]
pub attester_slashings_v5: Vec<(AttesterSlashing<T>, ForkVersion)>, pub attester_slashings_v5: Vec<(AttesterSlashing<T>, ForkVersion)>,
/// Attester slashings. /// Attester slashings.
#[superstruct(only(V12, V14))] #[superstruct(only(V12, V14, V15))]
pub attester_slashings: Vec<SigVerifiedOp<AttesterSlashing<T>, T>>, pub attester_slashings: Vec<SigVerifiedOp<AttesterSlashing<T>, T>>,
/// [DEPRECATED] Proposer slashings. /// [DEPRECATED] Proposer slashings.
#[superstruct(only(V5))] #[superstruct(only(V5))]
pub proposer_slashings_v5: Vec<ProposerSlashing>, pub proposer_slashings_v5: Vec<ProposerSlashing>,
/// Proposer slashings with fork information. /// Proposer slashings with fork information.
#[superstruct(only(V12, V14))] #[superstruct(only(V12, V14, V15))]
pub proposer_slashings: Vec<SigVerifiedOp<ProposerSlashing, T>>, pub proposer_slashings: Vec<SigVerifiedOp<ProposerSlashing, T>>,
/// [DEPRECATED] Voluntary exits. /// [DEPRECATED] Voluntary exits.
#[superstruct(only(V5))] #[superstruct(only(V5))]
pub voluntary_exits_v5: Vec<SignedVoluntaryExit>, pub voluntary_exits_v5: Vec<SignedVoluntaryExit>,
/// Voluntary exits with fork information. /// Voluntary exits with fork information.
#[superstruct(only(V12, V14))] #[superstruct(only(V12, V14, V15))]
pub voluntary_exits: Vec<SigVerifiedOp<SignedVoluntaryExit, T>>, pub voluntary_exits: Vec<SigVerifiedOp<SignedVoluntaryExit, T>>,
/// BLS to Execution Changes /// BLS to Execution Changes
#[superstruct(only(V14))] #[superstruct(only(V14, V15))]
pub bls_to_execution_changes: Vec<SigVerifiedOp<SignedBlsToExecutionChange, T>>, pub bls_to_execution_changes: Vec<SigVerifiedOp<SignedBlsToExecutionChange, T>>,
/// Validator indices with BLS to Execution Changes to be broadcast at the
/// Capella fork.
#[superstruct(only(V15))]
pub capella_bls_change_broadcast_indices: Vec<u64>,
} }
impl<T: EthSpec> PersistedOperationPool<T> { impl<T: EthSpec> PersistedOperationPool<T> {
@ -110,18 +116,26 @@ impl<T: EthSpec> PersistedOperationPool<T> {
.map(|bls_to_execution_change| (**bls_to_execution_change).clone()) .map(|bls_to_execution_change| (**bls_to_execution_change).clone())
.collect(); .collect();
PersistedOperationPool::V14(PersistedOperationPoolV14 { let capella_bls_change_broadcast_indices = operation_pool
.bls_to_execution_changes
.read()
.iter_pre_capella_indices()
.copied()
.collect();
PersistedOperationPool::V15(PersistedOperationPoolV15 {
attestations, attestations,
sync_contributions, sync_contributions,
attester_slashings, attester_slashings,
proposer_slashings, proposer_slashings,
voluntary_exits, voluntary_exits,
bls_to_execution_changes, bls_to_execution_changes,
capella_bls_change_broadcast_indices,
}) })
} }
/// Reconstruct an `OperationPool`. /// Reconstruct an `OperationPool`.
pub fn into_operation_pool(self) -> Result<OperationPool<T>, OpPoolError> { pub fn into_operation_pool(mut self) -> Result<OperationPool<T>, OpPoolError> {
let attester_slashings = RwLock::new(self.attester_slashings()?.iter().cloned().collect()); let attester_slashings = RwLock::new(self.attester_slashings()?.iter().cloned().collect());
let proposer_slashings = RwLock::new( let proposer_slashings = RwLock::new(
self.proposer_slashings()? self.proposer_slashings()?
@ -142,33 +156,43 @@ impl<T: EthSpec> PersistedOperationPool<T> {
PersistedOperationPool::V5(_) | PersistedOperationPool::V12(_) => { PersistedOperationPool::V5(_) | PersistedOperationPool::V12(_) => {
return Err(OpPoolError::IncorrectOpPoolVariant) return Err(OpPoolError::IncorrectOpPoolVariant)
} }
PersistedOperationPool::V14(ref pool) => { PersistedOperationPool::V14(_) | PersistedOperationPool::V15(_) => {
let mut map = AttestationMap::default(); let mut map = AttestationMap::default();
for (att, attesting_indices) in pool.attestations.clone() { for (att, attesting_indices) in self.attestations()?.clone() {
map.insert(att, attesting_indices); map.insert(att, attesting_indices);
} }
RwLock::new(map) RwLock::new(map)
} }
}; };
let bls_to_execution_changes = match self { let mut bls_to_execution_changes = BlsToExecutionChanges::default();
PersistedOperationPool::V5(_) | PersistedOperationPool::V12(_) => { if let Ok(persisted_changes) = self.bls_to_execution_changes_mut() {
return Err(OpPoolError::IncorrectOpPoolVariant) let persisted_changes = mem::take(persisted_changes);
let broadcast_indices =
if let Ok(indices) = self.capella_bls_change_broadcast_indices_mut() {
mem::take(indices).into_iter().collect()
} else {
HashSet::new()
};
for bls_to_execution_change in persisted_changes {
let received_pre_capella = if broadcast_indices
.contains(&bls_to_execution_change.as_inner().message.validator_index)
{
ReceivedPreCapella::Yes
} else {
ReceivedPreCapella::No
};
bls_to_execution_changes.insert(bls_to_execution_change, received_pre_capella);
} }
PersistedOperationPool::V14(pool) => { }
let mut bls_to_execution_changes = BlsToExecutionChanges::default();
for bls_to_execution_change in pool.bls_to_execution_changes {
bls_to_execution_changes.insert(bls_to_execution_change);
}
RwLock::new(bls_to_execution_changes)
}
};
let op_pool = OperationPool { let op_pool = OperationPool {
attestations, attestations,
sync_contributions, sync_contributions,
attester_slashings, attester_slashings,
proposer_slashings, proposer_slashings,
voluntary_exits, voluntary_exits,
bls_to_execution_changes, bls_to_execution_changes: RwLock::new(bls_to_execution_changes),
reward_cache: Default::default(), reward_cache: Default::default(),
_phantom: Default::default(), _phantom: Default::default(),
}; };
@ -204,6 +228,20 @@ impl<T: EthSpec> StoreItem for PersistedOperationPoolV12<T> {
} }
} }
impl<T: EthSpec> StoreItem for PersistedOperationPoolV14<T> {
fn db_column() -> DBColumn {
DBColumn::OpPool
}
fn as_store_bytes(&self) -> Vec<u8> {
self.as_ssz_bytes()
}
fn from_store_bytes(bytes: &[u8]) -> Result<Self, StoreError> {
PersistedOperationPoolV14::from_ssz_bytes(bytes).map_err(Into::into)
}
}
/// Deserialization for `PersistedOperationPool` defaults to `PersistedOperationPool::V12`. /// Deserialization for `PersistedOperationPool` defaults to `PersistedOperationPool::V12`.
impl<T: EthSpec> StoreItem for PersistedOperationPool<T> { impl<T: EthSpec> StoreItem for PersistedOperationPool<T> {
fn db_column() -> DBColumn { fn db_column() -> DBColumn {
@ -216,8 +254,8 @@ impl<T: EthSpec> StoreItem for PersistedOperationPool<T> {
fn from_store_bytes(bytes: &[u8]) -> Result<Self, StoreError> { fn from_store_bytes(bytes: &[u8]) -> Result<Self, StoreError> {
// Default deserialization to the latest variant. // Default deserialization to the latest variant.
PersistedOperationPoolV14::from_ssz_bytes(bytes) PersistedOperationPoolV15::from_ssz_bytes(bytes)
.map(Self::V14) .map(Self::V15)
.map_err(Into::into) .map_err(Into::into)
} }
} }

View File

@ -4,7 +4,7 @@ use ssz::{Decode, Encode};
use ssz_derive::{Decode, Encode}; use ssz_derive::{Decode, Encode};
use types::{Checkpoint, Hash256, Slot}; use types::{Checkpoint, Hash256, Slot};
pub const CURRENT_SCHEMA_VERSION: SchemaVersion = SchemaVersion(14); pub const CURRENT_SCHEMA_VERSION: SchemaVersion = SchemaVersion(15);
// All the keys that get stored under the `BeaconMeta` column. // All the keys that get stored under the `BeaconMeta` column.
// //

View File

@ -97,7 +97,7 @@ You can opt-in to reconstructing all of the historic states by providing the
The database keeps track of three markers to determine the availability of historic blocks and The database keeps track of three markers to determine the availability of historic blocks and
states: states:
* `oldest_block_slot`: All blocks with slots less than or equal to this value are available in the * `oldest_block_slot`: All blocks with slots greater than or equal to this value are available in the
database. Additionally, the genesis block is always available. database. Additionally, the genesis block is always available.
* `state_lower_limit`: All states with slots _less than or equal to_ this value are available in * `state_lower_limit`: All states with slots _less than or equal to_ this value are available in
the database. The minimum value is 0, indicating that the genesis state is always available. the database. The minimum value is 0, indicating that the genesis state is always available.

View File

@ -115,11 +115,7 @@ impl Comparison {
let mut children = vec![]; let mut children = vec![];
for i in 0..std::cmp::max(a.len(), b.len()) { for i in 0..std::cmp::max(a.len(), b.len()) {
children.push(FieldComparison::new( children.push(FieldComparison::new(format!("{i}"), &a.get(i), &b.get(i)));
format!("{:}", i),
&a.get(i),
&b.get(i),
));
} }
Self::parent(field_name, a == b, children) Self::parent(field_name, a == b, children)
@ -164,8 +160,8 @@ impl FieldComparison {
Self { Self {
field_name, field_name,
equal: a == b, equal: a == b,
a: format!("{:?}", a), a: format!("{a:?}"),
b: format!("{:?}", b), b: format!("{b:?}"),
} }
} }

View File

@ -32,7 +32,7 @@ pub fn compare_fields_derive(input: TokenStream) -> TokenStream {
_ => panic!("compare_fields_derive only supports named struct fields."), _ => panic!("compare_fields_derive only supports named struct fields."),
}; };
let field_name = format!("{:}", ident_a); let field_name = ident_a.to_string();
let ident_b = ident_a.clone(); let ident_b = ident_a.clone();
let quote = if is_slice(field) { let quote = if is_slice(field) {

View File

@ -7,7 +7,6 @@ pub mod json_str;
pub mod list_of_bytes_lists; pub mod list_of_bytes_lists;
pub mod quoted_u64_vec; pub mod quoted_u64_vec;
pub mod u256_hex_be; pub mod u256_hex_be;
pub mod u256_hex_be_opt;
pub mod u32_hex; pub mod u32_hex;
pub mod u64_hex_be; pub mod u64_hex_be;
pub mod u8_hex; pub mod u8_hex;

View File

@ -1,188 +0,0 @@
use ethereum_types::U256;
use serde::de::{Error, Visitor};
use serde::{de, Deserializer, Serialize, Serializer};
use std::fmt;
use std::str::FromStr;
pub fn serialize<S>(num: &Option<U256>, serializer: S) -> Result<S::Ok, S::Error>
where
S: Serializer,
{
num.serialize(serializer)
}
pub struct U256Visitor;
impl<'de> Visitor<'de> for U256Visitor {
type Value = Option<String>;
fn expecting(&self, formatter: &mut fmt::Formatter) -> fmt::Result {
formatter.write_str("a well formatted hex string")
}
fn visit_some<D>(self, deserializer: D) -> Result<Self::Value, D::Error>
where
D: Deserializer<'de>,
{
deserializer.deserialize_string(U256Visitor)
}
fn visit_none<E>(self) -> Result<Self::Value, E>
where
E: Error,
{
Ok(None)
}
fn visit_str<E>(self, value: &str) -> Result<Self::Value, E>
where
E: de::Error,
{
if !value.starts_with("0x") {
return Err(de::Error::custom("must start with 0x"));
}
let stripped = &value[2..];
if stripped.is_empty() {
Err(de::Error::custom(format!(
"quantity cannot be {:?}",
stripped
)))
} else if stripped == "0" {
Ok(Some(value.to_string()))
} else if stripped.starts_with('0') {
Err(de::Error::custom("cannot have leading zero"))
} else {
Ok(Some(value.to_string()))
}
}
}
pub fn deserialize<'de, D>(deserializer: D) -> Result<Option<U256>, D::Error>
where
D: Deserializer<'de>,
{
let decoded = deserializer.deserialize_option(U256Visitor)?;
decoded
.map(|decoded| {
U256::from_str(&decoded)
.map_err(|e| de::Error::custom(format!("Invalid U256 string: {}", e)))
})
.transpose()
}
#[cfg(test)]
mod test {
use ethereum_types::U256;
use serde::{Deserialize, Serialize};
use serde_json;
#[derive(Debug, PartialEq, Serialize, Deserialize)]
#[serde(transparent)]
struct Wrapper {
#[serde(with = "super")]
val: Option<U256>,
}
#[test]
fn encoding() {
assert_eq!(
&serde_json::to_string(&Wrapper {
val: Some(0.into())
})
.unwrap(),
"\"0x0\""
);
assert_eq!(
&serde_json::to_string(&Wrapper {
val: Some(1.into())
})
.unwrap(),
"\"0x1\""
);
assert_eq!(
&serde_json::to_string(&Wrapper {
val: Some(256.into())
})
.unwrap(),
"\"0x100\""
);
assert_eq!(
&serde_json::to_string(&Wrapper {
val: Some(65.into())
})
.unwrap(),
"\"0x41\""
);
assert_eq!(
&serde_json::to_string(&Wrapper {
val: Some(1024.into())
})
.unwrap(),
"\"0x400\""
);
assert_eq!(
&serde_json::to_string(&Wrapper {
val: Some(U256::max_value() - 1)
})
.unwrap(),
"\"0xfffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffe\""
);
assert_eq!(
&serde_json::to_string(&Wrapper {
val: Some(U256::max_value())
})
.unwrap(),
"\"0xffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff\""
);
}
#[test]
fn decoding() {
assert_eq!(
serde_json::from_str::<Wrapper>("\"0x0\"").unwrap(),
Wrapper {
val: Some(0.into())
},
);
assert_eq!(
serde_json::from_str::<Wrapper>("\"0x41\"").unwrap(),
Wrapper {
val: Some(65.into())
},
);
assert_eq!(
serde_json::from_str::<Wrapper>("\"0x400\"").unwrap(),
Wrapper {
val: Some(1024.into())
},
);
assert_eq!(
serde_json::from_str::<Wrapper>(
"\"0xfffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffe\""
)
.unwrap(),
Wrapper {
val: Some(U256::max_value() - 1)
},
);
assert_eq!(
serde_json::from_str::<Wrapper>(
"\"0xffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff\""
)
.unwrap(),
Wrapper {
val: Some(U256::max_value())
},
);
assert_eq!(
serde_json::from_str::<Wrapper>("null").unwrap(),
Wrapper { val: None },
);
serde_json::from_str::<Wrapper>("\"0x\"").unwrap_err();
serde_json::from_str::<Wrapper>("\"0x0400\"").unwrap_err();
serde_json::from_str::<Wrapper>("\"400\"").unwrap_err();
serde_json::from_str::<Wrapper>("\"ff\"").unwrap_err();
}
}

View File

@ -343,10 +343,10 @@ pub fn get_new_eth1_data<T: EthSpec>(
/// Contains a partial set of checks from the `process_execution_payload` function: /// 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 /// https://github.com/ethereum/consensus-specs/blob/v1.1.5/specs/merge/beacon-chain.md#process_execution_payload
pub fn partially_verify_execution_payload<'payload, T: EthSpec, Payload: AbstractExecPayload<T>>( pub fn partially_verify_execution_payload<T: EthSpec, Payload: AbstractExecPayload<T>>(
state: &BeaconState<T>, state: &BeaconState<T>,
block_slot: Slot, block_slot: Slot,
payload: Payload::Ref<'payload>, payload: Payload::Ref<'_>,
spec: &ChainSpec, spec: &ChainSpec,
) -> Result<(), BlockProcessingError> { ) -> Result<(), BlockProcessingError> {
if is_merge_transition_complete(state) { if is_merge_transition_complete(state) {
@ -385,9 +385,9 @@ pub fn partially_verify_execution_payload<'payload, T: EthSpec, Payload: Abstrac
/// Partially equivalent to the `process_execution_payload` function: /// 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 /// https://github.com/ethereum/consensus-specs/blob/v1.1.5/specs/merge/beacon-chain.md#process_execution_payload
pub fn process_execution_payload<'payload, T: EthSpec, Payload: AbstractExecPayload<T>>( pub fn process_execution_payload<T: EthSpec, Payload: AbstractExecPayload<T>>(
state: &mut BeaconState<T>, state: &mut BeaconState<T>,
payload: Payload::Ref<'payload>, payload: Payload::Ref<'_>,
spec: &ChainSpec, spec: &ChainSpec,
) -> Result<(), BlockProcessingError> { ) -> Result<(), BlockProcessingError> {
partially_verify_execution_payload::<T, Payload>(state, state.slot(), payload, spec)?; partially_verify_execution_payload::<T, Payload>(state, state.slot(), payload, spec)?;
@ -516,9 +516,9 @@ pub fn get_expected_withdrawals<T: EthSpec>(
} }
/// Apply withdrawals to the state. /// Apply withdrawals to the state.
pub fn process_withdrawals<'payload, T: EthSpec, Payload: AbstractExecPayload<T>>( pub fn process_withdrawals<T: EthSpec, Payload: AbstractExecPayload<T>>(
state: &mut BeaconState<T>, state: &mut BeaconState<T>,
payload: Payload::Ref<'payload>, payload: Payload::Ref<'_>,
spec: &ChainSpec, spec: &ChainSpec,
) -> Result<(), BlockProcessingError> { ) -> Result<(), BlockProcessingError> {
match state { match state {

View File

@ -9,9 +9,9 @@ use crate::VerifySignatures;
use safe_arith::SafeArith; use safe_arith::SafeArith;
use types::consts::altair::{PARTICIPATION_FLAG_WEIGHTS, PROPOSER_WEIGHT, WEIGHT_DENOMINATOR}; use types::consts::altair::{PARTICIPATION_FLAG_WEIGHTS, PROPOSER_WEIGHT, WEIGHT_DENOMINATOR};
pub fn process_operations<'a, T: EthSpec, Payload: AbstractExecPayload<T>>( pub fn process_operations<T: EthSpec, Payload: AbstractExecPayload<T>>(
state: &mut BeaconState<T>, state: &mut BeaconState<T>,
block_body: BeaconBlockBodyRef<'a, T, Payload>, block_body: BeaconBlockBodyRef<T, Payload>,
verify_signatures: VerifySignatures, verify_signatures: VerifySignatures,
ctxt: &mut ConsensusContext<T>, ctxt: &mut ConsensusContext<T>,
spec: &ChainSpec, spec: &ChainSpec,
@ -237,9 +237,9 @@ pub fn process_attester_slashings<T: EthSpec>(
} }
/// Wrapper function to handle calling the correct version of `process_attestations` based on /// Wrapper function to handle calling the correct version of `process_attestations` based on
/// the fork. /// the fork.
pub fn process_attestations<'a, T: EthSpec, Payload: AbstractExecPayload<T>>( pub fn process_attestations<T: EthSpec, Payload: AbstractExecPayload<T>>(
state: &mut BeaconState<T>, state: &mut BeaconState<T>,
block_body: BeaconBlockBodyRef<'a, T, Payload>, block_body: BeaconBlockBodyRef<T, Payload>,
verify_signatures: VerifySignatures, verify_signatures: VerifySignatures,
ctxt: &mut ConsensusContext<T>, ctxt: &mut ConsensusContext<T>,
spec: &ChainSpec, spec: &ChainSpec,

View File

@ -10,6 +10,7 @@ use tree_hash_derive::TreeHash;
arbitrary::Arbitrary, arbitrary::Arbitrary,
Debug, Debug,
PartialEq, PartialEq,
Eq,
Hash, Hash,
Clone, Clone,
Serialize, Serialize,

View File

@ -87,6 +87,16 @@ pub struct ExecutionPayload<T: EthSpec> {
pub withdrawals: Withdrawals<T>, pub withdrawals: Withdrawals<T>,
} }
impl<'a, T: EthSpec> ExecutionPayloadRef<'a, T> {
// this emulates clone on a normal reference type
pub fn clone_from_ref(&self) -> ExecutionPayload<T> {
map_execution_payload_ref!(&'a _, self, move |payload, cons| {
cons(payload);
payload.clone().into()
})
}
}
impl<T: EthSpec> ExecutionPayload<T> { impl<T: EthSpec> ExecutionPayload<T> {
pub fn from_ssz_bytes(bytes: &[u8], fork_name: ForkName) -> Result<Self, ssz::DecodeError> { pub fn from_ssz_bytes(bytes: &[u8], fork_name: ForkName) -> Result<Self, ssz::DecodeError> {
match fork_name { match fork_name {

View File

@ -10,6 +10,7 @@ use tree_hash_derive::TreeHash;
arbitrary::Arbitrary, arbitrary::Arbitrary,
Debug, Debug,
PartialEq, PartialEq,
Eq,
Hash, Hash,
Clone, Clone,
Serialize, Serialize,

View File

@ -127,7 +127,7 @@ impl<E: GenericExecutionEngine> TestRig<E> {
..Default::default() ..Default::default()
}; };
let execution_layer = let execution_layer =
ExecutionLayer::from_config(config, executor.clone(), log.clone(), &spec).unwrap(); ExecutionLayer::from_config(config, executor.clone(), log.clone()).unwrap();
ExecutionPair { ExecutionPair {
execution_engine, execution_engine,
execution_layer, execution_layer,
@ -146,7 +146,7 @@ impl<E: GenericExecutionEngine> TestRig<E> {
..Default::default() ..Default::default()
}; };
let execution_layer = let execution_layer =
ExecutionLayer::from_config(config, executor, log.clone(), &spec).unwrap(); ExecutionLayer::from_config(config, executor, log.clone()).unwrap();
ExecutionPair { ExecutionPair {
execution_engine, execution_engine,
execution_layer, execution_layer,