* some blob reprocessing work

* remove ForceBlockLookup

* reorder enum match arms in sync manager

* a lot more reprocessing work

* impl logic for triggerng blob lookups along with block lookups

* deal with rpc blobs in groups per block in the da checker. don't cache missing blob ids in the da checker.

* make single block lookup generic

* more work

* add delayed processing logic and combine some requests

* start fixing some compile errors

* fix compilation in main block lookup mod

* much work

* get things compiling

* parent blob lookups

* fix compile

* revert red/stevie changes

* fix up sync manager delay message logic

* add peer usefulness enum

* should remove lookup refactor

* consolidate retry error handling

* improve peer scoring during certain failures in parent lookups

* improve retry code

* drop parent lookup if either req has a peer disconnect during download

* refactor single block processed method

* processing peer refactor

* smol bugfix

* fix some todos

* fix lints

* fix lints

* fix compile in lookup tests

* fix lints

* fix lints

* fix existing block lookup tests

* renamings

* fix after merge

* cargo fmt

* compilation fix in beacon chain tests

* fix

* refactor lookup tests to work with multiple forks and response types

* make tests into macros

* wrap availability check error

* fix compile after merge

* add random blobs

* start fixing up lookup verify error handling

* some bug fixes and the start of deneb only tests

* make tests work for all forks

* track information about peer source

* error refactoring

* improve peer scoring

* fix test compilation

* make sure blobs are sent for processing after stream termination, delete copied tests

* add some tests and fix a bug

* smol bugfixes and moar tests

* add tests and fix some things

* compile after merge

* lots of refactoring

* retry on invalid block/blob

* merge unknown parent messages before current slot lookup

* get tests compiling

* penalize blob peer on invalid blobs

* Check disk on in-memory cache miss

* Update beacon_node/beacon_chain/src/data_availability_checker/overflow_lru_cache.rs

* Update beacon_node/network/src/sync/network_context.rs

Co-authored-by: Divma <26765164+divagant-martian@users.noreply.github.com>

* fix bug in matching blocks and blobs in range sync

* pr feedback

* fix conflicts

* upgrade logs from warn to crit when we receive incorrect response in range

* synced_and_connected_within_tolerance -> should_search_for_block

* remove todo

* add data gas used and update excess data gas to u64

* Fix Broken Overflow Tests

* payload verification with commitments

* fix merge conflicts

* restore payload file

* Restore payload file

* remove todo

* add max blob commitments per block

* c-kzg lib update

* Fix ef tests

* Abstract over minimal/mainnet spec in kzg crate

* Start integrating new KZG

* checkpoint sync without alignment

* checkpoint sync without alignment

* add import

* add import

* query for checkpoint state by slot rather than state root (teku doesn't serve by state root)

* query for checkpoint state by slot rather than state root (teku doesn't serve by state root)

* loosen check

* get state first and query by most recent block root

* Revert "loosen check"

This reverts commit 069d13dd63aa794a3505db9f17bd1a6b73f0be81.

* get state first and query by most recent block root

* merge max blobs change

* simplify delay logic

* rename unknown parent sync message variants

* rename parameter, block_slot -> slot

* add some docs to the lookup module

* use interval instead of sleep

* drop request if blocks and blobs requests both return `None` for `Id`

* clean up `find_single_lookup` logic

* add lookup source enum

* clean up `find_single_lookup` logic

* add docs to find_single_lookup_request

* move LookupSource our of param where unnecessary

* remove unnecessary todo

* query for block by `state.latest_block_header.slot`

* fix lint

* fix merge transition ef tests

* fix test

* fix test

* fix observed  blob sidecars test

* Add some metrics (#33)

* fix protocol limits for blobs by root

* Update Engine API for 1:1 Structure Method

* make beacon chain tests to fix devnet 6 changes

* get ckzg working and fix some tests

* fix remaining tests

* fix lints

* Fix KZG linking issues

* remove unused dep

* lockfile

* test fixes

* remove dbgs

* remove unwrap

* cleanup tx generator

* small fixes

* fixing fixes

* more self reivew

* more self review

* refactor genesis header initialization

* refactor mock el instantiations

* fix compile

* fix network test, make sure they run for each fork

* pr feedback

* fix last test (hopefully)

---------

Co-authored-by: Pawan Dhananjay <pawandhananjay@gmail.com>
Co-authored-by: Mark Mackey <mark@sigmaprime.io>
Co-authored-by: Divma <26765164+divagant-martian@users.noreply.github.com>
Co-authored-by: Michael Sproul <michael@sigmaprime.io>
This commit is contained in:
realbigsean 2023-06-29 15:35:43 -04:00 committed by GitHub
parent 4c9fcf1e83
commit adbb62f7f3
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
69 changed files with 2114 additions and 1338 deletions

View File

@ -374,14 +374,15 @@ jobs:
run: rustup update stable run: rustup update stable
- name: Run cargo audit to identify known security vulnerabilities reported to the RustSec Advisory Database - name: Run cargo audit to identify known security vulnerabilities reported to the RustSec Advisory Database
run: make audit run: make audit
cargo-vendor: # TODO(sean): re-enable this when we can figure it out with c-kzg
name: cargo-vendor # cargo-vendor:
runs-on: ubuntu-latest # name: cargo-vendor
needs: cargo-fmt # runs-on: ubuntu-latest
steps: # needs: cargo-fmt
- uses: actions/checkout@v3 # steps:
- name: Run cargo vendor to make sure dependencies can be vendored for packaging, reproducibility and archival purpose # - uses: actions/checkout@v3
run: CARGO_HOME=$(readlink -f $HOME) make vendor # - name: Run cargo vendor to make sure dependencies can be vendored for packaging, reproducibility and archival purpose
# run: CARGO_HOME=$(readlink -f $HOME) make vendor
cargo-udeps: cargo-udeps:
name: cargo-udeps name: cargo-udeps
runs-on: ubuntu-latest runs-on: ubuntu-latest

786
Cargo.lock generated

File diff suppressed because it is too large Load Diff

View File

@ -114,9 +114,8 @@ use store::{
use task_executor::{ShutdownReason, TaskExecutor}; use task_executor::{ShutdownReason, TaskExecutor};
use tokio_stream::Stream; use tokio_stream::Stream;
use tree_hash::TreeHash; use tree_hash::TreeHash;
use types::beacon_block_body::KzgCommitments;
use types::beacon_state::CloneConfig; use types::beacon_state::CloneConfig;
use types::blob_sidecar::{BlobSidecarList, Blobs}; use types::blob_sidecar::BlobSidecarList;
use types::consts::deneb::MIN_EPOCHS_FOR_BLOB_SIDECARS_REQUESTS; use types::consts::deneb::MIN_EPOCHS_FOR_BLOB_SIDECARS_REQUESTS;
use types::*; use types::*;
@ -466,7 +465,7 @@ pub struct BeaconChain<T: BeaconChainTypes> {
pub genesis_backfill_slot: Slot, pub genesis_backfill_slot: Slot,
pub proposal_blob_cache: BlobCache<T::EthSpec>, pub proposal_blob_cache: BlobCache<T::EthSpec>,
pub data_availability_checker: Arc<DataAvailabilityChecker<T>>, pub data_availability_checker: Arc<DataAvailabilityChecker<T>>,
pub kzg: Option<Arc<Kzg>>, pub kzg: Option<Arc<Kzg<<T::EthSpec as EthSpec>::Kzg>>>,
} }
type BeaconBlockAndState<T, Payload> = (BeaconBlock<T, Payload>, BeaconState<T>); type BeaconBlockAndState<T, Payload> = (BeaconBlock<T, Payload>, BeaconState<T>);
@ -2914,7 +2913,14 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
) -> Result<AvailabilityProcessingStatus, BlockError<T::EthSpec>> { ) -> Result<AvailabilityProcessingStatus, BlockError<T::EthSpec>> {
let availability = cache_fn(self.clone())?; let availability = cache_fn(self.clone())?;
match availability { match availability {
Availability::Available(block) => self.import_available_block(block).await, Availability::Available(block) => {
// This is the time since start of the slot where all the components of the block have become available
let delay =
get_slot_delay_ms(timestamp_now(), block.block.slot(), &self.slot_clock);
metrics::observe_duration(&metrics::BLOCK_AVAILABILITY_DELAY, delay);
// Block is fully available, import into fork choice
self.import_available_block(block).await
}
Availability::MissingComponents(block_root) => Ok( Availability::MissingComponents(block_root) => Ok(
AvailabilityProcessingStatus::MissingComponents(slot, block_root), AvailabilityProcessingStatus::MissingComponents(slot, block_root),
), ),
@ -4903,7 +4909,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
//FIXME(sean) //FIXME(sean)
// - add a new timer for processing here // - add a new timer for processing here
if let Some(blobs) = blobs_opt { if let (Some(blobs), Some(proofs)) = (blobs_opt, proofs_opt) {
let kzg = self let kzg = self
.kzg .kzg
.as_ref() .as_ref()
@ -4924,14 +4930,10 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
))); )));
} }
let kzg_proofs = if let Some(proofs) = proofs_opt { let kzg_proofs = Vec::from(proofs);
Vec::from(proofs)
} else {
Self::compute_blob_kzg_proofs(kzg, &blobs, expected_kzg_commitments, slot)?
};
kzg_utils::validate_blobs::<T::EthSpec>( kzg_utils::validate_blobs::<T::EthSpec>(
kzg, kzg.as_ref(),
expected_kzg_commitments, expected_kzg_commitments,
&blobs, &blobs,
&kzg_proofs, &kzg_proofs,
@ -4982,29 +4984,6 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
Ok((block, state)) Ok((block, state))
} }
fn compute_blob_kzg_proofs(
kzg: &Arc<Kzg>,
blobs: &Blobs<T::EthSpec>,
expected_kzg_commitments: &KzgCommitments<T::EthSpec>,
slot: Slot,
) -> Result<Vec<KzgProof>, BlockProductionError> {
blobs
.iter()
.enumerate()
.map(|(blob_index, blob)| {
let kzg_commitment = expected_kzg_commitments.get(blob_index).ok_or(
BlockProductionError::MissingKzgCommitment(format!(
"Missing KZG commitment for slot {} blob index {}",
slot, blob_index
)),
)?;
kzg_utils::compute_blob_kzg_proof::<T::EthSpec>(kzg, blob, *kzg_commitment)
.map_err(BlockProductionError::KzgError)
})
.collect::<Result<Vec<KzgProof>, BlockProductionError>>()
}
/// This method must be called whenever an execution engine indicates that a payload is /// This method must be called whenever an execution engine indicates that a payload is
/// invalid. /// invalid.
/// ///

View File

