Documents and simplifies syncing code (#738)
* Add duplication prevention to gossipsub * Clean up topic logs * Add content addressed messages for gossip * Support BlocksByRange step parameter * Add initial docs to syncing * Adds documentation and simplify code in sync * Remove unnecessary comment * Fix fmt issue * Add batch log to sync
This commit is contained in:
parent
3acb3cc640
commit
01ac60d638
@ -66,7 +66,7 @@ pub struct RPC<TSubstream> {
|
|||||||
/// Queue of events to processed.
|
/// Queue of events to processed.
|
||||||
events: Vec<NetworkBehaviourAction<RPCEvent, RPCMessage>>,
|
events: Vec<NetworkBehaviourAction<RPCEvent, RPCMessage>>,
|
||||||
/// Pins the generic substream.
|
/// Pins the generic substream.
|
||||||
marker: PhantomData<(TSubstream)>,
|
marker: PhantomData<TSubstream>,
|
||||||
/// Slog logger for RPC behaviour.
|
/// Slog logger for RPC behaviour.
|
||||||
log: slog::Logger,
|
log: slog::Logger,
|
||||||
}
|
}
|
||||||
|
@ -16,36 +16,11 @@
|
|||||||
//! further peers connect, this process is run in parallel with those peers, until our head is
|
//! further peers connect, this process is run in parallel with those peers, until our head is
|
||||||
//! within `SLOT_IMPORT_TOLERANCE` of all connected peers.
|
//! within `SLOT_IMPORT_TOLERANCE` of all connected peers.
|
||||||
//!
|
//!
|
||||||
//! Batch Syncing
|
//! ## Batch Syncing
|
||||||
//!
|
//!
|
||||||
//! This syncing process start by requesting `BLOCKS_PER_REQUEST` blocks from a peer with an
|
//! See `RangeSync` for further details.
|
||||||
//! unknown chain (with a greater slot height) starting from our current head slot. If the earliest
|
|
||||||
//! block returned is known to us, then the group of blocks returned form part of a known chain,
|
|
||||||
//! and we process this batch of blocks, before requesting more batches forward and processing
|
|
||||||
//! those in turn until we reach the peer's chain's head. If the first batch doesn't contain a
|
|
||||||
//! block we know of, we must iteratively request blocks backwards (until our latest finalized head
|
|
||||||
//! slot) until we find a common ancestor before we can start processing the blocks. If no common
|
|
||||||
//! ancestor is found, the peer has a chain which is not part of our finalized head slot and we
|
|
||||||
//! drop the peer and the downloaded blocks.
|
|
||||||
//! Once we are fully synced with all known peers, the state of the manager becomes `Regular` which
|
|
||||||
//! then allows for parent lookups of propagated blocks.
|
|
||||||
//!
|
//!
|
||||||
//! A schematic version of this logic with two chain variations looks like the following.
|
//! ## Parent Lookup
|
||||||
//!
|
|
||||||
//! |----------------------|---------------------------------|
|
|
||||||
//! ^finalized head ^current local head ^remotes head
|
|
||||||
//!
|
|
||||||
//!
|
|
||||||
//! An example of the remotes chain diverging before our current head.
|
|
||||||
//! |---------------------------|
|
|
||||||
//! ^---------------------------------------------|
|
|
||||||
//! ^chain diverges |initial batch| ^remotes head
|
|
||||||
//!
|
|
||||||
//! In this example, we cannot process the initial batch as it is not on a known chain. We must
|
|
||||||
//! then backwards sync until we reach a common chain to begin forwarding batch syncing.
|
|
||||||
//!
|
|
||||||
//!
|
|
||||||
//! Parent Lookup
|
|
||||||
//!
|
//!
|
||||||
//! When a block with an unknown parent is received and we are in `Regular` sync mode, the block is
|
//! When a block with an unknown parent is received and we are in `Regular` sync mode, the block is
|
||||||
//! queued for lookup. A round-robin approach is used to request the parent from the known list of
|
//! queued for lookup. A round-robin approach is used to request the parent from the known list of
|
||||||
|
@ -23,6 +23,7 @@ const BLOCKS_PER_BATCH: u64 = 50;
|
|||||||
/// The number of times to retry a batch before the chain is considered failed and removed.
|
/// The number of times to retry a batch before the chain is considered failed and removed.
|
||||||
const MAX_BATCH_RETRIES: u8 = 5;
|
const MAX_BATCH_RETRIES: u8 = 5;
|
||||||
|
|
||||||
|
/// A collection of sequential blocks that are requested from peers in a single RPC request.
|
||||||
#[derive(PartialEq)]
|
#[derive(PartialEq)]
|
||||||
pub struct Batch<T: EthSpec> {
|
pub struct Batch<T: EthSpec> {
|
||||||
/// The ID of the batch, batches are ID's sequentially.
|
/// The ID of the batch, batches are ID's sequentially.
|
||||||
@ -55,6 +56,9 @@ impl<T: EthSpec> PartialOrd for Batch<T> {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/// A return type for functions that act on a `Chain` which informs the caller whether the chain
|
||||||
|
/// has been completed and should be removed or to be kept if further processing is
|
||||||
|
/// required.
|
||||||
pub enum ProcessingResult {
|
pub enum ProcessingResult {
|
||||||
KeepChain,
|
KeepChain,
|
||||||
RemoveChain,
|
RemoveChain,
|
||||||
@ -86,6 +90,9 @@ impl<T: EthSpec> Batch<T> {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/// A chain of blocks that need to be downloaded. Peers who claim to contain the target head
|
||||||
|
/// root are grouped into the peer pool and queried for batches when downloading the
|
||||||
|
/// chain.
|
||||||
pub struct SyncingChain<T: BeaconChainTypes> {
|
pub struct SyncingChain<T: BeaconChainTypes> {
|
||||||
/// The original start slot when this chain was initialised.
|
/// The original start slot when this chain was initialised.
|
||||||
pub start_slot: Slot,
|
pub start_slot: Slot,
|
||||||
@ -127,7 +134,7 @@ pub enum ChainSyncingState {
|
|||||||
/// The chain is undergoing syncing.
|
/// The chain is undergoing syncing.
|
||||||
Syncing,
|
Syncing,
|
||||||
/// The chain is temporarily paused whilst an error is rectified.
|
/// The chain is temporarily paused whilst an error is rectified.
|
||||||
Paused,
|
_Paused,
|
||||||
}
|
}
|
||||||
|
|
||||||
impl<T: BeaconChainTypes> SyncingChain<T> {
|
impl<T: BeaconChainTypes> SyncingChain<T> {
|
||||||
@ -154,6 +161,12 @@ impl<T: BeaconChainTypes> SyncingChain<T> {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/// A batch of blocks has been received. This function gets run on all chains and should
|
||||||
|
/// return Some if the request id matches a pending request on this chain, or None if it does
|
||||||
|
/// not.
|
||||||
|
///
|
||||||
|
/// If the request corresponds to a pending batch, this function processes the completed
|
||||||
|
/// batch.
|
||||||
pub fn on_block_response(
|
pub fn on_block_response(
|
||||||
&mut self,
|
&mut self,
|
||||||
chain: &Weak<BeaconChain<T>>,
|
chain: &Weak<BeaconChain<T>>,
|
||||||
@ -174,6 +187,9 @@ impl<T: BeaconChainTypes> SyncingChain<T> {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/// A completed batch has been received, process the batch.
|
||||||
|
/// This will return `ProcessingResult::KeepChain` if the chain has not completed or
|
||||||
|
/// failed indicating that further batches are required.
|
||||||
fn process_completed_batch(
|
fn process_completed_batch(
|
||||||
&mut self,
|
&mut self,
|
||||||
chain: Weak<BeaconChain<T>>,
|
chain: Weak<BeaconChain<T>>,
|
||||||
@ -226,57 +242,52 @@ impl<T: BeaconChainTypes> SyncingChain<T> {
|
|||||||
//blocks here, manage the queue and process them in another thread as they become
|
//blocks here, manage the queue and process them in another thread as they become
|
||||||
//available.
|
//available.
|
||||||
|
|
||||||
if self.state != ChainSyncingState::Paused {
|
if self.state == ChainSyncingState::Syncing {
|
||||||
// pre-emptively request more blocks from peers whilst we process current blocks,
|
// pre-emptively request more blocks from peers whilst we process current blocks,
|
||||||
self.send_range_request(network, current_peer);
|
self.send_range_request(network, current_peer, log);
|
||||||
|
}
|
||||||
|
|
||||||
// Try and process batches sequentially in the ordered list.
|
// Try and process batches sequentially in the ordered list.
|
||||||
let current_process_id = self.to_be_processed_id;
|
let current_process_id = self.to_be_processed_id;
|
||||||
|
// keep track of the number of successful batches to decide whether to run fork choice
|
||||||
|
let mut successful_block_process = false;
|
||||||
|
|
||||||
for batch in self
|
for batch in self
|
||||||
.completed_batches
|
.completed_batches
|
||||||
.iter()
|
.iter()
|
||||||
.filter(|batch| batch.id >= current_process_id)
|
.filter(|batch| batch.id >= current_process_id)
|
||||||
{
|
{
|
||||||
if batch.id == self.to_be_processed_id {
|
if batch.id != self.to_be_processed_id {
|
||||||
|
// there are no batches to be processed at the moment
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
|
||||||
if batch.downloaded_blocks.is_empty() {
|
if batch.downloaded_blocks.is_empty() {
|
||||||
// the batch was empty, progress to the next block
|
// the batch was empty, progress to the next block
|
||||||
self.to_be_processed_id += 1;
|
self.to_be_processed_id += 1;
|
||||||
continue;
|
continue;
|
||||||
} else {
|
}
|
||||||
let mut successes = 0;
|
|
||||||
|
// process the batch
|
||||||
|
// Keep track of successful batches. Run fork choice after all waiting batches have
|
||||||
|
// been processed.
|
||||||
debug!(log, "Processing batch"; "batch_id" => batch.id);
|
debug!(log, "Processing batch"; "batch_id" => batch.id);
|
||||||
match process_batch(chain.clone(), batch, &mut successes, log) {
|
match process_batch(chain.clone(), batch, log) {
|
||||||
Ok(_) => {
|
Ok(_) => {
|
||||||
// batch was successfully processed
|
// batch was successfully processed
|
||||||
self.last_processed_id = self.to_be_processed_id;
|
self.last_processed_id = self.to_be_processed_id;
|
||||||
self.to_be_processed_id += 1;
|
self.to_be_processed_id += 1;
|
||||||
|
successful_block_process = true;
|
||||||
if let Some(chain) = chain.upgrade() {
|
|
||||||
match chain.fork_choice() {
|
|
||||||
Ok(()) => trace!(
|
|
||||||
log,
|
|
||||||
"Fork choice success";
|
|
||||||
"location" => "batch import success"
|
|
||||||
),
|
|
||||||
Err(e) => error!(
|
|
||||||
log,
|
|
||||||
"Fork choice failed";
|
|
||||||
"error" => format!("{:?}", e),
|
|
||||||
"location" => "batch import success"
|
|
||||||
),
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
Err(e) => {
|
Err(e) => {
|
||||||
warn!(log, "Block processing error"; "error"=> format!("{:?}", e));
|
warn!(log, "Block processing error"; "error"=> format!("{:?}", e));
|
||||||
|
|
||||||
if successes > 0 {
|
if successful_block_process {
|
||||||
if let Some(chain) = chain.upgrade() {
|
if let Some(chain) = chain.upgrade() {
|
||||||
match chain.fork_choice() {
|
match chain.fork_choice() {
|
||||||
Ok(()) => trace!(
|
Ok(()) => trace!(
|
||||||
log,
|
log,
|
||||||
"Fork choice success";
|
"Fork choice success";
|
||||||
"block_imports" => successes,
|
|
||||||
"location" => "batch import error"
|
"location" => "batch import error"
|
||||||
),
|
),
|
||||||
Err(e) => error!(
|
Err(e) => error!(
|
||||||
@ -299,30 +310,41 @@ impl<T: BeaconChainTypes> SyncingChain<T> {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
} else {
|
// If we have processed batches, run fork choice
|
||||||
// there are no more batches to be processed, end
|
if successful_block_process {
|
||||||
break;
|
if let Some(chain) = chain.upgrade() {
|
||||||
|
match chain.fork_choice() {
|
||||||
|
Ok(()) => trace!(
|
||||||
|
log,
|
||||||
|
"Fork choice success";
|
||||||
|
"location" => "batch import success"
|
||||||
|
),
|
||||||
|
Err(e) => error!(
|
||||||
|
log,
|
||||||
|
"Fork choice failed";
|
||||||
|
"error" => format!("{:?}", e),
|
||||||
|
"location" => "batch import success"
|
||||||
|
),
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
// remove any validated batches
|
// remove any validated batches
|
||||||
let last_processed_id = self.last_processed_id;
|
let last_processed_id = self.last_processed_id;
|
||||||
self.completed_batches
|
self.completed_batches
|
||||||
.retain(|batch| batch.id >= last_processed_id);
|
.retain(|batch| batch.id >= last_processed_id);
|
||||||
|
|
||||||
// check if the chain has completed syncing, if not, request another batch from this peer
|
// check if the chain has completed syncing
|
||||||
if self.start_slot + self.last_processed_id * BLOCKS_PER_BATCH >= self.target_head_slot
|
if self.start_slot + self.last_processed_id * BLOCKS_PER_BATCH >= self.target_head_slot {
|
||||||
{
|
|
||||||
// chain is completed
|
// chain is completed
|
||||||
ProcessingResult::RemoveChain
|
ProcessingResult::RemoveChain
|
||||||
} else {
|
} else {
|
||||||
// chain is not completed
|
// chain is not completed
|
||||||
ProcessingResult::KeepChain
|
ProcessingResult::KeepChain
|
||||||
}
|
}
|
||||||
} else {
|
|
||||||
ProcessingResult::KeepChain
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/// An invalid batch has been received that could not be processed.
|
||||||
fn handle_invalid_batch(
|
fn handle_invalid_batch(
|
||||||
&mut self,
|
&mut self,
|
||||||
_chain: Weak<BeaconChain<T>>,
|
_chain: Weak<BeaconChain<T>>,
|
||||||
@ -340,9 +362,10 @@ impl<T: BeaconChainTypes> SyncingChain<T> {
|
|||||||
//
|
//
|
||||||
|
|
||||||
//TODO: Implement this logic
|
//TODO: Implement this logic
|
||||||
// Currently just fail the chain, and drop all associated peers
|
// Currently just fail the chain, and drop all associated peers, removing them from the
|
||||||
for peer_id in self.peer_pool.iter() {
|
// peer pool, to prevent re-status
|
||||||
network.downvote_peer(peer_id.clone());
|
for peer_id in self.peer_pool.drain() {
|
||||||
|
network.downvote_peer(peer_id);
|
||||||
}
|
}
|
||||||
ProcessingResult::RemoveChain
|
ProcessingResult::RemoveChain
|
||||||
}
|
}
|
||||||
@ -352,6 +375,9 @@ impl<T: BeaconChainTypes> SyncingChain<T> {
|
|||||||
}
|
}
|
||||||
|
|
||||||
// Either a new chain, or an old one with a peer list
|
// Either a new chain, or an old one with a peer list
|
||||||
|
/// This chain has been requested to start syncing.
|
||||||
|
///
|
||||||
|
/// This could be new chain, or an old chain that is being resumed.
|
||||||
pub fn start_syncing(
|
pub fn start_syncing(
|
||||||
&mut self,
|
&mut self,
|
||||||
network: &mut SyncNetworkContext,
|
network: &mut SyncNetworkContext,
|
||||||
@ -415,14 +441,15 @@ impl<T: BeaconChainTypes> SyncingChain<T> {
|
|||||||
|
|
||||||
for peer_id in peers {
|
for peer_id in peers {
|
||||||
// send a blocks by range request to the peer
|
// send a blocks by range request to the peer
|
||||||
self.send_range_request(network, peer_id);
|
self.send_range_request(network, peer_id, log);
|
||||||
}
|
}
|
||||||
|
|
||||||
self.state = ChainSyncingState::Syncing;
|
self.state = ChainSyncingState::Syncing;
|
||||||
}
|
}
|
||||||
|
|
||||||
// A peer has been added, start batch requests for this peer
|
/// A peer has been added.
|
||||||
// this should only be called for a syncing chain
|
///
|
||||||
|
/// If the chain is active, this starts requesting batches from this peer.
|
||||||
pub fn peer_added(
|
pub fn peer_added(
|
||||||
&mut self,
|
&mut self,
|
||||||
network: &mut SyncNetworkContext,
|
network: &mut SyncNetworkContext,
|
||||||
@ -436,24 +463,32 @@ impl<T: BeaconChainTypes> SyncingChain<T> {
|
|||||||
}
|
}
|
||||||
|
|
||||||
// find the next batch and request it from the peer
|
// find the next batch and request it from the peer
|
||||||
self.send_range_request(network, peer_id);
|
self.send_range_request(network, peer_id, log);
|
||||||
}
|
}
|
||||||
|
|
||||||
// Re-STATUS all the peers in this chain
|
/// Sends a STATUS message to all peers in the peer pool.
|
||||||
pub fn status_peers(&self, chain: Weak<BeaconChain<T>>, network: &mut SyncNetworkContext) {
|
pub fn status_peers(&self, chain: Weak<BeaconChain<T>>, network: &mut SyncNetworkContext) {
|
||||||
for peer_id in self.peer_pool.iter() {
|
for peer_id in self.peer_pool.iter() {
|
||||||
network.status_peer(chain.clone(), peer_id.clone());
|
network.status_peer(chain.clone(), peer_id.clone());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
fn send_range_request(&mut self, network: &mut SyncNetworkContext, peer_id: PeerId) {
|
/// Requests the next required batch from the provided peer.
|
||||||
|
fn send_range_request(
|
||||||
|
&mut self,
|
||||||
|
network: &mut SyncNetworkContext,
|
||||||
|
peer_id: PeerId,
|
||||||
|
log: &slog::Logger,
|
||||||
|
) {
|
||||||
// find the next pending batch and request it from the peer
|
// find the next pending batch and request it from the peer
|
||||||
if let Some(batch) = self.get_next_batch(peer_id) {
|
if let Some(batch) = self.get_next_batch(peer_id) {
|
||||||
|
debug!(log, "Requesting batch"; "start_slot" => batch.start_slot, "end_slot" => batch.end_slot, "id" => batch.id, "peer" => format!("{:?}", batch.current_peer), "head_root"=> format!("{}", batch.head_root));
|
||||||
// send the batch
|
// send the batch
|
||||||
self.send_batch(network, batch);
|
self.send_batch(network, batch);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/// Requests the provided batch from the provided peer.
|
||||||
fn send_batch(&mut self, network: &mut SyncNetworkContext, batch: Batch<T::EthSpec>) {
|
fn send_batch(&mut self, network: &mut SyncNetworkContext, batch: Batch<T::EthSpec>) {
|
||||||
let request = batch.to_blocks_by_range_request();
|
let request = batch.to_blocks_by_range_request();
|
||||||
if let Ok(request_id) = network.blocks_by_range_request(batch.current_peer.clone(), request)
|
if let Ok(request_id) = network.blocks_by_range_request(batch.current_peer.clone(), request)
|
||||||
@ -463,6 +498,8 @@ impl<T: BeaconChainTypes> SyncingChain<T> {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/// Returns the next required batch from the chain if it exists. If there are no more batches
|
||||||
|
/// required, `None` is returned.
|
||||||
fn get_next_batch(&mut self, peer_id: PeerId) -> Option<Batch<T::EthSpec>> {
|
fn get_next_batch(&mut self, peer_id: PeerId) -> Option<Batch<T::EthSpec>> {
|
||||||
let batch_start_slot =
|
let batch_start_slot =
|
||||||
self.start_slot + self.to_be_downloaded_id.saturating_sub(1) * BLOCKS_PER_BATCH;
|
self.start_slot + self.to_be_downloaded_id.saturating_sub(1) * BLOCKS_PER_BATCH;
|
||||||
@ -493,9 +530,12 @@ impl<T: BeaconChainTypes> SyncingChain<T> {
|
|||||||
))
|
))
|
||||||
}
|
}
|
||||||
|
|
||||||
// Checks if the request_id is associated with this chain. If so, attempts to re-request the
|
/// An RPC error has occurred.
|
||||||
// batch. If the batch has exceeded the number of retries, returns Some(true), indicating
|
///
|
||||||
// the chain should be dropped.
|
/// Checks if the request_id is associated with this chain. If so, attempts to re-request the
|
||||||
|
/// batch. If the batch has exceeded the number of retries, returns
|
||||||
|
/// Some(`ProcessingResult::RemoveChain)`. Returns `None` if the request isn't related to
|
||||||
|
/// this chain.
|
||||||
pub fn inject_error(
|
pub fn inject_error(
|
||||||
&mut self,
|
&mut self,
|
||||||
network: &mut SyncNetworkContext,
|
network: &mut SyncNetworkContext,
|
||||||
@ -512,6 +552,11 @@ impl<T: BeaconChainTypes> SyncingChain<T> {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/// A batch has failed.
|
||||||
|
///
|
||||||
|
/// Attempts to re-request from another peer in the peer pool (if possible) and returns
|
||||||
|
/// `ProcessingResult::RemoveChain` if the number of retries on the batch exceeds
|
||||||
|
/// `MAX_BATCH_RETRIES`.
|
||||||
pub fn failed_batch(
|
pub fn failed_batch(
|
||||||
&mut self,
|
&mut self,
|
||||||
network: &mut SyncNetworkContext,
|
network: &mut SyncNetworkContext,
|
||||||
@ -542,7 +587,6 @@ impl<T: BeaconChainTypes> SyncingChain<T> {
|
|||||||
fn process_batch<T: BeaconChainTypes>(
|
fn process_batch<T: BeaconChainTypes>(
|
||||||
chain: Weak<BeaconChain<T>>,
|
chain: Weak<BeaconChain<T>>,
|
||||||
batch: &Batch<T::EthSpec>,
|
batch: &Batch<T::EthSpec>,
|
||||||
successes: &mut usize,
|
|
||||||
log: &Logger,
|
log: &Logger,
|
||||||
) -> Result<(), String> {
|
) -> Result<(), String> {
|
||||||
for block in &batch.downloaded_blocks {
|
for block in &batch.downloaded_blocks {
|
||||||
@ -558,8 +602,6 @@ fn process_batch<T: BeaconChainTypes>(
|
|||||||
"slot" => block.slot,
|
"slot" => block.slot,
|
||||||
"block_root" => format!("{}", block_root),
|
"block_root" => format!("{}", block_root),
|
||||||
);
|
);
|
||||||
|
|
||||||
*successes += 1
|
|
||||||
}
|
}
|
||||||
BlockProcessingOutcome::ParentUnknown { parent } => {
|
BlockProcessingOutcome::ParentUnknown { parent } => {
|
||||||
// blocks should be sequential and all parents should exist
|
// blocks should be sequential and all parents should exist
|
||||||
|
@ -1,3 +1,8 @@
|
|||||||
|
//! This provides the logic for the finalized and head chains.
|
||||||
|
//!
|
||||||
|
//! Each chain type is stored in it's own vector. A variety of helper functions are given along
|
||||||
|
//! with this struct to to simplify the logic of the other layers of sync.
|
||||||
|
|
||||||
use super::chain::{ChainSyncingState, ProcessingResult, SyncingChain};
|
use super::chain::{ChainSyncingState, ProcessingResult, SyncingChain};
|
||||||
use crate::message_processor::PeerSyncInfo;
|
use crate::message_processor::PeerSyncInfo;
|
||||||
use crate::sync::network_context::SyncNetworkContext;
|
use crate::sync::network_context::SyncNetworkContext;
|
||||||
@ -8,32 +13,47 @@ use std::sync::Weak;
|
|||||||
use types::EthSpec;
|
use types::EthSpec;
|
||||||
use types::{Hash256, Slot};
|
use types::{Hash256, Slot};
|
||||||
|
|
||||||
|
/// The state of the long range/batch sync.
|
||||||
pub enum SyncState {
|
pub enum SyncState {
|
||||||
|
/// A finalized chain is being synced.
|
||||||
Finalized,
|
Finalized,
|
||||||
|
/// There are no finalized chains and we are syncing one more head chains.
|
||||||
Head,
|
Head,
|
||||||
|
/// There are no head or finalized chains and no long range sync is in progress.
|
||||||
Idle,
|
Idle,
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/// A collection of finalized and head chains currently being processed.
|
||||||
pub struct ChainCollection<T: BeaconChainTypes> {
|
pub struct ChainCollection<T: BeaconChainTypes> {
|
||||||
|
/// The beacon chain for processing.
|
||||||
|
beacon_chain: Weak<BeaconChain<T>>,
|
||||||
|
/// The set of finalized chains being synced.
|
||||||
finalized_chains: Vec<SyncingChain<T>>,
|
finalized_chains: Vec<SyncingChain<T>>,
|
||||||
|
/// The set of head chains being synced.
|
||||||
head_chains: Vec<SyncingChain<T>>,
|
head_chains: Vec<SyncingChain<T>>,
|
||||||
|
/// The current sync state of the process.
|
||||||
sync_state: SyncState,
|
sync_state: SyncState,
|
||||||
}
|
}
|
||||||
|
|
||||||
impl<T: BeaconChainTypes> ChainCollection<T> {
|
impl<T: BeaconChainTypes> ChainCollection<T> {
|
||||||
pub fn new() -> Self {
|
pub fn new(beacon_chain: Weak<BeaconChain<T>>) -> Self {
|
||||||
ChainCollection {
|
ChainCollection {
|
||||||
sync_state: SyncState::Idle,
|
sync_state: SyncState::Idle,
|
||||||
finalized_chains: Vec::new(),
|
finalized_chains: Vec::new(),
|
||||||
head_chains: Vec::new(),
|
head_chains: Vec::new(),
|
||||||
|
beacon_chain,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/// The current syncing state.
|
||||||
pub fn sync_state(&self) -> &SyncState {
|
pub fn sync_state(&self) -> &SyncState {
|
||||||
&self.sync_state
|
&self.sync_state
|
||||||
}
|
}
|
||||||
|
|
||||||
// if a finalized chain just completed, we assume we waiting for head syncing, unless a fully
|
/// A fully synced peer has joined.
|
||||||
// sync peer joins.
|
///
|
||||||
|
/// We could be awaiting a head sync. If we are in the head syncing state, without any head
|
||||||
|
/// chains, then update the state to idle.
|
||||||
pub fn fully_synced_peer_found(&mut self) {
|
pub fn fully_synced_peer_found(&mut self) {
|
||||||
if let SyncState::Head = self.sync_state {
|
if let SyncState::Head = self.sync_state {
|
||||||
if self.head_chains.is_empty() {
|
if self.head_chains.is_empty() {
|
||||||
@ -42,47 +62,15 @@ impl<T: BeaconChainTypes> ChainCollection<T> {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// after a finalized chain completes, the state should be waiting for a head chain
|
/// After a finalized chain completes this function is called. It ensures the state is set to
|
||||||
|
/// `SyncState::Head` indicating we are awaiting new peers to connect before we can consider
|
||||||
|
/// the state as idle.
|
||||||
pub fn set_head_sync(&mut self) {
|
pub fn set_head_sync(&mut self) {
|
||||||
if let SyncState::Idle = self.sync_state {
|
if let SyncState::Idle = self.sync_state {
|
||||||
self.sync_state = SyncState::Head;
|
self.sync_state = SyncState::Head;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
fn finalized_syncing_index(&self) -> Option<usize> {
|
|
||||||
self.finalized_chains
|
|
||||||
.iter()
|
|
||||||
.enumerate()
|
|
||||||
.find_map(|(index, chain)| {
|
|
||||||
if chain.state == ChainSyncingState::Syncing {
|
|
||||||
Some(index)
|
|
||||||
} else {
|
|
||||||
None
|
|
||||||
}
|
|
||||||
})
|
|
||||||
}
|
|
||||||
|
|
||||||
pub fn purge_finalized(&mut self, local_finalized_slot: Slot) {
|
|
||||||
self.finalized_chains
|
|
||||||
.retain(|chain| chain.target_head_slot > local_finalized_slot);
|
|
||||||
}
|
|
||||||
|
|
||||||
pub fn purge_head(&mut self, head_slot: Slot) {
|
|
||||||
self.head_chains
|
|
||||||
.retain(|chain| chain.target_head_slot > head_slot);
|
|
||||||
}
|
|
||||||
|
|
||||||
fn get_chain<'a>(
|
|
||||||
chain: &'a mut [SyncingChain<T>],
|
|
||||||
target_head_root: Hash256,
|
|
||||||
target_head_slot: Slot,
|
|
||||||
) -> Option<&'a mut SyncingChain<T>> {
|
|
||||||
chain.iter_mut().find(|iter_chain| {
|
|
||||||
iter_chain.target_head_root == target_head_root
|
|
||||||
&& iter_chain.target_head_slot == target_head_slot
|
|
||||||
})
|
|
||||||
}
|
|
||||||
|
|
||||||
/// Finds any finalized chain if it exists.
|
/// Finds any finalized chain if it exists.
|
||||||
pub fn get_finalized_mut(
|
pub fn get_finalized_mut(
|
||||||
&mut self,
|
&mut self,
|
||||||
@ -109,35 +97,23 @@ impl<T: BeaconChainTypes> ChainCollection<T> {
|
|||||||
)
|
)
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Checks if a new finalized state should become the syncing chain. Updates the state of the
|
/// Updates the state of the chain collection.
|
||||||
/// collection.
|
///
|
||||||
pub fn update_finalized(
|
/// This removes any out-dated chains, swaps to any higher priority finalized chains and
|
||||||
&mut self,
|
/// updates the state of the collection.
|
||||||
beacon_chain: Weak<BeaconChain<T>>,
|
pub fn update_finalized(&mut self, network: &mut SyncNetworkContext, log: &slog::Logger) {
|
||||||
network: &mut SyncNetworkContext,
|
let local_slot = match self.beacon_chain.upgrade() {
|
||||||
log: &slog::Logger,
|
Some(chain) => PeerSyncInfo::from(&chain)
|
||||||
) {
|
.finalized_epoch
|
||||||
let local_info = match beacon_chain.upgrade() {
|
.start_slot(T::EthSpec::slots_per_epoch()),
|
||||||
Some(chain) => PeerSyncInfo::from(&chain),
|
|
||||||
None => {
|
None => {
|
||||||
warn!(log, "Beacon chain dropped. Chains not updated");
|
warn!(log, "Beacon chain dropped. Chains not updated");
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
let local_slot = local_info
|
|
||||||
.finalized_epoch
|
|
||||||
.start_slot(T::EthSpec::slots_per_epoch());
|
|
||||||
|
|
||||||
// Remove any outdated finalized chains
|
// Remove any outdated finalized chains
|
||||||
self.purge_finalized(local_slot);
|
self.purge_outdated_chains(network);
|
||||||
self.finalized_chains
|
|
||||||
.retain(|chain| !chain.peer_pool.is_empty());
|
|
||||||
|
|
||||||
// Remove any outdated head chains
|
|
||||||
self.purge_head(local_info.head_slot);
|
|
||||||
self.finalized_chains
|
|
||||||
.retain(|chain| !chain.peer_pool.is_empty());
|
|
||||||
|
|
||||||
// Check if any chains become the new syncing chain
|
// Check if any chains become the new syncing chain
|
||||||
if let Some(index) = self.finalized_syncing_index() {
|
if let Some(index) = self.finalized_syncing_index() {
|
||||||
@ -172,7 +148,7 @@ impl<T: BeaconChainTypes> ChainCollection<T> {
|
|||||||
chain.start_syncing(network, local_slot, log);
|
chain.start_syncing(network, local_slot, log);
|
||||||
self.sync_state = SyncState::Finalized;
|
self.sync_state = SyncState::Finalized;
|
||||||
} else {
|
} else {
|
||||||
// There are no finalized chains, update the state
|
// There are no finalized chains, update the state.
|
||||||
if self.head_chains.is_empty() {
|
if self.head_chains.is_empty() {
|
||||||
self.sync_state = SyncState::Idle;
|
self.sync_state = SyncState::Idle;
|
||||||
} else {
|
} else {
|
||||||
@ -181,7 +157,7 @@ impl<T: BeaconChainTypes> ChainCollection<T> {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Add a new finalized chain to the collection
|
/// Add a new finalized chain to the collection.
|
||||||
pub fn new_finalized_chain(
|
pub fn new_finalized_chain(
|
||||||
&mut self,
|
&mut self,
|
||||||
local_finalized_slot: Slot,
|
local_finalized_slot: Slot,
|
||||||
@ -197,7 +173,7 @@ impl<T: BeaconChainTypes> ChainCollection<T> {
|
|||||||
));
|
));
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Add a new finalized chain to the collection
|
/// Add a new finalized chain to the collection and starts syncing it.
|
||||||
pub fn new_head_chain(
|
pub fn new_head_chain(
|
||||||
&mut self,
|
&mut self,
|
||||||
network: &mut SyncNetworkContext,
|
network: &mut SyncNetworkContext,
|
||||||
@ -221,10 +197,14 @@ impl<T: BeaconChainTypes> ChainCollection<T> {
|
|||||||
self.head_chains.push(new_head_chain);
|
self.head_chains.push(new_head_chain);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/// Returns if `true` if any finalized chains exist, `false` otherwise.
|
||||||
pub fn is_finalizing_sync(&self) -> bool {
|
pub fn is_finalizing_sync(&self) -> bool {
|
||||||
!self.finalized_chains.is_empty()
|
!self.finalized_chains.is_empty()
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/// Given a chain iterator, runs a given function on each chain until the function returns
|
||||||
|
/// `Some`. This allows the `RangeSync` struct to loop over chains and optionally remove the
|
||||||
|
/// chain from the collection if the function results in completing the chain.
|
||||||
fn request_function<'a, F, I>(chain: I, mut func: F) -> Option<(usize, ProcessingResult)>
|
fn request_function<'a, F, I>(chain: I, mut func: F) -> Option<(usize, ProcessingResult)>
|
||||||
where
|
where
|
||||||
I: Iterator<Item = &'a mut SyncingChain<T>>,
|
I: Iterator<Item = &'a mut SyncingChain<T>>,
|
||||||
@ -235,6 +215,7 @@ impl<T: BeaconChainTypes> ChainCollection<T> {
|
|||||||
.find_map(|(index, chain)| Some((index, func(chain)?)))
|
.find_map(|(index, chain)| Some((index, func(chain)?)))
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/// Runs a function on all finalized chains.
|
||||||
pub fn finalized_request<F>(&mut self, func: F) -> Option<(usize, ProcessingResult)>
|
pub fn finalized_request<F>(&mut self, func: F) -> Option<(usize, ProcessingResult)>
|
||||||
where
|
where
|
||||||
F: FnMut(&mut SyncingChain<T>) -> Option<ProcessingResult>,
|
F: FnMut(&mut SyncingChain<T>) -> Option<ProcessingResult>,
|
||||||
@ -242,6 +223,7 @@ impl<T: BeaconChainTypes> ChainCollection<T> {
|
|||||||
ChainCollection::request_function(self.finalized_chains.iter_mut(), func)
|
ChainCollection::request_function(self.finalized_chains.iter_mut(), func)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/// Runs a function on all head chains.
|
||||||
pub fn head_request<F>(&mut self, func: F) -> Option<(usize, ProcessingResult)>
|
pub fn head_request<F>(&mut self, func: F) -> Option<(usize, ProcessingResult)>
|
||||||
where
|
where
|
||||||
F: FnMut(&mut SyncingChain<T>) -> Option<ProcessingResult>,
|
F: FnMut(&mut SyncingChain<T>) -> Option<ProcessingResult>,
|
||||||
@ -249,7 +231,7 @@ impl<T: BeaconChainTypes> ChainCollection<T> {
|
|||||||
ChainCollection::request_function(self.head_chains.iter_mut(), func)
|
ChainCollection::request_function(self.head_chains.iter_mut(), func)
|
||||||
}
|
}
|
||||||
|
|
||||||
#[allow(dead_code)]
|
/// Runs a function on all finalized and head chains.
|
||||||
pub fn head_finalized_request<F>(&mut self, func: F) -> Option<(usize, ProcessingResult)>
|
pub fn head_finalized_request<F>(&mut self, func: F) -> Option<(usize, ProcessingResult)>
|
||||||
where
|
where
|
||||||
F: FnMut(&mut SyncingChain<T>) -> Option<ProcessingResult>,
|
F: FnMut(&mut SyncingChain<T>) -> Option<ProcessingResult>,
|
||||||
@ -262,10 +244,53 @@ impl<T: BeaconChainTypes> ChainCollection<T> {
|
|||||||
)
|
)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/// Removes any outdated finalized or head chains.
|
||||||
|
///
|
||||||
|
/// This removes chains with no peers, or chains whose start block slot is less than our current
|
||||||
|
/// finalized block slot.
|
||||||
|
pub fn purge_outdated_chains(&mut self, network: &mut SyncNetworkContext) {
|
||||||
|
// Remove any chains that have no peers
|
||||||
|
self.finalized_chains
|
||||||
|
.retain(|chain| !chain.peer_pool.is_empty());
|
||||||
|
self.head_chains.retain(|chain| !chain.peer_pool.is_empty());
|
||||||
|
|
||||||
|
let local_info = match self.beacon_chain.upgrade() {
|
||||||
|
Some(chain) => PeerSyncInfo::from(&chain),
|
||||||
|
None => {
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
let local_finalized_slot = local_info
|
||||||
|
.finalized_epoch
|
||||||
|
.start_slot(T::EthSpec::slots_per_epoch());
|
||||||
|
|
||||||
|
// Remove chains that are out-dated and re-status their peers
|
||||||
|
let beacon_chain_clone = self.beacon_chain.clone();
|
||||||
|
self.finalized_chains.retain(|chain| {
|
||||||
|
if chain.target_head_slot <= local_finalized_slot {
|
||||||
|
chain.status_peers(beacon_chain_clone.clone(), network);
|
||||||
|
false
|
||||||
|
} else {
|
||||||
|
true
|
||||||
|
}
|
||||||
|
});
|
||||||
|
self.head_chains.retain(|chain| {
|
||||||
|
if chain.target_head_slot <= local_finalized_slot {
|
||||||
|
chain.status_peers(beacon_chain_clone.clone(), network);
|
||||||
|
false
|
||||||
|
} else {
|
||||||
|
true
|
||||||
|
}
|
||||||
|
});
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Removes and returns a finalized chain from the collection.
|
||||||
pub fn remove_finalized_chain(&mut self, index: usize) -> SyncingChain<T> {
|
pub fn remove_finalized_chain(&mut self, index: usize) -> SyncingChain<T> {
|
||||||
self.finalized_chains.swap_remove(index)
|
self.finalized_chains.swap_remove(index)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/// Removes and returns a head chain from the collection.
|
||||||
pub fn remove_head_chain(&mut self, index: usize) -> SyncingChain<T> {
|
pub fn remove_head_chain(&mut self, index: usize) -> SyncingChain<T> {
|
||||||
self.head_chains.swap_remove(index)
|
self.head_chains.swap_remove(index)
|
||||||
}
|
}
|
||||||
@ -273,12 +298,55 @@ impl<T: BeaconChainTypes> ChainCollection<T> {
|
|||||||
/// Removes a chain from either finalized or head chains based on the index. Using a request
|
/// Removes a chain from either finalized or head chains based on the index. Using a request
|
||||||
/// iterates of finalized chains before head chains. Thus an index that is greater than the
|
/// iterates of finalized chains before head chains. Thus an index that is greater than the
|
||||||
/// finalized chain length, indicates a head chain.
|
/// finalized chain length, indicates a head chain.
|
||||||
pub fn remove_chain(&mut self, index: usize) -> SyncingChain<T> {
|
///
|
||||||
if index >= self.finalized_chains.len() {
|
/// This will re-status the chains peers on removal. The index must exist.
|
||||||
|
pub fn remove_chain(
|
||||||
|
&mut self,
|
||||||
|
network: &mut SyncNetworkContext,
|
||||||
|
index: usize,
|
||||||
|
log: &slog::Logger,
|
||||||
|
) {
|
||||||
|
let chain = if index >= self.finalized_chains.len() {
|
||||||
let index = index - self.finalized_chains.len();
|
let index = index - self.finalized_chains.len();
|
||||||
self.head_chains.swap_remove(index)
|
let chain = self.head_chains.swap_remove(index);
|
||||||
|
chain.status_peers(self.beacon_chain.clone(), network);
|
||||||
|
chain
|
||||||
} else {
|
} else {
|
||||||
self.finalized_chains.swap_remove(index)
|
let chain = self.finalized_chains.swap_remove(index);
|
||||||
|
chain.status_peers(self.beacon_chain.clone(), network);
|
||||||
|
chain
|
||||||
|
};
|
||||||
|
|
||||||
|
debug!(log, "Chain was removed"; "start_slot" => chain.start_slot.as_u64(), "end_slot" => chain.target_head_slot.as_u64());
|
||||||
|
|
||||||
|
// update the state
|
||||||
|
self.update_finalized(network, log);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/// Returns the index of finalized chain that is currently syncing. Returns `None` if no
|
||||||
|
/// finalized chain is currently syncing.
|
||||||
|
fn finalized_syncing_index(&self) -> Option<usize> {
|
||||||
|
self.finalized_chains
|
||||||
|
.iter()
|
||||||
|
.enumerate()
|
||||||
|
.find_map(|(index, chain)| {
|
||||||
|
if chain.state == ChainSyncingState::Syncing {
|
||||||
|
Some(index)
|
||||||
|
} else {
|
||||||
|
None
|
||||||
|
}
|
||||||
|
})
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Returns a chain given the target head root and slot.
|
||||||
|
fn get_chain<'a>(
|
||||||
|
chain: &'a mut [SyncingChain<T>],
|
||||||
|
target_head_root: Hash256,
|
||||||
|
target_head_slot: Slot,
|
||||||
|
) -> Option<&'a mut SyncingChain<T>> {
|
||||||
|
chain.iter_mut().find(|iter_chain| {
|
||||||
|
iter_chain.target_head_root == target_head_root
|
||||||
|
&& iter_chain.target_head_slot == target_head_slot
|
||||||
|
})
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -1,3 +1,44 @@
|
|||||||
|
//! This contains the logic for the long range (batch) sync strategy.
|
||||||
|
//!
|
||||||
|
//! The general premise is to group peers by their self-proclaimed finalized blocks and head
|
||||||
|
//! blocks. Once grouped, the peers become sources to download a specific `Chain`. A `Chain` is a
|
||||||
|
//! collection of blocks that terminates at the specified target head.
|
||||||
|
//!
|
||||||
|
//! This sync strategy can be separated into two distinct forms:
|
||||||
|
//! - Finalized Chain Sync
|
||||||
|
//! - Head Chain Sync
|
||||||
|
//!
|
||||||
|
//! ## Finalized chain sync
|
||||||
|
//!
|
||||||
|
//! This occurs when a peer connects that claims to have a finalized head slot that is greater
|
||||||
|
//! than our own. In this case, we form a chain from our last finalized slot, to their claimed
|
||||||
|
//! finalized slot. Any peer that also claims to have this last finalized slot is added to a pool
|
||||||
|
//! of peers from which batches of blocks may be downloaded. Blocks are downloaded until
|
||||||
|
//! the finalized slot of the chain is reached. Once reached, all peers within the pool are sent a
|
||||||
|
//! STATUS message to potentially start a head chain sync, or check if further finalized chains
|
||||||
|
//! need to be downloaded.
|
||||||
|
//!
|
||||||
|
//! A few interesting notes about finalized chain syncing:
|
||||||
|
//! - Only one finalized chain can sync at a time.
|
||||||
|
//! - The finalized chain with the largest peer pool takes priority.
|
||||||
|
//! - As one finalized chain completes, others are checked to see if we they can be continued,
|
||||||
|
//! otherwise they are removed.
|
||||||
|
//!
|
||||||
|
//! ## Head Chain Sync
|
||||||
|
//!
|
||||||
|
//! If a peer joins and there is no active finalized chains being synced, and it's head is
|
||||||
|
//! beyond our `SLOT_IMPORT_TOLERANCE` a chain is formed starting from this peers finalized slot
|
||||||
|
//! (this has been necessarily downloaded by our node, otherwise we would start a finalized chain
|
||||||
|
//! sync) to this peers head slot. Any other peers that match this head slot and head root, are
|
||||||
|
//! added to this chain's peer pool, which will be downloaded in parallel.
|
||||||
|
//!
|
||||||
|
//! Unlike finalized chains, head chains can be synced in parallel.
|
||||||
|
//!
|
||||||
|
//! ## Batch Syncing
|
||||||
|
//!
|
||||||
|
//! Each chain is downloaded in batches of blocks. The batched blocks are processed sequentially
|
||||||
|
//! and further batches are requested as current blocks are being processed.
|
||||||
|
|
||||||
use super::chain::ProcessingResult;
|
use super::chain::ProcessingResult;
|
||||||
use super::chain_collection::{ChainCollection, SyncState};
|
use super::chain_collection::{ChainCollection, SyncState};
|
||||||
use crate::message_processor::PeerSyncInfo;
|
use crate::message_processor::PeerSyncInfo;
|
||||||
@ -10,38 +51,50 @@ use std::collections::HashSet;
|
|||||||
use std::sync::Weak;
|
use std::sync::Weak;
|
||||||
use types::{BeaconBlock, EthSpec};
|
use types::{BeaconBlock, EthSpec};
|
||||||
|
|
||||||
//TODO: The code becomes cleaner if finalized_chains and head_chains were merged into a single
|
/// The primary object dealing with long range/batch syncing. This contains all the active and
|
||||||
// object. This will prevent code duplication. Rather than keeping the current syncing
|
/// non-active chains that need to be processed before the syncing is considered complete. This
|
||||||
// finalized chain in index 0, it should be stored in this object under an option. Then lookups can
|
/// holds the current state of the long range sync.
|
||||||
// occur over the single object containing both finalized and head chains, which would then
|
|
||||||
// behave similarly.
|
|
||||||
|
|
||||||
pub struct RangeSync<T: BeaconChainTypes> {
|
pub struct RangeSync<T: BeaconChainTypes> {
|
||||||
/// The beacon chain for processing
|
/// The beacon chain for processing.
|
||||||
beacon_chain: Weak<BeaconChain<T>>,
|
beacon_chain: Weak<BeaconChain<T>>,
|
||||||
|
/// A collection of chains that need to be downloaded. This stores any head or finalized chains
|
||||||
|
/// that need to be downloaded.
|
||||||
chains: ChainCollection<T>,
|
chains: ChainCollection<T>,
|
||||||
/// Known peers to the RangeSync, that need to be re-status'd once finalized chains are
|
/// Peers that join whilst a finalized chain is being download, sit in this set. Once the
|
||||||
/// completed.
|
/// finalized chain(s) complete, these peer's get STATUS'ed to update their head slot before
|
||||||
|
/// the head chains are formed and downloaded.
|
||||||
awaiting_head_peers: HashSet<PeerId>,
|
awaiting_head_peers: HashSet<PeerId>,
|
||||||
|
/// The syncing logger.
|
||||||
log: slog::Logger,
|
log: slog::Logger,
|
||||||
}
|
}
|
||||||
|
|
||||||
impl<T: BeaconChainTypes> RangeSync<T> {
|
impl<T: BeaconChainTypes> RangeSync<T> {
|
||||||
pub fn new(beacon_chain: Weak<BeaconChain<T>>, log: slog::Logger) -> Self {
|
pub fn new(beacon_chain: Weak<BeaconChain<T>>, log: slog::Logger) -> Self {
|
||||||
RangeSync {
|
RangeSync {
|
||||||
beacon_chain,
|
beacon_chain: beacon_chain.clone(),
|
||||||
chains: ChainCollection::new(),
|
chains: ChainCollection::new(beacon_chain),
|
||||||
awaiting_head_peers: HashSet::new(),
|
awaiting_head_peers: HashSet::new(),
|
||||||
log,
|
log,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// Notify the collection that a fully synced peer was found. This allows updating the state
|
/// The `chains` collection stores the current state of syncing. Once a finalized chain
|
||||||
// if we were awaiting a head state.
|
/// completes, it's state is pre-emptively set to `SyncState::Head`. This ensures that
|
||||||
|
/// during the transition period of finalized to head, the sync manager doesn't start
|
||||||
|
/// requesting blocks from gossipsub.
|
||||||
|
///
|
||||||
|
/// On re-status, a peer that has no head to download indicates that this state can be set to
|
||||||
|
/// idle as there are in fact no head chains to download. This function notifies the chain
|
||||||
|
/// collection that the state can safely be set to idle.
|
||||||
pub fn fully_synced_peer_found(&mut self) {
|
pub fn fully_synced_peer_found(&mut self) {
|
||||||
self.chains.fully_synced_peer_found()
|
self.chains.fully_synced_peer_found()
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/// A useful peer has been added. The SyncManager has identified this peer as needing either
|
||||||
|
/// a finalized or head chain sync. This processes the peer and starts/resumes any chain that
|
||||||
|
/// may need to be synced as a result. A new peer, may increase the peer pool of a finalized
|
||||||
|
/// chain, this may result in a different finalized chain from syncing as finalized chains are
|
||||||
|
/// prioritised by peer-pool size.
|
||||||
pub fn add_peer(
|
pub fn add_peer(
|
||||||
&mut self,
|
&mut self,
|
||||||
network: &mut SyncNetworkContext,
|
network: &mut SyncNetworkContext,
|
||||||
@ -70,13 +123,12 @@ impl<T: BeaconChainTypes> RangeSync<T> {
|
|||||||
.finalized_epoch
|
.finalized_epoch
|
||||||
.start_slot(T::EthSpec::slots_per_epoch());
|
.start_slot(T::EthSpec::slots_per_epoch());
|
||||||
|
|
||||||
// firstly, remove any out-of-date chains
|
|
||||||
self.chains.purge_finalized(local_finalized_slot);
|
|
||||||
self.chains.purge_head(local_info.head_slot);
|
|
||||||
|
|
||||||
// remove peer from any chains
|
// remove peer from any chains
|
||||||
self.remove_peer(network, &peer_id);
|
self.remove_peer(network, &peer_id);
|
||||||
|
|
||||||
|
// remove any out-of-date chains
|
||||||
|
self.chains.purge_outdated_chains(network);
|
||||||
|
|
||||||
if remote_finalized_slot > local_info.head_slot {
|
if remote_finalized_slot > local_info.head_slot {
|
||||||
debug!(self.log, "Finalization sync peer joined"; "peer_id" => format!("{:?}", peer_id));
|
debug!(self.log, "Finalization sync peer joined"; "peer_id" => format!("{:?}", peer_id));
|
||||||
// Finalized chain search
|
// Finalized chain search
|
||||||
@ -97,8 +149,7 @@ impl<T: BeaconChainTypes> RangeSync<T> {
|
|||||||
chain.peer_added(network, peer_id, &self.log);
|
chain.peer_added(network, peer_id, &self.log);
|
||||||
|
|
||||||
// check if the new peer's addition will favour a new syncing chain.
|
// check if the new peer's addition will favour a new syncing chain.
|
||||||
self.chains
|
self.chains.update_finalized(network, &self.log);
|
||||||
.update_finalized(self.beacon_chain.clone(), network, &self.log);
|
|
||||||
} else {
|
} else {
|
||||||
// there is no finalized chain that matches this peer's last finalized target
|
// there is no finalized chain that matches this peer's last finalized target
|
||||||
// create a new finalized chain
|
// create a new finalized chain
|
||||||
@ -110,8 +161,7 @@ impl<T: BeaconChainTypes> RangeSync<T> {
|
|||||||
remote_finalized_slot,
|
remote_finalized_slot,
|
||||||
peer_id,
|
peer_id,
|
||||||
);
|
);
|
||||||
self.chains
|
self.chains.update_finalized(network, &self.log);
|
||||||
.update_finalized(self.beacon_chain.clone(), network, &self.log);
|
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
if self.chains.is_finalizing_sync() {
|
if self.chains.is_finalizing_sync() {
|
||||||
@ -145,11 +195,14 @@ impl<T: BeaconChainTypes> RangeSync<T> {
|
|||||||
&self.log,
|
&self.log,
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
self.chains
|
self.chains.update_finalized(network, &self.log);
|
||||||
.update_finalized(self.beacon_chain.clone(), network, &self.log);
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/// A `BlocksByRange` response has been received from the network.
|
||||||
|
///
|
||||||
|
/// This function finds the chain that made this request. Once found, processes the result.
|
||||||
|
/// This request could complete a chain or simply add to its progress.
|
||||||
pub fn blocks_by_range_response(
|
pub fn blocks_by_range_response(
|
||||||
&mut self,
|
&mut self,
|
||||||
network: &mut SyncNetworkContext,
|
network: &mut SyncNetworkContext,
|
||||||
@ -162,7 +215,6 @@ impl<T: BeaconChainTypes> RangeSync<T> {
|
|||||||
// `connected_peers` number of head chains, which should be relatively small and this
|
// `connected_peers` number of head chains, which should be relatively small and this
|
||||||
// lookup should not be very expensive. However, we could add an extra index that maps the
|
// lookup should not be very expensive. However, we could add an extra index that maps the
|
||||||
// request id to index of the vector to avoid O(N) searches and O(N) hash lookups.
|
// request id to index of the vector to avoid O(N) searches and O(N) hash lookups.
|
||||||
// Note to future sync-rewriter/profiler: Michael approves of these O(N) searches.
|
|
||||||
|
|
||||||
let chain_ref = &self.beacon_chain;
|
let chain_ref = &self.beacon_chain;
|
||||||
let log_ref = &self.log;
|
let log_ref = &self.log;
|
||||||
@ -177,8 +229,7 @@ impl<T: BeaconChainTypes> RangeSync<T> {
|
|||||||
chain.status_peers(self.beacon_chain.clone(), network);
|
chain.status_peers(self.beacon_chain.clone(), network);
|
||||||
|
|
||||||
// update the state of the collection
|
// update the state of the collection
|
||||||
self.chains
|
self.chains.update_finalized(network, &self.log);
|
||||||
.update_finalized(self.beacon_chain.clone(), network, &self.log);
|
|
||||||
|
|
||||||
// set the state to a head sync, to inform the manager that we are awaiting a
|
// set the state to a head sync, to inform the manager that we are awaiting a
|
||||||
// head chain.
|
// head chain.
|
||||||
@ -207,8 +258,7 @@ impl<T: BeaconChainTypes> RangeSync<T> {
|
|||||||
chain.status_peers(self.beacon_chain.clone(), network);
|
chain.status_peers(self.beacon_chain.clone(), network);
|
||||||
|
|
||||||
// update the state of the collection
|
// update the state of the collection
|
||||||
self.chains
|
self.chains.update_finalized(network, &self.log);
|
||||||
.update_finalized(self.beacon_chain.clone(), network, &self.log);
|
|
||||||
}
|
}
|
||||||
Some(_) => {}
|
Some(_) => {}
|
||||||
None => {
|
None => {
|
||||||
@ -221,6 +271,7 @@ impl<T: BeaconChainTypes> RangeSync<T> {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/// Public method to indicate the current state of the long range sync.
|
||||||
pub fn is_syncing(&self) -> bool {
|
pub fn is_syncing(&self) -> bool {
|
||||||
match self.chains.sync_state() {
|
match self.chains.sync_state() {
|
||||||
SyncState::Finalized => true,
|
SyncState::Finalized => true,
|
||||||
@ -229,6 +280,8 @@ impl<T: BeaconChainTypes> RangeSync<T> {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/// A peer has disconnected. This removes the peer from any ongoing chains and mappings. A
|
||||||
|
/// disconnected peer could remove a chain
|
||||||
pub fn peer_disconnect(&mut self, network: &mut SyncNetworkContext, peer_id: &PeerId) {
|
pub fn peer_disconnect(&mut self, network: &mut SyncNetworkContext, peer_id: &PeerId) {
|
||||||
// if the peer is in the awaiting head mapping, remove it
|
// if the peer is in the awaiting head mapping, remove it
|
||||||
self.awaiting_head_peers.remove(&peer_id);
|
self.awaiting_head_peers.remove(&peer_id);
|
||||||
@ -237,8 +290,7 @@ impl<T: BeaconChainTypes> RangeSync<T> {
|
|||||||
self.remove_peer(network, peer_id);
|
self.remove_peer(network, peer_id);
|
||||||
|
|
||||||
// update the state of the collection
|
// update the state of the collection
|
||||||
self.chains
|
self.chains.update_finalized(network, &self.log);
|
||||||
.update_finalized(self.beacon_chain.clone(), network, &self.log);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/// When a peer gets removed, both the head and finalized chains need to be searched to check which pool the peer is in. The chain may also have a batch or batches awaiting
|
/// When a peer gets removed, both the head and finalized chains need to be searched to check which pool the peer is in. The chain may also have a batch or batches awaiting
|
||||||
@ -271,22 +323,17 @@ impl<T: BeaconChainTypes> RangeSync<T> {
|
|||||||
}) {
|
}) {
|
||||||
Some((index, ProcessingResult::RemoveChain)) => {
|
Some((index, ProcessingResult::RemoveChain)) => {
|
||||||
// the chain needed to be removed
|
// the chain needed to be removed
|
||||||
let chain = self.chains.remove_chain(index);
|
debug!(self.log, "Chain being removed due to failed batch");
|
||||||
debug!(self.log, "Chain was removed due batch failing"; "start_slot" => chain.start_slot.as_u64(), "end_slot" => chain.target_head_slot.as_u64());
|
self.chains.remove_chain(network, index, &self.log);
|
||||||
// the chain has been removed, re-status it's peers
|
|
||||||
chain.status_peers(self.beacon_chain.clone(), network);
|
|
||||||
// update the state of the collection
|
|
||||||
self.chains
|
|
||||||
.update_finalized(self.beacon_chain.clone(), network, &self.log);
|
|
||||||
}
|
}
|
||||||
_ => {} // chain didn't need to be removed, ignore
|
_ => {} // chain didn't need to be removed, ignore
|
||||||
}
|
}
|
||||||
|
|
||||||
// remove any chains that no longer have any peers
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// An RPC Error occurred, if it's a pending batch, re-request it if possible, if there have
|
/// An RPC error has occurred.
|
||||||
// been too many attempts, remove the chain
|
///
|
||||||
|
/// Check to see if the request corresponds to a pending batch. If so, re-request it if possible, if there have
|
||||||
|
/// been too many failed attempts for the batch, remove the chain.
|
||||||
pub fn inject_error(
|
pub fn inject_error(
|
||||||
&mut self,
|
&mut self,
|
||||||
network: &mut SyncNetworkContext,
|
network: &mut SyncNetworkContext,
|
||||||
@ -300,13 +347,8 @@ impl<T: BeaconChainTypes> RangeSync<T> {
|
|||||||
}) {
|
}) {
|
||||||
Some((_, ProcessingResult::KeepChain)) => {} // error handled chain persists
|
Some((_, ProcessingResult::KeepChain)) => {} // error handled chain persists
|
||||||
Some((index, ProcessingResult::RemoveChain)) => {
|
Some((index, ProcessingResult::RemoveChain)) => {
|
||||||
let chain = self.chains.remove_chain(index);
|
debug!(self.log, "Chain being removed due to RPC error");
|
||||||
debug!(self.log, "Chain was removed due to error"; "start_slot" => chain.start_slot.as_u64(), "end_slot" => chain.target_head_slot.as_u64());
|
self.chains.remove_chain(network, index, &self.log)
|
||||||
// the chain has failed, re-status it's peers
|
|
||||||
chain.status_peers(self.beacon_chain.clone(), network);
|
|
||||||
// update the state of the collection
|
|
||||||
self.chains
|
|
||||||
.update_finalized(self.beacon_chain.clone(), network, &self.log);
|
|
||||||
}
|
}
|
||||||
None => {} // request wasn't in the finalized chains, check the head chains
|
None => {} // request wasn't in the finalized chains, check the head chains
|
||||||
}
|
}
|
||||||
|
Loading…
Reference in New Issue
Block a user