This commit is contained in:
realbigsean 2022-12-30 11:00:14 -05:00
parent 222a514506
commit d8f7277beb
No known key found for this signature in database
GPG Key ID: B372B64D866BF8CC
18 changed files with 109 additions and 302 deletions

View File

@ -6,7 +6,6 @@ use crate::attestation_verification::{
use crate::attester_cache::{AttesterCache, AttesterCacheKey}; use crate::attester_cache::{AttesterCache, AttesterCacheKey};
use crate::beacon_proposer_cache::compute_proposer_duties_from_head; use crate::beacon_proposer_cache::compute_proposer_duties_from_head;
use crate::beacon_proposer_cache::BeaconProposerCache; use crate::beacon_proposer_cache::BeaconProposerCache;
use crate::blob_verification::{BlobError, VerifiedBlobsSidecar};
use crate::block_times_cache::BlockTimesCache; use crate::block_times_cache::BlockTimesCache;
use crate::block_verification::{ use crate::block_verification::{
check_block_is_finalized_descendant, check_block_relevancy, get_block_root, check_block_is_finalized_descendant, check_block_relevancy, get_block_root,
@ -1818,23 +1817,6 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
}) })
} }
/// Accepts some `BlobsSidecar` received over from the network and attempts to verify it,
/// returning `Ok(_)` if it is valid to be (re)broadcast on the gossip network.
pub fn verify_blobs_sidecar_for_gossip<'a>(
&self,
blobs_sidecar: &'a BlobsSidecar<T::EthSpec>,
) -> Result<VerifiedBlobsSidecar<'a, T>, BlobError> {
metrics::inc_counter(&metrics::BLOBS_SIDECAR_PROCESSING_REQUESTS);
let _timer = metrics::start_timer(&metrics::BLOBS_SIDECAR_GOSSIP_VERIFICATION_TIMES);
VerifiedBlobsSidecar::verify(blobs_sidecar, self).map(|v| {
if let Some(_event_handler) = self.event_handler.as_ref() {
// TODO: Handle sse events
}
metrics::inc_counter(&metrics::BLOBS_SIDECAR_PROCESSING_SUCCESSES);
v
})
}
/// Accepts some 'LightClientFinalityUpdate' from the network and attempts to verify it /// Accepts some 'LightClientFinalityUpdate' from the network and attempts to verify it
pub fn verify_finality_update_for_gossip( pub fn verify_finality_update_for_gossip(
self: &Arc<Self>, self: &Arc<Self>,
@ -4479,7 +4461,6 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
.try_into() .try_into()
.map_err(|_| BlockProductionError::InvalidPayloadFork)?, .map_err(|_| BlockProductionError::InvalidPayloadFork)?,
bls_to_execution_changes: bls_to_execution_changes.into(), bls_to_execution_changes: bls_to_execution_changes.into(),
//FIXME(sean) get blobs
blob_kzg_commitments: VariableList::from(kzg_commitments), blob_kzg_commitments: VariableList::from(kzg_commitments),
}, },
}), }),

View File