@ -4,8 +4,8 @@ use state_processing::state_advance::partial_state_advance;
use std::sync::Arc; use std::sync::Arc;
use crate::beacon_chain::{ use crate::beacon_chain::{
BeaconChain, BeaconChainTypes, MAXIMUM_GOSSIP_CLOCK_DISPARITY, BeaconChain, BeaconChainTypes, BLOCK_PROCESSING_CACHE_LOCK_TIMEOUT,
VALIDATOR_PUBKEY_CACHE_LOCK_TIMEOUT, MAXIMUM_GOSSIP_CLOCK_DISPARITY, VALIDATOR_PUBKEY_CACHE_LOCK_TIMEOUT,
}; };
use crate::data_availability_checker::{ use crate::data_availability_checker::{
AvailabilityCheckError, AvailabilityPendingBlock, AvailableBlock, AvailabilityCheckError, AvailabilityPendingBlock, AvailableBlock,
@ -20,9 +20,9 @@ use ssz_types::FixedVector;
use std::borrow::Cow; use std::borrow::Cow;
use types::blob_sidecar::{BlobIdentifier, FixedBlobSidecarList}; use types::blob_sidecar::{BlobIdentifier, FixedBlobSidecarList};
use types::{ use types::{
BeaconBlockRef, BeaconState, BeaconStateError, BlobSidecar, ChainSpec, CloneConfig, Epoch, BeaconBlockRef, BeaconState, BeaconStateError, BlobSidecar, BlobSidecarList, ChainSpec,
EthSpec, FullPayload, Hash256, KzgCommitment, RelativeEpoch, SignedBeaconBlock, CloneConfig, Epoch, EthSpec, FullPayload, Hash256, KzgCommitment, RelativeEpoch,
SignedBeaconBlockHeader, SignedBlobSidecar, Slot, SignedBeaconBlock, SignedBeaconBlockHeader, SignedBlobSidecar, Slot,
}; };
#[derive(Debug)] #[derive(Debug)]
@ -240,36 +240,72 @@ pub fn validate_blob_sidecar_for_gossip<T: BeaconChainTypes>(
"block_root" => %block_root, "block_root" => %block_root,
"index" => %blob_index, "index" => %blob_index,
); );
// The cached head state is in the same epoch as the blob or the state has already been if let Some(mut snapshot) = chain
// advanced to the blob's epoch .snapshot_cache
let snapshot = &chain.canonical_head.cached_head().snapshot; .try_read_for(BLOCK_PROCESSING_CACHE_LOCK_TIMEOUT)
if snapshot.beacon_state.current_epoch() == blob_slot.epoch(T::EthSpec::slots_per_epoch()) { .and_then(|snapshot_cache| {
snapshot_cache.get_cloned(block_parent_root, CloneConfig::committee_caches_only())
})
{
if snapshot.beacon_state.slot() == blob_slot {
debug!(
chain.log,
"Cloning snapshot cache state for blob verification";
"block_root" => %block_root,
"index" => %blob_index,
);
( (
snapshot snapshot
.beacon_state .beacon_state
.get_beacon_proposer_index(blob_slot, &chain.spec)?, .get_beacon_proposer_index(blob_slot, &chain.spec)?,
snapshot.beacon_state.fork(), snapshot.beacon_state.fork(),
) )
} else {
debug!(
chain.log,
"Cloning and advancing snapshot cache state for blob verification";
"block_root" => %block_root,
"index" => %blob_index,
);
let state = cheap_state_advance_to_obtain_committees(
&mut snapshot.beacon_state,
Some(snapshot.beacon_block_root),
blob_slot,
&chain.spec,
)?;
(
state.get_beacon_proposer_index(blob_slot, &chain.spec)?,
state.fork(),
)
}
} }
// Need to advance the state to get the proposer index // Need to advance the state to get the proposer index
else { else {
// Reaching this condition too often might be an issue since we could theoretically have
// 5 threads (4 blob indices + 1 block) cloning the state.
// We shouldn't be seeing this condition a lot because we try to advance the state
// 3 seconds before the start of a slot. However, if this becomes an issue during testing, we should
// consider sending a blob for reprocessing to reduce the number of state clones.
warn!( warn!(
chain.log, chain.log,
"Cached head not advanced for blob verification"; "Snapshot cache miss for blob verification";
"block_root" => %block_root, "block_root" => %block_root,
"index" => %blob_index, "index" => %blob_index,
"action" => "contact the devs if you see this msg too often"
); );
// The state produced is only valid for determining proposer/attester shuffling indices.
let mut cloned_state = snapshot.clone_with(CloneConfig::committee_caches_only()); let parent_block = chain
.get_blinded_block(&block_parent_root)
.map_err(BlobError::BeaconChainError)?
.ok_or_else(|| {
BlobError::from(BeaconChainError::MissingBeaconBlock(block_parent_root))
})?;
let mut parent_state = chain
.get_state(&parent_block.state_root(), Some(parent_block.slot()))?
.ok_or_else(|| {
BeaconChainError::DBInconsistent(format!(
"Missing state {:?}",
parent_block.state_root()
))
})?;
let state = cheap_state_advance_to_obtain_committees( let state = cheap_state_advance_to_obtain_committees(
&mut cloned_state.beacon_state, &mut parent_state,
None, Some(parent_block.state_root()),
blob_slot, blob_slot,
&chain.spec, &chain.spec,
)?; )?;
@ -449,8 +485,9 @@ impl<T: EthSpec> KzgVerifiedBlob<T> {
/// Returns an error if the kzg verification check fails. /// Returns an error if the kzg verification check fails.
pub fn verify_kzg_for_blob<T: EthSpec>( pub fn verify_kzg_for_blob<T: EthSpec>(
blob: Arc<BlobSidecar<T>>, blob: Arc<BlobSidecar<T>>,
kzg: &Kzg, kzg: &Kzg<T::Kzg>,
) -> Result<KzgVerifiedBlob<T>, AvailabilityCheckError> { ) -> Result<KzgVerifiedBlob<T>, AvailabilityCheckError> {
let _timer = crate::metrics::start_timer(&crate::metrics::KZG_VERIFICATION_SINGLE_TIMES);
//TODO(sean) remove clone //TODO(sean) remove clone
if validate_blob::<T>(kzg, blob.blob.clone(), blob.kzg_commitment, blob.kzg_proof) if validate_blob::<T>(kzg, blob.blob.clone(), blob.kzg_commitment, blob.kzg_proof)
.map_err(AvailabilityCheckError::Kzg)? .map_err(AvailabilityCheckError::Kzg)?
@ -468,8 +505,9 @@ pub fn verify_kzg_for_blob<T: EthSpec>(
/// in a loop since this function kzg verifies a list of blobs more efficiently. /// in a loop since this function kzg verifies a list of blobs more efficiently.
pub fn verify_kzg_for_blob_list<T: EthSpec>( pub fn verify_kzg_for_blob_list<T: EthSpec>(
blob_list: Vec<Arc<BlobSidecar<T>>>, blob_list: Vec<Arc<BlobSidecar<T>>>,
kzg: &Kzg, kzg: &Kzg<T::Kzg>,
) -> Result<KzgVerifiedBlobList<T>, AvailabilityCheckError> { ) -> Result<KzgVerifiedBlobList<T>, AvailabilityCheckError> {
let _timer = crate::metrics::start_timer(&crate::metrics::KZG_VERIFICATION_BATCH_TIMES);
let (blobs, (commitments, proofs)): (Vec<_>, (Vec<_>, Vec<_>)) = blob_list let (blobs, (commitments, proofs)): (Vec<_>, (Vec<_>, Vec<_>)) = blob_list
.clone() .clone()
.into_iter() .into_iter()
@ -612,6 +650,15 @@ pub enum BlockWrapper<E: EthSpec> {
} }
impl<E: EthSpec> BlockWrapper<E> { impl<E: EthSpec> BlockWrapper<E> {
pub fn new(block: Arc<SignedBeaconBlock<E>>, blobs: Option<BlobSidecarList<E>>) -> Self {
match blobs {
Some(blobs) => {
let blobs = FixedVector::from(blobs.into_iter().map(Some).collect::<Vec<_>>());
BlockWrapper::BlockAndBlobs(block, blobs)
}
None => BlockWrapper::Block(block),
}
}
pub fn deconstruct(self) -> (Arc<SignedBeaconBlock<E>>, Option<FixedBlobSidecarList<E>>) { pub fn deconstruct(self) -> (Arc<SignedBeaconBlock<E>>, Option<FixedBlobSidecarList<E>>) {
match self { match self {
BlockWrapper::Block(block) => (block, None), BlockWrapper::Block(block) => (block, None),

View File

@ -11,7 +11,6 @@ use kzg::Kzg;
use slog::{debug, error}; use slog::{debug, error};
use slot_clock::SlotClock; use slot_clock::SlotClock;
use ssz_types::{Error, FixedVector, VariableList}; use ssz_types::{Error, FixedVector, VariableList};
use state_processing::per_block_processing::deneb::deneb::verify_kzg_commitments_against_transactions;
use std::collections::HashSet; use std::collections::HashSet;
use std::sync::Arc; use std::sync::Arc;
use strum::IntoStaticStr; use strum::IntoStaticStr;
@ -21,7 +20,7 @@ use types::blob_sidecar::{BlobIdentifier, BlobSidecar, FixedBlobSidecarList};
use types::consts::deneb::MIN_EPOCHS_FOR_BLOB_SIDECARS_REQUESTS; use types::consts::deneb::MIN_EPOCHS_FOR_BLOB_SIDECARS_REQUESTS;
use types::ssz_tagged_signed_beacon_block; use types::ssz_tagged_signed_beacon_block;
use types::{ use types::{
BeaconBlockRef, BlobSidecarList, ChainSpec, Epoch, EthSpec, ExecPayload, FullPayload, Hash256, BeaconBlockRef, BlobSidecarList, ChainSpec, Epoch, EthSpec, FullPayload, Hash256,
SignedBeaconBlock, SignedBeaconBlockHeader, Slot, SignedBeaconBlock, SignedBeaconBlockHeader, Slot,
}; };
@ -85,7 +84,7 @@ impl From<ssz::DecodeError> for AvailabilityCheckError {
pub struct DataAvailabilityChecker<T: BeaconChainTypes> { pub struct DataAvailabilityChecker<T: BeaconChainTypes> {
availability_cache: Arc<OverflowLRUCache<T>>, availability_cache: Arc<OverflowLRUCache<T>>,
slot_clock: T::SlotClock, slot_clock: T::SlotClock,
kzg: Option<Arc<Kzg>>, kzg: Option<Arc<Kzg<<T::EthSpec as EthSpec>::Kzg>>>,
spec: ChainSpec, spec: ChainSpec,
} }
@ -114,7 +113,7 @@ impl<T: EthSpec> Availability<T> {
impl<T: BeaconChainTypes> DataAvailabilityChecker<T> { impl<T: BeaconChainTypes> DataAvailabilityChecker<T> {
pub fn new( pub fn new(
slot_clock: T::SlotClock, slot_clock: T::SlotClock,
kzg: Option<Arc<Kzg>>, kzg: Option<Arc<Kzg<<T::EthSpec as EthSpec>::Kzg>>>,
store: BeaconStore<T>, store: BeaconStore<T>,
spec: ChainSpec, spec: ChainSpec,
) -> Result<Self, AvailabilityCheckError> { ) -> Result<Self, AvailabilityCheckError> {
@ -296,23 +295,8 @@ impl<T: BeaconChainTypes> DataAvailabilityChecker<T> {
&self, &self,
block: &Arc<SignedBeaconBlock<T::EthSpec, FullPayload<T::EthSpec>>>, block: &Arc<SignedBeaconBlock<T::EthSpec, FullPayload<T::EthSpec>>>,
) -> Result<BlobRequirements, AvailabilityCheckError> { ) -> Result<BlobRequirements, AvailabilityCheckError> {
let verified_blobs = if let (Ok(block_kzg_commitments), Ok(payload)) = ( let verified_blobs =
block.message().body().blob_kzg_commitments(), if let Ok(block_kzg_commitments) = block.message().body().blob_kzg_commitments() {
block.message().body().execution_payload(),
) {
if let Some(transactions) = payload.transactions() {
let verified = verify_kzg_commitments_against_transactions::<T::EthSpec>(
transactions,
block_kzg_commitments,
)
.map_err(|e| AvailabilityCheckError::TxKzgCommitmentMismatch(format!("{e:?}")))?;
if !verified {
return Err(AvailabilityCheckError::TxKzgCommitmentMismatch(
"a commitment and version didn't match".to_string(),
));
}
}
if self.da_check_required(block.epoch()) { if self.da_check_required(block.epoch()) {
if block_kzg_commitments.is_empty() { if block_kzg_commitments.is_empty() {
BlobRequirements::EmptyBlobs BlobRequirements::EmptyBlobs

View File

@ -18,7 +18,7 @@ use proto_array::{Block as ProtoBlock, ExecutionStatus};
use slog::{debug, warn}; use slog::{debug, warn};
use slot_clock::SlotClock; use slot_clock::SlotClock;
use state_processing::per_block_processing::{ use state_processing::per_block_processing::{
compute_timestamp_at_slot, get_expected_withdrawals, is_execution_enabled, self, compute_timestamp_at_slot, get_expected_withdrawals, is_execution_enabled,
is_merge_transition_complete, partially_verify_execution_payload, is_merge_transition_complete, partially_verify_execution_payload,
}; };
use std::sync::Arc; use std::sync::Arc;
@ -68,15 +68,16 @@ impl<T: BeaconChainTypes> PayloadNotifier<T> {
// the block as optimistically imported. This is particularly relevant in the case // the block as optimistically imported. This is particularly relevant in the case
// where we do not send the block to the EL at all. // where we do not send the block to the EL at all.
let block_message = block.message(); let block_message = block.message();
let payload = block_message.execution_payload()?;
partially_verify_execution_payload::<_, FullPayload<_>>( partially_verify_execution_payload::<_, FullPayload<_>>(
state, state,
block.slot(), block.slot(),
payload, block_message.body(),
&chain.spec, &chain.spec,
) )
.map_err(BlockError::PerBlockProcessingError)?; .map_err(BlockError::PerBlockProcessingError)?;
let payload = block_message.execution_payload()?;
match notify_execution_layer { match notify_execution_layer {
NotifyExecutionLayer::No if chain.config.optimistic_finalized_sync => { NotifyExecutionLayer::No if chain.config.optimistic_finalized_sync => {
// Verify the block hash here in Lighthouse and immediately mark the block as // Verify the block hash here in Lighthouse and immediately mark the block as
@ -139,6 +140,14 @@ async fn notify_new_payload<'a, T: BeaconChainTypes>(
block: BeaconBlockRef<'a, T::EthSpec>, block: BeaconBlockRef<'a, T::EthSpec>,
) -> Result<PayloadVerificationStatus, BlockError<T::EthSpec>> { ) -> Result<PayloadVerificationStatus, BlockError<T::EthSpec>> {
let execution_payload = block.execution_payload()?; let execution_payload = block.execution_payload()?;
let versioned_hashes = block.body().blob_kzg_commitments().ok().map(|commitments| {
commitments
.into_iter()
.map(|commitment| {
per_block_processing::deneb::deneb::kzg_commitment_to_versioned_hash(commitment)
})
.collect::<Vec<_>>()
});
let execution_layer = chain let execution_layer = chain
.execution_layer .execution_layer
@ -146,7 +155,7 @@ async fn notify_new_payload<'a, T: BeaconChainTypes>(
.ok_or(ExecutionPayloadError::NoExecutionConnection)?; .ok_or(ExecutionPayloadError::NoExecutionConnection)?;
let new_payload_response = execution_layer let new_payload_response = execution_layer
.notify_new_payload(&execution_payload.into()) .notify_new_payload(&execution_payload.into(), versioned_hashes)
.await; .await;
match new_payload_response { match new_payload_response {

View File

@ -1,24 +1,23 @@
use kzg::{Error as KzgError, Kzg, BYTES_PER_BLOB}; use kzg::{Error as KzgError, Kzg, KzgPreset};
use types::{Blob, EthSpec, Hash256, KzgCommitment, KzgProof}; use types::{Blob, EthSpec, Hash256, KzgCommitment, KzgProof};
/// Converts a blob ssz List object to an array to be used with the kzg /// Converts a blob ssz List object to an array to be used with the kzg
/// crypto library. /// crypto library.
fn ssz_blob_to_crypto_blob<T: EthSpec>(blob: Blob<T>) -> kzg::Blob { fn ssz_blob_to_crypto_blob<T: EthSpec>(
let blob_vec: Vec<u8> = blob.into(); blob: Blob<T>,
let mut arr = [0; BYTES_PER_BLOB]; ) -> Result<<<T as EthSpec>::Kzg as KzgPreset>::Blob, KzgError> {
arr.copy_from_slice(&blob_vec); T::blob_from_bytes(blob.to_vec().as_slice())
arr.into()
} }
/// Validate a single blob-commitment-proof triplet from a `BlobSidecar`. /// Validate a single blob-commitment-proof triplet from a `BlobSidecar`.
pub fn validate_blob<T: EthSpec>( pub fn validate_blob<T: EthSpec>(
kzg: &Kzg, kzg: &Kzg<T::Kzg>,
blob: Blob<T>, blob: Blob<T>,
kzg_commitment: KzgCommitment, kzg_commitment: KzgCommitment,
kzg_proof: KzgProof, kzg_proof: KzgProof,
) -> Result<bool, KzgError> { ) -> Result<bool, KzgError> {
kzg.verify_blob_kzg_proof( kzg.verify_blob_kzg_proof(
ssz_blob_to_crypto_blob::<T>(blob), ssz_blob_to_crypto_blob::<T>(blob)?,
kzg_commitment, kzg_commitment,
kzg_proof, kzg_proof,
) )
@ -26,51 +25,64 @@ pub fn validate_blob<T: EthSpec>(
/// Validate a batch of blob-commitment-proof triplets from multiple `BlobSidecars`. /// Validate a batch of blob-commitment-proof triplets from multiple `BlobSidecars`.
pub fn validate_blobs<T: EthSpec>( pub fn validate_blobs<T: EthSpec>(
kzg: &Kzg, kzg: &Kzg<T::Kzg>,
expected_kzg_commitments: &[KzgCommitment], expected_kzg_commitments: &[KzgCommitment],
blobs: &[Blob<T>], blobs: &[Blob<T>],
kzg_proofs: &[KzgProof], kzg_proofs: &[KzgProof],
) -> Result<bool, KzgError> { ) -> Result<bool, KzgError> {
// TODO(sean) batch verification fails with a single element, it's unclear to me why
if blobs.len() == 1 && kzg_proofs.len() == 1 && expected_kzg_commitments.len() == 1 {
if let (Some(blob), Some(kzg_proof), Some(kzg_commitment)) = (
blobs.get(0),
kzg_proofs.get(0),
expected_kzg_commitments.get(0),
) {
return validate_blob::<T>(kzg, blob.clone(), *kzg_commitment, *kzg_proof);
} else {
return Ok(false);
}
}
let blobs = blobs let blobs = blobs
.iter() .iter()
.map(|blob| ssz_blob_to_crypto_blob::<T>(blob.clone())) // Avoid this clone .map(|blob| ssz_blob_to_crypto_blob::<T>(blob.clone())) // Avoid this clone
.collect::<Vec<_>>(); .collect::<Result<Vec<_>, KzgError>>()?;
kzg.verify_blob_kzg_proof_batch(&blobs, expected_kzg_commitments, kzg_proofs) kzg.verify_blob_kzg_proof_batch(&blobs, expected_kzg_commitments, kzg_proofs)
} }
/// Compute the kzg proof given an ssz blob and its kzg commitment. /// Compute the kzg proof given an ssz blob and its kzg commitment.
pub fn compute_blob_kzg_proof<T: EthSpec>( pub fn compute_blob_kzg_proof<T: EthSpec>(
kzg: &Kzg, kzg: &Kzg<T::Kzg>,
blob: &Blob<T>, blob: &Blob<T>,
kzg_commitment: KzgCommitment, kzg_commitment: KzgCommitment,
) -> Result<KzgProof, KzgError> { ) -> Result<KzgProof, KzgError> {
// Avoid this blob clone // Avoid this blob clone
kzg.compute_blob_kzg_proof(ssz_blob_to_crypto_blob::<T>(blob.clone()), kzg_commitment) kzg.compute_blob_kzg_proof(ssz_blob_to_crypto_blob::<T>(blob.clone())?, kzg_commitment)
} }
/// Compute the kzg commitment for a given blob. /// Compute the kzg commitment for a given blob.
pub fn blob_to_kzg_commitment<T: EthSpec>( pub fn blob_to_kzg_commitment<T: EthSpec>(
kzg: &Kzg, kzg: &Kzg<T::Kzg>,
blob: Blob<T>, blob: Blob<T>,
) -> Result<KzgCommitment, KzgError> { ) -> Result<KzgCommitment, KzgError> {
kzg.blob_to_kzg_commitment(ssz_blob_to_crypto_blob::<T>(blob)) kzg.blob_to_kzg_commitment(ssz_blob_to_crypto_blob::<T>(blob)?)
} }
/// Compute the kzg proof for a given blob and an evaluation point z. /// Compute the kzg proof for a given blob and an evaluation point z.
pub fn compute_kzg_proof<T: EthSpec>( pub fn compute_kzg_proof<T: EthSpec>(
kzg: &Kzg, kzg: &Kzg<T::Kzg>,
blob: Blob<T>, blob: Blob<T>,
z: Hash256, z: Hash256,
) -> Result<(KzgProof, Hash256), KzgError> { ) -> Result<(KzgProof, Hash256), KzgError> {
let z = z.0.into(); let z = z.0.into();
kzg.compute_kzg_proof(ssz_blob_to_crypto_blob::<T>(blob), z) kzg.compute_kzg_proof(ssz_blob_to_crypto_blob::<T>(blob)?, z)
.map(|(proof, z)| (proof, Hash256::from_slice(&z.to_vec()))) .map(|(proof, z)| (proof, Hash256::from_slice(&z.to_vec())))
} }
/// Verify a `kzg_proof` for a `kzg_commitment` that evaluating a polynomial at `z` results in `y` /// Verify a `kzg_proof` for a `kzg_commitment` that evaluating a polynomial at `z` results in `y`
pub fn verify_kzg_proof<T: EthSpec>( pub fn verify_kzg_proof<T: EthSpec>(
kzg: &Kzg, kzg: &Kzg<T::Kzg>,
kzg_commitment: KzgCommitment, kzg_commitment: KzgCommitment,
kzg_proof: KzgProof, kzg_proof: KzgProof,
z: Hash256, z: Hash256,

View File

@ -1027,6 +1027,24 @@ lazy_static! {
"beacon_aggregated_attestation_subsets_total", "beacon_aggregated_attestation_subsets_total",
"Count of new aggregated attestations that are subsets of already known aggregates" "Count of new aggregated attestations that are subsets of already known aggregates"
); );
/*
* Kzg related metrics
*/
pub static ref KZG_VERIFICATION_SINGLE_TIMES: Result<Histogram> =
try_create_histogram("kzg_verification_single_seconds", "Runtime of single kzg verification");
pub static ref KZG_VERIFICATION_BATCH_TIMES: Result<Histogram> =
try_create_histogram("kzg_verification_batch_seconds", "Runtime of batched kzg verification");
/*
* Availability related metrics
*/
pub static ref BLOCK_AVAILABILITY_DELAY: Result<Histogram> = try_create_histogram_with_buckets(
"block_availability_delay",
"Duration between start of the slot and the time at which all components of the block are available.",
// Create a custom bucket list for greater granularity in block delay
Ok(vec![0.1, 0.2, 0.3,0.4,0.5,0.75,1.0,1.25,1.5,1.75,2.0,2.5,3.0,3.5,4.0,5.0,6.0,7.0,8.0,9.0,10.0,15.0,20.0])
);
} }
/// Scrape the `beacon_chain` for metrics that are not constantly updated (e.g., the present slot, /// Scrape the `beacon_chain` for metrics that are not constantly updated (e.g., the present slot,

View File

@ -16,6 +16,7 @@ use crate::{
}; };
use bls::get_withdrawal_credentials; use bls::get_withdrawal_credentials;
use eth2::types::BlockContentsTuple; use eth2::types::BlockContentsTuple;
use execution_layer::test_utils::generate_genesis_header;
use execution_layer::{ use execution_layer::{
auth::JwtKey, auth::JwtKey,
test_utils::{ test_utils::{
@ -30,8 +31,8 @@ use int_to_bytes::int_to_bytes32;
use kzg::{Kzg, TrustedSetup}; use kzg::{Kzg, TrustedSetup};
use merkle_proof::MerkleTree; use merkle_proof::MerkleTree;
use operation_pool::ReceivedPreCapella; use operation_pool::ReceivedPreCapella;
use parking_lot::Mutex;
use parking_lot::RwLockWriteGuard; use parking_lot::RwLockWriteGuard;
use parking_lot::{Mutex, RwLock};
use rand::rngs::StdRng; use rand::rngs::StdRng;
use rand::Rng; use rand::Rng;
use rand::SeedableRng; use rand::SeedableRng;
@ -53,6 +54,7 @@ use std::str::FromStr;
use std::sync::Arc; use std::sync::Arc;
use std::time::Duration; use std::time::Duration;
use store::{config::StoreConfig, HotColdDB, ItemStore, LevelDB, MemoryStore}; use store::{config::StoreConfig, HotColdDB, ItemStore, LevelDB, MemoryStore};
use task_executor::TaskExecutor;
use task_executor::{test_utils::TestRuntime, ShutdownReason}; use task_executor::{test_utils::TestRuntime, ShutdownReason};
use tree_hash::TreeHash; use tree_hash::TreeHash;
use types::sync_selection_proof::SyncSelectionProof; use types::sync_selection_proof::SyncSelectionProof;
@ -195,11 +197,12 @@ impl<E: EthSpec> Builder<EphemeralHarnessType<E>> {
.unwrap(), .unwrap(),
); );
let mutator = move |builder: BeaconChainBuilder<_>| { let mutator = move |builder: BeaconChainBuilder<_>| {
let header = generate_genesis_header::<E>(builder.get_spec(), false);
let genesis_state = interop_genesis_state_with_eth1::<E>( let genesis_state = interop_genesis_state_with_eth1::<E>(
&validator_keypairs, &validator_keypairs,
HARNESS_GENESIS_TIME, HARNESS_GENESIS_TIME,
Hash256::from_slice(DEFAULT_ETH1_BLOCK_HASH), Hash256::from_slice(DEFAULT_ETH1_BLOCK_HASH),
None, header,
builder.get_spec(), builder.get_spec(),
) )
.expect("should generate interop state"); .expect("should generate interop state");
@ -256,11 +259,12 @@ impl<E: EthSpec> Builder<DiskHarnessType<E>> {
.expect("cannot build without validator keypairs"); .expect("cannot build without validator keypairs");
let mutator = move |builder: BeaconChainBuilder<_>| { let mutator = move |builder: BeaconChainBuilder<_>| {
let header = generate_genesis_header::<E>(builder.get_spec(), false);
let genesis_state = interop_genesis_state_with_eth1::<E>( let genesis_state = interop_genesis_state_with_eth1::<E>(
&validator_keypairs, &validator_keypairs,
HARNESS_GENESIS_TIME, HARNESS_GENESIS_TIME,
Hash256::from_slice(DEFAULT_ETH1_BLOCK_HASH), Hash256::from_slice(DEFAULT_ETH1_BLOCK_HASH),
None, header,
builder.get_spec(), builder.get_spec(),
) )
.expect("should generate interop state"); .expect("should generate interop state");
@ -392,7 +396,7 @@ where
self self
} }
pub fn execution_layer(mut self, urls: &[&str]) -> Self { pub fn execution_layer_from_urls(mut self, urls: &[&str]) -> Self {
assert!( assert!(
self.execution_layer.is_none(), self.execution_layer.is_none(),
"execution layer already defined" "execution layer already defined"
@ -421,6 +425,11 @@ where
self self
} }
pub fn execution_layer(mut self, el: Option<ExecutionLayer<E>>) -> Self {
self.execution_layer = el;
self
}
pub fn recalculate_fork_times_with_genesis(mut self, genesis_time: u64) -> Self { pub fn recalculate_fork_times_with_genesis(mut self, genesis_time: u64) -> Self {
let mock = self let mock = self
.mock_execution_layer .mock_execution_layer
@ -434,7 +443,7 @@ where
spec.capella_fork_epoch.map(|epoch| { spec.capella_fork_epoch.map(|epoch| {
genesis_time + spec.seconds_per_slot * E::slots_per_epoch() * epoch.as_u64() genesis_time + spec.seconds_per_slot * E::slots_per_epoch() * epoch.as_u64()
}); });
mock.server.execution_block_generator().deneb_time = spec.deneb_fork_epoch.map(|epoch| { mock.server.execution_block_generator().cancun_time = spec.deneb_fork_epoch.map(|epoch| {
genesis_time + spec.seconds_per_slot * E::slots_per_epoch() * epoch.as_u64() genesis_time + spec.seconds_per_slot * E::slots_per_epoch() * epoch.as_u64()
}); });
@ -442,30 +451,11 @@ where
} }
pub fn mock_execution_layer(mut self) -> Self { pub fn mock_execution_layer(mut self) -> Self {
let spec = self.spec.clone().expect("cannot build without spec"); let mock = mock_execution_layer_from_parts::<E>(
let shanghai_time = spec.capella_fork_epoch.map(|epoch| { self.spec.as_ref().expect("cannot build without spec"),
HARNESS_GENESIS_TIME + spec.seconds_per_slot * E::slots_per_epoch() * epoch.as_u64()
});
let deneb_time = spec.deneb_fork_epoch.map(|epoch| {
HARNESS_GENESIS_TIME + spec.seconds_per_slot * E::slots_per_epoch() * epoch.as_u64()
});
let trusted_setup: TrustedSetup =
serde_json::from_reader(eth2_network_config::TRUSTED_SETUP)
.map_err(|e| format!("Unable to read trusted setup file: {}", e))
.expect("should have trusted setup");
let kzg = Kzg::new_from_trusted_setup(trusted_setup).expect("should create kzg");
let mock = MockExecutionLayer::new(
self.runtime.task_executor.clone(), self.runtime.task_executor.clone(),
DEFAULT_TERMINAL_BLOCK,
shanghai_time,
deneb_time,
None, None,
Some(JwtKey::from_slice(&DEFAULT_JWT_SECRET).unwrap()),
spec,
None, None,
Some(kzg),
); );
self.execution_layer = Some(mock.el.clone()); self.execution_layer = Some(mock.el.clone());
self.mock_execution_layer = Some(mock); self.mock_execution_layer = Some(mock);
@ -480,29 +470,12 @@ where
// Get a random unused port // Get a random unused port
let port = unused_port::unused_tcp4_port().unwrap(); let port = unused_port::unused_tcp4_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();
let spec = self.spec.as_ref().expect("cannot build without spec");
let spec = self.spec.clone().expect("cannot build without spec"); let mock_el = mock_execution_layer_from_parts::<E>(
let shanghai_time = spec.capella_fork_epoch.map(|epoch| { spec,
HARNESS_GENESIS_TIME + spec.seconds_per_slot * E::slots_per_epoch() * epoch.as_u64()
});
let deneb_time = spec.deneb_fork_epoch.map(|epoch| {
HARNESS_GENESIS_TIME + spec.seconds_per_slot * E::slots_per_epoch() * epoch.as_u64()
});
let trusted_setup: TrustedSetup =
serde_json::from_reader(eth2_network_config::TRUSTED_SETUP)
.map_err(|e| format!("Unable to read trusted setup file: {}", e))
.expect("should have trusted setup");
let kzg = Kzg::new_from_trusted_setup(trusted_setup).expect("should create kzg");
let mock_el = MockExecutionLayer::new(
self.runtime.task_executor.clone(), self.runtime.task_executor.clone(),
DEFAULT_TERMINAL_BLOCK,
shanghai_time,
deneb_time,
builder_threshold,
Some(JwtKey::from_slice(&DEFAULT_JWT_SECRET).unwrap()),
spec.clone(),
Some(builder_url.clone()), Some(builder_url.clone()),
Some(kzg), builder_threshold,
) )
.move_to_terminal_block(); .move_to_terminal_block();
@ -512,7 +485,7 @@ where
mock_el_url, mock_el_url,
builder_url, builder_url,
beacon_url, beacon_url,
spec, spec.clone(),
self.runtime.task_executor.clone(), self.runtime.task_executor.clone(),
)); ));
self.execution_layer = Some(mock_el.el.clone()); self.execution_layer = Some(mock_el.el.clone());
@ -547,7 +520,7 @@ where
.validator_keypairs .validator_keypairs
.expect("cannot build without validator keypairs"); .expect("cannot build without validator keypairs");
let trusted_setup: TrustedSetup = let trusted_setup: TrustedSetup =
serde_json::from_reader(eth2_network_config::TRUSTED_SETUP) serde_json::from_reader(eth2_network_config::get_trusted_setup::<E::Kzg>())
.map_err(|e| format!("Unable to read trusted setup file: {}", e)) .map_err(|e| format!("Unable to read trusted setup file: {}", e))
.unwrap(); .unwrap();
@ -603,11 +576,44 @@ where
runtime: self.runtime, runtime: self.runtime,
mock_execution_layer: self.mock_execution_layer, mock_execution_layer: self.mock_execution_layer,
mock_builder: self.mock_builder.map(Arc::new), mock_builder: self.mock_builder.map(Arc::new),
blob_signature_cache: <_>::default(),
rng: make_rng(), rng: make_rng(),
} }
} }
} }
pub fn mock_execution_layer_from_parts<T: EthSpec>(
spec: &ChainSpec,
task_executor: TaskExecutor,
builder_url: Option<SensitiveUrl>,
builder_threshold: Option<u128>,
) -> MockExecutionLayer<T> {
let shanghai_time = spec.capella_fork_epoch.map(|epoch| {
HARNESS_GENESIS_TIME + spec.seconds_per_slot * T::slots_per_epoch() * epoch.as_u64()
});
let cancun_time = spec.deneb_fork_epoch.map(|epoch| {
HARNESS_GENESIS_TIME + spec.seconds_per_slot * T::slots_per_epoch() * epoch.as_u64()
});
let trusted_setup: TrustedSetup =
serde_json::from_reader(eth2_network_config::get_trusted_setup::<T::Kzg>())
.map_err(|e| format!("Unable to read trusted setup file: {}", e))
.expect("should have trusted setup");
let kzg = Kzg::new_from_trusted_setup(trusted_setup).expect("should create kzg");
MockExecutionLayer::new(
task_executor,
DEFAULT_TERMINAL_BLOCK,
shanghai_time,
cancun_time,
builder_threshold,
Some(JwtKey::from_slice(&DEFAULT_JWT_SECRET).unwrap()),
spec.clone(),
builder_url,
Some(kzg),
)
}
/// A testing harness which can instantiate a `BeaconChain` and populate it with blocks and /// A testing harness which can instantiate a `BeaconChain` and populate it with blocks and
/// attestations. /// attestations.
/// ///
@ -629,9 +635,29 @@ pub struct BeaconChainHarness<T: BeaconChainTypes> {
pub mock_execution_layer: Option<MockExecutionLayer<T::EthSpec>>, pub mock_execution_layer: Option<MockExecutionLayer<T::EthSpec>>,
pub mock_builder: Option<Arc<TestingBuilder<T::EthSpec>>>, pub mock_builder: Option<Arc<TestingBuilder<T::EthSpec>>>,
/// Cache for blob signature because we don't need them for import, but we do need them
/// to test gossip validation. We always make them during block production but drop them
/// before storing them in the db.
pub blob_signature_cache: Arc<RwLock<HashMap<BlobSignatureKey, Signature>>>,
pub rng: Mutex<StdRng>, pub rng: Mutex<StdRng>,
} }
#[derive(Clone, Debug, Hash, Eq, PartialEq)]
pub struct BlobSignatureKey {
block_root: Hash256,
blob_index: u64,
}
impl BlobSignatureKey {
pub fn new(block_root: Hash256, blob_index: u64) -> Self {
Self {
block_root,
blob_index,
}
}
}
pub type CommitteeAttestations<E> = Vec<(Attestation<E>, SubnetId)>; pub type CommitteeAttestations<E> = Vec<(Attestation<E>, SubnetId)>;
pub type HarnessAttestations<E> = pub type HarnessAttestations<E> =
Vec<(CommitteeAttestations<E>, Option<SignedAggregateAndProof<E>>)>; Vec<(CommitteeAttestations<E>, Option<SignedAggregateAndProof<E>>)>;
@ -663,6 +689,20 @@ where
.execution_block_generator() .execution_block_generator()
} }
pub fn get_head_block(&self) -> BlockWrapper<E> {
let block = self.chain.head_beacon_block();
let block_root = block.canonical_root();
let blobs = self.chain.get_blobs(&block_root).unwrap();
BlockWrapper::new(block, blobs)
}
pub fn get_full_block(&self, block_root: &Hash256) -> BlockWrapper<E> {
let block = self.chain.get_blinded_block(block_root).unwrap().unwrap();
let full_block = self.chain.store.make_full_block(block_root, block).unwrap();
let blobs = self.chain.get_blobs(block_root).unwrap();
BlockWrapper::new(Arc::new(full_block), blobs)
}
pub fn get_all_validators(&self) -> Vec<usize> { pub fn get_all_validators(&self) -> Vec<usize> {
(0..self.validator_keypairs.len()).collect() (0..self.validator_keypairs.len()).collect()
} }
@ -824,7 +864,7 @@ where
.proposal_blob_cache .proposal_blob_cache
.pop(&signed_block.canonical_root()) .pop(&signed_block.canonical_root())
{ {
let signed_blobs = Vec::from(blobs) let signed_blobs: SignedBlobSidecarList<E> = Vec::from(blobs)
.into_iter() .into_iter()
.map(|blob| { .map(|blob| {
blob.sign( blob.sign(
@ -836,6 +876,13 @@ where
}) })
.collect::<Vec<_>>() .collect::<Vec<_>>()
.into(); .into();
let mut guard = self.blob_signature_cache.write();
for blob in &signed_blobs {
guard.insert(
BlobSignatureKey::new(blob.message.block_root, blob.message.index),
blob.signature.clone(),
);
}
(signed_block, Some(signed_blobs)) (signed_block, Some(signed_blobs))
} else { } else {
(signed_block, None) (signed_block, None)

View File

@ -68,6 +68,7 @@ async fn produces_attestations() {
.store .store
.make_full_block(&block_root, blinded_block) .make_full_block(&block_root, blinded_block)
.unwrap(); .unwrap();
let blobs = chain.get_blobs(&block_root).unwrap();
let epoch_boundary_slot = state let epoch_boundary_slot = state
.current_epoch() .current_epoch()
@ -132,7 +133,8 @@ async fn produces_attestations() {
assert_eq!(data.target.epoch, state.current_epoch(), "bad target epoch"); assert_eq!(data.target.epoch, state.current_epoch(), "bad target epoch");
assert_eq!(data.target.root, target_root, "bad target root"); assert_eq!(data.target.root, target_root, "bad target root");
let block_wrapper: BlockWrapper<MainnetEthSpec> = Arc::new(block.clone()).into(); let block_wrapper =
BlockWrapper::<MainnetEthSpec>::new(Arc::new(block.clone()), blobs.clone());
let beacon_chain::blob_verification::MaybeAvailableBlock::Available(available_block) = chain let beacon_chain::blob_verification::MaybeAvailableBlock::Available(available_block) = chain
.data_availability_checker .data_availability_checker
.check_availability(block_wrapper) .check_availability(block_wrapper)
@ -202,7 +204,12 @@ async fn early_attester_cache_old_request() {
.get_block(&head.beacon_block_root) .get_block(&head.beacon_block_root)
.unwrap(); .unwrap();
let block_wrapper: BlockWrapper<MainnetEthSpec> = head.beacon_block.clone().into(); let head_blobs = harness
.chain
.get_blobs(&head.beacon_block_root)
.expect("should get blobs");
let block_wrapper = BlockWrapper::<MainnetEthSpec>::new(head.beacon_block.clone(), head_blobs);
let beacon_chain::blob_verification::MaybeAvailableBlock::Available(available_block) = harness.chain let beacon_chain::blob_verification::MaybeAvailableBlock::Available(available_block) = harness.chain
.data_availability_checker .data_availability_checker
.check_availability(block_wrapper) .check_availability(block_wrapper)

View File

@ -797,7 +797,6 @@ async fn unaggregated_gossip_verification() {
a.data.target.epoch = early_slot.epoch(E::slots_per_epoch()); a.data.target.epoch = early_slot.epoch(E::slots_per_epoch());
}, },
|tester, err| { |tester, err| {
dbg!(&err);
assert!(matches!( assert!(matches!(
err, err,
AttnError::PastSlot { AttnError::PastSlot {

View File

@ -1,6 +1,7 @@
#![cfg(not(debug_assertions))] #![cfg(not(debug_assertions))]
use beacon_chain::blob_verification::BlockWrapper; use beacon_chain::blob_verification::BlockWrapper;
use beacon_chain::test_utils::BlobSignatureKey;
use beacon_chain::{ use beacon_chain::{
blob_verification::AsBlock, blob_verification::AsBlock,
test_utils::{AttestationStrategy, BeaconChainHarness, BlockStrategy, EphemeralHarnessType}, test_utils::{AttestationStrategy, BeaconChainHarness, BlockStrategy, EphemeralHarnessType},
@ -36,7 +37,7 @@ lazy_static! {
static ref KEYPAIRS: Vec<Keypair> = types::test_utils::generate_deterministic_keypairs(VALIDATOR_COUNT); static ref KEYPAIRS: Vec<Keypair> = types::test_utils::generate_deterministic_keypairs(VALIDATOR_COUNT);
} }
async fn get_chain_segment() -> Vec<BeaconSnapshot<E>> { async fn get_chain_segment() -> (Vec<BeaconSnapshot<E>>, Vec<Option<BlobSidecarList<E>>>) {
let harness = get_harness(VALIDATOR_COUNT); let harness = get_harness(VALIDATOR_COUNT);
harness harness
@ -48,6 +49,7 @@ async fn get_chain_segment() -> Vec<BeaconSnapshot<E>> {
.await; .await;
let mut segment = Vec::with_capacity(CHAIN_SEGMENT_LENGTH); let mut segment = Vec::with_capacity(CHAIN_SEGMENT_LENGTH);
let mut segment_blobs = Vec::with_capacity(CHAIN_SEGMENT_LENGTH);
for snapshot in harness for snapshot in harness
.chain .chain
.chain_dump() .chain_dump()
@ -66,8 +68,76 @@ async fn get_chain_segment() -> Vec<BeaconSnapshot<E>> {
beacon_block: Arc::new(full_block), beacon_block: Arc::new(full_block),
beacon_state: snapshot.beacon_state, beacon_state: snapshot.beacon_state,
}); });
segment_blobs.push(
harness
.chain
.get_blobs(&snapshot.beacon_block_root)
.unwrap(),
)
} }
segment (segment, segment_blobs)
}
async fn get_chain_segment_with_signed_blobs() -> (
Vec<BeaconSnapshot<E>>,
Vec<Option<VariableList<SignedBlobSidecar<E>, <E as EthSpec>::MaxBlobsPerBlock>>>,
) {
let harness = get_harness(VALIDATOR_COUNT);
harness
.extend_chain(
CHAIN_SEGMENT_LENGTH,
BlockStrategy::OnCanonicalHead,
AttestationStrategy::AllValidators,
)
.await;
let mut segment = Vec::with_capacity(CHAIN_SEGMENT_LENGTH);
let mut segment_blobs = Vec::with_capacity(CHAIN_SEGMENT_LENGTH);
for snapshot in harness
.chain
.chain_dump()
.expect("should dump chain")
.into_iter()
.skip(1)
{
let full_block = harness
.chain
.get_block(&snapshot.beacon_block_root)
.await
.unwrap()
.unwrap();
segment.push(BeaconSnapshot {
beacon_block_root: snapshot.beacon_block_root,
beacon_block: Arc::new(full_block),
beacon_state: snapshot.beacon_state,
});
let signed_blobs = harness
.chain
.get_blobs(&snapshot.beacon_block_root)
.unwrap()
.map(|blobs| {
let blobs = blobs
.into_iter()
.map(|blob| {
let block_root = blob.block_root;
let blob_index = blob.index;
SignedBlobSidecar {
message: blob,
signature: harness
.blob_signature_cache
.read()
.get(&BlobSignatureKey::new(block_root, blob_index))
.unwrap()
.clone(),
}
})
.collect::<Vec<_>>();
VariableList::from(blobs)
});
segment_blobs.push(signed_blobs)
}
(segment, segment_blobs)
} }
fn get_harness(validator_count: usize) -> BeaconChainHarness<EphemeralHarnessType<E>> { fn get_harness(validator_count: usize) -> BeaconChainHarness<EphemeralHarnessType<E>> {
@ -83,10 +153,14 @@ fn get_harness(validator_count: usize) -> BeaconChainHarness<EphemeralHarnessTyp
harness harness
} }
fn chain_segment_blocks(chain_segment: &[BeaconSnapshot<E>]) -> Vec<Arc<SignedBeaconBlock<E>>> { fn chain_segment_blocks(
chain_segment: &[BeaconSnapshot<E>],
blobs: &[Option<BlobSidecarList<E>>],
) -> Vec<BlockWrapper<E>> {
chain_segment chain_segment
.iter() .iter()
.map(|snapshot| snapshot.beacon_block.clone().into()) .zip(blobs.into_iter())
.map(|(snapshot, blobs)| BlockWrapper::new(snapshot.beacon_block.clone(), blobs.clone()))
.collect() .collect()
} }
@ -142,8 +216,8 @@ fn update_parent_roots(snapshots: &mut [BeaconSnapshot<E>]) {
#[tokio::test] #[tokio::test]
async fn chain_segment_full_segment() { async fn chain_segment_full_segment() {
let harness = get_harness(VALIDATOR_COUNT); let harness = get_harness(VALIDATOR_COUNT);
let chain_segment = get_chain_segment().await; let (chain_segment, chain_segment_blobs) = get_chain_segment().await;
let blocks: Vec<BlockWrapper<E>> = chain_segment_blocks(&chain_segment) let blocks: Vec<BlockWrapper<E>> = chain_segment_blocks(&chain_segment, &chain_segment_blobs)
.into_iter() .into_iter()
.map(|block| block.into()) .map(|block| block.into())
.collect(); .collect();
@ -181,8 +255,9 @@ async fn chain_segment_full_segment() {
async fn chain_segment_varying_chunk_size() { async fn chain_segment_varying_chunk_size() {
for chunk_size in &[1, 2, 3, 5, 31, 32, 33, 42] { for chunk_size in &[1, 2, 3, 5, 31, 32, 33, 42] {
let harness = get_harness(VALIDATOR_COUNT); let harness = get_harness(VALIDATOR_COUNT);
let chain_segment = get_chain_segment().await; let (chain_segment, chain_segment_blobs) = get_chain_segment().await;
let blocks: Vec<BlockWrapper<E>> = chain_segment_blocks(&chain_segment) let blocks: Vec<BlockWrapper<E>> =
chain_segment_blocks(&chain_segment, &chain_segment_blobs)
.into_iter() .into_iter()
.map(|block| block.into()) .map(|block| block.into())
.collect(); .collect();
@ -214,7 +289,7 @@ async fn chain_segment_varying_chunk_size() {
#[tokio::test] #[tokio::test]
async fn chain_segment_non_linear_parent_roots() { async fn chain_segment_non_linear_parent_roots() {
let harness = get_harness(VALIDATOR_COUNT); let harness = get_harness(VALIDATOR_COUNT);
let chain_segment = get_chain_segment().await; let (chain_segment, chain_segment_blobs) = get_chain_segment().await;
harness harness
.chain .chain
@ -224,7 +299,8 @@ async fn chain_segment_non_linear_parent_roots() {
/* /*
* Test with a block removed. * Test with a block removed.
*/ */
let mut blocks: Vec<BlockWrapper<E>> = chain_segment_blocks(&chain_segment) let mut blocks: Vec<BlockWrapper<E>> =
chain_segment_blocks(&chain_segment, &chain_segment_blobs)
.into_iter() .into_iter()
.map(|block| block.into()) .map(|block| block.into())
.collect(); .collect();
@ -245,7 +321,8 @@ async fn chain_segment_non_linear_parent_roots() {
/* /*
* Test with a modified parent root. * Test with a modified parent root.
*/ */
let mut blocks: Vec<BlockWrapper<E>> = chain_segment_blocks(&chain_segment) let mut blocks: Vec<BlockWrapper<E>> =
chain_segment_blocks(&chain_segment, &chain_segment_blobs)
.into_iter() .into_iter()
.map(|block| block.into()) .map(|block| block.into())
.collect(); .collect();
@ -270,7 +347,7 @@ async fn chain_segment_non_linear_parent_roots() {
#[tokio::test] #[tokio::test]
async fn chain_segment_non_linear_slots() { async fn chain_segment_non_linear_slots() {
let harness = get_harness(VALIDATOR_COUNT); let harness = get_harness(VALIDATOR_COUNT);
let chain_segment = get_chain_segment().await; let (chain_segment, chain_segment_blobs) = get_chain_segment().await;
harness harness
.chain .chain
.slot_clock .slot_clock
@ -280,7 +357,8 @@ async fn chain_segment_non_linear_slots() {
* Test where a child is lower than the parent. * Test where a child is lower than the parent.
*/ */
let mut blocks: Vec<BlockWrapper<E>> = chain_segment_blocks(&chain_segment) let mut blocks: Vec<BlockWrapper<E>> =
chain_segment_blocks(&chain_segment, &chain_segment_blobs)
.into_iter() .into_iter()
.map(|block| block.into()) .map(|block| block.into())
.collect(); .collect();
@ -304,7 +382,8 @@ async fn chain_segment_non_linear_slots() {
* Test where a child is equal to the parent. * Test where a child is equal to the parent.
*/ */
let mut blocks: Vec<BlockWrapper<E>> = chain_segment_blocks(&chain_segment) let mut blocks: Vec<BlockWrapper<E>> =
chain_segment_blocks(&chain_segment, &chain_segment_blobs)
.into_iter() .into_iter()
.map(|block| block.into()) .map(|block| block.into())
.collect(); .collect();
@ -327,6 +406,7 @@ async fn chain_segment_non_linear_slots() {
async fn assert_invalid_signature( async fn assert_invalid_signature(
chain_segment: &[BeaconSnapshot<E>], chain_segment: &[BeaconSnapshot<E>],
chain_segment_blobs: &[Option<BlobSidecarList<E>>],
harness: &BeaconChainHarness<EphemeralHarnessType<E>>, harness: &BeaconChainHarness<EphemeralHarnessType<E>>,
block_index: usize, block_index: usize,
snapshots: &[BeaconSnapshot<E>], snapshots: &[BeaconSnapshot<E>],
@ -334,7 +414,8 @@ async fn assert_invalid_signature(
) { ) {
let blocks: Vec<BlockWrapper<E>> = snapshots let blocks: Vec<BlockWrapper<E>> = snapshots
.iter() .iter()
.map(|snapshot| snapshot.beacon_block.clone().into()) .zip(chain_segment_blobs.iter())
.map(|(snapshot, blobs)| BlockWrapper::new(snapshot.beacon_block.clone(), blobs.clone()))
.collect(); .collect();
// Ensure the block will be rejected if imported in a chain segment. // Ensure the block will be rejected if imported in a chain segment.
@ -358,7 +439,8 @@ async fn assert_invalid_signature(
let ancestor_blocks = chain_segment let ancestor_blocks = chain_segment
.iter() .iter()
.take(block_index) .take(block_index)
.map(|snapshot| snapshot.beacon_block.clone().into()) .zip(chain_segment_blobs.iter())
.map(|(snapshot, blobs)| BlockWrapper::new(snapshot.beacon_block.clone(), blobs.clone()))
.collect(); .collect();
// We don't care if this fails, we just call this to ensure that all prior blocks have been // We don't care if this fails, we just call this to ensure that all prior blocks have been
// imported prior to this test. // imported prior to this test.
@ -372,7 +454,10 @@ async fn assert_invalid_signature(
.chain .chain
.process_block( .process_block(
snapshots[block_index].beacon_block.canonical_root(), snapshots[block_index].beacon_block.canonical_root(),
BlockWrapper::new(
snapshots[block_index].beacon_block.clone(), snapshots[block_index].beacon_block.clone(),
chain_segment_blobs[block_index].clone(),
),
NotifyExecutionLayer::Yes, NotifyExecutionLayer::Yes,
) )
.await; .await;
@ -403,7 +488,7 @@ async fn get_invalid_sigs_harness(
} }
#[tokio::test] #[tokio::test]
async fn invalid_signature_gossip_block() { async fn invalid_signature_gossip_block() {
let chain_segment = get_chain_segment().await; let (chain_segment, chain_segment_blobs) = get_chain_segment().await;
for &block_index in BLOCK_INDICES { for &block_index in BLOCK_INDICES {
// Ensure the block will be rejected if imported on its own (without gossip checking). // Ensure the block will be rejected if imported on its own (without gossip checking).
let harness = get_invalid_sigs_harness(&chain_segment).await; let harness = get_invalid_sigs_harness(&chain_segment).await;
@ -421,7 +506,10 @@ async fn invalid_signature_gossip_block() {
let ancestor_blocks = chain_segment let ancestor_blocks = chain_segment
.iter() .iter()
.take(block_index) .take(block_index)
.map(|snapshot| snapshot.beacon_block.clone().into()) .zip(chain_segment_blobs.iter())
.map(|(snapshot, blobs)| {
BlockWrapper::new(snapshot.beacon_block.clone(), blobs.clone())
})
.collect(); .collect();
harness harness
.chain .chain
@ -449,7 +537,7 @@ async fn invalid_signature_gossip_block() {
#[tokio::test] #[tokio::test]
async fn invalid_signature_block_proposal() { async fn invalid_signature_block_proposal() {
let chain_segment = get_chain_segment().await; let (chain_segment, chain_segment_blobs) = get_chain_segment().await;
for &block_index in BLOCK_INDICES { for &block_index in BLOCK_INDICES {
let harness = get_invalid_sigs_harness(&chain_segment).await; let harness = get_invalid_sigs_harness(&chain_segment).await;
let mut snapshots = chain_segment.clone(); let mut snapshots = chain_segment.clone();
@ -464,7 +552,10 @@ async fn invalid_signature_block_proposal() {
)); ));
let blocks: Vec<BlockWrapper<E>> = snapshots let blocks: Vec<BlockWrapper<E>> = snapshots
.iter() .iter()
.map(|snapshot| snapshot.beacon_block.clone().into()) .zip(chain_segment_blobs.iter())
.map(|(snapshot, blobs)| {
BlockWrapper::new(snapshot.beacon_block.clone(), blobs.clone())
})
.collect::<Vec<_>>(); .collect::<Vec<_>>();
// Ensure the block will be rejected if imported in a chain segment. // Ensure the block will be rejected if imported in a chain segment.
assert!( assert!(
@ -483,7 +574,7 @@ async fn invalid_signature_block_proposal() {
#[tokio::test] #[tokio::test]
async fn invalid_signature_randao_reveal() { async fn invalid_signature_randao_reveal() {
let chain_segment = get_chain_segment().await; let (chain_segment, chain_segment_blobs) = get_chain_segment().await;
for &block_index in BLOCK_INDICES { for &block_index in BLOCK_INDICES {
let harness = get_invalid_sigs_harness(&chain_segment).await; let harness = get_invalid_sigs_harness(&chain_segment).await;
let mut snapshots = chain_segment.clone(); let mut snapshots = chain_segment.clone();
@ -497,13 +588,21 @@ async fn invalid_signature_randao_reveal() {
Arc::new(SignedBeaconBlock::from_block(block, signature)); Arc::new(SignedBeaconBlock::from_block(block, signature));
update_parent_roots(&mut snapshots); update_parent_roots(&mut snapshots);
update_proposal_signatures(&mut snapshots, &harness); update_proposal_signatures(&mut snapshots, &harness);
assert_invalid_signature(&chain_segment, &harness, block_index, &snapshots, "randao").await; assert_invalid_signature(
&chain_segment,
&chain_segment_blobs,
&harness,
block_index,
&snapshots,
"randao",
)
.await;
} }
} }
#[tokio::test] #[tokio::test]
async fn invalid_signature_proposer_slashing() { async fn invalid_signature_proposer_slashing() {
let chain_segment = get_chain_segment().await; let (chain_segment, chain_segment_blobs) = get_chain_segment().await;
for &block_index in BLOCK_INDICES { for &block_index in BLOCK_INDICES {
let harness = get_invalid_sigs_harness(&chain_segment).await; let harness = get_invalid_sigs_harness(&chain_segment).await;
let mut snapshots = chain_segment.clone(); let mut snapshots = chain_segment.clone();
@ -533,6 +632,7 @@ async fn invalid_signature_proposer_slashing() {
update_proposal_signatures(&mut snapshots, &harness); update_proposal_signatures(&mut snapshots, &harness);
assert_invalid_signature( assert_invalid_signature(
&chain_segment, &chain_segment,
&chain_segment_blobs,
&harness, &harness,
block_index, block_index,
&snapshots, &snapshots,
@ -544,7 +644,7 @@ async fn invalid_signature_proposer_slashing() {
#[tokio::test] #[tokio::test]
async fn invalid_signature_attester_slashing() { async fn invalid_signature_attester_slashing() {
let chain_segment = get_chain_segment().await; let (chain_segment, chain_segment_blobs) = get_chain_segment().await;
for &block_index in BLOCK_INDICES { for &block_index in BLOCK_INDICES {
let harness = get_invalid_sigs_harness(&chain_segment).await; let harness = get_invalid_sigs_harness(&chain_segment).await;
let mut snapshots = chain_segment.clone(); let mut snapshots = chain_segment.clone();
@ -585,6 +685,7 @@ async fn invalid_signature_attester_slashing() {
update_proposal_signatures(&mut snapshots, &harness); update_proposal_signatures(&mut snapshots, &harness);
assert_invalid_signature( assert_invalid_signature(
&chain_segment, &chain_segment,
&chain_segment_blobs,
&harness, &harness,
block_index, block_index,
&snapshots, &snapshots,
@ -596,7 +697,7 @@ async fn invalid_signature_attester_slashing() {
#[tokio::test] #[tokio::test]
async fn invalid_signature_attestation() { async fn invalid_signature_attestation() {
let chain_segment = get_chain_segment().await; let (chain_segment, chain_segment_blobs) = get_chain_segment().await;
let mut checked_attestation = false; let mut checked_attestation = false;
for &block_index in BLOCK_INDICES { for &block_index in BLOCK_INDICES {
@ -615,6 +716,7 @@ async fn invalid_signature_attestation() {
update_proposal_signatures(&mut snapshots, &harness); update_proposal_signatures(&mut snapshots, &harness);
assert_invalid_signature( assert_invalid_signature(
&chain_segment, &chain_segment,
&chain_segment_blobs,
&harness, &harness,
block_index, block_index,
&snapshots, &snapshots,
@ -633,7 +735,7 @@ async fn invalid_signature_attestation() {
#[tokio::test] #[tokio::test]
async fn invalid_signature_deposit() { async fn invalid_signature_deposit() {
let chain_segment = get_chain_segment().await; let (chain_segment, chain_segment_blobs) = get_chain_segment().await;
for &block_index in BLOCK_INDICES { for &block_index in BLOCK_INDICES {
// Note: an invalid deposit signature is permitted! // Note: an invalid deposit signature is permitted!
let harness = get_invalid_sigs_harness(&chain_segment).await; let harness = get_invalid_sigs_harness(&chain_segment).await;
@ -663,7 +765,10 @@ async fn invalid_signature_deposit() {
update_proposal_signatures(&mut snapshots, &harness); update_proposal_signatures(&mut snapshots, &harness);
let blocks: Vec<BlockWrapper<E>> = snapshots let blocks: Vec<BlockWrapper<E>> = snapshots
.iter() .iter()
.map(|snapshot| snapshot.beacon_block.clone().into()) .zip(chain_segment_blobs.iter())
.map(|(snapshot, blobs)| {
BlockWrapper::new(snapshot.beacon_block.clone(), blobs.clone())
})
.collect(); .collect();
assert!( assert!(
!matches!( !matches!(
@ -681,7 +786,7 @@ async fn invalid_signature_deposit() {
#[tokio::test] #[tokio::test]
async fn invalid_signature_exit() { async fn invalid_signature_exit() {
let chain_segment = get_chain_segment().await; let (chain_segment, chain_segment_blobs) = get_chain_segment().await;
for &block_index in BLOCK_INDICES { for &block_index in BLOCK_INDICES {
let harness = get_invalid_sigs_harness(&chain_segment).await; let harness = get_invalid_sigs_harness(&chain_segment).await;
let mut snapshots = chain_segment.clone(); let mut snapshots = chain_segment.clone();
@ -708,6 +813,7 @@ async fn invalid_signature_exit() {
update_proposal_signatures(&mut snapshots, &harness); update_proposal_signatures(&mut snapshots, &harness);
assert_invalid_signature( assert_invalid_signature(
&chain_segment, &chain_segment,
&chain_segment_blobs,
&harness, &harness,
block_index, block_index,
&snapshots, &snapshots,
@ -727,7 +833,7 @@ fn unwrap_err<T, E>(result: Result<T, E>) -> E {
#[tokio::test] #[tokio::test]
async fn block_gossip_verification() { async fn block_gossip_verification() {
let harness = get_harness(VALIDATOR_COUNT); let harness = get_harness(VALIDATOR_COUNT);
let chain_segment = get_chain_segment().await; let (chain_segment, chain_segment_blobs) = get_chain_segment_with_signed_blobs().await;
let block_index = CHAIN_SEGMENT_LENGTH - 2; let block_index = CHAIN_SEGMENT_LENGTH - 2;
@ -737,7 +843,10 @@ async fn block_gossip_verification() {
.set_slot(chain_segment[block_index].beacon_block.slot().as_u64()); .set_slot(chain_segment[block_index].beacon_block.slot().as_u64());
// Import the ancestors prior to the block we're testing. // Import the ancestors prior to the block we're testing.
for snapshot in &chain_segment[0..block_index] { for (snapshot, blobs_opt) in chain_segment[0..block_index]
.iter()
.zip(chain_segment_blobs.iter())
{
let gossip_verified = harness let gossip_verified = harness
.chain .chain
.verify_block_for_gossip(snapshot.beacon_block.clone().into()) .verify_block_for_gossip(snapshot.beacon_block.clone().into())
@ -753,6 +862,21 @@ async fn block_gossip_verification() {
) )
.await .await
.expect("should import valid gossip verified block"); .expect("should import valid gossip verified block");
if let Some(blobs) = blobs_opt {
for blob in blobs {
let blob_index = blob.message.index;
let gossip_verified = harness
.chain
.verify_blob_sidecar_for_gossip(blob.clone(), blob_index)
.expect("should obtain gossip verified blob");
harness
.chain
.process_blob(gossip_verified)
.await
.expect("should import valid gossip verified blob");
}
}
} }
// Recompute the head to ensure we cache the latest view of fork choice. // Recompute the head to ensure we cache the latest view of fork choice.
@ -1010,14 +1134,25 @@ async fn verify_block_for_gossip_slashing_detection() {
harness.advance_slot(); harness.advance_slot();
let state = harness.get_current_state(); let state = harness.get_current_state();
let ((block1, _), _) = harness.make_block(state.clone(), Slot::new(1)).await; let ((block1, blobs1), _) = harness.make_block(state.clone(), Slot::new(1)).await;
let ((block2, _), _) = harness.make_block(state, Slot::new(1)).await; let ((block2, _blobs2), _) = harness.make_block(state, Slot::new(1)).await;
let verified_block = harness let verified_block = harness
.chain .chain
.verify_block_for_gossip(Arc::new(block1).into()) .verify_block_for_gossip(Arc::new(block1).into())
.await .await
.unwrap(); .unwrap();
if let Some(blobs) = blobs1 {
for blob in blobs {
let blob_index = blob.message.index;
let verified_blob = harness
.chain
.verify_blob_sidecar_for_gossip(blob, blob_index)
.unwrap();
harness.chain.process_blob(verified_blob).await.unwrap();
}
}
harness harness
.chain .chain
.process_block( .process_block(

View File

@ -167,7 +167,7 @@ impl InvalidPayloadRig {
async fn build_blocks(&mut self, num_blocks: u64, is_valid: Payload) -> Vec<Hash256> { async fn build_blocks(&mut self, num_blocks: u64, is_valid: Payload) -> Vec<Hash256> {
let mut roots = Vec::with_capacity(num_blocks as usize); let mut roots = Vec::with_capacity(num_blocks as usize);
for _ in 0..num_blocks { for _ in 0..num_blocks {
roots.push(self.import_block(is_valid.clone()).await); roots.push(self.import_block(is_valid).await);
} }
roots roots
} }
@ -815,13 +815,16 @@ async fn switches_heads() {
}) })
.await; .await;
// The fork block should become the head. // NOTE: The `import_block` method above will cause the `ExecutionStatus` of the
assert_eq!(rig.harness.head_block_root(), fork_block_root); // `fork_block_root`'s payload to switch from `Optimistic` to `Invalid`. This means it *won't*
// be set as head, it's parent block will instead. This is an issue with the mock EL and/or
// the payload invalidation rig.
assert_eq!(rig.harness.head_block_root(), fork_parent_root);
// The fork block has not yet been validated. // The fork block has not yet been validated.
assert!(rig assert!(rig
.execution_status(fork_block_root) .execution_status(fork_block_root)
.is_strictly_optimistic()); .is_optimistic_or_invalid());
for root in blocks { for root in blocks {
let slot = rig let slot = rig
@ -1420,13 +1423,13 @@ async fn build_optimistic_chain(
.server .server
.all_get_block_by_hash_requests_return_natural_value(); .all_get_block_by_hash_requests_return_natural_value();
return rig; rig
} }
#[tokio::test] #[tokio::test]
async fn optimistic_transition_block_valid_unfinalized() { async fn optimistic_transition_block_valid_unfinalized() {
let ttd = 42; let ttd = 42;
let num_blocks = 16 as usize; let num_blocks = 16_usize;
let rig = build_optimistic_chain(ttd, ttd, num_blocks).await; let rig = build_optimistic_chain(ttd, ttd, num_blocks).await;
let post_transition_block_root = rig let post_transition_block_root = rig
@ -1480,7 +1483,7 @@ async fn optimistic_transition_block_valid_unfinalized() {
#[tokio::test] #[tokio::test]
async fn optimistic_transition_block_valid_finalized() { async fn optimistic_transition_block_valid_finalized() {
let ttd = 42; let ttd = 42;
let num_blocks = 130 as usize; let num_blocks = 130_usize;
let rig = build_optimistic_chain(ttd, ttd, num_blocks).await; let rig = build_optimistic_chain(ttd, ttd, num_blocks).await;
let post_transition_block_root = rig let post_transition_block_root = rig
@ -1535,7 +1538,7 @@ async fn optimistic_transition_block_valid_finalized() {
async fn optimistic_transition_block_invalid_unfinalized() { async fn optimistic_transition_block_invalid_unfinalized() {
let block_ttd = 42; let block_ttd = 42;
let rig_ttd = 1337; let rig_ttd = 1337;
let num_blocks = 22 as usize; let num_blocks = 22_usize;
let rig = build_optimistic_chain(block_ttd, rig_ttd, num_blocks).await; let rig = build_optimistic_chain(block_ttd, rig_ttd, num_blocks).await;
let post_transition_block_root = rig let post_transition_block_root = rig
@ -1611,7 +1614,7 @@ async fn optimistic_transition_block_invalid_unfinalized() {
async fn optimistic_transition_block_invalid_unfinalized_syncing_ee() { async fn optimistic_transition_block_invalid_unfinalized_syncing_ee() {
let block_ttd = 42; let block_ttd = 42;
let rig_ttd = 1337; let rig_ttd = 1337;
let num_blocks = 22 as usize; let num_blocks = 22_usize;
let rig = build_optimistic_chain(block_ttd, rig_ttd, num_blocks).await; let rig = build_optimistic_chain(block_ttd, rig_ttd, num_blocks).await;
let post_transition_block_root = rig let post_transition_block_root = rig
@ -1724,7 +1727,7 @@ async fn optimistic_transition_block_invalid_unfinalized_syncing_ee() {
async fn optimistic_transition_block_invalid_finalized() { async fn optimistic_transition_block_invalid_finalized() {
let block_ttd = 42; let block_ttd = 42;
let rig_ttd = 1337; let rig_ttd = 1337;
let num_blocks = 130 as usize; let num_blocks = 130_usize;
let rig = build_optimistic_chain(block_ttd, rig_ttd, num_blocks).await; let rig = build_optimistic_chain(block_ttd, rig_ttd, num_blocks).await;
let post_transition_block_root = rig let post_transition_block_root = rig

View File

@ -1,10 +1,12 @@
#![cfg(not(debug_assertions))] #![cfg(not(debug_assertions))]
use beacon_chain::attestation_verification::Error as AttnError; use beacon_chain::attestation_verification::Error as AttnError;
use beacon_chain::blob_verification::BlockWrapper;
use beacon_chain::builder::BeaconChainBuilder; use beacon_chain::builder::BeaconChainBuilder;
use beacon_chain::schema_change::migrate_schema; use beacon_chain::schema_change::migrate_schema;
use beacon_chain::test_utils::{ use beacon_chain::test_utils::{
test_spec, AttestationStrategy, BeaconChainHarness, BlockStrategy, DiskHarnessType, mock_execution_layer_from_parts, test_spec, AttestationStrategy, BeaconChainHarness,
BlockStrategy, DiskHarnessType,
}; };
use beacon_chain::validator_monitor::DEFAULT_INDIVIDUAL_TRACKING_THRESHOLD; use beacon_chain::validator_monitor::DEFAULT_INDIVIDUAL_TRACKING_THRESHOLD;
use beacon_chain::{ use beacon_chain::{
@ -12,7 +14,7 @@ use beacon_chain::{
migrate::MigratorConfig, BeaconChain, BeaconChainError, BeaconChainTypes, BeaconSnapshot, migrate::MigratorConfig, BeaconChain, BeaconChainError, BeaconChainTypes, BeaconSnapshot,
ChainConfig, NotifyExecutionLayer, ServerSentEventHandler, WhenSlotSkipped, ChainConfig, NotifyExecutionLayer, ServerSentEventHandler, WhenSlotSkipped,
}; };
use eth2_network_config::TRUSTED_SETUP; use eth2_network_config::get_trusted_setup;
use kzg::TrustedSetup; use kzg::TrustedSetup;
use lazy_static::lazy_static; use lazy_static::lazy_static;
use logging::test_logger; use logging::test_logger;
@ -2113,13 +2115,20 @@ async fn weak_subjectivity_sync() {
let store = get_store(&temp2); let store = get_store(&temp2);
let spec = test_spec::<E>(); let spec = test_spec::<E>();
let seconds_per_slot = spec.seconds_per_slot; let seconds_per_slot = spec.seconds_per_slot;
let trusted_setup: TrustedSetup = serde_json::from_reader(TRUSTED_SETUP) let trusted_setup: TrustedSetup =
serde_json::from_reader(get_trusted_setup::<<E as EthSpec>::Kzg>())
.map_err(|e| println!("Unable to read trusted setup file: {}", e)) .map_err(|e| println!("Unable to read trusted setup file: {}", e))
.unwrap(); .unwrap();
let mock = mock_execution_layer_from_parts(
&harness.spec,
harness.runtime.task_executor.clone(),
None,
None,
);
// Initialise a new beacon chain from the finalized checkpoint // Initialise a new beacon chain from the finalized checkpoint
let beacon_chain = Arc::new( let beacon_chain = BeaconChainBuilder::<DiskHarnessType<E>>::new(MinimalEthSpec)
BeaconChainBuilder::<DiskHarnessType<E>>::new(MinimalEthSpec)
.store(store.clone()) .store(store.clone())
.custom_spec(test_spec::<E>()) .custom_spec(test_spec::<E>())
.task_executor(harness.chain.task_executor.clone()) .task_executor(harness.chain.task_executor.clone())
@ -2137,11 +2146,13 @@ async fn weak_subjectivity_sync() {
log.clone(), log.clone(),
1, 1,
))) )))
.execution_layer(Some(mock.el))
.monitor_validators(true, vec![], DEFAULT_INDIVIDUAL_TRACKING_THRESHOLD, log) .monitor_validators(true, vec![], DEFAULT_INDIVIDUAL_TRACKING_THRESHOLD, log)
.trusted_setup(trusted_setup) .trusted_setup(trusted_setup)
.build() .build()
.expect("should build"), .expect("should build");
);
let beacon_chain = Arc::new(beacon_chain);
// Apply blocks forward to reach head. // Apply blocks forward to reach head.
let chain_dump = harness.chain.chain_dump().unwrap(); let chain_dump = harness.chain.chain_dump().unwrap();
@ -2150,12 +2161,14 @@ async fn weak_subjectivity_sync() {
assert_eq!(new_blocks[0].beacon_block.slot(), wss_slot + 1); assert_eq!(new_blocks[0].beacon_block.slot(), wss_slot + 1);
for snapshot in new_blocks { for snapshot in new_blocks {
let block_root = snapshot.beacon_block_root;
let full_block = harness let full_block = harness
.chain .chain
.get_block(&snapshot.beacon_block_root) .get_block(&snapshot.beacon_block_root)
.await .await
.unwrap() .unwrap()
.unwrap(); .unwrap();
let blobs = harness.chain.get_blobs(&block_root).expect("blobs");
let slot = full_block.slot(); let slot = full_block.slot();
let state_root = full_block.state_root(); let state_root = full_block.state_root();
@ -2163,7 +2176,7 @@ async fn weak_subjectivity_sync() {
beacon_chain beacon_chain
.process_block( .process_block(
full_block.canonical_root(), full_block.canonical_root(),
Arc::new(full_block), BlockWrapper::new(Arc::new(full_block), blobs),
NotifyExecutionLayer::Yes, NotifyExecutionLayer::Yes,
) )
.await .await
@ -2210,16 +2223,19 @@ async fn weak_subjectivity_sync() {
let mut available_blocks = vec![]; let mut available_blocks = vec![];
for blinded in historical_blocks { for blinded in historical_blocks {
let block_root = blinded.canonical_root();
let full_block = harness let full_block = harness
.chain .chain
.get_block(&blinded.canonical_root()) .get_block(&block_root)
.await .await
.expect("should get block") .expect("should get block")
.expect("should get block"); .expect("should get block");
let blobs = harness.chain.get_blobs(&block_root).expect("blobs");
if let MaybeAvailableBlock::Available(block) = harness if let MaybeAvailableBlock::Available(block) = harness
.chain .chain
.data_availability_checker .data_availability_checker
.check_availability(full_block.into()) .check_availability(BlockWrapper::new(Arc::new(full_block), blobs))
.expect("should check availability") .expect("should check availability")
{ {
available_blocks.push(block); available_blocks.push(block);
@ -2338,7 +2354,7 @@ async fn finalizes_after_resuming_from_db() {
.default_spec() .default_spec()
.keypairs(KEYPAIRS[0..validator_count].to_vec()) .keypairs(KEYPAIRS[0..validator_count].to_vec())
.resumed_disk_store(store) .resumed_disk_store(store)
.mock_execution_layer() .execution_layer(original_chain.execution_layer.clone())
.build(); .build();
assert_chains_pretty_much_the_same(&original_chain, &resumed_harness.chain); assert_chains_pretty_much_the_same(&original_chain, &resumed_harness.chain);

View File

@ -684,7 +684,7 @@ async fn run_skip_slot_test(skip_slots: u64) {
.chain .chain
.process_block( .process_block(
harness_a.chain.head_snapshot().beacon_block_root, harness_a.chain.head_snapshot().beacon_block_root,
harness_a.chain.head_snapshot().beacon_block.clone(), harness_a.get_head_block(),
NotifyExecutionLayer::Yes, NotifyExecutionLayer::Yes,
) )
.await .await

View File

@ -37,6 +37,7 @@ impl<T: EthSpec> ExecutionLayer<T> {
None None
}; };
let rlp_data_gas_used = payload.data_gas_used().ok();
let rlp_excess_data_gas = payload.excess_data_gas().ok(); let rlp_excess_data_gas = payload.excess_data_gas().ok();
// Construct the block header. // Construct the block header.
@ -45,6 +46,7 @@ impl<T: EthSpec> ExecutionLayer<T> {
KECCAK_EMPTY_LIST_RLP.as_fixed_bytes().into(), KECCAK_EMPTY_LIST_RLP.as_fixed_bytes().into(),
rlp_transactions_root, rlp_transactions_root,
rlp_withdrawals_root, rlp_withdrawals_root,
rlp_data_gas_used.copied(),
rlp_excess_data_gas.copied(), rlp_excess_data_gas.copied(),
); );
@ -97,6 +99,9 @@ pub fn rlp_encode_block_header(header: &ExecutionBlockHeader) -> Vec<u8> {
if let Some(withdrawals_root) = &header.withdrawals_root { if let Some(withdrawals_root) = &header.withdrawals_root {
rlp_header_stream.append(withdrawals_root); rlp_header_stream.append(withdrawals_root);
} }
if let Some(data_gas_used) = &header.data_gas_used {
rlp_header_stream.append(data_gas_used);
}
if let Some(excess_data_gas) = &header.excess_data_gas { if let Some(excess_data_gas) = &header.excess_data_gas {
rlp_header_stream.append(excess_data_gas); rlp_header_stream.append(excess_data_gas);
} }
@ -146,6 +151,7 @@ mod test {
nonce: Hash64::zero(), nonce: Hash64::zero(),
base_fee_per_gas: 0x036b_u64.into(), base_fee_per_gas: 0x036b_u64.into(),
withdrawals_root: None, withdrawals_root: None,
data_gas_used: None,
excess_data_gas: None, excess_data_gas: None,
}; };
let expected_rlp = "f90200a0e0a94a7a3c9617401586b1a27025d2d9671332d22d540e0af72b069170380f2aa01dcc4de8dec75d7aab85b567b6ccd41ad312451b948a7413f0a142fd40d4934794ba5e000000000000000000000000000000000000a0ec3c94b18b8a1cff7d60f8d258ec723312932928626b4c9355eb4ab3568ec7f7a050f738580ed699f0469702c7ccc63ed2e51bc034be9479b7bff4e68dee84accfa029b0562f7140574dd0d50dee8a271b22e1a0a7b78fca58f7c60370d8317ba2a9b9010000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000830200000188016345785d8a00008301553482079e42a0000000000000000000000000000000000000000000000000000000000000000088000000000000000082036b"; let expected_rlp = "f90200a0e0a94a7a3c9617401586b1a27025d2d9671332d22d540e0af72b069170380f2aa01dcc4de8dec75d7aab85b567b6ccd41ad312451b948a7413f0a142fd40d4934794ba5e000000000000000000000000000000000000a0ec3c94b18b8a1cff7d60f8d258ec723312932928626b4c9355eb4ab3568ec7f7a050f738580ed699f0469702c7ccc63ed2e51bc034be9479b7bff4e68dee84accfa029b0562f7140574dd0d50dee8a271b22e1a0a7b78fca58f7c60370d8317ba2a9b9010000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000830200000188016345785d8a00008301553482079e42a0000000000000000000000000000000000000000000000000000000000000000088000000000000000082036b";
@ -175,6 +181,7 @@ mod test {
nonce: Hash64::zero(), nonce: Hash64::zero(),
base_fee_per_gas: 0x036b_u64.into(), base_fee_per_gas: 0x036b_u64.into(),
withdrawals_root: None, withdrawals_root: None,
data_gas_used: None,
excess_data_gas: None, excess_data_gas: None,
}; };
let expected_rlp = "f901fda0927ca537f06c783a3a2635b8805eef1c8c2124f7444ad4a3389898dd832f2dbea01dcc4de8dec75d7aab85b567b6ccd41ad312451b948a7413f0a142fd40d4934794ba5e000000000000000000000000000000000000a0e97859b065bd8dbbb4519c7cb935024de2484c2b7f881181b4360492f0b06b82a050f738580ed699f0469702c7ccc63ed2e51bc034be9479b7bff4e68dee84accfa029b0562f7140574dd0d50dee8a271b22e1a0a7b78fca58f7c60370d8317ba2a9b9010000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000800188016345785d8a00008301553482079e42a0000000000000000000000000000000000000000000000000000000000002000088000000000000000082036b"; let expected_rlp = "f901fda0927ca537f06c783a3a2635b8805eef1c8c2124f7444ad4a3389898dd832f2dbea01dcc4de8dec75d7aab85b567b6ccd41ad312451b948a7413f0a142fd40d4934794ba5e000000000000000000000000000000000000a0e97859b065bd8dbbb4519c7cb935024de2484c2b7f881181b4360492f0b06b82a050f738580ed699f0469702c7ccc63ed2e51bc034be9479b7bff4e68dee84accfa029b0562f7140574dd0d50dee8a271b22e1a0a7b78fca58f7c60370d8317ba2a9b9010000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000800188016345785d8a00008301553482079e42a0000000000000000000000000000000000000000000000000000000000002000088000000000000000082036b";
@ -205,6 +212,7 @@ mod test {
nonce: Hash64::zero(), nonce: Hash64::zero(),
base_fee_per_gas: 0x34187b238_u64.into(), base_fee_per_gas: 0x34187b238_u64.into(),
withdrawals_root: None, withdrawals_root: None,
data_gas_used: None,
excess_data_gas: None, excess_data_gas: None,
}; };
let expected_hash = let expected_hash =

View File

@ -7,7 +7,7 @@ use crate::http::{
}; };
use crate::BlobTxConversionError; use crate::BlobTxConversionError;
use eth2::types::{SsePayloadAttributes, SsePayloadAttributesV1, SsePayloadAttributesV2}; use eth2::types::{SsePayloadAttributes, SsePayloadAttributesV1, SsePayloadAttributesV2};
pub use ethers_core::types::Transaction; use ethers_core::types::Transaction;
use ethers_core::utils::rlp::{self, Decodable, Rlp}; use ethers_core::utils::rlp::{self, Decodable, Rlp};
use http::deposit_methods::RpcError; use http::deposit_methods::RpcError;
pub use json_structures::{JsonWithdrawal, TransitionConfigurationV1}; pub use json_structures::{JsonWithdrawal, TransitionConfigurationV1};
@ -190,8 +190,11 @@ pub struct ExecutionBlockWithTransactions<T: EthSpec> {
#[superstruct(only(Capella, Deneb))] #[superstruct(only(Capella, Deneb))]
pub withdrawals: Vec<JsonWithdrawal>, pub withdrawals: Vec<JsonWithdrawal>,
#[superstruct(only(Deneb))] #[superstruct(only(Deneb))]
#[serde(with = "serde_utils::u256_hex_be")] #[serde(with = "serde_utils::u64_hex_be")]
pub excess_data_gas: Uint256, pub data_gas_used: u64,
#[superstruct(only(Deneb))]
#[serde(with = "serde_utils::u64_hex_be")]
pub excess_data_gas: u64,
} }
impl<T: EthSpec> TryFrom<ExecutionPayload<T>> for ExecutionBlockWithTransactions<T> { impl<T: EthSpec> TryFrom<ExecutionPayload<T>> for ExecutionBlockWithTransactions<T> {
@ -268,6 +271,7 @@ impl<T: EthSpec> TryFrom<ExecutionPayload<T>> for ExecutionBlockWithTransactions
.into_iter() .into_iter()
.map(|withdrawal| withdrawal.into()) .map(|withdrawal| withdrawal.into())
.collect(), .collect(),
data_gas_used: block.data_gas_used,
excess_data_gas: block.excess_data_gas, excess_data_gas: block.excess_data_gas,
}), }),
}; };
@ -510,6 +514,7 @@ impl<E: EthSpec> ExecutionPayloadBodyV1<E> {
block_hash: header.block_hash, block_hash: header.block_hash,
transactions: self.transactions, transactions: self.transactions,
withdrawals, withdrawals,
data_gas_used: header.data_gas_used,
excess_data_gas: header.excess_data_gas, excess_data_gas: header.excess_data_gas,
})) }))
} else { } else {

View File

@ -11,7 +11,7 @@ use std::collections::HashSet;
use tokio::sync::Mutex; use tokio::sync::Mutex;
use std::time::{Duration, Instant}; use std::time::{Duration, Instant};
use types::EthSpec; use types::{EthSpec, VersionedHash};
pub use deposit_log::{DepositLog, Log}; pub use deposit_log::{DepositLog, Log};
pub use reqwest::Client; pub use reqwest::Client;
@ -807,9 +807,13 @@ impl HttpJsonRpc {
pub async fn new_payload_v3<T: EthSpec>( pub async fn new_payload_v3<T: EthSpec>(
&self, &self,
execution_payload: ExecutionPayload<T>, execution_payload: ExecutionPayloadDeneb<T>,
versioned_hashes: Vec<VersionedHash>,
) -> Result<PayloadStatusV1, Error> { ) -> Result<PayloadStatusV1, Error> {
let params = json!([JsonExecutionPayload::from(execution_payload)]); let params = json!([
JsonExecutionPayload::V3(execution_payload.into()),
versioned_hashes
]);
let response: JsonPayloadStatusV1 = self let response: JsonPayloadStatusV1 = self
.rpc_request( .rpc_request(
@ -887,26 +891,6 @@ impl HttpJsonRpc {
let params = json!([JsonPayloadIdRequest::from(payload_id)]); let params = json!([JsonPayloadIdRequest::from(payload_id)]);
match fork_name { match fork_name {
ForkName::Merge => {
let response: JsonGetPayloadResponseV1<T> = self
.rpc_request(
ENGINE_GET_PAYLOAD_V2,
params,
ENGINE_GET_PAYLOAD_TIMEOUT * self.execution_timeout_multiplier,
)
.await?;
Ok(JsonGetPayloadResponse::V1(response).into())
}
ForkName::Capella => {
let response: JsonGetPayloadResponseV2<T> = self
.rpc_request(
ENGINE_GET_PAYLOAD_V2,
params,
ENGINE_GET_PAYLOAD_TIMEOUT * self.execution_timeout_multiplier,
)
.await?;
Ok(JsonGetPayloadResponse::V2(response).into())
}
ForkName::Deneb => { ForkName::Deneb => {
let response: JsonGetPayloadResponseV3<T> = self let response: JsonGetPayloadResponseV3<T> = self
.rpc_request( .rpc_request(
@ -917,7 +901,7 @@ impl HttpJsonRpc {
.await?; .await?;
Ok(JsonGetPayloadResponse::V3(response).into()) Ok(JsonGetPayloadResponse::V3(response).into())
} }
ForkName::Base | ForkName::Altair => Err(Error::UnsupportedForkVariant(format!( _ => Err(Error::UnsupportedForkVariant(format!(
"called get_payload_v3 with {}", "called get_payload_v3 with {}",
fork_name fork_name
))), ))),
@ -1099,11 +1083,12 @@ impl HttpJsonRpc {
pub async fn new_payload<T: EthSpec>( pub async fn new_payload<T: EthSpec>(
&self, &self,
execution_payload: ExecutionPayload<T>, execution_payload: ExecutionPayload<T>,
versioned_hashes_opt: Option<Vec<VersionedHash>>,
) -> Result<PayloadStatusV1, Error> { ) -> Result<PayloadStatusV1, Error> {
let engine_capabilities = self.get_engine_capabilities(None).await?; let engine_capabilities = self.get_engine_capabilities(None).await?;
if engine_capabilities.new_payload_v3 { match execution_payload {
self.new_payload_v3(execution_payload).await ExecutionPayload::Merge(_) | ExecutionPayload::Capella(_) => {
} else if engine_capabilities.new_payload_v2 { if engine_capabilities.new_payload_v2 {
self.new_payload_v2(execution_payload).await self.new_payload_v2(execution_payload).await
} else if engine_capabilities.new_payload_v1 { } else if engine_capabilities.new_payload_v1 {
self.new_payload_v1(execution_payload).await self.new_payload_v1(execution_payload).await
@ -1111,6 +1096,19 @@ impl HttpJsonRpc {
Err(Error::RequiredMethodUnsupported("engine_newPayload")) Err(Error::RequiredMethodUnsupported("engine_newPayload"))
} }
} }
ExecutionPayload::Deneb(execution_payload_deneb) => {
let Some(versioned_hashes) = versioned_hashes_opt else {
return Err(Error::IncorrectStateVariant);
};
if engine_capabilities.new_payload_v3 {
self.new_payload_v3(execution_payload_deneb, versioned_hashes)
.await
} else {
Err(Error::RequiredMethodUnsupported("engine_newPayloadV3"))
}
}
}
}
// automatically selects the latest version of // automatically selects the latest version of
// get_payload that the execution engine supports // get_payload that the execution engine supports
@ -1120,9 +1118,9 @@ impl HttpJsonRpc {
payload_id: PayloadId, payload_id: PayloadId,
) -> Result<GetPayloadResponse<T>, Error> { ) -> Result<GetPayloadResponse<T>, Error> {
let engine_capabilities = self.get_engine_capabilities(None).await?; let engine_capabilities = self.get_engine_capabilities(None).await?;
if engine_capabilities.get_payload_v3 { match fork_name {
self.get_payload_v3(fork_name, payload_id).await ForkName::Merge | ForkName::Capella => {
} else if engine_capabilities.get_payload_v2 { if engine_capabilities.get_payload_v2 {
self.get_payload_v2(fork_name, payload_id).await self.get_payload_v2(fork_name, payload_id).await
} else if engine_capabilities.new_payload_v1 { } else if engine_capabilities.new_payload_v1 {
self.get_payload_v1(payload_id).await self.get_payload_v1(payload_id).await
@ -1130,6 +1128,19 @@ impl HttpJsonRpc {
Err(Error::RequiredMethodUnsupported("engine_getPayload")) Err(Error::RequiredMethodUnsupported("engine_getPayload"))
} }
} }
ForkName::Deneb => {
if engine_capabilities.get_payload_v3 {
self.get_payload_v3(fork_name, payload_id).await
} else {
Err(Error::RequiredMethodUnsupported("engine_getPayloadV3"))
}
}
ForkName::Base | ForkName::Altair => Err(Error::UnsupportedForkVariant(format!(
"called get_payload with {}",
fork_name
))),
}
}
// automatically selects the latest version of // automatically selects the latest version of
// forkchoice_updated that the execution engine supports // forkchoice_updated that the execution engine supports

View File

@ -99,8 +99,11 @@ pub struct JsonExecutionPayload<T: EthSpec> {
#[superstruct(only(V2, V3))] #[superstruct(only(V2, V3))]
pub withdrawals: VariableList<JsonWithdrawal, T::MaxWithdrawalsPerPayload>, pub withdrawals: VariableList<JsonWithdrawal, T::MaxWithdrawalsPerPayload>,
#[superstruct(only(V3))] #[superstruct(only(V3))]
#[serde(with = "serde_utils::u256_hex_be")] #[serde(with = "serde_utils::u64_hex_be")]
pub excess_data_gas: Uint256, pub data_gas_used: u64,
#[superstruct(only(V3))]
#[serde(with = "serde_utils::u64_hex_be")]
pub excess_data_gas: u64,
} }
impl<T: EthSpec> From<ExecutionPayloadMerge<T>> for JsonExecutionPayloadV1<T> { impl<T: EthSpec> From<ExecutionPayloadMerge<T>> for JsonExecutionPayloadV1<T> {
@ -172,6 +175,7 @@ impl<T: EthSpec> From<ExecutionPayloadDeneb<T>> for JsonExecutionPayloadV3<T> {
.map(Into::into) .map(Into::into)
.collect::<Vec<_>>() .collect::<Vec<_>>()
.into(), .into(),
data_gas_used: payload.data_gas_used,
excess_data_gas: payload.excess_data_gas, excess_data_gas: payload.excess_data_gas,
} }
} }
@ -256,6 +260,7 @@ impl<T: EthSpec> From<JsonExecutionPayloadV3<T>> for ExecutionPayloadDeneb<T> {
.map(Into::into) .map(Into::into)
.collect::<Vec<_>>() .collect::<Vec<_>>()
.into(), .into(),
data_gas_used: payload.data_gas_used,
excess_data_gas: payload.excess_data_gas, excess_data_gas: payload.excess_data_gas,
} }
} }

View File

@ -15,8 +15,7 @@ use engines::{Engine, EngineError};
pub use engines::{EngineState, ForkchoiceState}; pub use engines::{EngineState, ForkchoiceState};
use eth2::types::{builder_bid::SignedBuilderBid, ForkVersionedResponse}; use eth2::types::{builder_bid::SignedBuilderBid, ForkVersionedResponse};
use ethers_core::abi::ethereum_types::FromStrRadixErr; use ethers_core::abi::ethereum_types::FromStrRadixErr;
use ethers_core::types::transaction::eip2930::AccessListItem; use ethers_core::types::Transaction as EthersTransaction;
use ethers_core::types::{Transaction as EthersTransaction, U64};
use fork_choice::ForkchoiceUpdateParameters; use fork_choice::ForkchoiceUpdateParameters;
use lru::LruCache; use lru::LruCache;
use payload_status::process_payload_status; use payload_status::process_payload_status;
@ -25,7 +24,6 @@ use sensitive_url::SensitiveUrl;
use serde::{Deserialize, Serialize}; use serde::{Deserialize, Serialize};
use slog::{crit, debug, error, info, trace, warn, Logger}; use slog::{crit, debug, error, info, trace, warn, Logger};
use slot_clock::SlotClock; use slot_clock::SlotClock;
use ssz::Encode;
use std::collections::HashMap; use std::collections::HashMap;
use std::fmt; use std::fmt;
use std::future::Future; use std::future::Future;
@ -43,8 +41,6 @@ use tokio_stream::wrappers::WatchStream;
use tree_hash::TreeHash; use tree_hash::TreeHash;
use types::beacon_block_body::KzgCommitments; use types::beacon_block_body::KzgCommitments;
use types::blob_sidecar::Blobs; use types::blob_sidecar::Blobs;
use types::consts::deneb::BLOB_TX_TYPE;
use types::transaction::{AccessTuple, BlobTransaction, EcdsaSignature, SignedBlobTransaction};
use types::{AbstractExecPayload, BeaconStateError, ExecPayload, VersionedHash}; use types::{AbstractExecPayload, BeaconStateError, ExecPayload, VersionedHash};
use types::{ use types::{
BlindedPayload, BlockType, ChainSpec, Epoch, ExecutionBlockHash, ExecutionPayload, BlindedPayload, BlockType, ChainSpec, Epoch, ExecutionBlockHash, ExecutionPayload,
@ -1217,6 +1213,7 @@ impl<T: EthSpec> ExecutionLayer<T> {
pub async fn notify_new_payload( pub async fn notify_new_payload(
&self, &self,
execution_payload: &ExecutionPayload<T>, execution_payload: &ExecutionPayload<T>,
versioned_hashes: Option<Vec<VersionedHash>>,
) -> Result<PayloadStatus, Error> { ) -> Result<PayloadStatus, Error> {
let _timer = metrics::start_timer_vec( let _timer = metrics::start_timer_vec(
&metrics::EXECUTION_LAYER_REQUEST_TIMES, &metrics::EXECUTION_LAYER_REQUEST_TIMES,
@ -1233,7 +1230,11 @@ impl<T: EthSpec> ExecutionLayer<T> {
let result = self let result = self
.engine() .engine()
.request(|engine| engine.api.new_payload(execution_payload.clone())) .request(|engine| {
engine
.api
.new_payload(execution_payload.clone(), versioned_hashes)
})
.await; .await;
if let Ok(status) = &result { if let Ok(status) = &result {
@ -1786,7 +1787,7 @@ impl<T: EthSpec> ExecutionLayer<T> {
VariableList::new( VariableList::new(
transactions transactions
.into_iter() .into_iter()
.map(ethers_tx_to_bytes::<T>) .map(ethers_tx_to_ssz::<T>)
.collect::<Result<Vec<_>, BlobTxConversionError>>()?, .collect::<Result<Vec<_>, BlobTxConversionError>>()?,
) )
.map_err(BlobTxConversionError::SszError) .map_err(BlobTxConversionError::SszError)
@ -1863,6 +1864,7 @@ impl<T: EthSpec> ExecutionLayer<T> {
block_hash: deneb_block.block_hash, block_hash: deneb_block.block_hash,
transactions: convert_transactions(deneb_block.transactions)?, transactions: convert_transactions(deneb_block.transactions)?,
withdrawals, withdrawals,
data_gas_used: deneb_block.data_gas_used,
excess_data_gas: deneb_block.excess_data_gas, excess_data_gas: deneb_block.excess_data_gas,
}) })
} }
@ -2170,159 +2172,35 @@ impl From<serde_json::Error> for BlobTxConversionError {
} }
} }
/// A utility function to convert a `ethers-rs` `Transaction` into the correct bytes encoding based fn random_valid_tx<T: EthSpec>(
/// on transaction type. That means RLP encoding if this is a transaction other than a
/// `BLOB_TX_TYPE` transaction in which case, SSZ encoding will be used.
fn ethers_tx_to_bytes<T: EthSpec>(
transaction: EthersTransaction,
) -> Result<Transaction<T::MaxBytesPerTransaction>, BlobTxConversionError> { ) -> Result<Transaction<T::MaxBytesPerTransaction>, BlobTxConversionError> {
let tx_type = transaction // Calculate transaction bytes. We don't care about the contents of the transaction.
.transaction_type let transaction: EthersTransaction = serde_json::from_str(
.ok_or(BlobTxConversionError::NoTransactionType)? r#"{
.as_u64(); "blockHash":"0x1d59ff54b1eb26b013ce3cb5fc9dab3705b415a67127a003c3e61eb445bb8df2",
"blockNumber":"0x5daf3b",
let tx = if BLOB_TX_TYPE as u64 == tx_type { "from":"0xa7d9ddbe1f17865597fbd27ec712455208b6b76d",
let EthersTransaction { "gas":"0xc350",
hash: _, "gasPrice":"0x4a817c800",
nonce, "hash":"0x88df016429689c079f3b2f6ad39fa052532c56795b733da78a91ebe6a713944b",
block_hash: _, "input":"0x68656c6c6f21",
block_number: _, "nonce":"0x15",
transaction_index: _, "to":"0xf02c1c8e6114b1dbe8937a39260b5b0a374432bb",
from: _, "transactionIndex":"0x41",
to, "value":"0xf3dbb76162000",
value, "v":"0x25",
gas_price: _, "r":"0x1b5e176d927f8e9ab405058b2d2457392da3e20f328b16ddabcebc33eaac5fea",
gas, "s":"0x4ba69724e8f69de52f0125ad8b3c5c2cef33019bac3249e2c0a2192766d1721c"
input, }"#,
v,
r,
s,
transaction_type: _,
access_list,
max_priority_fee_per_gas,
max_fee_per_gas,
chain_id,
other,
} = transaction;
// ******************** BlobTransaction fields ********************
// chainId
let chain_id = chain_id.ok_or(BlobTxConversionError::NoChainId)?;
// nonce
let nonce = if nonce > Uint256::from(u64::MAX) {
return Err(BlobTxConversionError::NonceTooLarge);
} else {
nonce.as_u64()
};
// maxPriorityFeePerGas
let max_priority_fee_per_gas =
max_priority_fee_per_gas.ok_or(BlobTxConversionError::MaxPriorityFeePerGasMissing)?;
// maxFeePerGas
let max_fee_per_gas = max_fee_per_gas.ok_or(BlobTxConversionError::MaxFeePerGasMissing)?;
// gas
let gas = if gas > Uint256::from(u64::MAX) {
return Err(BlobTxConversionError::GasTooHigh);
} else {
gas.as_u64()
};
// data (a.k.a input)
let data = VariableList::new(input.to_vec())?;
// accessList
let access_list = VariableList::new(
access_list
.ok_or(BlobTxConversionError::AccessListMissing)?
.0
.into_iter()
.map(|access_tuple| {
let AccessListItem {
address,
storage_keys,
} = access_tuple;
Ok(AccessTuple {
address,
storage_keys: VariableList::new(storage_keys)?,
})
})
.collect::<Result<Vec<AccessTuple>, BlobTxConversionError>>()?,
)?;
// ******************** BlobTransaction `other` fields ********************
//
// Here we use the `other` field in the `ethers-rs` `Transaction` type because
// `ethers-rs` does not yet support SSZ and therefore the blobs transaction type.
// maxFeePerDataGas
let max_fee_per_data_gas = Uint256::from_str_radix(
other
.get("maxFeePerDataGas")
.ok_or(BlobTxConversionError::MaxFeePerDataGasMissing)?
.as_str()
.ok_or(BlobTxConversionError::MaxFeePerDataGasMissing)?,
16,
) )
.map_err(BlobTxConversionError::FromStrRadix)?; .unwrap();
ethers_tx_to_ssz::<T>(transaction)
// versionedHashes
let versioned_hashes = other
.get("versionedHashes")
.ok_or(BlobTxConversionError::VersionedHashesMissing)?
.as_array()
.ok_or(BlobTxConversionError::VersionedHashesMissing)?
.iter()
.map(|versioned_hash| {
let hash_bytes = serde_utils::hex::decode(
versioned_hash
.as_str()
.ok_or(BlobTxConversionError::VersionedHashesMissing)?,
)
.map_err(BlobTxConversionError::FromHex)?;
if hash_bytes.len() != Hash256::ssz_fixed_len() {
Err(BlobTxConversionError::InvalidVersionedHashBytesLen)
} else {
Ok(Hash256::from_slice(&hash_bytes))
} }
})
.collect::<Result<Vec<VersionedHash>, BlobTxConversionError>>()?;
let message = BlobTransaction {
chain_id,
nonce,
max_priority_fee_per_gas,
max_fee_per_gas,
gas,
to,
value,
data,
access_list,
max_fee_per_data_gas,
versioned_hashes: VariableList::new(versioned_hashes)?,
};
// ******************** EcdsaSignature fields ******************** fn ethers_tx_to_ssz<T: EthSpec>(
tx: EthersTransaction,
let y_parity = if v == U64::zero() { ) -> Result<Transaction<T::MaxBytesPerTransaction>, BlobTxConversionError> {
false VariableList::new(tx.rlp().to_vec()).map_err(Into::into)
} else if v == U64::one() {
true
} else {
return Err(BlobTxConversionError::InvalidYParity);
};
let signature = EcdsaSignature { y_parity, r, s };
// The `BLOB_TX_TYPE` should prepend the SSZ encoded `SignedBlobTransaction`.
let mut signed_tx = SignedBlobTransaction { message, signature }.as_ssz_bytes();
signed_tx.insert(0, BLOB_TX_TYPE);
signed_tx
} else {
transaction.rlp().to_vec()
};
VariableList::new(tx).map_err(Into::into)
} }
fn noop<T: EthSpec>( fn noop<T: EthSpec>(

View File

@ -6,24 +6,23 @@ use crate::{
}, },
ExecutionBlock, PayloadAttributes, PayloadId, PayloadStatusV1, PayloadStatusV1Status, ExecutionBlock, PayloadAttributes, PayloadId, PayloadStatusV1, PayloadStatusV1Status,
}, },
BlobsBundleV1, ExecutionBlockWithTransactions, random_valid_tx, BlobsBundleV1, ExecutionBlockWithTransactions,
}; };
use kzg::{Kzg, BYTES_PER_BLOB, BYTES_PER_FIELD_ELEMENT, FIELD_ELEMENTS_PER_BLOB}; use kzg::Kzg;
use rand::RngCore; use rand::thread_rng;
use serde::{Deserialize, Serialize}; use serde::{Deserialize, Serialize};
use ssz::Encode;
use std::collections::HashMap; use std::collections::HashMap;
use std::sync::Arc; use std::sync::Arc;
use tree_hash::TreeHash; use tree_hash::TreeHash;
use tree_hash_derive::TreeHash; use tree_hash_derive::TreeHash;
use types::consts::deneb::BLOB_TX_TYPE;
use types::transaction::{BlobTransaction, EcdsaSignature, SignedBlobTransaction};
use types::{ use types::{
Blob, EthSpec, ExecutionBlockHash, ExecutionPayload, ExecutionPayloadCapella, BlobSidecar, ChainSpec, EthSpec, ExecutionBlockHash, ExecutionPayload, ExecutionPayloadCapella,
ExecutionPayloadDeneb, ExecutionPayloadMerge, ForkName, Hash256, Transaction, Transactions, ExecutionPayloadDeneb, ExecutionPayloadHeader, ExecutionPayloadMerge, ForkName, Hash256,
Uint256, Transactions, Uint256,
}; };
use super::DEFAULT_TERMINAL_BLOCK;
const GAS_LIMIT: u64 = 16384; const GAS_LIMIT: u64 = 16384;
const GAS_USED: u64 = GAS_LIMIT - 1; const GAS_USED: u64 = GAS_LIMIT - 1;
@ -125,12 +124,12 @@ pub struct ExecutionBlockGenerator<T: EthSpec> {
* Post-merge fork triggers * Post-merge fork triggers
*/ */
pub shanghai_time: Option<u64>, // withdrawals pub shanghai_time: Option<u64>, // withdrawals
pub deneb_time: Option<u64>, // 4844 pub cancun_time: Option<u64>, // deneb
/* /*
* deneb stuff * deneb stuff
*/ */
pub blobs_bundles: HashMap<PayloadId, BlobsBundleV1<T>>, pub blobs_bundles: HashMap<PayloadId, BlobsBundleV1<T>>,
pub kzg: Option<Arc<Kzg>>, pub kzg: Option<Arc<Kzg<T::Kzg>>>,
} }
impl<T: EthSpec> ExecutionBlockGenerator<T> { impl<T: EthSpec> ExecutionBlockGenerator<T> {
@ -139,8 +138,8 @@ impl<T: EthSpec> ExecutionBlockGenerator<T> {
terminal_block_number: u64, terminal_block_number: u64,
terminal_block_hash: ExecutionBlockHash, terminal_block_hash: ExecutionBlockHash,
shanghai_time: Option<u64>, shanghai_time: Option<u64>,
deneb_time: Option<u64>, cancun_time: Option<u64>,
kzg: Option<Kzg>, kzg: Option<Kzg<T::Kzg>>,
) -> Self { ) -> Self {
let mut gen = Self { let mut gen = Self {
head_block: <_>::default(), head_block: <_>::default(),
@ -154,7 +153,7 @@ impl<T: EthSpec> ExecutionBlockGenerator<T> {
next_payload_id: 0, next_payload_id: 0,
payload_ids: <_>::default(), payload_ids: <_>::default(),
shanghai_time, shanghai_time,
deneb_time, cancun_time,
blobs_bundles: <_>::default(), blobs_bundles: <_>::default(),
kzg: kzg.map(Arc::new), kzg: kzg.map(Arc::new),
}; };
@ -189,7 +188,7 @@ impl<T: EthSpec> ExecutionBlockGenerator<T> {
} }
pub fn get_fork_at_timestamp(&self, timestamp: u64) -> ForkName { pub fn get_fork_at_timestamp(&self, timestamp: u64) -> ForkName {
match self.deneb_time { match self.cancun_time {
Some(fork_time) if timestamp >= fork_time => ForkName::Deneb, Some(fork_time) if timestamp >= fork_time => ForkName::Deneb,
_ => match self.shanghai_time { _ => match self.shanghai_time {
Some(fork_time) if timestamp >= fork_time => ForkName::Capella, Some(fork_time) if timestamp >= fork_time => ForkName::Capella,
@ -270,10 +269,15 @@ impl<T: EthSpec> ExecutionBlockGenerator<T> {
finalized_block_hash finalized_block_hash
)); ));
} }
let parent_hash = if block_number == 0 { let block = if block_number == 0 {
ExecutionBlockHash::zero() generate_genesis_block(self.terminal_total_difficulty, self.terminal_block_number)?
} else if let Some(block) = self.block_by_number(block_number - 1) { } else if let Some(block) = self.block_by_number(block_number - 1) {
block.block_hash() generate_pow_block(
self.terminal_total_difficulty,
self.terminal_block_number,
block_number,
block.block_hash(),
)?
} else { } else {
return Err(format!( return Err(format!(
"parent with block number {} not found", "parent with block number {} not found",
@ -281,13 +285,6 @@ impl<T: EthSpec> ExecutionBlockGenerator<T> {
)); ));
}; };
let block = generate_pow_block(
self.terminal_total_difficulty,
self.terminal_block_number,
block_number,
parent_hash,
)?;
// Insert block into block tree // Insert block into block tree
self.insert_block(Block::PoW(block))?; self.insert_block(Block::PoW(block))?;
@ -348,10 +345,10 @@ impl<T: EthSpec> ExecutionBlockGenerator<T> {
Ok(hash) Ok(hash)
} }
// This does not reject duplicate blocks inserted. This lets us re-use the same execution
// block generator for multiple beacon chains which is useful in testing.
pub fn insert_block(&mut self, block: Block<T>) -> Result<ExecutionBlockHash, String> { pub fn insert_block(&mut self, block: Block<T>) -> Result<ExecutionBlockHash, String> {
if self.blocks.contains_key(&block.block_hash()) { if block.parent_hash() != ExecutionBlockHash::zero()
return Err(format!("{:?} is already known", block.block_hash()));
} else if block.parent_hash() != ExecutionBlockHash::zero()
&& !self.blocks.contains_key(&block.parent_hash()) && !self.blocks.contains_key(&block.parent_hash())
{ {
return Err(format!("parent block {:?} is unknown", block.parent_hash())); return Err(format!("parent block {:?} is unknown", block.parent_hash()));
@ -410,8 +407,7 @@ impl<T: EthSpec> ExecutionBlockGenerator<T> {
} }
pub fn get_blobs_bundle(&mut self, id: &PayloadId) -> Option<BlobsBundleV1<T>> { pub fn get_blobs_bundle(&mut self, id: &PayloadId) -> Option<BlobsBundleV1<T>> {
// remove it to free memory self.blobs_bundles.get(id).cloned()
self.blobs_bundles.remove(id)
} }
pub fn new_payload(&mut self, payload: ExecutionPayload<T>) -> PayloadStatusV1 { pub fn new_payload(&mut self, payload: ExecutionPayload<T>) -> PayloadStatusV1 {
@ -450,6 +446,14 @@ impl<T: EthSpec> ExecutionBlockGenerator<T> {
forkchoice_state: ForkchoiceState, forkchoice_state: ForkchoiceState,
payload_attributes: Option<PayloadAttributes>, payload_attributes: Option<PayloadAttributes>,
) -> Result<JsonForkchoiceUpdatedV1Response, String> { ) -> Result<JsonForkchoiceUpdatedV1Response, String> {
// This is meant to cover starting post-merge transition at genesis. Useful for
// testing Capella forks and later.
if let Some(genesis_pow_block) = self.block_by_number(0) {
if genesis_pow_block.block_hash() == forkchoice_state.head_block_hash {
self.terminal_block_hash = forkchoice_state.head_block_hash;
}
}
if let Some(payload) = self if let Some(payload) = self
.pending_payloads .pending_payloads
.remove(&forkchoice_state.head_block_hash) .remove(&forkchoice_state.head_block_hash)
@ -518,8 +522,7 @@ impl<T: EthSpec> ExecutionBlockGenerator<T> {
block_hash: ExecutionBlockHash::zero(), block_hash: ExecutionBlockHash::zero(),
transactions: vec![].into(), transactions: vec![].into(),
}), }),
PayloadAttributes::V2(pa) => { PayloadAttributes::V2(pa) => match self.get_fork_at_timestamp(pa.timestamp) {
match self.get_fork_at_timestamp(pa.timestamp) {
ForkName::Merge => ExecutionPayload::Merge(ExecutionPayloadMerge { ForkName::Merge => ExecutionPayload::Merge(ExecutionPayloadMerge {
parent_hash: forkchoice_state.head_block_hash, parent_hash: forkchoice_state.head_block_hash,
fee_recipient: pa.suggested_fee_recipient, fee_recipient: pa.suggested_fee_recipient,
@ -536,8 +539,7 @@ impl<T: EthSpec> ExecutionBlockGenerator<T> {
block_hash: ExecutionBlockHash::zero(), block_hash: ExecutionBlockHash::zero(),
transactions: vec![].into(), transactions: vec![].into(),
}), }),
ForkName::Capella => { ForkName::Capella => ExecutionPayload::Capella(ExecutionPayloadCapella {
ExecutionPayload::Capella(ExecutionPayloadCapella {
parent_hash: forkchoice_state.head_block_hash, parent_hash: forkchoice_state.head_block_hash,
fee_recipient: pa.suggested_fee_recipient, fee_recipient: pa.suggested_fee_recipient,
receipts_root: Hash256::repeat_byte(42), receipts_root: Hash256::repeat_byte(42),
@ -553,10 +555,8 @@ impl<T: EthSpec> ExecutionBlockGenerator<T> {
block_hash: ExecutionBlockHash::zero(), block_hash: ExecutionBlockHash::zero(),
transactions: vec![].into(), transactions: vec![].into(),
withdrawals: pa.withdrawals.clone().into(), withdrawals: pa.withdrawals.clone().into(),
}) }),
} ForkName::Deneb => ExecutionPayload::Deneb(ExecutionPayloadDeneb {
ForkName::Deneb => {
ExecutionPayload::Deneb(ExecutionPayloadDeneb {
parent_hash: forkchoice_state.head_block_hash, parent_hash: forkchoice_state.head_block_hash,
fee_recipient: pa.suggested_fee_recipient, fee_recipient: pa.suggested_fee_recipient,
receipts_root: Hash256::repeat_byte(42), receipts_root: Hash256::repeat_byte(42),
@ -572,13 +572,11 @@ impl<T: EthSpec> ExecutionBlockGenerator<T> {
block_hash: ExecutionBlockHash::zero(), block_hash: ExecutionBlockHash::zero(),
transactions: vec![].into(), transactions: vec![].into(),
withdrawals: pa.withdrawals.clone().into(), withdrawals: pa.withdrawals.clone().into(),
// FIXME(deneb) maybe this should be set to something? data_gas_used: 0,
excess_data_gas: Uint256::one(), excess_data_gas: 0,
}) }),
}
_ => unreachable!(), _ => unreachable!(),
} },
}
}; };
match execution_payload.fork_name() { match execution_payload.fork_name() {
@ -631,61 +629,22 @@ impl<T: EthSpec> ExecutionBlockGenerator<T> {
pub fn generate_random_blobs<T: EthSpec>( pub fn generate_random_blobs<T: EthSpec>(
n_blobs: usize, n_blobs: usize,
kzg: &Kzg, kzg: &Kzg<T::Kzg>,
) -> Result<(BlobsBundleV1<T>, Transactions<T>), String> { ) -> Result<(BlobsBundleV1<T>, Transactions<T>), String> {
let mut bundle = BlobsBundleV1::<T>::default(); let mut bundle = BlobsBundleV1::<T>::default();
let mut transactions = vec![]; let mut transactions = vec![];
for blob_index in 0..n_blobs { for blob_index in 0..n_blobs {
// fill a vector with random bytes let random_valid_sidecar = BlobSidecar::<T>::random_valid(&mut thread_rng(), kzg)?;
let mut blob_bytes = [0u8; BYTES_PER_BLOB];
rand::thread_rng().fill_bytes(&mut blob_bytes);
// Ensure that the blob is canonical by ensuring that
// each field element contained in the blob is < BLS_MODULUS
for i in 0..FIELD_ELEMENTS_PER_BLOB {
blob_bytes[i * BYTES_PER_FIELD_ELEMENT + BYTES_PER_FIELD_ELEMENT - 1] = 0;
}
let blob = Blob::<T>::new(Vec::from(blob_bytes)) let BlobSidecar {
.map_err(|e| format!("error constructing random blob: {:?}", e))?; blob,
kzg_commitment,
kzg_proof,
..
} = random_valid_sidecar;
let commitment = kzg let tx = random_valid_tx::<T>()
.blob_to_kzg_commitment(blob_bytes.into()) .map_err(|e| format!("error creating valid tx SSZ bytes: {:?}", e))?;
.map_err(|e| format!("error computing kzg commitment: {:?}", e))?;
let proof = kzg
.compute_blob_kzg_proof(blob_bytes.into(), commitment)
.map_err(|e| format!("error computing kzg proof: {:?}", e))?;
let versioned_hash = commitment.calculate_versioned_hash();
let blob_transaction = BlobTransaction {
chain_id: Default::default(),
nonce: 0,
max_priority_fee_per_gas: Default::default(),
max_fee_per_gas: Default::default(),
gas: 100000,
to: None,
value: Default::default(),
data: Default::default(),
access_list: Default::default(),
max_fee_per_data_gas: Default::default(),
versioned_hashes: vec![versioned_hash].into(),
};
let bad_signature = EcdsaSignature {
y_parity: false,
r: Uint256::from(0),
s: Uint256::from(0),
};
let signed_blob_transaction = SignedBlobTransaction {
message: blob_transaction,
signature: bad_signature,
};
// calculate transaction bytes
let tx_bytes = [BLOB_TX_TYPE]
.into_iter()
.chain(signed_blob_transaction.as_ssz_bytes().into_iter())
.collect::<Vec<_>>();
let tx = Transaction::<T::MaxBytesPerTransaction>::from(tx_bytes);
transactions.push(tx); transactions.push(tx);
bundle bundle
@ -694,11 +653,11 @@ pub fn generate_random_blobs<T: EthSpec>(
.map_err(|_| format!("blobs are full, blob index: {:?}", blob_index))?; .map_err(|_| format!("blobs are full, blob index: {:?}", blob_index))?;
bundle bundle
.commitments .commitments
.push(commitment) .push(kzg_commitment)
.map_err(|_| format!("blobs are full, blob index: {:?}", blob_index))?; .map_err(|_| format!("blobs are full, blob index: {:?}", blob_index))?;
bundle bundle
.proofs .proofs
.push(proof) .push(kzg_proof)
.map_err(|_| format!("blobs are full, blob index: {:?}", blob_index))?; .map_err(|_| format!("blobs are full, blob index: {:?}", blob_index))?;
} }
@ -709,6 +668,51 @@ fn payload_id_from_u64(n: u64) -> PayloadId {
n.to_le_bytes() n.to_le_bytes()
} }
pub fn generate_genesis_header<T: EthSpec>(
spec: &ChainSpec,
post_transition_merge: bool,
) -> Option<ExecutionPayloadHeader<T>> {
let genesis_fork = spec.fork_name_at_slot::<T>(spec.genesis_slot);
let genesis_block_hash =
generate_genesis_block(spec.terminal_total_difficulty, DEFAULT_TERMINAL_BLOCK)
.ok()
.map(|block| block.block_hash);
match genesis_fork {
ForkName::Base | ForkName::Altair => None,
ForkName::Merge => {
if post_transition_merge {
let mut header = ExecutionPayloadHeader::Merge(<_>::default());
*header.block_hash_mut() = genesis_block_hash.unwrap_or_default();
Some(header)
} else {
Some(ExecutionPayloadHeader::<T>::Merge(<_>::default()))
}
}
ForkName::Capella => {
let mut header = ExecutionPayloadHeader::Capella(<_>::default());
*header.block_hash_mut() = genesis_block_hash.unwrap_or_default();
Some(header)
}
ForkName::Deneb => {
let mut header = ExecutionPayloadHeader::Capella(<_>::default());
*header.block_hash_mut() = genesis_block_hash.unwrap_or_default();
Some(header)
}
}
}
pub fn generate_genesis_block(
terminal_total_difficulty: Uint256,
terminal_block_number: u64,
) -> Result<PoWBlock, String> {
generate_pow_block(
terminal_total_difficulty,
terminal_block_number,
0,
ExecutionBlockHash::zero(),
)
}
pub fn generate_pow_block( pub fn generate_pow_block(
terminal_total_difficulty: Uint256, terminal_total_difficulty: Uint256,
terminal_block_number: u64, terminal_block_number: u64,

View File

@ -43,12 +43,12 @@ impl<T: EthSpec> MockExecutionLayer<T> {
executor: TaskExecutor, executor: TaskExecutor,
terminal_block: u64, terminal_block: u64,
shanghai_time: Option<u64>, shanghai_time: Option<u64>,
deneb_time: Option<u64>, cancun_time: Option<u64>,
builder_threshold: Option<u128>, builder_threshold: Option<u128>,
jwt_key: Option<JwtKey>, jwt_key: Option<JwtKey>,
spec: ChainSpec, spec: ChainSpec,
builder_url: Option<SensitiveUrl>, builder_url: Option<SensitiveUrl>,
kzg: Option<Kzg>, kzg: Option<Kzg<T::Kzg>>,
) -> Self { ) -> Self {
let handle = executor.handle().unwrap(); let handle = executor.handle().unwrap();
@ -60,7 +60,7 @@ impl<T: EthSpec> MockExecutionLayer<T> {
terminal_block, terminal_block,
spec.terminal_block_hash, spec.terminal_block_hash,
shanghai_time, shanghai_time,
deneb_time, cancun_time,
kzg, kzg,
); );
@ -204,7 +204,7 @@ impl<T: EthSpec> MockExecutionLayer<T> {
Some(payload.clone()) Some(payload.clone())
); );
let status = self.el.notify_new_payload(&payload).await.unwrap(); let status = self.el.notify_new_payload(&payload, None).await.unwrap();
assert_eq!(status, PayloadStatus::Valid); assert_eq!(status, PayloadStatus::Valid);
// Use junk values for slot/head-root to ensure there is no payload supplied. // Use junk values for slot/head-root to ensure there is no payload supplied.

View File

@ -25,7 +25,8 @@ use warp::{http::StatusCode, Filter, Rejection};
use crate::EngineCapabilities; use crate::EngineCapabilities;
pub use execution_block_generator::{ pub use execution_block_generator::{
generate_pow_block, generate_random_blobs, Block, ExecutionBlockGenerator, generate_genesis_header, generate_pow_block, generate_random_blobs, 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};
@ -65,7 +66,7 @@ pub struct MockExecutionConfig {
pub terminal_block: u64, pub terminal_block: u64,
pub terminal_block_hash: ExecutionBlockHash, pub terminal_block_hash: ExecutionBlockHash,
pub shanghai_time: Option<u64>, pub shanghai_time: Option<u64>,
pub deneb_time: Option<u64>, pub cancun_time: Option<u64>,
} }
impl Default for MockExecutionConfig { impl Default for MockExecutionConfig {
@ -77,7 +78,7 @@ impl Default for MockExecutionConfig {
terminal_block_hash: ExecutionBlockHash::zero(), terminal_block_hash: ExecutionBlockHash::zero(),
server_config: Config::default(), server_config: Config::default(),
shanghai_time: None, shanghai_time: None,
deneb_time: None, cancun_time: None,
} }
} }
} }
@ -106,7 +107,7 @@ impl<T: EthSpec> MockServer<T> {
pub fn new_with_config( pub fn new_with_config(
handle: &runtime::Handle, handle: &runtime::Handle,
config: MockExecutionConfig, config: MockExecutionConfig,
kzg: Option<Kzg>, kzg: Option<Kzg<T::Kzg>>,
) -> Self { ) -> Self {
let MockExecutionConfig { let MockExecutionConfig {
jwt_key, jwt_key,
@ -115,7 +116,7 @@ impl<T: EthSpec> MockServer<T> {
terminal_block_hash, terminal_block_hash,
server_config, server_config,
shanghai_time, shanghai_time,
deneb_time, cancun_time,
} = config; } = config;
let last_echo_request = Arc::new(RwLock::new(None)); let last_echo_request = Arc::new(RwLock::new(None));
let preloaded_responses = Arc::new(Mutex::new(vec![])); let preloaded_responses = Arc::new(Mutex::new(vec![]));
@ -124,7 +125,7 @@ impl<T: EthSpec> MockServer<T> {
terminal_block, terminal_block,
terminal_block_hash, terminal_block_hash,
shanghai_time, shanghai_time,
deneb_time, cancun_time,
kzg, kzg,
); );
@ -186,8 +187,8 @@ impl<T: EthSpec> MockServer<T> {
terminal_block: u64, terminal_block: u64,
terminal_block_hash: ExecutionBlockHash, terminal_block_hash: ExecutionBlockHash,
shanghai_time: Option<u64>, shanghai_time: Option<u64>,
deneb_time: Option<u64>, cancun_time: Option<u64>,
kzg: Option<Kzg>, kzg: Option<Kzg<T::Kzg>>,
) -> Self { ) -> Self {
Self::new_with_config( Self::new_with_config(
handle, handle,
@ -198,7 +199,7 @@ impl<T: EthSpec> MockServer<T> {
terminal_block, terminal_block,
terminal_block_hash, terminal_block_hash,
shanghai_time, shanghai_time,
deneb_time, cancun_time,
}, },
kzg, kzg,
) )

View File

@ -4,6 +4,7 @@ use beacon_chain::{
StateSkipConfig, StateSkipConfig,
}; };
use eth2::types::{IndexedErrorMessage, StateId, SyncSubcommittee}; use eth2::types::{IndexedErrorMessage, StateId, SyncSubcommittee};
use execution_layer::test_utils::generate_genesis_header;
use genesis::{bls_withdrawal_credentials, interop_genesis_state_with_withdrawal_credentials}; use genesis::{bls_withdrawal_credentials, interop_genesis_state_with_withdrawal_credentials};
use http_api::test_utils::*; use http_api::test_utils::*;
use std::collections::HashSet; use std::collections::HashSet;
@ -357,12 +358,13 @@ async fn bls_to_execution_changes_update_all_around_capella_fork() {
.iter() .iter()
.map(|keypair| bls_withdrawal_credentials(&keypair.as_ref().unwrap().pk, &spec)) .map(|keypair| bls_withdrawal_credentials(&keypair.as_ref().unwrap().pk, &spec))
.collect::<Vec<_>>(); .collect::<Vec<_>>();
let header = generate_genesis_header(&spec, true);
let genesis_state = interop_genesis_state_with_withdrawal_credentials( let genesis_state = interop_genesis_state_with_withdrawal_credentials(
&validator_keypairs, &validator_keypairs,
&withdrawal_credentials, &withdrawal_credentials,
HARNESS_GENESIS_TIME, HARNESS_GENESIS_TIME,
Hash256::from_slice(DEFAULT_ETH1_BLOCK_HASH), Hash256::from_slice(DEFAULT_ETH1_BLOCK_HASH),
None, header,
&spec, &spec,
) )
.unwrap(); .unwrap();

View File

@ -47,6 +47,12 @@ const MAX_INBOUND_SUBSTREAMS: usize = 32;
#[derive(Debug, Clone, Copy, Hash, Eq, PartialEq)] #[derive(Debug, Clone, Copy, Hash, Eq, PartialEq)]
pub struct SubstreamId(usize); pub struct SubstreamId(usize);
impl SubstreamId {
pub fn new(id: usize) -> Self {
Self(id)
}
}
type InboundSubstream<TSpec> = InboundFramed<NegotiatedSubstream, TSpec>; type InboundSubstream<TSpec> = InboundFramed<NegotiatedSubstream, TSpec>;
/// Events the handler emits to the behaviour. /// Events the handler emits to the behaviour.
@ -753,6 +759,9 @@ where
if matches!(info.protocol, Protocol::BlocksByRange) { if matches!(info.protocol, Protocol::BlocksByRange) {
debug!(self.log, "BlocksByRange Response sent"; "duration" => Instant::now().duration_since(info.request_start_time).as_secs()); debug!(self.log, "BlocksByRange Response sent"; "duration" => Instant::now().duration_since(info.request_start_time).as_secs());
} }
if matches!(info.protocol, Protocol::BlobsByRange) {
debug!(self.log, "BlobsByRange Response sent"; "duration" => Instant::now().duration_since(info.request_start_time).as_secs());
}
// There is nothing more to process on this substream as it has // There is nothing more to process on this substream as it has
// been closed. Move on to the next one. // been closed. Move on to the next one.
@ -776,6 +785,9 @@ where
if matches!(info.protocol, Protocol::BlocksByRange) { if matches!(info.protocol, Protocol::BlocksByRange) {
debug!(self.log, "BlocksByRange Response failed"; "duration" => info.request_start_time.elapsed().as_secs()); debug!(self.log, "BlocksByRange Response failed"; "duration" => info.request_start_time.elapsed().as_secs());
} }
if matches!(info.protocol, Protocol::BlobsByRange) {
debug!(self.log, "BlobsByRange Response failed"; "duration" => info.request_start_time.elapsed().as_secs());
}
break; break;
} }
// The sending future has not completed. Leave the state as busy and // The sending future has not completed. Leave the state as busy and

View File

@ -6,7 +6,7 @@ use serde::Serialize;
use ssz::Encode; use ssz::Encode;
use ssz_derive::{Decode, Encode}; use ssz_derive::{Decode, Encode};
use ssz_types::{ use ssz_types::{
typenum::{U1024, U256, U768}, typenum::{U1024, U128, U256, U768},
VariableList, VariableList,
}; };
use std::marker::PhantomData; use std::marker::PhantomData;
@ -28,6 +28,7 @@ pub const MAX_REQUEST_BLOCKS: u64 = 1024;
pub type MaxErrorLen = U256; pub type MaxErrorLen = U256;
pub const MAX_ERROR_LEN: u64 = 256; pub const MAX_ERROR_LEN: u64 = 256;
pub type MaxRequestBlocksDeneb = U128;
pub const MAX_REQUEST_BLOCKS_DENEB: u64 = 128; pub const MAX_REQUEST_BLOCKS_DENEB: u64 = 128;
// TODO: this is calculated as MAX_REQUEST_BLOCKS_DENEB * MAX_BLOBS_PER_BLOCK and // TODO: this is calculated as MAX_REQUEST_BLOCKS_DENEB * MAX_BLOBS_PER_BLOCK and
@ -295,7 +296,7 @@ pub struct BlobsByRangeRequest {
/// The starting slot to request blobs. /// The starting slot to request blobs.
pub start_slot: u64, pub start_slot: u64,
/// The number of blobs from the start slot. /// The number of slots from the start slot.
pub count: u64, pub count: u64,
} }

View File

@ -248,8 +248,10 @@ where
Err(RateLimitedErr::TooLarge) => { Err(RateLimitedErr::TooLarge) => {
// we set the batch sizes, so this is a coding/config err for most protocols // we set the batch sizes, so this is a coding/config err for most protocols
let protocol = req.versioned_protocol().protocol(); let protocol = req.versioned_protocol().protocol();
if matches!(protocol, Protocol::BlocksByRange) { if matches!(protocol, Protocol::BlocksByRange)
debug!(self.log, "Blocks by range request will never be processed"; "request" => %req); || matches!(protocol, Protocol::BlobsByRange)
{
debug!(self.log, "By range request will never be processed"; "request" => %req, "protocol" => %protocol);
} else { } else {
crit!(self.log, "Request size too large to ever be processed"; "protocol" => %protocol); crit!(self.log, "Request size too large to ever be processed"; "protocol" => %protocol);
} }

View File

@ -101,6 +101,20 @@ lazy_static! {
]) ])
.as_ssz_bytes() .as_ssz_bytes()
.len(); .len();
pub static ref BLOBS_BY_ROOT_REQUEST_MIN: usize =
VariableList::<Hash256, MaxRequestBlobSidecars>::from(Vec::<Hash256>::new())
.as_ssz_bytes()
.len();
pub static ref BLOBS_BY_ROOT_REQUEST_MAX: usize =
VariableList::<Hash256, MaxRequestBlobSidecars>::from(vec![
Hash256::zero();
MAX_REQUEST_BLOB_SIDECARS
as usize
])
.as_ssz_bytes()
.len();
pub static ref ERROR_TYPE_MIN: usize = pub static ref ERROR_TYPE_MIN: usize =
VariableList::<u8, MaxErrorLen>::from(Vec::<u8>::new()) VariableList::<u8, MaxErrorLen>::from(Vec::<u8>::new())
.as_ssz_bytes() .as_ssz_bytes()
@ -113,13 +127,6 @@ lazy_static! {
]) ])
.as_ssz_bytes() .as_ssz_bytes()
.len(); .len();
pub static ref BLOB_SIDECAR_MIN: usize = BlobSidecar::<MainnetEthSpec>::empty().as_ssz_bytes().len();
pub static ref BLOB_SIDECAR_MAX: usize = BlobSidecar::<MainnetEthSpec>::max_size();
//FIXME(sean) these are underestimates
pub static ref SIGNED_BLOCK_AND_BLOBS_MIN: usize = *BLOB_SIDECAR_MIN + *SIGNED_BEACON_BLOCK_BASE_MIN;
pub static ref SIGNED_BLOCK_AND_BLOBS_MAX: usize =*BLOB_SIDECAR_MAX + *SIGNED_BEACON_BLOCK_DENEB_MAX;
} }
/// The maximum bytes that can be sent across the RPC pre-merge. /// The maximum bytes that can be sent across the RPC pre-merge.
@ -395,8 +402,7 @@ impl ProtocolId {
<BlobsByRangeRequest as Encode>::ssz_fixed_len(), <BlobsByRangeRequest as Encode>::ssz_fixed_len(),
), ),
Protocol::BlobsByRoot => { Protocol::BlobsByRoot => {
// TODO: This looks wrong to me RpcLimits::new(*BLOBS_BY_ROOT_REQUEST_MIN, *BLOBS_BY_ROOT_REQUEST_MAX)
RpcLimits::new(*BLOCKS_BY_ROOT_REQUEST_MIN, *BLOCKS_BY_ROOT_REQUEST_MAX)
} }
Protocol::Ping => RpcLimits::new( Protocol::Ping => RpcLimits::new(
<Ping as Encode>::ssz_fixed_len(), <Ping as Encode>::ssz_fixed_len(),
@ -420,8 +426,8 @@ impl ProtocolId {
Protocol::Goodbye => RpcLimits::new(0, 0), // Goodbye request has no response Protocol::Goodbye => RpcLimits::new(0, 0), // Goodbye request has no response
Protocol::BlocksByRange => rpc_block_limits_by_fork(fork_context.current_fork()), Protocol::BlocksByRange => rpc_block_limits_by_fork(fork_context.current_fork()),
Protocol::BlocksByRoot => rpc_block_limits_by_fork(fork_context.current_fork()), Protocol::BlocksByRoot => rpc_block_limits_by_fork(fork_context.current_fork()),
Protocol::BlobsByRange => RpcLimits::new(*BLOB_SIDECAR_MIN, *BLOB_SIDECAR_MAX), Protocol::BlobsByRange => rpc_blob_limits::<T>(),
Protocol::BlobsByRoot => RpcLimits::new(*BLOB_SIDECAR_MIN, *BLOB_SIDECAR_MAX), Protocol::BlobsByRoot => rpc_blob_limits::<T>(),
Protocol::Ping => RpcLimits::new( Protocol::Ping => RpcLimits::new(
<Ping as Encode>::ssz_fixed_len(), <Ping as Encode>::ssz_fixed_len(),
<Ping as Encode>::ssz_fixed_len(), <Ping as Encode>::ssz_fixed_len(),
@ -476,6 +482,13 @@ impl ProtocolId {
} }
} }
pub fn rpc_blob_limits<T: EthSpec>() -> RpcLimits {
RpcLimits::new(
BlobSidecar::<T>::empty().as_ssz_bytes().len(),
BlobSidecar::<T>::max_size(),
)
}
impl ProtocolName for ProtocolId { impl ProtocolName for ProtocolId {
fn protocol_name(&self) -> &[u8] { fn protocol_name(&self) -> &[u8] {
self.protocol_id.as_bytes() self.protocol_id.as_bytes()

View File

@ -9,8 +9,9 @@ use std::time::Duration;
use tokio::runtime::Runtime; use tokio::runtime::Runtime;
use tokio::time::sleep; use tokio::time::sleep;
use types::{ use types::{
BeaconBlock, BeaconBlockAltair, BeaconBlockBase, BeaconBlockMerge, EmptyBlock, Epoch, EthSpec, BeaconBlock, BeaconBlockAltair, BeaconBlockBase, BeaconBlockMerge, BlobSidecar, EmptyBlock,
ForkContext, ForkName, Hash256, MinimalEthSpec, Signature, SignedBeaconBlock, Slot, Epoch, EthSpec, ForkContext, ForkName, Hash256, MinimalEthSpec, Signature, SignedBeaconBlock,
Slot,
}; };
mod common; mod common;
@ -259,6 +260,111 @@ fn test_blocks_by_range_chunked_rpc() {
}) })
} }
// Tests a streamed BlobsByRange RPC Message
#[test]
#[allow(clippy::single_match)]
fn test_blobs_by_range_chunked_rpc() {
// set up the logging. The level and enabled logging or not
let log_level = Level::Debug;
let enable_logging = false;
let slot_count = 32;
let messages_to_send = 34;
let log = common::build_log(log_level, enable_logging);
let rt = Arc::new(Runtime::new().unwrap());
rt.block_on(async {
// get sender/receiver
let (mut sender, mut receiver) =
common::build_node_pair(Arc::downgrade(&rt), &log, ForkName::Deneb).await;
// BlobsByRange Request
let rpc_request = Request::BlobsByRange(BlobsByRangeRequest {
start_slot: 0,
count: slot_count,
});
// BlocksByRange Response
let blob = BlobSidecar::<E>::default();
let rpc_response = Response::BlobsByRange(Some(Arc::new(blob)));
// keep count of the number of messages received
let mut messages_received = 0;
let request_id = messages_to_send as usize;
// build the sender future
let sender_future = async {
loop {
match sender.next_event().await {
NetworkEvent::PeerConnectedOutgoing(peer_id) => {
// Send a STATUS message
debug!(log, "Sending RPC");
sender.send_request(peer_id, request_id, rpc_request.clone());
}
NetworkEvent::ResponseReceived {
peer_id: _,
id: _,
response,
} => {
warn!(log, "Sender received a response");
match response {
Response::BlobsByRange(Some(_)) => {
assert_eq!(response, rpc_response.clone());
messages_received += 1;
warn!(log, "Chunk received");
}
Response::BlobsByRange(None) => {
// should be exactly `messages_to_send` messages before terminating
assert_eq!(messages_received, messages_to_send);
// end the test
return;
}
_ => panic!("Invalid RPC received"),
}
}
_ => {} // Ignore other behaviour events
}
}
};
// build the receiver future
let receiver_future = async {
loop {
match receiver.next_event().await {
NetworkEvent::RequestReceived {
peer_id,
id,
request,
} => {
if request == rpc_request {
// send the response
warn!(log, "Receiver got request");
for _ in 0..messages_to_send {
// Send first third of responses as base blocks,
// second as altair and third as merge.
receiver.send_response(peer_id, id, rpc_response.clone());
}
// send the stream termination
receiver.send_response(peer_id, id, Response::BlobsByRange(None));
}
}
_ => {} // Ignore other events
}
}
};
tokio::select! {
_ = sender_future => {}
_ = receiver_future => {}
_ = sleep(Duration::from_secs(30)) => {
panic!("Future timed out");
}
}
})
}
// Tests rejection of blocks over `MAX_RPC_SIZE`. // Tests rejection of blocks over `MAX_RPC_SIZE`.
#[test] #[test]
#[allow(clippy::single_match)] #[allow(clippy::single_match)]

View File

@ -225,7 +225,7 @@ pub const GOSSIP_ATTESTATION_BATCH: &str = "gossip_attestation_batch";
pub const GOSSIP_AGGREGATE: &str = "gossip_aggregate"; pub const GOSSIP_AGGREGATE: &str = "gossip_aggregate";
pub const GOSSIP_AGGREGATE_BATCH: &str = "gossip_aggregate_batch"; pub const GOSSIP_AGGREGATE_BATCH: &str = "gossip_aggregate_batch";
pub const GOSSIP_BLOCK: &str = "gossip_block"; pub const GOSSIP_BLOCK: &str = "gossip_block";
pub const GOSSIP_BLOCK_AND_BLOBS_SIDECAR: &str = "gossip_block_and_blobs_sidecar"; pub const GOSSIP_BLOBS_SIDECAR: &str = "gossip_blobs_sidecar";
pub const DELAYED_IMPORT_BLOCK: &str = "delayed_import_block"; pub const DELAYED_IMPORT_BLOCK: &str = "delayed_import_block";
pub const GOSSIP_VOLUNTARY_EXIT: &str = "gossip_voluntary_exit"; pub const GOSSIP_VOLUNTARY_EXIT: &str = "gossip_voluntary_exit";
pub const GOSSIP_PROPOSER_SLASHING: &str = "gossip_proposer_slashing"; pub const GOSSIP_PROPOSER_SLASHING: &str = "gossip_proposer_slashing";
@ -1002,7 +1002,7 @@ impl<T: BeaconChainTypes> Work<T> {
Work::GossipAggregate { .. } => GOSSIP_AGGREGATE, Work::GossipAggregate { .. } => GOSSIP_AGGREGATE,
Work::GossipAggregateBatch { .. } => GOSSIP_AGGREGATE_BATCH, Work::GossipAggregateBatch { .. } => GOSSIP_AGGREGATE_BATCH,
Work::GossipBlock { .. } => GOSSIP_BLOCK, Work::GossipBlock { .. } => GOSSIP_BLOCK,
Work::GossipSignedBlobSidecar { .. } => GOSSIP_BLOCK_AND_BLOBS_SIDECAR, Work::GossipSignedBlobSidecar { .. } => GOSSIP_BLOBS_SIDECAR,
Work::DelayedImportBlock { .. } => DELAYED_IMPORT_BLOCK, Work::DelayedImportBlock { .. } => DELAYED_IMPORT_BLOCK,
Work::GossipVoluntaryExit { .. } => GOSSIP_VOLUNTARY_EXIT, Work::GossipVoluntaryExit { .. } => GOSSIP_VOLUNTARY_EXIT,
Work::GossipProposerSlashing { .. } => GOSSIP_PROPOSER_SLASHING, Work::GossipProposerSlashing { .. } => GOSSIP_PROPOSER_SLASHING,

View File

@ -1,4 +1,3 @@
#![cfg(not(debug_assertions))] // Tests are too slow in debug.
#![cfg(test)] #![cfg(test)]
use crate::beacon_processor::work_reprocessing_queue::{ use crate::beacon_processor::work_reprocessing_queue::{
@ -7,14 +6,16 @@ use crate::beacon_processor::work_reprocessing_queue::{
use crate::beacon_processor::*; use crate::beacon_processor::*;
use crate::{service::NetworkMessage, sync::SyncMessage}; use crate::{service::NetworkMessage, sync::SyncMessage};
use beacon_chain::test_utils::{ use beacon_chain::test_utils::{
AttestationStrategy, BeaconChainHarness, BlockStrategy, EphemeralHarnessType, test_spec, AttestationStrategy, BeaconChainHarness, BlockStrategy, EphemeralHarnessType,
}; };
use beacon_chain::{BeaconChain, ChainConfig, MAXIMUM_GOSSIP_CLOCK_DISPARITY}; use beacon_chain::{BeaconChain, ChainConfig, WhenSlotSkipped, MAXIMUM_GOSSIP_CLOCK_DISPARITY};
use lighthouse_network::discovery::ConnectionId;
use lighthouse_network::rpc::SubstreamId;
use lighthouse_network::{ use lighthouse_network::{
discv5::enr::{CombinedKey, EnrBuilder}, discv5::enr::{CombinedKey, EnrBuilder},
rpc::methods::{MetaData, MetaDataV2}, rpc::methods::{MetaData, MetaDataV2},
types::{EnrAttestationBitfield, EnrSyncCommitteeBitfield}, types::{EnrAttestationBitfield, EnrSyncCommitteeBitfield},
MessageId, NetworkGlobals, PeerId, MessageId, NetworkGlobals, PeerId, Response,
}; };
use slot_clock::SlotClock; use slot_clock::SlotClock;
use std::cmp; use std::cmp;
@ -23,8 +24,8 @@ use std::sync::Arc;
use std::time::Duration; use std::time::Duration;
use tokio::sync::mpsc; use tokio::sync::mpsc;
use types::{ use types::{
Attestation, AttesterSlashing, Epoch, EthSpec, MainnetEthSpec, ProposerSlashing, Attestation, AttesterSlashing, Epoch, MainnetEthSpec, ProposerSlashing, SignedBeaconBlock,
SignedBeaconBlock, SignedVoluntaryExit, SubnetId, SignedBlobSidecarList, SignedVoluntaryExit, Slot, SubnetId,
}; };
type E = MainnetEthSpec; type E = MainnetEthSpec;
@ -45,6 +46,7 @@ const STANDARD_TIMEOUT: Duration = Duration::from_secs(10);
struct TestRig { struct TestRig {
chain: Arc<BeaconChain<T>>, chain: Arc<BeaconChain<T>>,
next_block: Arc<SignedBeaconBlock<E>>, next_block: Arc<SignedBeaconBlock<E>>,
next_blobs: Option<SignedBlobSidecarList<E>>,
attestations: Vec<(Attestation<E>, SubnetId)>, attestations: Vec<(Attestation<E>, SubnetId)>,
next_block_attestations: Vec<(Attestation<E>, SubnetId)>, next_block_attestations: Vec<(Attestation<E>, SubnetId)>,
next_block_aggregate_attestations: Vec<SignedAggregateAndProof<E>>, next_block_aggregate_attestations: Vec<SignedAggregateAndProof<E>>,
@ -74,14 +76,15 @@ impl TestRig {
} }
pub async fn new_with_chain_config(chain_length: u64, chain_config: ChainConfig) -> Self { pub async fn new_with_chain_config(chain_length: u64, chain_config: ChainConfig) -> Self {
let mut spec = test_spec::<MainnetEthSpec>();
// 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();
spec.shard_committee_period = 2; spec.shard_committee_period = 2;
let harness = BeaconChainHarness::builder(MainnetEthSpec) let harness = BeaconChainHarness::builder(MainnetEthSpec)
.spec(spec) .spec(spec)
.deterministic_keypairs(VALIDATOR_COUNT) .deterministic_keypairs(VALIDATOR_COUNT)
.fresh_ephemeral_store() .fresh_ephemeral_store()
.mock_execution_layer()
.chain_config(chain_config) .chain_config(chain_config)
.build(); .build();
@ -211,6 +214,7 @@ impl TestRig {
Self { Self {
chain, chain,
next_block: Arc::new(next_block_tuple.0), next_block: Arc::new(next_block_tuple.0),
next_blobs: next_block_tuple.1,
attestations, attestations,
next_block_attestations, next_block_attestations,
next_block_aggregate_attestations, next_block_aggregate_attestations,
@ -246,6 +250,22 @@ impl TestRig {
.unwrap(); .unwrap();
} }
pub fn enqueue_gossip_blob(&self, blob_index: usize) {
if let Some(blobs) = self.next_blobs.as_ref() {
let blob = blobs.get(blob_index).unwrap();
self.beacon_processor_tx
.try_send(WorkEvent::gossip_signed_blob_sidecar(
junk_message_id(),
junk_peer_id(),
Client::default(),
blob_index as u64,
blob.clone(),
Duration::from_secs(0),
))
.unwrap();
}
}
pub fn enqueue_rpc_block(&self) { pub fn enqueue_rpc_block(&self) {
let event = WorkEvent::rpc_beacon_block( let event = WorkEvent::rpc_beacon_block(
self.next_block.canonical_root(), self.next_block.canonical_root(),
@ -268,6 +288,36 @@ impl TestRig {
self.beacon_processor_tx.try_send(event).unwrap(); self.beacon_processor_tx.try_send(event).unwrap();
} }
pub fn enqueue_single_lookup_rpc_blobs(&self) {
if let Some(blobs) = self.next_blobs.clone() {
let blobs = FixedBlobSidecarList::from(
blobs
.into_iter()
.map(|b| Some(b.message))
.collect::<Vec<_>>(),
);
let event = WorkEvent::rpc_blobs(
self.next_block.canonical_root(),
blobs,
std::time::Duration::default(),
BlockProcessType::SingleBlock { id: 1 },
);
self.beacon_processor_tx.try_send(event).unwrap();
}
}
pub fn enqueue_blobs_by_range_request(&self, count: u64) {
let event = WorkEvent::blobs_by_range_request(
PeerId::random(),
(ConnectionId::new(42), SubstreamId::new(24)),
BlobsByRangeRequest {
start_slot: 0,
count,
},
);
self.beacon_processor_tx.try_send(event).unwrap();
}
pub fn enqueue_backfill_batch(&self) { pub fn enqueue_backfill_batch(&self) {
let event = WorkEvent::chain_segment( let event = WorkEvent::chain_segment(
ChainSegmentProcessId::BackSyncBatchId(Epoch::default()), ChainSegmentProcessId::BackSyncBatchId(Epoch::default()),
@ -491,6 +541,13 @@ async fn import_gossip_block_acceptably_early() {
rig.assert_event_journal(&[GOSSIP_BLOCK, WORKER_FREED, NOTHING_TO_DO]) rig.assert_event_journal(&[GOSSIP_BLOCK, WORKER_FREED, NOTHING_TO_DO])
.await; .await;
let num_blobs = rig.next_blobs.as_ref().map(|b| b.len()).unwrap_or(0);
for i in 0..num_blobs {
rig.enqueue_gossip_blob(i);
rig.assert_event_journal(&[GOSSIP_BLOBS_SIDECAR, WORKER_FREED, NOTHING_TO_DO])
.await;
}
// Note: this section of the code is a bit race-y. We're assuming that we can set the slot clock // Note: this section of the code is a bit race-y. We're assuming that we can set the slot clock
// and check the head in the time between the block arrived early and when its due for // and check the head in the time between the block arrived early and when its due for
// processing. // processing.
@ -499,6 +556,7 @@ async fn import_gossip_block_acceptably_early() {
// processing, instead of just ADDITIONAL_QUEUED_BLOCK_DELAY. Speak to @paulhauner if this test // processing, instead of just ADDITIONAL_QUEUED_BLOCK_DELAY. Speak to @paulhauner if this test
// starts failing. // starts failing.
rig.chain.slot_clock.set_slot(rig.next_block.slot().into()); rig.chain.slot_clock.set_slot(rig.next_block.slot().into());
assert!( assert!(
rig.head_root() != rig.next_block.canonical_root(), rig.head_root() != rig.next_block.canonical_root(),
"block not yet imported" "block not yet imported"
@ -566,6 +624,19 @@ async fn import_gossip_block_at_current_slot() {
rig.assert_event_journal(&[GOSSIP_BLOCK, WORKER_FREED, NOTHING_TO_DO]) rig.assert_event_journal(&[GOSSIP_BLOCK, WORKER_FREED, NOTHING_TO_DO])
.await; .await;
let num_blobs = rig
.next_blobs
.as_ref()
.map(|blobs| blobs.len())
.unwrap_or(0);
for i in 0..num_blobs {
rig.enqueue_gossip_blob(i);
rig.assert_event_journal(&[GOSSIP_BLOBS_SIDECAR, WORKER_FREED, NOTHING_TO_DO])
.await;
}
assert_eq!( assert_eq!(
rig.head_root(), rig.head_root(),
rig.next_block.canonical_root(), rig.next_block.canonical_root(),
@ -618,20 +689,34 @@ async fn attestation_to_unknown_block_processed(import_method: BlockImportMethod
); );
// Send the block and ensure that the attestation is received back and imported. // Send the block and ensure that the attestation is received back and imported.
let num_blobs = rig
let block_event = match import_method { .next_blobs
.as_ref()
.map(|blobs| blobs.len())
.unwrap_or(0);
let mut events = vec![];
match import_method {
BlockImportMethod::Gossip => { BlockImportMethod::Gossip => {
rig.enqueue_gossip_block(); rig.enqueue_gossip_block();
GOSSIP_BLOCK events.push(GOSSIP_BLOCK);
for i in 0..num_blobs {
rig.enqueue_gossip_blob(i);
events.push(GOSSIP_BLOBS_SIDECAR);
}
} }
BlockImportMethod::Rpc => { BlockImportMethod::Rpc => {
rig.enqueue_rpc_block(); rig.enqueue_rpc_block();
RPC_BLOCK events.push(RPC_BLOCK);
rig.enqueue_single_lookup_rpc_blobs();
if num_blobs > 0 {
events.push(RPC_BLOB);
}
} }
}; };
rig.assert_event_journal_contains_ordered(&[block_event, UNKNOWN_BLOCK_ATTESTATION]) events.push(UNKNOWN_BLOCK_ATTESTATION);
.await;
rig.assert_event_journal_contains_ordered(&events).await;
// Run fork choice, since it isn't run when processing an RPC block. At runtime it is the // Run fork choice, since it isn't run when processing an RPC block. At runtime it is the
// responsibility of the sync manager to do this. // responsibility of the sync manager to do this.
@ -687,20 +772,34 @@ async fn aggregate_attestation_to_unknown_block(import_method: BlockImportMethod
); );
// Send the block and ensure that the attestation is received back and imported. // Send the block and ensure that the attestation is received back and imported.
let num_blobs = rig
let block_event = match import_method { .next_blobs
.as_ref()
.map(|blobs| blobs.len())
.unwrap_or(0);
let mut events = vec![];
match import_method {
BlockImportMethod::Gossip => { BlockImportMethod::Gossip => {
rig.enqueue_gossip_block(); rig.enqueue_gossip_block();
GOSSIP_BLOCK events.push(GOSSIP_BLOCK);
for i in 0..num_blobs {
rig.enqueue_gossip_blob(i);
events.push(GOSSIP_BLOBS_SIDECAR);
}
} }
BlockImportMethod::Rpc => { BlockImportMethod::Rpc => {
rig.enqueue_rpc_block(); rig.enqueue_rpc_block();
RPC_BLOCK events.push(RPC_BLOCK);
rig.enqueue_single_lookup_rpc_blobs();
if num_blobs > 0 {
events.push(RPC_BLOB);
}
} }
}; };
rig.assert_event_journal_contains_ordered(&[block_event, UNKNOWN_BLOCK_AGGREGATE]) events.push(UNKNOWN_BLOCK_AGGREGATE);
.await;
rig.assert_event_journal_contains_ordered(&events).await;
// Run fork choice, since it isn't run when processing an RPC block. At runtime it is the // Run fork choice, since it isn't run when processing an RPC block. At runtime it is the
// responsibility of the sync manager to do this. // responsibility of the sync manager to do this.
@ -868,9 +967,15 @@ async fn test_rpc_block_reprocessing() {
// Insert the next block into the duplicate cache manually // Insert the next block into the duplicate cache manually
let handle = rig.duplicate_cache.check_and_insert(next_block_root); let handle = rig.duplicate_cache.check_and_insert(next_block_root);
rig.enqueue_single_lookup_rpc_block(); rig.enqueue_single_lookup_rpc_block();
rig.assert_event_journal(&[RPC_BLOCK, WORKER_FREED, NOTHING_TO_DO]) rig.assert_event_journal(&[RPC_BLOCK, WORKER_FREED, NOTHING_TO_DO])
.await; .await;
rig.enqueue_single_lookup_rpc_blobs();
if rig.next_blobs.as_ref().map(|b| b.len()).unwrap_or(0) > 0 {
rig.assert_event_journal(&[RPC_BLOB, WORKER_FREED, NOTHING_TO_DO])
.await;
}
// next_block shouldn't be processed since it couldn't get the // next_block shouldn't be processed since it couldn't get the
// duplicate cache handle // duplicate cache handle
assert_ne!(next_block_root, rig.head_root()); assert_ne!(next_block_root, rig.head_root());
@ -934,3 +1039,47 @@ async fn test_backfill_sync_processing_rate_limiting_disabled() {
) )
.await; .await;
} }
#[tokio::test]
async fn test_blobs_by_range() {
if test_spec::<E>().deneb_fork_epoch.is_none() {
return;
};
let mut rig = TestRig::new(64).await;
let slot_count = 32;
rig.enqueue_blobs_by_range_request(slot_count);
let mut blob_count = 0;
for slot in 0..slot_count {
let root = rig
.chain
.block_root_at_slot(Slot::new(slot), WhenSlotSkipped::None)
.unwrap();
blob_count += root
.and_then(|root| {
rig.chain
.get_blobs(&root)
.unwrap_or_default()
.map(|blobs| blobs.len())
})
.unwrap_or(0);
}
let mut actual_count = 0;
while let Some(next) = rig._network_rx.recv().await {
if let NetworkMessage::SendResponse {
peer_id: _,
response: Response::BlobsByRange(blob),
id: _,
} = next
{
if blob.is_some() {
actual_count += 1;
} else {
break;
}
} else {
panic!("unexpected message {:?}", next);
}
}
assert_eq!(blob_count, actual_count);
}

View File

@ -8,7 +8,7 @@ use beacon_chain::{
light_client_optimistic_update_verification::Error as LightClientOptimisticUpdateError, light_client_optimistic_update_verification::Error as LightClientOptimisticUpdateError,
observed_operations::ObservationOutcome, observed_operations::ObservationOutcome,
sync_committee_verification::{self, Error as SyncCommitteeError}, sync_committee_verification::{self, Error as SyncCommitteeError},
validator_monitor::get_block_delay_ms, validator_monitor::{get_block_delay_ms, get_slot_delay_ms},
AvailabilityProcessingStatus, BeaconChainError, BeaconChainTypes, BlockError, ForkChoiceError, AvailabilityProcessingStatus, BeaconChainError, BeaconChainTypes, BlockError, ForkChoiceError,
GossipVerifiedBlock, NotifyExecutionLayer, GossipVerifiedBlock, NotifyExecutionLayer,
}; };
@ -659,11 +659,15 @@ impl<T: BeaconChainTypes> Worker<T> {
_peer_client: Client, _peer_client: Client,
blob_index: u64, blob_index: u64,
signed_blob: SignedBlobSidecar<T::EthSpec>, signed_blob: SignedBlobSidecar<T::EthSpec>,
_seen_duration: Duration, seen_duration: Duration,
) { ) {
let slot = signed_blob.message.slot; let slot = signed_blob.message.slot;
let root = signed_blob.message.block_root; let root = signed_blob.message.block_root;
let index = signed_blob.message.index; let index = signed_blob.message.index;
let delay = get_slot_delay_ms(seen_duration, slot, &self.chain.slot_clock);
// Log metrics to track delay from other nodes on the network.
metrics::observe_duration(&metrics::BEACON_BLOB_GOSSIP_SLOT_START_DELAY_TIME, delay);
metrics::set_gauge(&metrics::BEACON_BLOB_LAST_DELAY, delay.as_millis() as i64);
match self match self
.chain .chain
.verify_blob_sidecar_for_gossip(signed_blob, blob_index) .verify_blob_sidecar_for_gossip(signed_blob, blob_index)
@ -676,8 +680,9 @@ impl<T: BeaconChainTypes> Worker<T> {
"root" => %root, "root" => %root,
"index" => %index "index" => %index
); );
metrics::inc_counter(&metrics::BEACON_PROCESSOR_GOSSIP_BLOB_VERIFIED_TOTAL);
self.propagate_validation_result(message_id, peer_id, MessageAcceptance::Accept); self.propagate_validation_result(message_id, peer_id, MessageAcceptance::Accept);
self.process_gossip_verified_blob(peer_id, gossip_verified_blob, _seen_duration) self.process_gossip_verified_blob(peer_id, gossip_verified_blob, seen_duration)
.await .await
} }
Err(err) => { Err(err) => {

View File

@ -10,7 +10,7 @@ use lighthouse_network::rpc::methods::{
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};
use slog::{debug, error, warn}; use slog::{debug, error, trace, warn};
use slot_clock::SlotClock; use slot_clock::SlotClock;
use std::collections::{hash_map::Entry, HashMap}; use std::collections::{hash_map::Entry, HashMap};
use task_executor::TaskExecutor; use task_executor::TaskExecutor;
@ -778,7 +778,7 @@ impl<T: BeaconChainTypes> Worker<T> {
} }
} }
Ok(None) => { Ok(None) => {
debug!( trace!(
self.log, self.log,
"No blobs in the store for block root"; "No blobs in the store for block root";
"request" => ?req, "request" => ?req,

View File

@ -106,6 +106,19 @@ lazy_static! {
"beacon_processor_gossip_block_early_seconds", "beacon_processor_gossip_block_early_seconds",
"Whenever a gossip block is received early this metrics is set to how early that block was." "Whenever a gossip block is received early this metrics is set to how early that block was."
); );
// Gossip blobs.
pub static ref BEACON_PROCESSOR_GOSSIP_BLOB_QUEUE_TOTAL: Result<IntGauge> = try_create_int_gauge(
"beacon_processor_gossip_blob_queue_total",
"Count of blocks from gossip waiting to be verified."
);
pub static ref BEACON_PROCESSOR_GOSSIP_BLOB_VERIFIED_TOTAL: Result<IntCounter> = try_create_int_counter(
"beacon_processor_gossip_blob_verified_total",
"Total number of gossip blob verified for propagation."
);
pub static ref BEACON_PROCESSOR_GOSSIP_BLOB_IMPORTED_TOTAL: Result<IntCounter> = try_create_int_counter(
"beacon_processor_gossip_blob_imported_total",
"Total number of gossip blobs imported to fork choice, etc."
);
// Gossip Exits. // Gossip Exits.
pub static ref BEACON_PROCESSOR_EXIT_QUEUE_TOTAL: Result<IntGauge> = try_create_int_gauge( pub static ref BEACON_PROCESSOR_EXIT_QUEUE_TOTAL: Result<IntGauge> = try_create_int_gauge(
"beacon_processor_exit_queue_total", "beacon_processor_exit_queue_total",
@ -374,6 +387,35 @@ lazy_static! {
"Count of times when a gossip block arrived from the network later than the attestation deadline.", "Count of times when a gossip block arrived from the network later than the attestation deadline.",
); );
/*
* Blob Delay Metrics
*/
pub static ref BEACON_BLOB_GOSSIP_PROPAGATION_VERIFICATION_DELAY_TIME: Result<Histogram> = try_create_histogram_with_buckets(
"beacon_blob_gossip_propagation_verification_delay_time",
"Duration between when the blob is received and when it is verified for propagation.",
// [0.001, 0.002, 0.005, 0.01, 0.02, 0.05, 0.1, 0.2, 0.5]
decimal_buckets(-3,-1)
);
pub static ref BEACON_BLOB_GOSSIP_SLOT_START_DELAY_TIME: Result<Histogram> = try_create_histogram_with_buckets(
"beacon_blob_gossip_slot_start_delay_time",
"Duration between when the blob is received and the start of the slot it belongs to.",
// Create a custom bucket list for greater granularity in block delay
Ok(vec![0.1, 0.2, 0.3,0.4,0.5,0.75,1.0,1.25,1.5,1.75,2.0,2.5,3.0,3.5,4.0,5.0,6.0,7.0,8.0,9.0,10.0,15.0,20.0])
// NOTE: Previous values, which we may want to switch back to.
// [0.1, 0.2, 0.5, 1, 2, 5, 10, 20, 50]
//decimal_buckets(-1,2)
);
pub static ref BEACON_BLOB_LAST_DELAY: Result<IntGauge> = try_create_int_gauge(
"beacon_blob_last_delay",
"Keeps track of the last blob's delay from the start of the slot"
);
pub static ref BEACON_BLOB_GOSSIP_ARRIVED_LATE_TOTAL: Result<IntCounter> = try_create_int_counter(
"beacon_blob_gossip_arrived_late_total",
"Count of times when a gossip blob arrived from the network later than the attestation deadline.",
);
/* /*
* Attestation reprocessing queue metrics. * Attestation reprocessing queue metrics.
*/ */

View File

@ -1324,7 +1324,6 @@ mod deneb_only {
fn parent_blob_response(mut self) -> Self { fn parent_blob_response(mut self) -> Self {
for blob in &self.parent_blobs { for blob in &self.parent_blobs {
dbg!("sendingblob");
self.bl.parent_lookup_blob_response( self.bl.parent_lookup_blob_response(
self.parent_blob_req_id.expect("parent blob request id"), self.parent_blob_req_id.expect("parent blob request id"),
self.peer_id, self.peer_id,
@ -1334,7 +1333,6 @@ mod deneb_only {
); );
assert_eq!(self.bl.parent_lookups.len(), 1); assert_eq!(self.bl.parent_lookups.len(), 1);
} }
dbg!("sending stream terminator");
self.bl.parent_lookup_blob_response( self.bl.parent_lookup_blob_response(
self.parent_blob_req_id.expect("blob request id"), self.parent_blob_req_id.expect("blob request id"),
self.peer_id, self.peer_id,

View File

@ -82,7 +82,7 @@ The `modernity` is:
The `features` is: The `features` is:
* `-dev` for a development build with `minimal-spec` preset enabled. * `-dev` for a development build with `minimal` preset enabled (`spec-minimal` feature).
* empty for a standard build with no custom feature enabled. * empty for a standard build with no custom feature enabled.

File diff suppressed because one or more lines are too long

View File

@ -13,10 +13,11 @@
use discv5::enr::{CombinedKey, Enr}; use discv5::enr::{CombinedKey, Enr};
use eth2_config::{instantiate_hardcoded_nets, HardcodedNet}; use eth2_config::{instantiate_hardcoded_nets, HardcodedNet};
use kzg::TrustedSetup; use kzg::{KzgPreset, KzgPresetId, TrustedSetup};
use std::fs::{create_dir_all, File}; use std::fs::{create_dir_all, File};
use std::io::{Read, Write}; use std::io::{Read, Write};
use std::path::PathBuf; use std::path::PathBuf;
use std::str::FromStr;
use types::{BeaconState, ChainSpec, Config, Epoch, EthSpec, EthSpecId}; use types::{BeaconState, ChainSpec, Config, Epoch, EthSpec, EthSpecId};
pub const DEPLOY_BLOCK_FILE: &str = "deploy_block.txt"; pub const DEPLOY_BLOCK_FILE: &str = "deploy_block.txt";
@ -38,9 +39,26 @@ pub const DEFAULT_HARDCODED_NETWORK: &str = "mainnet";
/// ///
/// This is done to ensure that testnets also inherit the high security and /// This is done to ensure that testnets also inherit the high security and
/// randomness of the mainnet kzg trusted setup ceremony. /// randomness of the mainnet kzg trusted setup ceremony.
pub const TRUSTED_SETUP: &[u8] = const TRUSTED_SETUP: &[u8] =
include_bytes!("../built_in_network_configs/testing_trusted_setups.json"); include_bytes!("../built_in_network_configs/testing_trusted_setups.json");
const TRUSTED_SETUP_MINIMAL: &[u8] =
include_bytes!("../built_in_network_configs/minimal_testing_trusted_setups.json");
pub fn get_trusted_setup<P: KzgPreset>() -> &'static [u8] {
match P::spec_name() {
KzgPresetId::Mainnet => TRUSTED_SETUP,
KzgPresetId::Minimal => TRUSTED_SETUP_MINIMAL,
}
}
pub fn get_trusted_setup_from_id(id: KzgPresetId) -> &'static [u8] {
match id {
KzgPresetId::Mainnet => TRUSTED_SETUP,
KzgPresetId::Minimal => TRUSTED_SETUP_MINIMAL,
}
}
/// Specifies an Eth2 network. /// Specifies an Eth2 network.
/// ///
/// See the crate-level documentation for more details. /// See the crate-level documentation for more details.
@ -73,7 +91,9 @@ impl Eth2NetworkConfig {
let kzg_trusted_setup = if let Some(epoch) = config.deneb_fork_epoch { let kzg_trusted_setup = if let Some(epoch) = config.deneb_fork_epoch {
// Only load the trusted setup if the deneb fork epoch is set // Only load the trusted setup if the deneb fork epoch is set
if epoch.value != Epoch::max_value() { if epoch.value != Epoch::max_value() {
let trusted_setup: TrustedSetup = serde_json::from_reader(TRUSTED_SETUP) let trusted_setup_bytes =
get_trusted_setup_from_id(KzgPresetId::from_str(&config.preset_base)?);
let trusted_setup: TrustedSetup = serde_json::from_reader(trusted_setup_bytes)
.map_err(|e| format!("Unable to read trusted setup file: {}", e))?; .map_err(|e| format!("Unable to read trusted setup file: {}", e))?;
Some(trusted_setup) Some(trusted_setup)
} else { } else {
@ -239,7 +259,9 @@ impl Eth2NetworkConfig {
let kzg_trusted_setup = if let Some(epoch) = config.deneb_fork_epoch { let kzg_trusted_setup = if let Some(epoch) = config.deneb_fork_epoch {
// Only load the trusted setup if the deneb fork epoch is set // Only load the trusted setup if the deneb fork epoch is set
if epoch.value != Epoch::max_value() { if epoch.value != Epoch::max_value() {
let trusted_setup: TrustedSetup = serde_json::from_reader(TRUSTED_SETUP) let trusted_setup: TrustedSetup = serde_json::from_reader(
get_trusted_setup_from_id(KzgPresetId::from_str(&config.preset_base)?),
)
.map_err(|e| format!("Unable to read trusted setup file: {}", e))?; .map_err(|e| format!("Unable to read trusted setup file: {}", e))?;
Some(trusted_setup) Some(trusted_setup)
} else { } else {

View File

@ -13,7 +13,6 @@ pub use self::verify_attester_slashing::{
pub use self::verify_proposer_slashing::verify_proposer_slashing; pub use self::verify_proposer_slashing::verify_proposer_slashing;
pub use altair::sync_committee::process_sync_aggregate; pub use altair::sync_committee::process_sync_aggregate;
pub use block_signature_verifier::{BlockSignatureVerifier, ParallelSignatureSets}; pub use block_signature_verifier::{BlockSignatureVerifier, ParallelSignatureSets};
pub use deneb::deneb::process_blob_kzg_commitments;
pub use is_valid_indexed_attestation::is_valid_indexed_attestation; pub use is_valid_indexed_attestation::is_valid_indexed_attestation;
pub use process_operations::process_operations; pub use process_operations::process_operations;
pub use verify_attestation::{ pub use verify_attestation::{
@ -163,11 +162,11 @@ pub fn per_block_processing<T: EthSpec, Payload: AbstractExecPayload<T>>(
// `process_randao` as the former depends on the `randao_mix` computed with the reveal of the // `process_randao` as the former depends on the `randao_mix` computed with the reveal of the
// previous block. // previous block.
if is_execution_enabled(state, block.body()) { if is_execution_enabled(state, block.body()) {
let payload = block.body().execution_payload()?; let body = block.body();
if state_processing_strategy == StateProcessingStrategy::Accurate { if state_processing_strategy == StateProcessingStrategy::Accurate {
process_withdrawals::<T, Payload>(state, payload, spec)?; process_withdrawals::<T, Payload>(state, body.execution_payload()?, spec)?;
} }
process_execution_payload::<T, Payload>(state, payload, spec)?; process_execution_payload::<T, Payload>(state, body, spec)?;
} }
process_randao(state, block, verify_randao, ctxt, spec)?; process_randao(state, block, verify_randao, ctxt, spec)?;
@ -184,8 +183,6 @@ pub fn per_block_processing<T: EthSpec, Payload: AbstractExecPayload<T>>(
)?; )?;
} }
process_blob_kzg_commitments(block.body(), ctxt)?;
Ok(()) Ok(())
} }
@ -350,9 +347,10 @@ pub fn get_new_eth1_data<T: EthSpec>(
pub fn partially_verify_execution_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<'_>, body: BeaconBlockBodyRef<T, Payload>,
spec: &ChainSpec, spec: &ChainSpec,
) -> Result<(), BlockProcessingError> { ) -> Result<(), BlockProcessingError> {
let payload = body.execution_payload()?;
if is_merge_transition_complete(state) { if is_merge_transition_complete(state) {
block_verify!( block_verify!(
payload.parent_hash() == state.latest_execution_payload_header()?.block_hash(), payload.parent_hash() == state.latest_execution_payload_header()?.block_hash(),
@ -379,6 +377,17 @@ pub fn partially_verify_execution_payload<T: EthSpec, Payload: AbstractExecPaylo
} }
); );
if let Ok(blob_commitments) = body.blob_kzg_commitments() {
// Verify commitments are under the limit.
block_verify!(
blob_commitments.len() <= T::max_blobs_per_block(),
BlockProcessingError::ExecutionInvalidBlobsLen {
max: T::max_blobs_per_block(),
actual: blob_commitments.len(),
}
);
}
Ok(()) Ok(())
} }
@ -391,11 +400,11 @@ pub fn partially_verify_execution_payload<T: EthSpec, Payload: AbstractExecPaylo
/// 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<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<'_>, body: BeaconBlockBodyRef<T, Payload>,
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(), body, spec)?;
let payload = body.execution_payload()?;
match state.latest_execution_payload_header_mut()? { match state.latest_execution_payload_header_mut()? {
ExecutionPayloadHeaderRefMut::Merge(header_mut) => { ExecutionPayloadHeaderRefMut::Merge(header_mut) => {
match payload.to_execution_payload_header() { match payload.to_execution_payload_header() {
@ -423,15 +432,19 @@ pub fn process_execution_payload<T: EthSpec, Payload: AbstractExecPayload<T>>(
/// These functions will definitely be called before the merge. Their entire purpose is to check if /// These functions will definitely be called before the merge. Their entire purpose is to check if
/// the merge has happened or if we're on the transition block. Thus we don't want to propagate /// the merge has happened or if we're on the transition block. Thus we don't want to propagate
/// errors from the `BeaconState` being an earlier variant than `BeaconStateMerge` as we'd have to /// errors from the `BeaconState` being an earlier variant than `BeaconStateMerge` as we'd have to
/// repeaetedly write code to treat these errors as false. /// repeatedly write code to treat these errors as false.
/// https://github.com/ethereum/consensus-specs/blob/dev/specs/bellatrix/beacon-chain.md#is_merge_transition_complete /// https://github.com/ethereum/consensus-specs/blob/dev/specs/bellatrix/beacon-chain.md#is_merge_transition_complete
pub fn is_merge_transition_complete<T: EthSpec>(state: &BeaconState<T>) -> bool { pub fn is_merge_transition_complete<T: EthSpec>(state: &BeaconState<T>) -> bool {
match state {
// We must check defaultness against the payload header with 0x0 roots, as that's what's meant // We must check defaultness against the payload header with 0x0 roots, as that's what's meant
// by `ExecutionPayloadHeader()` in the spec. // by `ExecutionPayloadHeader()` in the spec.
state BeaconState::Merge(_) => state
.latest_execution_payload_header() .latest_execution_payload_header()
.map(|header| !header.is_default_with_zero_roots()) .map(|header| !header.is_default_with_zero_roots())
.unwrap_or(false) .unwrap_or(false),
BeaconState::Deneb(_) | BeaconState::Capella(_) => true,
BeaconState::Base(_) | BeaconState::Altair(_) => false,
}
} }
/// https://github.com/ethereum/consensus-specs/blob/dev/specs/bellatrix/beacon-chain.md#is_merge_transition_block /// https://github.com/ethereum/consensus-specs/blob/dev/specs/bellatrix/beacon-chain.md#is_merge_transition_block
pub fn is_merge_transition_block<T: EthSpec, Payload: AbstractExecPayload<T>>( pub fn is_merge_transition_block<T: EthSpec, Payload: AbstractExecPayload<T>>(

View File

@ -1,125 +1,8 @@
use crate::{BlockProcessingError, ConsensusContext};
use ethereum_hashing::hash_fixed; use ethereum_hashing::hash_fixed;
use itertools::{EitherOrBoth, Itertools}; use types::consts::deneb::VERSIONED_HASH_VERSION_KZG;
use safe_arith::SafeArith; use types::{KzgCommitment, VersionedHash};
use ssz::Decode;
use types::consts::deneb::{BLOB_TX_TYPE, VERSIONED_HASH_VERSION_KZG};
use types::{
AbstractExecPayload, BeaconBlockBodyRef, EthSpec, ExecPayload, KzgCommitment, Transaction,
Transactions, VersionedHash,
};
pub fn process_blob_kzg_commitments<T: EthSpec, Payload: AbstractExecPayload<T>>( pub fn kzg_commitment_to_versioned_hash(kzg_commitment: &KzgCommitment) -> VersionedHash {
block_body: BeaconBlockBodyRef<T, Payload>,
ctxt: &mut ConsensusContext<T>,
) -> Result<(), BlockProcessingError> {
// Return early if this check has already been run.
if ctxt.kzg_commitments_consistent() {
return Ok(());
}
if let (Ok(payload), Ok(kzg_commitments)) = (
block_body.execution_payload(),
block_body.blob_kzg_commitments(),
) {
if let Some(transactions) = payload.transactions() {
if !verify_kzg_commitments_against_transactions::<T>(transactions, kzg_commitments)? {
return Err(BlockProcessingError::BlobVersionHashMismatch);
}
}
}
Ok(())
}
pub fn verify_kzg_commitments_against_transactions<T: EthSpec>(
transactions: &Transactions<T>,
kzg_commitments: &[KzgCommitment],
) -> Result<bool, BlockProcessingError> {
let nested_iter = transactions
.into_iter()
.filter(|tx| {
tx.first()
.map(|tx_type| *tx_type == BLOB_TX_TYPE)
.unwrap_or(false)
})
.map(|tx| tx_peek_versioned_hashes::<T>(tx));
itertools::process_results(nested_iter, |iter| {
let zipped_iter = iter
.flatten()
// Need to use `itertools::zip_longest` here because just zipping hides if one iter is shorter
// and `itertools::zip_eq` panics.
.zip_longest(kzg_commitments.iter())
.enumerate()
.map(|(index, next)| match next {
EitherOrBoth::Both(hash, commitment) => Ok((hash?, commitment)),
// The number of versioned hashes from the blob transactions exceeds the number of
// commitments in the block.
EitherOrBoth::Left(_) => Err(BlockProcessingError::BlobNumCommitmentsMismatch {
commitments_processed_in_block: index,
commitments_processed_in_transactions: index.safe_add(1)?,
}),
// The number of commitments in the block exceeds the number of versioned hashes
// in the blob transactions.
EitherOrBoth::Right(_) => Err(BlockProcessingError::BlobNumCommitmentsMismatch {
commitments_processed_in_block: index.safe_add(1)?,
commitments_processed_in_transactions: index,
}),
});
itertools::process_results(zipped_iter, |mut iter| {
iter.all(|(tx_versioned_hash, commitment)| {
tx_versioned_hash == kzg_commitment_to_versioned_hash(commitment)
})
})
})?
}
/// Only transactions of type `BLOB_TX_TYPE` should be passed into this function.
fn tx_peek_versioned_hashes<T: EthSpec>(
opaque_tx: &Transaction<T::MaxBytesPerTransaction>,
) -> Result<
impl IntoIterator<Item = Result<VersionedHash, BlockProcessingError>> + '_,
BlockProcessingError,
> {
let tx_len = opaque_tx.len();
let message_offset = 1.safe_add(u32::from_ssz_bytes(opaque_tx.get(1..5).ok_or(
BlockProcessingError::BlobVersionHashIndexOutOfBounds {
length: tx_len,
index: 5,
},
)?)?)?;
let message_offset_usize = message_offset as usize;
// field offset: 32 + 8 + 32 + 32 + 8 + 4 + 32 + 4 + 4 + 32 = 188
let versioned_hashes_offset = message_offset.safe_add(u32::from_ssz_bytes(
opaque_tx
.get(message_offset_usize.safe_add(188)?..message_offset_usize.safe_add(192)?)
.ok_or(BlockProcessingError::BlobVersionHashIndexOutOfBounds {
length: tx_len,
index: message_offset_usize.safe_add(192)?,
})?,
)?)?;
let num_hashes = tx_len
.safe_sub(versioned_hashes_offset as usize)?
.safe_div(32)?;
Ok((0..num_hashes).map(move |i| {
let next_version_hash_index =
(versioned_hashes_offset as usize).safe_add(i.safe_mul(32)?)?;
let bytes = opaque_tx
.get(next_version_hash_index..next_version_hash_index.safe_add(32)?)
.ok_or(BlockProcessingError::BlobVersionHashIndexOutOfBounds {
length: tx_len,
index: (next_version_hash_index).safe_add(32)?,
})?;
Ok(VersionedHash::from_slice(bytes))
}))
}
fn kzg_commitment_to_versioned_hash(kzg_commitment: &KzgCommitment) -> VersionedHash {
let mut hashed_commitment = hash_fixed(&kzg_commitment.0); let mut hashed_commitment = hash_fixed(&kzg_commitment.0);
hashed_commitment[0] = VERSIONED_HASH_VERSION_KZG; hashed_commitment[0] = VERSIONED_HASH_VERSION_KZG;
VersionedHash::from(hashed_commitment) VersionedHash::from(hashed_commitment)

View File

@ -76,6 +76,10 @@ pub enum BlockProcessingError {
expected: u64, expected: u64,
found: u64, found: u64,
}, },
ExecutionInvalidBlobsLen {
max: usize,
actual: usize,
},
ExecutionInvalid, ExecutionInvalid,
ConsensusContext(ContextError), ConsensusContext(ContextError),
WithdrawalsRootMismatch { WithdrawalsRootMismatch {

View File

@ -9,7 +9,8 @@ use superstruct::superstruct;
use test_random_derive::TestRandom; use test_random_derive::TestRandom;
use tree_hash_derive::TreeHash; use tree_hash_derive::TreeHash;
pub type KzgCommitments<T> = VariableList<KzgCommitment, <T as EthSpec>::MaxBlobsPerBlock>; pub type KzgCommitments<T> =
VariableList<KzgCommitment, <T as EthSpec>::MaxBlobCommitmentsPerBlock>;
/// The body of a `BeaconChain` block, containing operations. /// The body of a `BeaconChain` block, containing operations.
/// ///

View File

@ -2,7 +2,8 @@ use crate::test_utils::TestRandom;
use crate::{Blob, ChainSpec, Domain, EthSpec, Fork, Hash256, SignedBlobSidecar, SignedRoot, Slot}; use crate::{Blob, ChainSpec, Domain, EthSpec, Fork, Hash256, SignedBlobSidecar, SignedRoot, Slot};
use bls::SecretKey; use bls::SecretKey;
use derivative::Derivative; use derivative::Derivative;
use kzg::{KzgCommitment, KzgProof}; use kzg::{Kzg, KzgCommitment, KzgPreset, KzgProof};
use rand::Rng;
use serde_derive::{Deserialize, Serialize}; use serde_derive::{Deserialize, Serialize};
use ssz::Encode; use ssz::Encode;
use ssz_derive::{Decode, Encode}; use ssz_derive::{Decode, Encode};
@ -93,6 +94,38 @@ impl<T: EthSpec> BlobSidecar<T> {
Self::default() Self::default()
} }
pub fn random_valid<R: Rng>(rng: &mut R, kzg: &Kzg<T::Kzg>) -> Result<Self, String> {
let mut blob_bytes = vec![0u8; T::Kzg::BYTES_PER_BLOB];
rng.fill_bytes(&mut blob_bytes);
// Ensure that the blob is canonical by ensuring that
// each field element contained in the blob is < BLS_MODULUS
for i in 0..T::Kzg::FIELD_ELEMENTS_PER_BLOB {
let Some(byte) = blob_bytes.get_mut(i.checked_mul(T::Kzg::BYTES_PER_FIELD_ELEMENT).ok_or("overflow".to_string())?) else {
return Err(format!("blob byte index out of bounds: {:?}", i));
};
*byte = 0;
}
let blob = Blob::<T>::new(blob_bytes)
.map_err(|e| format!("error constructing random blob: {:?}", e))?;
let kzg_blob = T::blob_from_bytes(&blob).unwrap();
let commitment = kzg
.blob_to_kzg_commitment(kzg_blob.clone())
.map_err(|e| format!("error computing kzg commitment: {:?}", e))?;
let proof = kzg
.compute_blob_kzg_proof(kzg_blob, commitment)
.map_err(|e| format!("error computing kzg proof: {:?}", e))?;
Ok(Self {
blob,
kzg_commitment: commitment,
kzg_proof: proof,
..Default::default()
})
}
#[allow(clippy::integer_arithmetic)] #[allow(clippy::integer_arithmetic)]
pub fn max_size() -> usize { pub fn max_size() -> usize {
// Fixed part // Fixed part

View File

@ -34,7 +34,6 @@ pub mod deneb {
.expect("should initialize BLS_MODULUS"); .expect("should initialize BLS_MODULUS");
pub static ref MIN_EPOCHS_FOR_BLOB_SIDECARS_REQUESTS: Epoch = Epoch::from(4096_u64); pub static ref MIN_EPOCHS_FOR_BLOB_SIDECARS_REQUESTS: Epoch = Epoch::from(4096_u64);
} }
pub const BLOB_TX_TYPE: u8 = 3;
pub const VERSIONED_HASH_VERSION_KZG: u8 = 1; pub const VERSIONED_HASH_VERSION_KZG: u8 = 1;
pub const BLOB_SIDECAR_SUBNET_COUNT: u64 = 6; pub const BLOB_SIDECAR_SUBNET_COUNT: u64 = 6;
} }

View File

@ -1,5 +1,6 @@
use crate::*; use crate::*;
use kzg::{BlobTrait, KzgPreset, MainnetKzgPreset, MinimalKzgPreset};
use safe_arith::SafeArith; use safe_arith::SafeArith;
use serde_derive::{Deserialize, Serialize}; use serde_derive::{Deserialize, Serialize};
use ssz_types::typenum::{ use ssz_types::typenum::{
@ -51,6 +52,8 @@ impl fmt::Display for EthSpecId {
pub trait EthSpec: pub trait EthSpec:
'static + Default + Sync + Send + Clone + Debug + PartialEq + Eq + for<'a> arbitrary::Arbitrary<'a> 'static + Default + Sync + Send + Clone + Debug + PartialEq + Eq + for<'a> arbitrary::Arbitrary<'a>
{ {
type Kzg: KzgPreset;
/* /*
* Constants * Constants
*/ */
@ -106,6 +109,7 @@ pub trait EthSpec:
* New in Deneb * New in Deneb
*/ */
type MaxBlobsPerBlock: Unsigned + Clone + Sync + Send + Debug + PartialEq + Unpin; type MaxBlobsPerBlock: Unsigned + Clone + Sync + Send + Debug + PartialEq + Unpin;
type MaxBlobCommitmentsPerBlock: Unsigned + Clone + Sync + Send + Debug + PartialEq + Unpin;
type FieldElementsPerBlob: Unsigned + Clone + Sync + Send + Debug + PartialEq; type FieldElementsPerBlob: Unsigned + Clone + Sync + Send + Debug + PartialEq;
type BytesPerFieldElement: Unsigned + Clone + Sync + Send + Debug + PartialEq; type BytesPerFieldElement: Unsigned + Clone + Sync + Send + Debug + PartialEq;
/* /*
@ -256,6 +260,10 @@ pub trait EthSpec:
Self::MaxBlobsPerBlock::to_usize() Self::MaxBlobsPerBlock::to_usize()
} }
fn blob_from_bytes(bytes: &[u8]) -> Result<<Self::Kzg as KzgPreset>::Blob, kzg::Error> {
<Self::Kzg as KzgPreset>::Blob::from_bytes(bytes)
}
/// Returns the `BYTES_PER_BLOB` constant for this specification. /// Returns the `BYTES_PER_BLOB` constant for this specification.
fn bytes_per_blob() -> usize { fn bytes_per_blob() -> usize {
Self::BytesPerBlob::to_usize() Self::BytesPerBlob::to_usize()
@ -275,6 +283,8 @@ macro_rules! params_from_eth_spec {
pub struct MainnetEthSpec; pub struct MainnetEthSpec;
impl EthSpec for MainnetEthSpec { impl EthSpec for MainnetEthSpec {
type Kzg = MainnetKzgPreset;
type JustificationBitsLength = U4; type JustificationBitsLength = U4;
type SubnetBitfieldLength = U64; type SubnetBitfieldLength = U64;
type MaxValidatorsPerCommittee = U2048; type MaxValidatorsPerCommittee = U2048;
@ -300,6 +310,7 @@ impl EthSpec for MainnetEthSpec {
type MinGasLimit = U5000; type MinGasLimit = U5000;
type MaxExtraDataBytes = U32; type MaxExtraDataBytes = U32;
type MaxBlobsPerBlock = U6; type MaxBlobsPerBlock = U6;
type MaxBlobCommitmentsPerBlock = U4096;
type BytesPerFieldElement = U32; type BytesPerFieldElement = U32;
type FieldElementsPerBlob = U4096; type FieldElementsPerBlob = U4096;
type BytesPerBlob = U131072; type BytesPerBlob = U131072;
@ -323,6 +334,8 @@ impl EthSpec for MainnetEthSpec {
pub struct MinimalEthSpec; pub struct MinimalEthSpec;
impl EthSpec for MinimalEthSpec { impl EthSpec for MinimalEthSpec {
type Kzg = MinimalKzgPreset;
type SlotsPerEpoch = U8; type SlotsPerEpoch = U8;
type EpochsPerEth1VotingPeriod = U4; type EpochsPerEth1VotingPeriod = U4;
type SlotsPerHistoricalRoot = U64; type SlotsPerHistoricalRoot = U64;
@ -335,6 +348,7 @@ impl EthSpec for MinimalEthSpec {
type MaxWithdrawalsPerPayload = U4; type MaxWithdrawalsPerPayload = U4;
type FieldElementsPerBlob = U4; //FIXME(sean) this is spec'd out currently but will likely change type FieldElementsPerBlob = U4; //FIXME(sean) this is spec'd out currently but will likely change
type BytesPerBlob = U128; //FIXME(sean) this is spec'd out currently but will likely change type BytesPerBlob = U128; //FIXME(sean) this is spec'd out currently but will likely change
type MaxBlobCommitmentsPerBlock = U16;
params_from_eth_spec!(MainnetEthSpec { params_from_eth_spec!(MainnetEthSpec {
JustificationBitsLength, JustificationBitsLength,
@ -374,6 +388,8 @@ impl EthSpec for MinimalEthSpec {
pub struct GnosisEthSpec; pub struct GnosisEthSpec;
impl EthSpec for GnosisEthSpec { impl EthSpec for GnosisEthSpec {
type Kzg = MainnetKzgPreset;
type JustificationBitsLength = U4; type JustificationBitsLength = U4;
type SubnetBitfieldLength = U64; type SubnetBitfieldLength = U64;
type MaxValidatorsPerCommittee = U2048; type MaxValidatorsPerCommittee = U2048;
@ -404,6 +420,7 @@ impl EthSpec for GnosisEthSpec {
type MaxBlsToExecutionChanges = U16; type MaxBlsToExecutionChanges = U16;
type MaxWithdrawalsPerPayload = U8; type MaxWithdrawalsPerPayload = U8;
type MaxBlobsPerBlock = U6; type MaxBlobsPerBlock = U6;
type MaxBlobCommitmentsPerBlock = U4096;
type FieldElementsPerBlob = U4096; type FieldElementsPerBlob = U4096;
type BytesPerFieldElement = U32; type BytesPerFieldElement = U32;
type BytesPerBlob = U131072; type BytesPerBlob = U131072;

View File

@ -26,6 +26,7 @@ use metastruct::metastruct;
#[derive(Debug, Clone, PartialEq, Eq, Hash)] #[derive(Debug, Clone, PartialEq, Eq, Hash)]
#[metastruct(mappings(map_execution_block_header_fields_except_withdrawals(exclude( #[metastruct(mappings(map_execution_block_header_fields_except_withdrawals(exclude(
withdrawals_root, withdrawals_root,
data_gas_used,
excess_data_gas excess_data_gas
)),))] )),))]
pub struct ExecutionBlockHeader { pub struct ExecutionBlockHeader {
@ -46,7 +47,8 @@ pub struct ExecutionBlockHeader {
pub nonce: Hash64, pub nonce: Hash64,
pub base_fee_per_gas: Uint256, pub base_fee_per_gas: Uint256,
pub withdrawals_root: Option<Hash256>, pub withdrawals_root: Option<Hash256>,
pub excess_data_gas: Option<Uint256>, pub data_gas_used: Option<u64>,
pub excess_data_gas: Option<u64>,
} }
impl ExecutionBlockHeader { impl ExecutionBlockHeader {
@ -55,7 +57,8 @@ impl ExecutionBlockHeader {
rlp_empty_list_root: Hash256, rlp_empty_list_root: Hash256,
rlp_transactions_root: Hash256, rlp_transactions_root: Hash256,
rlp_withdrawals_root: Option<Hash256>, rlp_withdrawals_root: Option<Hash256>,
rlp_excess_data_gas: Option<Uint256>, rlp_data_gas_used: Option<u64>,
rlp_excess_data_gas: Option<u64>,
) -> Self { ) -> Self {
// Most of these field mappings are defined in EIP-3675 except for `mixHash`, which is // Most of these field mappings are defined in EIP-3675 except for `mixHash`, which is
// defined in EIP-4399. // defined in EIP-4399.
@ -77,6 +80,7 @@ impl ExecutionBlockHeader {
nonce: Hash64::zero(), nonce: Hash64::zero(),
base_fee_per_gas: payload.base_fee_per_gas(), base_fee_per_gas: payload.base_fee_per_gas(),
withdrawals_root: rlp_withdrawals_root, withdrawals_root: rlp_withdrawals_root,
data_gas_used: rlp_data_gas_used,
excess_data_gas: rlp_excess_data_gas, excess_data_gas: rlp_excess_data_gas,
} }
} }

View File

@ -84,9 +84,13 @@ pub struct ExecutionPayload<T: EthSpec> {
#[superstruct(only(Capella, Deneb))] #[superstruct(only(Capella, Deneb))]
pub withdrawals: Withdrawals<T>, pub withdrawals: Withdrawals<T>,
#[superstruct(only(Deneb))] #[superstruct(only(Deneb))]
#[serde(with = "serde_utils::quoted_u256")] #[serde(with = "serde_utils::quoted_u64")]
#[superstruct(getter(copy))] #[superstruct(getter(copy))]
pub excess_data_gas: Uint256, pub data_gas_used: u64,
#[superstruct(only(Deneb))]
#[serde(with = "serde_utils::quoted_u64")]
#[superstruct(getter(copy))]
pub excess_data_gas: u64,
} }
impl<'a, T: EthSpec> ExecutionPayloadRef<'a, T> { impl<'a, T: EthSpec> ExecutionPayloadRef<'a, T> {

View File

@ -78,9 +78,13 @@ pub struct ExecutionPayloadHeader<T: EthSpec> {
#[superstruct(getter(copy))] #[superstruct(getter(copy))]
pub withdrawals_root: Hash256, pub withdrawals_root: Hash256,
#[superstruct(only(Deneb))] #[superstruct(only(Deneb))]
#[serde(with = "serde_utils::quoted_u256")] #[serde(with = "serde_utils::quoted_u64")]
#[superstruct(getter(copy))] #[superstruct(getter(copy))]
pub excess_data_gas: Uint256, pub data_gas_used: u64,
#[superstruct(only(Deneb))]
#[serde(with = "serde_utils::quoted_u64")]
#[superstruct(getter(copy))]
pub excess_data_gas: u64,
} }
impl<T: EthSpec> ExecutionPayloadHeader<T> { impl<T: EthSpec> ExecutionPayloadHeader<T> {
@ -151,8 +155,8 @@ impl<T: EthSpec> ExecutionPayloadHeaderCapella<T> {
block_hash: self.block_hash, block_hash: self.block_hash,
transactions_root: self.transactions_root, transactions_root: self.transactions_root,
withdrawals_root: self.withdrawals_root, withdrawals_root: self.withdrawals_root,
// TODO: verify if this is correct data_gas_used: 0,
excess_data_gas: Uint256::zero(), excess_data_gas: 0,
} }
} }
} }
@ -217,6 +221,7 @@ impl<'a, T: EthSpec> From<&'a ExecutionPayloadDeneb<T>> for ExecutionPayloadHead
block_hash: payload.block_hash, block_hash: payload.block_hash,
transactions_root: payload.transactions.tree_hash_root(), transactions_root: payload.transactions.tree_hash_root(),
withdrawals_root: payload.withdrawals.tree_hash_root(), withdrawals_root: payload.withdrawals.tree_hash_root(),
data_gas_used: payload.data_gas_used,
excess_data_gas: payload.excess_data_gas, excess_data_gas: payload.excess_data_gas,
} }
} }

View File

@ -99,7 +99,6 @@ pub mod sqlite;
pub mod blob_sidecar; pub mod blob_sidecar;
pub mod signed_blob; pub mod signed_blob;
pub mod transaction;
use ethereum_types::{H160, H256}; use ethereum_types::{H160, H256};

View File

@ -1,44 +0,0 @@
use crate::{Hash256, Uint256, VersionedHash};
use ethereum_types::Address;
use ssz_derive::{Decode, Encode};
use ssz_types::typenum::{U16777216, U4096};
use ssz_types::VariableList;
pub type MaxCalldataSize = U16777216;
pub type MaxAccessListSize = U16777216;
pub type MaxAccessListStorageKeys = U16777216;
pub type MaxVersionedHashesListSize = U4096;
#[derive(Debug, Clone, PartialEq, Encode, Decode)]
pub struct SignedBlobTransaction {
pub message: BlobTransaction,
pub signature: EcdsaSignature,
}
#[derive(Debug, Clone, PartialEq, Encode, Decode)]
pub struct BlobTransaction {
pub chain_id: Uint256,
pub nonce: u64,
pub max_priority_fee_per_gas: Uint256,
pub max_fee_per_gas: Uint256,
pub gas: u64,
pub to: Option<Address>,
pub value: Uint256,
pub data: VariableList<u8, MaxCalldataSize>,
pub access_list: VariableList<AccessTuple, MaxAccessListSize>,
pub max_fee_per_data_gas: Uint256,
pub versioned_hashes: VariableList<VersionedHash, MaxVersionedHashesListSize>,
}
#[derive(Debug, Clone, PartialEq, Encode, Decode)]
pub struct AccessTuple {
pub address: Address,
pub storage_keys: VariableList<Hash256, MaxAccessListStorageKeys>,
}
#[derive(Debug, Clone, PartialEq, Encode, Decode)]
pub struct EcdsaSignature {
pub y_parity: bool,
pub r: Uint256,
pub s: Uint256,
}

View File

@ -16,10 +16,11 @@ serde_derive = "1.0.116"
ethereum_serde_utils = "0.5.0" ethereum_serde_utils = "0.5.0"
hex = "0.4.2" hex = "0.4.2"
ethereum_hashing = "1.0.0-beta.2" ethereum_hashing = "1.0.0-beta.2"
c-kzg = {git = "https://github.com/ethereum/c-kzg-4844", rev = "fd24cf8e1e2f09a96b4e62a595b4e49f046ce6cf" } c-kzg = { git = "https://github.com/ethereum/c-kzg-4844", rev = "13cec820c08f45318f82ed4e0da0300042758b92" , features = ["mainnet-spec"]}
c_kzg_min = { package = "c-kzg", git = "https://github.com/ethereum//c-kzg-4844", rev = "13cec820c08f45318f82ed4e0da0300042758b92", features = ["minimal-spec"], optional = true }
arbitrary = { version = "1.0", features = ["derive"], optional = true } arbitrary = { version = "1.0", features = ["derive"], optional = true }
[features] [features]
default = ["mainnet-spec"] # TODO(deneb): enabled by default for convenience, would need more cfg magic to disable
mainnet-spec = ["c-kzg/mainnet-spec"] default = ["c_kzg_min"]
minimal-spec = ["c-kzg/minimal-spec"] minimal-spec = ["c_kzg_min"]

View File

@ -1,4 +1,4 @@
use c_kzg::{Bytes48, BYTES_PER_COMMITMENT}; use c_kzg::BYTES_PER_COMMITMENT;
use derivative::Derivative; use derivative::Derivative;
use ethereum_hashing::hash_fixed; use ethereum_hashing::hash_fixed;
use serde::de::{Deserialize, Deserializer}; use serde::de::{Deserialize, Deserializer};
@ -14,7 +14,7 @@ pub const BLOB_COMMITMENT_VERSION_KZG: u8 = 0x01;
#[derive(Derivative, Clone, Copy, Encode, Decode)] #[derive(Derivative, Clone, Copy, Encode, Decode)]
#[derivative(PartialEq, Eq, Hash)] #[derivative(PartialEq, Eq, Hash)]
#[ssz(struct_behaviour = "transparent")] #[ssz(struct_behaviour = "transparent")]
pub struct KzgCommitment(pub [u8; BYTES_PER_COMMITMENT]); pub struct KzgCommitment(pub [u8; c_kzg::BYTES_PER_COMMITMENT]);
impl KzgCommitment { impl KzgCommitment {
pub fn calculate_versioned_hash(&self) -> Hash256 { pub fn calculate_versioned_hash(&self) -> Hash256 {
@ -24,7 +24,13 @@ impl KzgCommitment {
} }
} }
impl From<KzgCommitment> for Bytes48 { impl From<KzgCommitment> for c_kzg::Bytes48 {
fn from(value: KzgCommitment) -> Self {
value.0.into()
}
}
impl From<KzgCommitment> for c_kzg_min::Bytes48 {
fn from(value: KzgCommitment) -> Self { fn from(value: KzgCommitment) -> Self {
value.0.into() value.0.into()
} }

View File

@ -1,4 +1,4 @@
use c_kzg::{Bytes48, BYTES_PER_PROOF}; use c_kzg::BYTES_PER_PROOF;
use serde::de::{Deserialize, Deserializer}; use serde::de::{Deserialize, Deserializer};
use serde::ser::{Serialize, Serializer}; use serde::ser::{Serialize, Serializer};
use ssz_derive::{Decode, Encode}; use ssz_derive::{Decode, Encode};
@ -11,7 +11,13 @@ use tree_hash::{PackedEncoding, TreeHash};
#[ssz(struct_behaviour = "transparent")] #[ssz(struct_behaviour = "transparent")]
pub struct KzgProof(pub [u8; BYTES_PER_PROOF]); pub struct KzgProof(pub [u8; BYTES_PER_PROOF]);
impl From<KzgProof> for Bytes48 { impl From<KzgProof> for c_kzg::Bytes48 {
fn from(value: KzgProof) -> Self {
value.0.into()
}
}
impl From<KzgProof> for c_kzg_min::Bytes48 {
fn from(value: KzgProof) -> Self { fn from(value: KzgProof) -> Self {
value.0.into() value.0.into()
} }

View File

@ -2,30 +2,263 @@ mod kzg_commitment;
mod kzg_proof; mod kzg_proof;
mod trusted_setup; mod trusted_setup;
use serde_derive::{Deserialize, Serialize};
use std::fmt::Debug;
use std::ops::Deref;
use std::str::FromStr;
pub use crate::{kzg_commitment::KzgCommitment, kzg_proof::KzgProof, trusted_setup::TrustedSetup}; pub use crate::{kzg_commitment::KzgCommitment, kzg_proof::KzgProof, trusted_setup::TrustedSetup};
pub use c_kzg::{ pub use c_kzg::{Bytes32, Bytes48};
Blob, Error as CKzgError, KzgSettings, BYTES_PER_BLOB, BYTES_PER_FIELD_ELEMENT,
FIELD_ELEMENTS_PER_BLOB,
};
use c_kzg::{Bytes32, Bytes48};
use std::path::PathBuf;
#[derive(Debug)] #[derive(Debug)]
pub enum Error { pub enum Error {
InvalidTrustedSetup(CKzgError), InvalidTrustedSetup(CryptoError),
InvalidKzgProof(CKzgError), InvalidKzgProof(CryptoError),
InvalidBytes(CKzgError), InvalidBytes(CryptoError),
KzgProofComputationFailed(CKzgError), KzgProofComputationFailed(CryptoError),
InvalidBlob(CKzgError), InvalidBlob(CryptoError),
InvalidBytesForBlob(CryptoError),
} }
#[derive(Debug)]
pub enum CryptoError {
CKzg(c_kzg::Error),
CKzgMin(c_kzg_min::Error),
}
impl From<c_kzg::Error> for CryptoError {
fn from(e: c_kzg::Error) -> Self {
Self::CKzg(e)
}
}
impl From<c_kzg_min::Error> for CryptoError {
fn from(e: c_kzg_min::Error) -> Self {
Self::CKzgMin(e)
}
}
pub trait BlobTrait: Sized + Clone {
fn from_bytes(bytes: &[u8]) -> Result<Self, Error>;
}
pub enum KzgPresetId {
Mainnet,
Minimal,
}
impl FromStr for KzgPresetId {
type Err = String;
fn from_str(s: &str) -> Result<Self, Self::Err> {
match s {
"mainnet" => Ok(KzgPresetId::Mainnet),
"minimal" => Ok(KzgPresetId::Minimal),
_ => Err(format!("Unknown eth spec: {}", s)),
}
}
}
pub trait KzgPreset:
'static + Default + Sync + Send + Clone + Debug + PartialEq + Eq + for<'a> arbitrary::Arbitrary<'a>
{
type KzgSettings: Debug + Sync + Send;
type Blob: BlobTrait;
type Bytes32: From<[u8; 32]> + Deref<Target = [u8; 32]>;
type Bytes48: From<KzgCommitment> + From<KzgProof>;
type Error: Into<CryptoError>;
const BYTES_PER_BLOB: usize;
const BYTES_PER_FIELD_ELEMENT: usize;
const FIELD_ELEMENTS_PER_BLOB: usize;
fn spec_name() -> KzgPresetId;
fn bytes32_in(bytes: Bytes32) -> Self::Bytes32 {
let bytes: [u8; 32] = *bytes;
Self::Bytes32::from(bytes)
}
fn bytes32_out(bytes: Self::Bytes32) -> Bytes32 {
let bytes: [u8; 32] = *bytes;
Bytes32::from(bytes)
}
fn load_trusted_setup(trusted_setup: TrustedSetup) -> Result<Self::KzgSettings, CryptoError>;
fn compute_blob_kzg_proof(
blob: Self::Blob,
kzg_commitment: KzgCommitment,
trusted_setup: &Self::KzgSettings,
) -> Result<KzgProof, CryptoError>;
fn verify_blob_kzg_proof(
blob: Self::Blob,
kzg_commitment: KzgCommitment,
kzg_proof: KzgProof,
trusted_setup: &Self::KzgSettings,
) -> Result<bool, CryptoError>;
fn verify_blob_kzg_proof_batch(
blobs: &[Self::Blob],
commitments_bytes: &[Self::Bytes48],
proofs_bytes: &[Self::Bytes48],
trusted_setup: &Self::KzgSettings,
) -> Result<bool, CryptoError>;
fn blob_to_kzg_commitment(
blob: Self::Blob,
trusted_setup: &Self::KzgSettings,
) -> Result<KzgCommitment, CryptoError>;
fn compute_kzg_proof(
blob: Self::Blob,
z: Self::Bytes32,
trusted_setup: &Self::KzgSettings,
) -> Result<(KzgProof, Self::Bytes32), CryptoError>;
fn verify_kzg_proof(
kzg_commitment: KzgCommitment,
z: Self::Bytes32,
y: Self::Bytes32,
kzg_proof: KzgProof,
trusted_setup: &Self::KzgSettings,
) -> Result<bool, CryptoError>;
}
macro_rules! implement_kzg_preset {
($preset_type:ident, $module_name:ident, $preset_id:ident) => {
impl KzgPreset for $preset_type {
type KzgSettings = $module_name::KzgSettings;
type Blob = $module_name::Blob;
type Bytes32 = $module_name::Bytes32;
type Bytes48 = $module_name::Bytes48;
type Error = $module_name::Error;
const BYTES_PER_BLOB: usize = $module_name::BYTES_PER_BLOB;
const BYTES_PER_FIELD_ELEMENT: usize = $module_name::BYTES_PER_FIELD_ELEMENT;
const FIELD_ELEMENTS_PER_BLOB: usize = $module_name::FIELD_ELEMENTS_PER_BLOB;
fn spec_name() -> KzgPresetId {
KzgPresetId::$preset_id
}
fn load_trusted_setup(
trusted_setup: TrustedSetup,
) -> Result<Self::KzgSettings, CryptoError> {
$module_name::KzgSettings::load_trusted_setup(
trusted_setup.g1_points(),
trusted_setup.g2_points(),
)
.map_err(CryptoError::from)
}
fn compute_blob_kzg_proof(
blob: Self::Blob,
kzg_commitment: KzgCommitment,
trusted_setup: &Self::KzgSettings,
) -> Result<KzgProof, CryptoError> {
$module_name::KzgProof::compute_blob_kzg_proof(
blob,
kzg_commitment.into(),
trusted_setup,
)
.map(|proof| KzgProof(proof.to_bytes().into_inner()))
.map_err(CryptoError::from)
}
fn verify_blob_kzg_proof(
blob: Self::Blob,
kzg_commitment: KzgCommitment,
kzg_proof: KzgProof,
trusted_setup: &Self::KzgSettings,
) -> Result<bool, CryptoError> {
$module_name::KzgProof::verify_blob_kzg_proof(
blob,
kzg_commitment.into(),
kzg_proof.into(),
trusted_setup,
)
.map_err(CryptoError::from)
}
fn verify_blob_kzg_proof_batch(
blobs: &[Self::Blob],
commitments_bytes: &[Self::Bytes48],
proofs_bytes: &[Self::Bytes48],
trusted_setup: &Self::KzgSettings,
) -> Result<bool, CryptoError> {
$module_name::KzgProof::verify_blob_kzg_proof_batch(
blobs,
commitments_bytes,
proofs_bytes,
trusted_setup,
)
.map_err(CryptoError::from)
}
fn blob_to_kzg_commitment(
blob: Self::Blob,
trusted_setup: &Self::KzgSettings,
) -> Result<KzgCommitment, CryptoError> {
$module_name::KzgCommitment::blob_to_kzg_commitment(blob, trusted_setup)
.map(|com| KzgCommitment(com.to_bytes().into_inner()))
.map_err(CryptoError::from)
}
fn compute_kzg_proof(
blob: Self::Blob,
z: Self::Bytes32,
trusted_setup: &Self::KzgSettings,
) -> Result<(KzgProof, Self::Bytes32), CryptoError> {
$module_name::KzgProof::compute_kzg_proof(blob, z, trusted_setup)
.map(|(proof, y)| (KzgProof(proof.to_bytes().into_inner()), y))
.map_err(CryptoError::from)
}
fn verify_kzg_proof(
kzg_commitment: KzgCommitment,
z: Self::Bytes32,
y: Self::Bytes32,
kzg_proof: KzgProof,
trusted_setup: &Self::KzgSettings,
) -> Result<bool, CryptoError> {
$module_name::KzgProof::verify_kzg_proof(
kzg_commitment.into(),
z,
y,
kzg_proof.into(),
trusted_setup,
)
.map_err(CryptoError::from)
}
}
impl BlobTrait for $module_name::Blob {
fn from_bytes(bytes: &[u8]) -> Result<Self, Error> {
Self::from_bytes(bytes)
.map_err(CryptoError::from)
.map_err(Error::InvalidBlob)
}
}
};
}
#[derive(Clone, PartialEq, Eq, Debug, Default, Serialize, Deserialize, arbitrary::Arbitrary)]
pub struct MainnetKzgPreset;
#[derive(Clone, PartialEq, Eq, Debug, Default, Serialize, Deserialize, arbitrary::Arbitrary)]
pub struct MinimalKzgPreset;
implement_kzg_preset!(MainnetKzgPreset, c_kzg, Mainnet);
implement_kzg_preset!(MinimalKzgPreset, c_kzg_min, Minimal);
/// A wrapper over a kzg library that holds the trusted setup parameters. /// A wrapper over a kzg library that holds the trusted setup parameters.
#[derive(Debug)] #[derive(Debug)]
pub struct Kzg { pub struct Kzg<P: KzgPreset> {
trusted_setup: KzgSettings, trusted_setup: P::KzgSettings,
} }
impl Kzg { impl<P: KzgPreset> Kzg<P> {
/// Load the kzg trusted setup parameters from a vec of G1 and G2 points. /// Load the kzg trusted setup parameters from a vec of G1 and G2 points.
/// ///
/// The number of G1 points should be equal to FIELD_ELEMENTS_PER_BLOB /// The number of G1 points should be equal to FIELD_ELEMENTS_PER_BLOB
@ -33,22 +266,7 @@ impl Kzg {
/// The number of G2 points should be equal to 65. /// The number of G2 points should be equal to 65.
pub fn new_from_trusted_setup(trusted_setup: TrustedSetup) -> Result<Self, Error> { pub fn new_from_trusted_setup(trusted_setup: TrustedSetup) -> Result<Self, Error> {
Ok(Self { Ok(Self {
trusted_setup: KzgSettings::load_trusted_setup( trusted_setup: P::load_trusted_setup(trusted_setup)
trusted_setup.g1_points(),
trusted_setup.g2_points(),
)
.map_err(Error::InvalidTrustedSetup)?,
})
}
/// Loads a trusted setup given the path to the file containing the trusted setup values.
/// The format is specified in `c_kzg::KzgSettings::load_trusted_setup_file`.
///
/// Note: This function will likely be deprecated. Use `Kzg::new_from_trusted_setup` instead.
#[deprecated]
pub fn new_from_file(file_path: PathBuf) -> Result<Self, Error> {
Ok(Self {
trusted_setup: KzgSettings::load_trusted_setup_file(file_path)
.map_err(Error::InvalidTrustedSetup)?, .map_err(Error::InvalidTrustedSetup)?,
}) })
} }
@ -56,27 +274,21 @@ impl Kzg {
/// Compute the kzg proof given a blob and its kzg commitment. /// Compute the kzg proof given a blob and its kzg commitment.
pub fn compute_blob_kzg_proof( pub fn compute_blob_kzg_proof(
&self, &self,
blob: Blob, blob: P::Blob,
kzg_commitment: KzgCommitment, kzg_commitment: KzgCommitment,
) -> Result<KzgProof, Error> { ) -> Result<KzgProof, Error> {
c_kzg::KzgProof::compute_blob_kzg_proof(blob, kzg_commitment.into(), &self.trusted_setup) P::compute_blob_kzg_proof(blob, kzg_commitment, &self.trusted_setup)
.map_err(Error::KzgProofComputationFailed) .map_err(Error::KzgProofComputationFailed)
.map(|proof| KzgProof(proof.to_bytes().into_inner()))
} }
/// Verify a kzg proof given the blob, kzg commitment and kzg proof. /// Verify a kzg proof given the blob, kzg commitment and kzg proof.
pub fn verify_blob_kzg_proof( pub fn verify_blob_kzg_proof(
&self, &self,
blob: Blob, blob: P::Blob,
kzg_commitment: KzgCommitment, kzg_commitment: KzgCommitment,
kzg_proof: KzgProof, kzg_proof: KzgProof,
) -> Result<bool, Error> { ) -> Result<bool, Error> {
c_kzg::KzgProof::verify_blob_kzg_proof( P::verify_blob_kzg_proof(blob, kzg_commitment, kzg_proof, &self.trusted_setup)
blob,
kzg_commitment.into(),
kzg_proof.into(),
&self.trusted_setup,
)
.map_err(Error::InvalidKzgProof) .map_err(Error::InvalidKzgProof)
} }
@ -86,22 +298,21 @@ impl Kzg {
/// TODO(pawan): test performance against a parallelized rayon impl. /// TODO(pawan): test performance against a parallelized rayon impl.
pub fn verify_blob_kzg_proof_batch( pub fn verify_blob_kzg_proof_batch(
&self, &self,
blobs: &[Blob], blobs: &[P::Blob],
kzg_commitments: &[KzgCommitment], kzg_commitments: &[KzgCommitment],
kzg_proofs: &[KzgProof], kzg_proofs: &[KzgProof],
) -> Result<bool, Error> { ) -> Result<bool, Error> {
let commitments_bytes = kzg_commitments let commitments_bytes = kzg_commitments
.iter() .iter()
.map(|comm| Bytes48::from_bytes(&comm.0)) .map(|comm| P::Bytes48::from(*comm))
.collect::<Result<Vec<Bytes48>, _>>() .collect::<Vec<_>>();
.map_err(Error::InvalidBytes)?;
let proofs_bytes = kzg_proofs let proofs_bytes = kzg_proofs
.iter() .iter()
.map(|proof| Bytes48::from_bytes(&proof.0)) .map(|proof| P::Bytes48::from(*proof))
.collect::<Result<Vec<Bytes48>, _>>() .collect::<Vec<_>>();
.map_err(Error::InvalidBytes)?;
c_kzg::KzgProof::verify_blob_kzg_proof_batch( P::verify_blob_kzg_proof_batch(
blobs, blobs,
&commitments_bytes, &commitments_bytes,
&proofs_bytes, &proofs_bytes,
@ -111,17 +322,19 @@ impl Kzg {
} }
/// Converts a blob to a kzg commitment. /// Converts a blob to a kzg commitment.
pub fn blob_to_kzg_commitment(&self, blob: Blob) -> Result<KzgCommitment, Error> { pub fn blob_to_kzg_commitment(&self, blob: P::Blob) -> Result<KzgCommitment, Error> {
c_kzg::KzgCommitment::blob_to_kzg_commitment(blob, &self.trusted_setup) P::blob_to_kzg_commitment(blob, &self.trusted_setup).map_err(Error::InvalidBlob)
.map_err(Error::InvalidBlob)
.map(|com| KzgCommitment(com.to_bytes().into_inner()))
} }
/// Computes the kzg proof for a given `blob` and an evaluation point `z` /// Computes the kzg proof for a given `blob` and an evaluation point `z`
pub fn compute_kzg_proof(&self, blob: Blob, z: Bytes32) -> Result<(KzgProof, Bytes32), Error> { pub fn compute_kzg_proof(
c_kzg::KzgProof::compute_kzg_proof(blob, z, &self.trusted_setup) &self,
blob: P::Blob,
z: Bytes32,
) -> Result<(KzgProof, Bytes32), Error> {
P::compute_kzg_proof(blob, P::bytes32_in(z), &self.trusted_setup)
.map_err(Error::KzgProofComputationFailed) .map_err(Error::KzgProofComputationFailed)
.map(|(proof, y)| (KzgProof(proof.to_bytes().into_inner()), y)) .map(|(proof, y)| (proof, P::bytes32_out(y)))
} }
/// Verifies a `kzg_proof` for a `kzg_commitment` that evaluating a polynomial at `z` results in `y` /// Verifies a `kzg_proof` for a `kzg_commitment` that evaluating a polynomial at `z` results in `y`
@ -132,11 +345,11 @@ impl Kzg {
y: Bytes32, y: Bytes32,
kzg_proof: KzgProof, kzg_proof: KzgProof,
) -> Result<bool, Error> { ) -> Result<bool, Error> {
c_kzg::KzgProof::verify_kzg_proof( P::verify_kzg_proof(
kzg_commitment.into(), kzg_commitment,
z, P::bytes32_in(z),
y, P::bytes32_in(y),
kzg_proof.into(), kzg_proof,
&self.trusted_setup, &self.trusted_setup,
) )
.map_err(Error::InvalidKzgProof) .map_err(Error::InvalidKzgProof)

View File

@ -21,7 +21,7 @@ struct G2Point([u8; BYTES_PER_G2_POINT]);
/// See https://github.com/ethereum/consensus-specs/blob/dev/presets/mainnet/trusted_setups/testing_trusted_setups.json /// See https://github.com/ethereum/consensus-specs/blob/dev/presets/mainnet/trusted_setups/testing_trusted_setups.json
#[derive(Debug, Clone, PartialEq, Serialize, Deserialize)] #[derive(Debug, Clone, PartialEq, Serialize, Deserialize)]
pub struct TrustedSetup { pub struct TrustedSetup {
#[serde(rename = "setup_G1")] #[serde(rename = "setup_G1_lagrange")]
#[serde(deserialize_with = "deserialize_g1_points")] #[serde(deserialize_with = "deserialize_g1_points")]
g1_points: Vec<G1Point>, g1_points: Vec<G1Point>,
#[serde(rename = "setup_G2")] #[serde(rename = "setup_G2")]

View File

@ -1,7 +1,7 @@
use account_utils::eth2_keystore::keypair_from_secret; use account_utils::eth2_keystore::keypair_from_secret;
use clap::ArgMatches; use clap::ArgMatches;
use clap_utils::{parse_optional, parse_required, parse_ssz_optional}; use clap_utils::{parse_optional, parse_required, parse_ssz_optional};
use eth2_network_config::{Eth2NetworkConfig, TRUSTED_SETUP}; use eth2_network_config::{get_trusted_setup, Eth2NetworkConfig};
use eth2_wallet::bip39::Seed; use eth2_wallet::bip39::Seed;
use eth2_wallet::bip39::{Language, Mnemonic}; use eth2_wallet::bip39::{Language, Mnemonic};
use eth2_wallet::{recover_validator_secret_from_mnemonic, KeyType}; use eth2_wallet::{recover_validator_secret_from_mnemonic, KeyType};
@ -199,7 +199,8 @@ pub fn run<T: EthSpec>(testnet_dir_path: PathBuf, matches: &ArgMatches) -> Resul
let kzg_trusted_setup = if let Some(epoch) = spec.deneb_fork_epoch { let kzg_trusted_setup = if let Some(epoch) = spec.deneb_fork_epoch {
// Only load the trusted setup if the deneb fork epoch is set // Only load the trusted setup if the deneb fork epoch is set
if epoch != Epoch::max_value() { if epoch != Epoch::max_value() {
let trusted_setup: TrustedSetup = serde_json::from_reader(TRUSTED_SETUP) let trusted_setup: TrustedSetup =
serde_json::from_reader(get_trusted_setup::<T::Kzg>())
.map_err(|e| format!("Unable to read trusted setup file: {}", e))?; .map_err(|e| format!("Unable to read trusted setup file: {}", e))?;
Some(trusted_setup) Some(trusted_setup)
} else { } else {

View File

@ -108,9 +108,9 @@ echo $GENESIS_TIME
CAPELLA_TIME=$((GENESIS_TIME + (CAPELLA_FORK_EPOCH * 32 * SECONDS_PER_SLOT))) CAPELLA_TIME=$((GENESIS_TIME + (CAPELLA_FORK_EPOCH * 32 * SECONDS_PER_SLOT)))
echo $CAPELLA_TIME echo $CAPELLA_TIME
sed -i 's/"shanghaiTime".*$/"shanghaiTime": '"$CAPELLA_TIME"',/g' $genesis_file sed -i 's/"shanghaiTime".*$/"shanghaiTime": '"$CAPELLA_TIME"',/g' $genesis_file
DENEB_TIME=$((GENESIS_TIME + (DENEB_FORK_EPOCH * 32 * SECONDS_PER_SLOT))) CANCUN_TIME=$((GENESIS_TIME + (DENEB_FORK_EPOCH * 32 * SECONDS_PER_SLOT)))
echo $DENEB_TIME echo $CANCUN_TIME
sed -i 's/"shardingForkTime".*$/"shardingForkTime": '"$DENEB_TIME"',/g' $genesis_file sed -i 's/"shardingForkTime".*$/"shardingForkTime": '"$CANCUN_TIME"',/g' $genesis_file
cat $genesis_file cat $genesis_file
# Delay to let boot_enr.yaml to be created # Delay to let boot_enr.yaml to be created

View File

@ -1,4 +1,4 @@
TESTS_TAG := v1.3.0 TESTS_TAG := v1.4.0-alpha.2
TESTS = general minimal mainnet TESTS = general minimal mainnet
TARBALLS = $(patsubst %,%-$(TESTS_TAG).tar.gz,$(TESTS)) TARBALLS = $(patsubst %,%-$(TESTS_TAG).tar.gz,$(TESTS))

View File

@ -54,7 +54,7 @@ excluded_paths = [
# FIXME(sean) # FIXME(sean)
"tests/mainnet/capella/light_client/single_merkle_proof/BeaconBlockBody/*", "tests/mainnet/capella/light_client/single_merkle_proof/BeaconBlockBody/*",
"tests/mainnet/deneb/light_client/single_merkle_proof/BeaconBlockBody/*", "tests/mainnet/deneb/light_client/single_merkle_proof/BeaconBlockBody/*",
"tests/general/deneb/kzg" "tests/.*/eip6110"
] ]

View File

@ -31,7 +31,7 @@ impl<E: EthSpec> Case for KZGBlobToKZGCommitment<E> {
} }
fn result(&self, _case_index: usize, _fork_name: ForkName) -> Result<(), Error> { fn result(&self, _case_index: usize, _fork_name: ForkName) -> Result<(), Error> {
let kzg = get_kzg()?; let kzg = get_kzg::<E::Kzg>()?;
let commitment = parse_blob::<E>(&self.input.blob).and_then(|blob| { let commitment = parse_blob::<E>(&self.input.blob).and_then(|blob| {
blob_to_kzg_commitment::<E>(&kzg, blob).map_err(|e| { blob_to_kzg_commitment::<E>(&kzg, blob).map_err(|e| {

View File

@ -38,7 +38,7 @@ impl<E: EthSpec> Case for KZGComputeBlobKZGProof<E> {
Ok((blob, commitment)) Ok((blob, commitment))
}; };
let kzg = get_kzg()?; let kzg = get_kzg::<E::Kzg>()?;
let proof = parse_input(&self.input).and_then(|(blob, commitment)| { let proof = parse_input(&self.input).and_then(|(blob, commitment)| {
compute_blob_kzg_proof::<E>(&kzg, &blob, commitment) compute_blob_kzg_proof::<E>(&kzg, &blob, commitment)
.map_err(|e| Error::InternalError(format!("Failed to compute kzg proof: {:?}", e))) .map_err(|e| Error::InternalError(format!("Failed to compute kzg proof: {:?}", e)))

View File

@ -45,7 +45,7 @@ impl<E: EthSpec> Case for KZGComputeKZGProof<E> {
Ok((blob, z)) Ok((blob, z))
}; };
let kzg = get_kzg()?; let kzg = get_kzg::<E::Kzg>()?;
let proof = parse_input(&self.input).and_then(|(blob, z)| { let proof = parse_input(&self.input).and_then(|(blob, z)| {
compute_kzg_proof::<E>(&kzg, blob, z) compute_kzg_proof::<E>(&kzg, blob, z)
.map_err(|e| Error::InternalError(format!("Failed to compute kzg proof: {:?}", e))) .map_err(|e| Error::InternalError(format!("Failed to compute kzg proof: {:?}", e)))

View File

@ -1,15 +1,15 @@
use super::*; use super::*;
use crate::case_result::compare_result; use crate::case_result::compare_result;
use beacon_chain::kzg_utils::validate_blob; use beacon_chain::kzg_utils::validate_blob;
use eth2_network_config::TRUSTED_SETUP; use eth2_network_config::get_trusted_setup;
use kzg::{Kzg, KzgCommitment, KzgProof, TrustedSetup}; use kzg::{Kzg, KzgCommitment, KzgPreset, KzgProof, TrustedSetup};
use serde_derive::Deserialize; use serde_derive::Deserialize;
use std::convert::TryInto; use std::convert::TryInto;
use std::marker::PhantomData; use std::marker::PhantomData;
use types::Blob; use types::Blob;
pub fn get_kzg() -> Result<Kzg, Error> { pub fn get_kzg<P: KzgPreset>() -> Result<Kzg<P>, Error> {
let trusted_setup: TrustedSetup = serde_json::from_reader(TRUSTED_SETUP) let trusted_setup: TrustedSetup = serde_json::from_reader(get_trusted_setup::<P>())
.map_err(|e| Error::InternalError(format!("Failed to initialize kzg: {:?}", e)))?; .map_err(|e| Error::InternalError(format!("Failed to initialize kzg: {:?}", e)))?;
Kzg::new_from_trusted_setup(trusted_setup) Kzg::new_from_trusted_setup(trusted_setup)
.map_err(|e| Error::InternalError(format!("Failed to initialize kzg: {:?}", e))) .map_err(|e| Error::InternalError(format!("Failed to initialize kzg: {:?}", e)))
@ -81,7 +81,7 @@ impl<E: EthSpec> Case for KZGVerifyBlobKZGProof<E> {
Ok((blob, commitment, proof)) Ok((blob, commitment, proof))
}; };
let kzg = get_kzg()?; let kzg = get_kzg::<E::Kzg>()?;
let result = parse_input(&self.input).and_then(|(blob, commitment, proof)| { let result = parse_input(&self.input).and_then(|(blob, commitment, proof)| {
validate_blob::<E>(&kzg, blob, commitment, proof) validate_blob::<E>(&kzg, blob, commitment, proof)
.map_err(|e| Error::InternalError(format!("Failed to validate blob: {:?}", e))) .map_err(|e| Error::InternalError(format!("Failed to validate blob: {:?}", e)))

View File

@ -51,7 +51,7 @@ impl<E: EthSpec> Case for KZGVerifyBlobKZGProofBatch<E> {
Ok((commitments, blobs, proofs)) Ok((commitments, blobs, proofs))
}; };
let kzg = get_kzg()?; let kzg = get_kzg::<E::Kzg>()?;
let result = parse_input(&self.input).and_then(|(commitments, blobs, proofs)| { let result = parse_input(&self.input).and_then(|(commitments, blobs, proofs)| {
validate_blobs::<E>(&kzg, &commitments, &blobs, &proofs) validate_blobs::<E>(&kzg, &commitments, &blobs, &proofs)
.map_err(|e| Error::InternalError(format!("Failed to validate blobs: {:?}", e))) .map_err(|e| Error::InternalError(format!("Failed to validate blobs: {:?}", e)))

View File

@ -41,7 +41,7 @@ impl<E: EthSpec> Case for KZGVerifyKZGProof<E> {
Ok((commitment, z, y, proof)) Ok((commitment, z, y, proof))
}; };
let kzg = get_kzg()?; let kzg = get_kzg::<E::Kzg>()?;
let result = parse_input(&self.input).and_then(|(commitment, z, y, proof)| { let result = parse_input(&self.input).and_then(|(commitment, z, y, proof)| {
verify_kzg_proof::<E>(&kzg, commitment, proof, z, y) verify_kzg_proof::<E>(&kzg, commitment, proof, z, y)
.map_err(|e| Error::InternalError(format!("Failed to validate proof: {:?}", e))) .map_err(|e| Error::InternalError(format!("Failed to validate proof: {:?}", e)))

View File

@ -4,6 +4,7 @@ use crate::case_result::compare_beacon_state_results_without_caches;
use crate::decode::{ssz_decode_file, ssz_decode_file_with, ssz_decode_state, yaml_decode_file}; use crate::decode::{ssz_decode_file, ssz_decode_file_with, ssz_decode_state, yaml_decode_file};
use crate::testing_spec; use crate::testing_spec;
use serde_derive::Deserialize; use serde_derive::Deserialize;
use ssz::Decode;
use state_processing::{ use state_processing::{
per_block_processing::{ per_block_processing::{
errors::BlockProcessingError, errors::BlockProcessingError,
@ -19,7 +20,8 @@ use state_processing::{
use std::fmt::Debug; use std::fmt::Debug;
use std::path::Path; use std::path::Path;
use types::{ use types::{
Attestation, AttesterSlashing, BeaconBlock, BeaconState, BlindedPayload, ChainSpec, Deposit, Attestation, AttesterSlashing, BeaconBlock, BeaconBlockBody, BeaconBlockBodyCapella,
BeaconBlockBodyDeneb, BeaconBlockBodyMerge, BeaconState, BlindedPayload, ChainSpec, Deposit,
EthSpec, ExecutionPayload, ForkName, FullPayload, ProposerSlashing, SignedBlsToExecutionChange, EthSpec, ExecutionPayload, ForkName, FullPayload, ProposerSlashing, SignedBlsToExecutionChange,
SignedVoluntaryExit, SyncAggregate, SignedVoluntaryExit, SyncAggregate,
}; };
@ -259,13 +261,13 @@ impl<E: EthSpec> Operation<E> for SyncAggregate<E> {
} }
} }
impl<E: EthSpec> Operation<E> for FullPayload<E> { impl<E: EthSpec> Operation<E> for BeaconBlockBody<E, FullPayload<E>> {
fn handler_name() -> String { fn handler_name() -> String {
"execution_payload".into() "execution_payload".into()
} }
fn filename() -> String { fn filename() -> String {
"execution_payload.ssz_snappy".into() "body.ssz_snappy".into()
} }
fn is_enabled_for_fork(fork_name: ForkName) -> bool { fn is_enabled_for_fork(fork_name: ForkName) -> bool {
@ -274,9 +276,13 @@ impl<E: EthSpec> Operation<E> for FullPayload<E> {
fn decode(path: &Path, fork_name: ForkName, _spec: &ChainSpec) -> Result<Self, Error> { fn decode(path: &Path, fork_name: ForkName, _spec: &ChainSpec) -> Result<Self, Error> {
ssz_decode_file_with(path, |bytes| { ssz_decode_file_with(path, |bytes| {
ExecutionPayload::from_ssz_bytes(bytes, fork_name) Ok(match fork_name {
ForkName::Merge => BeaconBlockBody::Merge(<_>::from_ssz_bytes(bytes)?),
ForkName::Capella => BeaconBlockBody::Capella(<_>::from_ssz_bytes(bytes)?),
ForkName::Deneb => BeaconBlockBody::Deneb(<_>::from_ssz_bytes(bytes)?),
_ => panic!(),
})
}) })
.map(Into::into)
} }
fn apply_to( fn apply_to(
@ -296,13 +302,13 @@ impl<E: EthSpec> Operation<E> for FullPayload<E> {
} }
} }
} }
impl<E: EthSpec> Operation<E> for BlindedPayload<E> { impl<E: EthSpec> Operation<E> for BeaconBlockBody<E, BlindedPayload<E>> {
fn handler_name() -> String { fn handler_name() -> String {
"execution_payload".into() "execution_payload".into()
} }
fn filename() -> String { fn filename() -> String {
"execution_payload.ssz_snappy".into() "body.ssz_snappy".into()
} }
fn is_enabled_for_fork(fork_name: ForkName) -> bool { fn is_enabled_for_fork(fork_name: ForkName) -> bool {
@ -311,9 +317,22 @@ impl<E: EthSpec> Operation<E> for BlindedPayload<E> {
fn decode(path: &Path, fork_name: ForkName, _spec: &ChainSpec) -> Result<Self, Error> { fn decode(path: &Path, fork_name: ForkName, _spec: &ChainSpec) -> Result<Self, Error> {
ssz_decode_file_with(path, |bytes| { ssz_decode_file_with(path, |bytes| {
ExecutionPayload::from_ssz_bytes(bytes, fork_name) Ok(match fork_name {
ForkName::Merge => {
let inner = <BeaconBlockBodyMerge<E, FullPayload<E>>>::from_ssz_bytes(bytes)?;
BeaconBlockBody::Merge(inner.clone_as_blinded())
}
ForkName::Capella => {
let inner = <BeaconBlockBodyCapella<E, FullPayload<E>>>::from_ssz_bytes(bytes)?;
BeaconBlockBody::Capella(inner.clone_as_blinded())
}
ForkName::Deneb => {
let inner = <BeaconBlockBodyDeneb<E, FullPayload<E>>>::from_ssz_bytes(bytes)?;
BeaconBlockBody::Deneb(inner.clone_as_blinded())
}
_ => panic!(),
})
}) })
.map(Into::into)
} }
fn apply_to( fn apply_to(

View File

@ -72,14 +72,14 @@ fn operations_sync_aggregate() {
#[test] #[test]
fn operations_execution_payload_full() { fn operations_execution_payload_full() {
OperationsHandler::<MinimalEthSpec, FullPayload<_>>::default().run(); OperationsHandler::<MinimalEthSpec, BeaconBlockBody<_, FullPayload<_>>>::default().run();
OperationsHandler::<MainnetEthSpec, FullPayload<_>>::default().run(); OperationsHandler::<MainnetEthSpec, BeaconBlockBody<_, FullPayload<_>>>::default().run();
} }
#[test] #[test]
fn operations_execution_payload_blinded() { fn operations_execution_payload_blinded() {
OperationsHandler::<MinimalEthSpec, BlindedPayload<_>>::default().run(); OperationsHandler::<MinimalEthSpec, BeaconBlockBody<_, BlindedPayload<_>>>::default().run();
OperationsHandler::<MainnetEthSpec, BlindedPayload<_>>::default().run(); OperationsHandler::<MainnetEthSpec, BeaconBlockBody<_, BlindedPayload<_>>>::default().run();
} }
#[test] #[test]

View File

@ -371,7 +371,7 @@ impl<E: GenericExecutionEngine> TestRig<E> {
let status = self let status = self
.ee_a .ee_a
.execution_layer .execution_layer
.notify_new_payload(&valid_payload) .notify_new_payload(&valid_payload, None)
.await .await
.unwrap(); .unwrap();
assert_eq!(status, PayloadStatus::Valid); assert_eq!(status, PayloadStatus::Valid);
@ -424,7 +424,7 @@ impl<E: GenericExecutionEngine> TestRig<E> {
let status = self let status = self
.ee_a .ee_a
.execution_layer .execution_layer
.notify_new_payload(&invalid_payload) .notify_new_payload(&invalid_payload, None)
.await .await
.unwrap(); .unwrap();
assert!(matches!( assert!(matches!(
@ -486,7 +486,7 @@ impl<E: GenericExecutionEngine> TestRig<E> {
let status = self let status = self
.ee_a .ee_a
.execution_layer .execution_layer
.notify_new_payload(&second_payload) .notify_new_payload(&second_payload, None)
.await .await
.unwrap(); .unwrap();
assert_eq!(status, PayloadStatus::Valid); assert_eq!(status, PayloadStatus::Valid);
@ -533,7 +533,7 @@ impl<E: GenericExecutionEngine> TestRig<E> {
let status = self let status = self
.ee_b .ee_b
.execution_layer .execution_layer
.notify_new_payload(&second_payload) .notify_new_payload(&second_payload, None)
.await .await
.unwrap(); .unwrap();
// TODO: we should remove the `Accepted` status here once Geth fixes it // TODO: we should remove the `Accepted` status here once Geth fixes it
@ -574,7 +574,7 @@ impl<E: GenericExecutionEngine> TestRig<E> {
let status = self let status = self
.ee_b .ee_b
.execution_layer .execution_layer
.notify_new_payload(&valid_payload) .notify_new_payload(&valid_payload, None)
.await .await
.unwrap(); .unwrap();
assert_eq!(status, PayloadStatus::Valid); assert_eq!(status, PayloadStatus::Valid);
@ -588,7 +588,7 @@ impl<E: GenericExecutionEngine> TestRig<E> {
let status = self let status = self
.ee_b .ee_b
.execution_layer .execution_layer
.notify_new_payload(&second_payload) .notify_new_payload(&second_payload, None)
.await .await
.unwrap(); .unwrap();
assert_eq!(status, PayloadStatus::Valid); assert_eq!(status, PayloadStatus::Valid);