//! This crate provides an abstraction over one or more *execution engines*. An execution engine
//! was formerly known as an "eth1 node", like Geth, Nethermind, Erigon, etc.
//!
//! This crate only provides useful functionality for "The Merge", it does not provide any of the
//! deposit-contract functionality that the `beacon_node/eth1` crate already provides.
use crate::payload_cache::PayloadCache;
use auth::{strip_prefix, Auth, JwtKey};
use builder_client::BuilderHttpClient;
pub use engine_api::EngineCapabilities;
use engine_api::Error as ApiError;
pub use engine_api::*;
pub use engine_api::{http, http::deposit_methods, http::HttpJsonRpc};
use engines::{Engine, EngineError};
pub use engines::{EngineState, ForkchoiceState};
use eth2::types::{builder_bid::SignedBuilderBid, BlobsBundle, ForkVersionedResponse};
use eth2::types::{FullPayloadContents, SignedBlockContents};
use ethers_core::abi::ethereum_types::FromStrRadixErr;
use ethers_core::types::Transaction as EthersTransaction;
use fork_choice::ForkchoiceUpdateParameters;
use lru::LruCache;
use payload_status::process_payload_status;
pub use payload_status::PayloadStatus;
use sensitive_url::SensitiveUrl;
use serde::{Deserialize, Serialize};
use slog::{crit, debug, error, info, trace, warn, Logger};
use slot_clock::SlotClock;
use std::collections::HashMap;
use std::fmt;
use std::future::Future;
use std::io::Write;
use std::path::PathBuf;
use std::sync::Arc;
use std::time::{Duration, Instant, SystemTime, UNIX_EPOCH};
use strum::AsRefStr;
use task_executor::TaskExecutor;
use tokio::{
sync::{Mutex, MutexGuard, RwLock},
time::sleep,
};
use tokio_stream::wrappers::WatchStream;
use tree_hash::TreeHash;
use types::beacon_block_body::KzgCommitments;
use types::blob_sidecar::BlobItems;
use types::builder_bid::BuilderBid;
use types::{
AbstractExecPayload, BeaconStateError, BlindedPayload, BlockType, ChainSpec, Epoch,
ExecPayload, ExecutionPayloadCapella, ExecutionPayloadDeneb, ExecutionPayloadMerge,
};
use types::{KzgProofs, Sidecar};
use types::{ProposerPreparationData, PublicKeyBytes, Signature, Slot, Transaction};
mod block_hash;
mod engine_api;
pub mod engines;
mod keccak;
mod metrics;
pub mod payload_cache;
mod payload_status;
pub mod test_utils;
/// Indicates the default jwt authenticated execution endpoint.
pub const DEFAULT_EXECUTION_ENDPOINT: &str = "http://localhost:8551/";
/// Name for the default file used for the jwt secret.
pub const DEFAULT_JWT_FILE: &str = "jwt.hex";
/// Each time the `ExecutionLayer` retrieves a block from an execution node, it stores that block
/// in an LRU cache to avoid redundant lookups. This is the size of that cache.
const EXECUTION_BLOCKS_LRU_CACHE_SIZE: usize = 128;
/// A fee recipient address for use during block production. Only used as a very last resort if
/// there is no address provided by the user.
///
/// ## Note
///
/// This is *not* the zero-address, since Geth has been known to return errors for a coinbase of
/// 0x00..00.
const DEFAULT_SUGGESTED_FEE_RECIPIENT: [u8; 20] =
[0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1];
/// A payload alongside some information about where it came from.
pub enum ProvenancedPayload
{
/// A good old fashioned farm-to-table payload from your local EE.
Local(P),
/// A payload from a builder (e.g. mev-boost).
Builder(P),
}
impl> TryFrom>
for ProvenancedPayload>
{
type Error = Error;
fn try_from(value: BuilderBid) -> Result {
let block_proposal_contents = match value {
BuilderBid::Merge(builder_bid) => BlockProposalContents::Payload {
payload: ExecutionPayloadHeader::Merge(builder_bid.header)
.try_into()
.map_err(|_| Error::InvalidPayloadConversion)?,
block_value: builder_bid.value,
},
BuilderBid::Capella(builder_bid) => BlockProposalContents::Payload {
payload: ExecutionPayloadHeader::Capella(builder_bid.header)
.try_into()
.map_err(|_| Error::InvalidPayloadConversion)?,
block_value: builder_bid.value,
},
BuilderBid::Deneb(builder_bid) => BlockProposalContents::PayloadAndBlobs {
payload: ExecutionPayloadHeader::Deneb(builder_bid.header)
.try_into()
.map_err(|_| Error::InvalidPayloadConversion)?,
block_value: builder_bid.value,
kzg_commitments: builder_bid.blinded_blobs_bundle.commitments,
blobs: BlobItems::try_from_blob_roots(builder_bid.blinded_blobs_bundle.blob_roots)
.map_err(Error::InvalidBlobConversion)?,
proofs: builder_bid.blinded_blobs_bundle.proofs,
},
};
Ok(ProvenancedPayload::Builder(block_proposal_contents))
}
}
#[derive(Debug)]
pub enum Error {
NoEngine,
NoPayloadBuilder,
ApiError(ApiError),
Builder(builder_client::Error),
NoHeaderFromBuilder,
CannotProduceHeader,
EngineError(Box),
NotSynced,
ShuttingDown,
FeeRecipientUnspecified,
MissingLatestValidHash,
BlockHashMismatch {
computed: ExecutionBlockHash,
payload: ExecutionBlockHash,
transactions_root: Hash256,
},
InvalidJWTSecret(String),
InvalidForkForPayload,
InvalidPayloadBody(String),
InvalidPayloadConversion,
InvalidBlobConversion(String),
BeaconStateError(BeaconStateError),
}
impl From for Error {
fn from(e: BeaconStateError) -> Self {
Error::BeaconStateError(e)
}
}
impl From for Error {
fn from(e: ApiError) -> Self {
Error::ApiError(e)
}
}
pub enum BlockProposalContents> {
Payload {
payload: Payload,
block_value: Uint256,
},
PayloadAndBlobs {
payload: Payload,
block_value: Uint256,
kzg_commitments: KzgCommitments,
blobs: >::BlobItems,
proofs: KzgProofs,
},
}
impl> TryFrom>
for BlockProposalContents
{
type Error = Error;
fn try_from(response: GetPayloadResponse) -> Result {
let (execution_payload, block_value, maybe_bundle) = response.into();
match maybe_bundle {
Some(bundle) => Ok(Self::PayloadAndBlobs {
payload: execution_payload.into(),
block_value,
kzg_commitments: bundle.commitments,
blobs: BlobItems::try_from_blobs(bundle.blobs)
.map_err(Error::InvalidBlobConversion)?,
proofs: bundle.proofs,
}),
None => Ok(Self::Payload {
payload: execution_payload.into(),
block_value,
}),
}
}
}
#[allow(clippy::type_complexity)]
impl> BlockProposalContents {
pub fn deconstruct(
self,
) -> (
Payload,
Option>,
Option<>::BlobItems>,
Option>,
) {
match self {
Self::Payload {
payload,
block_value: _,
} => (payload, None, None, None),
Self::PayloadAndBlobs {
payload,
block_value: _,
kzg_commitments,
blobs,
proofs,
} => (payload, Some(kzg_commitments), Some(blobs), Some(proofs)),
}
}
pub fn payload(&self) -> &Payload {
match self {
Self::Payload { payload, .. } => payload,
Self::PayloadAndBlobs { payload, .. } => payload,
}
}
pub fn to_payload(self) -> Payload {
match self {
Self::Payload { payload, .. } => payload,
Self::PayloadAndBlobs { payload, .. } => payload,
}
}
pub fn block_value(&self) -> &Uint256 {
match self {
Self::Payload { block_value, .. } => block_value,
Self::PayloadAndBlobs { block_value, .. } => block_value,
}
}
pub fn default_at_fork(fork_name: ForkName) -> Result {
Ok(match fork_name {
ForkName::Base | ForkName::Altair | ForkName::Merge | ForkName::Capella => {
BlockProposalContents::Payload {
payload: Payload::default_at_fork(fork_name)?,
block_value: Uint256::zero(),
}
}
ForkName::Deneb => BlockProposalContents::PayloadAndBlobs {
payload: Payload::default_at_fork(fork_name)?,
block_value: Uint256::zero(),
blobs: Payload::default_blobs_at_fork(fork_name)?,
kzg_commitments: VariableList::default(),
proofs: VariableList::default(),
},
})
}
}
#[derive(Clone, PartialEq)]
pub struct ProposerPreparationDataEntry {
update_epoch: Epoch,
preparation_data: ProposerPreparationData,
}
#[derive(Hash, PartialEq, Eq)]
pub struct ProposerKey {
slot: Slot,
head_block_root: Hash256,
}
#[derive(PartialEq, Clone)]
pub struct Proposer {
validator_index: u64,
payload_attributes: PayloadAttributes,
}
/// Information from the beacon chain that is necessary for querying the builder API.
pub struct BuilderParams {
pub pubkey: PublicKeyBytes,
pub slot: Slot,
pub chain_health: ChainHealth,
}
pub enum ChainHealth {
Healthy,
Unhealthy(FailedCondition),
Optimistic,
PreMerge,
}
#[derive(Debug)]
pub enum FailedCondition {
Skips,
SkipsPerEpoch,
EpochsSinceFinalization,
}
type PayloadContentsRefTuple<'a, T> = (ExecutionPayloadRef<'a, T>, Option<&'a BlobsBundle>);
struct Inner {
engine: Arc,
builder: Option,
execution_engine_forkchoice_lock: Mutex<()>,
suggested_fee_recipient: Option,
proposer_preparation_data: Mutex>,
execution_blocks: Mutex>,
proposers: RwLock>,
executor: TaskExecutor,
payload_cache: PayloadCache,
builder_profit_threshold: Uint256,
log: Logger,
always_prefer_builder_payload: bool,
ignore_builder_override_suggestion_threshold: f32,
/// Track whether the last `newPayload` call errored.
///
/// This is used *only* in the informational sync status endpoint, so that a VC using this
/// node can prefer another node with a healthier EL.
last_new_payload_errored: RwLock,
}
#[derive(Debug, Default, Clone, Serialize, Deserialize)]
pub struct Config {
/// Endpoint urls for EL nodes that are running the engine api.
pub execution_endpoints: Vec,
/// Endpoint urls for services providing the builder api.
pub builder_url: Option,
/// User agent to send with requests to the builder API.
pub builder_user_agent: Option,
/// JWT secrets for the above endpoints running the engine api.
pub secret_files: Vec,
/// The default fee recipient to use on the beacon node if none if provided from
/// the validator client during block preparation.
pub suggested_fee_recipient: Option,
/// An optional id for the beacon node that will be passed to the EL in the JWT token claim.
pub jwt_id: Option,
/// An optional client version for the beacon node that will be passed to the EL in the JWT token claim.
pub jwt_version: Option,
/// Default directory for the jwt secret if not provided through cli.
pub default_datadir: PathBuf,
/// The minimum value of an external payload for it to be considered in a proposal.
pub builder_profit_threshold: u128,
pub execution_timeout_multiplier: Option,
pub always_prefer_builder_payload: bool,
pub ignore_builder_override_suggestion_threshold: f32,
}
/// Provides access to one execution engine and provides a neat interface for consumption by the
/// `BeaconChain`.
#[derive(Clone)]
pub struct ExecutionLayer {
inner: Arc>,
}
/// This function will return the percentage difference between 2 U256 values, using `base_value`
/// as the denominator. It is accurate to 7 decimal places which is about the precision of
/// an f32.
///
/// If some error is encountered in the calculation, None will be returned.
fn percentage_difference_u256(base_value: Uint256, comparison_value: Uint256) -> Option {
if base_value == Uint256::zero() {
return None;
}
// this is the total supply of ETH in WEI
let max_value = Uint256::from(12u8) * Uint256::exp10(25);
if base_value > max_value || comparison_value > max_value {
return None;
}
// Now we should be able to calculate the difference without division by zero or overflow
const PRECISION: usize = 7;
let precision_factor = Uint256::exp10(PRECISION);
let scaled_difference = if base_value <= comparison_value {
(comparison_value - base_value) * precision_factor
} else {
(base_value - comparison_value) * precision_factor
};
let scaled_proportion = scaled_difference / base_value;
// max value of scaled difference is 1.2 * 10^33, well below the max value of a u128 / f64 / f32
let percentage =
100.0f64 * scaled_proportion.low_u128() as f64 / precision_factor.low_u128() as f64;
if base_value <= comparison_value {
Some(percentage as f32)
} else {
Some(-percentage as f32)
}
}
impl ExecutionLayer {
/// Instantiate `Self` with an Execution engine specified in `Config`, using JSON-RPC via HTTP.
pub fn from_config(config: Config, executor: TaskExecutor, log: Logger) -> Result {
let Config {
execution_endpoints: urls,
builder_url,
builder_user_agent,
secret_files,
suggested_fee_recipient,
jwt_id,
jwt_version,
default_datadir,
builder_profit_threshold,
execution_timeout_multiplier,
always_prefer_builder_payload,
ignore_builder_override_suggestion_threshold,
} = config;
if urls.len() > 1 {
warn!(log, "Only the first execution engine url will be used");
}
let execution_url = urls.into_iter().next().ok_or(Error::NoEngine)?;
// Use the default jwt secret path if not provided via cli.
let secret_file = secret_files
.into_iter()
.next()
.unwrap_or_else(|| default_datadir.join(DEFAULT_JWT_FILE));
let jwt_key = if secret_file.exists() {
// Read secret from file if it already exists
std::fs::read_to_string(&secret_file)
.map_err(|e| format!("Failed to read JWT secret file. Error: {:?}", e))
.and_then(|ref s| {
let secret = JwtKey::from_slice(
&hex::decode(strip_prefix(s.trim_end()))
.map_err(|e| format!("Invalid hex string: {:?}", e))?,
)?;
Ok(secret)
})
.map_err(Error::InvalidJWTSecret)
} else {
// Create a new file and write a randomly generated secret to it if file does not exist
warn!(log, "No JWT found on disk. Generating"; "path" => %secret_file.display());
std::fs::File::options()
.write(true)
.create_new(true)
.open(&secret_file)
.map_err(|e| format!("Failed to open JWT secret file. Error: {:?}", e))
.and_then(|mut f| {
let secret = auth::JwtKey::random();
f.write_all(secret.hex_string().as_bytes())
.map_err(|e| format!("Failed to write to JWT secret file: {:?}", e))?;
Ok(secret)
})
.map_err(Error::InvalidJWTSecret)
}?;
let engine: Engine = {
let auth = Auth::new(jwt_key, jwt_id, jwt_version);
debug!(log, "Loaded execution endpoint"; "endpoint" => %execution_url, "jwt_path" => ?secret_file.as_path());
let api = HttpJsonRpc::new_with_auth(execution_url, auth, execution_timeout_multiplier)
.map_err(Error::ApiError)?;
Engine::new(api, executor.clone(), &log)
};
let builder = builder_url
.map(|url| {
let builder_client = BuilderHttpClient::new(url.clone(), builder_user_agent)
.map_err(Error::Builder)?;
info!(
log,
"Using external block builder";
"builder_url" => ?url,
"builder_profit_threshold" => builder_profit_threshold,
"local_user_agent" => builder_client.get_user_agent(),
);
Ok::<_, Error>(builder_client)
})
.transpose()?;
let inner = Inner {
engine: Arc::new(engine),
builder,
execution_engine_forkchoice_lock: <_>::default(),
suggested_fee_recipient,
proposer_preparation_data: Mutex::new(HashMap::new()),
proposers: RwLock::new(HashMap::new()),
execution_blocks: Mutex::new(LruCache::new(EXECUTION_BLOCKS_LRU_CACHE_SIZE)),
executor,
payload_cache: PayloadCache::default(),
builder_profit_threshold: Uint256::from(builder_profit_threshold),
log,
always_prefer_builder_payload,
ignore_builder_override_suggestion_threshold,
last_new_payload_errored: RwLock::new(false),
};
Ok(Self {
inner: Arc::new(inner),
})
}
}
impl ExecutionLayer {
fn engine(&self) -> &Arc {
&self.inner.engine
}
pub fn builder(&self) -> &Option {
&self.inner.builder
}
/// Cache a full payload, keyed on the `tree_hash_root` of the payload
fn cache_payload(
&self,
payload_and_blobs: PayloadContentsRefTuple,
) -> Option> {
let (payload_ref, maybe_json_blobs_bundle) = payload_and_blobs;
let payload = payload_ref.clone_from_ref();
let maybe_blobs_bundle = maybe_json_blobs_bundle
.cloned()
.map(|blobs_bundle| BlobsBundle {
commitments: blobs_bundle.commitments,
proofs: blobs_bundle.proofs,
blobs: blobs_bundle.blobs,
});
self.inner
.payload_cache
.put(FullPayloadContents::new(payload, maybe_blobs_bundle))
}
/// Attempt to retrieve a full payload from the payload cache by the payload root
pub fn get_payload_by_root(&self, root: &Hash256) -> Option> {
self.inner.payload_cache.get(root)
}
pub fn executor(&self) -> &TaskExecutor {
&self.inner.executor
}
/// Get the current difficulty of the PoW chain.
pub async fn get_current_difficulty(&self) -> Result {
let block = self
.engine()
.api
.get_block_by_number(BlockByNumberQuery::Tag(LATEST_TAG))
.await?
.ok_or(ApiError::ExecutionHeadBlockNotFound)?;
Ok(block.total_difficulty)
}
/// Note: this function returns a mutex guard, be careful to avoid deadlocks.
async fn execution_blocks(
&self,
) -> MutexGuard<'_, LruCache> {
self.inner.execution_blocks.lock().await
}
/// Gives access to a channel containing if the last engine state is online or not.
///
/// This can be called several times.
pub async fn get_responsiveness_watch(&self) -> WatchStream {
self.engine().watch_state().await
}
/// Note: this function returns a mutex guard, be careful to avoid deadlocks.
async fn proposer_preparation_data(
&self,
) -> MutexGuard<'_, HashMap> {
self.inner.proposer_preparation_data.lock().await
}
fn proposers(&self) -> &RwLock> {
&self.inner.proposers
}
fn log(&self) -> &Logger {
&self.inner.log
}
pub async fn execution_engine_forkchoice_lock(&self) -> MutexGuard<'_, ()> {
self.inner.execution_engine_forkchoice_lock.lock().await
}
/// Convenience function to allow spawning a task without waiting for the result.
pub fn spawn(&self, generate_future: F, name: &'static str)
where
F: FnOnce(Self) -> U,
U: Future