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 task_executor::TaskExecutor;
use types::beacon_block_body::{KzgCommitmentOpts, KzgCommitments}; use types::beacon_block_body::{KzgCommitmentOpts, KzgCommitments};
use types::blob_sidecar::{BlobIdentifier, BlobSidecar, FixedBlobSidecarList}; 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}; use types::{BlobSidecarList, ChainSpec, Epoch, EthSpec, Hash256, SignedBeaconBlock, Slot};
mod availability_view; mod availability_view;
@ -424,7 +423,8 @@ impl<T: BeaconChainTypes> DataAvailabilityChecker<T> {
.map(|current_epoch| { .map(|current_epoch| {
std::cmp::max( std::cmp::max(
fork_epoch, 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( let cutoff_epoch = std::cmp::max(
finalized_epoch + 1, finalized_epoch + 1,
std::cmp::max( 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, deneb_fork_epoch,
), ),
); );

View File

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

View File

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

View File

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

View File

@ -4,7 +4,7 @@ use std::{
time::Duration, time::Duration,
}; };
use super::{methods, rate_limiter::Quota, Protocol}; use super::{rate_limiter::Quota, Protocol};
use serde::{Deserialize, Serialize}; 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_META_DATA_QUOTA: Quota = Quota::n_every(2, 5);
pub const DEFAULT_STATUS_QUOTA: Quota = Quota::n_every(5, 15); pub const DEFAULT_STATUS_QUOTA: Quota = Quota::n_every(5, 15);
pub const DEFAULT_GOODBYE_QUOTA: Quota = Quota::one_every(10); pub const DEFAULT_GOODBYE_QUOTA: Quota = Quota::one_every(10);
pub const DEFAULT_BLOCKS_BY_RANGE_QUOTA: Quota = pub const DEFAULT_BLOCKS_BY_RANGE_QUOTA: Quota = Quota::n_every(1024, 10);
Quota::n_every(methods::MAX_REQUEST_BLOCKS, 10);
pub const DEFAULT_BLOCKS_BY_ROOT_QUOTA: Quota = Quota::n_every(128, 10); pub const DEFAULT_BLOCKS_BY_ROOT_QUOTA: Quota = Quota::n_every(128, 10);
pub const DEFAULT_BLOBS_BY_RANGE_QUOTA: Quota = pub const DEFAULT_BLOBS_BY_RANGE_QUOTA: Quota = Quota::n_every(768, 10);
Quota::n_every(methods::MAX_REQUEST_BLOB_SIDECARS, 10);
pub const DEFAULT_BLOBS_BY_ROOT_QUOTA: Quota = Quota::n_every(128, 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); 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 serde::Serialize;
use ssz::Encode; use ssz::Encode;
use ssz_derive::{Decode, Encode}; use ssz_derive::{Decode, Encode};
use ssz_types::{ use ssz_types::{typenum::U256, VariableList};
typenum::{U1024, U128, U256, U768},
VariableList,
};
use std::marker::PhantomData; use std::marker::PhantomData;
use std::ops::Deref; use std::ops::Deref;
use std::sync::Arc; use std::sync::Arc;
use strum::IntoStaticStr; use strum::IntoStaticStr;
use superstruct::superstruct; use superstruct::superstruct;
use types::blob_sidecar::BlobIdentifier; use types::blob_sidecar::BlobIdentifier;
use types::consts::deneb::MAX_BLOBS_PER_BLOCK;
use types::{ use types::{
blob_sidecar::BlobSidecar, Epoch, EthSpec, Hash256, LightClientBootstrap, SignedBeaconBlock, blob_sidecar::BlobSidecar, ChainSpec, Epoch, EthSpec, Hash256, LightClientBootstrap,
Slot, 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. /// Maximum length of error message.
pub type MaxErrorLen = U256; pub type MaxErrorLen = U256;
pub const MAX_ERROR_LEN: u64 = 256; pub const MAX_ERROR_LEN: u64 = 256;
pub type MaxRequestBlocksDeneb = U128;
pub const MAX_REQUEST_BLOCKS_DENEB: u64 = 128;
pub 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. /// Wrapper over SSZ List to represent error message in rpc responses.
#[derive(Debug, Clone)] #[derive(Debug, Clone)]
pub struct ErrorType(pub VariableList<u8, MaxErrorLen>); pub struct ErrorType(pub VariableList<u8, MaxErrorLen>);
@ -344,22 +330,23 @@ impl OldBlocksByRangeRequest {
} }
/// Request a number of beacon block bodies from a peer. /// Request a number of beacon block bodies from a peer.
#[superstruct( #[superstruct(variants(V1, V2), variant_attributes(derive(Clone, Debug, PartialEq)))]
variants(V1, V2),
variant_attributes(derive(Encode, Decode, Clone, Debug, PartialEq))
)]
#[derive(Clone, Debug, PartialEq)] #[derive(Clone, Debug, PartialEq)]
pub struct BlocksByRootRequest { pub struct BlocksByRootRequest {
/// The list of beacon block bodies being requested. /// The list of beacon block bodies being requested.
pub block_roots: VariableList<Hash256, MaxRequestBlocks>, pub block_roots: RuntimeVariableList<Hash256>,
} }
impl BlocksByRootRequest { 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 }) 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 }) Self::V1(BlocksByRootRequestV1 { block_roots })
} }
} }
@ -368,7 +355,15 @@ impl BlocksByRootRequest {
#[derive(Clone, Debug, PartialEq)] #[derive(Clone, Debug, PartialEq)]
pub struct BlobsByRootRequest { pub struct BlobsByRootRequest {
/// The list of beacon block roots being requested. /// 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 */ /* RPC Handling and Grouping */

View File

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

View File

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

View File

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

View File

@ -1,8 +1,7 @@
use libp2p::gossipsub::{IdentTopic as Topic, TopicHash}; use libp2p::gossipsub::{IdentTopic as Topic, TopicHash};
use serde::{Deserialize, Serialize}; use serde::{Deserialize, Serialize};
use strum::AsRefStr; use strum::AsRefStr;
use types::consts::deneb::BLOB_SIDECAR_SUBNET_COUNT; use types::{ChainSpec, EthSpec, ForkName, SubnetId, SyncSubnetId};
use types::{EthSpec, ForkName, SubnetId, SyncSubnetId};
use crate::Subnet; use crate::Subnet;
@ -44,7 +43,7 @@ pub const LIGHT_CLIENT_GOSSIP_TOPICS: [GossipKind; 2] = [
pub const DENEB_CORE_TOPICS: [GossipKind; 0] = []; pub const DENEB_CORE_TOPICS: [GossipKind; 0] = [];
/// Returns the core topics associated with each fork that are new to the previous fork /// 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 { match fork_name {
ForkName::Base => BASE_CORE_TOPICS.to_vec(), ForkName::Base => BASE_CORE_TOPICS.to_vec(),
ForkName::Altair => ALTAIR_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 => { ForkName::Deneb => {
// All of deneb blob topics are core topics // All of deneb blob topics are core topics
let mut deneb_blob_topics = Vec::new(); 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)); deneb_blob_topics.push(GossipKind::BlobSidecar(i));
} }
let mut deneb_topics = DENEB_CORE_TOPICS.to_vec(); 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 /// 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. /// 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> { pub fn core_topics_to_subscribe<T: EthSpec>(
let mut topics = fork_core_topics::<T>(&current_fork); 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() { 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); topics.extend(previous_fork_topics);
current_fork = previous_fork; current_fork = previous_fork;
} }
@ -435,14 +437,18 @@ mod tests {
#[test] #[test]
fn test_core_topics_to_subscribe() { fn test_core_topics_to_subscribe() {
type E = MainnetEthSpec; type E = MainnetEthSpec;
let spec = E::default_spec();
let mut all_topics = Vec::new(); 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.append(&mut deneb_core_topics);
all_topics.extend(CAPELLA_CORE_TOPICS); all_topics.extend(CAPELLA_CORE_TOPICS);
all_topics.extend(ALTAIR_CORE_TOPICS); all_topics.extend(ALTAIR_CORE_TOPICS);
all_topics.extend(BASE_CORE_TOPICS); all_topics.extend(BASE_CORE_TOPICS);
let latest_fork = *ForkName::list_all().last().unwrap(); 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; .await;
// BlocksByRoot Request // BlocksByRoot Request
let rpc_request = let rpc_request = Request::BlocksByRoot(BlocksByRootRequest::new(
Request::BlocksByRoot(BlocksByRootRequest::new(VariableList::from(vec![ 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),
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 // BlocksByRoot Response
let full_block = BeaconBlock::Base(BeaconBlockBase::<E>::full(&spec)); let full_block = BeaconBlock::Base(BeaconBlockBase::<E>::full(&spec));
@ -876,8 +878,8 @@ fn test_tcp_blocks_by_root_chunked_rpc_terminates_correctly() {
.await; .await;
// BlocksByRoot Request // BlocksByRoot Request
let rpc_request = let rpc_request = Request::BlocksByRoot(BlocksByRootRequest::new(
Request::BlocksByRoot(BlocksByRootRequest::new(VariableList::from(vec![ 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),
@ -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), Hash256::from_low_u64_be(0),
Hash256::from_low_u64_be(0), Hash256::from_low_u64_be(0),
]))); ],
&spec,
));
// BlocksByRoot Response // BlocksByRoot Response
let full_block = BeaconBlock::Base(BeaconBlockBase::<E>::full(&spec)); 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_chain::{BeaconChainError, BeaconChainTypes, HistoricalBlockError, WhenSlotSkipped};
use beacon_processor::SendOnDrop; use beacon_processor::SendOnDrop;
use itertools::process_results; use itertools::process_results;
use lighthouse_network::rpc::methods::{ use lighthouse_network::rpc::methods::{BlobsByRangeRequest, BlobsByRootRequest};
BlobsByRangeRequest, BlobsByRootRequest, MAX_REQUEST_BLOB_SIDECARS, MAX_REQUEST_BLOCKS_DENEB,
};
use lighthouse_network::rpc::StatusMessage; use lighthouse_network::rpc::StatusMessage;
use lighthouse_network::rpc::*; use lighthouse_network::rpc::*;
use lighthouse_network::{PeerId, PeerRequestId, ReportSource, Response, SyncInfo}; use lighthouse_network::{PeerId, PeerRequestId, ReportSource, Response, SyncInfo};
@ -222,12 +220,14 @@ impl<T: BeaconChainTypes> NetworkBeaconProcessor<T> {
request_id: PeerRequestId, request_id: PeerRequestId,
request: BlobsByRootRequest, 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. // No blob ids requested.
return; return;
}; };
let requested_indices = request let requested_indices = request
.blob_ids .blob_ids
.as_slice()
.iter() .iter()
.map(|id| id.index) .map(|id| id.index)
.collect::<Vec<_>>(); .collect::<Vec<_>>();
@ -235,9 +235,9 @@ impl<T: BeaconChainTypes> NetworkBeaconProcessor<T> {
let send_response = true; let send_response = true;
let mut blob_list_results = HashMap::new(); 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. // 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); self.send_response(peer_id, Response::BlobsByRoot(Some(blob)), request_id);
send_blob_count += 1; send_blob_count += 1;
} else { } else {
@ -248,7 +248,7 @@ impl<T: BeaconChainTypes> NetworkBeaconProcessor<T> {
let blob_list_result = match blob_list_results.entry(root) { let blob_list_result = match blob_list_results.entry(root) {
Entry::Vacant(entry) => { 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(), Entry::Occupied(entry) => entry.into_mut(),
}; };
@ -256,7 +256,7 @@ impl<T: BeaconChainTypes> NetworkBeaconProcessor<T> {
match blob_list_result.as_ref() { match blob_list_result.as_ref() {
Ok(blobs_sidecar_list) => { Ok(blobs_sidecar_list) => {
'inner: for blob_sidecar in blobs_sidecar_list.iter() { 'inner: for blob_sidecar in blobs_sidecar_list.iter() {
if blob_sidecar.index == index { if blob_sidecar.index == *index {
self.send_response( self.send_response(
peer_id, peer_id,
Response::BlobsByRoot(Some(blob_sidecar.clone())), Response::BlobsByRoot(Some(blob_sidecar.clone())),
@ -346,14 +346,17 @@ impl<T: BeaconChainTypes> NetworkBeaconProcessor<T> {
); );
// Should not send more than max request blocks // Should not send more than max request blocks
let max_request_size = self.chain.epoch().map_or(MAX_REQUEST_BLOCKS, |epoch| { let max_request_size =
match self.chain.spec.fork_name_at_epoch(epoch) { self.chain
ForkName::Deneb => MAX_REQUEST_BLOCKS_DENEB, .epoch()
ForkName::Base | ForkName::Altair | ForkName::Merge | ForkName::Capella => { .map_or(self.chain.spec.max_request_blocks, |epoch| {
MAX_REQUEST_BLOCKS 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 { if *req.count() > max_request_size {
return self.send_error_response( return self.send_error_response(
peer_id, peer_id,
@ -586,7 +589,7 @@ impl<T: BeaconChainTypes> NetworkBeaconProcessor<T> {
); );
// Should not send more than max request blocks // 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( return self.send_error_response(
peer_id, peer_id,
RPCResponseErrorCode::InvalidRequest, RPCResponseErrorCode::InvalidRequest,

View File

@ -716,9 +716,10 @@ impl<T: BeaconChainTypes> NetworkService<T> {
} }
let mut subscribed_topics: Vec<GossipTopic> = vec![]; let mut subscribed_topics: Vec<GossipTopic> = vec![];
for topic_kind in for topic_kind in core_topics_to_subscribe::<T::EthSpec>(
core_topics_to_subscribe::<T::EthSpec>(self.fork_context.current_fork()) self.fork_context.current_fork(),
{ &self.fork_context.spec,
) {
for fork_digest in self.required_gossip_fork_digests() { for fork_digest in self.required_gossip_fork_digests() {
let topic = GossipTopic::new( let topic = GossipTopic::new(
topic_kind.clone(), topic_kind.clone(),
@ -945,7 +946,10 @@ impl<T: BeaconChainTypes> NetworkService<T> {
} }
fn subscribed_core_topics(&self) -> bool { 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 core_topics: HashSet<&GossipKind> = HashSet::from_iter(&core_topics);
let subscriptions = self.network_globals.gossipsub_subscriptions.read(); let subscriptions = self.network_globals.gossipsub_subscriptions.read();
let subscribed_topics: HashSet<&GossipKind> = 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::methods::BlobsByRootRequest;
use lighthouse_network::rpc::BlocksByRootRequest; use lighthouse_network::rpc::BlocksByRootRequest;
use rand::prelude::IteratorRandom; use rand::prelude::IteratorRandom;
use ssz_types::VariableList;
use std::ops::IndexMut; use std::ops::IndexMut;
use std::sync::Arc; use std::sync::Arc;
use std::time::Duration; use std::time::Duration;
use types::blob_sidecar::{BlobIdentifier, FixedBlobSidecarList}; use types::blob_sidecar::{BlobIdentifier, FixedBlobSidecarList};
use types::{BlobSidecar, EthSpec, Hash256, SignedBeaconBlock}; use types::{BlobSidecar, ChainSpec, EthSpec, Hash256, SignedBeaconBlock};
#[derive(Debug, Copy, Clone)] #[derive(Debug, Copy, Clone)]
pub enum ResponseType { pub enum ResponseType {
@ -87,11 +86,14 @@ pub trait RequestState<L: Lookup, T: BeaconChainTypes> {
/* Request building methods */ /* Request building methods */
/// Construct a new request. /// 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. // Verify and construct request.
self.too_many_attempts()?; self.too_many_attempts()?;
let peer = self.get_peer()?; let peer = self.get_peer()?;
let request = self.new_request(); let request = self.new_request(spec);
Ok((peer, request)) Ok((peer, request))
} }
@ -108,7 +110,7 @@ pub trait RequestState<L: Lookup, T: BeaconChainTypes> {
} }
// Construct request. // Construct request.
let (peer_id, request) = self.build_request()?; let (peer_id, request) = self.build_request(&cx.chain.spec)?;
// Update request state. // Update request state.
self.get_state_mut().state = State::Downloading { peer_id }; self.get_state_mut().state = State::Downloading { peer_id };
@ -151,7 +153,7 @@ pub trait RequestState<L: Lookup, T: BeaconChainTypes> {
} }
/// Initialize `Self::RequestType`. /// Initialize `Self::RequestType`.
fn new_request(&self) -> Self::RequestType; fn new_request(&self, spec: &ChainSpec) -> Self::RequestType;
/// Send the request to the network service. /// Send the request to the network service.
fn make_request( 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 VerifiedResponseType = Arc<SignedBeaconBlock<T::EthSpec>>;
type ReconstructedResponseType = RpcBlock<T::EthSpec>; type ReconstructedResponseType = RpcBlock<T::EthSpec>;
fn new_request(&self) -> BlocksByRootRequest { fn new_request(&self, spec: &ChainSpec) -> BlocksByRootRequest {
BlocksByRootRequest::new(VariableList::from(vec![self.requested_block_root])) BlocksByRootRequest::new(vec![self.requested_block_root], spec)
} }
fn make_request( fn make_request(
@ -353,10 +355,9 @@ impl<L: Lookup, T: BeaconChainTypes> RequestState<L, T> for BlobRequestState<L,
type VerifiedResponseType = FixedBlobSidecarList<T::EthSpec>; type VerifiedResponseType = FixedBlobSidecarList<T::EthSpec>;
type ReconstructedResponseType = 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_id_vec: Vec<BlobIdentifier> = self.requested_ids.clone().into();
let blob_ids = VariableList::from(blob_id_vec); BlobsByRootRequest::new(blob_id_vec, spec)
BlobsByRootRequest { blob_ids }
} }
fn make_request( fn make_request(

View File

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

View File

@ -52,7 +52,6 @@ use beacon_chain::{
AvailabilityProcessingStatus, BeaconChain, BeaconChainTypes, BlockError, EngineState, AvailabilityProcessingStatus, BeaconChain, BeaconChainTypes, BlockError, EngineState,
}; };
use futures::StreamExt; use futures::StreamExt;
use lighthouse_network::rpc::methods::MAX_REQUEST_BLOCKS;
use lighthouse_network::rpc::RPCError; use lighthouse_network::rpc::RPCError;
use lighthouse_network::types::{NetworkGlobals, SyncState}; use lighthouse_network::types::{NetworkGlobals, SyncState};
use lighthouse_network::SyncInfo; use lighthouse_network::SyncInfo;
@ -230,7 +229,7 @@ pub fn spawn<T: BeaconChainTypes>(
log: slog::Logger, log: slog::Logger,
) { ) {
assert!( 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" "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); 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 let indices = blob_request
.blob_ids .blob_ids
.as_slice()
.iter() .iter()
.map(|id| id.index) .map(|id| id.index)
.collect::<Vec<_>>(); .collect::<Vec<_>>();

View File

@ -40,7 +40,6 @@ use std::path::Path;
use std::sync::Arc; use std::sync::Arc;
use std::time::Duration; use std::time::Duration;
use types::blob_sidecar::BlobSidecarList; use types::blob_sidecar::BlobSidecarList;
use types::consts::deneb::MIN_EPOCHS_FOR_BLOB_SIDECARS_REQUESTS;
use types::*; use types::*;
/// On-disk database that stores finalized states efficiently. /// 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_current_epoch = self.get_split_slot().epoch(E::slots_per_epoch()) + 2;
let min_data_availability_boundary = std::cmp::max( let min_data_availability_boundary = std::cmp::max(
deneb_fork_epoch, 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) self.try_prune_blobs(force, min_data_availability_boundary)

View File

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

View File

@ -4,6 +4,7 @@ use int_to_bytes::int_to_bytes4;
use serde::Deserialize; use serde::Deserialize;
use serde::{Deserializer, Serialize, Serializer}; use serde::{Deserializer, Serialize, Serializer};
use serde_utils::quoted_u64::MaybeQuoted; use serde_utils::quoted_u64::MaybeQuoted;
use ssz::Encode;
use std::fs::File; use std::fs::File;
use std::path::Path; use std::path::Path;
use std::time::Duration; use std::time::Duration;
@ -172,22 +173,40 @@ pub struct ChainSpec {
*/ */
pub boot_nodes: Vec<String>, pub boot_nodes: Vec<String>,
pub network_id: u8, pub network_id: u8,
pub attestation_propagation_slot_range: u64,
pub maximum_gossip_clock_disparity_millis: u64,
pub target_aggregators_per_committee: 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 gossip_max_size: u64,
pub max_request_blocks: u64,
pub epochs_per_subnet_subscription: u64,
pub min_epochs_for_block_requests: u64, pub min_epochs_for_block_requests: u64,
pub max_chunk_size: u64, pub max_chunk_size: u64,
pub ttfb_timeout: u64, pub ttfb_timeout: u64,
pub resp_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_invalid_snappy: [u8; 4],
pub message_domain_valid_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_extra_bits: u8,
pub attestation_subnet_prefix_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 * Application params
*/ */
@ -487,6 +506,25 @@ impl ChainSpec {
Duration::from_secs(self.resp_timeout) 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. /// Returns a `ChainSpec` compatible with the Ethereum Foundation specification.
pub fn mainnet() -> Self { pub fn mainnet() -> Self {
Self { Self {
@ -648,12 +686,12 @@ impl ChainSpec {
*/ */
boot_nodes: vec![], boot_nodes: vec![],
network_id: 1, // mainnet network id network_id: 1, // mainnet network id
attestation_propagation_slot_range: 32, attestation_propagation_slot_range: default_attestation_propagation_slot_range(),
attestation_subnet_count: 64, attestation_subnet_count: 64,
subnets_per_node: 2, 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, 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(), gossip_max_size: default_gossip_max_size(),
min_epochs_for_block_requests: default_min_epochs_for_block_requests(), min_epochs_for_block_requests: default_min_epochs_for_block_requests(),
max_chunk_size: default_max_chunk_size(), max_chunk_size: default_max_chunk_size(),
@ -663,6 +701,23 @@ impl ChainSpec {
message_domain_valid_snappy: default_message_domain_valid_snappy(), message_domain_valid_snappy: default_message_domain_valid_snappy(),
attestation_subnet_extra_bits: default_attestation_subnet_extra_bits(), attestation_subnet_extra_bits: default_attestation_subnet_extra_bits(),
attestation_subnet_prefix_bits: default_attestation_subnet_prefix_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 * Application specific
*/ */
@ -892,12 +947,12 @@ impl ChainSpec {
*/ */
boot_nodes: vec![], boot_nodes: vec![],
network_id: 100, // Gnosis Chain network id 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, attestation_subnet_count: 64,
subnets_per_node: 4, // Make this larger than usual to avoid network damage 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, 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(), gossip_max_size: default_gossip_max_size(),
min_epochs_for_block_requests: default_min_epochs_for_block_requests(), min_epochs_for_block_requests: default_min_epochs_for_block_requests(),
max_chunk_size: default_max_chunk_size(), max_chunk_size: default_max_chunk_size(),
@ -907,6 +962,22 @@ impl ChainSpec {
message_domain_valid_snappy: default_message_domain_valid_snappy(), message_domain_valid_snappy: default_message_domain_valid_snappy(),
attestation_subnet_extra_bits: default_attestation_subnet_extra_bits(), attestation_subnet_extra_bits: default_attestation_subnet_extra_bits(),
attestation_subnet_prefix_bits: default_attestation_subnet_prefix_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 * Application specific
@ -1032,6 +1103,12 @@ pub struct Config {
#[serde(default = "default_gossip_max_size")] #[serde(default = "default_gossip_max_size")]
#[serde(with = "serde_utils::quoted_u64")] #[serde(with = "serde_utils::quoted_u64")]
gossip_max_size: 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(default = "default_min_epochs_for_block_requests")]
#[serde(with = "serde_utils::quoted_u64")] #[serde(with = "serde_utils::quoted_u64")]
min_epochs_for_block_requests: u64, min_epochs_for_block_requests: u64,
@ -1044,6 +1121,12 @@ pub struct Config {
#[serde(default = "default_resp_timeout")] #[serde(default = "default_resp_timeout")]
#[serde(with = "serde_utils::quoted_u64")] #[serde(with = "serde_utils::quoted_u64")]
resp_timeout: 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(default = "default_message_domain_invalid_snappy")]
#[serde(with = "serde_utils::bytes_4_hex")] #[serde(with = "serde_utils::bytes_4_hex")]
message_domain_invalid_snappy: [u8; 4], message_domain_invalid_snappy: [u8; 4],
@ -1056,6 +1139,18 @@ pub struct Config {
#[serde(default = "default_attestation_subnet_prefix_bits")] #[serde(default = "default_attestation_subnet_prefix_bits")]
#[serde(with = "serde_utils::quoted_u8")] #[serde(with = "serde_utils::quoted_u8")]
attestation_subnet_prefix_bits: 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] { fn default_bellatrix_fork_version() -> [u8; 4] {
@ -1141,6 +1236,70 @@ const fn default_attestation_subnet_prefix_bits() -> u8 {
6 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 { impl Default for Config {
fn default() -> Self { fn default() -> Self {
let chain_spec = MainnetEthSpec::default_spec(); let chain_spec = MainnetEthSpec::default_spec();
@ -1243,14 +1402,22 @@ impl Config {
deposit_contract_address: spec.deposit_contract_address, deposit_contract_address: spec.deposit_contract_address,
gossip_max_size: spec.gossip_max_size, 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, min_epochs_for_block_requests: spec.min_epochs_for_block_requests,
max_chunk_size: spec.max_chunk_size, max_chunk_size: spec.max_chunk_size,
ttfb_timeout: spec.ttfb_timeout, ttfb_timeout: spec.ttfb_timeout,
resp_timeout: spec.resp_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_invalid_snappy: spec.message_domain_invalid_snappy,
message_domain_valid_snappy: spec.message_domain_valid_snappy, message_domain_valid_snappy: spec.message_domain_valid_snappy,
attestation_subnet_extra_bits: spec.attestation_subnet_extra_bits, attestation_subnet_extra_bits: spec.attestation_subnet_extra_bits,
attestation_subnet_prefix_bits: spec.attestation_subnet_prefix_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, message_domain_valid_snappy,
attestation_subnet_extra_bits, attestation_subnet_extra_bits,
attestation_subnet_prefix_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; } = self;
if preset_base != T::spec_name().to_string().as_str() { if preset_base != T::spec_name().to_string().as_str() {
@ -1356,6 +1531,22 @@ impl Config {
message_domain_valid_snappy, message_domain_valid_snappy,
attestation_subnet_extra_bits, attestation_subnet_extra_bits,
attestation_subnet_prefix_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() ..chain_spec.clone()
}) })
} }

View File

@ -22,11 +22,3 @@ pub mod altair {
pub mod merge { pub mod merge {
pub const INTERVALS_PER_SLOT: u64 = 3; 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>, current_fork: RwLock<ForkName>,
fork_to_digest: HashMap<ForkName, [u8; 4]>, fork_to_digest: HashMap<ForkName, [u8; 4]>,
digest_to_fork: HashMap<[u8; 4], ForkName>, digest_to_fork: HashMap<[u8; 4], ForkName>,
pub spec: ChainSpec,
} }
impl ForkContext { impl ForkContext {
@ -73,6 +74,7 @@ impl ForkContext {
current_fork: RwLock::new(spec.fork_name_at_slot::<T>(current_slot)), current_fork: RwLock::new(spec.fork_name_at_slot::<T>(current_slot)),
fork_to_digest, fork_to_digest,
digest_to_fork, digest_to_fork,
spec: spec.clone(),
} }
} }

View File

@ -101,6 +101,7 @@ pub mod sqlite;
pub mod blob_sidecar; pub mod blob_sidecar;
pub mod light_client_header; pub mod light_client_header;
pub mod non_zero_usize; pub mod non_zero_usize;
pub mod runtime_var_list;
use ethereum_types::{H160, H256}; 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_preparation_data::ProposerPreparationData;
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};
pub use crate::runtime_var_list::RuntimeVariableList;
pub use crate::selection_proof::SelectionProof; pub use crate::selection_proof::SelectionProof;
pub use crate::shuffling_id::AttestationShufflingId; pub use crate::shuffling_id::AttestationShufflingId;
pub use crate::signed_aggregate_and_proof::SignedAggregateAndProof; pub use crate::signed_aggregate_and_proof::SignedAggregateAndProof;
@ -216,7 +218,7 @@ pub use bls::{
Signature, SignatureBytes, 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 ssz_types::{typenum, typenum::Unsigned, BitList, BitVector, FixedVector, VariableList};
pub use superstruct::superstruct; 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; 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::{ pub use c_kzg::{
Blob, Bytes32, Bytes48, KzgSettings, BYTES_PER_BLOB, BYTES_PER_COMMITMENT, Blob, Bytes32, Bytes48, KzgSettings, BYTES_PER_BLOB, BYTES_PER_COMMITMENT,
BYTES_PER_FIELD_ELEMENT, BYTES_PER_PROOF, FIELD_ELEMENTS_PER_BLOB, BYTES_PER_FIELD_ELEMENT, BYTES_PER_PROOF, FIELD_ELEMENTS_PER_BLOB,
}; };
#[derive(Debug)] #[derive(Debug)]
pub enum Error { pub enum Error {
/// An error from the underlying kzg library. /// An error from the underlying kzg library.