@ -460,7 +460,7 @@ where
if is_terminal_block_hash_set && !is_activation_epoch_reached { if is_terminal_block_hash_set && !is_activation_epoch_reached {
// Use the "empty" payload if there's a terminal block hash, but we haven't reached the // Use the "empty" payload if there's a terminal block hash, but we haven't reached the
// terminal block epoch yet. // terminal block epoch yet.
return Ok(BlockProposalContents::default_at_fork(fork)); return BlockProposalContents::default_at_fork(fork).map_err(Into::into);
} }
let terminal_pow_block_hash = execution_layer let terminal_pow_block_hash = execution_layer
@ -473,7 +473,7 @@ where
} else { } else {
// If the merge transition hasn't occurred yet and the EL hasn't found the terminal // If the merge transition hasn't occurred yet and the EL hasn't found the terminal
// block, return an "empty" payload. // block, return an "empty" payload.
return Ok(BlockProposalContents::default_at_fork(fork)); return BlockProposalContents::default_at_fork(fork).map_err(Into::into);
} }
} else { } else {
latest_execution_payload_header_block_hash latest_execution_payload_header_block_hash

View File

@ -350,12 +350,14 @@ impl From<Withdrawal> for JsonWithdrawal {
impl From<JsonWithdrawal> for Withdrawal { impl From<JsonWithdrawal> for Withdrawal {
fn from(jw: JsonWithdrawal) -> Self { fn from(jw: JsonWithdrawal) -> Self {
// This comparison is to avoid a scenarion where the EE gives us too large a number this
// panics when it attempts to case to a `u64`.
let amount = std::cmp::max(jw.amount / 1000000000, Uint256::from(u64::MAX));
Self { Self {
index: jw.index, index: jw.index,
validator_index: jw.validator_index, validator_index: jw.validator_index,
address: jw.address, address: jw.address,
//FIXME(sean) if EE gives us too large a number this panics amount: amount.as_u64(),
amount: (jw.amount / 1000000000).as_u64(),
} }
} }
} }

View File

@ -35,7 +35,7 @@ use tokio::{
time::sleep, time::sleep,
}; };
use tokio_stream::wrappers::WatchStream; use tokio_stream::wrappers::WatchStream;
use types::{AbstractExecPayload, Blob, ExecPayload, KzgCommitment}; use types::{AbstractExecPayload, BeaconStateError, Blob, ExecPayload, KzgCommitment};
use types::{ use types::{
BlindedPayload, BlockType, ChainSpec, Epoch, ExecutionBlockHash, ForkName, BlindedPayload, BlockType, ChainSpec, Epoch, ExecutionBlockHash, ForkName,
ProposerPreparationData, PublicKeyBytes, Signature, SignedBeaconBlock, Slot, Uint256, ProposerPreparationData, PublicKeyBytes, Signature, SignedBeaconBlock, Slot, Uint256,
@ -95,6 +95,13 @@ pub enum Error {
FeeRecipientUnspecified, FeeRecipientUnspecified,
MissingLatestValidHash, MissingLatestValidHash,
InvalidJWTSecret(String), InvalidJWTSecret(String),
BeaconStateError(BeaconStateError),
}
impl From<BeaconStateError> for Error {
fn from(e: BeaconStateError) -> Self {
Error::BeaconStateError(e)
}
} }
impl From<ApiError> for Error { impl From<ApiError> for Error {
@ -153,17 +160,17 @@ impl<T: EthSpec, Payload: AbstractExecPayload<T>> BlockProposalContents<T, Paylo
} => Some(blobs), } => Some(blobs),
} }
} }
pub fn default_at_fork(fork_name: ForkName) -> Self { pub fn default_at_fork(fork_name: ForkName) -> Result<Self, BeaconStateError> {
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::default_at_fork(fork_name)?)
} }
ForkName::Eip4844 => BlockProposalContents::PayloadAndBlobs { ForkName::Eip4844 => BlockProposalContents::PayloadAndBlobs {
payload: Payload::default_at_fork(fork_name), payload: Payload::default_at_fork(fork_name)?,
blobs: vec![], blobs: vec![],
kzg_commitments: vec![], kzg_commitments: vec![],
}, },
} })
} }
} }
@ -803,10 +810,6 @@ impl<T: EthSpec> ExecutionLayer<T> {
spec, spec,
) { ) {
Ok(()) => Ok(ProvenancedPayload::Builder( Ok(()) => Ok(ProvenancedPayload::Builder(
//FIXME(sean) the builder API needs to be updated
// NOTE the comment above was removed in the
// rebase with unstable.. I think it goes
// here now?
BlockProposalContents::Payload(relay.data.message.header), BlockProposalContents::Payload(relay.data.message.header),
)), )),
Err(reason) if !reason.payload_invalid() => { Err(reason) if !reason.payload_invalid() => {
@ -858,19 +861,11 @@ impl<T: EthSpec> ExecutionLayer<T> {
spec, spec,
) { ) {
Ok(()) => Ok(ProvenancedPayload::Builder( Ok(()) => Ok(ProvenancedPayload::Builder(
//FIXME(sean) the builder API needs to be updated
// NOTE the comment above was removed in the
// rebase with unstable.. I think it goes
// here now?
BlockProposalContents::Payload(relay.data.message.header), BlockProposalContents::Payload(relay.data.message.header),
)), )),
// 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(
//FIXME(sean) the builder API needs to be updated
// NOTE the comment above was removed in the
// rebase with unstable.. I think it goes
// here now?
BlockProposalContents::Payload(relay.data.message.header), BlockProposalContents::Payload(relay.data.message.header),
)), )),
Err(reason) => { Err(reason) => {

View File

@ -189,6 +189,11 @@ async fn reconstruct_block<T: BeaconChainTypes>(
.spec .spec
.fork_name_at_epoch(block.slot().epoch(T::EthSpec::slots_per_epoch())), .fork_name_at_epoch(block.slot().epoch(T::EthSpec::slots_per_epoch())),
) )
.map_err(|e| {
warp_utils::reject::custom_server_error(format!(
"Default payload construction error: {e:?}"
))
})?
.into() .into()
// If we already have an execution payload with this transactions root cached, use it. // If we already have an execution payload with this transactions root cached, use it.
} else if let Some(cached_payload) = } else if let Some(cached_payload) =

View File

