Initial merge changes
Added Execution Payload from Rayonism Fork Updated new Containers to match Merge Spec Updated BeaconBlockBody for Merge Spec Completed updating BeaconState and BeaconBlockBody Modified ExecutionPayload<T> to use Transaction<T> Mostly Finished Changes for beacon-chain.md Added some things for fork-choice.md Update to match new fork-choice.md/fork.md changes ran cargo fmt Added Missing Pieces in eth2_libp2p for Merge fix ef test Various Changes to Conform Closer to Merge Spec
This commit is contained in:
parent
fe75a0a9a1
commit
5687c56d51
1
Cargo.lock
generated
1
Cargo.lock
generated
@ -1923,6 +1923,7 @@ dependencies = [
|
|||||||
"eth2_ssz 0.4.0 (registry+https://github.com/rust-lang/crates.io-index)",
|
"eth2_ssz 0.4.0 (registry+https://github.com/rust-lang/crates.io-index)",
|
||||||
"eth2_ssz_derive 0.3.0 (registry+https://github.com/rust-lang/crates.io-index)",
|
"eth2_ssz_derive 0.3.0 (registry+https://github.com/rust-lang/crates.io-index)",
|
||||||
"proto_array",
|
"proto_array",
|
||||||
|
"state_processing",
|
||||||
"store",
|
"store",
|
||||||
"types",
|
"types",
|
||||||
]
|
]
|
||||||
|
@ -2838,6 +2838,11 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
|
|||||||
SyncAggregate::new()
|
SyncAggregate::new()
|
||||||
}))
|
}))
|
||||||
};
|
};
|
||||||
|
// Closure to fetch a sync aggregate in cases where it is required.
|
||||||
|
let get_execution_payload = || -> Result<ExecutionPayload<_>, BlockProductionError> {
|
||||||
|
// TODO: actually get the payload from eth1 node..
|
||||||
|
Ok(ExecutionPayload::default())
|
||||||
|
};
|
||||||
|
|
||||||
let inner_block = match state {
|
let inner_block = match state {
|
||||||
BeaconState::Base(_) => BeaconBlock::Base(BeaconBlockBase {
|
BeaconState::Base(_) => BeaconBlock::Base(BeaconBlockBase {
|
||||||
@ -2876,6 +2881,28 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
|
|||||||
},
|
},
|
||||||
})
|
})
|
||||||
}
|
}
|
||||||
|
BeaconState::Merge(_) => {
|
||||||
|
let sync_aggregate = get_sync_aggregate()?;
|
||||||
|
let execution_payload = get_execution_payload()?;
|
||||||
|
BeaconBlock::Merge(BeaconBlockMerge {
|
||||||
|
slot,
|
||||||
|
proposer_index,
|
||||||
|
parent_root,
|
||||||
|
state_root: Hash256::zero(),
|
||||||
|
body: BeaconBlockBodyMerge {
|
||||||
|
randao_reveal,
|
||||||
|
eth1_data,
|
||||||
|
graffiti,
|
||||||
|
proposer_slashings: proposer_slashings.into(),
|
||||||
|
attester_slashings: attester_slashings.into(),
|
||||||
|
attestations,
|
||||||
|
deposits,
|
||||||
|
voluntary_exits: voluntary_exits.into(),
|
||||||
|
sync_aggregate,
|
||||||
|
execution_payload,
|
||||||
|
},
|
||||||
|
})
|
||||||
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
let block = SignedBeaconBlock::from_block(
|
let block = SignedBeaconBlock::from_block(
|
||||||
|
@ -48,7 +48,7 @@ use crate::{
|
|||||||
BLOCK_PROCESSING_CACHE_LOCK_TIMEOUT, MAXIMUM_GOSSIP_CLOCK_DISPARITY,
|
BLOCK_PROCESSING_CACHE_LOCK_TIMEOUT, MAXIMUM_GOSSIP_CLOCK_DISPARITY,
|
||||||
VALIDATOR_PUBKEY_CACHE_LOCK_TIMEOUT,
|
VALIDATOR_PUBKEY_CACHE_LOCK_TIMEOUT,
|
||||||
},
|
},
|
||||||
metrics, BeaconChain, BeaconChainError, BeaconChainTypes,
|
eth1_chain, metrics, BeaconChain, BeaconChainError, BeaconChainTypes,
|
||||||
};
|
};
|
||||||
use fork_choice::{ForkChoice, ForkChoiceStore};
|
use fork_choice::{ForkChoice, ForkChoiceStore};
|
||||||
use parking_lot::RwLockReadGuard;
|
use parking_lot::RwLockReadGuard;
|
||||||
@ -56,6 +56,7 @@ use proto_array::Block as ProtoBlock;
|
|||||||
use slog::{debug, error, Logger};
|
use slog::{debug, error, Logger};
|
||||||
use slot_clock::SlotClock;
|
use slot_clock::SlotClock;
|
||||||
use ssz::Encode;
|
use ssz::Encode;
|
||||||
|
use state_processing::per_block_processing::{is_execution_enabled, is_merge_complete};
|
||||||
use state_processing::{
|
use state_processing::{
|
||||||
block_signature_verifier::{BlockSignatureVerifier, Error as BlockSignatureVerifierError},
|
block_signature_verifier::{BlockSignatureVerifier, Error as BlockSignatureVerifierError},
|
||||||
per_block_processing, per_slot_processing,
|
per_block_processing, per_slot_processing,
|
||||||
@ -68,9 +69,9 @@ use std::io::Write;
|
|||||||
use store::{Error as DBError, HotColdDB, HotStateSummary, KeyValueStore, StoreOp};
|
use store::{Error as DBError, HotColdDB, HotStateSummary, KeyValueStore, StoreOp};
|
||||||
use tree_hash::TreeHash;
|
use tree_hash::TreeHash;
|
||||||
use types::{
|
use types::{
|
||||||
BeaconBlockRef, BeaconState, BeaconStateError, ChainSpec, CloneConfig, Epoch, EthSpec, Hash256,
|
BeaconBlockRef, BeaconState, BeaconStateError, ChainSpec, CloneConfig, Epoch, EthSpec,
|
||||||
InconsistentFork, PublicKey, PublicKeyBytes, RelativeEpoch, SignedBeaconBlock,
|
ExecutionPayload, Hash256, InconsistentFork, PublicKey, PublicKeyBytes, RelativeEpoch,
|
||||||
SignedBeaconBlockHeader, Slot,
|
SignedBeaconBlock, SignedBeaconBlockHeader, Slot,
|
||||||
};
|
};
|
||||||
|
|
||||||
/// Maximum block slot number. Block with slots bigger than this constant will NOT be processed.
|
/// Maximum block slot number. Block with slots bigger than this constant will NOT be processed.
|
||||||
@ -223,6 +224,66 @@ pub enum BlockError<T: EthSpec> {
|
|||||||
///
|
///
|
||||||
/// The block is invalid and the peer is faulty.
|
/// The block is invalid and the peer is faulty.
|
||||||
InconsistentFork(InconsistentFork),
|
InconsistentFork(InconsistentFork),
|
||||||
|
/// There was an error while validating the ExecutionPayload
|
||||||
|
///
|
||||||
|
/// ## Peer scoring
|
||||||
|
///
|
||||||
|
/// See `ExecutionPayloadError` for scoring information
|
||||||
|
ExecutionPayloadError(ExecutionPayloadError),
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Returned when block validation failed due to some issue verifying
|
||||||
|
/// the execution payload.
|
||||||
|
#[derive(Debug)]
|
||||||
|
pub enum ExecutionPayloadError {
|
||||||
|
/// There's no eth1 connection (mandatory after merge)
|
||||||
|
///
|
||||||
|
/// ## Peer scoring
|
||||||
|
///
|
||||||
|
/// As this is our fault, do not penalize the peer
|
||||||
|
NoEth1Connection,
|
||||||
|
/// Error occurred during engine_executePayload
|
||||||
|
///
|
||||||
|
/// ## Peer scoring
|
||||||
|
///
|
||||||
|
/// Some issue with our configuration, do not penalize peer
|
||||||
|
Eth1VerificationError(eth1_chain::Error),
|
||||||
|
/// The execution engine returned INVALID for the payload
|
||||||
|
///
|
||||||
|
/// ## Peer scoring
|
||||||
|
///
|
||||||
|
/// The block is invalid and the peer is faulty
|
||||||
|
RejectedByExecutionEngine,
|
||||||
|
/// The execution payload is empty when is shouldn't be
|
||||||
|
///
|
||||||
|
/// ## Peer scoring
|
||||||
|
///
|
||||||
|
/// The block is invalid and the peer is faulty
|
||||||
|
PayloadEmpty,
|
||||||
|
/// The execution payload timestamp does not match the slot
|
||||||
|
///
|
||||||
|
/// ## Peer scoring
|
||||||
|
///
|
||||||
|
/// The block is invalid and the peer is faulty
|
||||||
|
InvalidPayloadTimestamp,
|
||||||
|
/// The gas used in the block exceeds the gas limit
|
||||||
|
///
|
||||||
|
/// ## Peer scoring
|
||||||
|
///
|
||||||
|
/// The block is invalid and the peer is faulty
|
||||||
|
GasUsedExceedsLimit,
|
||||||
|
/// The payload block hash equals the parent hash
|
||||||
|
///
|
||||||
|
/// ## Peer scoring
|
||||||
|
///
|
||||||
|
/// The block is invalid and the peer is faulty
|
||||||
|
BlockHashEqualsParentHash,
|
||||||
|
/// The execution payload transaction list data exceeds size limits
|
||||||
|
///
|
||||||
|
/// ## Peer scoring
|
||||||
|
///
|
||||||
|
/// The block is invalid and the peer is faulty
|
||||||
|
TransactionDataExceedsSizeLimit,
|
||||||
}
|
}
|
||||||
|
|
||||||
impl<T: EthSpec> std::fmt::Display for BlockError<T> {
|
impl<T: EthSpec> std::fmt::Display for BlockError<T> {
|
||||||
@ -668,6 +729,18 @@ impl<T: BeaconChainTypes> GossipVerifiedBlock<T> {
|
|||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// TODO: avoid this by adding field to fork-choice to determine if merge-block has been imported
|
||||||
|
let (parent, block) = if let Some(snapshot) = parent {
|
||||||
|
(Some(snapshot), block)
|
||||||
|
} else {
|
||||||
|
let (snapshot, block) = load_parent(block, chain)?;
|
||||||
|
(Some(snapshot), block)
|
||||||
|
};
|
||||||
|
let state = &parent.as_ref().unwrap().pre_state;
|
||||||
|
|
||||||
|
// validate the block's execution_payload
|
||||||
|
validate_execution_payload(block.message(), state)?;
|
||||||
|
|
||||||
Ok(Self {
|
Ok(Self {
|
||||||
block,
|
block,
|
||||||
block_root,
|
block_root,
|
||||||
@ -989,6 +1062,34 @@ impl<'a, T: BeaconChainTypes> FullyVerifiedBlock<'a, T> {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// This is the soonest we can run these checks as they must be called AFTER per_slot_processing
|
||||||
|
if is_execution_enabled(&state, block.message().body()) {
|
||||||
|
let eth1_chain = chain
|
||||||
|
.eth1_chain
|
||||||
|
.as_ref()
|
||||||
|
.ok_or(BlockError::ExecutionPayloadError(
|
||||||
|
ExecutionPayloadError::NoEth1Connection,
|
||||||
|
))?;
|
||||||
|
|
||||||
|
if !eth1_chain
|
||||||
|
.on_payload(block.message().body().execution_payload().ok_or(
|
||||||
|
BlockError::InconsistentFork(InconsistentFork {
|
||||||
|
fork_at_slot: eth2::types::ForkName::Merge,
|
||||||
|
object_fork: block.message().body().fork_name(),
|
||||||
|
}),
|
||||||
|
)?)
|
||||||
|
.map_err(|e| {
|
||||||
|
BlockError::ExecutionPayloadError(ExecutionPayloadError::Eth1VerificationError(
|
||||||
|
e,
|
||||||
|
))
|
||||||
|
})?
|
||||||
|
{
|
||||||
|
return Err(BlockError::ExecutionPayloadError(
|
||||||
|
ExecutionPayloadError::RejectedByExecutionEngine,
|
||||||
|
));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
// If the block is sufficiently recent, notify the validator monitor.
|
// If the block is sufficiently recent, notify the validator monitor.
|
||||||
if let Some(slot) = chain.slot_clock.now() {
|
if let Some(slot) = chain.slot_clock.now() {
|
||||||
let epoch = slot.epoch(T::EthSpec::slots_per_epoch());
|
let epoch = slot.epoch(T::EthSpec::slots_per_epoch());
|
||||||
@ -1097,6 +1198,38 @@ impl<'a, T: BeaconChainTypes> FullyVerifiedBlock<'a, T> {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/// Validate the gossip block's execution_payload according to the checks described here:
|
||||||
|
/// https://github.com/ethereum/consensus-specs/blob/dev/specs/merge/p2p-interface.md#beacon_block
|
||||||
|
fn validate_execution_payload<E: EthSpec>(
|
||||||
|
block: BeaconBlockRef<'_, E>,
|
||||||
|
state: &BeaconState<E>,
|
||||||
|
) -> Result<(), BlockError<E>> {
|
||||||
|
if !is_execution_enabled(state, block.body()) {
|
||||||
|
return Ok(());
|
||||||
|
}
|
||||||
|
let execution_payload = block
|
||||||
|
.body()
|
||||||
|
.execution_payload()
|
||||||
|
// TODO: this really should never error so maybe
|
||||||
|
// we should make this simpler..
|
||||||
|
.ok_or(BlockError::InconsistentFork(InconsistentFork {
|
||||||
|
fork_at_slot: eth2::types::ForkName::Merge,
|
||||||
|
object_fork: block.body().fork_name(),
|
||||||
|
}))?;
|
||||||
|
|
||||||
|
if is_merge_complete(state) {
|
||||||
|
if *execution_payload == <ExecutionPayload<E>>::default() {
|
||||||
|
return Err(BlockError::ExecutionPayloadError(
|
||||||
|
ExecutionPayloadError::PayloadEmpty,
|
||||||
|
));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// TODO: finish these
|
||||||
|
|
||||||
|
Ok(())
|
||||||
|
}
|
||||||
|
|
||||||
/// Check that the count of skip slots between the block and its parent does not exceed our maximum
|
/// Check that the count of skip slots between the block and its parent does not exceed our maximum
|
||||||
/// value.
|
/// value.
|
||||||
///
|
///
|
||||||
|
@ -15,8 +15,8 @@ use std::time::{SystemTime, UNIX_EPOCH};
|
|||||||
use store::{DBColumn, Error as StoreError, StoreItem};
|
use store::{DBColumn, Error as StoreError, StoreItem};
|
||||||
use task_executor::TaskExecutor;
|
use task_executor::TaskExecutor;
|
||||||
use types::{
|
use types::{
|
||||||
BeaconState, BeaconStateError, ChainSpec, Deposit, Eth1Data, EthSpec, Hash256, Slot, Unsigned,
|
BeaconState, BeaconStateError, ChainSpec, Deposit, Eth1Data, EthSpec, ExecutionPayload,
|
||||||
DEPOSIT_TREE_DEPTH,
|
Hash256, Slot, Unsigned, DEPOSIT_TREE_DEPTH,
|
||||||
};
|
};
|
||||||
|
|
||||||
type BlockNumber = u64;
|
type BlockNumber = u64;
|
||||||
@ -53,6 +53,8 @@ pub enum Error {
|
|||||||
UnknownPreviousEth1BlockHash,
|
UnknownPreviousEth1BlockHash,
|
||||||
/// An arithmetic error occurred.
|
/// An arithmetic error occurred.
|
||||||
ArithError(safe_arith::ArithError),
|
ArithError(safe_arith::ArithError),
|
||||||
|
/// Unable to execute payload
|
||||||
|
UnableToExecutePayload(String),
|
||||||
}
|
}
|
||||||
|
|
||||||
impl From<safe_arith::ArithError> for Error {
|
impl From<safe_arith::ArithError> for Error {
|
||||||
@ -274,6 +276,15 @@ where
|
|||||||
)
|
)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
pub fn on_payload(&self, execution_payload: &ExecutionPayload<E>) -> Result<bool, Error> {
|
||||||
|
if self.use_dummy_backend {
|
||||||
|
let dummy_backend: DummyEth1ChainBackend<E> = DummyEth1ChainBackend::default();
|
||||||
|
dummy_backend.on_payload(execution_payload)
|
||||||
|
} else {
|
||||||
|
self.backend.on_payload(execution_payload)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/// Instantiate `Eth1Chain` from a persisted `SszEth1`.
|
/// Instantiate `Eth1Chain` from a persisted `SszEth1`.
|
||||||
///
|
///
|
||||||
/// The `Eth1Chain` will have the same caches as the persisted `SszEth1`.
|
/// The `Eth1Chain` will have the same caches as the persisted `SszEth1`.
|
||||||
@ -334,6 +345,9 @@ pub trait Eth1ChainBackend<T: EthSpec>: Sized + Send + Sync {
|
|||||||
/// an idea of how up-to-date the remote eth1 node is.
|
/// an idea of how up-to-date the remote eth1 node is.
|
||||||
fn head_block(&self) -> Option<Eth1Block>;
|
fn head_block(&self) -> Option<Eth1Block>;
|
||||||
|
|
||||||
|
/// Verifies the execution payload
|
||||||
|
fn on_payload(&self, execution_payload: &ExecutionPayload<T>) -> Result<bool, Error>;
|
||||||
|
|
||||||
/// Encode the `Eth1ChainBackend` instance to bytes.
|
/// Encode the `Eth1ChainBackend` instance to bytes.
|
||||||
fn as_bytes(&self) -> Vec<u8>;
|
fn as_bytes(&self) -> Vec<u8>;
|
||||||
|
|
||||||
@ -388,6 +402,10 @@ impl<T: EthSpec> Eth1ChainBackend<T> for DummyEth1ChainBackend<T> {
|
|||||||
None
|
None
|
||||||
}
|
}
|
||||||
|
|
||||||
|
fn on_payload(&self, _execution_payload: &ExecutionPayload<T>) -> Result<bool, Error> {
|
||||||
|
Ok(true)
|
||||||
|
}
|
||||||
|
|
||||||
/// Return empty Vec<u8> for dummy backend.
|
/// Return empty Vec<u8> for dummy backend.
|
||||||
fn as_bytes(&self) -> Vec<u8> {
|
fn as_bytes(&self) -> Vec<u8> {
|
||||||
Vec::new()
|
Vec::new()
|
||||||
@ -556,6 +574,15 @@ impl<T: EthSpec> Eth1ChainBackend<T> for CachingEth1Backend<T> {
|
|||||||
self.core.head_block()
|
self.core.head_block()
|
||||||
}
|
}
|
||||||
|
|
||||||
|
fn on_payload(&self, execution_payload: &ExecutionPayload<T>) -> Result<bool, Error> {
|
||||||
|
futures::executor::block_on(async move {
|
||||||
|
self.core
|
||||||
|
.on_payload(execution_payload.clone())
|
||||||
|
.await
|
||||||
|
.map_err(|e| Error::UnableToExecutePayload(format!("{:?}", e)))
|
||||||
|
})
|
||||||
|
}
|
||||||
|
|
||||||
/// Return encoded byte representation of the block and deposit caches.
|
/// Return encoded byte representation of the block and deposit caches.
|
||||||
fn as_bytes(&self) -> Vec<u8> {
|
fn as_bytes(&self) -> Vec<u8> {
|
||||||
self.core.as_bytes()
|
self.core.as_bytes()
|
||||||
|
@ -44,7 +44,7 @@ pub use self::errors::{BeaconChainError, BlockProductionError};
|
|||||||
pub use self::historical_blocks::HistoricalBlockError;
|
pub use self::historical_blocks::HistoricalBlockError;
|
||||||
pub use attestation_verification::Error as AttestationError;
|
pub use attestation_verification::Error as AttestationError;
|
||||||
pub use beacon_fork_choice_store::{BeaconForkChoiceStore, Error as ForkChoiceStoreError};
|
pub use beacon_fork_choice_store::{BeaconForkChoiceStore, Error as ForkChoiceStoreError};
|
||||||
pub use block_verification::{BlockError, GossipVerifiedBlock};
|
pub use block_verification::{BlockError, ExecutionPayloadError, GossipVerifiedBlock};
|
||||||
pub use eth1_chain::{Eth1Chain, Eth1ChainBackend};
|
pub use eth1_chain::{Eth1Chain, Eth1ChainBackend};
|
||||||
pub use events::ServerSentEventHandler;
|
pub use events::ServerSentEventHandler;
|
||||||
pub use metrics::scrape_for_metrics;
|
pub use metrics::scrape_for_metrics;
|
||||||
|
@ -19,7 +19,7 @@ use std::fmt;
|
|||||||
use std::ops::Range;
|
use std::ops::Range;
|
||||||
use std::str::FromStr;
|
use std::str::FromStr;
|
||||||
use std::time::Duration;
|
use std::time::Duration;
|
||||||
use types::Hash256;
|
use types::{Hash256, PowBlock, Uint256};
|
||||||
|
|
||||||
/// `keccak("DepositEvent(bytes,bytes,bytes,bytes,bytes)")`
|
/// `keccak("DepositEvent(bytes,bytes,bytes,bytes,bytes)")`
|
||||||
pub const DEPOSIT_EVENT_TOPIC: &str =
|
pub const DEPOSIT_EVENT_TOPIC: &str =
|
||||||
@ -49,6 +49,7 @@ pub enum Eth1Id {
|
|||||||
#[derive(Clone, Copy)]
|
#[derive(Clone, Copy)]
|
||||||
pub enum BlockQuery {
|
pub enum BlockQuery {
|
||||||
Number(u64),
|
Number(u64),
|
||||||
|
Hash(Hash256),
|
||||||
Latest,
|
Latest,
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -135,13 +136,6 @@ pub async fn get_chain_id(endpoint: &SensitiveUrl, timeout: Duration) -> Result<
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
#[derive(Debug, PartialEq, Clone)]
|
|
||||||
pub struct Block {
|
|
||||||
pub hash: Hash256,
|
|
||||||
pub timestamp: u64,
|
|
||||||
pub number: u64,
|
|
||||||
}
|
|
||||||
|
|
||||||
/// Returns the current block number.
|
/// Returns the current block number.
|
||||||
///
|
///
|
||||||
/// Uses HTTP JSON RPC at `endpoint`. E.g., `http://localhost:8545`.
|
/// Uses HTTP JSON RPC at `endpoint`. E.g., `http://localhost:8545`.
|
||||||
@ -156,40 +150,74 @@ pub async fn get_block_number(endpoint: &SensitiveUrl, timeout: Duration) -> Res
|
|||||||
.map_err(|e| format!("Failed to get block number: {}", e))
|
.map_err(|e| format!("Failed to get block number: {}", e))
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Gets a block hash by block number.
|
/// Gets a block by hash or block number.
|
||||||
///
|
///
|
||||||
/// Uses HTTP JSON RPC at `endpoint`. E.g., `http://localhost:8545`.
|
/// Uses HTTP JSON RPC at `endpoint`. E.g., `http://localhost:8545`.
|
||||||
pub async fn get_block(
|
pub async fn get_block(
|
||||||
endpoint: &SensitiveUrl,
|
endpoint: &SensitiveUrl,
|
||||||
query: BlockQuery,
|
query: BlockQuery,
|
||||||
timeout: Duration,
|
timeout: Duration,
|
||||||
) -> Result<Block, String> {
|
) -> Result<PowBlock, String> {
|
||||||
let query_param = match query {
|
let query_param = match query {
|
||||||
BlockQuery::Number(block_number) => format!("0x{:x}", block_number),
|
BlockQuery::Number(block_number) => format!("0x{:x}", block_number),
|
||||||
|
BlockQuery::Hash(hash) => format!("{:?}", hash), // debug formatting ensures output not truncated
|
||||||
BlockQuery::Latest => "latest".to_string(),
|
BlockQuery::Latest => "latest".to_string(),
|
||||||
};
|
};
|
||||||
|
let rpc_method = match query {
|
||||||
|
BlockQuery::Number(_) | BlockQuery::Latest => "eth_getBlockByNumber",
|
||||||
|
BlockQuery::Hash(_) => "eth_getBlockByHash",
|
||||||
|
};
|
||||||
let params = json!([
|
let params = json!([
|
||||||
query_param,
|
query_param,
|
||||||
false // do not return full tx objects.
|
false // do not return full tx objects.
|
||||||
]);
|
]);
|
||||||
|
|
||||||
let response_body = send_rpc_request(endpoint, "eth_getBlockByNumber", params, timeout).await?;
|
let response_body = send_rpc_request(endpoint, rpc_method, params, timeout).await?;
|
||||||
let response = response_result_or_error(&response_body)
|
let response = response_result_or_error(&response_body)
|
||||||
.map_err(|e| format!("eth_getBlockByNumber failed: {}", e))?;
|
.map_err(|e| format!("{} failed: {}", rpc_method, e))?;
|
||||||
|
|
||||||
let hash: Vec<u8> = hex_to_bytes(
|
let block_hash: Vec<u8> = hex_to_bytes(
|
||||||
response
|
response
|
||||||
.get("hash")
|
.get("hash")
|
||||||
.ok_or("No hash for block")?
|
.ok_or("No hash for block")?
|
||||||
.as_str()
|
.as_str()
|
||||||
.ok_or("Block hash was not string")?,
|
.ok_or("Block hash was not string")?,
|
||||||
)?;
|
)?;
|
||||||
let hash: Hash256 = if hash.len() == 32 {
|
let block_hash: Hash256 = if block_hash.len() == 32 {
|
||||||
Hash256::from_slice(&hash)
|
Hash256::from_slice(&block_hash)
|
||||||
} else {
|
} else {
|
||||||
return Err(format!("Block has was not 32 bytes: {:?}", hash));
|
return Err(format!("Block hash was not 32 bytes: {:?}", block_hash));
|
||||||
};
|
};
|
||||||
|
|
||||||
|
let parent_hash: Vec<u8> = hex_to_bytes(
|
||||||
|
response
|
||||||
|
.get("parentHash")
|
||||||
|
.ok_or("No parent hash for block")?
|
||||||
|
.as_str()
|
||||||
|
.ok_or("Parent hash was not string")?,
|
||||||
|
)?;
|
||||||
|
let parent_hash: Hash256 = if parent_hash.len() == 32 {
|
||||||
|
Hash256::from_slice(&parent_hash)
|
||||||
|
} else {
|
||||||
|
return Err(format!("parent hash was not 32 bytes: {:?}", parent_hash));
|
||||||
|
};
|
||||||
|
|
||||||
|
let total_difficulty_str = response
|
||||||
|
.get("totalDifficulty")
|
||||||
|
.ok_or("No total difficulty for block")?
|
||||||
|
.as_str()
|
||||||
|
.ok_or("Total difficulty was not a string")?;
|
||||||
|
let total_difficulty = Uint256::from_str(total_difficulty_str)
|
||||||
|
.map_err(|e| format!("total_difficulty from_str {:?}", e))?;
|
||||||
|
|
||||||
|
let difficulty_str = response
|
||||||
|
.get("difficulty")
|
||||||
|
.ok_or("No difficulty for block")?
|
||||||
|
.as_str()
|
||||||
|
.ok_or("Difficulty was not a string")?;
|
||||||
|
let difficulty =
|
||||||
|
Uint256::from_str(difficulty_str).map_err(|e| format!("difficulty from_str {:?}", e))?;
|
||||||
|
|
||||||
let timestamp = hex_to_u64_be(
|
let timestamp = hex_to_u64_be(
|
||||||
response
|
response
|
||||||
.get("timestamp")
|
.get("timestamp")
|
||||||
@ -198,7 +226,7 @@ pub async fn get_block(
|
|||||||
.ok_or("Block timestamp was not string")?,
|
.ok_or("Block timestamp was not string")?,
|
||||||
)?;
|
)?;
|
||||||
|
|
||||||
let number = hex_to_u64_be(
|
let block_number = hex_to_u64_be(
|
||||||
response
|
response
|
||||||
.get("number")
|
.get("number")
|
||||||
.ok_or("No number for block")?
|
.ok_or("No number for block")?
|
||||||
@ -206,14 +234,20 @@ pub async fn get_block(
|
|||||||
.ok_or("Block number was not string")?,
|
.ok_or("Block number was not string")?,
|
||||||
)?;
|
)?;
|
||||||
|
|
||||||
if number <= usize::max_value() as u64 {
|
if block_number <= usize::max_value() as u64 {
|
||||||
Ok(Block {
|
Ok(PowBlock {
|
||||||
hash,
|
block_hash,
|
||||||
|
parent_hash,
|
||||||
|
total_difficulty,
|
||||||
|
difficulty,
|
||||||
timestamp,
|
timestamp,
|
||||||
number,
|
block_number,
|
||||||
})
|
})
|
||||||
} else {
|
} else {
|
||||||
Err(format!("Block number {} is larger than a usize", number))
|
Err(format!(
|
||||||
|
"Block number {} is larger than a usize",
|
||||||
|
block_number
|
||||||
|
))
|
||||||
}
|
}
|
||||||
.map_err(|e| format!("Failed to get block number: {}", e))
|
.map_err(|e| format!("Failed to get block number: {}", e))
|
||||||
}
|
}
|
||||||
|
@ -21,7 +21,7 @@ use std::sync::Arc;
|
|||||||
use std::time::{SystemTime, UNIX_EPOCH};
|
use std::time::{SystemTime, UNIX_EPOCH};
|
||||||
use tokio::sync::RwLock as TRwLock;
|
use tokio::sync::RwLock as TRwLock;
|
||||||
use tokio::time::{interval_at, Duration, Instant};
|
use tokio::time::{interval_at, Duration, Instant};
|
||||||
use types::{ChainSpec, EthSpec, Unsigned};
|
use types::{ChainSpec, EthSpec, ExecutionPayload, Unsigned};
|
||||||
|
|
||||||
/// Indicates the default eth1 network id we use for the deposit contract.
|
/// Indicates the default eth1 network id we use for the deposit contract.
|
||||||
pub const DEFAULT_NETWORK_ID: Eth1Id = Eth1Id::Goerli;
|
pub const DEFAULT_NETWORK_ID: Eth1Id = Eth1Id::Goerli;
|
||||||
@ -331,6 +331,8 @@ pub enum SingleEndpointError {
|
|||||||
GetDepositCountFailed(String),
|
GetDepositCountFailed(String),
|
||||||
/// Failed to read the deposit contract root from the eth1 node.
|
/// Failed to read the deposit contract root from the eth1 node.
|
||||||
GetDepositLogsFailed(String),
|
GetDepositLogsFailed(String),
|
||||||
|
/// Failed to run engine_ExecutePayload
|
||||||
|
EngineExecutePayloadFailed,
|
||||||
}
|
}
|
||||||
|
|
||||||
#[derive(Debug, PartialEq)]
|
#[derive(Debug, PartialEq)]
|
||||||
@ -669,6 +671,21 @@ impl Service {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/// This is were we call out to engine_executePayload to determine if payload is valid
|
||||||
|
pub async fn on_payload<T: EthSpec>(
|
||||||
|
&self,
|
||||||
|
_execution_payload: ExecutionPayload<T>,
|
||||||
|
) -> Result<bool, Error> {
|
||||||
|
let endpoints = self.init_endpoints();
|
||||||
|
|
||||||
|
// TODO: call engine_executePayload and figure out how backup endpoint works..
|
||||||
|
endpoints
|
||||||
|
.first_success(|_e| async move { Ok(true) })
|
||||||
|
.await
|
||||||
|
.map(|(res, _)| res)
|
||||||
|
.map_err(Error::FallbackError)
|
||||||
|
}
|
||||||
|
|
||||||
/// Update the deposit and block cache, returning an error if either fail.
|
/// Update the deposit and block cache, returning an error if either fail.
|
||||||
///
|
///
|
||||||
/// ## Returns
|
/// ## Returns
|
||||||
@ -1242,7 +1259,7 @@ async fn download_eth1_block(
|
|||||||
});
|
});
|
||||||
|
|
||||||
// Performs a `get_blockByNumber` call to an eth1 node.
|
// Performs a `get_blockByNumber` call to an eth1 node.
|
||||||
let http_block = get_block(
|
let pow_block = get_block(
|
||||||
endpoint,
|
endpoint,
|
||||||
block_number_opt
|
block_number_opt
|
||||||
.map(BlockQuery::Number)
|
.map(BlockQuery::Number)
|
||||||
@ -1253,9 +1270,9 @@ async fn download_eth1_block(
|
|||||||
.await?;
|
.await?;
|
||||||
|
|
||||||
Ok(Eth1Block {
|
Ok(Eth1Block {
|
||||||
hash: http_block.hash,
|
hash: pow_block.block_hash,
|
||||||
number: http_block.number,
|
number: pow_block.block_number,
|
||||||
timestamp: http_block.timestamp,
|
timestamp: pow_block.timestamp,
|
||||||
deposit_root,
|
deposit_root,
|
||||||
deposit_count,
|
deposit_count,
|
||||||
})
|
})
|
||||||
|
@ -1,6 +1,6 @@
|
|||||||
#![cfg(test)]
|
#![cfg(test)]
|
||||||
use environment::{Environment, EnvironmentBuilder};
|
use environment::{Environment, EnvironmentBuilder};
|
||||||
use eth1::http::{get_deposit_count, get_deposit_logs_in_range, get_deposit_root, Block, Log};
|
use eth1::http::{get_deposit_count, get_deposit_logs_in_range, get_deposit_root, Log};
|
||||||
use eth1::{Config, Service};
|
use eth1::{Config, Service};
|
||||||
use eth1::{DepositCache, DEFAULT_CHAIN_ID, DEFAULT_NETWORK_ID};
|
use eth1::{DepositCache, DEFAULT_CHAIN_ID, DEFAULT_NETWORK_ID};
|
||||||
use eth1_test_rig::GanacheEth1Instance;
|
use eth1_test_rig::GanacheEth1Instance;
|
||||||
@ -571,8 +571,9 @@ mod deposit_tree {
|
|||||||
mod http {
|
mod http {
|
||||||
use super::*;
|
use super::*;
|
||||||
use eth1::http::BlockQuery;
|
use eth1::http::BlockQuery;
|
||||||
|
use types::PowBlock;
|
||||||
|
|
||||||
async fn get_block(eth1: &GanacheEth1Instance, block_number: u64) -> Block {
|
async fn get_block(eth1: &GanacheEth1Instance, block_number: u64) -> PowBlock {
|
||||||
eth1::http::get_block(
|
eth1::http::get_block(
|
||||||
&SensitiveUrl::parse(eth1.endpoint().as_str()).unwrap(),
|
&SensitiveUrl::parse(eth1.endpoint().as_str()).unwrap(),
|
||||||
BlockQuery::Number(block_number),
|
BlockQuery::Number(block_number),
|
||||||
@ -639,7 +640,7 @@ mod http {
|
|||||||
// Check the block hash.
|
// Check the block hash.
|
||||||
let new_block = get_block(ð1, block_number).await;
|
let new_block = get_block(ð1, block_number).await;
|
||||||
assert_ne!(
|
assert_ne!(
|
||||||
new_block.hash, old_block.hash,
|
new_block.block_hash, old_block.block_hash,
|
||||||
"block hash should change with each deposit"
|
"block hash should change with each deposit"
|
||||||
);
|
);
|
||||||
|
|
||||||
@ -661,7 +662,7 @@ mod http {
|
|||||||
// Check to ensure the block root is changing
|
// Check to ensure the block root is changing
|
||||||
assert_ne!(
|
assert_ne!(
|
||||||
new_root,
|
new_root,
|
||||||
Some(new_block.hash),
|
Some(new_block.block_hash),
|
||||||
"the deposit root should be different to the block hash"
|
"the deposit root should be different to the block hash"
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
@ -209,7 +209,9 @@ pub fn gossipsub_config(fork_context: Arc<ForkContext>) -> GossipsubConfig {
|
|||||||
) -> Vec<u8> {
|
) -> Vec<u8> {
|
||||||
let topic_bytes = message.topic.as_str().as_bytes();
|
let topic_bytes = message.topic.as_str().as_bytes();
|
||||||
match fork_context.current_fork() {
|
match fork_context.current_fork() {
|
||||||
ForkName::Altair => {
|
// according to: https://github.com/ethereum/consensus-specs/blob/dev/specs/merge/p2p-interface.md#the-gossip-domain-gossipsub
|
||||||
|
// the derivation of the message-id remains the same in the merge
|
||||||
|
ForkName::Altair | ForkName::Merge => {
|
||||||
let topic_len_bytes = topic_bytes.len().to_le_bytes();
|
let topic_len_bytes = topic_bytes.len().to_le_bytes();
|
||||||
let mut vec = Vec::with_capacity(
|
let mut vec = Vec::with_capacity(
|
||||||
prefix.len() + topic_len_bytes.len() + topic_bytes.len() + message.data.len(),
|
prefix.len() + topic_len_bytes.len() + topic_bytes.len() + message.data.len(),
|
||||||
|
@ -17,7 +17,7 @@ use std::sync::Arc;
|
|||||||
use tokio_util::codec::{Decoder, Encoder};
|
use tokio_util::codec::{Decoder, Encoder};
|
||||||
use types::{
|
use types::{
|
||||||
EthSpec, ForkContext, ForkName, SignedBeaconBlock, SignedBeaconBlockAltair,
|
EthSpec, ForkContext, ForkName, SignedBeaconBlock, SignedBeaconBlockAltair,
|
||||||
SignedBeaconBlockBase,
|
SignedBeaconBlockBase, SignedBeaconBlockMerge,
|
||||||
};
|
};
|
||||||
use unsigned_varint::codec::Uvi;
|
use unsigned_varint::codec::Uvi;
|
||||||
|
|
||||||
@ -375,7 +375,7 @@ fn handle_error<T>(
|
|||||||
}
|
}
|
||||||
|
|
||||||
/// Returns `Some(context_bytes)` for encoding RPC responses that require context bytes.
|
/// Returns `Some(context_bytes)` for encoding RPC responses that require context bytes.
|
||||||
/// Returns `None` when context bytes are not required.
|
/// Returns `None` when context bytes are not required.
|
||||||
fn context_bytes<T: EthSpec>(
|
fn context_bytes<T: EthSpec>(
|
||||||
protocol: &ProtocolId,
|
protocol: &ProtocolId,
|
||||||
fork_context: &ForkContext,
|
fork_context: &ForkContext,
|
||||||
@ -383,23 +383,25 @@ fn context_bytes<T: EthSpec>(
|
|||||||
) -> Option<[u8; CONTEXT_BYTES_LEN]> {
|
) -> Option<[u8; CONTEXT_BYTES_LEN]> {
|
||||||
// Add the context bytes if required
|
// Add the context bytes if required
|
||||||
if protocol.has_context_bytes() {
|
if protocol.has_context_bytes() {
|
||||||
if let RPCCodedResponse::Success(RPCResponse::BlocksByRange(res)) = resp {
|
if let RPCCodedResponse::Success(rpc_variant) = resp {
|
||||||
if let SignedBeaconBlock::Altair { .. } = **res {
|
if let RPCResponse::BlocksByRange(ref_box_block)
|
||||||
// Altair context being `None` implies that "altair never happened".
|
| RPCResponse::BlocksByRoot(ref_box_block) = rpc_variant
|
||||||
// This code should be unreachable if altair is disabled since only Version::V1 would be valid in that case.
|
{
|
||||||
return fork_context.to_context_bytes(ForkName::Altair);
|
return match **ref_box_block {
|
||||||
} else if let SignedBeaconBlock::Base { .. } = **res {
|
// NOTE: If you are adding another fork type here, be sure to modify the
|
||||||
return Some(fork_context.genesis_context_bytes());
|
// `fork_context.to_context_bytes()` function to support it as well!
|
||||||
}
|
SignedBeaconBlock::Merge { .. } => {
|
||||||
}
|
// TODO: check this
|
||||||
|
// Merge context being `None` implies that "merge never happened".
|
||||||
if let RPCCodedResponse::Success(RPCResponse::BlocksByRoot(res)) = resp {
|
fork_context.to_context_bytes(ForkName::Merge)
|
||||||
if let SignedBeaconBlock::Altair { .. } = **res {
|
}
|
||||||
// Altair context being `None` implies that "altair never happened".
|
SignedBeaconBlock::Altair { .. } => {
|
||||||
// This code should be unreachable if altair is disabled since only Version::V1 would be valid in that case.
|
// Altair context being `None` implies that "altair never happened".
|
||||||
return fork_context.to_context_bytes(ForkName::Altair);
|
// This code should be unreachable if altair is disabled since only Version::V1 would be valid in that case.
|
||||||
} else if let SignedBeaconBlock::Base { .. } = **res {
|
fork_context.to_context_bytes(ForkName::Altair)
|
||||||
return Some(fork_context.genesis_context_bytes());
|
}
|
||||||
|
SignedBeaconBlock::Base { .. } => Some(fork_context.genesis_context_bytes()),
|
||||||
|
};
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -559,6 +561,12 @@ fn handle_v2_response<T: EthSpec>(
|
|||||||
ForkName::Base => Ok(Some(RPCResponse::BlocksByRange(Box::new(
|
ForkName::Base => Ok(Some(RPCResponse::BlocksByRange(Box::new(
|
||||||
SignedBeaconBlock::Base(SignedBeaconBlockBase::from_ssz_bytes(decoded_buffer)?),
|
SignedBeaconBlock::Base(SignedBeaconBlockBase::from_ssz_bytes(decoded_buffer)?),
|
||||||
)))),
|
)))),
|
||||||
|
// TODO: check this (though it seems okay)
|
||||||
|
ForkName::Merge => Ok(Some(RPCResponse::BlocksByRange(Box::new(
|
||||||
|
SignedBeaconBlock::Merge(SignedBeaconBlockMerge::from_ssz_bytes(
|
||||||
|
decoded_buffer,
|
||||||
|
)?),
|
||||||
|
)))),
|
||||||
},
|
},
|
||||||
Protocol::BlocksByRoot => match fork_name {
|
Protocol::BlocksByRoot => match fork_name {
|
||||||
ForkName::Altair => Ok(Some(RPCResponse::BlocksByRoot(Box::new(
|
ForkName::Altair => Ok(Some(RPCResponse::BlocksByRoot(Box::new(
|
||||||
@ -569,6 +577,12 @@ fn handle_v2_response<T: EthSpec>(
|
|||||||
ForkName::Base => Ok(Some(RPCResponse::BlocksByRoot(Box::new(
|
ForkName::Base => Ok(Some(RPCResponse::BlocksByRoot(Box::new(
|
||||||
SignedBeaconBlock::Base(SignedBeaconBlockBase::from_ssz_bytes(decoded_buffer)?),
|
SignedBeaconBlock::Base(SignedBeaconBlockBase::from_ssz_bytes(decoded_buffer)?),
|
||||||
)))),
|
)))),
|
||||||
|
// TODO: check this (though it seems right)
|
||||||
|
ForkName::Merge => Ok(Some(RPCResponse::BlocksByRoot(Box::new(
|
||||||
|
SignedBeaconBlock::Merge(SignedBeaconBlockMerge::from_ssz_bytes(
|
||||||
|
decoded_buffer,
|
||||||
|
)?),
|
||||||
|
)))),
|
||||||
},
|
},
|
||||||
_ => Err(RPCError::ErrorResponse(
|
_ => Err(RPCError::ErrorResponse(
|
||||||
RPCResponseErrorCode::InvalidRequest,
|
RPCResponseErrorCode::InvalidRequest,
|
||||||
|
@ -21,8 +21,8 @@ use tokio_util::{
|
|||||||
compat::{Compat, FuturesAsyncReadCompatExt},
|
compat::{Compat, FuturesAsyncReadCompatExt},
|
||||||
};
|
};
|
||||||
use types::{
|
use types::{
|
||||||
BeaconBlock, BeaconBlockAltair, BeaconBlockBase, EthSpec, ForkContext, Hash256, MainnetEthSpec,
|
BeaconBlock, BeaconBlockAltair, BeaconBlockBase, BeaconBlockMerge, EthSpec, ForkContext,
|
||||||
Signature, SignedBeaconBlock,
|
Hash256, MainnetEthSpec, Signature, SignedBeaconBlock,
|
||||||
};
|
};
|
||||||
|
|
||||||
lazy_static! {
|
lazy_static! {
|
||||||
@ -53,6 +53,20 @@ lazy_static! {
|
|||||||
)
|
)
|
||||||
.as_ssz_bytes()
|
.as_ssz_bytes()
|
||||||
.len();
|
.len();
|
||||||
|
|
||||||
|
pub static ref SIGNED_BEACON_BLOCK_MERGE_MIN: usize = SignedBeaconBlock::<MainnetEthSpec>::from_block(
|
||||||
|
BeaconBlock::Merge(BeaconBlockMerge::<MainnetEthSpec>::empty(&MainnetEthSpec::default_spec())),
|
||||||
|
Signature::empty(),
|
||||||
|
)
|
||||||
|
.as_ssz_bytes()
|
||||||
|
.len();
|
||||||
|
pub static ref SIGNED_BEACON_BLOCK_MERGE_MAX: usize = SignedBeaconBlock::<MainnetEthSpec>::from_block(
|
||||||
|
BeaconBlock::Merge(BeaconBlockMerge::full(&MainnetEthSpec::default_spec())),
|
||||||
|
Signature::empty(),
|
||||||
|
)
|
||||||
|
.as_ssz_bytes()
|
||||||
|
.len();
|
||||||
|
|
||||||
pub static ref BLOCKS_BY_ROOT_REQUEST_MIN: usize =
|
pub static ref BLOCKS_BY_ROOT_REQUEST_MIN: usize =
|
||||||
VariableList::<Hash256, MaxRequestBlocks>::from(Vec::<Hash256>::new())
|
VariableList::<Hash256, MaxRequestBlocks>::from(Vec::<Hash256>::new())
|
||||||
.as_ssz_bytes()
|
.as_ssz_bytes()
|
||||||
@ -253,12 +267,18 @@ 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 => RpcLimits::new(
|
Protocol::BlocksByRange => RpcLimits::new(
|
||||||
std::cmp::min(
|
std::cmp::min(
|
||||||
*SIGNED_BEACON_BLOCK_ALTAIR_MIN,
|
std::cmp::min(
|
||||||
*SIGNED_BEACON_BLOCK_BASE_MIN,
|
*SIGNED_BEACON_BLOCK_ALTAIR_MIN,
|
||||||
|
*SIGNED_BEACON_BLOCK_BASE_MIN,
|
||||||
|
),
|
||||||
|
*SIGNED_BEACON_BLOCK_MERGE_MIN,
|
||||||
),
|
),
|
||||||
std::cmp::max(
|
std::cmp::max(
|
||||||
*SIGNED_BEACON_BLOCK_ALTAIR_MAX,
|
std::cmp::max(
|
||||||
*SIGNED_BEACON_BLOCK_BASE_MAX,
|
*SIGNED_BEACON_BLOCK_ALTAIR_MAX,
|
||||||
|
*SIGNED_BEACON_BLOCK_BASE_MAX,
|
||||||
|
),
|
||||||
|
*SIGNED_BEACON_BLOCK_MERGE_MAX,
|
||||||
),
|
),
|
||||||
),
|
),
|
||||||
Protocol::BlocksByRoot => RpcLimits::new(
|
Protocol::BlocksByRoot => RpcLimits::new(
|
||||||
|
@ -10,7 +10,8 @@ use std::io::{Error, ErrorKind};
|
|||||||
use types::{
|
use types::{
|
||||||
Attestation, AttesterSlashing, EthSpec, ForkContext, ForkName, ProposerSlashing,
|
Attestation, AttesterSlashing, EthSpec, ForkContext, ForkName, ProposerSlashing,
|
||||||
SignedAggregateAndProof, SignedBeaconBlock, SignedBeaconBlockAltair, SignedBeaconBlockBase,
|
SignedAggregateAndProof, SignedBeaconBlock, SignedBeaconBlockAltair, SignedBeaconBlockBase,
|
||||||
SignedContributionAndProof, SignedVoluntaryExit, SubnetId, SyncCommitteeMessage, SyncSubnetId,
|
SignedBeaconBlockMerge, SignedContributionAndProof, SignedVoluntaryExit, SubnetId,
|
||||||
|
SyncCommitteeMessage, SyncSubnetId,
|
||||||
};
|
};
|
||||||
|
|
||||||
#[derive(Debug, Clone, PartialEq)]
|
#[derive(Debug, Clone, PartialEq)]
|
||||||
@ -161,6 +162,10 @@ impl<T: EthSpec> PubsubMessage<T> {
|
|||||||
SignedBeaconBlockAltair::from_ssz_bytes(data)
|
SignedBeaconBlockAltair::from_ssz_bytes(data)
|
||||||
.map_err(|e| format!("{:?}", e))?,
|
.map_err(|e| format!("{:?}", e))?,
|
||||||
),
|
),
|
||||||
|
Some(ForkName::Merge) => SignedBeaconBlock::<T>::Merge(
|
||||||
|
SignedBeaconBlockMerge::from_ssz_bytes(data)
|
||||||
|
.map_err(|e| format!("{:?}", e))?,
|
||||||
|
),
|
||||||
None => {
|
None => {
|
||||||
return Err(format!(
|
return Err(format!(
|
||||||
"Unknown gossipsub fork digest: {:?}",
|
"Unknown gossipsub fork digest: {:?}",
|
||||||
|
@ -5,7 +5,8 @@ use beacon_chain::{
|
|||||||
observed_operations::ObservationOutcome,
|
observed_operations::ObservationOutcome,
|
||||||
sync_committee_verification::Error as SyncCommitteeError,
|
sync_committee_verification::Error as SyncCommitteeError,
|
||||||
validator_monitor::get_block_delay_ms,
|
validator_monitor::get_block_delay_ms,
|
||||||
BeaconChainError, BeaconChainTypes, BlockError, ForkChoiceError, GossipVerifiedBlock,
|
BeaconChainError, BeaconChainTypes, BlockError, ExecutionPayloadError, ForkChoiceError,
|
||||||
|
GossipVerifiedBlock,
|
||||||
};
|
};
|
||||||
use lighthouse_network::{Client, MessageAcceptance, MessageId, PeerAction, PeerId, ReportSource};
|
use lighthouse_network::{Client, MessageAcceptance, MessageId, PeerAction, PeerId, ReportSource};
|
||||||
use slog::{crit, debug, error, info, trace, warn};
|
use slog::{crit, debug, error, info, trace, warn};
|
||||||
@ -746,6 +747,16 @@ impl<T: BeaconChainTypes> Worker<T> {
|
|||||||
self.propagate_validation_result(message_id, peer_id, MessageAcceptance::Ignore);
|
self.propagate_validation_result(message_id, peer_id, MessageAcceptance::Ignore);
|
||||||
return None;
|
return None;
|
||||||
}
|
}
|
||||||
|
// TODO: check that this is what we're supposed to do when we don't want to
|
||||||
|
// penalize a peer for our configuration issue
|
||||||
|
// in the verification process BUT is this the proper way to handle it?
|
||||||
|
Err(e @BlockError::ExecutionPayloadError(ExecutionPayloadError::Eth1VerificationError(_)))
|
||||||
|
| Err(e @BlockError::ExecutionPayloadError(ExecutionPayloadError::NoEth1Connection)) => {
|
||||||
|
debug!(self.log, "Could not verify block for gossip, ignoring the block";
|
||||||
|
"error" => %e);
|
||||||
|
self.propagate_validation_result(message_id, peer_id, MessageAcceptance::Ignore);
|
||||||
|
return;
|
||||||
|
}
|
||||||
Err(e @ BlockError::StateRootMismatch { .. })
|
Err(e @ BlockError::StateRootMismatch { .. })
|
||||||
| Err(e @ BlockError::IncorrectBlockProposer { .. })
|
| Err(e @ BlockError::IncorrectBlockProposer { .. })
|
||||||
| Err(e @ BlockError::BlockSlotLimitReached)
|
| Err(e @ BlockError::BlockSlotLimitReached)
|
||||||
@ -759,6 +770,8 @@ impl<T: BeaconChainTypes> Worker<T> {
|
|||||||
| Err(e @ BlockError::TooManySkippedSlots { .. })
|
| Err(e @ BlockError::TooManySkippedSlots { .. })
|
||||||
| Err(e @ BlockError::WeakSubjectivityConflict)
|
| Err(e @ BlockError::WeakSubjectivityConflict)
|
||||||
| Err(e @ BlockError::InconsistentFork(_))
|
| Err(e @ BlockError::InconsistentFork(_))
|
||||||
|
// TODO: is this what we should be doing when block verification fails?
|
||||||
|
| Err(e @BlockError::ExecutionPayloadError(_))
|
||||||
| Err(e @ BlockError::GenesisBlock) => {
|
| Err(e @ BlockError::GenesisBlock) => {
|
||||||
warn!(self.log, "Could not verify block for gossip, rejecting the block";
|
warn!(self.log, "Could not verify block for gossip, rejecting the block";
|
||||||
"error" => %e);
|
"error" => %e);
|
||||||
|
@ -883,6 +883,10 @@ impl<E: EthSpec, Hot: ItemStore<E>, Cold: ItemStore<E>> HotColdDB<E, Hot, Cold>
|
|||||||
&mut block.message.state_root,
|
&mut block.message.state_root,
|
||||||
&mut block.message.parent_root,
|
&mut block.message.parent_root,
|
||||||
),
|
),
|
||||||
|
SignedBeaconBlock::Merge(block) => (
|
||||||
|
&mut block.message.state_root,
|
||||||
|
&mut block.message.parent_root,
|
||||||
|
),
|
||||||
};
|
};
|
||||||
|
|
||||||
*state_root = Hash256::zero();
|
*state_root = Hash256::zero();
|
||||||
|
@ -14,8 +14,8 @@ use types::*;
|
|||||||
///
|
///
|
||||||
/// Utilises lazy-loading from separate storage for its vector fields.
|
/// Utilises lazy-loading from separate storage for its vector fields.
|
||||||
#[superstruct(
|
#[superstruct(
|
||||||
variants(Base, Altair),
|
variants(Base, Altair, Merge),
|
||||||
variant_attributes(derive(Debug, PartialEq, Clone, Encode, Decode),)
|
variant_attributes(derive(Debug, PartialEq, Clone, Encode, Decode))
|
||||||
)]
|
)]
|
||||||
#[derive(Debug, PartialEq, Clone, Encode)]
|
#[derive(Debug, PartialEq, Clone, Encode)]
|
||||||
#[ssz(enum_behaviour = "transparent")]
|
#[ssz(enum_behaviour = "transparent")]
|
||||||
@ -66,9 +66,9 @@ where
|
|||||||
pub current_epoch_attestations: VariableList<PendingAttestation<T>, T::MaxPendingAttestations>,
|
pub current_epoch_attestations: VariableList<PendingAttestation<T>, T::MaxPendingAttestations>,
|
||||||
|
|
||||||
// Participation (Altair and later)
|
// Participation (Altair and later)
|
||||||
#[superstruct(only(Altair))]
|
#[superstruct(only(Altair, Merge))]
|
||||||
pub previous_epoch_participation: VariableList<ParticipationFlags, T::ValidatorRegistryLimit>,
|
pub previous_epoch_participation: VariableList<ParticipationFlags, T::ValidatorRegistryLimit>,
|
||||||
#[superstruct(only(Altair))]
|
#[superstruct(only(Altair, Merge))]
|
||||||
pub current_epoch_participation: VariableList<ParticipationFlags, T::ValidatorRegistryLimit>,
|
pub current_epoch_participation: VariableList<ParticipationFlags, T::ValidatorRegistryLimit>,
|
||||||
|
|
||||||
// Finality
|
// Finality
|
||||||
@ -78,14 +78,18 @@ where
|
|||||||
pub finalized_checkpoint: Checkpoint,
|
pub finalized_checkpoint: Checkpoint,
|
||||||
|
|
||||||
// Inactivity
|
// Inactivity
|
||||||
#[superstruct(only(Altair))]
|
#[superstruct(only(Altair, Merge))]
|
||||||
pub inactivity_scores: VariableList<u64, T::ValidatorRegistryLimit>,
|
pub inactivity_scores: VariableList<u64, T::ValidatorRegistryLimit>,
|
||||||
|
|
||||||
// Light-client sync committees
|
// Light-client sync committees
|
||||||
#[superstruct(only(Altair))]
|
#[superstruct(only(Altair, Merge))]
|
||||||
pub current_sync_committee: Arc<SyncCommittee<T>>,
|
pub current_sync_committee: Arc<SyncCommittee<T>>,
|
||||||
#[superstruct(only(Altair))]
|
#[superstruct(only(Altair, Merge))]
|
||||||
pub next_sync_committee: Arc<SyncCommittee<T>>,
|
pub next_sync_committee: Arc<SyncCommittee<T>>,
|
||||||
|
|
||||||
|
// Execution
|
||||||
|
#[superstruct(only(Merge))]
|
||||||
|
pub latest_execution_payload_header: ExecutionPayloadHeader<T>,
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Implement the conversion function from BeaconState -> PartialBeaconState.
|
/// Implement the conversion function from BeaconState -> PartialBeaconState.
|
||||||
@ -160,6 +164,20 @@ impl<T: EthSpec> PartialBeaconState<T> {
|
|||||||
inactivity_scores
|
inactivity_scores
|
||||||
]
|
]
|
||||||
),
|
),
|
||||||
|
BeaconState::Merge(s) => impl_from_state_forgetful!(
|
||||||
|
s,
|
||||||
|
outer,
|
||||||
|
Merge,
|
||||||
|
PartialBeaconStateMerge,
|
||||||
|
[
|
||||||
|
previous_epoch_participation,
|
||||||
|
current_epoch_participation,
|
||||||
|
current_sync_committee,
|
||||||
|
next_sync_committee,
|
||||||
|
inactivity_scores,
|
||||||
|
latest_execution_payload_header
|
||||||
|
]
|
||||||
|
),
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -334,6 +352,19 @@ impl<E: EthSpec> TryInto<BeaconState<E>> for PartialBeaconState<E> {
|
|||||||
inactivity_scores
|
inactivity_scores
|
||||||
]
|
]
|
||||||
),
|
),
|
||||||
|
PartialBeaconState::Merge(inner) => impl_try_into_beacon_state!(
|
||||||
|
inner,
|
||||||
|
Merge,
|
||||||
|
BeaconStateMerge,
|
||||||
|
[
|
||||||
|
previous_epoch_participation,
|
||||||
|
current_epoch_participation,
|
||||||
|
current_sync_committee,
|
||||||
|
next_sync_committee,
|
||||||
|
inactivity_scores,
|
||||||
|
latest_execution_payload_header
|
||||||
|
]
|
||||||
|
),
|
||||||
};
|
};
|
||||||
Ok(state)
|
Ok(state)
|
||||||
}
|
}
|
||||||
|
@ -8,6 +8,7 @@ edition = "2018"
|
|||||||
|
|
||||||
[dependencies]
|
[dependencies]
|
||||||
types = { path = "../types" }
|
types = { path = "../types" }
|
||||||
|
state_processing = { path = "../state_processing" }
|
||||||
proto_array = { path = "../proto_array" }
|
proto_array = { path = "../proto_array" }
|
||||||
eth2_ssz = "0.4.0"
|
eth2_ssz = "0.4.0"
|
||||||
eth2_ssz_derive = "0.3.0"
|
eth2_ssz_derive = "0.3.0"
|
||||||
|
@ -2,9 +2,11 @@ use std::marker::PhantomData;
|
|||||||
|
|
||||||
use proto_array::{Block as ProtoBlock, ProtoArrayForkChoice};
|
use proto_array::{Block as ProtoBlock, ProtoArrayForkChoice};
|
||||||
use ssz_derive::{Decode, Encode};
|
use ssz_derive::{Decode, Encode};
|
||||||
|
use state_processing::per_block_processing::is_merge_block;
|
||||||
use types::{
|
use types::{
|
||||||
AttestationShufflingId, BeaconBlock, BeaconState, BeaconStateError, ChainSpec, Checkpoint,
|
AttestationShufflingId, BeaconBlock, BeaconState, BeaconStateError, ChainSpec, Checkpoint,
|
||||||
Epoch, EthSpec, Hash256, IndexedAttestation, RelativeEpoch, SignedBeaconBlock, Slot,
|
Epoch, EthSpec, Hash256, IndexedAttestation, PowBlock, RelativeEpoch, SignedBeaconBlock, Slot,
|
||||||
|
Uint256,
|
||||||
};
|
};
|
||||||
|
|
||||||
use crate::ForkChoiceStore;
|
use crate::ForkChoiceStore;
|
||||||
@ -60,6 +62,10 @@ pub enum InvalidBlock {
|
|||||||
finalized_root: Hash256,
|
finalized_root: Hash256,
|
||||||
block_ancestor: Option<Hash256>,
|
block_ancestor: Option<Hash256>,
|
||||||
},
|
},
|
||||||
|
InvalidTerminalPowBlock {
|
||||||
|
block_total_difficulty: Uint256,
|
||||||
|
parent_total_difficulty: Uint256,
|
||||||
|
},
|
||||||
}
|
}
|
||||||
|
|
||||||
#[derive(Debug)]
|
#[derive(Debug)]
|
||||||
@ -231,6 +237,14 @@ where
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/// https://github.com/ethereum/consensus-specs/blob/dev/specs/merge/fork-choice.md#is_valid_terminal_pow_block
|
||||||
|
fn is_valid_terminal_pow_block(block: &PowBlock, parent: &PowBlock, spec: &ChainSpec) -> bool {
|
||||||
|
let is_total_difficulty_reached = block.total_difficulty >= spec.terminal_total_difficulty;
|
||||||
|
let is_parent_total_difficulty_valid = parent.total_difficulty < spec.terminal_total_difficulty;
|
||||||
|
|
||||||
|
is_total_difficulty_reached && is_parent_total_difficulty_valid
|
||||||
|
}
|
||||||
|
|
||||||
impl<T, E> ForkChoice<T, E>
|
impl<T, E> ForkChoice<T, E>
|
||||||
where
|
where
|
||||||
T: ForkChoiceStore<E>,
|
T: ForkChoiceStore<E>,
|
||||||
@ -489,6 +503,19 @@ where
|
|||||||
}));
|
}));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// https://github.com/ethereum/consensus-specs/blob/dev/specs/merge/fork-choice.md#on_block
|
||||||
|
if is_merge_block(state, block.body()) {
|
||||||
|
// TODO: get POW blocks from eth1 chain here as indicated in the merge spec link ^
|
||||||
|
let pow_block = PowBlock::default();
|
||||||
|
let pow_parent = PowBlock::default();
|
||||||
|
if !is_valid_terminal_pow_block(&pow_block, &pow_parent, spec) {
|
||||||
|
return Err(Error::InvalidBlock(InvalidBlock::InvalidTerminalPowBlock {
|
||||||
|
block_total_difficulty: pow_block.total_difficulty,
|
||||||
|
parent_total_difficulty: pow_parent.total_difficulty,
|
||||||
|
}));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
// Update justified checkpoint.
|
// Update justified checkpoint.
|
||||||
if state.current_justified_checkpoint().epoch > self.fc_store.justified_checkpoint().epoch {
|
if state.current_justified_checkpoint().epoch > self.fc_store.justified_checkpoint().epoch {
|
||||||
if state.current_justified_checkpoint().epoch
|
if state.current_justified_checkpoint().epoch
|
||||||
|
@ -1,6 +1,6 @@
|
|||||||
use super::*;
|
use super::*;
|
||||||
use core::num::NonZeroUsize;
|
use core::num::NonZeroUsize;
|
||||||
use ethereum_types::{H256, U128, U256};
|
use ethereum_types::{H160, H256, U128, U256};
|
||||||
use smallvec::SmallVec;
|
use smallvec::SmallVec;
|
||||||
use std::sync::Arc;
|
use std::sync::Arc;
|
||||||
|
|
||||||
@ -256,6 +256,27 @@ impl<T: Decode> Decode for Arc<T> {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
impl Decode for H160 {
|
||||||
|
fn is_ssz_fixed_len() -> bool {
|
||||||
|
true
|
||||||
|
}
|
||||||
|
|
||||||
|
fn ssz_fixed_len() -> usize {
|
||||||
|
20
|
||||||
|
}
|
||||||
|
|
||||||
|
fn from_ssz_bytes(bytes: &[u8]) -> Result<Self, DecodeError> {
|
||||||
|
let len = bytes.len();
|
||||||
|
let expected = <Self as Decode>::ssz_fixed_len();
|
||||||
|
|
||||||
|
if len != expected {
|
||||||
|
Err(DecodeError::InvalidByteLength { len, expected })
|
||||||
|
} else {
|
||||||
|
Ok(Self::from_slice(bytes))
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
impl Decode for H256 {
|
impl Decode for H256 {
|
||||||
fn is_ssz_fixed_len() -> bool {
|
fn is_ssz_fixed_len() -> bool {
|
||||||
true
|
true
|
||||||
|
@ -1,6 +1,6 @@
|
|||||||
use super::*;
|
use super::*;
|
||||||
use core::num::NonZeroUsize;
|
use core::num::NonZeroUsize;
|
||||||
use ethereum_types::{H256, U128, U256};
|
use ethereum_types::{H160, H256, U128, U256};
|
||||||
use smallvec::SmallVec;
|
use smallvec::SmallVec;
|
||||||
use std::sync::Arc;
|
use std::sync::Arc;
|
||||||
|
|
||||||
@ -305,6 +305,24 @@ impl Encode for NonZeroUsize {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
impl Encode for H160 {
|
||||||
|
fn is_ssz_fixed_len() -> bool {
|
||||||
|
true
|
||||||
|
}
|
||||||
|
|
||||||
|
fn ssz_fixed_len() -> usize {
|
||||||
|
20
|
||||||
|
}
|
||||||
|
|
||||||
|
fn ssz_bytes_len(&self) -> usize {
|
||||||
|
20
|
||||||
|
}
|
||||||
|
|
||||||
|
fn ssz_append(&self, buf: &mut Vec<u8>) {
|
||||||
|
buf.extend_from_slice(self.as_bytes());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
impl Encode for H256 {
|
impl Encode for H256 {
|
||||||
fn is_ssz_fixed_len() -> bool {
|
fn is_ssz_fixed_len() -> bool {
|
||||||
true
|
true
|
||||||
|
@ -33,7 +33,7 @@ pub fn slash_validator<T: EthSpec>(
|
|||||||
|
|
||||||
let min_slashing_penalty_quotient = match state {
|
let min_slashing_penalty_quotient = match state {
|
||||||
BeaconState::Base(_) => spec.min_slashing_penalty_quotient,
|
BeaconState::Base(_) => spec.min_slashing_penalty_quotient,
|
||||||
BeaconState::Altair(_) => spec.min_slashing_penalty_quotient_altair,
|
BeaconState::Altair(_) | BeaconState::Merge(_) => spec.min_slashing_penalty_quotient_altair,
|
||||||
};
|
};
|
||||||
decrease_balance(
|
decrease_balance(
|
||||||
state,
|
state,
|
||||||
@ -48,7 +48,7 @@ pub fn slash_validator<T: EthSpec>(
|
|||||||
validator_effective_balance.safe_div(spec.whistleblower_reward_quotient)?;
|
validator_effective_balance.safe_div(spec.whistleblower_reward_quotient)?;
|
||||||
let proposer_reward = match state {
|
let proposer_reward = match state {
|
||||||
BeaconState::Base(_) => whistleblower_reward.safe_div(spec.proposer_reward_quotient)?,
|
BeaconState::Base(_) => whistleblower_reward.safe_div(spec.proposer_reward_quotient)?,
|
||||||
BeaconState::Altair(_) => whistleblower_reward
|
BeaconState::Altair(_) | BeaconState::Merge(_) => whistleblower_reward
|
||||||
.safe_mul(PROPOSER_WEIGHT)?
|
.safe_mul(PROPOSER_WEIGHT)?
|
||||||
.safe_div(WEIGHT_DENOMINATOR)?,
|
.safe_div(WEIGHT_DENOMINATOR)?,
|
||||||
};
|
};
|
||||||
|
@ -52,6 +52,8 @@ pub fn initialize_beacon_state_from_eth1<T: EthSpec>(
|
|||||||
state.fork_mut().previous_version = spec.altair_fork_version;
|
state.fork_mut().previous_version = spec.altair_fork_version;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// TODO: handle upgrade_to_merge() here
|
||||||
|
|
||||||
// Now that we have our validators, initialize the caches (including the committees)
|
// Now that we have our validators, initialize the caches (including the committees)
|
||||||
state.build_all_caches(spec)?;
|
state.build_all_caches(spec)?;
|
||||||
|
|
||||||
|
@ -149,6 +149,10 @@ pub fn per_block_processing<T: EthSpec>(
|
|||||||
)?;
|
)?;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
if is_execution_enabled(state, block.body()) {
|
||||||
|
process_execution_payload(state, block.body().execution_payload().unwrap(), spec)?
|
||||||
|
}
|
||||||
|
|
||||||
Ok(())
|
Ok(())
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -283,3 +287,135 @@ pub fn get_new_eth1_data<T: EthSpec>(
|
|||||||
Ok(None)
|
Ok(None)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/// https://github.com/ethereum/consensus-specs/blob/dev/specs/merge/beacon-chain.md#is_valid_gas_limit
|
||||||
|
pub fn is_valid_gas_limit<T: EthSpec>(
|
||||||
|
payload: &ExecutionPayload<T>,
|
||||||
|
parent: &ExecutionPayloadHeader<T>,
|
||||||
|
) -> bool {
|
||||||
|
// check if payload used too much gas
|
||||||
|
if payload.gas_used > payload.gas_limit {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
// check if payload changed the gas limit too much
|
||||||
|
if payload.gas_limit >= parent.gas_limit + parent.gas_limit / T::gas_limit_denominator() {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
if payload.gas_limit <= parent.gas_limit - parent.gas_limit / T::gas_limit_denominator() {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
// check if the gas limit is at least the minimum gas limit
|
||||||
|
if payload.gas_limit < T::min_gas_limit() {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
/// https://github.com/ethereum/consensus-specs/blob/dev/specs/merge/beacon-chain.md#process_execution_payload
|
||||||
|
pub fn process_execution_payload<T: EthSpec>(
|
||||||
|
state: &mut BeaconState<T>,
|
||||||
|
payload: &ExecutionPayload<T>,
|
||||||
|
spec: &ChainSpec,
|
||||||
|
) -> Result<(), BlockProcessingError> {
|
||||||
|
if is_merge_complete(state) {
|
||||||
|
block_verify!(
|
||||||
|
payload.parent_hash == state.latest_execution_payload_header()?.block_hash,
|
||||||
|
BlockProcessingError::ExecutionHashChainIncontiguous {
|
||||||
|
expected: state.latest_execution_payload_header()?.block_hash,
|
||||||
|
found: payload.parent_hash,
|
||||||
|
}
|
||||||
|
);
|
||||||
|
block_verify!(
|
||||||
|
payload.block_number
|
||||||
|
== state
|
||||||
|
.latest_execution_payload_header()?
|
||||||
|
.block_number
|
||||||
|
.safe_add(1)?,
|
||||||
|
BlockProcessingError::ExecutionBlockNumberIncontiguous {
|
||||||
|
expected: state
|
||||||
|
.latest_execution_payload_header()?
|
||||||
|
.block_number
|
||||||
|
.safe_add(1)?,
|
||||||
|
found: payload.block_number,
|
||||||
|
}
|
||||||
|
);
|
||||||
|
block_verify!(
|
||||||
|
payload.random == *state.get_randao_mix(state.current_epoch())?,
|
||||||
|
BlockProcessingError::ExecutionRandaoMismatch {
|
||||||
|
expected: *state.get_randao_mix(state.current_epoch())?,
|
||||||
|
found: payload.random,
|
||||||
|
}
|
||||||
|
);
|
||||||
|
block_verify!(
|
||||||
|
is_valid_gas_limit(payload, state.latest_execution_payload_header()?),
|
||||||
|
BlockProcessingError::ExecutionInvalidGasLimit {
|
||||||
|
used: payload.gas_used,
|
||||||
|
limit: payload.gas_limit,
|
||||||
|
}
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
let timestamp = compute_timestamp_at_slot(state, spec)?;
|
||||||
|
block_verify!(
|
||||||
|
payload.timestamp == timestamp,
|
||||||
|
BlockProcessingError::ExecutionInvalidTimestamp {
|
||||||
|
expected: timestamp,
|
||||||
|
found: payload.timestamp,
|
||||||
|
}
|
||||||
|
);
|
||||||
|
|
||||||
|
*state.latest_execution_payload_header_mut()? = ExecutionPayloadHeader {
|
||||||
|
parent_hash: payload.parent_hash,
|
||||||
|
coinbase: payload.coinbase,
|
||||||
|
state_root: payload.state_root,
|
||||||
|
receipt_root: payload.receipt_root,
|
||||||
|
logs_bloom: payload.logs_bloom.clone(),
|
||||||
|
random: payload.random,
|
||||||
|
block_number: payload.block_number,
|
||||||
|
gas_limit: payload.gas_limit,
|
||||||
|
gas_used: payload.gas_used,
|
||||||
|
timestamp: payload.timestamp,
|
||||||
|
base_fee_per_gas: payload.base_fee_per_gas,
|
||||||
|
block_hash: payload.block_hash,
|
||||||
|
transactions_root: payload.transactions.tree_hash_root(),
|
||||||
|
};
|
||||||
|
|
||||||
|
Ok(())
|
||||||
|
}
|
||||||
|
|
||||||
|
/// 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
|
||||||
|
/// errors from the `BeaconState` being an earlier variant than `BeaconStateMerge` as we'd have to
|
||||||
|
/// repeaetedly write code to treat these errors as false.
|
||||||
|
/// https://github.com/ethereum/consensus-specs/blob/dev/specs/merge/beacon-chain.md#is_merge_complete
|
||||||
|
pub fn is_merge_complete<T: EthSpec>(state: &BeaconState<T>) -> bool {
|
||||||
|
state
|
||||||
|
.latest_execution_payload_header()
|
||||||
|
.map(|header| *header != <ExecutionPayloadHeader<T>>::default())
|
||||||
|
.unwrap_or(false)
|
||||||
|
}
|
||||||
|
/// https://github.com/ethereum/consensus-specs/blob/dev/specs/merge/beacon-chain.md#is_merge_block
|
||||||
|
pub fn is_merge_block<T: EthSpec>(state: &BeaconState<T>, body: BeaconBlockBodyRef<T>) -> bool {
|
||||||
|
body.execution_payload()
|
||||||
|
.map(|payload| !is_merge_complete(state) && *payload != <ExecutionPayload<T>>::default())
|
||||||
|
.unwrap_or(false)
|
||||||
|
}
|
||||||
|
/// https://github.com/ethereum/consensus-specs/blob/dev/specs/merge/beacon-chain.md#is_execution_enabled
|
||||||
|
pub fn is_execution_enabled<T: EthSpec>(
|
||||||
|
state: &BeaconState<T>,
|
||||||
|
body: BeaconBlockBodyRef<T>,
|
||||||
|
) -> bool {
|
||||||
|
is_merge_block(state, body) || is_merge_complete(state)
|
||||||
|
}
|
||||||
|
|
||||||
|
/// https://github.com/ethereum/consensus-specs/blob/dev/specs/merge/beacon-chain.md#compute_timestamp_at_slot
|
||||||
|
pub fn compute_timestamp_at_slot<T: EthSpec>(
|
||||||
|
state: &BeaconState<T>,
|
||||||
|
spec: &ChainSpec,
|
||||||
|
) -> Result<u64, ArithError> {
|
||||||
|
let slots_since_genesis = state.slot().as_u64().safe_sub(spec.genesis_slot.as_u64())?;
|
||||||
|
slots_since_genesis
|
||||||
|
.safe_mul(spec.seconds_per_slot)
|
||||||
|
.and_then(|since_genesis| state.genesis_time().safe_add(since_genesis))
|
||||||
|
}
|
||||||
|
@ -57,6 +57,26 @@ pub enum BlockProcessingError {
|
|||||||
ArithError(ArithError),
|
ArithError(ArithError),
|
||||||
InconsistentBlockFork(InconsistentFork),
|
InconsistentBlockFork(InconsistentFork),
|
||||||
InconsistentStateFork(InconsistentFork),
|
InconsistentStateFork(InconsistentFork),
|
||||||
|
ExecutionHashChainIncontiguous {
|
||||||
|
expected: Hash256,
|
||||||
|
found: Hash256,
|
||||||
|
},
|
||||||
|
ExecutionBlockNumberIncontiguous {
|
||||||
|
expected: u64,
|
||||||
|
found: u64,
|
||||||
|
},
|
||||||
|
ExecutionRandaoMismatch {
|
||||||
|
expected: Hash256,
|
||||||
|
found: Hash256,
|
||||||
|
},
|
||||||
|
ExecutionInvalidGasLimit {
|
||||||
|
used: u64,
|
||||||
|
limit: u64,
|
||||||
|
},
|
||||||
|
ExecutionInvalidTimestamp {
|
||||||
|
expected: u64,
|
||||||
|
found: u64,
|
||||||
|
},
|
||||||
}
|
}
|
||||||
|
|
||||||
impl From<BeaconStateError> for BlockProcessingError {
|
impl From<BeaconStateError> for BlockProcessingError {
|
||||||
|
@ -228,7 +228,7 @@ pub fn process_attestations<'a, T: EthSpec>(
|
|||||||
BeaconBlockBodyRef::Base(_) => {
|
BeaconBlockBodyRef::Base(_) => {
|
||||||
base::process_attestations(state, block_body.attestations(), verify_signatures, spec)?;
|
base::process_attestations(state, block_body.attestations(), verify_signatures, spec)?;
|
||||||
}
|
}
|
||||||
BeaconBlockBodyRef::Altair(_) => {
|
BeaconBlockBodyRef::Altair(_) | BeaconBlockBodyRef::Merge(_) => {
|
||||||
altair::process_attestations(
|
altair::process_attestations(
|
||||||
state,
|
state,
|
||||||
block_body.attestations(),
|
block_body.attestations(),
|
||||||
|
@ -35,7 +35,7 @@ pub fn process_epoch<T: EthSpec>(
|
|||||||
|
|
||||||
match state {
|
match state {
|
||||||
BeaconState::Base(_) => base::process_epoch(state, spec),
|
BeaconState::Base(_) => base::process_epoch(state, spec),
|
||||||
BeaconState::Altair(_) => altair::process_epoch(state, spec),
|
BeaconState::Altair(_) | BeaconState::Merge(_) => altair::process_epoch(state, spec),
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1,4 +1,4 @@
|
|||||||
use crate::upgrade::upgrade_to_altair;
|
use crate::upgrade::{upgrade_to_altair, upgrade_to_merge};
|
||||||
use crate::{per_epoch_processing::EpochProcessingSummary, *};
|
use crate::{per_epoch_processing::EpochProcessingSummary, *};
|
||||||
use safe_arith::{ArithError, SafeArith};
|
use safe_arith::{ArithError, SafeArith};
|
||||||
use types::*;
|
use types::*;
|
||||||
@ -44,11 +44,17 @@ pub fn per_slot_processing<T: EthSpec>(
|
|||||||
|
|
||||||
state.slot_mut().safe_add_assign(1)?;
|
state.slot_mut().safe_add_assign(1)?;
|
||||||
|
|
||||||
// If the Altair fork epoch is reached, perform an irregular state upgrade.
|
// Process fork upgrades here. Note that multiple upgrades can potentially run
|
||||||
if state.slot().safe_rem(T::slots_per_epoch())? == 0
|
// in sequence if they are scheduled in the same Epoch (common in testnets)
|
||||||
&& spec.altair_fork_epoch == Some(state.current_epoch())
|
if state.slot().safe_rem(T::slots_per_epoch())? == 0 {
|
||||||
{
|
// If the Altair fork epoch is reached, perform an irregular state upgrade.
|
||||||
upgrade_to_altair(state, spec)?;
|
if spec.altair_fork_epoch == Some(state.current_epoch()) {
|
||||||
|
upgrade_to_altair(state, spec)?;
|
||||||
|
}
|
||||||
|
// If the Merge fork epoch is reached, perform an irregular state upgrade.
|
||||||
|
if spec.merge_fork_epoch == Some(state.current_epoch()) {
|
||||||
|
upgrade_to_merge(state, spec)?;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
Ok(summary)
|
Ok(summary)
|
||||||
|
@ -1,3 +1,5 @@
|
|||||||
pub mod altair;
|
pub mod altair;
|
||||||
|
pub mod merge;
|
||||||
|
|
||||||
pub use altair::upgrade_to_altair;
|
pub use altair::upgrade_to_altair;
|
||||||
|
pub use merge::upgrade_to_merge;
|
||||||
|
72
consensus/state_processing/src/upgrade/merge.rs
Normal file
72
consensus/state_processing/src/upgrade/merge.rs
Normal file
@ -0,0 +1,72 @@
|
|||||||
|
use std::mem;
|
||||||
|
use types::{
|
||||||
|
BeaconState, BeaconStateError as Error, BeaconStateMerge, ChainSpec, EthSpec,
|
||||||
|
ExecutionPayloadHeader, Fork,
|
||||||
|
};
|
||||||
|
|
||||||
|
/// Transform a `Altair` state into an `Merge` state.
|
||||||
|
pub fn upgrade_to_merge<E: EthSpec>(
|
||||||
|
pre_state: &mut BeaconState<E>,
|
||||||
|
spec: &ChainSpec,
|
||||||
|
) -> Result<(), Error> {
|
||||||
|
let epoch = pre_state.current_epoch();
|
||||||
|
let pre = pre_state.as_altair_mut()?;
|
||||||
|
|
||||||
|
// Where possible, use something like `mem::take` to move fields from behind the &mut
|
||||||
|
// reference. For other fields that don't have a good default value, use `clone`.
|
||||||
|
//
|
||||||
|
// Fixed size vectors get cloned because replacing them would require the same size
|
||||||
|
// allocation as cloning.
|
||||||
|
let post = BeaconState::Merge(BeaconStateMerge {
|
||||||
|
// Versioning
|
||||||
|
genesis_time: pre.genesis_time,
|
||||||
|
genesis_validators_root: pre.genesis_validators_root,
|
||||||
|
slot: pre.slot,
|
||||||
|
fork: Fork {
|
||||||
|
previous_version: pre.fork.current_version,
|
||||||
|
current_version: spec.merge_fork_version,
|
||||||
|
epoch,
|
||||||
|
},
|
||||||
|
// History
|
||||||
|
latest_block_header: pre.latest_block_header.clone(),
|
||||||
|
block_roots: pre.block_roots.clone(),
|
||||||
|
state_roots: pre.state_roots.clone(),
|
||||||
|
historical_roots: mem::take(&mut pre.historical_roots),
|
||||||
|
// Eth1
|
||||||
|
eth1_data: pre.eth1_data.clone(),
|
||||||
|
eth1_data_votes: mem::take(&mut pre.eth1_data_votes),
|
||||||
|
eth1_deposit_index: pre.eth1_deposit_index,
|
||||||
|
// Registry
|
||||||
|
validators: mem::take(&mut pre.validators),
|
||||||
|
balances: mem::take(&mut pre.balances),
|
||||||
|
// Randomness
|
||||||
|
randao_mixes: pre.randao_mixes.clone(),
|
||||||
|
// Slashings
|
||||||
|
slashings: pre.slashings.clone(),
|
||||||
|
// `Participation
|
||||||
|
previous_epoch_participation: mem::take(&mut pre.previous_epoch_participation),
|
||||||
|
current_epoch_participation: mem::take(&mut pre.current_epoch_participation),
|
||||||
|
// Finality
|
||||||
|
justification_bits: pre.justification_bits.clone(),
|
||||||
|
previous_justified_checkpoint: pre.previous_justified_checkpoint,
|
||||||
|
current_justified_checkpoint: pre.current_justified_checkpoint,
|
||||||
|
finalized_checkpoint: pre.finalized_checkpoint,
|
||||||
|
// Inactivity
|
||||||
|
inactivity_scores: mem::take(&mut pre.inactivity_scores),
|
||||||
|
// Sync committees
|
||||||
|
current_sync_committee: pre.current_sync_committee.clone(),
|
||||||
|
next_sync_committee: pre.next_sync_committee.clone(),
|
||||||
|
// Execution
|
||||||
|
latest_execution_payload_header: <ExecutionPayloadHeader<E>>::default(),
|
||||||
|
// Caches
|
||||||
|
total_active_balance: pre.total_active_balance,
|
||||||
|
committee_caches: mem::take(&mut pre.committee_caches),
|
||||||
|
pubkey_cache: mem::take(&mut pre.pubkey_cache),
|
||||||
|
exit_cache: mem::take(&mut pre.exit_cache),
|
||||||
|
tree_hash_cache: mem::take(&mut pre.tree_hash_cache),
|
||||||
|
});
|
||||||
|
|
||||||
|
*pre_state = post;
|
||||||
|
|
||||||
|
Ok(())
|
||||||
|
}
|
@ -1,5 +1,5 @@
|
|||||||
use super::*;
|
use super::*;
|
||||||
use ethereum_types::{H256, U128, U256};
|
use ethereum_types::{H160, H256, U128, U256};
|
||||||
|
|
||||||
fn int_to_hash256(int: u64) -> Hash256 {
|
fn int_to_hash256(int: u64) -> Hash256 {
|
||||||
let mut bytes = [0; HASHSIZE];
|
let mut bytes = [0; HASHSIZE];
|
||||||
@ -126,6 +126,28 @@ impl TreeHash for U256 {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
impl TreeHash for H160 {
|
||||||
|
fn tree_hash_type() -> TreeHashType {
|
||||||
|
TreeHashType::Vector
|
||||||
|
}
|
||||||
|
|
||||||
|
fn tree_hash_packed_encoding(&self) -> Vec<u8> {
|
||||||
|
let mut result = vec![0; 32];
|
||||||
|
result[0..20].copy_from_slice(self.as_bytes());
|
||||||
|
result
|
||||||
|
}
|
||||||
|
|
||||||
|
fn tree_hash_packing_factor() -> usize {
|
||||||
|
1
|
||||||
|
}
|
||||||
|
|
||||||
|
fn tree_hash_root(&self) -> Hash256 {
|
||||||
|
let mut result = [0; 32];
|
||||||
|
result[0..20].copy_from_slice(self.as_bytes());
|
||||||
|
Hash256::from_slice(&result)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
impl TreeHash for H256 {
|
impl TreeHash for H256 {
|
||||||
fn tree_hash_type() -> TreeHashType {
|
fn tree_hash_type() -> TreeHashType {
|
||||||
TreeHashType::Vector
|
TreeHashType::Vector
|
||||||
|
@ -1,5 +1,6 @@
|
|||||||
use crate::beacon_block_body::{
|
use crate::beacon_block_body::{
|
||||||
BeaconBlockBodyAltair, BeaconBlockBodyBase, BeaconBlockBodyRef, BeaconBlockBodyRefMut,
|
BeaconBlockBodyAltair, BeaconBlockBodyBase, BeaconBlockBodyMerge, BeaconBlockBodyRef,
|
||||||
|
BeaconBlockBodyRefMut,
|
||||||
};
|
};
|
||||||
use crate::test_utils::TestRandom;
|
use crate::test_utils::TestRandom;
|
||||||
use crate::*;
|
use crate::*;
|
||||||
@ -14,7 +15,7 @@ use tree_hash_derive::TreeHash;
|
|||||||
|
|
||||||
/// A block of the `BeaconChain`.
|
/// A block of the `BeaconChain`.
|
||||||
#[superstruct(
|
#[superstruct(
|
||||||
variants(Base, Altair),
|
variants(Base, Altair, Merge),
|
||||||
variant_attributes(
|
variant_attributes(
|
||||||
derive(
|
derive(
|
||||||
Debug,
|
Debug,
|
||||||
@ -55,6 +56,8 @@ pub struct BeaconBlock<T: EthSpec> {
|
|||||||
pub body: BeaconBlockBodyBase<T>,
|
pub body: BeaconBlockBodyBase<T>,
|
||||||
#[superstruct(only(Altair), partial_getter(rename = "body_altair"))]
|
#[superstruct(only(Altair), partial_getter(rename = "body_altair"))]
|
||||||
pub body: BeaconBlockBodyAltair<T>,
|
pub body: BeaconBlockBodyAltair<T>,
|
||||||
|
#[superstruct(only(Merge), partial_getter(rename = "body_merge"))]
|
||||||
|
pub body: BeaconBlockBodyMerge<T>,
|
||||||
}
|
}
|
||||||
|
|
||||||
impl<T: EthSpec> SignedRoot for BeaconBlock<T> {}
|
impl<T: EthSpec> SignedRoot for BeaconBlock<T> {}
|
||||||
@ -63,7 +66,9 @@ impl<'a, T: EthSpec> SignedRoot for BeaconBlockRef<'a, T> {}
|
|||||||
impl<T: EthSpec> BeaconBlock<T> {
|
impl<T: EthSpec> BeaconBlock<T> {
|
||||||
/// Returns an empty block to be used during genesis.
|
/// Returns an empty block to be used during genesis.
|
||||||
pub fn empty(spec: &ChainSpec) -> Self {
|
pub fn empty(spec: &ChainSpec) -> Self {
|
||||||
if spec.altair_fork_epoch == Some(T::genesis_epoch()) {
|
if spec.merge_fork_epoch == Some(T::genesis_epoch()) {
|
||||||
|
Self::Merge(BeaconBlockMerge::empty(spec))
|
||||||
|
} else if spec.altair_fork_epoch == Some(T::genesis_epoch()) {
|
||||||
Self::Altair(BeaconBlockAltair::empty(spec))
|
Self::Altair(BeaconBlockAltair::empty(spec))
|
||||||
} else {
|
} else {
|
||||||
Self::Base(BeaconBlockBase::empty(spec))
|
Self::Base(BeaconBlockBase::empty(spec))
|
||||||
@ -171,6 +176,7 @@ impl<'a, T: EthSpec> BeaconBlockRef<'a, T> {
|
|||||||
let object_fork = match self {
|
let object_fork = match self {
|
||||||
BeaconBlockRef::Base { .. } => ForkName::Base,
|
BeaconBlockRef::Base { .. } => ForkName::Base,
|
||||||
BeaconBlockRef::Altair { .. } => ForkName::Altair,
|
BeaconBlockRef::Altair { .. } => ForkName::Altair,
|
||||||
|
BeaconBlockRef::Merge { .. } => ForkName::Merge,
|
||||||
};
|
};
|
||||||
|
|
||||||
if fork_at_slot == object_fork {
|
if fork_at_slot == object_fork {
|
||||||
@ -188,6 +194,7 @@ impl<'a, T: EthSpec> BeaconBlockRef<'a, T> {
|
|||||||
match self {
|
match self {
|
||||||
BeaconBlockRef::Base(block) => BeaconBlockBodyRef::Base(&block.body),
|
BeaconBlockRef::Base(block) => BeaconBlockBodyRef::Base(&block.body),
|
||||||
BeaconBlockRef::Altair(block) => BeaconBlockBodyRef::Altair(&block.body),
|
BeaconBlockRef::Altair(block) => BeaconBlockBodyRef::Altair(&block.body),
|
||||||
|
BeaconBlockRef::Merge(block) => BeaconBlockBodyRef::Merge(&block.body),
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -196,6 +203,7 @@ impl<'a, T: EthSpec> BeaconBlockRef<'a, T> {
|
|||||||
match self {
|
match self {
|
||||||
BeaconBlockRef::Base(block) => block.body.tree_hash_root(),
|
BeaconBlockRef::Base(block) => block.body.tree_hash_root(),
|
||||||
BeaconBlockRef::Altair(block) => block.body.tree_hash_root(),
|
BeaconBlockRef::Altair(block) => block.body.tree_hash_root(),
|
||||||
|
BeaconBlockRef::Merge(block) => block.body.tree_hash_root(),
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -230,6 +238,7 @@ impl<'a, T: EthSpec> BeaconBlockRefMut<'a, T> {
|
|||||||
match self {
|
match self {
|
||||||
BeaconBlockRefMut::Base(block) => BeaconBlockBodyRefMut::Base(&mut block.body),
|
BeaconBlockRefMut::Base(block) => BeaconBlockBodyRefMut::Base(&mut block.body),
|
||||||
BeaconBlockRefMut::Altair(block) => BeaconBlockBodyRefMut::Altair(&mut block.body),
|
BeaconBlockRefMut::Altair(block) => BeaconBlockBodyRefMut::Altair(&mut block.body),
|
||||||
|
BeaconBlockRefMut::Merge(block) => BeaconBlockBodyRefMut::Merge(&mut block.body),
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -411,6 +420,61 @@ impl<T: EthSpec> BeaconBlockAltair<T> {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
impl<T: EthSpec> BeaconBlockMerge<T> {
|
||||||
|
/// Returns an empty Merge block to be used during genesis.
|
||||||
|
pub fn empty(spec: &ChainSpec) -> Self {
|
||||||
|
BeaconBlockMerge {
|
||||||
|
slot: spec.genesis_slot,
|
||||||
|
proposer_index: 0,
|
||||||
|
parent_root: Hash256::zero(),
|
||||||
|
state_root: Hash256::zero(),
|
||||||
|
body: BeaconBlockBodyMerge {
|
||||||
|
randao_reveal: Signature::empty(),
|
||||||
|
eth1_data: Eth1Data {
|
||||||
|
deposit_root: Hash256::zero(),
|
||||||
|
block_hash: Hash256::zero(),
|
||||||
|
deposit_count: 0,
|
||||||
|
},
|
||||||
|
graffiti: Graffiti::default(),
|
||||||
|
proposer_slashings: VariableList::empty(),
|
||||||
|
attester_slashings: VariableList::empty(),
|
||||||
|
attestations: VariableList::empty(),
|
||||||
|
deposits: VariableList::empty(),
|
||||||
|
voluntary_exits: VariableList::empty(),
|
||||||
|
sync_aggregate: SyncAggregate::empty(),
|
||||||
|
execution_payload: ExecutionPayload::empty(),
|
||||||
|
},
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Return an Merge block where the block has maximum size.
|
||||||
|
pub fn full(spec: &ChainSpec) -> Self {
|
||||||
|
let altair_block = BeaconBlockAltair::full(spec);
|
||||||
|
BeaconBlockMerge {
|
||||||
|
slot: spec.genesis_slot,
|
||||||
|
proposer_index: 0,
|
||||||
|
parent_root: Hash256::zero(),
|
||||||
|
state_root: Hash256::zero(),
|
||||||
|
body: BeaconBlockBodyMerge {
|
||||||
|
proposer_slashings: altair_block.body.proposer_slashings,
|
||||||
|
attester_slashings: altair_block.body.attester_slashings,
|
||||||
|
attestations: altair_block.body.attestations,
|
||||||
|
deposits: altair_block.body.deposits,
|
||||||
|
voluntary_exits: altair_block.body.voluntary_exits,
|
||||||
|
sync_aggregate: altair_block.body.sync_aggregate,
|
||||||
|
randao_reveal: Signature::empty(),
|
||||||
|
eth1_data: Eth1Data {
|
||||||
|
deposit_root: Hash256::zero(),
|
||||||
|
block_hash: Hash256::zero(),
|
||||||
|
deposit_count: 0,
|
||||||
|
},
|
||||||
|
graffiti: Graffiti::default(),
|
||||||
|
execution_payload: ExecutionPayload::default(),
|
||||||
|
},
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
#[cfg(test)]
|
#[cfg(test)]
|
||||||
mod tests {
|
mod tests {
|
||||||
use super::*;
|
use super::*;
|
||||||
|
@ -11,7 +11,7 @@ use tree_hash_derive::TreeHash;
|
|||||||
///
|
///
|
||||||
/// This *superstruct* abstracts over the hard-fork.
|
/// This *superstruct* abstracts over the hard-fork.
|
||||||
#[superstruct(
|
#[superstruct(
|
||||||
variants(Base, Altair),
|
variants(Base, Altair, Merge),
|
||||||
variant_attributes(
|
variant_attributes(
|
||||||
derive(
|
derive(
|
||||||
Debug,
|
Debug,
|
||||||
@ -26,7 +26,9 @@ use tree_hash_derive::TreeHash;
|
|||||||
),
|
),
|
||||||
serde(bound = "T: EthSpec", deny_unknown_fields),
|
serde(bound = "T: EthSpec", deny_unknown_fields),
|
||||||
cfg_attr(feature = "arbitrary-fuzz", derive(arbitrary::Arbitrary))
|
cfg_attr(feature = "arbitrary-fuzz", derive(arbitrary::Arbitrary))
|
||||||
)
|
),
|
||||||
|
cast_error(ty = "Error", expr = "Error::IncorrectStateVariant"),
|
||||||
|
partial_getter_error(ty = "Error", expr = "Error::IncorrectStateVariant")
|
||||||
)]
|
)]
|
||||||
#[derive(Debug, PartialEq, Clone, Serialize, Deserialize)]
|
#[derive(Debug, PartialEq, Clone, Serialize, Deserialize)]
|
||||||
#[serde(untagged)]
|
#[serde(untagged)]
|
||||||
@ -41,8 +43,10 @@ pub struct BeaconBlockBody<T: EthSpec> {
|
|||||||
pub attestations: VariableList<Attestation<T>, T::MaxAttestations>,
|
pub attestations: VariableList<Attestation<T>, T::MaxAttestations>,
|
||||||
pub deposits: VariableList<Deposit, T::MaxDeposits>,
|
pub deposits: VariableList<Deposit, T::MaxDeposits>,
|
||||||
pub voluntary_exits: VariableList<SignedVoluntaryExit, T::MaxVoluntaryExits>,
|
pub voluntary_exits: VariableList<SignedVoluntaryExit, T::MaxVoluntaryExits>,
|
||||||
#[superstruct(only(Altair))]
|
#[superstruct(only(Altair, Merge))]
|
||||||
pub sync_aggregate: SyncAggregate<T>,
|
pub sync_aggregate: SyncAggregate<T>,
|
||||||
|
#[superstruct(only(Merge))]
|
||||||
|
pub execution_payload: ExecutionPayload<T>,
|
||||||
}
|
}
|
||||||
|
|
||||||
impl<'a, T: EthSpec> BeaconBlockBodyRef<'a, T> {
|
impl<'a, T: EthSpec> BeaconBlockBodyRef<'a, T> {
|
||||||
@ -51,6 +55,25 @@ impl<'a, T: EthSpec> BeaconBlockBodyRef<'a, T> {
|
|||||||
match self {
|
match self {
|
||||||
BeaconBlockBodyRef::Base(_) => None,
|
BeaconBlockBodyRef::Base(_) => None,
|
||||||
BeaconBlockBodyRef::Altair(inner) => Some(&inner.sync_aggregate),
|
BeaconBlockBodyRef::Altair(inner) => Some(&inner.sync_aggregate),
|
||||||
|
BeaconBlockBodyRef::Merge(inner) => Some(&inner.sync_aggregate),
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Access the execution payload from the block's body, if one exists.
|
||||||
|
pub fn execution_payload(self) -> Option<&'a ExecutionPayload<T>> {
|
||||||
|
match self {
|
||||||
|
BeaconBlockBodyRef::Base(_) => None,
|
||||||
|
BeaconBlockBodyRef::Altair(_) => None,
|
||||||
|
BeaconBlockBodyRef::Merge(inner) => Some(&inner.execution_payload),
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Get the fork_name of this object
|
||||||
|
pub fn fork_name(self) -> ForkName {
|
||||||
|
match self {
|
||||||
|
BeaconBlockBodyRef::Base { .. } => ForkName::Base,
|
||||||
|
BeaconBlockBodyRef::Altair { .. } => ForkName::Altair,
|
||||||
|
BeaconBlockBodyRef::Merge { .. } => ForkName::Merge,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -172,7 +172,7 @@ impl From<BeaconStateHash> for Hash256 {
|
|||||||
|
|
||||||
/// The state of the `BeaconChain` at some slot.
|
/// The state of the `BeaconChain` at some slot.
|
||||||
#[superstruct(
|
#[superstruct(
|
||||||
variants(Base, Altair),
|
variants(Base, Altair, Merge),
|
||||||
variant_attributes(
|
variant_attributes(
|
||||||
derive(
|
derive(
|
||||||
Derivative,
|
Derivative,
|
||||||
@ -250,9 +250,9 @@ where
|
|||||||
pub current_epoch_attestations: VariableList<PendingAttestation<T>, T::MaxPendingAttestations>,
|
pub current_epoch_attestations: VariableList<PendingAttestation<T>, T::MaxPendingAttestations>,
|
||||||
|
|
||||||
// Participation (Altair and later)
|
// Participation (Altair and later)
|
||||||
#[superstruct(only(Altair))]
|
#[superstruct(only(Altair, Merge))]
|
||||||
pub previous_epoch_participation: VariableList<ParticipationFlags, T::ValidatorRegistryLimit>,
|
pub previous_epoch_participation: VariableList<ParticipationFlags, T::ValidatorRegistryLimit>,
|
||||||
#[superstruct(only(Altair))]
|
#[superstruct(only(Altair, Merge))]
|
||||||
pub current_epoch_participation: VariableList<ParticipationFlags, T::ValidatorRegistryLimit>,
|
pub current_epoch_participation: VariableList<ParticipationFlags, T::ValidatorRegistryLimit>,
|
||||||
|
|
||||||
// Finality
|
// Finality
|
||||||
@ -267,15 +267,19 @@ where
|
|||||||
|
|
||||||
// Inactivity
|
// Inactivity
|
||||||
#[serde(with = "ssz_types::serde_utils::quoted_u64_var_list")]
|
#[serde(with = "ssz_types::serde_utils::quoted_u64_var_list")]
|
||||||
#[superstruct(only(Altair))]
|
#[superstruct(only(Altair, Merge))]
|
||||||
pub inactivity_scores: VariableList<u64, T::ValidatorRegistryLimit>,
|
pub inactivity_scores: VariableList<u64, T::ValidatorRegistryLimit>,
|
||||||
|
|
||||||
// Light-client sync committees
|
// Light-client sync committees
|
||||||
#[superstruct(only(Altair))]
|
#[superstruct(only(Altair, Merge))]
|
||||||
pub current_sync_committee: Arc<SyncCommittee<T>>,
|
pub current_sync_committee: Arc<SyncCommittee<T>>,
|
||||||
#[superstruct(only(Altair))]
|
#[superstruct(only(Altair, Merge))]
|
||||||
pub next_sync_committee: Arc<SyncCommittee<T>>,
|
pub next_sync_committee: Arc<SyncCommittee<T>>,
|
||||||
|
|
||||||
|
// Execution
|
||||||
|
#[superstruct(only(Merge))]
|
||||||
|
pub latest_execution_payload_header: ExecutionPayloadHeader<T>,
|
||||||
|
|
||||||
// Caching (not in the spec)
|
// Caching (not in the spec)
|
||||||
#[serde(skip_serializing, skip_deserializing)]
|
#[serde(skip_serializing, skip_deserializing)]
|
||||||
#[ssz(skip_serializing, skip_deserializing)]
|
#[ssz(skip_serializing, skip_deserializing)]
|
||||||
@ -384,6 +388,7 @@ impl<T: EthSpec> BeaconState<T> {
|
|||||||
let object_fork = match self {
|
let object_fork = match self {
|
||||||
BeaconState::Base { .. } => ForkName::Base,
|
BeaconState::Base { .. } => ForkName::Base,
|
||||||
BeaconState::Altair { .. } => ForkName::Altair,
|
BeaconState::Altair { .. } => ForkName::Altair,
|
||||||
|
BeaconState::Merge { .. } => ForkName::Merge,
|
||||||
};
|
};
|
||||||
|
|
||||||
if fork_at_slot == object_fork {
|
if fork_at_slot == object_fork {
|
||||||
@ -1089,6 +1094,7 @@ impl<T: EthSpec> BeaconState<T> {
|
|||||||
match self {
|
match self {
|
||||||
BeaconState::Base(state) => (&mut state.validators, &mut state.balances),
|
BeaconState::Base(state) => (&mut state.validators, &mut state.balances),
|
||||||
BeaconState::Altair(state) => (&mut state.validators, &mut state.balances),
|
BeaconState::Altair(state) => (&mut state.validators, &mut state.balances),
|
||||||
|
BeaconState::Merge(state) => (&mut state.validators, &mut state.balances),
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -1284,11 +1290,13 @@ impl<T: EthSpec> BeaconState<T> {
|
|||||||
match self {
|
match self {
|
||||||
BeaconState::Base(_) => Err(BeaconStateError::IncorrectStateVariant),
|
BeaconState::Base(_) => Err(BeaconStateError::IncorrectStateVariant),
|
||||||
BeaconState::Altair(state) => Ok(&mut state.current_epoch_participation),
|
BeaconState::Altair(state) => Ok(&mut state.current_epoch_participation),
|
||||||
|
BeaconState::Merge(state) => Ok(&mut state.current_epoch_participation),
|
||||||
}
|
}
|
||||||
} else if epoch == self.previous_epoch() {
|
} else if epoch == self.previous_epoch() {
|
||||||
match self {
|
match self {
|
||||||
BeaconState::Base(_) => Err(BeaconStateError::IncorrectStateVariant),
|
BeaconState::Base(_) => Err(BeaconStateError::IncorrectStateVariant),
|
||||||
BeaconState::Altair(state) => Ok(&mut state.previous_epoch_participation),
|
BeaconState::Altair(state) => Ok(&mut state.previous_epoch_participation),
|
||||||
|
BeaconState::Merge(state) => Ok(&mut state.previous_epoch_participation),
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
Err(BeaconStateError::EpochOutOfBounds)
|
Err(BeaconStateError::EpochOutOfBounds)
|
||||||
@ -1572,6 +1580,7 @@ impl<T: EthSpec> BeaconState<T> {
|
|||||||
let mut res = match self {
|
let mut res = match self {
|
||||||
BeaconState::Base(inner) => BeaconState::Base(inner.clone()),
|
BeaconState::Base(inner) => BeaconState::Base(inner.clone()),
|
||||||
BeaconState::Altair(inner) => BeaconState::Altair(inner.clone()),
|
BeaconState::Altair(inner) => BeaconState::Altair(inner.clone()),
|
||||||
|
BeaconState::Merge(inner) => BeaconState::Merge(inner.clone()),
|
||||||
};
|
};
|
||||||
if config.committee_caches {
|
if config.committee_caches {
|
||||||
*res.committee_caches_mut() = self.committee_caches().clone();
|
*res.committee_caches_mut() = self.committee_caches().clone();
|
||||||
|
@ -127,6 +127,10 @@ pub struct ChainSpec {
|
|||||||
pub altair_fork_version: [u8; 4],
|
pub altair_fork_version: [u8; 4],
|
||||||
/// The Altair fork epoch is optional, with `None` representing "Altair never happens".
|
/// The Altair fork epoch is optional, with `None` representing "Altair never happens".
|
||||||
pub altair_fork_epoch: Option<Epoch>,
|
pub altair_fork_epoch: Option<Epoch>,
|
||||||
|
pub merge_fork_version: [u8; 4],
|
||||||
|
/// The Merge fork epoch is optional, with `None` representing "Merge never happens".
|
||||||
|
pub merge_fork_epoch: Option<Epoch>,
|
||||||
|
pub terminal_total_difficulty: Uint256,
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* Networking
|
* Networking
|
||||||
@ -156,7 +160,7 @@ impl ChainSpec {
|
|||||||
) -> EnrForkId {
|
) -> EnrForkId {
|
||||||
EnrForkId {
|
EnrForkId {
|
||||||
fork_digest: self.fork_digest::<T>(slot, genesis_validators_root),
|
fork_digest: self.fork_digest::<T>(slot, genesis_validators_root),
|
||||||
next_fork_version: self.next_fork_version(),
|
next_fork_version: self.next_fork_version::<T>(slot),
|
||||||
next_fork_epoch: self
|
next_fork_epoch: self
|
||||||
.next_fork_epoch::<T>(slot)
|
.next_fork_epoch::<T>(slot)
|
||||||
.map(|(_, e)| e)
|
.map(|(_, e)| e)
|
||||||
@ -178,10 +182,12 @@ impl ChainSpec {
|
|||||||
|
|
||||||
/// Returns the `next_fork_version`.
|
/// Returns the `next_fork_version`.
|
||||||
///
|
///
|
||||||
/// Since `next_fork_version = current_fork_version` if no future fork is planned,
|
/// `next_fork_version = current_fork_version` if no future fork is planned,
|
||||||
/// this function returns `altair_fork_version` until the next fork is planned.
|
pub fn next_fork_version<E: EthSpec>(&self, slot: Slot) -> [u8; 4] {
|
||||||
pub fn next_fork_version(&self) -> [u8; 4] {
|
match self.next_fork_epoch::<E>(slot) {
|
||||||
self.altair_fork_version
|
Some((fork, _)) => self.fork_version_for_name(fork),
|
||||||
|
None => self.fork_version_for_name(self.fork_name_at_slot::<E>(slot)),
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Returns the epoch of the next scheduled fork along with its corresponding `ForkName`.
|
/// Returns the epoch of the next scheduled fork along with its corresponding `ForkName`.
|
||||||
@ -201,9 +207,12 @@ impl ChainSpec {
|
|||||||
|
|
||||||
/// Returns the name of the fork which is active at `epoch`.
|
/// Returns the name of the fork which is active at `epoch`.
|
||||||
pub fn fork_name_at_epoch(&self, epoch: Epoch) -> ForkName {
|
pub fn fork_name_at_epoch(&self, epoch: Epoch) -> ForkName {
|
||||||
match self.altair_fork_epoch {
|
match self.merge_fork_epoch {
|
||||||
Some(fork_epoch) if epoch >= fork_epoch => ForkName::Altair,
|
Some(fork_epoch) if epoch >= fork_epoch => ForkName::Merge,
|
||||||
_ => ForkName::Base,
|
_ => match self.altair_fork_epoch {
|
||||||
|
Some(fork_epoch) if epoch >= fork_epoch => ForkName::Altair,
|
||||||
|
_ => ForkName::Base,
|
||||||
|
},
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -212,6 +221,7 @@ impl ChainSpec {
|
|||||||
match fork_name {
|
match fork_name {
|
||||||
ForkName::Base => self.genesis_fork_version,
|
ForkName::Base => self.genesis_fork_version,
|
||||||
ForkName::Altair => self.altair_fork_version,
|
ForkName::Altair => self.altair_fork_version,
|
||||||
|
ForkName::Merge => self.merge_fork_version,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -220,6 +230,7 @@ impl ChainSpec {
|
|||||||
match fork_name {
|
match fork_name {
|
||||||
ForkName::Base => Some(Epoch::new(0)),
|
ForkName::Base => Some(Epoch::new(0)),
|
||||||
ForkName::Altair => self.altair_fork_epoch,
|
ForkName::Altair => self.altair_fork_epoch,
|
||||||
|
ForkName::Merge => self.merge_fork_epoch,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -467,6 +478,9 @@ impl ChainSpec {
|
|||||||
domain_contribution_and_proof: 9,
|
domain_contribution_and_proof: 9,
|
||||||
altair_fork_version: [0x01, 0x00, 0x00, 0x00],
|
altair_fork_version: [0x01, 0x00, 0x00, 0x00],
|
||||||
altair_fork_epoch: Some(Epoch::new(74240)),
|
altair_fork_epoch: Some(Epoch::new(74240)),
|
||||||
|
merge_fork_version: [0x02, 0x00, 0x00, 0x00],
|
||||||
|
merge_fork_epoch: None,
|
||||||
|
terminal_total_difficulty: Uint256::MAX,
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* Network specific
|
* Network specific
|
||||||
|
@ -19,3 +19,13 @@ pub mod altair {
|
|||||||
|
|
||||||
pub const NUM_FLAG_INDICES: usize = 3;
|
pub const NUM_FLAG_INDICES: usize = 3;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
pub mod merge_testing {
|
||||||
|
use ethereum_types::H256;
|
||||||
|
pub const GENESIS_GAS_LIMIT: u64 = 30_000_000;
|
||||||
|
pub const GENESIS_BASE_FEE_PER_GAS: H256 = H256([
|
||||||
|
0x00, 0xca, 0x9a, 0x3b, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00,
|
||||||
|
0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00,
|
||||||
|
0x00, 0x00,
|
||||||
|
]);
|
||||||
|
}
|
||||||
|
@ -9,6 +9,9 @@ use ssz_types::typenum::{
|
|||||||
use std::fmt::{self, Debug};
|
use std::fmt::{self, Debug};
|
||||||
use std::str::FromStr;
|
use std::str::FromStr;
|
||||||
|
|
||||||
|
use ssz_types::typenum::{bit::B0, UInt, U1048576, U16384, U256, U625};
|
||||||
|
pub type U5000 = UInt<UInt<UInt<U625, B0>, B0>, B0>; // 625 * 8 = 5000
|
||||||
|
|
||||||
const MAINNET: &str = "mainnet";
|
const MAINNET: &str = "mainnet";
|
||||||
const MINIMAL: &str = "minimal";
|
const MINIMAL: &str = "minimal";
|
||||||
|
|
||||||
@ -80,6 +83,14 @@ pub trait EthSpec: 'static + Default + Sync + Send + Clone + Debug + PartialEq +
|
|||||||
type SyncCommitteeSize: Unsigned + Clone + Sync + Send + Debug + PartialEq;
|
type SyncCommitteeSize: Unsigned + Clone + Sync + Send + Debug + PartialEq;
|
||||||
/// The number of `sync_committee` subnets.
|
/// The number of `sync_committee` subnets.
|
||||||
type SyncCommitteeSubnetCount: Unsigned + Clone + Sync + Send + Debug + PartialEq;
|
type SyncCommitteeSubnetCount: Unsigned + Clone + Sync + Send + Debug + PartialEq;
|
||||||
|
/*
|
||||||
|
* New in Merge
|
||||||
|
*/
|
||||||
|
type MaxBytesPerOpaqueTransaction: Unsigned + Clone + Sync + Send + Debug + PartialEq;
|
||||||
|
type MaxTransactionsPerPayload: Unsigned + Clone + Sync + Send + Debug + PartialEq;
|
||||||
|
type BytesPerLogsBloom: Unsigned + Clone + Sync + Send + Debug + PartialEq;
|
||||||
|
type GasLimitDenominator: Unsigned + Clone + Sync + Send + Debug + PartialEq;
|
||||||
|
type MinGasLimit: Unsigned + Clone + Sync + Send + Debug + PartialEq;
|
||||||
/*
|
/*
|
||||||
* Derived values (set these CAREFULLY)
|
* Derived values (set these CAREFULLY)
|
||||||
*/
|
*/
|
||||||
@ -187,6 +198,31 @@ pub trait EthSpec: 'static + Default + Sync + Send + Clone + Debug + PartialEq +
|
|||||||
fn sync_subcommittee_size() -> usize {
|
fn sync_subcommittee_size() -> usize {
|
||||||
Self::SyncSubcommitteeSize::to_usize()
|
Self::SyncSubcommitteeSize::to_usize()
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/// Returns the `MAX_BYTES_PER_OPAQUE_TRANSACTION` constant for this specification.
|
||||||
|
fn max_bytes_per_opaque_transaction() -> usize {
|
||||||
|
Self::MaxBytesPerOpaqueTransaction::to_usize()
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Returns the `MAX_TRANSACTIONS_PER_PAYLOAD` constant for this specification.
|
||||||
|
fn max_transactions_per_payload() -> usize {
|
||||||
|
Self::MaxTransactionsPerPayload::to_usize()
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Returns the `BYTES_PER_LOGS_BLOOM` constant for this specification.
|
||||||
|
fn bytes_per_logs_bloom() -> usize {
|
||||||
|
Self::BytesPerLogsBloom::to_usize()
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Returns the `GAS_LIMIT_DENOMINATOR` constant for this specification.
|
||||||
|
fn gas_limit_denominator() -> u64 {
|
||||||
|
Self::GasLimitDenominator::to_u64()
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Returns the `MIN_GAS_LIMIT` constant for this specification.
|
||||||
|
fn min_gas_limit() -> u64 {
|
||||||
|
Self::MinGasLimit::to_u64()
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Macro to inherit some type values from another EthSpec.
|
/// Macro to inherit some type values from another EthSpec.
|
||||||
@ -221,6 +257,11 @@ impl EthSpec for MainnetEthSpec {
|
|||||||
type MaxVoluntaryExits = U16;
|
type MaxVoluntaryExits = U16;
|
||||||
type SyncCommitteeSize = U512;
|
type SyncCommitteeSize = U512;
|
||||||
type SyncCommitteeSubnetCount = U4;
|
type SyncCommitteeSubnetCount = U4;
|
||||||
|
type MaxBytesPerOpaqueTransaction = U1048576;
|
||||||
|
type MaxTransactionsPerPayload = U16384;
|
||||||
|
type BytesPerLogsBloom = U256;
|
||||||
|
type GasLimitDenominator = U1024;
|
||||||
|
type MinGasLimit = U5000;
|
||||||
type SyncSubcommitteeSize = U128; // 512 committee size / 4 sync committee subnet count
|
type SyncSubcommitteeSize = U128; // 512 committee size / 4 sync committee subnet count
|
||||||
type MaxPendingAttestations = U4096; // 128 max attestations * 32 slots per epoch
|
type MaxPendingAttestations = U4096; // 128 max attestations * 32 slots per epoch
|
||||||
type SlotsPerEth1VotingPeriod = U2048; // 64 epochs * 32 slots per epoch
|
type SlotsPerEth1VotingPeriod = U2048; // 64 epochs * 32 slots per epoch
|
||||||
@ -262,7 +303,12 @@ impl EthSpec for MinimalEthSpec {
|
|||||||
MaxAttesterSlashings,
|
MaxAttesterSlashings,
|
||||||
MaxAttestations,
|
MaxAttestations,
|
||||||
MaxDeposits,
|
MaxDeposits,
|
||||||
MaxVoluntaryExits
|
MaxVoluntaryExits,
|
||||||
|
MaxBytesPerOpaqueTransaction,
|
||||||
|
MaxTransactionsPerPayload,
|
||||||
|
BytesPerLogsBloom,
|
||||||
|
GasLimitDenominator,
|
||||||
|
MinGasLimit
|
||||||
});
|
});
|
||||||
|
|
||||||
fn default_spec() -> ChainSpec {
|
fn default_spec() -> ChainSpec {
|
||||||
|
174
consensus/types/src/execution_payload.rs
Normal file
174
consensus/types/src/execution_payload.rs
Normal file
@ -0,0 +1,174 @@
|
|||||||
|
use crate::{test_utils::TestRandom, *};
|
||||||
|
use serde_derive::{Deserialize, Serialize};
|
||||||
|
use ssz_derive::{Decode, Encode};
|
||||||
|
use std::{ops::Index, slice::SliceIndex};
|
||||||
|
use test_random_derive::TestRandom;
|
||||||
|
use tree_hash_derive::TreeHash;
|
||||||
|
|
||||||
|
#[derive(Debug, Clone, PartialEq, Serialize, Deserialize, Encode, Decode, TreeHash)]
|
||||||
|
#[ssz(enum_behaviour = "union")]
|
||||||
|
#[tree_hash(enum_behaviour = "union")]
|
||||||
|
pub enum Transaction<T: EthSpec> {
|
||||||
|
OpaqueTransaction(VariableList<u8, T::MaxBytesPerOpaqueTransaction>),
|
||||||
|
}
|
||||||
|
|
||||||
|
impl<T: EthSpec, I: SliceIndex<[u8]>> Index<I> for Transaction<T> {
|
||||||
|
type Output = I::Output;
|
||||||
|
|
||||||
|
#[inline]
|
||||||
|
fn index(&self, index: I) -> &Self::Output {
|
||||||
|
match self {
|
||||||
|
Self::OpaqueTransaction(v) => Index::index(v, index),
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
impl<T: EthSpec> From<VariableList<u8, T::MaxBytesPerOpaqueTransaction>> for Transaction<T> {
|
||||||
|
fn from(list: VariableList<u8, <T as EthSpec>::MaxBytesPerOpaqueTransaction>) -> Self {
|
||||||
|
Self::OpaqueTransaction(list)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
#[cfg_attr(feature = "arbitrary-fuzz", derive(arbitrary::Arbitrary))]
|
||||||
|
#[derive(
|
||||||
|
Default, Debug, Clone, PartialEq, Serialize, Deserialize, Encode, Decode, TreeHash, TestRandom,
|
||||||
|
)]
|
||||||
|
pub struct ExecutionPayload<T: EthSpec> {
|
||||||
|
pub parent_hash: Hash256,
|
||||||
|
pub coinbase: Address,
|
||||||
|
pub state_root: Hash256,
|
||||||
|
pub receipt_root: Hash256,
|
||||||
|
#[serde(with = "serde_logs_bloom")]
|
||||||
|
pub logs_bloom: FixedVector<u8, T::BytesPerLogsBloom>,
|
||||||
|
pub random: Hash256,
|
||||||
|
#[serde(with = "eth2_serde_utils::quoted_u64")]
|
||||||
|
pub block_number: u64,
|
||||||
|
#[serde(with = "eth2_serde_utils::quoted_u64")]
|
||||||
|
pub gas_limit: u64,
|
||||||
|
#[serde(with = "eth2_serde_utils::quoted_u64")]
|
||||||
|
pub gas_used: u64,
|
||||||
|
#[serde(with = "eth2_serde_utils::quoted_u64")]
|
||||||
|
pub timestamp: u64,
|
||||||
|
pub base_fee_per_gas: Hash256,
|
||||||
|
pub block_hash: Hash256,
|
||||||
|
#[serde(with = "serde_transactions")]
|
||||||
|
#[test_random(default)]
|
||||||
|
pub transactions: VariableList<Transaction<T>, T::MaxTransactionsPerPayload>,
|
||||||
|
}
|
||||||
|
|
||||||
|
impl<T: EthSpec> ExecutionPayload<T> {
|
||||||
|
// TODO: check this whole thing later
|
||||||
|
pub fn empty() -> Self {
|
||||||
|
Self {
|
||||||
|
parent_hash: Hash256::zero(),
|
||||||
|
coinbase: Address::default(),
|
||||||
|
state_root: Hash256::zero(),
|
||||||
|
receipt_root: Hash256::zero(),
|
||||||
|
logs_bloom: FixedVector::default(),
|
||||||
|
random: Hash256::zero(),
|
||||||
|
block_number: 0,
|
||||||
|
gas_limit: 0,
|
||||||
|
gas_used: 0,
|
||||||
|
timestamp: 0,
|
||||||
|
base_fee_per_gas: Hash256::zero(),
|
||||||
|
block_hash: Hash256::zero(),
|
||||||
|
transactions: VariableList::empty(),
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Serializes the `logs_bloom` field.
|
||||||
|
pub mod serde_logs_bloom {
|
||||||
|
use super::*;
|
||||||
|
use eth2_serde_utils::hex::PrefixedHexVisitor;
|
||||||
|
use serde::{Deserializer, Serializer};
|
||||||
|
|
||||||
|
pub fn serialize<S, U>(bytes: &FixedVector<u8, U>, serializer: S) -> Result<S::Ok, S::Error>
|
||||||
|
where
|
||||||
|
S: Serializer,
|
||||||
|
U: Unsigned,
|
||||||
|
{
|
||||||
|
let mut hex_string: String = "0x".to_string();
|
||||||
|
hex_string.push_str(&hex::encode(&bytes[..]));
|
||||||
|
|
||||||
|
serializer.serialize_str(&hex_string)
|
||||||
|
}
|
||||||
|
|
||||||
|
pub fn deserialize<'de, D, U>(deserializer: D) -> Result<FixedVector<u8, U>, D::Error>
|
||||||
|
where
|
||||||
|
D: Deserializer<'de>,
|
||||||
|
U: Unsigned,
|
||||||
|
{
|
||||||
|
let vec = deserializer.deserialize_string(PrefixedHexVisitor)?;
|
||||||
|
|
||||||
|
FixedVector::new(vec)
|
||||||
|
.map_err(|e| serde::de::Error::custom(format!("invalid logs bloom: {:?}", e)))
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Serializes the `transactions` field.
|
||||||
|
pub mod serde_transactions {
|
||||||
|
use super::*;
|
||||||
|
use eth2_serde_utils::hex;
|
||||||
|
use serde::ser::SerializeSeq;
|
||||||
|
use serde::{de, Deserializer, Serializer};
|
||||||
|
use std::marker::PhantomData;
|
||||||
|
|
||||||
|
pub struct ListOfBytesListVisitor<T: EthSpec> {
|
||||||
|
_t: PhantomData<T>,
|
||||||
|
}
|
||||||
|
impl<'a, T> serde::de::Visitor<'a> for ListOfBytesListVisitor<T>
|
||||||
|
where
|
||||||
|
T: EthSpec,
|
||||||
|
{
|
||||||
|
type Value = VariableList<Transaction<T>, T::MaxTransactionsPerPayload>;
|
||||||
|
|
||||||
|
fn expecting(&self, formatter: &mut std::fmt::Formatter) -> std::fmt::Result {
|
||||||
|
write!(formatter, "a list of 0x-prefixed byte lists")
|
||||||
|
}
|
||||||
|
|
||||||
|
fn visit_seq<A>(self, mut seq: A) -> Result<Self::Value, A::Error>
|
||||||
|
where
|
||||||
|
A: serde::de::SeqAccess<'a>,
|
||||||
|
{
|
||||||
|
let mut outer = VariableList::default();
|
||||||
|
|
||||||
|
while let Some(val) = seq.next_element::<String>()? {
|
||||||
|
let inner_vec = hex::decode(&val).map_err(de::Error::custom)?;
|
||||||
|
let inner = VariableList::new(inner_vec).map_err(|e| {
|
||||||
|
serde::de::Error::custom(format!("invalid transaction: {:?}", e))
|
||||||
|
})?;
|
||||||
|
outer.push(inner.into()).map_err(|e| {
|
||||||
|
serde::de::Error::custom(format!("too many transactions: {:?}", e))
|
||||||
|
})?;
|
||||||
|
}
|
||||||
|
|
||||||
|
Ok(outer)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
pub fn serialize<S, T>(
|
||||||
|
value: &VariableList<Transaction<T>, T::MaxTransactionsPerPayload>,
|
||||||
|
serializer: S,
|
||||||
|
) -> Result<S::Ok, S::Error>
|
||||||
|
where
|
||||||
|
S: Serializer,
|
||||||
|
T: EthSpec,
|
||||||
|
{
|
||||||
|
let mut seq = serializer.serialize_seq(Some(value.len()))?;
|
||||||
|
for val in value {
|
||||||
|
seq.serialize_element(&hex::encode(&val[..]))?;
|
||||||
|
}
|
||||||
|
seq.end()
|
||||||
|
}
|
||||||
|
|
||||||
|
pub fn deserialize<'de, D, T>(
|
||||||
|
deserializer: D,
|
||||||
|
) -> Result<VariableList<Transaction<T>, T::MaxTransactionsPerPayload>, D::Error>
|
||||||
|
where
|
||||||
|
D: Deserializer<'de>,
|
||||||
|
T: EthSpec,
|
||||||
|
{
|
||||||
|
deserializer.deserialize_any(ListOfBytesListVisitor { _t: PhantomData })
|
||||||
|
}
|
||||||
|
}
|
37
consensus/types/src/execution_payload_header.rs
Normal file
37
consensus/types/src/execution_payload_header.rs
Normal file
@ -0,0 +1,37 @@
|
|||||||
|
use crate::{execution_payload::serde_logs_bloom, test_utils::TestRandom, *};
|
||||||
|
use serde_derive::{Deserialize, Serialize};
|
||||||
|
use ssz_derive::{Decode, Encode};
|
||||||
|
use test_random_derive::TestRandom;
|
||||||
|
use tree_hash_derive::TreeHash;
|
||||||
|
|
||||||
|
#[cfg_attr(feature = "arbitrary-fuzz", derive(arbitrary::Arbitrary))]
|
||||||
|
#[derive(
|
||||||
|
Default, Debug, Clone, PartialEq, Serialize, Deserialize, Encode, Decode, TreeHash, TestRandom,
|
||||||
|
)]
|
||||||
|
pub struct ExecutionPayloadHeader<T: EthSpec> {
|
||||||
|
pub parent_hash: Hash256,
|
||||||
|
pub coinbase: Address,
|
||||||
|
pub state_root: Hash256,
|
||||||
|
pub receipt_root: Hash256,
|
||||||
|
#[serde(with = "serde_logs_bloom")]
|
||||||
|
pub logs_bloom: FixedVector<u8, T::BytesPerLogsBloom>,
|
||||||
|
pub random: Hash256,
|
||||||
|
#[serde(with = "eth2_serde_utils::quoted_u64")]
|
||||||
|
pub block_number: u64,
|
||||||
|
#[serde(with = "eth2_serde_utils::quoted_u64")]
|
||||||
|
pub gas_limit: u64,
|
||||||
|
#[serde(with = "eth2_serde_utils::quoted_u64")]
|
||||||
|
pub gas_used: u64,
|
||||||
|
#[serde(with = "eth2_serde_utils::quoted_u64")]
|
||||||
|
pub timestamp: u64,
|
||||||
|
pub base_fee_per_gas: Hash256,
|
||||||
|
pub block_hash: Hash256,
|
||||||
|
pub transactions_root: Hash256,
|
||||||
|
}
|
||||||
|
|
||||||
|
impl<T: EthSpec> ExecutionPayloadHeader<T> {
|
||||||
|
// TODO: check this whole thing later
|
||||||
|
pub fn empty() -> Self {
|
||||||
|
Self::default()
|
||||||
|
}
|
||||||
|
}
|
@ -35,6 +35,15 @@ impl ForkContext {
|
|||||||
));
|
));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// Only add Merge to list of forks if it's enabled
|
||||||
|
// Note: `merge_fork_epoch == None` implies merge hasn't been activated yet on the config.
|
||||||
|
if spec.merge_fork_epoch.is_some() {
|
||||||
|
fork_to_digest.push((
|
||||||
|
ForkName::Merge,
|
||||||
|
ChainSpec::compute_fork_digest(spec.merge_fork_version, genesis_validators_root),
|
||||||
|
));
|
||||||
|
}
|
||||||
|
|
||||||
let fork_to_digest: HashMap<ForkName, [u8; 4]> = fork_to_digest.into_iter().collect();
|
let fork_to_digest: HashMap<ForkName, [u8; 4]> = fork_to_digest.into_iter().collect();
|
||||||
|
|
||||||
let digest_to_fork = fork_to_digest
|
let digest_to_fork = fork_to_digest
|
||||||
|
@ -10,6 +10,7 @@ use std::str::FromStr;
|
|||||||
pub enum ForkName {
|
pub enum ForkName {
|
||||||
Base,
|
Base,
|
||||||
Altair,
|
Altair,
|
||||||
|
Merge,
|
||||||
}
|
}
|
||||||
|
|
||||||
impl ForkName {
|
impl ForkName {
|
||||||
@ -24,10 +25,17 @@ impl ForkName {
|
|||||||
match self {
|
match self {
|
||||||
ForkName::Base => {
|
ForkName::Base => {
|
||||||
spec.altair_fork_epoch = None;
|
spec.altair_fork_epoch = None;
|
||||||
|
spec.merge_fork_epoch = None;
|
||||||
spec
|
spec
|
||||||
}
|
}
|
||||||
ForkName::Altair => {
|
ForkName::Altair => {
|
||||||
spec.altair_fork_epoch = Some(Epoch::new(0));
|
spec.altair_fork_epoch = Some(Epoch::new(0));
|
||||||
|
spec.merge_fork_epoch = None;
|
||||||
|
spec
|
||||||
|
}
|
||||||
|
ForkName::Merge => {
|
||||||
|
spec.altair_fork_epoch = None;
|
||||||
|
spec.merge_fork_epoch = Some(Epoch::new(0));
|
||||||
spec
|
spec
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -40,6 +48,7 @@ impl ForkName {
|
|||||||
match self {
|
match self {
|
||||||
ForkName::Base => None,
|
ForkName::Base => None,
|
||||||
ForkName::Altair => Some(ForkName::Base),
|
ForkName::Altair => Some(ForkName::Base),
|
||||||
|
ForkName::Merge => Some(ForkName::Altair),
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -49,7 +58,8 @@ impl ForkName {
|
|||||||
pub fn next_fork(self) -> Option<ForkName> {
|
pub fn next_fork(self) -> Option<ForkName> {
|
||||||
match self {
|
match self {
|
||||||
ForkName::Base => Some(ForkName::Altair),
|
ForkName::Base => Some(ForkName::Altair),
|
||||||
ForkName::Altair => None,
|
ForkName::Altair => Some(ForkName::Merge),
|
||||||
|
ForkName::Merge => None,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -98,6 +108,7 @@ impl FromStr for ForkName {
|
|||||||
Ok(match fork_name.to_lowercase().as_ref() {
|
Ok(match fork_name.to_lowercase().as_ref() {
|
||||||
"phase0" | "base" => ForkName::Base,
|
"phase0" | "base" => ForkName::Base,
|
||||||
"altair" => ForkName::Altair,
|
"altair" => ForkName::Altair,
|
||||||
|
"merge" => ForkName::Merge,
|
||||||
_ => return Err(()),
|
_ => return Err(()),
|
||||||
})
|
})
|
||||||
}
|
}
|
||||||
@ -108,6 +119,7 @@ impl Display for ForkName {
|
|||||||
match self {
|
match self {
|
||||||
ForkName::Base => "phase0".fmt(f),
|
ForkName::Base => "phase0".fmt(f),
|
||||||
ForkName::Altair => "altair".fmt(f),
|
ForkName::Altair => "altair".fmt(f),
|
||||||
|
ForkName::Merge => "merge".fmt(f),
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -139,7 +151,7 @@ mod test {
|
|||||||
|
|
||||||
#[test]
|
#[test]
|
||||||
fn previous_and_next_fork_consistent() {
|
fn previous_and_next_fork_consistent() {
|
||||||
assert_eq!(ForkName::Altair.next_fork(), None);
|
assert_eq!(ForkName::Merge.next_fork(), None);
|
||||||
assert_eq!(ForkName::Base.previous_fork(), None);
|
assert_eq!(ForkName::Base.previous_fork(), None);
|
||||||
|
|
||||||
for (prev_fork, fork) in ForkName::list_all().into_iter().tuple_windows() {
|
for (prev_fork, fork) in ForkName::list_all().into_iter().tuple_windows() {
|
||||||
|
@ -37,6 +37,8 @@ pub mod deposit_message;
|
|||||||
pub mod enr_fork_id;
|
pub mod enr_fork_id;
|
||||||
pub mod eth1_data;
|
pub mod eth1_data;
|
||||||
pub mod eth_spec;
|
pub mod eth_spec;
|
||||||
|
pub mod execution_payload;
|
||||||
|
pub mod execution_payload_header;
|
||||||
pub mod fork;
|
pub mod fork;
|
||||||
pub mod fork_data;
|
pub mod fork_data;
|
||||||
pub mod fork_name;
|
pub mod fork_name;
|
||||||
@ -45,6 +47,7 @@ pub mod graffiti;
|
|||||||
pub mod historical_batch;
|
pub mod historical_batch;
|
||||||
pub mod indexed_attestation;
|
pub mod indexed_attestation;
|
||||||
pub mod pending_attestation;
|
pub mod pending_attestation;
|
||||||
|
pub mod pow_block;
|
||||||
pub mod proposer_slashing;
|
pub mod proposer_slashing;
|
||||||
pub mod relative_epoch;
|
pub mod relative_epoch;
|
||||||
pub mod selection_proof;
|
pub mod selection_proof;
|
||||||
@ -90,11 +93,12 @@ pub use crate::attestation_data::AttestationData;
|
|||||||
pub use crate::attestation_duty::AttestationDuty;
|
pub use crate::attestation_duty::AttestationDuty;
|
||||||
pub use crate::attester_slashing::AttesterSlashing;
|
pub use crate::attester_slashing::AttesterSlashing;
|
||||||
pub use crate::beacon_block::{
|
pub use crate::beacon_block::{
|
||||||
BeaconBlock, BeaconBlockAltair, BeaconBlockBase, BeaconBlockRef, BeaconBlockRefMut,
|
BeaconBlock, BeaconBlockAltair, BeaconBlockBase, BeaconBlockMerge, BeaconBlockRef,
|
||||||
|
BeaconBlockRefMut,
|
||||||
};
|
};
|
||||||
pub use crate::beacon_block_body::{
|
pub use crate::beacon_block_body::{
|
||||||
BeaconBlockBody, BeaconBlockBodyAltair, BeaconBlockBodyBase, BeaconBlockBodyRef,
|
BeaconBlockBody, BeaconBlockBodyAltair, BeaconBlockBodyBase, BeaconBlockBodyMerge,
|
||||||
BeaconBlockBodyRefMut,
|
BeaconBlockBodyRef, BeaconBlockBodyRefMut,
|
||||||
};
|
};
|
||||||
pub use crate::beacon_block_header::BeaconBlockHeader;
|
pub use crate::beacon_block_header::BeaconBlockHeader;
|
||||||
pub use crate::beacon_committee::{BeaconCommittee, OwnedBeaconCommittee};
|
pub use crate::beacon_committee::{BeaconCommittee, OwnedBeaconCommittee};
|
||||||
@ -109,6 +113,8 @@ pub use crate::deposit_message::DepositMessage;
|
|||||||
pub use crate::enr_fork_id::EnrForkId;
|
pub use crate::enr_fork_id::EnrForkId;
|
||||||
pub use crate::eth1_data::Eth1Data;
|
pub use crate::eth1_data::Eth1Data;
|
||||||
pub use crate::eth_spec::EthSpecId;
|
pub use crate::eth_spec::EthSpecId;
|
||||||
|
pub use crate::execution_payload::ExecutionPayload;
|
||||||
|
pub use crate::execution_payload_header::ExecutionPayloadHeader;
|
||||||
pub use crate::fork::Fork;
|
pub use crate::fork::Fork;
|
||||||
pub use crate::fork_context::ForkContext;
|
pub use crate::fork_context::ForkContext;
|
||||||
pub use crate::fork_data::ForkData;
|
pub use crate::fork_data::ForkData;
|
||||||
@ -120,6 +126,7 @@ pub use crate::indexed_attestation::IndexedAttestation;
|
|||||||
pub use crate::participation_flags::ParticipationFlags;
|
pub use crate::participation_flags::ParticipationFlags;
|
||||||
pub use crate::participation_list::ParticipationList;
|
pub use crate::participation_list::ParticipationList;
|
||||||
pub use crate::pending_attestation::PendingAttestation;
|
pub use crate::pending_attestation::PendingAttestation;
|
||||||
|
pub use crate::pow_block::PowBlock;
|
||||||
pub use crate::preset::{AltairPreset, BasePreset};
|
pub use crate::preset::{AltairPreset, BasePreset};
|
||||||
pub use crate::proposer_slashing::ProposerSlashing;
|
pub use crate::proposer_slashing::ProposerSlashing;
|
||||||
pub use crate::relative_epoch::{Error as RelativeEpochError, RelativeEpoch};
|
pub use crate::relative_epoch::{Error as RelativeEpochError, RelativeEpoch};
|
||||||
@ -128,6 +135,7 @@ pub use crate::shuffling_id::AttestationShufflingId;
|
|||||||
pub use crate::signed_aggregate_and_proof::SignedAggregateAndProof;
|
pub use crate::signed_aggregate_and_proof::SignedAggregateAndProof;
|
||||||
pub use crate::signed_beacon_block::{
|
pub use crate::signed_beacon_block::{
|
||||||
SignedBeaconBlock, SignedBeaconBlockAltair, SignedBeaconBlockBase, SignedBeaconBlockHash,
|
SignedBeaconBlock, SignedBeaconBlockAltair, SignedBeaconBlockBase, SignedBeaconBlockHash,
|
||||||
|
SignedBeaconBlockMerge,
|
||||||
};
|
};
|
||||||
pub use crate::signed_beacon_block_header::SignedBeaconBlockHeader;
|
pub use crate::signed_beacon_block_header::SignedBeaconBlockHeader;
|
||||||
pub use crate::signed_contribution_and_proof::SignedContributionAndProof;
|
pub use crate::signed_contribution_and_proof::SignedContributionAndProof;
|
||||||
@ -150,6 +158,7 @@ pub use crate::voluntary_exit::VoluntaryExit;
|
|||||||
|
|
||||||
pub type CommitteeIndex = u64;
|
pub type CommitteeIndex = u64;
|
||||||
pub type Hash256 = H256;
|
pub type Hash256 = H256;
|
||||||
|
pub type Uint256 = ethereum_types::U256;
|
||||||
pub type Address = H160;
|
pub type Address = H160;
|
||||||
pub type ForkVersion = [u8; 4];
|
pub type ForkVersion = [u8; 4];
|
||||||
|
|
||||||
|
13
consensus/types/src/pow_block.rs
Normal file
13
consensus/types/src/pow_block.rs
Normal file
@ -0,0 +1,13 @@
|
|||||||
|
use crate::*;
|
||||||
|
|
||||||
|
#[cfg_attr(feature = "arbitrary-fuzz", derive(arbitrary::Arbitrary))]
|
||||||
|
#[derive(Default, Debug, PartialEq, Clone)]
|
||||||
|
pub struct PowBlock {
|
||||||
|
pub block_hash: Hash256,
|
||||||
|
pub parent_hash: Hash256,
|
||||||
|
pub total_difficulty: Uint256,
|
||||||
|
pub difficulty: Uint256,
|
||||||
|
// needed to unify with other parts of codebase
|
||||||
|
pub timestamp: u64,
|
||||||
|
pub block_number: u64,
|
||||||
|
}
|
@ -37,7 +37,7 @@ impl From<SignedBeaconBlockHash> for Hash256 {
|
|||||||
|
|
||||||
/// A `BeaconBlock` and a signature from its proposer.
|
/// A `BeaconBlock` and a signature from its proposer.
|
||||||
#[superstruct(
|
#[superstruct(
|
||||||
variants(Base, Altair),
|
variants(Base, Altair, Merge),
|
||||||
variant_attributes(
|
variant_attributes(
|
||||||
derive(
|
derive(
|
||||||
Debug,
|
Debug,
|
||||||
@ -64,6 +64,8 @@ pub struct SignedBeaconBlock<E: EthSpec> {
|
|||||||
pub message: BeaconBlockBase<E>,
|
pub message: BeaconBlockBase<E>,
|
||||||
#[superstruct(only(Altair), partial_getter(rename = "message_altair"))]
|
#[superstruct(only(Altair), partial_getter(rename = "message_altair"))]
|
||||||
pub message: BeaconBlockAltair<E>,
|
pub message: BeaconBlockAltair<E>,
|
||||||
|
#[superstruct(only(Merge), partial_getter(rename = "message_merge"))]
|
||||||
|
pub message: BeaconBlockMerge<E>,
|
||||||
pub signature: Signature,
|
pub signature: Signature,
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -116,6 +118,9 @@ impl<E: EthSpec> SignedBeaconBlock<E> {
|
|||||||
BeaconBlock::Altair(message) => {
|
BeaconBlock::Altair(message) => {
|
||||||
SignedBeaconBlock::Altair(SignedBeaconBlockAltair { message, signature })
|
SignedBeaconBlock::Altair(SignedBeaconBlockAltair { message, signature })
|
||||||
}
|
}
|
||||||
|
BeaconBlock::Merge(message) => {
|
||||||
|
SignedBeaconBlock::Merge(SignedBeaconBlockMerge { message, signature })
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -129,6 +134,7 @@ impl<E: EthSpec> SignedBeaconBlock<E> {
|
|||||||
SignedBeaconBlock::Altair(block) => {
|
SignedBeaconBlock::Altair(block) => {
|
||||||
(BeaconBlock::Altair(block.message), block.signature)
|
(BeaconBlock::Altair(block.message), block.signature)
|
||||||
}
|
}
|
||||||
|
SignedBeaconBlock::Merge(block) => (BeaconBlock::Merge(block.message), block.signature),
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -137,6 +143,7 @@ impl<E: EthSpec> SignedBeaconBlock<E> {
|
|||||||
match self {
|
match self {
|
||||||
SignedBeaconBlock::Base(inner) => BeaconBlockRef::Base(&inner.message),
|
SignedBeaconBlock::Base(inner) => BeaconBlockRef::Base(&inner.message),
|
||||||
SignedBeaconBlock::Altair(inner) => BeaconBlockRef::Altair(&inner.message),
|
SignedBeaconBlock::Altair(inner) => BeaconBlockRef::Altair(&inner.message),
|
||||||
|
SignedBeaconBlock::Merge(inner) => BeaconBlockRef::Merge(&inner.message),
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -145,6 +152,7 @@ impl<E: EthSpec> SignedBeaconBlock<E> {
|
|||||||
match self {
|
match self {
|
||||||
SignedBeaconBlock::Base(inner) => BeaconBlockRefMut::Base(&mut inner.message),
|
SignedBeaconBlock::Base(inner) => BeaconBlockRefMut::Base(&mut inner.message),
|
||||||
SignedBeaconBlock::Altair(inner) => BeaconBlockRefMut::Altair(&mut inner.message),
|
SignedBeaconBlock::Altair(inner) => BeaconBlockRefMut::Altair(&mut inner.message),
|
||||||
|
SignedBeaconBlock::Merge(inner) => BeaconBlockRefMut::Merge(&mut inner.message),
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -14,6 +14,7 @@ mod public_key_bytes;
|
|||||||
mod secret_key;
|
mod secret_key;
|
||||||
mod signature;
|
mod signature;
|
||||||
mod signature_bytes;
|
mod signature_bytes;
|
||||||
|
mod uint256;
|
||||||
|
|
||||||
pub fn test_random_instance<T: TestRandom>() -> T {
|
pub fn test_random_instance<T: TestRandom>() -> T {
|
||||||
let mut rng = XorShiftRng::from_seed([0x42; 16]);
|
let mut rng = XorShiftRng::from_seed([0x42; 16]);
|
||||||
|
@ -1,10 +1,10 @@
|
|||||||
use super::*;
|
use super::*;
|
||||||
use crate::Hash256;
|
use crate::Uint256;
|
||||||
|
|
||||||
impl TestRandom for Hash256 {
|
impl TestRandom for Uint256 {
|
||||||
fn random_for_test(rng: &mut impl RngCore) -> Self {
|
fn random_for_test(rng: &mut impl RngCore) -> Self {
|
||||||
let mut key_bytes = vec![0; 32];
|
let mut key_bytes = [0; 32];
|
||||||
rng.fill_bytes(&mut key_bytes);
|
rng.fill_bytes(&mut key_bytes);
|
||||||
Hash256::from_slice(&key_bytes[..])
|
Self::from_little_endian(&key_bytes[..])
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
10
consensus/types/src/test_utils/test_random/uint256.rs
Normal file
10
consensus/types/src/test_utils/test_random/uint256.rs
Normal file
@ -0,0 +1,10 @@
|
|||||||
|
use super::*;
|
||||||
|
use crate::Hash256;
|
||||||
|
|
||||||
|
impl TestRandom for Hash256 {
|
||||||
|
fn random_for_test(rng: &mut impl RngCore) -> Self {
|
||||||
|
let mut key_bytes = vec![0; 32];
|
||||||
|
rng.fill_bytes(&mut key_bytes);
|
||||||
|
Hash256::from_slice(&key_bytes[..])
|
||||||
|
}
|
||||||
|
}
|
@ -77,5 +77,6 @@ pub fn previous_fork(fork_name: ForkName) -> ForkName {
|
|||||||
match fork_name {
|
match fork_name {
|
||||||
ForkName::Base => ForkName::Base,
|
ForkName::Base => ForkName::Base,
|
||||||
ForkName::Altair => ForkName::Base,
|
ForkName::Altair => ForkName::Base,
|
||||||
|
ForkName::Merge => ForkName::Altair, // TODO: Check this when tests are released..
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -94,10 +94,12 @@ impl<E: EthSpec> EpochTransition<E> for JustificationAndFinalization {
|
|||||||
spec,
|
spec,
|
||||||
)
|
)
|
||||||
}
|
}
|
||||||
BeaconState::Altair(_) => altair::process_justification_and_finalization(
|
BeaconState::Altair(_) | BeaconState::Merge(_) => {
|
||||||
state,
|
altair::process_justification_and_finalization(
|
||||||
&altair::ParticipationCache::new(state, spec).unwrap(),
|
state,
|
||||||
),
|
&altair::ParticipationCache::new(state, spec).unwrap(),
|
||||||
|
)
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -110,11 +112,13 @@ impl<E: EthSpec> EpochTransition<E> for RewardsAndPenalties {
|
|||||||
validator_statuses.process_attestations(state)?;
|
validator_statuses.process_attestations(state)?;
|
||||||
base::process_rewards_and_penalties(state, &mut validator_statuses, spec)
|
base::process_rewards_and_penalties(state, &mut validator_statuses, spec)
|
||||||
}
|
}
|
||||||
BeaconState::Altair(_) => altair::process_rewards_and_penalties(
|
BeaconState::Altair(_) | BeaconState::Merge(_) => {
|
||||||
state,
|
altair::process_rewards_and_penalties(
|
||||||
&altair::ParticipationCache::new(state, spec).unwrap(),
|
state,
|
||||||
spec,
|
&altair::ParticipationCache::new(state, spec).unwrap(),
|
||||||
),
|
spec,
|
||||||
|
)
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -138,7 +142,7 @@ impl<E: EthSpec> EpochTransition<E> for Slashings {
|
|||||||
spec,
|
spec,
|
||||||
)?;
|
)?;
|
||||||
}
|
}
|
||||||
BeaconState::Altair(_) => {
|
BeaconState::Altair(_) | BeaconState::Merge(_) => {
|
||||||
process_slashings(
|
process_slashings(
|
||||||
state,
|
state,
|
||||||
altair::ParticipationCache::new(state, spec)
|
altair::ParticipationCache::new(state, spec)
|
||||||
@ -197,7 +201,9 @@ impl<E: EthSpec> EpochTransition<E> for SyncCommitteeUpdates {
|
|||||||
fn run(state: &mut BeaconState<E>, spec: &ChainSpec) -> Result<(), EpochProcessingError> {
|
fn run(state: &mut BeaconState<E>, spec: &ChainSpec) -> Result<(), EpochProcessingError> {
|
||||||
match state {
|
match state {
|
||||||
BeaconState::Base(_) => Ok(()),
|
BeaconState::Base(_) => Ok(()),
|
||||||
BeaconState::Altair(_) => altair::process_sync_committee_updates(state, spec),
|
BeaconState::Altair(_) | BeaconState::Merge(_) => {
|
||||||
|
altair::process_sync_committee_updates(state, spec)
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -206,7 +212,7 @@ impl<E: EthSpec> EpochTransition<E> for InactivityUpdates {
|
|||||||
fn run(state: &mut BeaconState<E>, spec: &ChainSpec) -> Result<(), EpochProcessingError> {
|
fn run(state: &mut BeaconState<E>, spec: &ChainSpec) -> Result<(), EpochProcessingError> {
|
||||||
match state {
|
match state {
|
||||||
BeaconState::Base(_) => Ok(()),
|
BeaconState::Base(_) => Ok(()),
|
||||||
BeaconState::Altair(_) => altair::process_inactivity_updates(
|
BeaconState::Altair(_) | BeaconState::Merge(_) => altair::process_inactivity_updates(
|
||||||
state,
|
state,
|
||||||
&altair::ParticipationCache::new(state, spec).unwrap(),
|
&altair::ParticipationCache::new(state, spec).unwrap(),
|
||||||
spec,
|
spec,
|
||||||
@ -219,7 +225,9 @@ impl<E: EthSpec> EpochTransition<E> for ParticipationFlagUpdates {
|
|||||||
fn run(state: &mut BeaconState<E>, _: &ChainSpec) -> Result<(), EpochProcessingError> {
|
fn run(state: &mut BeaconState<E>, _: &ChainSpec) -> Result<(), EpochProcessingError> {
|
||||||
match state {
|
match state {
|
||||||
BeaconState::Base(_) => Ok(()),
|
BeaconState::Base(_) => Ok(()),
|
||||||
BeaconState::Altair(_) => altair::process_participation_flag_updates(state),
|
BeaconState::Altair(_) | BeaconState::Merge(_) => {
|
||||||
|
altair::process_participation_flag_updates(state)
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -267,7 +275,7 @@ impl<E: EthSpec, T: EpochTransition<E>> Case for EpochProcessing<E, T> {
|
|||||||
&& T::name() != "inactivity_updates"
|
&& T::name() != "inactivity_updates"
|
||||||
&& T::name() != "participation_flag_updates"
|
&& T::name() != "participation_flag_updates"
|
||||||
}
|
}
|
||||||
ForkName::Altair => true,
|
ForkName::Altair | ForkName::Merge => true, // TODO: revisit when tests are out
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -72,7 +72,7 @@ impl<E: EthSpec> Operation<E> for Attestation<E> {
|
|||||||
BeaconState::Base(_) => {
|
BeaconState::Base(_) => {
|
||||||
base::process_attestations(state, &[self.clone()], VerifySignatures::True, spec)
|
base::process_attestations(state, &[self.clone()], VerifySignatures::True, spec)
|
||||||
}
|
}
|
||||||
BeaconState::Altair(_) => altair::process_attestation(
|
BeaconState::Altair(_) | BeaconState::Merge(_) => altair::process_attestation(
|
||||||
state,
|
state,
|
||||||
self,
|
self,
|
||||||
0,
|
0,
|
||||||
|
@ -37,6 +37,9 @@ impl<E: EthSpec> LoadCase for TransitionTest<E> {
|
|||||||
ForkName::Altair => {
|
ForkName::Altair => {
|
||||||
spec.altair_fork_epoch = Some(metadata.fork_epoch);
|
spec.altair_fork_epoch = Some(metadata.fork_epoch);
|
||||||
}
|
}
|
||||||
|
ForkName::Merge => {
|
||||||
|
spec.merge_fork_epoch = Some(metadata.fork_epoch);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// Load blocks
|
// Load blocks
|
||||||
|
@ -34,6 +34,7 @@ pub trait Handler {
|
|||||||
let fork_name_str = match fork_name {
|
let fork_name_str = match fork_name {
|
||||||
ForkName::Base => "phase0",
|
ForkName::Base => "phase0",
|
||||||
ForkName::Altair => "altair",
|
ForkName::Altair => "altair",
|
||||||
|
ForkName::Merge => "merge", // TODO: check this
|
||||||
};
|
};
|
||||||
|
|
||||||
let handler_path = PathBuf::from(env!("CARGO_MANIFEST_DIR"))
|
let handler_path = PathBuf::from(env!("CARGO_MANIFEST_DIR"))
|
||||||
|
Loading…
Reference in New Issue
Block a user