diff --git a/beacon_node/beacon_chain/src/beacon_chain.rs b/beacon_node/beacon_chain/src/beacon_chain.rs index 2262325642d..13022b82699 100644 --- a/beacon_node/beacon_chain/src/beacon_chain.rs +++ b/beacon_node/beacon_chain/src/beacon_chain.rs @@ -3851,6 +3851,8 @@ impl BeaconChain { } if let Some(data_columns) = data_columns { + // TODO(das): `available_block includes all sampled columns, but we only need to store + // custody columns. To be clarified in spec. if !data_columns.is_empty() { debug!( self.log, "Writing data_columns to store"; diff --git a/beacon_node/beacon_chain/src/data_availability_checker.rs b/beacon_node/beacon_chain/src/data_availability_checker.rs index 4d5afdc8904..395f40c5dba 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker.rs @@ -108,13 +108,15 @@ impl DataAvailabilityChecker { spec.custody_requirement as usize }; - let custody_column_count = - custody_subnet_count.saturating_mul(spec.data_columns_per_subnet()); + let subnet_sampling_size = + std::cmp::max(custody_subnet_count, spec.samples_per_slot as usize); + let sampling_column_count = + subnet_sampling_size.saturating_mul(spec.data_columns_per_subnet()); let inner = DataAvailabilityCheckerInner::new( OVERFLOW_LRU_CAPACITY, store, - custody_column_count, + sampling_column_count, spec.clone(), )?; Ok(Self { @@ -125,10 +127,8 @@ impl DataAvailabilityChecker { }) } - pub fn get_custody_columns_count(&self) -> usize { - self.availability_cache - .custody_subnet_count() - .saturating_mul(self.spec.data_columns_per_subnet()) + pub fn get_sampling_column_count(&self) -> usize { + self.availability_cache.sampling_column_count() } /// Checks if the block root is currenlty in the availability cache awaiting import because @@ -141,9 +141,9 @@ impl DataAvailabilityChecker { .get_execution_valid_block(block_root) } - /// Return the set of imported blob indexes for `block_root`. Returns None if there is no block + /// Return the set of cached blob indexes for `block_root`. Returns None if there is no block /// component for `block_root`. - pub fn imported_blob_indexes(&self, block_root: &Hash256) -> Option> { + pub fn cached_blob_indexes(&self, block_root: &Hash256) -> Option> { self.availability_cache .peek_pending_components(block_root, |components| { components.map(|components| { @@ -156,9 +156,9 @@ impl DataAvailabilityChecker { }) } - /// Return the set of imported custody column indexes for `block_root`. Returns None if there is + /// Return the set of cached custody column indexes for `block_root`. Returns None if there is /// no block component for `block_root`. - pub fn imported_custody_column_indexes(&self, block_root: &Hash256) -> Option> { + pub fn cached_data_column_indexes(&self, block_root: &Hash256) -> Option> { self.availability_cache .peek_pending_components(block_root, |components| { components.map(|components| components.get_cached_data_columns_indices()) diff --git a/beacon_node/beacon_chain/src/data_availability_checker/overflow_lru_cache.rs b/beacon_node/beacon_chain/src/data_availability_checker/overflow_lru_cache.rs index 46ab08a8215..8f91bf34fcc 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker/overflow_lru_cache.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker/overflow_lru_cache.rs @@ -40,7 +40,7 @@ pub struct PendingComponents { pub enum BlockImportRequirement { AllBlobs, - CustodyColumns(usize), + ColumnSampling(usize), } impl PendingComponents { @@ -210,7 +210,7 @@ impl PendingComponents { .map_or(false, |num_expected_blobs| { num_expected_blobs == self.num_received_blobs() }), - BlockImportRequirement::CustodyColumns(num_expected_columns) => { + BlockImportRequirement::ColumnSampling(num_expected_columns) => { let num_received_data_columns = self.num_received_data_columns(); // No data columns when there are 0 blobs self.num_expected_blobs() @@ -281,7 +281,7 @@ impl PendingComponents { }; (Some(VariableList::new(verified_blobs)?), None) } - BlockImportRequirement::CustodyColumns(_) => { + BlockImportRequirement::ColumnSampling(_) => { let verified_data_columns = verified_data_columns .into_iter() .map(|d| d.into_inner()) @@ -353,8 +353,8 @@ pub struct DataAvailabilityCheckerInner { /// This cache holds a limited number of states in memory and reconstructs them /// from disk when necessary. This is necessary until we merge tree-states state_cache: StateLRUCache, - /// The number of data columns the node is custodying. - custody_column_count: usize, + /// The number of data columns the node is sampling via subnet sampling. + sampling_column_count: usize, spec: Arc, } @@ -362,19 +362,19 @@ impl DataAvailabilityCheckerInner { pub fn new( capacity: NonZeroUsize, beacon_store: BeaconStore, - custody_column_count: usize, + sampling_column_count: usize, spec: Arc, ) -> Result { Ok(Self { critical: RwLock::new(LruCache::new(capacity)), state_cache: StateLRUCache::new(beacon_store, spec.clone()), - custody_column_count, + sampling_column_count, spec, }) } - pub fn custody_subnet_count(&self) -> usize { - self.custody_column_count + pub fn sampling_column_count(&self) -> usize { + self.sampling_column_count } /// Returns true if the block root is known, without altering the LRU ordering @@ -440,8 +440,8 @@ impl DataAvailabilityCheckerInner { ) -> Result { let peer_das_enabled = self.spec.is_peer_das_enabled_for_epoch(epoch); if peer_das_enabled { - Ok(BlockImportRequirement::CustodyColumns( - self.custody_column_count, + Ok(BlockImportRequirement::ColumnSampling( + self.sampling_column_count, )) } else { Ok(BlockImportRequirement::AllBlobs) @@ -456,7 +456,7 @@ impl DataAvailabilityCheckerInner { block_import_requirement: &BlockImportRequirement, pending_components: &PendingComponents, ) -> bool { - let BlockImportRequirement::CustodyColumns(num_expected_columns) = block_import_requirement + let BlockImportRequirement::ColumnSampling(num_expected_columns) = block_import_requirement else { return false; }; diff --git a/beacon_node/beacon_chain/src/historical_blocks.rs b/beacon_node/beacon_chain/src/historical_blocks.rs index 1372211b175..a23b6ddc1e5 100644 --- a/beacon_node/beacon_chain/src/historical_blocks.rs +++ b/beacon_node/beacon_chain/src/historical_blocks.rs @@ -94,7 +94,9 @@ impl BeaconChain { // Blobs are stored per block, and data columns are each stored individually let n_blob_ops_per_block = if self.spec.is_peer_das_scheduled() { - self.data_availability_checker.get_custody_columns_count() + // TODO(das): `available_block includes all sampled columns, but we only need to store + // custody columns. To be clarified in spec PR. + self.data_availability_checker.get_sampling_column_count() } else { 1 }; diff --git a/beacon_node/http_api/src/publish_blocks.rs b/beacon_node/http_api/src/publish_blocks.rs index 16364b435a9..fceeb2dd231 100644 --- a/beacon_node/http_api/src/publish_blocks.rs +++ b/beacon_node/http_api/src/publish_blocks.rs @@ -389,18 +389,17 @@ pub async fn publish_block>( .count() > 0 { - let custody_columns_indices = &network_globals.custody_columns; - - let custody_columns = gossip_verified_data_columns + let sampling_columns_indices = &network_globals.sampling_columns; + let sampling_columns = gossip_verified_data_columns .into_iter() .flatten() - .filter(|data_column| custody_columns_indices.contains(&data_column.index())) + .filter(|data_column| sampling_columns_indices.contains(&data_column.index())) .collect(); // Importing the columns could trigger block import and network publication in the case // where the block was already seen on gossip. if let Err(e) = - Box::pin(chain.process_gossip_data_columns(custody_columns, publish_fn)).await + Box::pin(chain.process_gossip_data_columns(sampling_columns, publish_fn)).await { let msg = format!("Invalid data column: {e}"); return if let BroadcastValidation::Gossip = validation_level { diff --git a/beacon_node/lighthouse_network/src/types/globals.rs b/beacon_node/lighthouse_network/src/types/globals.rs index f271c9ff722..bcebd02a0ed 100644 --- a/beacon_node/lighthouse_network/src/types/globals.rs +++ b/beacon_node/lighthouse_network/src/types/globals.rs @@ -26,9 +26,9 @@ pub struct NetworkGlobals { pub sync_state: RwLock, /// The current state of the backfill sync. pub backfill_state: RwLock, - /// The computed custody subnets and columns is stored to avoid re-computing. - pub custody_subnets: Vec, - pub custody_columns: Vec, + /// The computed sampling subnets and columns is stored to avoid re-computing. + pub sampling_subnets: Vec, + pub sampling_columns: Vec, /// Network-related configuration. Immutable after initialization. pub config: Arc, /// Ethereum chain configuration. Immutable after initialization. @@ -45,24 +45,31 @@ impl NetworkGlobals { config: Arc, spec: Arc, ) -> Self { - let (custody_subnets, custody_columns) = if spec.is_peer_das_scheduled() { + let (sampling_subnets, sampling_columns) = if spec.is_peer_das_scheduled() { + let node_id = enr.node_id().raw(); + let custody_subnet_count = local_metadata .custody_subnet_count() .copied() .expect("custody subnet count must be set if PeerDAS is scheduled"); - let custody_subnets = DataColumnSubnetId::compute_custody_subnets::( - enr.node_id().raw(), - custody_subnet_count, + + let subnet_sampling_size = std::cmp::max(custody_subnet_count, spec.samples_per_slot); + + let sampling_subnets = DataColumnSubnetId::compute_custody_subnets::( + node_id, + subnet_sampling_size, &spec, ) - .expect("custody subnet count must be valid") + .expect("sampling subnet count must be valid") .collect::>(); - let custody_columns = custody_subnets + + let sampling_columns = sampling_subnets .iter() .flat_map(|subnet| subnet.columns::(&spec)) .sorted() .collect(); - (custody_subnets, custody_columns) + + (sampling_subnets, sampling_columns) } else { (vec![], vec![]) }; @@ -76,8 +83,8 @@ impl NetworkGlobals { gossipsub_subscriptions: RwLock::new(HashSet::new()), sync_state: RwLock::new(SyncState::Stalled), backfill_state: RwLock::new(BackFillState::NotRequired), - custody_subnets, - custody_columns, + sampling_subnets, + sampling_columns, config, spec, } @@ -197,12 +204,13 @@ mod test { use types::{Epoch, EthSpec, MainnetEthSpec as E}; #[test] - fn test_custody_subnets() { + fn test_sampling_subnets() { let log = logging::test_logger(); let mut spec = E::default_spec(); spec.eip7594_fork_epoch = Some(Epoch::new(0)); let custody_subnet_count = spec.data_column_sidecar_subnet_count / 2; + let subnet_sampling_size = std::cmp::max(custody_subnet_count, spec.samples_per_slot); let metadata = get_metadata(custody_subnet_count); let config = Arc::new(NetworkConfig::default()); @@ -213,17 +221,20 @@ mod test { config, Arc::new(spec), ); - assert_eq!(globals.custody_subnets.len(), custody_subnet_count as usize); + assert_eq!( + globals.sampling_subnets.len(), + subnet_sampling_size as usize + ); } #[test] - fn test_custody_columns() { + fn test_sampling_columns() { let log = logging::test_logger(); let mut spec = E::default_spec(); spec.eip7594_fork_epoch = Some(Epoch::new(0)); let custody_subnet_count = spec.data_column_sidecar_subnet_count / 2; - let custody_columns_count = spec.number_of_columns / 2; + let subnet_sampling_size = std::cmp::max(custody_subnet_count, spec.samples_per_slot); let metadata = get_metadata(custody_subnet_count); let config = Arc::new(NetworkConfig::default()); @@ -234,7 +245,10 @@ mod test { config, Arc::new(spec), ); - assert_eq!(globals.custody_columns.len(), custody_columns_count); + assert_eq!( + globals.sampling_columns.len(), + subnet_sampling_size as usize + ); } fn get_metadata(custody_subnet_count: u64) -> MetaData { diff --git a/beacon_node/network/src/service.rs b/beacon_node/network/src/service.rs index f36d11ecdd1..5a66cb7f30d 100644 --- a/beacon_node/network/src/service.rs +++ b/beacon_node/network/src/service.rs @@ -813,7 +813,7 @@ impl NetworkService { } } } else { - for column_subnet in &self.network_globals.custody_subnets { + for column_subnet in &self.network_globals.sampling_subnets { for fork_digest in self.required_gossip_fork_digests() { let gossip_kind = Subnet::DataColumn(*column_subnet).into(); let topic = diff --git a/beacon_node/network/src/sync/block_lookups/tests.rs b/beacon_node/network/src/sync/block_lookups/tests.rs index 4c73e8f8d03..151333a2ef5 100644 --- a/beacon_node/network/src/sync/block_lookups/tests.rs +++ b/beacon_node/network/src/sync/block_lookups/tests.rs @@ -1,6 +1,6 @@ use crate::network_beacon_processor::NetworkBeaconProcessor; use crate::sync::manager::{BlockProcessType, SyncManager}; -use crate::sync::sampling::SamplingConfig; +use crate::sync::peer_sampling::SamplingConfig; use crate::sync::{SamplingId, SyncMessage}; use crate::NetworkMessage; use std::sync::Arc; @@ -2037,9 +2037,10 @@ fn custody_lookup_happy_path() { // Should not request blobs let id = r.expect_block_lookup_request(block.canonical_root()); r.complete_valid_block_request(id, block.into(), true); - let custody_column_count = spec.custody_requirement * spec.data_columns_per_subnet() as u64; + // for each slot we download `samples_per_slot` columns + let sample_column_count = spec.samples_per_slot * spec.data_columns_per_subnet() as u64; let custody_ids = - r.expect_only_data_columns_by_root_requests(block_root, custody_column_count as usize); + r.expect_only_data_columns_by_root_requests(block_root, sample_column_count as usize); r.complete_valid_custody_request(custody_ids, data_columns, false); r.expect_no_active_lookups(); } diff --git a/beacon_node/network/src/sync/manager.rs b/beacon_node/network/src/sync/manager.rs index f1417804849..c1f4fe54fb7 100644 --- a/beacon_node/network/src/sync/manager.rs +++ b/beacon_node/network/src/sync/manager.rs @@ -38,9 +38,9 @@ use super::block_lookups::BlockLookups; use super::network_context::{ BlockOrBlob, CustodyByRootResult, RangeRequestId, RpcEvent, SyncNetworkContext, }; +use super::peer_sampling::{Sampling, SamplingConfig, SamplingResult}; use super::peer_sync_info::{remote_sync_type, PeerSyncType}; use super::range_sync::{RangeSync, RangeSyncType, EPOCHS_PER_BATCH}; -use super::sampling::{Sampling, SamplingConfig, SamplingResult}; use crate::network_beacon_processor::{ChainSegmentProcessId, NetworkBeaconProcessor}; use crate::service::NetworkMessage; use crate::status::ToStatusMessage; diff --git a/beacon_node/network/src/sync/mod.rs b/beacon_node/network/src/sync/mod.rs index 6669add4453..1dca6f02ac2 100644 --- a/beacon_node/network/src/sync/mod.rs +++ b/beacon_node/network/src/sync/mod.rs @@ -6,9 +6,9 @@ mod block_lookups; mod block_sidecar_coupling; pub mod manager; mod network_context; +mod peer_sampling; mod peer_sync_info; mod range_sync; -mod sampling; pub use lighthouse_network::service::api_types::SamplingId; pub use manager::{BatchProcessResult, SyncMessage}; diff --git a/beacon_node/network/src/sync/network_context.rs b/beacon_node/network/src/sync/network_context.rs index dc35a141d21..492b703f8a2 100644 --- a/beacon_node/network/src/sync/network_context.rs +++ b/beacon_node/network/src/sync/network_context.rs @@ -418,13 +418,13 @@ impl SyncNetworkContext { false }; - let (expects_custody_columns, num_of_custody_column_req) = + let (expects_columns, num_of_column_req) = if matches!(batch_type, ByRangeRequestType::BlocksAndColumns) { - let custody_indexes = self.network_globals().custody_columns.clone(); + let column_indexes = self.network_globals().sampling_columns.clone(); let mut num_of_custody_column_req = 0; for (peer_id, columns_by_range_request) in - self.make_columns_by_range_requests(request, &custody_indexes)? + self.make_columns_by_range_requests(request, &column_indexes)? { requested_peers.push(peer_id); @@ -448,15 +448,15 @@ impl SyncNetworkContext { num_of_custody_column_req += 1; } - (Some(custody_indexes), Some(num_of_custody_column_req)) + (Some(column_indexes), Some(num_of_custody_column_req)) } else { (None, None) }; let info = RangeBlockComponentsRequest::new( expected_blobs, - expects_custody_columns, - num_of_custody_column_req, + expects_columns, + num_of_column_req, requested_peers, ); self.range_block_components_requests @@ -668,7 +668,7 @@ impl SyncNetworkContext { let imported_blob_indexes = self .chain .data_availability_checker - .imported_blob_indexes(&block_root) + .cached_blob_indexes(&block_root) .unwrap_or_default(); // Include only the blob indexes not yet imported (received through gossip) let indices = (0..expected_blobs as u64) @@ -786,13 +786,13 @@ impl SyncNetworkContext { let custody_indexes_imported = self .chain .data_availability_checker - .imported_custody_column_indexes(&block_root) + .cached_data_column_indexes(&block_root) .unwrap_or_default(); // Include only the blob indexes not yet imported (received through gossip) let custody_indexes_to_fetch = self .network_globals() - .custody_columns + .sampling_columns .clone() .into_iter() .filter(|index| !custody_indexes_imported.contains(index)) diff --git a/beacon_node/network/src/sync/sampling.rs b/beacon_node/network/src/sync/peer_sampling.rs similarity index 100% rename from beacon_node/network/src/sync/sampling.rs rename to beacon_node/network/src/sync/peer_sampling.rs diff --git a/beacon_node/network/src/sync/range_sync/chain.rs b/beacon_node/network/src/sync/range_sync/chain.rs index ed5946ada72..732e4a7bd16 100644 --- a/beacon_node/network/src/sync/range_sync/chain.rs +++ b/beacon_node/network/src/sync/range_sync/chain.rs @@ -444,9 +444,9 @@ impl SyncingChain { self.request_batches(network)?; } } - } else if !self.good_peers_on_custody_subnets(self.processing_target, network) { + } else if !self.good_peers_on_sampling_subnets(self.processing_target, network) { // This is to handle the case where no batch was sent for the current processing - // target when there is no custody peers available. This is a valid state and should not + // target when there is no sampling peers available. This is a valid state and should not // return an error. return Ok(KeepChain); } else { @@ -1075,10 +1075,10 @@ impl SyncingChain { // check if we have the batch for our optimistic start. If not, request it first. // We wait for this batch before requesting any other batches. if let Some(epoch) = self.optimistic_start { - if !self.good_peers_on_custody_subnets(epoch, network) { + if !self.good_peers_on_sampling_subnets(epoch, network) { debug!( self.log, - "Waiting for peers to be available on custody column subnets" + "Waiting for peers to be available on sampling column subnets" ); return Ok(KeepChain); } @@ -1107,14 +1107,18 @@ impl SyncingChain { Ok(KeepChain) } - /// Checks all custody column subnets for peers. Returns `true` if there is at least one peer in - /// every custody column subnet. - fn good_peers_on_custody_subnets(&self, epoch: Epoch, network: &SyncNetworkContext) -> bool { + /// Checks all sampling column subnets for peers. Returns `true` if there is at least one peer in + /// every sampling column subnet. + fn good_peers_on_sampling_subnets( + &self, + epoch: Epoch, + network: &SyncNetworkContext, + ) -> bool { if network.chain.spec.is_peer_das_enabled_for_epoch(epoch) { - // Require peers on all custody column subnets before sending batches + // Require peers on all sampling column subnets before sending batches let peers_on_all_custody_subnets = network .network_globals() - .custody_subnets + .sampling_subnets .iter() .all(|subnet_id| { let peer_count = network @@ -1167,11 +1171,11 @@ impl SyncingChain { return None; } - // don't send batch requests until we have peers on custody subnets + // don't send batch requests until we have peers on sampling subnets // TODO(das): this is a workaround to avoid sending out excessive block requests because // block and data column requests are currently coupled. This can be removed once we find a // way to decouple the requests and do retries individually, see issue #6258. - if !self.good_peers_on_custody_subnets(self.to_be_downloaded, network) { + if !self.good_peers_on_sampling_subnets(self.to_be_downloaded, network) { debug!( self.log, "Waiting for peers to be available on custody column subnets" diff --git a/common/eth2_network_config/built_in_network_configs/chiado/config.yaml b/common/eth2_network_config/built_in_network_configs/chiado/config.yaml index 74fca4c5010..1eca01bbeef 100644 --- a/common/eth2_network_config/built_in_network_configs/chiado/config.yaml +++ b/common/eth2_network_config/built_in_network_configs/chiado/config.yaml @@ -140,4 +140,5 @@ BLOB_SIDECAR_SUBNET_COUNT: 6 # DAS CUSTODY_REQUIREMENT: 4 DATA_COLUMN_SIDECAR_SUBNET_COUNT: 128 -NUMBER_OF_COLUMNS: 128 \ No newline at end of file +NUMBER_OF_COLUMNS: 128 +SAMPLES_PER_SLOT: 8 \ No newline at end of file diff --git a/common/eth2_network_config/built_in_network_configs/gnosis/config.yaml b/common/eth2_network_config/built_in_network_configs/gnosis/config.yaml index 07bd21b35c2..500555a2694 100644 --- a/common/eth2_network_config/built_in_network_configs/gnosis/config.yaml +++ b/common/eth2_network_config/built_in_network_configs/gnosis/config.yaml @@ -123,4 +123,5 @@ BLOB_SIDECAR_SUBNET_COUNT: 6 # DAS CUSTODY_REQUIREMENT: 4 DATA_COLUMN_SIDECAR_SUBNET_COUNT: 128 -NUMBER_OF_COLUMNS: 128 \ No newline at end of file +NUMBER_OF_COLUMNS: 128 +SAMPLES_PER_SLOT: 8 \ No newline at end of file diff --git a/common/eth2_network_config/built_in_network_configs/holesky/config.yaml b/common/eth2_network_config/built_in_network_configs/holesky/config.yaml index 67f1e5b6831..d67d77d3bea 100644 --- a/common/eth2_network_config/built_in_network_configs/holesky/config.yaml +++ b/common/eth2_network_config/built_in_network_configs/holesky/config.yaml @@ -127,4 +127,5 @@ BLOB_SIDECAR_SUBNET_COUNT: 6 # DAS CUSTODY_REQUIREMENT: 4 DATA_COLUMN_SIDECAR_SUBNET_COUNT: 128 -NUMBER_OF_COLUMNS: 128 \ No newline at end of file +NUMBER_OF_COLUMNS: 128 +SAMPLES_PER_SLOT: 8 \ No newline at end of file diff --git a/common/eth2_network_config/built_in_network_configs/mainnet/config.yaml b/common/eth2_network_config/built_in_network_configs/mainnet/config.yaml index acf4d83f323..18591fecdcd 100644 --- a/common/eth2_network_config/built_in_network_configs/mainnet/config.yaml +++ b/common/eth2_network_config/built_in_network_configs/mainnet/config.yaml @@ -149,4 +149,5 @@ BLOB_SIDECAR_SUBNET_COUNT: 6 # DAS CUSTODY_REQUIREMENT: 4 DATA_COLUMN_SIDECAR_SUBNET_COUNT: 128 -NUMBER_OF_COLUMNS: 128 \ No newline at end of file +NUMBER_OF_COLUMNS: 128 +SAMPLES_PER_SLOT: 8 \ No newline at end of file diff --git a/common/eth2_network_config/built_in_network_configs/sepolia/config.yaml b/common/eth2_network_config/built_in_network_configs/sepolia/config.yaml index 8b84d870103..b08a6180bf0 100644 --- a/common/eth2_network_config/built_in_network_configs/sepolia/config.yaml +++ b/common/eth2_network_config/built_in_network_configs/sepolia/config.yaml @@ -123,4 +123,5 @@ BLOB_SIDECAR_SUBNET_COUNT: 6 # DAS CUSTODY_REQUIREMENT: 4 DATA_COLUMN_SIDECAR_SUBNET_COUNT: 128 -NUMBER_OF_COLUMNS: 128 \ No newline at end of file +NUMBER_OF_COLUMNS: 128 +SAMPLES_PER_SLOT: 8 \ No newline at end of file diff --git a/consensus/types/src/chain_spec.rs b/consensus/types/src/chain_spec.rs index e31427121ec..7e933eea197 100644 --- a/consensus/types/src/chain_spec.rs +++ b/consensus/types/src/chain_spec.rs @@ -198,6 +198,7 @@ pub struct ChainSpec { pub custody_requirement: u64, pub data_column_sidecar_subnet_count: u64, pub number_of_columns: usize, + pub samples_per_slot: u64, /* * Networking @@ -811,6 +812,7 @@ impl ChainSpec { custody_requirement: 4, data_column_sidecar_subnet_count: 128, number_of_columns: 128, + samples_per_slot: 8, /* * Network specific @@ -1132,6 +1134,7 @@ impl ChainSpec { custody_requirement: 4, data_column_sidecar_subnet_count: 128, number_of_columns: 128, + samples_per_slot: 8, /* * Network specific */ @@ -1382,6 +1385,9 @@ pub struct Config { #[serde(default = "default_number_of_columns")] #[serde(with = "serde_utils::quoted_u64")] number_of_columns: u64, + #[serde(default = "default_samples_per_slot")] + #[serde(with = "serde_utils::quoted_u64")] + samples_per_slot: u64, } fn default_bellatrix_fork_version() -> [u8; 4] { @@ -1521,17 +1527,21 @@ const fn default_maximum_gossip_clock_disparity_millis() -> u64 { } const fn default_custody_requirement() -> u64 { - 1 + 4 } const fn default_data_column_sidecar_subnet_count() -> u64 { - 32 + 128 } const fn default_number_of_columns() -> u64 { 128 } +const fn default_samples_per_slot() -> u64 { + 8 +} + fn max_blocks_by_root_request_common(max_request_blocks: u64) -> usize { let max_request_blocks = max_request_blocks as usize; RuntimeVariableList::::from_vec( @@ -1727,6 +1737,7 @@ impl Config { custody_requirement: spec.custody_requirement, data_column_sidecar_subnet_count: spec.data_column_sidecar_subnet_count, number_of_columns: spec.number_of_columns as u64, + samples_per_slot: spec.samples_per_slot, } } @@ -1802,6 +1813,7 @@ impl Config { custody_requirement, data_column_sidecar_subnet_count, number_of_columns, + samples_per_slot, } = self; if preset_base != E::spec_name().to_string().as_str() { @@ -1881,6 +1893,7 @@ impl Config { custody_requirement, data_column_sidecar_subnet_count, number_of_columns: number_of_columns as usize, + samples_per_slot, ..chain_spec.clone() }) @@ -2125,6 +2138,7 @@ mod yaml_tests { CUSTODY_REQUIREMENT: 1 DATA_COLUMN_SIDECAR_SUBNET_COUNT: 128 NUMBER_OF_COLUMNS: 128 + SAMPLES_PER_SLOT: 8 "#; let chain_spec: Config = serde_yaml::from_str(spec).unwrap(); diff --git a/lighthouse/environment/tests/testnet_dir/config.yaml b/lighthouse/environment/tests/testnet_dir/config.yaml index 84e8274f06e..34e42a61f67 100644 --- a/lighthouse/environment/tests/testnet_dir/config.yaml +++ b/lighthouse/environment/tests/testnet_dir/config.yaml @@ -102,4 +102,5 @@ ATTESTATION_SUBNET_SHUFFLING_PREFIX_BITS: 3 # DAS CUSTODY_REQUIREMENT: 4 DATA_COLUMN_SIDECAR_SUBNET_COUNT: 128 -NUMBER_OF_COLUMNS: 128 \ No newline at end of file +NUMBER_OF_COLUMNS: 128 +SAMPLES_PER_SLOT: 8 \ No newline at end of file