@ -119,8 +119,8 @@ lazy_static! {
pub(crate) const MAX_RPC_SIZE: usize = 1_048_576; // 1M pub(crate) const MAX_RPC_SIZE: usize = 1_048_576; // 1M
/// The maximum bytes that can be sent across the RPC post-merge. /// The maximum bytes that can be sent across the RPC post-merge.
pub(crate) const MAX_RPC_SIZE_POST_MERGE: usize = 10 * 1_048_576; // 10M pub(crate) const MAX_RPC_SIZE_POST_MERGE: usize = 10 * 1_048_576; // 10M
//FIXME(sean) should these be the same?
pub(crate) const MAX_RPC_SIZE_POST_CAPELLA: usize = 10 * 1_048_576; // 10M pub(crate) const MAX_RPC_SIZE_POST_CAPELLA: usize = 10 * 1_048_576; // 10M
// FIXME(sean) should this be increased to account for blobs?
pub(crate) const MAX_RPC_SIZE_POST_EIP4844: usize = 10 * 1_048_576; // 10M pub(crate) const MAX_RPC_SIZE_POST_EIP4844: usize = 10 * 1_048_576; // 10M
/// The protocol prefix the RPC protocol id. /// The protocol prefix the RPC protocol id.
const PROTOCOL_PREFIX: &str = "/eth2/beacon_chain/req"; const PROTOCOL_PREFIX: &str = "/eth2/beacon_chain/req";

View File

@ -115,7 +115,8 @@ const MAX_AGGREGATED_ATTESTATION_REPROCESS_QUEUE_LEN: usize = 1_024;
/// before we start dropping them. /// before we start dropping them.
const MAX_GOSSIP_BLOCK_QUEUE_LEN: usize = 1_024; const MAX_GOSSIP_BLOCK_QUEUE_LEN: usize = 1_024;
//FIXME(sean) verify /// The maximum number of queued `SignedBeaconBlockAndBlobsSidecar` objects received on gossip that
/// will be stored before we start dropping them.
const MAX_GOSSIP_BLOCK_AND_BLOB_QUEUE_LEN: usize = 1_024; const MAX_GOSSIP_BLOCK_AND_BLOB_QUEUE_LEN: usize = 1_024;
/// The maximum number of queued `SignedBeaconBlock` objects received prior to their slot (but /// The maximum number of queued `SignedBeaconBlock` objects received prior to their slot (but
@ -1186,7 +1187,6 @@ impl<T: BeaconChainTypes> BeaconProcessor<T> {
// required to verify some attestations. // required to verify some attestations.
} else if let Some(item) = gossip_block_queue.pop() { } else if let Some(item) = gossip_block_queue.pop() {
self.spawn_worker(item, toolbox); self.spawn_worker(item, toolbox);
//FIXME(sean)
} else if let Some(item) = gossip_block_and_blobs_sidecar_queue.pop() { } else if let Some(item) = gossip_block_and_blobs_sidecar_queue.pop() {
self.spawn_worker(item, toolbox); self.spawn_worker(item, toolbox);
// Check the aggregates, *then* the unaggregates since we assume that // Check the aggregates, *then* the unaggregates since we assume that
@ -1675,23 +1675,9 @@ impl<T: BeaconChainTypes> BeaconProcessor<T> {
/* /*
* Verification for blobs sidecars received on gossip. * Verification for blobs sidecars received on gossip.
*/ */
Work::GossipBlockAndBlobsSidecar { Work::GossipBlockAndBlobsSidecar { .. } => {
message_id, warn!(self.log, "Unexpected block and blobs on gossip")
peer_id, }
peer_client,
block_and_blobs,
seen_timestamp,
} => task_spawner.spawn_async(async move {
worker
.process_gossip_block_and_blobs_sidecar(
message_id,
peer_id,
peer_client,
block_and_blobs,
seen_timestamp,
)
.await
}),
/* /*
* Import for blocks that we received earlier than their intended slot. * Import for blocks that we received earlier than their intended slot.
*/ */
@ -1892,19 +1878,9 @@ impl<T: BeaconChainTypes> BeaconProcessor<T> {
request, request,
) )
}), }),
Work::BlobsByRangeRequest { Work::BlobsByRangeRequest { .. } => {
peer_id, warn!(self.log.clone(), "Unexpected BlobsByRange Request")
request_id, }
request,
} => task_spawner.spawn_blocking_with_manual_send_idle(move |send_idle_on_drop| {
worker.handle_blobs_by_range_request(
sub_executor,
send_idle_on_drop,
peer_id,
request_id,
request,
)
}),
/* /*
* Processing of lightclient bootstrap requests from other peers. * Processing of lightclient bootstrap requests from other peers.
*/ */

View File

