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:
parent
c5383e393a
commit
2de3451011
@ -2893,7 +2893,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
|
|||||||
metrics::start_timer(&metrics::FORK_CHOICE_PROCESS_BLOCK_TIMES);
|
metrics::start_timer(&metrics::FORK_CHOICE_PROCESS_BLOCK_TIMES);
|
||||||
let block_delay = self
|
let block_delay = self
|
||||||
.slot_clock
|
.slot_clock
|
||||||
.seconds_from_current_slot_start(self.spec.seconds_per_slot)
|
.seconds_from_current_slot_start()
|
||||||
.ok_or(Error::UnableToComputeTimeAtSlot)?;
|
.ok_or(Error::UnableToComputeTimeAtSlot)?;
|
||||||
|
|
||||||
fork_choice
|
fork_choice
|
||||||
@ -3746,7 +3746,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
|
|||||||
|
|
||||||
let slot_delay = self
|
let slot_delay = self
|
||||||
.slot_clock
|
.slot_clock
|
||||||
.seconds_from_current_slot_start(self.spec.seconds_per_slot)
|
.seconds_from_current_slot_start()
|
||||||
.or_else(|| {
|
.or_else(|| {
|
||||||
warn!(
|
warn!(
|
||||||
self.log,
|
self.log,
|
||||||
|
@ -68,6 +68,8 @@ pub struct ChainConfig {
|
|||||||
///
|
///
|
||||||
/// This is useful for block builders and testing.
|
/// This is useful for block builders and testing.
|
||||||
pub always_prepare_payload: bool,
|
pub always_prepare_payload: bool,
|
||||||
|
/// Whether backfill sync processing should be rate-limited.
|
||||||
|
pub enable_backfill_rate_limiting: bool,
|
||||||
}
|
}
|
||||||
|
|
||||||
impl Default for ChainConfig {
|
impl Default for ChainConfig {
|
||||||
@ -94,6 +96,7 @@ impl Default for ChainConfig {
|
|||||||
optimistic_finalized_sync: true,
|
optimistic_finalized_sync: true,
|
||||||
shuffling_cache_size: crate::shuffling_cache::DEFAULT_CACHE_SIZE,
|
shuffling_cache_size: crate::shuffling_cache::DEFAULT_CACHE_SIZE,
|
||||||
always_prepare_payload: false,
|
always_prepare_payload: false,
|
||||||
|
enable_backfill_rate_limiting: true,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -61,6 +61,7 @@ use std::time::Duration;
|
|||||||
use std::{cmp, collections::HashSet};
|
use std::{cmp, collections::HashSet};
|
||||||
use task_executor::TaskExecutor;
|
use task_executor::TaskExecutor;
|
||||||
use tokio::sync::mpsc;
|
use tokio::sync::mpsc;
|
||||||
|
use tokio::sync::mpsc::error::TrySendError;
|
||||||
use types::{
|
use types::{
|
||||||
Attestation, AttesterSlashing, Hash256, LightClientFinalityUpdate, LightClientOptimisticUpdate,
|
Attestation, AttesterSlashing, Hash256, LightClientFinalityUpdate, LightClientOptimisticUpdate,
|
||||||
ProposerSlashing, SignedAggregateAndProof, SignedBeaconBlock, SignedBlsToExecutionChange,
|
ProposerSlashing, SignedAggregateAndProof, SignedBeaconBlock, SignedBlsToExecutionChange,
|
||||||
@ -77,7 +78,9 @@ mod tests;
|
|||||||
mod work_reprocessing_queue;
|
mod work_reprocessing_queue;
|
||||||
mod worker;
|
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};
|
pub use worker::{ChainSegmentProcessId, GossipAggregatePackage, GossipAttestationPackage};
|
||||||
|
|
||||||
/// The maximum size of the channel for work events to the `BeaconProcessor`.
|
/// 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 GOSSIP_LIGHT_CLIENT_OPTIMISTIC_UPDATE: &str = "light_client_optimistic_update";
|
||||||
pub const RPC_BLOCK: &str = "rpc_block";
|
pub const RPC_BLOCK: &str = "rpc_block";
|
||||||
pub const CHAIN_SEGMENT: &str = "chain_segment";
|
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 STATUS_PROCESSING: &str = "status_processing";
|
||||||
pub const BLOCKS_BY_RANGE_REQUEST: &str = "blocks_by_range_request";
|
pub const BLOCKS_BY_RANGE_REQUEST: &str = "blocks_by_range_request";
|
||||||
pub const BLOCKS_BY_ROOTS_REQUEST: &str = "blocks_by_roots_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,
|
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::GossipLightClientFinalityUpdate { .. } => GOSSIP_LIGHT_CLIENT_FINALITY_UPDATE,
|
||||||
Work::GossipLightClientOptimisticUpdate { .. } => GOSSIP_LIGHT_CLIENT_OPTIMISTIC_UPDATE,
|
Work::GossipLightClientOptimisticUpdate { .. } => GOSSIP_LIGHT_CLIENT_OPTIMISTIC_UPDATE,
|
||||||
Work::RpcBlock { .. } => RPC_BLOCK,
|
Work::RpcBlock { .. } => RPC_BLOCK,
|
||||||
|
Work::ChainSegment {
|
||||||
|
process_id: ChainSegmentProcessId::BackSyncBatchId { .. },
|
||||||
|
..
|
||||||
|
} => CHAIN_SEGMENT_BACKFILL,
|
||||||
Work::ChainSegment { .. } => CHAIN_SEGMENT,
|
Work::ChainSegment { .. } => CHAIN_SEGMENT,
|
||||||
Work::Status { .. } => STATUS_PROCESSING,
|
Work::Status { .. } => STATUS_PROCESSING,
|
||||||
Work::BlocksByRangeRequest { .. } => BLOCKS_BY_RANGE_REQUEST,
|
Work::BlocksByRangeRequest { .. } => BLOCKS_BY_RANGE_REQUEST,
|
||||||
@ -1054,23 +1065,23 @@ impl<T: BeaconChainTypes> BeaconProcessor<T> {
|
|||||||
FifoQueue::new(MAX_BLS_TO_EXECUTION_CHANGE_QUEUE_LEN);
|
FifoQueue::new(MAX_BLS_TO_EXECUTION_CHANGE_QUEUE_LEN);
|
||||||
|
|
||||||
let mut lcbootstrap_queue = FifoQueue::new(MAX_LIGHT_CLIENT_BOOTSTRAP_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
|
// Channels for sending work to the re-process scheduler (`work_reprocessing_tx`) and to
|
||||||
// receive them back once they are ready (`ready_work_rx`).
|
// 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 (ready_work_tx, ready_work_rx) = mpsc::channel(MAX_SCHEDULED_WORK_QUEUE_LEN);
|
||||||
let work_reprocessing_tx = {
|
let work_reprocessing_tx = spawn_reprocess_scheduler(
|
||||||
if let Some(chain) = self.beacon_chain.upgrade() {
|
|
||||||
spawn_reprocess_scheduler(
|
|
||||||
ready_work_tx,
|
ready_work_tx,
|
||||||
&self.executor,
|
&self.executor,
|
||||||
chain.slot_clock.clone(),
|
chain.slot_clock.clone(),
|
||||||
self.log.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 executor = self.executor.clone();
|
let executor = self.executor.clone();
|
||||||
|
|
||||||
@ -1083,12 +1094,55 @@ impl<T: BeaconChainTypes> BeaconProcessor<T> {
|
|||||||
reprocess_work_rx: ready_work_rx,
|
reprocess_work_rx: ready_work_rx,
|
||||||
};
|
};
|
||||||
|
|
||||||
|
let enable_backfill_rate_limiting = chain.config.enable_backfill_rate_limiting;
|
||||||
|
|
||||||
loop {
|
loop {
|
||||||
let work_event = match inbound_events.next().await {
|
let work_event = match inbound_events.next().await {
|
||||||
Some(InboundEvent::WorkerIdle) => {
|
Some(InboundEvent::WorkerIdle) => {
|
||||||
self.current_workers = self.current_workers.saturating_sub(1);
|
self.current_workers = self.current_workers.saturating_sub(1);
|
||||||
None
|
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::WorkEvent(event))
|
||||||
| Some(InboundEvent::ReprocessingWork(event)) => Some(event),
|
| Some(InboundEvent::ReprocessingWork(event)) => Some(event),
|
||||||
None => {
|
None => {
|
||||||
|
@ -9,7 +9,7 @@ use crate::{service::NetworkMessage, sync::SyncMessage};
|
|||||||
use beacon_chain::test_utils::{
|
use beacon_chain::test_utils::{
|
||||||
AttestationStrategy, BeaconChainHarness, BlockStrategy, EphemeralHarnessType,
|
AttestationStrategy, BeaconChainHarness, BlockStrategy, EphemeralHarnessType,
|
||||||
};
|
};
|
||||||
use beacon_chain::{BeaconChain, MAXIMUM_GOSSIP_CLOCK_DISPARITY};
|
use beacon_chain::{BeaconChain, ChainConfig, MAXIMUM_GOSSIP_CLOCK_DISPARITY};
|
||||||
use lighthouse_network::{
|
use lighthouse_network::{
|
||||||
discv5::enr::{CombinedKey, EnrBuilder},
|
discv5::enr::{CombinedKey, EnrBuilder},
|
||||||
rpc::methods::{MetaData, MetaDataV2},
|
rpc::methods::{MetaData, MetaDataV2},
|
||||||
@ -23,8 +23,8 @@ use std::sync::Arc;
|
|||||||
use std::time::Duration;
|
use std::time::Duration;
|
||||||
use tokio::sync::mpsc;
|
use tokio::sync::mpsc;
|
||||||
use types::{
|
use types::{
|
||||||
Attestation, AttesterSlashing, EthSpec, MainnetEthSpec, ProposerSlashing, SignedBeaconBlock,
|
Attestation, AttesterSlashing, Epoch, EthSpec, MainnetEthSpec, ProposerSlashing,
|
||||||
SignedVoluntaryExit, SubnetId,
|
SignedBeaconBlock, SignedVoluntaryExit, SubnetId,
|
||||||
};
|
};
|
||||||
|
|
||||||
type E = MainnetEthSpec;
|
type E = MainnetEthSpec;
|
||||||
@ -70,6 +70,10 @@ impl Drop for TestRig {
|
|||||||
|
|
||||||
impl TestRig {
|
impl TestRig {
|
||||||
pub async fn new(chain_length: u64) -> Self {
|
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.
|
// This allows for testing voluntary exits without building out a massive chain.
|
||||||
let mut spec = E::default_spec();
|
let mut spec = E::default_spec();
|
||||||
spec.shard_committee_period = 2;
|
spec.shard_committee_period = 2;
|
||||||
@ -78,6 +82,7 @@ impl TestRig {
|
|||||||
.spec(spec)
|
.spec(spec)
|
||||||
.deterministic_keypairs(VALIDATOR_COUNT)
|
.deterministic_keypairs(VALIDATOR_COUNT)
|
||||||
.fresh_ephemeral_store()
|
.fresh_ephemeral_store()
|
||||||
|
.chain_config(chain_config)
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
harness.advance_slot();
|
harness.advance_slot();
|
||||||
@ -261,6 +266,14 @@ impl TestRig {
|
|||||||
self.beacon_processor_tx.try_send(event).unwrap();
|
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) {
|
pub fn enqueue_unaggregated_attestation(&self) {
|
||||||
let (attestation, subnet_id) = self.attestations.first().unwrap().clone();
|
let (attestation, subnet_id) = self.attestations.first().unwrap().clone();
|
||||||
self.beacon_processor_tx
|
self.beacon_processor_tx
|
||||||
@ -873,3 +886,49 @@ async fn test_rpc_block_reprocessing() {
|
|||||||
// cache handle was dropped.
|
// cache handle was dropped.
|
||||||
assert_eq!(next_block_root, rig.head_root());
|
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;
|
||||||
|
}
|
||||||
|
@ -11,21 +11,25 @@
|
|||||||
//! Aggregated and unaggregated attestations that failed verification due to referencing an unknown
|
//! 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.
|
//! block will be re-queued until their block is imported, or until they expire.
|
||||||
use super::MAX_SCHEDULED_WORK_QUEUE_LEN;
|
use super::MAX_SCHEDULED_WORK_QUEUE_LEN;
|
||||||
|
use crate::beacon_processor::{ChainSegmentProcessId, Work, WorkEvent};
|
||||||
use crate::metrics;
|
use crate::metrics;
|
||||||
use crate::sync::manager::BlockProcessType;
|
use crate::sync::manager::BlockProcessType;
|
||||||
use beacon_chain::{BeaconChainTypes, GossipVerifiedBlock, MAXIMUM_GOSSIP_CLOCK_DISPARITY};
|
use beacon_chain::{BeaconChainTypes, GossipVerifiedBlock, MAXIMUM_GOSSIP_CLOCK_DISPARITY};
|
||||||
use fnv::FnvHashMap;
|
use fnv::FnvHashMap;
|
||||||
use futures::task::Poll;
|
use futures::task::Poll;
|
||||||
use futures::{Stream, StreamExt};
|
use futures::{Stream, StreamExt};
|
||||||
|
use itertools::Itertools;
|
||||||
use lighthouse_network::{MessageId, PeerId};
|
use lighthouse_network::{MessageId, PeerId};
|
||||||
use logging::TimeLatch;
|
use logging::TimeLatch;
|
||||||
use slog::{crit, debug, error, trace, warn, Logger};
|
use slog::{crit, debug, error, trace, warn, Logger};
|
||||||
use slot_clock::SlotClock;
|
use slot_clock::SlotClock;
|
||||||
use std::collections::{HashMap, HashSet};
|
use std::collections::{HashMap, HashSet};
|
||||||
|
use std::future::Future;
|
||||||
use std::pin::Pin;
|
use std::pin::Pin;
|
||||||
use std::sync::Arc;
|
use std::sync::Arc;
|
||||||
use std::task::Context;
|
use std::task::Context;
|
||||||
use std::time::Duration;
|
use std::time::Duration;
|
||||||
|
use strum::AsRefStr;
|
||||||
use task_executor::TaskExecutor;
|
use task_executor::TaskExecutor;
|
||||||
use tokio::sync::mpsc::{self, Receiver, Sender};
|
use tokio::sync::mpsc::{self, Receiver, Sender};
|
||||||
use tokio::time::error::Error as TimeError;
|
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.
|
/// How many light client updates we keep before new ones get dropped.
|
||||||
const MAXIMUM_QUEUED_LIGHT_CLIENT_UPDATES: usize = 128;
|
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.
|
/// Messages that the scheduler can receive.
|
||||||
|
#[derive(AsRefStr)]
|
||||||
pub enum ReprocessQueueMessage<T: BeaconChainTypes> {
|
pub enum ReprocessQueueMessage<T: BeaconChainTypes> {
|
||||||
/// A block that has been received early and we should queue for later processing.
|
/// A block that has been received early and we should queue for later processing.
|
||||||
EarlyBlock(QueuedGossipBlock<T>),
|
EarlyBlock(QueuedGossipBlock<T>),
|
||||||
@ -84,6 +102,8 @@ pub enum ReprocessQueueMessage<T: BeaconChainTypes> {
|
|||||||
UnknownBlockAggregate(QueuedAggregate<T::EthSpec>),
|
UnknownBlockAggregate(QueuedAggregate<T::EthSpec>),
|
||||||
/// A light client optimistic update that references a parent root that has not been seen as a parent.
|
/// A light client optimistic update that references a parent root that has not been seen as a parent.
|
||||||
UnknownLightClientOptimisticUpdate(QueuedLightClientUpdate<T::EthSpec>),
|
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.
|
/// 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>),
|
Unaggregate(QueuedUnaggregate<T::EthSpec>),
|
||||||
Aggregate(QueuedAggregate<T::EthSpec>),
|
Aggregate(QueuedAggregate<T::EthSpec>),
|
||||||
LightClientUpdate(QueuedLightClientUpdate<T::EthSpec>),
|
LightClientUpdate(QueuedLightClientUpdate<T::EthSpec>),
|
||||||
|
BackfillSync(QueuedBackfillBatch<T::EthSpec>),
|
||||||
}
|
}
|
||||||
|
|
||||||
/// An Attestation for which the corresponding block was not seen while processing, queued for
|
/// 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,
|
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.
|
/// Unifies the different messages processed by the block delay queue.
|
||||||
enum InboundEvent<T: BeaconChainTypes> {
|
enum InboundEvent<T: BeaconChainTypes> {
|
||||||
/// A gossip block that was queued for later processing and is ready for import.
|
/// A gossip block that was queued for later processing and is ready for import.
|
||||||
@ -155,6 +210,8 @@ enum InboundEvent<T: BeaconChainTypes> {
|
|||||||
ReadyAttestation(QueuedAttestationId),
|
ReadyAttestation(QueuedAttestationId),
|
||||||
/// A light client update that is ready for re-processing.
|
/// A light client update that is ready for re-processing.
|
||||||
ReadyLightClientUpdate(QueuedLightClientUpdateId),
|
ReadyLightClientUpdate(QueuedLightClientUpdateId),
|
||||||
|
/// A backfill batch that was queued is ready for processing.
|
||||||
|
ReadyBackfillSync(QueuedBackfillBatch<T::EthSpec>),
|
||||||
/// A `DelayQueue` returned an error.
|
/// A `DelayQueue` returned an error.
|
||||||
DelayQueueError(TimeError, &'static str),
|
DelayQueueError(TimeError, &'static str),
|
||||||
/// A message sent to the `ReprocessQueue`
|
/// A message sent to the `ReprocessQueue`
|
||||||
@ -191,6 +248,8 @@ struct ReprocessQueue<T: BeaconChainTypes> {
|
|||||||
queued_lc_updates: FnvHashMap<usize, (QueuedLightClientUpdate<T::EthSpec>, DelayKey)>,
|
queued_lc_updates: FnvHashMap<usize, (QueuedLightClientUpdate<T::EthSpec>, DelayKey)>,
|
||||||
/// Light Client Updates per parent_root.
|
/// Light Client Updates per parent_root.
|
||||||
awaiting_lc_updates_per_parent_root: HashMap<Hash256, Vec<QueuedLightClientUpdateId>>,
|
awaiting_lc_updates_per_parent_root: HashMap<Hash256, Vec<QueuedLightClientUpdateId>>,
|
||||||
|
/// Queued backfill batches
|
||||||
|
queued_backfill_batches: Vec<QueuedBackfillBatch<T::EthSpec>>,
|
||||||
|
|
||||||
/* Aux */
|
/* Aux */
|
||||||
/// Next attestation id, used for both aggregated and unaggregated attestations
|
/// Next attestation id, used for both aggregated and unaggregated attestations
|
||||||
@ -200,6 +259,8 @@ struct ReprocessQueue<T: BeaconChainTypes> {
|
|||||||
rpc_block_debounce: TimeLatch,
|
rpc_block_debounce: TimeLatch,
|
||||||
attestation_delay_debounce: TimeLatch,
|
attestation_delay_debounce: TimeLatch,
|
||||||
lc_update_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;
|
pub type QueuedLightClientUpdateId = usize;
|
||||||
@ -287,6 +348,20 @@ impl<T: BeaconChainTypes> Stream for ReprocessQueue<T> {
|
|||||||
Poll::Ready(None) | Poll::Pending => (),
|
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.
|
// Last empty the messages channel.
|
||||||
match self.work_reprocessing_rx.poll_recv(cx) {
|
match self.work_reprocessing_rx.poll_recv(cx) {
|
||||||
Poll::Ready(Some(message)) => return Poll::Ready(Some(InboundEvent::Msg(message))),
|
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(),
|
queued_unaggregates: FnvHashMap::default(),
|
||||||
awaiting_attestations_per_root: HashMap::new(),
|
awaiting_attestations_per_root: HashMap::new(),
|
||||||
awaiting_lc_updates_per_parent_root: HashMap::new(),
|
awaiting_lc_updates_per_parent_root: HashMap::new(),
|
||||||
|
queued_backfill_batches: Vec::new(),
|
||||||
next_attestation: 0,
|
next_attestation: 0,
|
||||||
next_lc_update: 0,
|
next_lc_update: 0,
|
||||||
early_block_debounce: TimeLatch::default(),
|
early_block_debounce: TimeLatch::default(),
|
||||||
rpc_block_debounce: TimeLatch::default(),
|
rpc_block_debounce: TimeLatch::default(),
|
||||||
attestation_delay_debounce: TimeLatch::default(),
|
attestation_delay_debounce: TimeLatch::default(),
|
||||||
lc_update_delay_debounce: TimeLatch::default(),
|
lc_update_delay_debounce: TimeLatch::default(),
|
||||||
|
next_backfill_batch_event: None,
|
||||||
|
slot_clock: Box::pin(slot_clock.clone()),
|
||||||
};
|
};
|
||||||
|
|
||||||
executor.spawn(
|
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.
|
// A block that was queued for later processing is now ready to be processed.
|
||||||
InboundEvent::ReadyGossipBlock(ready_block) => {
|
InboundEvent::ReadyGossipBlock(ready_block) => {
|
||||||
let block_root = ready_block.block.block_root;
|
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(
|
metrics::set_gauge_vec(
|
||||||
@ -809,4 +922,95 @@ impl<T: BeaconChainTypes> ReprocessQueue<T> {
|
|||||||
self.lc_updates_delay_queue.len() as i64,
|
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]
|
||||||
|
);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
@ -261,6 +261,14 @@ pub fn cli_app<'a, 'b>() -> App<'a, 'b> {
|
|||||||
.min_values(0)
|
.min_values(0)
|
||||||
.hidden(true)
|
.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 */
|
/* REST API related arguments */
|
||||||
.arg(
|
.arg(
|
||||||
Arg::with_name("http")
|
Arg::with_name("http")
|
||||||
|
@ -759,6 +759,10 @@ pub fn get_config<E: EthSpec>(
|
|||||||
client_config.always_prefer_builder_payload = true;
|
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)
|
Ok(client_config)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -104,12 +104,23 @@ pub trait SlotClock: Send + Sync + Sized + Clone {
|
|||||||
self.slot_duration() * 2 / INTERVALS_PER_SLOT as u32
|
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.
|
/// 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, seconds_per_slot: u64) -> Option<Duration> {
|
fn seconds_from_current_slot_start(&self) -> Option<Duration> {
|
||||||
self.now_duration()
|
self.now_duration()
|
||||||
.and_then(|now| now.checked_sub(self.genesis_duration()))
|
.and_then(|now| now.checked_sub(self.genesis_duration()))
|
||||||
.map(|duration_into_slot| {
|
.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,
|
||||||
|
)
|
||||||
})
|
})
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1052,6 +1052,19 @@ fn disable_upnp_flag() {
|
|||||||
.with_config(|config| assert!(!config.network.upnp_enabled));
|
.with_config(|config| assert!(!config.network.upnp_enabled));
|
||||||
}
|
}
|
||||||
#[test]
|
#[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() {
|
fn default_boot_nodes() {
|
||||||
let mainnet = vec![
|
let mainnet = vec![
|
||||||
// Lighthouse Team (Sigma Prime)
|
// Lighthouse Team (Sigma Prime)
|
||||||
|
@ -425,7 +425,7 @@ impl<E: EthSpec> Tester<E> {
|
|||||||
.harness
|
.harness
|
||||||
.chain
|
.chain
|
||||||
.slot_clock
|
.slot_clock
|
||||||
.seconds_from_current_slot_start(self.spec.seconds_per_slot)
|
.seconds_from_current_slot_start()
|
||||||
.unwrap();
|
.unwrap();
|
||||||
|
|
||||||
let result = self
|
let result = self
|
||||||
|
Loading…
Reference in New Issue
Block a user