Rate limiting backfill sync (#3936)

## Issue Addressed

#3212 

## Proposed Changes

- Introduce a new `rate_limiting_backfill_queue` - any new inbound backfill work events gets immediately sent to this FIFO queue **without any processing**
- Spawn a `backfill_scheduler` routine that pops a backfill event from the FIFO queue at specified intervals (currently halfway through a slot, or at 6s after slot start for 12s slots) and sends the event to `BeaconProcessor` via a `scheduled_backfill_work_tx` channel
- This channel gets polled last in the `InboundEvents`, and work event received is  wrapped in a `InboundEvent::ScheduledBackfillWork` enum variant, which gets processed immediately or queued by the `BeaconProcessor` (existing logic applies from here)

Diagram comparing backfill processing with / without rate-limiting: 
https://github.com/sigp/lighthouse/issues/3212#issuecomment-1386249922

See this comment for @paulhauner's  explanation and solution: https://github.com/sigp/lighthouse/issues/3212#issuecomment-1384674956

## Additional Info

I've compared this branch (with backfill processing rate limited to to 1 and 3 batches per slot) against the latest stable version. The CPU usage during backfill sync is reduced by ~5% - 20%, more details on this page:

https://hackmd.io/@jimmygchen/SJuVpJL3j

The above testing is done on Goerli (as I don't currently have hardware for Mainnet), I'm guessing the differences are likely to be bigger on mainnet due to block size.

### TODOs

- [x] Experiment with processing multiple batches per slot. (need to think about how to do this for different slot durations)
- [x] Add option to disable rate-limiting, enabed by default.
- [x] (No longer required now we're reusing the reprocessing queue) Complete the `backfill_scheduler` task when backfill sync is completed or not required
This commit is contained in:
Jimmy Chen 2023-04-03 03:02:55 +00:00
parent c5383e393a
commit 2de3451011
10 changed files with 380 additions and 24 deletions

View File

@ -2893,7 +2893,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
metrics::start_timer(&metrics::FORK_CHOICE_PROCESS_BLOCK_TIMES);
let block_delay = self
.slot_clock
.seconds_from_current_slot_start(self.spec.seconds_per_slot)
.seconds_from_current_slot_start()
.ok_or(Error::UnableToComputeTimeAtSlot)?;
fork_choice
@ -3746,7 +3746,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
let slot_delay = self
.slot_clock
.seconds_from_current_slot_start(self.spec.seconds_per_slot)
.seconds_from_current_slot_start()
.or_else(|| {
warn!(
self.log,

View File

@ -68,6 +68,8 @@ pub struct ChainConfig {
///
/// This is useful for block builders and testing.
pub always_prepare_payload: bool,
/// Whether backfill sync processing should be rate-limited.
pub enable_backfill_rate_limiting: bool,
}
impl Default for ChainConfig {
@ -94,6 +96,7 @@ impl Default for ChainConfig {
optimistic_finalized_sync: true,
shuffling_cache_size: crate::shuffling_cache::DEFAULT_CACHE_SIZE,
always_prepare_payload: false,
enable_backfill_rate_limiting: true,
}
}
}

View File

@ -61,6 +61,7 @@ use std::time::Duration;
use std::{cmp, collections::HashSet};
use task_executor::TaskExecutor;
use tokio::sync::mpsc;
use tokio::sync::mpsc::error::TrySendError;
use types::{
Attestation, AttesterSlashing, Hash256, LightClientFinalityUpdate, LightClientOptimisticUpdate,
ProposerSlashing, SignedAggregateAndProof, SignedBeaconBlock, SignedBlsToExecutionChange,
@ -77,7 +78,9 @@ mod tests;
mod work_reprocessing_queue;
mod worker;
use crate::beacon_processor::work_reprocessing_queue::QueuedGossipBlock;
use crate::beacon_processor::work_reprocessing_queue::{
QueuedBackfillBatch, QueuedGossipBlock, ReprocessQueueMessage,
};
pub use worker::{ChainSegmentProcessId, GossipAggregatePackage, GossipAttestationPackage};
/// The maximum size of the channel for work events to the `BeaconProcessor`.
@ -218,6 +221,7 @@ pub const GOSSIP_LIGHT_CLIENT_FINALITY_UPDATE: &str = "light_client_finality_upd
pub const GOSSIP_LIGHT_CLIENT_OPTIMISTIC_UPDATE: &str = "light_client_optimistic_update";
pub const RPC_BLOCK: &str = "rpc_block";
pub const CHAIN_SEGMENT: &str = "chain_segment";
pub const CHAIN_SEGMENT_BACKFILL: &str = "chain_segment_backfill";
pub const STATUS_PROCESSING: &str = "status_processing";
pub const BLOCKS_BY_RANGE_REQUEST: &str = "blocks_by_range_request";
pub const BLOCKS_BY_ROOTS_REQUEST: &str = "blocks_by_roots_request";
@ -738,6 +742,9 @@ impl<T: BeaconChainTypes> std::convert::From<ReadyWork<T>> for WorkEvent<T> {
seen_timestamp,
},
},
ReadyWork::BackfillSync(QueuedBackfillBatch { process_id, blocks }) => {
WorkEvent::chain_segment(process_id, blocks)
}
}
}
}
@ -893,6 +900,10 @@ impl<T: BeaconChainTypes> Work<T> {
Work::GossipLightClientFinalityUpdate { .. } => GOSSIP_LIGHT_CLIENT_FINALITY_UPDATE,
Work::GossipLightClientOptimisticUpdate { .. } => GOSSIP_LIGHT_CLIENT_OPTIMISTIC_UPDATE,
Work::RpcBlock { .. } => RPC_BLOCK,
Work::ChainSegment {
process_id: ChainSegmentProcessId::BackSyncBatchId { .. },
..
} => CHAIN_SEGMENT_BACKFILL,
Work::ChainSegment { .. } => CHAIN_SEGMENT,
Work::Status { .. } => STATUS_PROCESSING,
Work::BlocksByRangeRequest { .. } => BLOCKS_BY_RANGE_REQUEST,
@ -1054,23 +1065,23 @@ impl<T: BeaconChainTypes> BeaconProcessor<T> {
FifoQueue::new(MAX_BLS_TO_EXECUTION_CHANGE_QUEUE_LEN);
let mut lcbootstrap_queue = FifoQueue::new(MAX_LIGHT_CLIENT_BOOTSTRAP_QUEUE_LEN);
let chain = match self.beacon_chain.upgrade() {
Some(chain) => chain,
// No need to proceed any further if the beacon chain has been dropped, the client
// is shutting down.
None => return,
};
// Channels for sending work to the re-process scheduler (`work_reprocessing_tx`) and to
// receive them back once they are ready (`ready_work_rx`).
let (ready_work_tx, ready_work_rx) = mpsc::channel(MAX_SCHEDULED_WORK_QUEUE_LEN);
let work_reprocessing_tx = {
if let Some(chain) = self.beacon_chain.upgrade() {
spawn_reprocess_scheduler(
ready_work_tx,
&self.executor,
chain.slot_clock.clone(),
self.log.clone(),
)
} else {
// No need to proceed any further if the beacon chain has been dropped, the client
// is shutting down.
return;
}
};
let work_reprocessing_tx = spawn_reprocess_scheduler(
ready_work_tx,
&self.executor,
chain.slot_clock.clone(),
self.log.clone(),
);
let executor = self.executor.clone();
@ -1083,12 +1094,55 @@ impl<T: BeaconChainTypes> BeaconProcessor<T> {
reprocess_work_rx: ready_work_rx,
};
let enable_backfill_rate_limiting = chain.config.enable_backfill_rate_limiting;
loop {
let work_event = match inbound_events.next().await {
Some(InboundEvent::WorkerIdle) => {
self.current_workers = self.current_workers.saturating_sub(1);
None
}
Some(InboundEvent::WorkEvent(event)) if enable_backfill_rate_limiting => {
match QueuedBackfillBatch::try_from(event) {
Ok(backfill_batch) => {
match work_reprocessing_tx
.try_send(ReprocessQueueMessage::BackfillSync(backfill_batch))
{
Err(e) => {
warn!(
self.log,
"Unable to queue backfill work event. Will try to process now.";
"error" => %e
);
match e {
TrySendError::Full(reprocess_queue_message)
| TrySendError::Closed(reprocess_queue_message) => {
match reprocess_queue_message {
ReprocessQueueMessage::BackfillSync(
backfill_batch,
) => Some(backfill_batch.into()),
other => {
crit!(
self.log,
"Unexpected queue message type";
"message_type" => other.as_ref()
);
// This is an unhandled exception, drop the message.
continue;
}
}
}
}
}
Ok(..) => {
// backfill work sent to "reprocessing" queue. Process the next event.
continue;
}
}
}
Err(event) => Some(event),
}
}
Some(InboundEvent::WorkEvent(event))
| Some(InboundEvent::ReprocessingWork(event)) => Some(event),
None => {

View File

@ -9,7 +9,7 @@ use crate::{service::NetworkMessage, sync::SyncMessage};
use beacon_chain::test_utils::{
AttestationStrategy, BeaconChainHarness, BlockStrategy, EphemeralHarnessType,
};
use beacon_chain::{BeaconChain, MAXIMUM_GOSSIP_CLOCK_DISPARITY};
use beacon_chain::{BeaconChain, ChainConfig, MAXIMUM_GOSSIP_CLOCK_DISPARITY};
use lighthouse_network::{
discv5::enr::{CombinedKey, EnrBuilder},
rpc::methods::{MetaData, MetaDataV2},
@ -23,8 +23,8 @@ use std::sync::Arc;
use std::time::Duration;
use tokio::sync::mpsc;
use types::{
Attestation, AttesterSlashing, EthSpec, MainnetEthSpec, ProposerSlashing, SignedBeaconBlock,
SignedVoluntaryExit, SubnetId,
Attestation, AttesterSlashing, Epoch, EthSpec, MainnetEthSpec, ProposerSlashing,
SignedBeaconBlock, SignedVoluntaryExit, SubnetId,
};
type E = MainnetEthSpec;
@ -70,6 +70,10 @@ impl Drop for TestRig {
impl TestRig {
pub async fn new(chain_length: u64) -> Self {
Self::new_with_chain_config(chain_length, ChainConfig::default()).await
}
pub async fn new_with_chain_config(chain_length: u64, chain_config: ChainConfig) -> Self {
// This allows for testing voluntary exits without building out a massive chain.
let mut spec = E::default_spec();
spec.shard_committee_period = 2;
@ -78,6 +82,7 @@ impl TestRig {
.spec(spec)
.deterministic_keypairs(VALIDATOR_COUNT)
.fresh_ephemeral_store()
.chain_config(chain_config)
.build();
harness.advance_slot();
@ -261,6 +266,14 @@ impl TestRig {
self.beacon_processor_tx.try_send(event).unwrap();
}
pub fn enqueue_backfill_batch(&self) {
let event = WorkEvent::chain_segment(
ChainSegmentProcessId::BackSyncBatchId(Epoch::default()),
Vec::default(),
);
self.beacon_processor_tx.try_send(event).unwrap();
}
pub fn enqueue_unaggregated_attestation(&self) {
let (attestation, subnet_id) = self.attestations.first().unwrap().clone();
self.beacon_processor_tx
@ -873,3 +886,49 @@ async fn test_rpc_block_reprocessing() {
// cache handle was dropped.
assert_eq!(next_block_root, rig.head_root());
}
/// Ensure that backfill batches get rate-limited and processing is scheduled at specified intervals.
#[tokio::test]
async fn test_backfill_sync_processing() {
let mut rig = TestRig::new(SMALL_CHAIN).await;
// Note: to verify the exact event times in an integration test is not straight forward here
// (not straight forward to manipulate `TestingSlotClock` due to cloning of `SlotClock` in code)
// and makes the test very slow, hence timing calculation is unit tested separately in
// `work_reprocessing_queue`.
for _ in 0..1 {
rig.enqueue_backfill_batch();
// ensure queued batch is not processed until later
rig.assert_no_events_for(Duration::from_millis(100)).await;
// A new batch should be processed within a slot.
rig.assert_event_journal_with_timeout(
&[CHAIN_SEGMENT_BACKFILL, WORKER_FREED, NOTHING_TO_DO],
rig.chain.slot_clock.slot_duration(),
)
.await;
}
}
/// Ensure that backfill batches get processed as fast as they can when rate-limiting is disabled.
#[tokio::test]
async fn test_backfill_sync_processing_rate_limiting_disabled() {
let chain_config = ChainConfig {
enable_backfill_rate_limiting: false,
..Default::default()
};
let mut rig = TestRig::new_with_chain_config(SMALL_CHAIN, chain_config).await;
for _ in 0..3 {
rig.enqueue_backfill_batch();
}
// ensure all batches are processed
rig.assert_event_journal_with_timeout(
&[
CHAIN_SEGMENT_BACKFILL,
CHAIN_SEGMENT_BACKFILL,
CHAIN_SEGMENT_BACKFILL,
],
Duration::from_millis(100),
)
.await;
}

View File

@ -11,21 +11,25 @@
//! Aggregated and unaggregated attestations that failed verification due to referencing an unknown
//! block will be re-queued until their block is imported, or until they expire.
use super::MAX_SCHEDULED_WORK_QUEUE_LEN;
use crate::beacon_processor::{ChainSegmentProcessId, Work, WorkEvent};
use crate::metrics;
use crate::sync::manager::BlockProcessType;
use beacon_chain::{BeaconChainTypes, GossipVerifiedBlock, MAXIMUM_GOSSIP_CLOCK_DISPARITY};
use fnv::FnvHashMap;
use futures::task::Poll;
use futures::{Stream, StreamExt};
use itertools::Itertools;
use lighthouse_network::{MessageId, PeerId};
use logging::TimeLatch;
use slog::{crit, debug, error, trace, warn, Logger};
use slot_clock::SlotClock;
use std::collections::{HashMap, HashSet};
use std::future::Future;
use std::pin::Pin;
use std::sync::Arc;
use std::task::Context;
use std::time::Duration;
use strum::AsRefStr;
use task_executor::TaskExecutor;
use tokio::sync::mpsc::{self, Receiver, Sender};
use tokio::time::error::Error as TimeError;
@ -65,7 +69,21 @@ const MAXIMUM_QUEUED_ATTESTATIONS: usize = 16_384;
/// How many light client updates we keep before new ones get dropped.
const MAXIMUM_QUEUED_LIGHT_CLIENT_UPDATES: usize = 128;
// Process backfill batch 50%, 60%, 80% through each slot.
//
// Note: use caution to set these fractions in a way that won't cause panic-y
// arithmetic.
pub const BACKFILL_SCHEDULE_IN_SLOT: [(u32, u32); 3] = [
// One half: 6s on mainnet, 2.5s on Gnosis.
(1, 2),
// Three fifths: 7.2s on mainnet, 3s on Gnosis.
(3, 5),
// Four fifths: 9.6s on mainnet, 4s on Gnosis.
(4, 5),
];
/// Messages that the scheduler can receive.
#[derive(AsRefStr)]
pub enum ReprocessQueueMessage<T: BeaconChainTypes> {
/// A block that has been received early and we should queue for later processing.
EarlyBlock(QueuedGossipBlock<T>),
@ -84,6 +102,8 @@ pub enum ReprocessQueueMessage<T: BeaconChainTypes> {
UnknownBlockAggregate(QueuedAggregate<T::EthSpec>),
/// A light client optimistic update that references a parent root that has not been seen as a parent.
UnknownLightClientOptimisticUpdate(QueuedLightClientUpdate<T::EthSpec>),
/// A new backfill batch that needs to be scheduled for processing.
BackfillSync(QueuedBackfillBatch<T::EthSpec>),
}
/// Events sent by the scheduler once they are ready for re-processing.
@ -93,6 +113,7 @@ pub enum ReadyWork<T: BeaconChainTypes> {
Unaggregate(QueuedUnaggregate<T::EthSpec>),
Aggregate(QueuedAggregate<T::EthSpec>),
LightClientUpdate(QueuedLightClientUpdate<T::EthSpec>),
BackfillSync(QueuedBackfillBatch<T::EthSpec>),
}
/// An Attestation for which the corresponding block was not seen while processing, queued for
@ -144,6 +165,40 @@ pub struct QueuedRpcBlock<T: EthSpec> {
pub should_process: bool,
}
/// A backfill batch work that has been queued for processing later.
#[derive(Clone)]
pub struct QueuedBackfillBatch<E: EthSpec> {
pub process_id: ChainSegmentProcessId,
pub blocks: Vec<Arc<SignedBeaconBlock<E>>>,
}
impl<T: BeaconChainTypes> TryFrom<WorkEvent<T>> for QueuedBackfillBatch<T::EthSpec> {
type Error = WorkEvent<T>;
fn try_from(event: WorkEvent<T>) -> Result<Self, WorkEvent<T>> {
match event {
WorkEvent {
work:
Work::ChainSegment {
process_id: process_id @ ChainSegmentProcessId::BackSyncBatchId(_),
blocks,
},
..
} => Ok(QueuedBackfillBatch { process_id, blocks }),
_ => Err(event),
}
}
}
impl<T: BeaconChainTypes> From<QueuedBackfillBatch<T::EthSpec>> for WorkEvent<T> {
fn from(queued_backfill_batch: QueuedBackfillBatch<T::EthSpec>) -> WorkEvent<T> {
WorkEvent::chain_segment(
queued_backfill_batch.process_id,
queued_backfill_batch.blocks,
)
}
}
/// Unifies the different messages processed by the block delay queue.
enum InboundEvent<T: BeaconChainTypes> {
/// A gossip block that was queued for later processing and is ready for import.
@ -155,6 +210,8 @@ enum InboundEvent<T: BeaconChainTypes> {
ReadyAttestation(QueuedAttestationId),
/// A light client update that is ready for re-processing.
ReadyLightClientUpdate(QueuedLightClientUpdateId),
/// A backfill batch that was queued is ready for processing.
ReadyBackfillSync(QueuedBackfillBatch<T::EthSpec>),
/// A `DelayQueue` returned an error.
DelayQueueError(TimeError, &'static str),
/// A message sent to the `ReprocessQueue`
@ -191,6 +248,8 @@ struct ReprocessQueue<T: BeaconChainTypes> {
queued_lc_updates: FnvHashMap<usize, (QueuedLightClientUpdate<T::EthSpec>, DelayKey)>,
/// Light Client Updates per parent_root.
awaiting_lc_updates_per_parent_root: HashMap<Hash256, Vec<QueuedLightClientUpdateId>>,
/// Queued backfill batches
queued_backfill_batches: Vec<QueuedBackfillBatch<T::EthSpec>>,
/* Aux */
/// Next attestation id, used for both aggregated and unaggregated attestations
@ -200,6 +259,8 @@ struct ReprocessQueue<T: BeaconChainTypes> {
rpc_block_debounce: TimeLatch,
attestation_delay_debounce: TimeLatch,
lc_update_delay_debounce: TimeLatch,
next_backfill_batch_event: Option<Pin<Box<tokio::time::Sleep>>>,
slot_clock: Pin<Box<T::SlotClock>>,
}
pub type QueuedLightClientUpdateId = usize;
@ -287,6 +348,20 @@ impl<T: BeaconChainTypes> Stream for ReprocessQueue<T> {
Poll::Ready(None) | Poll::Pending => (),
}
if let Some(next_backfill_batch_event) = self.next_backfill_batch_event.as_mut() {
match next_backfill_batch_event.as_mut().poll(cx) {
Poll::Ready(_) => {
let maybe_batch = self.queued_backfill_batches.pop();
self.recompute_next_backfill_batch_event();
if let Some(batch) = maybe_batch {
return Poll::Ready(Some(InboundEvent::ReadyBackfillSync(batch)));
}
}
Poll::Pending => (),
}
}
// Last empty the messages channel.
match self.work_reprocessing_rx.poll_recv(cx) {
Poll::Ready(Some(message)) => return Poll::Ready(Some(InboundEvent::Msg(message))),
@ -323,12 +398,15 @@ pub fn spawn_reprocess_scheduler<T: BeaconChainTypes>(
queued_unaggregates: FnvHashMap::default(),
awaiting_attestations_per_root: HashMap::new(),
awaiting_lc_updates_per_parent_root: HashMap::new(),
queued_backfill_batches: Vec::new(),
next_attestation: 0,
next_lc_update: 0,
early_block_debounce: TimeLatch::default(),
rpc_block_debounce: TimeLatch::default(),
attestation_delay_debounce: TimeLatch::default(),
lc_update_delay_debounce: TimeLatch::default(),
next_backfill_batch_event: None,
slot_clock: Box::pin(slot_clock.clone()),
};
executor.spawn(
@ -679,6 +757,14 @@ impl<T: BeaconChainTypes> ReprocessQueue<T> {
}
}
}
InboundEvent::Msg(BackfillSync(queued_backfill_batch)) => {
self.queued_backfill_batches
.insert(0, queued_backfill_batch);
// only recompute if there is no `next_backfill_batch_event` already scheduled
if self.next_backfill_batch_event.is_none() {
self.recompute_next_backfill_batch_event();
}
}
// A block that was queued for later processing is now ready to be processed.
InboundEvent::ReadyGossipBlock(ready_block) => {
let block_root = ready_block.block.block_root;
@ -786,6 +872,33 @@ impl<T: BeaconChainTypes> ReprocessQueue<T> {
}
}
}
InboundEvent::ReadyBackfillSync(queued_backfill_batch) => {
let millis_from_slot_start = slot_clock
.millis_from_current_slot_start()
.map_or("null".to_string(), |duration| {
duration.as_millis().to_string()
});
debug!(
log,
"Sending scheduled backfill work";
"millis_from_slot_start" => millis_from_slot_start
);
if self
.ready_work_tx
.try_send(ReadyWork::BackfillSync(queued_backfill_batch.clone()))
.is_err()
{
error!(
log,
"Failed to send scheduled backfill work";
"info" => "sending work back to queue"
);
self.queued_backfill_batches
.insert(0, queued_backfill_batch);
}
}
}
metrics::set_gauge_vec(
@ -809,4 +922,95 @@ impl<T: BeaconChainTypes> ReprocessQueue<T> {
self.lc_updates_delay_queue.len() as i64,
);
}
fn recompute_next_backfill_batch_event(&mut self) {
// only recompute the `next_backfill_batch_event` if there are backfill batches in the queue
if !self.queued_backfill_batches.is_empty() {
self.next_backfill_batch_event = Some(Box::pin(tokio::time::sleep(
ReprocessQueue::<T>::duration_until_next_backfill_batch_event(&self.slot_clock),
)));
} else {
self.next_backfill_batch_event = None
}
}
/// Returns duration until the next scheduled processing time. The schedule ensure that backfill
/// processing is done in windows of time that aren't critical
fn duration_until_next_backfill_batch_event(slot_clock: &T::SlotClock) -> Duration {
let slot_duration = slot_clock.slot_duration();
slot_clock
.millis_from_current_slot_start()
.and_then(|duration_from_slot_start| {
BACKFILL_SCHEDULE_IN_SLOT
.into_iter()
// Convert fractions to seconds from slot start.
.map(|(multiplier, divisor)| (slot_duration / divisor) * multiplier)
.find_or_first(|&event_duration_from_slot_start| {
event_duration_from_slot_start > duration_from_slot_start
})
.map(|next_event_time| {
if duration_from_slot_start >= next_event_time {
// event is in the next slot, add duration to next slot
let duration_to_next_slot = slot_duration - duration_from_slot_start;
duration_to_next_slot + next_event_time
} else {
next_event_time - duration_from_slot_start
}
})
})
// If we can't read the slot clock, just wait another slot.
.unwrap_or(slot_duration)
}
}
#[cfg(test)]
mod tests {
use super::*;
use beacon_chain::builder::Witness;
use beacon_chain::eth1_chain::CachingEth1Backend;
use slot_clock::TestingSlotClock;
use store::MemoryStore;
use types::MainnetEthSpec as E;
use types::Slot;
type TestBeaconChainType =
Witness<TestingSlotClock, CachingEth1Backend<E>, E, MemoryStore<E>, MemoryStore<E>>;
#[test]
fn backfill_processing_schedule_calculation() {
let slot_duration = Duration::from_secs(12);
let slot_clock = TestingSlotClock::new(Slot::new(0), Duration::from_secs(0), slot_duration);
let current_slot_start = slot_clock.start_of(Slot::new(100)).unwrap();
slot_clock.set_current_time(current_slot_start);
let event_times = BACKFILL_SCHEDULE_IN_SLOT
.map(|(multiplier, divisor)| (slot_duration / divisor) * multiplier);
for &event_duration_from_slot_start in event_times.iter() {
let duration_to_next_event =
ReprocessQueue::<TestBeaconChainType>::duration_until_next_backfill_batch_event(
&slot_clock,
);
let current_time = slot_clock.millis_from_current_slot_start().unwrap();
assert_eq!(
duration_to_next_event,
event_duration_from_slot_start - current_time
);
slot_clock.set_current_time(current_slot_start + event_duration_from_slot_start)
}
// check for next event beyond the current slot
let duration_to_next_slot = slot_clock.duration_to_next_slot().unwrap();
let duration_to_next_event =
ReprocessQueue::<TestBeaconChainType>::duration_until_next_backfill_batch_event(
&slot_clock,
);
assert_eq!(
duration_to_next_event,
duration_to_next_slot + event_times[0]
);
}
}

View File

@ -261,6 +261,14 @@ pub fn cli_app<'a, 'b>() -> App<'a, 'b> {
.min_values(0)
.hidden(true)
)
.arg(
Arg::with_name("disable-backfill-rate-limiting")
.long("disable-backfill-rate-limiting")
.help("Disable the backfill sync rate-limiting. This allow users to just sync the entire chain as fast \
as possible, however it can result in resource contention which degrades staking performance. Stakers \
should generally choose to avoid this flag since backfill sync is not required for staking.")
.takes_value(false),
)
/* REST API related arguments */
.arg(
Arg::with_name("http")

View File

@ -759,6 +759,10 @@ pub fn get_config<E: EthSpec>(
client_config.always_prefer_builder_payload = true;
}
// Backfill sync rate-limiting
client_config.chain.enable_backfill_rate_limiting =
!cli_args.is_present("disable-backfill-rate-limiting");
Ok(client_config)
}

View File

@ -104,12 +104,23 @@ pub trait SlotClock: Send + Sync + Sized + Clone {
self.slot_duration() * 2 / INTERVALS_PER_SLOT as u32
}
/// Returns the `Duration` since the start of the current `Slot`. Useful in determining whether to apply proposer boosts.
fn seconds_from_current_slot_start(&self, seconds_per_slot: u64) -> Option<Duration> {
/// Returns the `Duration` since the start of the current `Slot` at seconds precision. Useful in determining whether to apply proposer boosts.
fn seconds_from_current_slot_start(&self) -> Option<Duration> {
self.now_duration()
.and_then(|now| now.checked_sub(self.genesis_duration()))
.map(|duration_into_slot| {
Duration::from_secs(duration_into_slot.as_secs() % seconds_per_slot)
Duration::from_secs(duration_into_slot.as_secs() % self.slot_duration().as_secs())
})
}
/// Returns the `Duration` since the start of the current `Slot` at milliseconds precision.
fn millis_from_current_slot_start(&self) -> Option<Duration> {
self.now_duration()
.and_then(|now| now.checked_sub(self.genesis_duration()))
.map(|duration_into_slot| {
Duration::from_millis(
(duration_into_slot.as_millis() % self.slot_duration().as_millis()) as u64,
)
})
}

View File

@ -1052,6 +1052,19 @@ fn disable_upnp_flag() {
.with_config(|config| assert!(!config.network.upnp_enabled));
}
#[test]
fn disable_backfill_rate_limiting_flag() {
CommandLineTest::new()
.flag("disable-backfill-rate-limiting", None)
.run_with_zero_port()
.with_config(|config| assert!(!config.chain.enable_backfill_rate_limiting));
}
#[test]
fn default_backfill_rate_limiting_flag() {
CommandLineTest::new()
.run_with_zero_port()
.with_config(|config| assert!(config.chain.enable_backfill_rate_limiting));
}
#[test]
fn default_boot_nodes() {
let mainnet = vec![
// Lighthouse Team (Sigma Prime)

View File

@ -425,7 +425,7 @@ impl<E: EthSpec> Tester<E> {
.harness
.chain
.slot_clock
.seconds_from_current_slot_start(self.spec.seconds_per_slot)
.seconds_from_current_slot_start()
.unwrap();
let result = self