@ -11,10 +11,7 @@ use beacon_chain::{
BeaconChainError, BeaconChainTypes, BlockError, CountUnrealized, ForkChoiceError, BeaconChainError, BeaconChainTypes, BlockError, CountUnrealized, ForkChoiceError,
GossipVerifiedBlock, NotifyExecutionLayer, GossipVerifiedBlock, NotifyExecutionLayer,
}; };
use lighthouse_network::{ use lighthouse_network::{Client, MessageAcceptance, MessageId, PeerAction, PeerId, ReportSource};
Client, MessageAcceptance, MessageId, PeerAction, PeerId, ReportSource,
SignedBeaconBlockAndBlobsSidecar,
};
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;
@ -699,19 +696,6 @@ impl<T: BeaconChainTypes> Worker<T> {
} }
} }
#[allow(clippy::too_many_arguments)]
pub async fn process_gossip_block_and_blobs_sidecar(
self,
_message_id: MessageId,
_peer_id: PeerId,
_peer_client: Client,
_block_and_blob: Arc<SignedBeaconBlockAndBlobsSidecar<T::EthSpec>>,
_seen_timestamp: Duration,
) {
//FIXME
unimplemented!()
}
/// Process the beacon block received from the gossip network and /// Process the beacon block received from the gossip network and
/// if it passes gossip propagation criteria, tell the network thread to forward it. /// if it passes gossip propagation criteria, tell the network thread to forward it.
/// ///

View File

@ -4,7 +4,6 @@ use crate::status::ToStatusMessage;
use crate::sync::SyncMessage; use crate::sync::SyncMessage;
use beacon_chain::{BeaconChainError, BeaconChainTypes, HistoricalBlockError, WhenSlotSkipped}; use beacon_chain::{BeaconChainError, BeaconChainTypes, HistoricalBlockError, WhenSlotSkipped};
use itertools::process_results; use itertools::process_results;
use lighthouse_network::rpc::methods::{BlobsByRangeRequest, MAX_REQUEST_BLOBS_SIDECARS};
use lighthouse_network::rpc::StatusMessage; use lighthouse_network::rpc::StatusMessage;
use lighthouse_network::rpc::*; use lighthouse_network::rpc::*;
use lighthouse_network::{PeerId, PeerRequestId, ReportSource, Response, SyncInfo}; use lighthouse_network::{PeerId, PeerRequestId, ReportSource, Response, SyncInfo};
@ -455,152 +454,4 @@ impl<T: BeaconChainTypes> Worker<T> {
"load_blocks_by_range_blocks", "load_blocks_by_range_blocks",
); );
} }
/// Handle a `BlobsByRange` request from the peer.
pub fn handle_blobs_by_range_request(
self,
_executor: TaskExecutor,
_send_on_drop: SendOnDrop,
peer_id: PeerId,
_request_id: PeerRequestId,
mut req: BlobsByRangeRequest,
) {
debug!(self.log, "Received BlobsByRange Request";
"peer_id" => %peer_id,
"count" => req.count,
"start_slot" => req.start_slot,
);
// Should not send more than max request blocks
if req.count > MAX_REQUEST_BLOBS_SIDECARS {
req.count = MAX_REQUEST_BLOBS_SIDECARS;
}
//FIXME(sean) create the blobs iter
// let forwards_block_root_iter = match self
// .chain
// .forwards_iter_block_roots(Slot::from(req.start_slot))
// {
// Ok(iter) => iter,
// Err(BeaconChainError::HistoricalBlockError(
// HistoricalBlockError::BlockOutOfRange {
// slot,
// oldest_block_slot,
// },
// )) => {
// debug!(self.log, "Range request failed during backfill"; "requested_slot" => slot, "oldest_known_slot" => oldest_block_slot);
// return self.send_error_response(
// peer_id,
// RPCResponseErrorCode::ResourceUnavailable,
// "Backfilling".into(),
// request_id,
// );
// }
// Err(e) => return error!(self.log, "Unable to obtain root iter"; "error" => ?e),
// };
//
// // Pick out the required blocks, ignoring skip-slots.
// let mut last_block_root = None;
// let maybe_block_roots = process_results(forwards_block_root_iter, |iter| {
// iter.take_while(|(_, slot)| slot.as_u64() < req.start_slot.saturating_add(req.count))
// // map skip slots to None
// .map(|(root, _)| {
// let result = if Some(root) == last_block_root {
// None
// } else {
// Some(root)
// };
// last_block_root = Some(root);
// result
// })
// .collect::<Vec<Option<Hash256>>>()
// });
//
// let block_roots = match maybe_block_roots {
// Ok(block_roots) => block_roots,
// Err(e) => return error!(self.log, "Error during iteration over blocks"; "error" => ?e),
// };
//
// // remove all skip slots
// let block_roots = block_roots.into_iter().flatten().collect::<Vec<_>>();
//
// // Fetching blocks is async because it may have to hit the execution layer for payloads.
// executor.spawn(
// async move {
// let mut blocks_sent = 0;
// let mut send_response = true;
//
// for root in block_roots {
// match self.chain.store.get_blobs(&root) {
// Ok(Some(blob)) => {
// blocks_sent += 1;
// self.send_network_message(NetworkMessage::SendResponse {
// peer_id,
// response: Response::BlobsByRange(Some(Arc::new(VariableList::new(vec![blob.message]).unwrap()))),
// id: request_id,
// });
// }
// Ok(None) => {
// error!(
// self.log,
// "Blob in the chain is not in the store";
// "request_root" => ?root
// );
// break;
// }
// Err(e) => {
// error!(
// self.log,
// "Error fetching block for peer";
// "block_root" => ?root,
// "error" => ?e
// );
// break;
// }
// }
// }
//
// let current_slot = self
// .chain
// .slot()
// .unwrap_or_else(|_| self.chain.slot_clock.genesis_slot());
//
// if blocks_sent < (req.count as usize) {
// debug!(
// self.log,
// "BlocksByRange Response processed";
// "peer" => %peer_id,
// "msg" => "Failed to return all requested blocks",
// "start_slot" => req.start_slot,
// "current_slot" => current_slot,
// "requested" => req.count,
// "returned" => blocks_sent
// );
// } else {
// debug!(
// self.log,
// "BlocksByRange Response processed";
// "peer" => %peer_id,
// "start_slot" => req.start_slot,
// "current_slot" => current_slot,
// "requested" => req.count,
// "returned" => blocks_sent
// );
// }
//
// if send_response {
// // send the stream terminator
// self.send_network_message(NetworkMessage::SendResponse {
// peer_id,
// response: Response::BlobsByRange(None),
// id: request_id,
// });
// }
//
// drop(send_on_drop);
// },
// "load_blocks_by_range_blocks",
// );
}
} }

