Runtime rpc request sizes (#4841)

* add runtime variable list type

* add configs to ChainSpec

* git rid of max request blocks type

* fix tests and lints

* remove todos

* git rid of old const usage

* fix decode impl

* add new config to `Config` api struct

* add docs fix compilt

* move methods for per-fork-spec to chainspec

* get values off chain spec

* fix compile

* remove min by root size

* add tests for runtime var list

---------

Co-authored-by: Jimmy Chen <jchen.tc@gmail.com>
This commit is contained in:
realbigsean 2024-01-08 18:23:47 -05:00 committed by GitHub
parent 5c8c8da8b1
commit b47e3f252e
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
25 changed files with 507 additions and 179 deletions

View File

@ -22,7 +22,6 @@ use std::sync::Arc;
use task_executor::TaskExecutor;
use types::beacon_block_body::{KzgCommitmentOpts, KzgCommitments};
use types::blob_sidecar::{BlobIdentifier, BlobSidecar, FixedBlobSidecarList};
use types::consts::deneb::MIN_EPOCHS_FOR_BLOB_SIDECARS_REQUESTS;
use types::{BlobSidecarList, ChainSpec, Epoch, EthSpec, Hash256, SignedBeaconBlock, Slot};
mod availability_view;
@ -424,7 +423,8 @@ impl<T: BeaconChainTypes> DataAvailabilityChecker<T> {
.map(|current_epoch| {
std::cmp::max(
fork_epoch,
current_epoch.saturating_sub(MIN_EPOCHS_FOR_BLOB_SIDECARS_REQUESTS),
current_epoch
.saturating_sub(self.spec.min_epochs_for_blob_sidecars_requests),
)
})
})
@ -517,7 +517,8 @@ async fn availability_cache_maintenance_service<T: BeaconChainTypes>(
let cutoff_epoch = std::cmp::max(
finalized_epoch + 1,
std::cmp::max(
current_epoch.saturating_sub(MIN_EPOCHS_FOR_BLOB_SIDECARS_REQUESTS),
current_epoch
.saturating_sub(chain.spec.min_epochs_for_blob_sidecars_requests),
deneb_fork_epoch,
),
);

View File

@ -60,7 +60,6 @@ use std::time::Duration;
use task_executor::TaskExecutor;
use tokio::sync::mpsc;
use tokio::sync::mpsc::error::TrySendError;
use types::consts::deneb::MAX_BLOBS_PER_BLOCK;
use types::{Attestation, Hash256, SignedAggregateAndProof, SubnetId};
use types::{EthSpec, Slot};
use work_reprocessing_queue::IgnoredRpcBlock;
@ -168,8 +167,7 @@ const MAX_BLOCKS_BY_RANGE_QUEUE_LEN: usize = 1_024;
/// The maximum number of queued `BlobsByRangeRequest` objects received from the network RPC that
/// will be stored before we start dropping them.
const MAX_BLOBS_BY_RANGE_QUEUE_LEN: usize =
MAX_BLOCKS_BY_RANGE_QUEUE_LEN * MAX_BLOBS_PER_BLOCK as usize;
const MAX_BLOBS_BY_RANGE_QUEUE_LEN: usize = 1024;
/// The maximum number of queued `BlocksByRootRequest` objects received from the network RPC that
/// will be stored before we start dropping them.

View File

@ -316,7 +316,7 @@ mod tests {
));
// Request limits
let limit = protocol_id.rpc_request_limits();
let limit = protocol_id.rpc_request_limits(&fork_context.spec);
let mut max = encode_len(limit.max + 1);
let mut codec = SSZSnappyOutboundCodec::<Spec>::new(
protocol_id.clone(),

View File

@ -15,10 +15,11 @@ use std::io::{Read, Write};
use std::marker::PhantomData;
use std::sync::Arc;
use tokio_util::codec::{Decoder, Encoder};
use types::ChainSpec;
use types::{
BlobSidecar, EthSpec, ForkContext, ForkName, Hash256, LightClientBootstrap, SignedBeaconBlock,
SignedBeaconBlockAltair, SignedBeaconBlockBase, SignedBeaconBlockCapella,
SignedBeaconBlockDeneb, SignedBeaconBlockMerge,
BlobSidecar, EthSpec, ForkContext, ForkName, Hash256, LightClientBootstrap,
RuntimeVariableList, SignedBeaconBlock, SignedBeaconBlockAltair, SignedBeaconBlockBase,
SignedBeaconBlockCapella, SignedBeaconBlockDeneb, SignedBeaconBlockMerge,
};
use unsigned_varint::codec::Uvi;
@ -140,7 +141,7 @@ impl<TSpec: EthSpec> Decoder for SSZSnappyInboundCodec<TSpec> {
// Should not attempt to decode rpc chunks with `length > max_packet_size` or not within bounds of
// packet size for ssz container corresponding to `self.protocol`.
let ssz_limits = self.protocol.rpc_request_limits();
let ssz_limits = self.protocol.rpc_request_limits(&self.fork_context.spec);
if ssz_limits.is_out_of_bounds(length, self.max_packet_size) {
return Err(RPCError::InvalidData(format!(
"RPC request length for protocol {:?} is out of bounds, length {}",
@ -161,7 +162,11 @@ impl<TSpec: EthSpec> Decoder for SSZSnappyInboundCodec<TSpec> {
let n = reader.get_ref().get_ref().position();
self.len = None;
let _read_bytes = src.split_to(n as usize);
handle_rpc_request(self.protocol.versioned_protocol, &decoded_buffer)
handle_rpc_request(
self.protocol.versioned_protocol,
&decoded_buffer,
&self.fork_context.spec,
)
}
Err(e) => handle_error(e, reader.get_ref().get_ref().position(), max_compressed_len),
}
@ -451,6 +456,7 @@ fn handle_length(
fn handle_rpc_request<T: EthSpec>(
versioned_protocol: SupportedProtocol,
decoded_buffer: &[u8],
spec: &ChainSpec,
) -> Result<Option<InboundRequest<T>>, RPCError> {
match versioned_protocol {
SupportedProtocol::StatusV1 => Ok(Some(InboundRequest::Status(
@ -467,12 +473,18 @@ fn handle_rpc_request<T: EthSpec>(
))),
SupportedProtocol::BlocksByRootV2 => Ok(Some(InboundRequest::BlocksByRoot(
BlocksByRootRequest::V2(BlocksByRootRequestV2 {
block_roots: VariableList::from_ssz_bytes(decoded_buffer)?,
block_roots: RuntimeVariableList::from_ssz_bytes(
decoded_buffer,
spec.max_request_blocks as usize,
)?,
}),
))),
SupportedProtocol::BlocksByRootV1 => Ok(Some(InboundRequest::BlocksByRoot(
BlocksByRootRequest::V1(BlocksByRootRequestV1 {
block_roots: VariableList::from_ssz_bytes(decoded_buffer)?,
block_roots: RuntimeVariableList::from_ssz_bytes(
decoded_buffer,
spec.max_request_blocks as usize,
)?,
}),
))),
SupportedProtocol::BlobsByRangeV1 => Ok(Some(InboundRequest::BlobsByRange(
@ -480,7 +492,10 @@ fn handle_rpc_request<T: EthSpec>(
))),
SupportedProtocol::BlobsByRootV1 => {
Ok(Some(InboundRequest::BlobsByRoot(BlobsByRootRequest {
blob_ids: VariableList::from_ssz_bytes(decoded_buffer)?,
blob_ids: RuntimeVariableList::from_ssz_bytes(
decoded_buffer,
spec.max_request_blob_sidecars as usize,
)?,
})))
}
SupportedProtocol::PingV1 => Ok(Some(InboundRequest::Ping(Ping {
@ -773,21 +788,22 @@ mod tests {
}
}
fn bbroot_request_v1() -> BlocksByRootRequest {
BlocksByRootRequest::new_v1(vec![Hash256::zero()].into())
fn bbroot_request_v1(spec: &ChainSpec) -> BlocksByRootRequest {
BlocksByRootRequest::new_v1(vec![Hash256::zero()], spec)
}
fn bbroot_request_v2() -> BlocksByRootRequest {
BlocksByRootRequest::new(vec![Hash256::zero()].into())
fn bbroot_request_v2(spec: &ChainSpec) -> BlocksByRootRequest {
BlocksByRootRequest::new(vec![Hash256::zero()], spec)
}
fn blbroot_request() -> BlobsByRootRequest {
BlobsByRootRequest {
blob_ids: VariableList::from(vec![BlobIdentifier {
fn blbroot_request(spec: &ChainSpec) -> BlobsByRootRequest {
BlobsByRootRequest::new(
vec![BlobIdentifier {
block_root: Hash256::zero(),
index: 0,
}]),
}
}],
spec,
)
}
fn ping_message() -> Ping {
@ -1391,22 +1407,22 @@ mod tests {
#[test]
fn test_encode_then_decode_request() {
let chain_spec = Spec::default_spec();
let requests: &[OutboundRequest<Spec>] = &[
OutboundRequest::Ping(ping_message()),
OutboundRequest::Status(status_message()),
OutboundRequest::Goodbye(GoodbyeReason::Fault),
OutboundRequest::BlocksByRange(bbrange_request_v1()),
OutboundRequest::BlocksByRange(bbrange_request_v2()),
OutboundRequest::BlocksByRoot(bbroot_request_v1()),
OutboundRequest::BlocksByRoot(bbroot_request_v2()),
OutboundRequest::BlocksByRoot(bbroot_request_v1(&chain_spec)),
OutboundRequest::BlocksByRoot(bbroot_request_v2(&chain_spec)),
OutboundRequest::MetaData(MetadataRequest::new_v1()),
OutboundRequest::BlobsByRange(blbrange_request()),
OutboundRequest::BlobsByRoot(blbroot_request()),
OutboundRequest::BlobsByRoot(blbroot_request(&chain_spec)),
OutboundRequest::MetaData(MetadataRequest::new_v2()),
];
let chain_spec = Spec::default_spec();
for req in requests.iter() {
for fork_name in ForkName::list_all() {
encode_then_decode_request(req.clone(), fork_name, &chain_spec);

View File

@ -4,7 +4,7 @@ use std::{
time::Duration,
};
use super::{methods, rate_limiter::Quota, Protocol};
use super::{rate_limiter::Quota, Protocol};
use serde::{Deserialize, Serialize};
@ -99,11 +99,9 @@ impl RateLimiterConfig {
pub const DEFAULT_META_DATA_QUOTA: Quota = Quota::n_every(2, 5);
pub const DEFAULT_STATUS_QUOTA: Quota = Quota::n_every(5, 15);
pub const DEFAULT_GOODBYE_QUOTA: Quota = Quota::one_every(10);
pub const DEFAULT_BLOCKS_BY_RANGE_QUOTA: Quota =
Quota::n_every(methods::MAX_REQUEST_BLOCKS, 10);
pub const DEFAULT_BLOCKS_BY_RANGE_QUOTA: Quota = Quota::n_every(1024, 10);
pub const DEFAULT_BLOCKS_BY_ROOT_QUOTA: Quota = Quota::n_every(128, 10);
pub const DEFAULT_BLOBS_BY_RANGE_QUOTA: Quota =
Quota::n_every(methods::MAX_REQUEST_BLOB_SIDECARS, 10);
pub const DEFAULT_BLOBS_BY_RANGE_QUOTA: Quota = Quota::n_every(768, 10);
pub const DEFAULT_BLOBS_BY_ROOT_QUOTA: Quota = Quota::n_every(128, 10);
pub const DEFAULT_LIGHT_CLIENT_BOOTSTRAP_QUOTA: Quota = Quota::one_every(10);
}

View File

@ -5,36 +5,22 @@ use regex::bytes::Regex;
use serde::Serialize;
use ssz::Encode;
use ssz_derive::{Decode, Encode};
use ssz_types::{
typenum::{U1024, U128, U256, U768},
VariableList,
};
use ssz_types::{typenum::U256, VariableList};
use std::marker::PhantomData;
use std::ops::Deref;
use std::sync::Arc;
use strum::IntoStaticStr;
use superstruct::superstruct;
use types::blob_sidecar::BlobIdentifier;
use types::consts::deneb::MAX_BLOBS_PER_BLOCK;
use types::{
blob_sidecar::BlobSidecar, Epoch, EthSpec, Hash256, LightClientBootstrap, SignedBeaconBlock,
Slot,
blob_sidecar::BlobSidecar, ChainSpec, Epoch, EthSpec, Hash256, LightClientBootstrap,
RuntimeVariableList, SignedBeaconBlock, Slot,
};
/// Maximum number of blocks in a single request.
pub type MaxRequestBlocks = U1024;
pub const MAX_REQUEST_BLOCKS: u64 = 1024;
/// Maximum length of error message.
pub type MaxErrorLen = U256;
pub const MAX_ERROR_LEN: u64 = 256;
pub type MaxRequestBlocksDeneb = U128;
pub const MAX_REQUEST_BLOCKS_DENEB: u64 = 128;
pub type MaxRequestBlobSidecars = U768;
pub const MAX_REQUEST_BLOB_SIDECARS: u64 = MAX_REQUEST_BLOCKS_DENEB * MAX_BLOBS_PER_BLOCK;
/// Wrapper over SSZ List to represent error message in rpc responses.
#[derive(Debug, Clone)]
pub struct ErrorType(pub VariableList<u8, MaxErrorLen>);
@ -344,22 +330,23 @@ impl OldBlocksByRangeRequest {
}
/// Request a number of beacon block bodies from a peer.
#[superstruct(
variants(V1, V2),
variant_attributes(derive(Encode, Decode, Clone, Debug, PartialEq))
)]
#[superstruct(variants(V1, V2), variant_attributes(derive(Clone, Debug, PartialEq)))]
#[derive(Clone, Debug, PartialEq)]
pub struct BlocksByRootRequest {
/// The list of beacon block bodies being requested.
pub block_roots: VariableList<Hash256, MaxRequestBlocks>,
pub block_roots: RuntimeVariableList<Hash256>,
}
impl BlocksByRootRequest {
pub fn new(block_roots: VariableList<Hash256, MaxRequestBlocks>) -> Self {
pub fn new(block_roots: Vec<Hash256>, spec: &ChainSpec) -> Self {
let block_roots =
RuntimeVariableList::from_vec(block_roots, spec.max_request_blocks as usize);
Self::V2(BlocksByRootRequestV2 { block_roots })
}
pub fn new_v1(block_roots: VariableList<Hash256, MaxRequestBlocks>) -> Self {
pub fn new_v1(block_roots: Vec<Hash256>, spec: &ChainSpec) -> Self {
let block_roots =
RuntimeVariableList::from_vec(block_roots, spec.max_request_blocks as usize);
Self::V1(BlocksByRootRequestV1 { block_roots })
}
}
@ -368,7 +355,15 @@ impl BlocksByRootRequest {
#[derive(Clone, Debug, PartialEq)]
pub struct BlobsByRootRequest {
/// The list of beacon block roots being requested.
pub blob_ids: VariableList<BlobIdentifier, MaxRequestBlobSidecars>,
pub blob_ids: RuntimeVariableList<BlobIdentifier>,
}
impl BlobsByRootRequest {
pub fn new(blob_ids: Vec<BlobIdentifier>, spec: &ChainSpec) -> Self {
let blob_ids =
RuntimeVariableList::from_vec(blob_ids, spec.max_request_blob_sidecars as usize);
Self { blob_ids }
}
}
/* RPC Handling and Grouping */

View File

@ -26,7 +26,7 @@ pub(crate) use protocol::InboundRequest;
pub use handler::SubstreamId;
pub use methods::{
BlocksByRangeRequest, BlocksByRootRequest, GoodbyeReason, LightClientBootstrapRequest,
MaxRequestBlocks, RPCResponseErrorCode, ResponseTermination, StatusMessage, MAX_REQUEST_BLOCKS,
RPCResponseErrorCode, ResponseTermination, StatusMessage,
};
pub(crate) use outbound::OutboundRequest;
pub use protocol::{max_rpc_size, Protocol, RPCError};

View File

@ -2,7 +2,6 @@ use super::methods::*;
use crate::rpc::{
codec::{base::BaseInboundCodec, ssz_snappy::SSZSnappyInboundCodec, InboundCodec},
methods::{MaxErrorLen, ResponseTermination, MAX_ERROR_LEN},
MaxRequestBlocks, MAX_REQUEST_BLOCKS,
};
use futures::future::BoxFuture;
use futures::prelude::{AsyncRead, AsyncWrite};
@ -22,7 +21,7 @@ use tokio_util::{
};
use types::{
BeaconBlock, BeaconBlockAltair, BeaconBlockBase, BeaconBlockCapella, BeaconBlockMerge,
BlobSidecar, EmptyBlock, EthSpec, ForkContext, ForkName, Hash256, MainnetEthSpec, Signature,
BlobSidecar, ChainSpec, EmptyBlock, EthSpec, ForkContext, ForkName, MainnetEthSpec, Signature,
SignedBeaconBlock,
};
@ -89,32 +88,6 @@ lazy_static! {
+ (<types::KzgCommitment as Encode>::ssz_fixed_len() * <MainnetEthSpec>::max_blobs_per_block())
+ ssz::BYTES_PER_LENGTH_OFFSET; // Length offset for the blob commitments field.
pub static ref BLOCKS_BY_ROOT_REQUEST_MIN: usize =
VariableList::<Hash256, MaxRequestBlocks>::from(Vec::<Hash256>::new())
.as_ssz_bytes()
.len();
pub static ref BLOCKS_BY_ROOT_REQUEST_MAX: usize =
VariableList::<Hash256, MaxRequestBlocks>::from(vec![
Hash256::zero();
MAX_REQUEST_BLOCKS
as usize
])
.as_ssz_bytes()
.len();
pub static ref BLOBS_BY_ROOT_REQUEST_MIN: usize =
VariableList::<Hash256, MaxRequestBlobSidecars>::from(Vec::<Hash256>::new())
.as_ssz_bytes()
.len();
pub static ref BLOBS_BY_ROOT_REQUEST_MAX: usize =
VariableList::<Hash256, MaxRequestBlobSidecars>::from(vec![
Hash256::zero();
MAX_REQUEST_BLOB_SIDECARS
as usize
])
.as_ssz_bytes()
.len();
pub static ref ERROR_TYPE_MIN: usize =
VariableList::<u8, MaxErrorLen>::from(Vec::<u8>::new())
.as_ssz_bytes()
@ -375,7 +348,7 @@ impl AsRef<str> for ProtocolId {
impl ProtocolId {
/// Returns min and max size for messages of given protocol id requests.
pub fn rpc_request_limits(&self) -> RpcLimits {
pub fn rpc_request_limits(&self, spec: &ChainSpec) -> RpcLimits {
match self.versioned_protocol.protocol() {
Protocol::Status => RpcLimits::new(
<StatusMessage as Encode>::ssz_fixed_len(),
@ -390,16 +363,12 @@ impl ProtocolId {
<OldBlocksByRangeRequestV2 as Encode>::ssz_fixed_len(),
<OldBlocksByRangeRequestV2 as Encode>::ssz_fixed_len(),
),
Protocol::BlocksByRoot => {
RpcLimits::new(*BLOCKS_BY_ROOT_REQUEST_MIN, *BLOCKS_BY_ROOT_REQUEST_MAX)
}
Protocol::BlocksByRoot => RpcLimits::new(0, spec.max_blocks_by_root_request),
Protocol::BlobsByRange => RpcLimits::new(
<BlobsByRangeRequest as Encode>::ssz_fixed_len(),
<BlobsByRangeRequest as Encode>::ssz_fixed_len(),
),
Protocol::BlobsByRoot => {
RpcLimits::new(*BLOBS_BY_ROOT_REQUEST_MIN, *BLOBS_BY_ROOT_REQUEST_MAX)
}
Protocol::BlobsByRoot => RpcLimits::new(0, spec.max_blobs_by_root_request),
Protocol::Ping => RpcLimits::new(
<Ping as Encode>::ssz_fixed_len(),
<Ping as Encode>::ssz_fixed_len(),

View File

@ -41,8 +41,7 @@ use std::{
};
use types::ForkName;
use types::{
consts::altair::SYNC_COMMITTEE_SUBNET_COUNT, consts::deneb::BLOB_SIDECAR_SUBNET_COUNT,
EnrForkId, EthSpec, ForkContext, Slot, SubnetId,
consts::altair::SYNC_COMMITTEE_SUBNET_COUNT, EnrForkId, EthSpec, ForkContext, Slot, SubnetId,
};
use utils::{build_transport, strip_peer_id, Context as ServiceContext, MAX_CONNECTIONS_PER_PEER};
@ -226,7 +225,7 @@ impl<AppReqId: ReqId, TSpec: EthSpec> Network<AppReqId, TSpec> {
let max_topics = ctx.chain_spec.attestation_subnet_count as usize
+ SYNC_COMMITTEE_SUBNET_COUNT as usize
+ BLOB_SIDECAR_SUBNET_COUNT as usize
+ ctx.chain_spec.blob_sidecar_subnet_count as usize
+ BASE_CORE_TOPICS.len()
+ ALTAIR_CORE_TOPICS.len()
+ CAPELLA_CORE_TOPICS.len()
@ -239,7 +238,7 @@ impl<AppReqId: ReqId, TSpec: EthSpec> Network<AppReqId, TSpec> {
possible_fork_digests,
ctx.chain_spec.attestation_subnet_count,
SYNC_COMMITTEE_SUBNET_COUNT,
BLOB_SIDECAR_SUBNET_COUNT,
ctx.chain_spec.blob_sidecar_subnet_count,
),
// during a fork we subscribe to both the old and new topics
max_subscribed_topics: max_topics * 4,
@ -639,7 +638,7 @@ impl<AppReqId: ReqId, TSpec: EthSpec> Network<AppReqId, TSpec> {
}
// Subscribe to core topics for the new fork
for kind in fork_core_topics::<TSpec>(&new_fork) {
for kind in fork_core_topics::<TSpec>(&new_fork, &self.fork_context.spec) {
let topic = GossipTopic::new(kind, GossipEncoding::default(), new_fork_digest);
self.subscribe(topic);
}

View File

@ -1,8 +1,7 @@
use libp2p::gossipsub::{IdentTopic as Topic, TopicHash};
use serde::{Deserialize, Serialize};
use strum::AsRefStr;
use types::consts::deneb::BLOB_SIDECAR_SUBNET_COUNT;
use types::{EthSpec, ForkName, SubnetId, SyncSubnetId};
use types::{ChainSpec, EthSpec, ForkName, SubnetId, SyncSubnetId};
use crate::Subnet;
@ -44,7 +43,7 @@ pub const LIGHT_CLIENT_GOSSIP_TOPICS: [GossipKind; 2] = [
pub const DENEB_CORE_TOPICS: [GossipKind; 0] = [];
/// Returns the core topics associated with each fork that are new to the previous fork
pub fn fork_core_topics<T: EthSpec>(fork_name: &ForkName) -> Vec<GossipKind> {
pub fn fork_core_topics<T: EthSpec>(fork_name: &ForkName, spec: &ChainSpec) -> Vec<GossipKind> {
match fork_name {
ForkName::Base => BASE_CORE_TOPICS.to_vec(),
ForkName::Altair => ALTAIR_CORE_TOPICS.to_vec(),
@ -53,7 +52,7 @@ pub fn fork_core_topics<T: EthSpec>(fork_name: &ForkName) -> Vec<GossipKind> {
ForkName::Deneb => {
// All of deneb blob topics are core topics
let mut deneb_blob_topics = Vec::new();
for i in 0..BLOB_SIDECAR_SUBNET_COUNT {
for i in 0..spec.blob_sidecar_subnet_count {
deneb_blob_topics.push(GossipKind::BlobSidecar(i));
}
let mut deneb_topics = DENEB_CORE_TOPICS.to_vec();
@ -65,10 +64,13 @@ pub fn fork_core_topics<T: EthSpec>(fork_name: &ForkName) -> Vec<GossipKind> {
/// Returns all the topics that we need to subscribe to for a given fork
/// including topics from older forks and new topics for the current fork.
pub fn core_topics_to_subscribe<T: EthSpec>(mut current_fork: ForkName) -> Vec<GossipKind> {
let mut topics = fork_core_topics::<T>(&current_fork);
pub fn core_topics_to_subscribe<T: EthSpec>(
mut current_fork: ForkName,
spec: &ChainSpec,
) -> Vec<GossipKind> {
let mut topics = fork_core_topics::<T>(&current_fork, spec);
while let Some(previous_fork) = current_fork.previous_fork() {
let previous_fork_topics = fork_core_topics::<T>(&previous_fork);
let previous_fork_topics = fork_core_topics::<T>(&previous_fork, spec);
topics.extend(previous_fork_topics);
current_fork = previous_fork;
}
@ -435,14 +437,18 @@ mod tests {
#[test]
fn test_core_topics_to_subscribe() {
type E = MainnetEthSpec;
let spec = E::default_spec();
let mut all_topics = Vec::new();
let mut deneb_core_topics = fork_core_topics::<E>(&ForkName::Deneb);
let mut deneb_core_topics = fork_core_topics::<E>(&ForkName::Deneb, &spec);
all_topics.append(&mut deneb_core_topics);
all_topics.extend(CAPELLA_CORE_TOPICS);
all_topics.extend(ALTAIR_CORE_TOPICS);
all_topics.extend(BASE_CORE_TOPICS);
let latest_fork = *ForkName::list_all().last().unwrap();
assert_eq!(core_topics_to_subscribe::<E>(latest_fork), all_topics);
assert_eq!(
core_topics_to_subscribe::<E>(latest_fork, &spec),
all_topics
);
}
}

View File

@ -743,15 +743,17 @@ fn test_tcp_blocks_by_root_chunked_rpc() {
.await;
// BlocksByRoot Request
let rpc_request =
Request::BlocksByRoot(BlocksByRootRequest::new(VariableList::from(vec![
let rpc_request = Request::BlocksByRoot(BlocksByRootRequest::new(
vec![
Hash256::from_low_u64_be(0),
Hash256::from_low_u64_be(0),
Hash256::from_low_u64_be(0),
Hash256::from_low_u64_be(0),
Hash256::from_low_u64_be(0),
Hash256::from_low_u64_be(0),
])));
],
&spec,
));
// BlocksByRoot Response
let full_block = BeaconBlock::Base(BeaconBlockBase::<E>::full(&spec));
@ -876,8 +878,8 @@ fn test_tcp_blocks_by_root_chunked_rpc_terminates_correctly() {
.await;
// BlocksByRoot Request
let rpc_request =
Request::BlocksByRoot(BlocksByRootRequest::new(VariableList::from(vec![
let rpc_request = Request::BlocksByRoot(BlocksByRootRequest::new(
vec![
Hash256::from_low_u64_be(0),
Hash256::from_low_u64_be(0),
Hash256::from_low_u64_be(0),
@ -888,7 +890,9 @@ fn test_tcp_blocks_by_root_chunked_rpc_terminates_correctly() {
Hash256::from_low_u64_be(0),
Hash256::from_low_u64_be(0),
Hash256::from_low_u64_be(0),
])));
],
&spec,
));
// BlocksByRoot Response
let full_block = BeaconBlock::Base(BeaconBlockBase::<E>::full(&spec));

View File

@ -5,9 +5,7 @@ use crate::sync::SyncMessage;
use beacon_chain::{BeaconChainError, BeaconChainTypes, HistoricalBlockError, WhenSlotSkipped};
use beacon_processor::SendOnDrop;
use itertools::process_results;
use lighthouse_network::rpc::methods::{
BlobsByRangeRequest, BlobsByRootRequest, MAX_REQUEST_BLOB_SIDECARS, MAX_REQUEST_BLOCKS_DENEB,
};
use lighthouse_network::rpc::methods::{BlobsByRangeRequest, BlobsByRootRequest};
use lighthouse_network::rpc::StatusMessage;
use lighthouse_network::rpc::*;
use lighthouse_network::{PeerId, PeerRequestId, ReportSource, Response, SyncInfo};
@ -222,12 +220,14 @@ impl<T: BeaconChainTypes> NetworkBeaconProcessor<T> {
request_id: PeerRequestId,
request: BlobsByRootRequest,
) {
let Some(requested_root) = request.blob_ids.first().map(|id| id.block_root) else {
let Some(requested_root) = request.blob_ids.as_slice().first().map(|id| id.block_root)
else {
// No blob ids requested.
return;
};
let requested_indices = request
.blob_ids
.as_slice()
.iter()
.map(|id| id.index)
.collect::<Vec<_>>();
@ -235,9 +235,9 @@ impl<T: BeaconChainTypes> NetworkBeaconProcessor<T> {
let send_response = true;
let mut blob_list_results = HashMap::new();
for id in request.blob_ids.into_iter() {
for id in request.blob_ids.as_slice() {
// First attempt to get the blobs from the RPC cache.
if let Ok(Some(blob)) = self.chain.data_availability_checker.get_blob(&id) {
if let Ok(Some(blob)) = self.chain.data_availability_checker.get_blob(id) {
self.send_response(peer_id, Response::BlobsByRoot(Some(blob)), request_id);
send_blob_count += 1;
} else {
@ -248,7 +248,7 @@ impl<T: BeaconChainTypes> NetworkBeaconProcessor<T> {
let blob_list_result = match blob_list_results.entry(root) {
Entry::Vacant(entry) => {
entry.insert(self.chain.get_blobs_checking_early_attester_cache(&root))
entry.insert(self.chain.get_blobs_checking_early_attester_cache(root))
}
Entry::Occupied(entry) => entry.into_mut(),
};
@ -256,7 +256,7 @@ impl<T: BeaconChainTypes> NetworkBeaconProcessor<T> {
match blob_list_result.as_ref() {
Ok(blobs_sidecar_list) => {
'inner: for blob_sidecar in blobs_sidecar_list.iter() {
if blob_sidecar.index == index {
if blob_sidecar.index == *index {
self.send_response(
peer_id,
Response::BlobsByRoot(Some(blob_sidecar.clone())),
@ -346,14 +346,17 @@ impl<T: BeaconChainTypes> NetworkBeaconProcessor<T> {
);
// Should not send more than max request blocks
let max_request_size = self.chain.epoch().map_or(MAX_REQUEST_BLOCKS, |epoch| {
match self.chain.spec.fork_name_at_epoch(epoch) {
ForkName::Deneb => MAX_REQUEST_BLOCKS_DENEB,
ForkName::Base | ForkName::Altair | ForkName::Merge | ForkName::Capella => {
MAX_REQUEST_BLOCKS
}
}
});
let max_request_size =
self.chain
.epoch()
.map_or(self.chain.spec.max_request_blocks, |epoch| {
match self.chain.spec.fork_name_at_epoch(epoch) {
ForkName::Deneb => self.chain.spec.max_request_blocks_deneb,
ForkName::Base | ForkName::Altair | ForkName::Merge | ForkName::Capella => {
self.chain.spec.max_request_blocks
}
}
});
if *req.count() > max_request_size {
return self.send_error_response(
peer_id,
@ -586,7 +589,7 @@ impl<T: BeaconChainTypes> NetworkBeaconProcessor<T> {
);
// Should not send more than max request blocks
if req.max_blobs_requested::<T::EthSpec>() > MAX_REQUEST_BLOB_SIDECARS {
if req.max_blobs_requested::<T::EthSpec>() > self.chain.spec.max_request_blob_sidecars {
return self.send_error_response(
peer_id,
RPCResponseErrorCode::InvalidRequest,

View File

@ -716,9 +716,10 @@ impl<T: BeaconChainTypes> NetworkService<T> {
}
let mut subscribed_topics: Vec<GossipTopic> = vec![];
for topic_kind in
core_topics_to_subscribe::<T::EthSpec>(self.fork_context.current_fork())
{
for topic_kind in core_topics_to_subscribe::<T::EthSpec>(
self.fork_context.current_fork(),
&self.fork_context.spec,
) {
for fork_digest in self.required_gossip_fork_digests() {
let topic = GossipTopic::new(
topic_kind.clone(),
@ -945,7 +946,10 @@ impl<T: BeaconChainTypes> NetworkService<T> {
}
fn subscribed_core_topics(&self) -> bool {
let core_topics = core_topics_to_subscribe::<T::EthSpec>(self.fork_context.current_fork());
let core_topics = core_topics_to_subscribe::<T::EthSpec>(
self.fork_context.current_fork(),
&self.fork_context.spec,
);
let core_topics: HashSet<&GossipKind> = HashSet::from_iter(&core_topics);
let subscriptions = self.network_globals.gossipsub_subscriptions.read();
let subscribed_topics: HashSet<&GossipKind> =

View File

@ -13,12 +13,11 @@ use beacon_chain::{get_block_root, BeaconChainTypes};
use lighthouse_network::rpc::methods::BlobsByRootRequest;
use lighthouse_network::rpc::BlocksByRootRequest;
use rand::prelude::IteratorRandom;
use ssz_types::VariableList;
use std::ops::IndexMut;
use std::sync::Arc;
use std::time::Duration;
use types::blob_sidecar::{BlobIdentifier, FixedBlobSidecarList};
use types::{BlobSidecar, EthSpec, Hash256, SignedBeaconBlock};
use types::{BlobSidecar, ChainSpec, EthSpec, Hash256, SignedBeaconBlock};
#[derive(Debug, Copy, Clone)]
pub enum ResponseType {
@ -87,11 +86,14 @@ pub trait RequestState<L: Lookup, T: BeaconChainTypes> {
/* Request building methods */
/// Construct a new request.
fn build_request(&mut self) -> Result<(PeerId, Self::RequestType), LookupRequestError> {
fn build_request(
&mut self,
spec: &ChainSpec,
) -> Result<(PeerId, Self::RequestType), LookupRequestError> {
// Verify and construct request.
self.too_many_attempts()?;
let peer = self.get_peer()?;
let request = self.new_request();
let request = self.new_request(spec);
Ok((peer, request))
}
@ -108,7 +110,7 @@ pub trait RequestState<L: Lookup, T: BeaconChainTypes> {
}
// Construct request.
let (peer_id, request) = self.build_request()?;
let (peer_id, request) = self.build_request(&cx.chain.spec)?;
// Update request state.
self.get_state_mut().state = State::Downloading { peer_id };
@ -151,7 +153,7 @@ pub trait RequestState<L: Lookup, T: BeaconChainTypes> {
}
/// Initialize `Self::RequestType`.
fn new_request(&self) -> Self::RequestType;
fn new_request(&self, spec: &ChainSpec) -> Self::RequestType;
/// Send the request to the network service.
fn make_request(
@ -254,8 +256,8 @@ impl<L: Lookup, T: BeaconChainTypes> RequestState<L, T> for BlockRequestState<L>
type VerifiedResponseType = Arc<SignedBeaconBlock<T::EthSpec>>;
type ReconstructedResponseType = RpcBlock<T::EthSpec>;
fn new_request(&self) -> BlocksByRootRequest {
BlocksByRootRequest::new(VariableList::from(vec![self.requested_block_root]))
fn new_request(&self, spec: &ChainSpec) -> BlocksByRootRequest {
BlocksByRootRequest::new(vec![self.requested_block_root], spec)
}
fn make_request(
@ -353,10 +355,9 @@ impl<L: Lookup, T: BeaconChainTypes> RequestState<L, T> for BlobRequestState<L,
type VerifiedResponseType = FixedBlobSidecarList<T::EthSpec>;
type ReconstructedResponseType = FixedBlobSidecarList<T::EthSpec>;
fn new_request(&self) -> BlobsByRootRequest {
fn new_request(&self, spec: &ChainSpec) -> BlobsByRootRequest {
let blob_id_vec: Vec<BlobIdentifier> = self.requested_ids.clone().into();
let blob_ids = VariableList::from(blob_id_vec);
BlobsByRootRequest { blob_ids }
BlobsByRootRequest::new(blob_id_vec, spec)
}
fn make_request(

View File

@ -575,7 +575,7 @@ mod tests {
HotColdDB::open_ephemeral(StoreConfig::default(), ChainSpec::minimal(), log.clone())
.expect("store");
let da_checker = Arc::new(
DataAvailabilityChecker::new(slot_clock, None, store.into(), &log, spec)
DataAvailabilityChecker::new(slot_clock, None, store.into(), &log, spec.clone())
.expect("data availability checker"),
);
let mut sl = SingleBlockLookup::<TestLookup1, T>::new(
@ -587,6 +587,7 @@ mod tests {
);
<BlockRequestState<TestLookup1> as RequestState<TestLookup1, T>>::build_request(
&mut sl.block_request_state,
&spec,
)
.unwrap();
sl.block_request_state.state.state = State::Downloading { peer_id };
@ -616,7 +617,7 @@ mod tests {
.expect("store");
let da_checker = Arc::new(
DataAvailabilityChecker::new(slot_clock, None, store.into(), &log, spec)
DataAvailabilityChecker::new(slot_clock, None, store.into(), &log, spec.clone())
.expect("data availability checker"),
);
@ -630,6 +631,7 @@ mod tests {
for _ in 1..TestLookup2::MAX_ATTEMPTS {
<BlockRequestState<TestLookup2> as RequestState<TestLookup2, T>>::build_request(
&mut sl.block_request_state,
&spec,
)
.unwrap();
sl.block_request_state.state.register_failure_downloading();
@ -638,6 +640,7 @@ mod tests {
// Now we receive the block and send it for processing
<BlockRequestState<TestLookup2> as RequestState<TestLookup2, T>>::build_request(
&mut sl.block_request_state,
&spec,
)
.unwrap();
sl.block_request_state.state.state = State::Downloading { peer_id };
@ -654,7 +657,8 @@ mod tests {
sl.block_request_state.state.register_failure_processing();
assert_eq!(
<BlockRequestState<TestLookup2> as RequestState<TestLookup2, T>>::build_request(
&mut sl.block_request_state
&mut sl.block_request_state,
&spec
),
Err(LookupRequestError::TooManyAttempts {
cannot_process: false

View File

@ -52,7 +52,6 @@ use beacon_chain::{
AvailabilityProcessingStatus, BeaconChain, BeaconChainTypes, BlockError, EngineState,
};
use futures::StreamExt;
use lighthouse_network::rpc::methods::MAX_REQUEST_BLOCKS;
use lighthouse_network::rpc::RPCError;
use lighthouse_network::types::{NetworkGlobals, SyncState};
use lighthouse_network::SyncInfo;
@ -230,7 +229,7 @@ pub fn spawn<T: BeaconChainTypes>(
log: slog::Logger,
) {
assert!(
MAX_REQUEST_BLOCKS >= T::EthSpec::slots_per_epoch() * EPOCHS_PER_BATCH,
beacon_chain.spec.max_request_blocks >= T::EthSpec::slots_per_epoch() * EPOCHS_PER_BATCH,
"Max blocks that can be requested in a single batch greater than max allowed blocks in a single request"
);

View File

@ -475,9 +475,15 @@ impl<T: BeaconChainTypes> SyncNetworkContext<T> {
};
let request_id = RequestId::Sync(sync_id);
if let Some(block_root) = blob_request.blob_ids.first().map(|id| id.block_root) {
if let Some(block_root) = blob_request
.blob_ids
.as_slice()
.first()
.map(|id| id.block_root)
{
let indices = blob_request
.blob_ids
.as_slice()
.iter()
.map(|id| id.index)
.collect::<Vec<_>>();

View File

@ -40,7 +40,6 @@ use std::path::Path;
use std::sync::Arc;
use std::time::Duration;
use types::blob_sidecar::BlobSidecarList;
use types::consts::deneb::MIN_EPOCHS_FOR_BLOB_SIDECARS_REQUESTS;
use types::*;
/// On-disk database that stores finalized states efficiently.
@ -2055,7 +2054,7 @@ impl<E: EthSpec, Hot: ItemStore<E>, Cold: ItemStore<E>> HotColdDB<E, Hot, Cold>
let min_current_epoch = self.get_split_slot().epoch(E::slots_per_epoch()) + 2;
let min_data_availability_boundary = std::cmp::max(
deneb_fork_epoch,
min_current_epoch.saturating_sub(MIN_EPOCHS_FOR_BLOB_SIDECARS_REQUESTS),
min_current_epoch.saturating_sub(self.spec.min_epochs_for_blob_sidecars_requests),
);
self.try_prune_blobs(force, min_data_availability_boundary)

View File

@ -1,6 +1,5 @@
use ethereum_hashing::hash_fixed;
use types::consts::deneb::VERSIONED_HASH_VERSION_KZG;
use types::{KzgCommitment, VersionedHash};
use types::{KzgCommitment, VersionedHash, VERSIONED_HASH_VERSION_KZG};
pub fn kzg_commitment_to_versioned_hash(kzg_commitment: &KzgCommitment) -> VersionedHash {
let mut hashed_commitment = hash_fixed(&kzg_commitment.0);

View File

@ -4,6 +4,7 @@ use int_to_bytes::int_to_bytes4;
use serde::Deserialize;
use serde::{Deserializer, Serialize, Serializer};
use serde_utils::quoted_u64::MaybeQuoted;
use ssz::Encode;
use std::fs::File;
use std::path::Path;
use std::time::Duration;
@ -172,22 +173,40 @@ pub struct ChainSpec {
*/
pub boot_nodes: Vec<String>,
pub network_id: u8,
pub attestation_propagation_slot_range: u64,
pub maximum_gossip_clock_disparity_millis: u64,
pub target_aggregators_per_committee: u64,
pub attestation_subnet_count: u64,
pub subnets_per_node: u8,
pub epochs_per_subnet_subscription: u64,
pub gossip_max_size: u64,
pub max_request_blocks: u64,
pub epochs_per_subnet_subscription: u64,
pub min_epochs_for_block_requests: u64,
pub max_chunk_size: u64,
pub ttfb_timeout: u64,
pub resp_timeout: u64,
pub attestation_propagation_slot_range: u64,
pub maximum_gossip_clock_disparity_millis: u64,
pub message_domain_invalid_snappy: [u8; 4],
pub message_domain_valid_snappy: [u8; 4],
pub subnets_per_node: u8,
pub attestation_subnet_count: u64,
pub attestation_subnet_extra_bits: u8,
pub attestation_subnet_prefix_bits: u8,
/*
* Networking Deneb
*/
pub max_request_blocks_deneb: u64,
pub max_request_blob_sidecars: u64,
pub min_epochs_for_blob_sidecars_requests: u64,
pub blob_sidecar_subnet_count: u64,
/*
* Networking Derived
*
* When adding fields here, make sure any values are derived again during `apply_to_chain_spec`.
*/
pub max_blocks_by_root_request: usize,
pub max_blocks_by_root_request_deneb: usize,
pub max_blobs_by_root_request: usize,
/*
* Application params
*/
@ -487,6 +506,25 @@ impl ChainSpec {
Duration::from_secs(self.resp_timeout)
}
pub fn max_blocks_by_root_request(&self, fork_name: ForkName) -> usize {
match fork_name {
ForkName::Base | ForkName::Altair | ForkName::Merge | ForkName::Capella => {
self.max_blocks_by_root_request
}
ForkName::Deneb => self.max_blocks_by_root_request_deneb,
}
}
pub fn max_request_blocks(&self, fork_name: ForkName) -> usize {
let max_request_blocks = match fork_name {
ForkName::Base | ForkName::Altair | ForkName::Merge | ForkName::Capella => {
self.max_request_blocks
}
ForkName::Deneb => self.max_request_blocks_deneb,
};
max_request_blocks as usize
}
/// Returns a `ChainSpec` compatible with the Ethereum Foundation specification.
pub fn mainnet() -> Self {
Self {
@ -648,12 +686,12 @@ impl ChainSpec {
*/
boot_nodes: vec![],
network_id: 1, // mainnet network id
attestation_propagation_slot_range: 32,
attestation_propagation_slot_range: default_attestation_propagation_slot_range(),
attestation_subnet_count: 64,
subnets_per_node: 2,
maximum_gossip_clock_disparity_millis: 500,
maximum_gossip_clock_disparity_millis: default_maximum_gossip_clock_disparity_millis(),
target_aggregators_per_committee: 16,
epochs_per_subnet_subscription: 256,
epochs_per_subnet_subscription: default_epochs_per_subnet_subscription(),
gossip_max_size: default_gossip_max_size(),
min_epochs_for_block_requests: default_min_epochs_for_block_requests(),
max_chunk_size: default_max_chunk_size(),
@ -663,6 +701,23 @@ impl ChainSpec {
message_domain_valid_snappy: default_message_domain_valid_snappy(),
attestation_subnet_extra_bits: default_attestation_subnet_extra_bits(),
attestation_subnet_prefix_bits: default_attestation_subnet_prefix_bits(),
max_request_blocks: default_max_request_blocks(),
/*
* Networking Deneb Specific
*/
max_request_blocks_deneb: default_max_request_blocks_deneb(),
max_request_blob_sidecars: default_max_request_blob_sidecars(),
min_epochs_for_blob_sidecars_requests: default_min_epochs_for_blob_sidecars_requests(),
blob_sidecar_subnet_count: default_blob_sidecar_subnet_count(),
/*
* Derived Deneb Specific
*/
max_blocks_by_root_request: default_max_blocks_by_root_request(),
max_blocks_by_root_request_deneb: default_max_blocks_by_root_request_deneb(),
max_blobs_by_root_request: default_max_blobs_by_root_request(),
/*
* Application specific
*/
@ -892,12 +947,12 @@ impl ChainSpec {
*/
boot_nodes: vec![],
network_id: 100, // Gnosis Chain network id
attestation_propagation_slot_range: 32,
attestation_propagation_slot_range: default_attestation_propagation_slot_range(),
attestation_subnet_count: 64,
subnets_per_node: 4, // Make this larger than usual to avoid network damage
maximum_gossip_clock_disparity_millis: 500,
maximum_gossip_clock_disparity_millis: default_maximum_gossip_clock_disparity_millis(),
target_aggregators_per_committee: 16,
epochs_per_subnet_subscription: 256,
epochs_per_subnet_subscription: default_epochs_per_subnet_subscription(),
gossip_max_size: default_gossip_max_size(),
min_epochs_for_block_requests: default_min_epochs_for_block_requests(),
max_chunk_size: default_max_chunk_size(),
@ -907,6 +962,22 @@ impl ChainSpec {
message_domain_valid_snappy: default_message_domain_valid_snappy(),
attestation_subnet_extra_bits: default_attestation_subnet_extra_bits(),
attestation_subnet_prefix_bits: default_attestation_subnet_prefix_bits(),
max_request_blocks: default_max_request_blocks(),
/*
* Networking Deneb Specific
*/
max_request_blocks_deneb: default_max_request_blocks_deneb(),
max_request_blob_sidecars: default_max_request_blob_sidecars(),
min_epochs_for_blob_sidecars_requests: default_min_epochs_for_blob_sidecars_requests(),
blob_sidecar_subnet_count: default_blob_sidecar_subnet_count(),
/*
* Derived Deneb Specific
*/
max_blocks_by_root_request: default_max_blocks_by_root_request(),
max_blocks_by_root_request_deneb: default_max_blocks_by_root_request_deneb(),
max_blobs_by_root_request: default_max_blobs_by_root_request(),
/*
* Application specific
@ -1032,6 +1103,12 @@ pub struct Config {
#[serde(default = "default_gossip_max_size")]
#[serde(with = "serde_utils::quoted_u64")]
gossip_max_size: u64,
#[serde(default = "default_max_request_blocks")]
#[serde(with = "serde_utils::quoted_u64")]
max_request_blocks: u64,
#[serde(default = "default_epochs_per_subnet_subscription")]
#[serde(with = "serde_utils::quoted_u64")]
epochs_per_subnet_subscription: u64,
#[serde(default = "default_min_epochs_for_block_requests")]
#[serde(with = "serde_utils::quoted_u64")]
min_epochs_for_block_requests: u64,
@ -1044,6 +1121,12 @@ pub struct Config {
#[serde(default = "default_resp_timeout")]
#[serde(with = "serde_utils::quoted_u64")]
resp_timeout: u64,
#[serde(default = "default_attestation_propagation_slot_range")]
#[serde(with = "serde_utils::quoted_u64")]
attestation_propagation_slot_range: u64,
#[serde(default = "default_maximum_gossip_clock_disparity_millis")]
#[serde(with = "serde_utils::quoted_u64")]
maximum_gossip_clock_disparity_millis: u64,
#[serde(default = "default_message_domain_invalid_snappy")]
#[serde(with = "serde_utils::bytes_4_hex")]
message_domain_invalid_snappy: [u8; 4],
@ -1056,6 +1139,18 @@ pub struct Config {
#[serde(default = "default_attestation_subnet_prefix_bits")]
#[serde(with = "serde_utils::quoted_u8")]
attestation_subnet_prefix_bits: u8,
#[serde(default = "default_max_request_blocks_deneb")]
#[serde(with = "serde_utils::quoted_u64")]
max_request_blocks_deneb: u64,
#[serde(default = "default_max_request_blob_sidecars")]
#[serde(with = "serde_utils::quoted_u64")]
max_request_blob_sidecars: u64,
#[serde(default = "default_min_epochs_for_blob_sidecars_requests")]
#[serde(with = "serde_utils::quoted_u64")]
min_epochs_for_blob_sidecars_requests: u64,
#[serde(default = "default_blob_sidecar_subnet_count")]
#[serde(with = "serde_utils::quoted_u64")]
blob_sidecar_subnet_count: u64,
}
fn default_bellatrix_fork_version() -> [u8; 4] {
@ -1141,6 +1236,70 @@ const fn default_attestation_subnet_prefix_bits() -> u8 {
6
}
const fn default_max_request_blocks() -> u64 {
1024
}
const fn default_max_request_blocks_deneb() -> u64 {
128
}
const fn default_max_request_blob_sidecars() -> u64 {
768
}
const fn default_min_epochs_for_blob_sidecars_requests() -> u64 {
4096
}
const fn default_blob_sidecar_subnet_count() -> u64 {
6
}
const fn default_epochs_per_subnet_subscription() -> u64 {
256
}
const fn default_attestation_propagation_slot_range() -> u64 {
32
}
const fn default_maximum_gossip_clock_disparity_millis() -> u64 {
500
}
fn max_blocks_by_root_request_common(max_request_blocks: u64) -> usize {
let max_request_blocks = max_request_blocks as usize;
RuntimeVariableList::<Hash256>::from_vec(
vec![Hash256::zero(); max_request_blocks],
max_request_blocks,
)
.as_ssz_bytes()
.len()
}
fn max_blobs_by_root_request_common(max_request_blob_sidecars: u64) -> usize {
let max_request_blob_sidecars = max_request_blob_sidecars as usize;
RuntimeVariableList::<Hash256>::from_vec(
vec![Hash256::zero(); max_request_blob_sidecars],
max_request_blob_sidecars,
)
.as_ssz_bytes()
.len()
}
fn default_max_blocks_by_root_request() -> usize {
max_blocks_by_root_request_common(default_max_request_blocks())
}
fn default_max_blocks_by_root_request_deneb() -> usize {
max_blocks_by_root_request_common(default_max_request_blocks_deneb())
}
fn default_max_blobs_by_root_request() -> usize {
max_blobs_by_root_request_common(default_max_request_blob_sidecars())
}
impl Default for Config {
fn default() -> Self {
let chain_spec = MainnetEthSpec::default_spec();
@ -1243,14 +1402,22 @@ impl Config {
deposit_contract_address: spec.deposit_contract_address,
gossip_max_size: spec.gossip_max_size,
max_request_blocks: spec.max_request_blocks,
epochs_per_subnet_subscription: spec.epochs_per_subnet_subscription,
min_epochs_for_block_requests: spec.min_epochs_for_block_requests,
max_chunk_size: spec.max_chunk_size,
ttfb_timeout: spec.ttfb_timeout,
resp_timeout: spec.resp_timeout,
attestation_propagation_slot_range: spec.attestation_propagation_slot_range,
maximum_gossip_clock_disparity_millis: spec.maximum_gossip_clock_disparity_millis,
message_domain_invalid_snappy: spec.message_domain_invalid_snappy,
message_domain_valid_snappy: spec.message_domain_valid_snappy,
attestation_subnet_extra_bits: spec.attestation_subnet_extra_bits,
attestation_subnet_prefix_bits: spec.attestation_subnet_prefix_bits,
max_request_blocks_deneb: spec.max_request_blocks_deneb,
max_request_blob_sidecars: spec.max_request_blob_sidecars,
min_epochs_for_blob_sidecars_requests: spec.min_epochs_for_blob_sidecars_requests,
blob_sidecar_subnet_count: spec.blob_sidecar_subnet_count,
}
}
@ -1307,6 +1474,14 @@ impl Config {
message_domain_valid_snappy,
attestation_subnet_extra_bits,
attestation_subnet_prefix_bits,
max_request_blocks,
epochs_per_subnet_subscription,
attestation_propagation_slot_range,
maximum_gossip_clock_disparity_millis,
max_request_blocks_deneb,
max_request_blob_sidecars,
min_epochs_for_blob_sidecars_requests,
blob_sidecar_subnet_count,
} = self;
if preset_base != T::spec_name().to_string().as_str() {
@ -1356,6 +1531,22 @@ impl Config {
message_domain_valid_snappy,
attestation_subnet_extra_bits,
attestation_subnet_prefix_bits,
max_request_blocks,
epochs_per_subnet_subscription,
attestation_propagation_slot_range,
maximum_gossip_clock_disparity_millis,
max_request_blocks_deneb,
max_request_blob_sidecars,
min_epochs_for_blob_sidecars_requests,
blob_sidecar_subnet_count,
// We need to re-derive any values that might have changed in the config.
max_blocks_by_root_request: max_blocks_by_root_request_common(max_request_blocks),
max_blocks_by_root_request_deneb: max_blocks_by_root_request_common(
max_request_blocks_deneb,
),
max_blobs_by_root_request: max_blobs_by_root_request_common(max_request_blob_sidecars),
..chain_spec.clone()
})
}

View File

@ -22,11 +22,3 @@ pub mod altair {
pub mod merge {
pub const INTERVALS_PER_SLOT: u64 = 3;
}
pub mod deneb {
use crate::Epoch;
pub const VERSIONED_HASH_VERSION_KZG: u8 = 1;
pub const BLOB_SIDECAR_SUBNET_COUNT: u64 = 6;
pub const MAX_BLOBS_PER_BLOCK: u64 = BLOB_SIDECAR_SUBNET_COUNT;
pub const MIN_EPOCHS_FOR_BLOB_SIDECARS_REQUESTS: Epoch = Epoch::new(4096);
}

View File

@ -9,6 +9,7 @@ pub struct ForkContext {
current_fork: RwLock<ForkName>,
fork_to_digest: HashMap<ForkName, [u8; 4]>,
digest_to_fork: HashMap<[u8; 4], ForkName>,
pub spec: ChainSpec,
}
impl ForkContext {
@ -73,6 +74,7 @@ impl ForkContext {
current_fork: RwLock::new(spec.fork_name_at_slot::<T>(current_slot)),
fork_to_digest,
digest_to_fork,
spec: spec.clone(),
}
}

View File

@ -101,6 +101,7 @@ pub mod sqlite;
pub mod blob_sidecar;
pub mod light_client_header;
pub mod non_zero_usize;
pub mod runtime_var_list;
use ethereum_types::{H160, H256};
@ -168,6 +169,7 @@ pub use crate::preset::{AltairPreset, BasePreset, BellatrixPreset, CapellaPreset
pub use crate::proposer_preparation_data::ProposerPreparationData;
pub use crate::proposer_slashing::ProposerSlashing;
pub use crate::relative_epoch::{Error as RelativeEpochError, RelativeEpoch};
pub use crate::runtime_var_list::RuntimeVariableList;
pub use crate::selection_proof::SelectionProof;
pub use crate::shuffling_id::AttestationShufflingId;
pub use crate::signed_aggregate_and_proof::SignedAggregateAndProof;
@ -216,7 +218,7 @@ pub use bls::{
Signature, SignatureBytes,
};
pub use kzg::{KzgCommitment, KzgProof};
pub use kzg::{KzgCommitment, KzgProof, VERSIONED_HASH_VERSION_KZG};
pub use ssz_types::{typenum, typenum::Unsigned, BitList, BitVector, FixedVector, VariableList};
pub use superstruct::superstruct;

View File

@ -0,0 +1,137 @@
use ssz::{Decode, Encode};
use ssz_derive::Encode;
#[derive(Debug, Clone, PartialEq, Encode)]
#[ssz(struct_behaviour = "transparent")]
pub struct RuntimeVariableList<T: Encode> {
vec: Vec<T>,
#[ssz(skip_serializing, skip_deserializing)]
max_len: usize,
}
impl<T: Encode + Decode + Clone> RuntimeVariableList<T> {
pub fn new(vec: Vec<T>, max_len: usize) -> Result<Self, ssz_types::Error> {
if vec.len() <= max_len {
Ok(Self { vec, max_len })
} else {
Err(ssz_types::Error::OutOfBounds {
i: vec.len(),
len: max_len,
})
}
}
pub fn from_vec(mut vec: Vec<T>, max_len: usize) -> Self {
vec.truncate(max_len);
Self { vec, max_len }
}
pub fn to_vec(&self) -> Vec<T> {
self.vec.clone()
}
pub fn as_slice(&self) -> &[T] {
self.vec.as_slice()
}
pub fn len(&self) -> usize {
self.vec.len()
}
pub fn is_empty(&self) -> bool {
self.vec.is_empty()
}
pub fn from_ssz_bytes(bytes: &[u8], max_len: usize) -> Result<Self, ssz::DecodeError> {
let vec = if bytes.is_empty() {
vec![]
} else if <T as Decode>::is_ssz_fixed_len() {
let num_items = bytes
.len()
.checked_div(<T as Decode>::ssz_fixed_len())
.ok_or(ssz::DecodeError::ZeroLengthItem)?;
if num_items > max_len {
return Err(ssz::DecodeError::BytesInvalid(format!(
"VariableList of {} items exceeds maximum of {}",
num_items, max_len
)));
}
bytes
.chunks(<T as Decode>::ssz_fixed_len())
.try_fold(Vec::with_capacity(num_items), |mut vec, chunk| {
vec.push(<T as Decode>::from_ssz_bytes(chunk)?);
Ok(vec)
})
.map(Into::into)?
} else {
ssz::decode_list_of_variable_length_items(bytes, Some(max_len))?
};
Ok(Self { vec, max_len })
}
}
#[cfg(test)]
mod test {
use ssz_types::{typenum::U4, VariableList};
use super::*;
#[test]
fn new() {
let vec = vec![42; 5];
let runtime_var_list: Result<RuntimeVariableList<u64>, _> =
RuntimeVariableList::new(vec, 4);
assert!(runtime_var_list.is_err());
let vec = vec![42; 3];
let runtime_var_list: Result<RuntimeVariableList<u64>, _> =
RuntimeVariableList::new(vec, 4);
assert!(runtime_var_list.is_ok());
let vec = vec![42; 4];
let runtime_var_list: Result<RuntimeVariableList<u64>, _> =
RuntimeVariableList::new(vec, 4);
assert!(runtime_var_list.is_ok());
}
#[test]
fn length() {
let vec = vec![42; 3];
let runtime_var_list: RuntimeVariableList<u64> =
RuntimeVariableList::new(vec.clone(), 4).unwrap();
let var_list: VariableList<u64, U4> = VariableList::from(vec.clone());
assert_eq!(&runtime_var_list.as_slice()[0..3], &vec[..]);
assert_eq!(runtime_var_list.as_slice(), &vec![42, 42, 42][..]);
assert_eq!(runtime_var_list.len(), var_list.len());
let vec = vec![];
let runtime_var_list: RuntimeVariableList<u64> = RuntimeVariableList::new(vec, 4).unwrap();
assert_eq!(runtime_var_list.as_slice(), &[] as &[u64]);
assert!(runtime_var_list.is_empty());
}
#[test]
fn encode() {
let runtime_var_list: RuntimeVariableList<u16> =
RuntimeVariableList::new(vec![0; 2], 2).unwrap();
assert_eq!(runtime_var_list.as_ssz_bytes(), vec![0, 0, 0, 0]);
assert_eq!(<RuntimeVariableList<u16> as Encode>::ssz_fixed_len(), 4);
}
#[test]
fn round_trip() {
let item = RuntimeVariableList::<u16>::new(vec![42; 8], 8).unwrap();
let encoded = &item.as_ssz_bytes();
assert_eq!(item.ssz_bytes_len(), encoded.len());
assert_eq!(RuntimeVariableList::from_ssz_bytes(encoded, 8), Ok(item));
let item = RuntimeVariableList::<u16>::new(vec![0; 8], 8).unwrap();
let encoded = &item.as_ssz_bytes();
assert_eq!(item.ssz_bytes_len(), encoded.len());
assert_eq!(RuntimeVariableList::from_ssz_bytes(encoded, 8), Ok(item));
}
}

View File

@ -4,12 +4,15 @@ mod trusted_setup;
use std::fmt::Debug;
pub use crate::{kzg_commitment::KzgCommitment, kzg_proof::KzgProof, trusted_setup::TrustedSetup};
pub use crate::{
kzg_commitment::{KzgCommitment, VERSIONED_HASH_VERSION_KZG},
kzg_proof::KzgProof,
trusted_setup::TrustedSetup,
};
pub use c_kzg::{
Blob, Bytes32, Bytes48, KzgSettings, BYTES_PER_BLOB, BYTES_PER_COMMITMENT,
BYTES_PER_FIELD_ELEMENT, BYTES_PER_PROOF, FIELD_ELEMENTS_PER_BLOB,
};
#[derive(Debug)]
pub enum Error {
/// An error from the underlying kzg library.