Devnet6 (#4404)
* 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:
parent
4c9fcf1e83
commit
adbb62f7f3
17
.github/workflows/test-suite.yml
vendored
17
.github/workflows/test-suite.yml
vendored
@ -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
786
Cargo.lock
generated
File diff suppressed because it is too large
Load Diff
@ -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.
|
||||||
///
|
///
|
||||||
|
@ -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),
|
||||||
|
@ -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
|
||||||
|
@ -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 {
|
||||||
|
@ -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,
|
||||||
|
@ -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,
|
||||||
|
@ -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)
|
||||||
|
@ -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)
|
||||||
|
@ -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 {
|
||||||
|
@ -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(
|
||||||
|
@ -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
|
||||||
|
@ -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);
|
||||||
|
@ -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
|
||||||
|
@ -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 =
|
||||||
|
@ -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 {
|
||||||
|
@ -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
|
||||||
|
@ -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,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -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>(
|
||||||
|
@ -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,
|
||||||
|
@ -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.
|
||||||
|
@ -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,
|
||||||
)
|
)
|
||||||
|
@ -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();
|
||||||
|
@ -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
|
||||||
|
@ -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,
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -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);
|
||||||
}
|
}
|
||||||
|
@ -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()
|
||||||
|
@ -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)]
|
||||||
|
@ -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,
|
||||||
|
@ -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);
|
||||||
|
}
|
||||||
|
@ -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) => {
|
||||||
|
@ -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,
|
||||||
|
@ -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.
|
||||||
*/
|
*/
|
||||||
|
@ -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,
|
||||||
|
@ -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
@ -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 {
|
||||||
|
@ -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>>(
|
||||||
|
@ -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)
|
||||||
|
@ -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 {
|
||||||
|
@ -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.
|
||||||
///
|
///
|
||||||
|
@ -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
|
||||||
|
@ -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;
|
||||||
}
|
}
|
||||||
|
@ -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;
|
||||||
|
@ -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,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -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> {
|
||||||
|
@ -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,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -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};
|
||||||
|
|
||||||
|
@ -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,
|
|
||||||
}
|
|
@ -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"]
|
@ -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()
|
||||||
}
|
}
|
||||||
|
@ -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()
|
||||||
}
|
}
|
||||||
|
@ -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)
|
||||||
|
@ -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")]
|
||||||
|
@ -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 {
|
||||||
|
@ -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
|
||||||
|
@ -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))
|
||||||
|
|
||||||
|
@ -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"
|
||||||
]
|
]
|
||||||
|
|
||||||
|
|
||||||
|
@ -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| {
|
||||||
|
@ -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)))
|
||||||
|
@ -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)))
|
||||||
|
@ -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)))
|
||||||
|
@ -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)))
|
||||||
|
@ -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)))
|
||||||
|
@ -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(
|
||||||
|
@ -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]
|
||||||
|
@ -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);
|
||||||
|
Loading…
Reference in New Issue
Block a user