View File

@ -47,7 +47,7 @@ use lighthouse_network::rpc::methods::MAX_REQUEST_BLOCKS;
use lighthouse_network::types::{NetworkGlobals, SyncState}; use lighthouse_network::types::{NetworkGlobals, SyncState};
use lighthouse_network::SyncInfo; use lighthouse_network::SyncInfo;
use lighthouse_network::{PeerAction, PeerId}; use lighthouse_network::{PeerAction, PeerId};
use slog::{crit, debug, error, info, trace, Logger}; use slog::{crit, debug, error, info, trace, warn, Logger};
use std::boxed::Box; use std::boxed::Box;
use std::ops::Sub; use std::ops::Sub;
use std::sync::Arc; use std::sync::Arc;
@ -592,8 +592,9 @@ impl<T: BeaconChainTypes> SyncManager<T> {
.block_lookups .block_lookups
.parent_chain_processed(chain_hash, result, &mut self.network), .parent_chain_processed(chain_hash, result, &mut self.network),
}, },
//FIXME(sean) SyncMessage::RpcBlob { .. } => {
SyncMessage::RpcBlob { .. } => todo!(), warn!(self.log, "Unexpected blob message received");
}
} }
} }

View File

@ -628,27 +628,6 @@ impl BeaconNodeHttpClient {
Ok(()) Ok(())
} }
/// `POST beacon/blobs`
///
/// Returns `Ok(None)` on a 404 error.
pub async fn post_beacon_blobs<T: EthSpec>(
&self,
block: &BlobsSidecar<T>,
) -> Result<(), Error> {
let mut path = self.eth_path(V1)?;
path.path_segments_mut()
.map_err(|()| Error::InvalidUrl(self.server.clone()))?
.push("beacon")
.push("blobs");
//FIXME(sean) should we re-use the proposal timeout? seems reasonable to..
self.post_with_timeout(path, block, self.timeouts.proposal)
.await?;
Ok(())
}
/// `POST beacon/blinded_blocks` /// `POST beacon/blinded_blocks`
/// ///
/// Returns `Ok(None)` on a 404 error. /// Returns `Ok(None)` on a 404 error.

View File

@ -247,11 +247,6 @@ pub trait EthSpec: 'static + Default + Sync + Send + Clone + Debug + PartialEq +
fn max_blobs_per_block() -> usize { fn max_blobs_per_block() -> usize {
Self::MaxBlobsPerBlock::to_usize() Self::MaxBlobsPerBlock::to_usize()
} }
/// FIXME: why is this called chunks_per_blob??
fn chunks_per_blob() -> usize {
Self::FieldElementsPerBlob::to_usize()
}
} }
/// Macro to inherit some type values from another EthSpec. /// Macro to inherit some type values from another EthSpec.

View File

@ -92,7 +92,7 @@ pub trait AbstractExecPayload<T: EthSpec>:
+ From<ExecutionPayloadEip4844<T>> + From<ExecutionPayloadEip4844<T>>
+ TryFrom<ExecutionPayloadHeaderEip4844<T>>; + TryFrom<ExecutionPayloadHeaderEip4844<T>>;
fn default_at_fork(fork_name: ForkName) -> Self; fn default_at_fork(fork_name: ForkName) -> Result<Self, Error>;
} }
#[superstruct( #[superstruct(
@ -372,13 +372,12 @@ impl<T: EthSpec> AbstractExecPayload<T> for FullPayload<T> {
type Capella = FullPayloadCapella<T>; type Capella = FullPayloadCapella<T>;
type Eip4844 = FullPayloadEip4844<T>; type Eip4844 = FullPayloadEip4844<T>;
fn default_at_fork(fork_name: ForkName) -> Self { fn default_at_fork(fork_name: ForkName) -> Result<Self, Error> {
match fork_name { match fork_name {
//FIXME(sean) error handling ForkName::Base | ForkName::Altair => Err(Error::IncorrectStateVariant),
ForkName::Base | ForkName::Altair => panic!(), ForkName::Merge => Ok(FullPayloadMerge::default().into()),
ForkName::Merge => FullPayloadMerge::default().into(), ForkName::Capella => Ok(FullPayloadCapella::default().into()),
ForkName::Capella => FullPayloadCapella::default().into(), ForkName::Eip4844 => Ok(FullPayloadEip4844::default().into()),
ForkName::Eip4844 => FullPayloadEip4844::default().into(),
} }
} }
} }
@ -882,13 +881,12 @@ impl<T: EthSpec> AbstractExecPayload<T> for BlindedPayload<T> {
type Capella = BlindedPayloadCapella<T>; type Capella = BlindedPayloadCapella<T>;
type Eip4844 = BlindedPayloadEip4844<T>; type Eip4844 = BlindedPayloadEip4844<T>;
fn default_at_fork(fork_name: ForkName) -> Self { fn default_at_fork(fork_name: ForkName) -> Result<Self, Error> {
match fork_name { match fork_name {
//FIXME(sean) error handling ForkName::Base | ForkName::Altair => Err(Error::IncorrectStateVariant),
ForkName::Base | ForkName::Altair => panic!(), ForkName::Merge => Ok(BlindedPayloadMerge::default().into()),
ForkName::Merge => BlindedPayloadMerge::default().into(), ForkName::Capella => Ok(BlindedPayloadCapella::default().into()),
ForkName::Capella => BlindedPayloadCapella::default().into(), ForkName::Eip4844 => Ok(BlindedPayloadEip4844::default().into()),
ForkName::Eip4844 => BlindedPayloadEip4844::default().into(),
} }
} }
} }

View File

@ -4,7 +4,10 @@ use ssz::Encode;
use std::fs::File; use std::fs::File;
use std::io::Write; use std::io::Write;
use std::time::{SystemTime, UNIX_EPOCH}; use std::time::{SystemTime, UNIX_EPOCH};
use types::{EthSpec, ExecutionPayloadHeader, ExecutionPayloadHeaderMerge}; use types::{
EthSpec, ExecutionPayloadHeader, ExecutionPayloadHeaderCapella, ExecutionPayloadHeaderEip4844,
ExecutionPayloadHeaderMerge, ForkName,
};
pub fn run<T: EthSpec>(matches: &ArgMatches) -> Result<(), String> { pub fn run<T: EthSpec>(matches: &ArgMatches) -> Result<(), String> {
let eth1_block_hash = parse_required(matches, "execution-block-hash")?; let eth1_block_hash = parse_required(matches, "execution-block-hash")?;
@ -17,17 +20,36 @@ pub fn run<T: EthSpec>(matches: &ArgMatches) -> Result<(), String> {
let base_fee_per_gas = parse_required(matches, "base-fee-per-gas")?; let base_fee_per_gas = parse_required(matches, "base-fee-per-gas")?;
let gas_limit = parse_required(matches, "gas-limit")?; let gas_limit = parse_required(matches, "gas-limit")?;
let file_name = matches.value_of("file").ok_or("No file supplied")?; let file_name = matches.value_of("file").ok_or("No file supplied")?;
let fork_name: ForkName = parse_optional(matches, "fork")?.unwrap_or(ForkName::Merge);
//FIXME(sean) let execution_payload_header: ExecutionPayloadHeader<T> = match fork_name {
let execution_payload_header: ExecutionPayloadHeader<T> = ForkName::Base | ForkName::Altair => return Err("invalid fork name".to_string()),
ExecutionPayloadHeader::Merge(ExecutionPayloadHeaderMerge { ForkName::Merge => ExecutionPayloadHeader::Merge(ExecutionPayloadHeaderMerge {
gas_limit, gas_limit,
base_fee_per_gas, base_fee_per_gas,
timestamp: genesis_time, timestamp: genesis_time,
block_hash: eth1_block_hash, block_hash: eth1_block_hash,
prev_randao: eth1_block_hash.into_root(), prev_randao: eth1_block_hash.into_root(),
..ExecutionPayloadHeaderMerge::default() ..ExecutionPayloadHeaderMerge::default()
}); }),
ForkName::Capella => ExecutionPayloadHeader::Capella(ExecutionPayloadHeaderCapella {
gas_limit,
base_fee_per_gas,
timestamp: genesis_time,
block_hash: eth1_block_hash,
prev_randao: eth1_block_hash.into_root(),
..ExecutionPayloadHeaderCapella::default()
}),
ForkName::Eip4844 => ExecutionPayloadHeader::Eip4844(ExecutionPayloadHeaderEip4844 {
gas_limit,
base_fee_per_gas,
timestamp: genesis_time,
block_hash: eth1_block_hash,
prev_randao: eth1_block_hash.into_root(),
..ExecutionPayloadHeaderEip4844::default()
}),
};
let mut file = File::create(file_name).map_err(|_| "Unable to create file".to_string())?; let mut file = File::create(file_name).map_err(|_| "Unable to create file".to_string())?;
let bytes = execution_payload_header.as_ssz_bytes(); let bytes = execution_payload_header.as_ssz_bytes();
file.write_all(bytes.as_slice()) file.write_all(bytes.as_slice())

View File

@ -371,7 +371,8 @@ fn main() {
.subcommand( .subcommand(
SubCommand::with_name("create-payload-header") SubCommand::with_name("create-payload-header")
.about("Generates an SSZ file containing bytes for an `ExecutionPayloadHeader`. \ .about("Generates an SSZ file containing bytes for an `ExecutionPayloadHeader`. \
Useful as input for `lcli new-testnet --execution-payload-header FILE`. ") Useful as input for `lcli new-testnet --execution-payload-header FILE`. If `--fork` \
is not provided, a payload header for the `Bellatrix` fork will be created.")
.arg( .arg(
Arg::with_name("execution-block-hash") Arg::with_name("execution-block-hash")
.long("execution-block-hash") .long("execution-block-hash")
@ -417,6 +418,14 @@ fn main() {
.takes_value(true) .takes_value(true)
.required(true) .required(true)
.help("Output file"), .help("Output file"),
).arg(
Arg::with_name("fork")
.long("fork")
.value_name("FORK")
.takes_value(true)
.default_value("bellatrix")
.help("The fork for which the execution payload header should be created.")
.possible_values(&["merge", "bellatrix", "capella", "eip4844"])
) )
) )
.subcommand( .subcommand(

View File

@ -9,8 +9,9 @@ use std::io::Read;
use std::path::PathBuf; use std::path::PathBuf;
use std::time::{SystemTime, UNIX_EPOCH}; use std::time::{SystemTime, UNIX_EPOCH};
use types::{ use types::{
test_utils::generate_deterministic_keypairs, Address, Config, EthSpec, ExecutionPayloadHeader, test_utils::generate_deterministic_keypairs, Address, Config, Epoch, EthSpec,
ExecutionPayloadHeaderMerge, ExecutionPayloadHeader, ExecutionPayloadHeaderCapella, ExecutionPayloadHeaderEip4844,
ExecutionPayloadHeaderMerge, ForkName,
}; };
pub fn run<T: EthSpec>(testnet_dir_path: PathBuf, matches: &ArgMatches) -> Result<(), String> { pub fn run<T: EthSpec>(testnet_dir_path: PathBuf, matches: &ArgMatches) -> Result<(), String> {
@ -80,9 +81,24 @@ pub fn run<T: EthSpec>(testnet_dir_path: PathBuf, matches: &ArgMatches) -> Resul
.map_err(|e| format!("Unable to open {}: {}", filename, e))?; .map_err(|e| format!("Unable to open {}: {}", filename, e))?;
file.read_to_end(&mut bytes) file.read_to_end(&mut bytes)
.map_err(|e| format!("Unable to read {}: {}", filename, e))?; .map_err(|e| format!("Unable to read {}: {}", filename, e))?;
//FIXME(sean) let fork_name = spec.fork_name_at_epoch(Epoch::new(0));
match fork_name {
ForkName::Base | ForkName::Altair => Err(ssz::DecodeError::BytesInvalid(
"genesis fork must be post-merge".to_string(),
)),
ForkName::Merge => {
ExecutionPayloadHeaderMerge::<T>::from_ssz_bytes(bytes.as_slice()) ExecutionPayloadHeaderMerge::<T>::from_ssz_bytes(bytes.as_slice())
.map(ExecutionPayloadHeader::Merge) .map(ExecutionPayloadHeader::Merge)
}
ForkName::Capella => {
ExecutionPayloadHeaderCapella::<T>::from_ssz_bytes(bytes.as_slice())
.map(ExecutionPayloadHeader::Capella)
}
ForkName::Eip4844 => {
ExecutionPayloadHeaderEip4844::<T>::from_ssz_bytes(bytes.as_slice())
.map(ExecutionPayloadHeader::Eip4844)
}
}
.map_err(|e| format!("SSZ decode failed: {:?}", e)) .map_err(|e| format!("SSZ decode failed: {:?}", e))
}) })
.transpose()?; .transpose()?;

View File

@ -37,7 +37,6 @@ pub enum Error {
pub enum SignableMessage<'a, T: EthSpec, Payload: AbstractExecPayload<T> = FullPayload<T>> { pub enum SignableMessage<'a, T: EthSpec, Payload: AbstractExecPayload<T> = FullPayload<T>> {
RandaoReveal(Epoch), RandaoReveal(Epoch),
BeaconBlock(&'a BeaconBlock<T, Payload>), BeaconBlock(&'a BeaconBlock<T, Payload>),
BlobsSidecar(&'a BlobsSidecar<T>),
AttestationData(&'a AttestationData), AttestationData(&'a AttestationData),
SignedAggregateAndProof(&'a AggregateAndProof<T>), SignedAggregateAndProof(&'a AggregateAndProof<T>),
SelectionProof(Slot), SelectionProof(Slot),
@ -59,7 +58,6 @@ impl<'a, T: EthSpec, Payload: AbstractExecPayload<T>> SignableMessage<'a, T, Pay
match self { match self {
SignableMessage::RandaoReveal(epoch) => epoch.signing_root(domain), SignableMessage::RandaoReveal(epoch) => epoch.signing_root(domain),
SignableMessage::BeaconBlock(b) => b.signing_root(domain), SignableMessage::BeaconBlock(b) => b.signing_root(domain),
SignableMessage::BlobsSidecar(b) => b.signing_root(domain),
SignableMessage::AttestationData(a) => a.signing_root(domain), SignableMessage::AttestationData(a) => a.signing_root(domain),
SignableMessage::SignedAggregateAndProof(a) => a.signing_root(domain), SignableMessage::SignedAggregateAndProof(a) => a.signing_root(domain),
SignableMessage::SelectionProof(slot) => slot.signing_root(domain), SignableMessage::SelectionProof(slot) => slot.signing_root(domain),
@ -182,7 +180,6 @@ impl SigningMethod {
Web3SignerObject::RandaoReveal { epoch } Web3SignerObject::RandaoReveal { epoch }
} }
SignableMessage::BeaconBlock(block) => Web3SignerObject::beacon_block(block)?, SignableMessage::BeaconBlock(block) => Web3SignerObject::beacon_block(block)?,
SignableMessage::BlobsSidecar(blob) => Web3SignerObject::BlobsSidecar(blob),
SignableMessage::AttestationData(a) => Web3SignerObject::Attestation(a), SignableMessage::AttestationData(a) => Web3SignerObject::Attestation(a),
SignableMessage::SignedAggregateAndProof(a) => { SignableMessage::SignedAggregateAndProof(a) => {
Web3SignerObject::AggregateAndProof(a) Web3SignerObject::AggregateAndProof(a)

View File

@ -11,7 +11,6 @@ pub enum MessageType {
AggregateAndProof, AggregateAndProof,
Attestation, Attestation,
BlockV2, BlockV2,
BlobsSidecar,
Deposit, Deposit,
RandaoReveal, RandaoReveal,
VoluntaryExit, VoluntaryExit,
@ -52,8 +51,6 @@ pub enum Web3SignerObject<'a, T: EthSpec, Payload: AbstractExecPayload<T>> {
#[serde(skip_serializing_if = "Option::is_none")] #[serde(skip_serializing_if = "Option::is_none")]
block_header: Option<BeaconBlockHeader>, block_header: Option<BeaconBlockHeader>,
}, },
//FIXME(sean) just guessing here
BlobsSidecar(&'a BlobsSidecar<T>),
#[allow(dead_code)] #[allow(dead_code)]
Deposit { Deposit {
pubkey: PublicKeyBytes, pubkey: PublicKeyBytes,
@ -114,7 +111,6 @@ impl<'a, T: EthSpec, Payload: AbstractExecPayload<T>> Web3SignerObject<'a, T, Pa
Web3SignerObject::AggregateAndProof(_) => MessageType::AggregateAndProof, Web3SignerObject::AggregateAndProof(_) => MessageType::AggregateAndProof,
Web3SignerObject::Attestation(_) => MessageType::Attestation, Web3SignerObject::Attestation(_) => MessageType::Attestation,
Web3SignerObject::BeaconBlock { .. } => MessageType::BlockV2, Web3SignerObject::BeaconBlock { .. } => MessageType::BlockV2,
Web3SignerObject::BlobsSidecar(_) => MessageType::BlobsSidecar,
Web3SignerObject::Deposit { .. } => MessageType::Deposit, Web3SignerObject::Deposit { .. } => MessageType::Deposit,
Web3SignerObject::RandaoReveal { .. } => MessageType::RandaoReveal, Web3SignerObject::RandaoReveal { .. } => MessageType::RandaoReveal,
Web3SignerObject::VoluntaryExit(_) => MessageType::VoluntaryExit, Web3SignerObject::VoluntaryExit(_) => MessageType::VoluntaryExit,