Sync fixes (#1716)

## Issue Addressed

chain state inconsistencies

## Proposed Changes
- a batch can be fake-failed by Range if it needs to move a peer to another chain. The peer will still send blocks/ errors / produce timeouts for those  requests, so check when we get a response from the RPC that the request id matches, instead of only the peer, since a re-request can be directed to the same peer.
- if an optimistic batch succeeds, store the attempt to avoid trying it again when quickly switching chains. Also, use it only if ahead of our current target, instead of the segment's start epoch
This commit is contained in:
divma 2020-10-04 23:49:14 +00:00 committed by Paul Hauner
parent e7eb99cb5e
commit 6997776494
No known key found for this signature in database
GPG Key ID: 5E2CFF9B75FA63DF
5 changed files with 81 additions and 70 deletions

View File

@ -366,7 +366,7 @@ impl<T: BeaconChainTypes> SyncManager<T> {
// check if the parent of this block isn't in our failed cache. If it is, this // check if the parent of this block isn't in our failed cache. If it is, this
// chain should be dropped and the peer downscored. // chain should be dropped and the peer downscored.
if self.failed_chains.contains(&block.message.parent_root) { if self.failed_chains.contains(&block.message.parent_root) {
debug!(self.log, "Parent chain ignored due to past failure"; "block" => format!("{:?}", block.message.parent_root), "slot" => block.message.slot); debug!(self.log, "Parent chain ignored due to past failure"; "block" => ?block.message.parent_root, "slot" => block.message.slot);
if !parent_request.downloaded_blocks.is_empty() { if !parent_request.downloaded_blocks.is_empty() {
// Add the root block to failed chains // Add the root block to failed chains
self.failed_chains self.failed_chains
@ -392,7 +392,7 @@ impl<T: BeaconChainTypes> SyncManager<T> {
// This can be allowed as some clients may implement pruning. We mildly // This can be allowed as some clients may implement pruning. We mildly
// tolerate this behaviour. // tolerate this behaviour.
if !single_block_request.block_returned { if !single_block_request.block_returned {
warn!(self.log, "Peer didn't respond with a block it referenced"; "referenced_block_hash" => format!("{}", single_block_request.hash), "peer_id" => format!("{}", peer_id)); warn!(self.log, "Peer didn't respond with a block it referenced"; "referenced_block_hash" => %single_block_request.hash, "peer_id" => %peer_id);
self.network self.network
.report_peer(peer_id, PeerAction::MidToleranceError); .report_peer(peer_id, PeerAction::MidToleranceError);
} }
@ -433,7 +433,7 @@ impl<T: BeaconChainTypes> SyncManager<T> {
error!( error!(
self.log, self.log,
"Failed to send sync block to processor"; "Failed to send sync block to processor";
"error" => format!("{:?}", e) "error" => ?e
); );
return None; return None;
} }
@ -465,7 +465,7 @@ impl<T: BeaconChainTypes> SyncManager<T> {
if expected_block_hash != block.canonical_root() { if expected_block_hash != block.canonical_root() {
// The peer that sent this, sent us the wrong block. // The peer that sent this, sent us the wrong block.
// We do not tolerate this behaviour. The peer is instantly disconnected and banned. // We do not tolerate this behaviour. The peer is instantly disconnected and banned.
warn!(self.log, "Peer sent incorrect block for single block lookup"; "peer_id" => format!("{}", peer_id)); warn!(self.log, "Peer sent incorrect block for single block lookup"; "peer_id" => %peer_id);
self.network.goodbye_peer(peer_id, GoodbyeReason::Fault); self.network.goodbye_peer(peer_id, GoodbyeReason::Fault);
return; return;
} }
@ -478,7 +478,7 @@ impl<T: BeaconChainTypes> SyncManager<T> {
// we have the correct block, try and process it // we have the correct block, try and process it
match block_result { match block_result {
Ok(block_root) => { Ok(block_root) => {
info!(self.log, "Processed block"; "block" => format!("{}", block_root)); info!(self.log, "Processed block"; "block" => %block_root);
match self.chain.fork_choice() { match self.chain.fork_choice() {
Ok(()) => trace!( Ok(()) => trace!(
@ -489,7 +489,7 @@ impl<T: BeaconChainTypes> SyncManager<T> {
Err(e) => error!( Err(e) => error!(
self.log, self.log,
"Fork choice failed"; "Fork choice failed";
"error" => format!("{:?}", e), "error" => ?e,
"location" => "single block" "location" => "single block"
), ),
} }
@ -502,10 +502,10 @@ impl<T: BeaconChainTypes> SyncManager<T> {
trace!(self.log, "Single block lookup already known"); trace!(self.log, "Single block lookup already known");
} }
Err(BlockError::BeaconChainError(e)) => { Err(BlockError::BeaconChainError(e)) => {
warn!(self.log, "Unexpected block processing error"; "error" => format!("{:?}", e)); warn!(self.log, "Unexpected block processing error"; "error" => ?e);
} }
outcome => { outcome => {
warn!(self.log, "Single block lookup failed"; "outcome" => format!("{:?}", outcome)); warn!(self.log, "Single block lookup failed"; "outcome" => ?outcome);
// This could be a range of errors. But we couldn't process the block. // This could be a range of errors. But we couldn't process the block.
// For now we consider this a mid tolerance error. // For now we consider this a mid tolerance error.
self.network self.network
@ -542,7 +542,7 @@ impl<T: BeaconChainTypes> SyncManager<T> {
if self.failed_chains.contains(&block.message.parent_root) if self.failed_chains.contains(&block.message.parent_root)
|| self.failed_chains.contains(&block_root) || self.failed_chains.contains(&block_root)
{ {
debug!(self.log, "Block is from a past failed chain. Dropping"; "block_root" => format!("{:?}", block_root), "block_slot" => block.message.slot); debug!(self.log, "Block is from a past failed chain. Dropping"; "block_root" => ?block_root, "block_slot" => block.message.slot);
return; return;
} }
@ -559,7 +559,7 @@ impl<T: BeaconChainTypes> SyncManager<T> {
} }
} }
debug!(self.log, "Unknown block received. Starting a parent lookup"; "block_slot" => block.message.slot, "block_hash" => format!("{}", block.canonical_root())); debug!(self.log, "Unknown block received. Starting a parent lookup"; "block_slot" => block.message.slot, "block_hash" => %block.canonical_root());
let parent_request = ParentRequests { let parent_request = ParentRequests {
downloaded_blocks: vec![block], downloaded_blocks: vec![block],
@ -636,10 +636,10 @@ impl<T: BeaconChainTypes> SyncManager<T> {
let head_slot = sync_info.head_slot; let head_slot = sync_info.head_slot;
let finalized_epoch = sync_info.finalized_epoch; let finalized_epoch = sync_info.finalized_epoch;
if peer_info.sync_status.update_synced(sync_info.into()) { if peer_info.sync_status.update_synced(sync_info.into()) {
debug!(self.log, "Peer transitioned sync state"; "new_state" => "synced", "peer_id" => format!("{}", peer_id), "head_slot" => head_slot, "finalized_epoch" => finalized_epoch); debug!(self.log, "Peer transitioned sync state"; "new_state" => "synced", "peer_id" => %peer_id, "head_slot" => head_slot, "finalized_epoch" => finalized_epoch);
} }
} else { } else {
crit!(self.log, "Status'd peer is unknown"; "peer_id" => format!("{}", peer_id)); crit!(self.log, "Status'd peer is unknown"; "peer_id" => %peer_id);
} }
self.update_sync_state(); self.update_sync_state();
} }
@ -650,10 +650,10 @@ impl<T: BeaconChainTypes> SyncManager<T> {
let head_slot = sync_info.head_slot; let head_slot = sync_info.head_slot;
let finalized_epoch = sync_info.finalized_epoch; let finalized_epoch = sync_info.finalized_epoch;
if peer_info.sync_status.update_advanced(sync_info.into()) { if peer_info.sync_status.update_advanced(sync_info.into()) {
debug!(self.log, "Peer transitioned sync state"; "new_state" => "advanced", "peer_id" => format!("{}", peer_id), "head_slot" => head_slot, "finalized_epoch" => finalized_epoch); debug!(self.log, "Peer transitioned sync state"; "new_state" => "advanced", "peer_id" => %peer_id, "head_slot" => head_slot, "finalized_epoch" => finalized_epoch);
} }
} else { } else {
crit!(self.log, "Status'd peer is unknown"; "peer_id" => format!("{}", peer_id)); crit!(self.log, "Status'd peer is unknown"; "peer_id" => %peer_id);
} }
self.update_sync_state(); self.update_sync_state();
} }
@ -664,10 +664,10 @@ impl<T: BeaconChainTypes> SyncManager<T> {
let head_slot = sync_info.head_slot; let head_slot = sync_info.head_slot;
let finalized_epoch = sync_info.finalized_epoch; let finalized_epoch = sync_info.finalized_epoch;
if peer_info.sync_status.update_behind(sync_info.into()) { if peer_info.sync_status.update_behind(sync_info.into()) {
debug!(self.log, "Peer transitioned sync state"; "new_state" => "behind", "peer_id" => format!("{}", peer_id), "head_slot" => head_slot, "finalized_epoch" => finalized_epoch); debug!(self.log, "Peer transitioned sync state"; "new_state" => "behind", "peer_id" => %peer_id, "head_slot" => head_slot, "finalized_epoch" => finalized_epoch);
} }
} else { } else {
crit!(self.log, "Status'd peer is unknown"; "peer_id" => format!("{}", peer_id)); crit!(self.log, "Status'd peer is unknown"; "peer_id" => %peer_id);
} }
self.update_sync_state(); self.update_sync_state();
} }
@ -675,7 +675,7 @@ impl<T: BeaconChainTypes> SyncManager<T> {
/// Updates the global sync state and logs any changes. /// Updates the global sync state and logs any changes.
fn update_sync_state(&mut self) { fn update_sync_state(&mut self) {
if let Some((old_state, new_state)) = self.network_globals.update_sync_state() { if let Some((old_state, new_state)) = self.network_globals.update_sync_state() {
info!(self.log, "Sync state updated"; "old_state" => format!("{}", old_state), "new_state" => format!("{}",new_state)); info!(self.log, "Sync state updated"; "old_state" => %old_state, "new_state" => %new_state);
// If we have become synced - Subscribe to all the core subnet topics // If we have become synced - Subscribe to all the core subnet topics
if new_state == eth2_libp2p::types::SyncState::Synced { if new_state == eth2_libp2p::types::SyncState::Synced {
self.network.subscribe_core_topics(); self.network.subscribe_core_topics();
@ -715,9 +715,9 @@ impl<T: BeaconChainTypes> SyncManager<T> {
let peer = parent_request.last_submitted_peer.clone(); let peer = parent_request.last_submitted_peer.clone();
warn!(self.log, "Peer sent invalid parent."; warn!(self.log, "Peer sent invalid parent.";
"peer_id" => format!("{:?}",peer), "peer_id" => %peer,
"received_block" => format!("{}", block_hash), "received_block" => %block_hash,
"expected_parent" => format!("{}", expected_hash), "expected_parent" => %expected_hash,
); );
// We try again, but downvote the peer. // We try again, but downvote the peer.
@ -772,7 +772,7 @@ impl<T: BeaconChainTypes> SyncManager<T> {
error!( error!(
self.log, self.log,
"Failed to send chain segment to processor"; "Failed to send chain segment to processor";
"error" => format!("{:?}", e) "error" => ?e
); );
} }
} }
@ -782,9 +782,9 @@ impl<T: BeaconChainTypes> SyncManager<T> {
// us the last block // us the last block
warn!( warn!(
self.log, "Invalid parent chain"; self.log, "Invalid parent chain";
"score_adjustment" => PeerAction::MidToleranceError.to_string(), "score_adjustment" => %PeerAction::MidToleranceError,
"outcome" => format!("{:?}", outcome), "outcome" => ?outcome,
"last_peer" => parent_request.last_submitted_peer.to_string(), "last_peer" => %parent_request.last_submitted_peer,
); );
// Add this chain to cache of failed chains // Add this chain to cache of failed chains
@ -827,7 +827,7 @@ impl<T: BeaconChainTypes> SyncManager<T> {
}; };
debug!(self.log, "Parent import failed"; debug!(self.log, "Parent import failed";
"block" => format!("{:?}",parent_request.downloaded_blocks[0].canonical_root()), "block" => ?parent_request.downloaded_blocks[0].canonical_root(),
"ancestors_found" => parent_request.downloaded_blocks.len(), "ancestors_found" => parent_request.downloaded_blocks.len(),
"reason" => error "reason" => error
); );

View File

@ -87,7 +87,7 @@ impl<T: EthSpec> SyncNetworkContext<T> {
request: BlocksByRangeRequest, request: BlocksByRangeRequest,
chain_id: ChainId, chain_id: ChainId,
batch_id: BatchId, batch_id: BatchId,
) -> Result<(), &'static str> { ) -> Result<SyncRequestId, &'static str> {
trace!( trace!(
self.log, self.log,
"Sending BlocksByRange Request"; "Sending BlocksByRange Request";
@ -97,7 +97,7 @@ impl<T: EthSpec> SyncNetworkContext<T> {
); );
let req_id = self.send_rpc_request(peer_id, Request::BlocksByRange(request))?; let req_id = self.send_rpc_request(peer_id, Request::BlocksByRange(request))?;
self.range_requests.insert(req_id, (chain_id, batch_id)); self.range_requests.insert(req_id, (chain_id, batch_id));
Ok(()) Ok(req_id)
} }
pub fn blocks_by_range_response( pub fn blocks_by_range_response(

View File

@ -1,3 +1,4 @@
use crate::sync::RequestId;
use eth2_libp2p::rpc::methods::BlocksByRangeRequest; use eth2_libp2p::rpc::methods::BlocksByRangeRequest;
use eth2_libp2p::PeerId; use eth2_libp2p::PeerId;
use ssz::Encode; use ssz::Encode;
@ -32,7 +33,7 @@ pub enum BatchState<T: EthSpec> {
/// The batch has failed either downloading or processing, but can be requested again. /// The batch has failed either downloading or processing, but can be requested again.
AwaitingDownload, AwaitingDownload,
/// The batch is being downloaded. /// The batch is being downloaded.
Downloading(PeerId, Vec<SignedBeaconBlock<T>>), Downloading(PeerId, Vec<SignedBeaconBlock<T>>, RequestId),
/// The batch has been completely downloaded and is ready for processing. /// The batch has been completely downloaded and is ready for processing.
AwaitingProcessing(PeerId, Vec<SignedBeaconBlock<T>>), AwaitingProcessing(PeerId, Vec<SignedBeaconBlock<T>>),
/// The batch is being processed. /// The batch is being processed.
@ -99,7 +100,7 @@ impl<T: EthSpec> BatchInfo<T> {
pub fn current_peer(&self) -> Option<&PeerId> { pub fn current_peer(&self) -> Option<&PeerId> {
match &self.state { match &self.state {
BatchState::AwaitingDownload | BatchState::Failed => None, BatchState::AwaitingDownload | BatchState::Failed => None,
BatchState::Downloading(peer_id, _) BatchState::Downloading(peer_id, _, _)
| BatchState::AwaitingProcessing(peer_id, _) | BatchState::AwaitingProcessing(peer_id, _)
| BatchState::Processing(Attempt { peer_id, .. }) | BatchState::Processing(Attempt { peer_id, .. })
| BatchState::AwaitingValidation(Attempt { peer_id, .. }) => Some(&peer_id), | BatchState::AwaitingValidation(Attempt { peer_id, .. }) => Some(&peer_id),
@ -126,9 +127,9 @@ impl<T: EthSpec> BatchInfo<T> {
/// Adds a block to a downloading batch. /// Adds a block to a downloading batch.
pub fn add_block(&mut self, block: SignedBeaconBlock<T>) { pub fn add_block(&mut self, block: SignedBeaconBlock<T>) {
match self.state.poison() { match self.state.poison() {
BatchState::Downloading(peer, mut blocks) => { BatchState::Downloading(peer, mut blocks, req_id) => {
blocks.push(block); blocks.push(block);
self.state = BatchState::Downloading(peer, blocks) self.state = BatchState::Downloading(peer, blocks, req_id)
} }
other => unreachable!("Add block for batch in wrong state: {:?}", other), other => unreachable!("Add block for batch in wrong state: {:?}", other),
} }
@ -148,7 +149,7 @@ impl<T: EthSpec> BatchInfo<T> {
), ),
> { > {
match self.state.poison() { match self.state.poison() {
BatchState::Downloading(peer, blocks) => { BatchState::Downloading(peer, blocks, _request_id) => {
// verify that blocks are in range // verify that blocks are in range
if let Some(last_slot) = blocks.last().map(|b| b.slot()) { if let Some(last_slot) = blocks.last().map(|b| b.slot()) {
// the batch is non-empty // the batch is non-empty
@ -189,7 +190,7 @@ impl<T: EthSpec> BatchInfo<T> {
#[must_use = "Batch may have failed"] #[must_use = "Batch may have failed"]
pub fn download_failed(&mut self) -> &BatchState<T> { pub fn download_failed(&mut self) -> &BatchState<T> {
match self.state.poison() { match self.state.poison() {
BatchState::Downloading(peer, _) => { BatchState::Downloading(peer, _, _request_id) => {
// register the attempt and check if the batch can be tried again // register the attempt and check if the batch can be tried again
self.failed_download_attempts.push(peer); self.failed_download_attempts.push(peer);
self.state = if self.failed_download_attempts.len() self.state = if self.failed_download_attempts.len()
@ -206,10 +207,10 @@ impl<T: EthSpec> BatchInfo<T> {
} }
} }
pub fn start_downloading_from_peer(&mut self, peer: PeerId) { pub fn start_downloading_from_peer(&mut self, peer: PeerId, request_id: RequestId) {
match self.state.poison() { match self.state.poison() {
BatchState::AwaitingDownload => { BatchState::AwaitingDownload => {
self.state = BatchState::Downloading(peer, Vec::new()); self.state = BatchState::Downloading(peer, Vec::new(), request_id);
} }
other => unreachable!("Starting download for batch in wrong state: {:?}", other), other => unreachable!("Starting download for batch in wrong state: {:?}", other),
} }
@ -333,9 +334,13 @@ impl<T: EthSpec> std::fmt::Debug for BatchState<T> {
BatchState::AwaitingProcessing(ref peer, ref blocks) => { BatchState::AwaitingProcessing(ref peer, ref blocks) => {
write!(f, "AwaitingProcessing({}, {} blocks)", peer, blocks.len()) write!(f, "AwaitingProcessing({}, {} blocks)", peer, blocks.len())
} }
BatchState::Downloading(peer, blocks) => { BatchState::Downloading(peer, blocks, request_id) => write!(
write!(f, "Downloading({}, {} blocks)", peer, blocks.len()) f,
} "Downloading({}, {} blocks, {})",
peer,
blocks.len(),
request_id
),
BatchState::Poisoned => f.write_str("Poisoned"), BatchState::Poisoned => f.write_str("Poisoned"),
} }
} }

View File

@ -1,7 +1,7 @@
use super::batch::{BatchInfo, BatchState}; use super::batch::{BatchInfo, BatchState};
use crate::beacon_processor::ProcessId; use crate::beacon_processor::ProcessId;
use crate::beacon_processor::WorkEvent as BeaconWorkEvent; use crate::beacon_processor::WorkEvent as BeaconWorkEvent;
use crate::sync::{network_context::SyncNetworkContext, BatchProcessResult}; use crate::sync::{network_context::SyncNetworkContext, BatchProcessResult, RequestId};
use beacon_chain::{BeaconChain, BeaconChainTypes}; use beacon_chain::{BeaconChain, BeaconChainTypes};
use eth2_libp2p::{PeerAction, PeerId}; use eth2_libp2p::{PeerAction, PeerId};
use fnv::FnvHashMap; use fnv::FnvHashMap;
@ -75,9 +75,9 @@ pub struct SyncingChain<T: BeaconChainTypes> {
/// If a block is imported for this batch, the chain advances to this point. /// If a block is imported for this batch, the chain advances to this point.
optimistic_start: Option<BatchId>, optimistic_start: Option<BatchId>,
/// When a batch for an optimistic start fails processing, it is stored to avoid trying it /// When a batch for an optimistic start is tried (either successful or not), it is stored to
/// again due to chain stopping/re-starting on chain switching. /// avoid trying it again due to chain stopping/re-starting on chain switching.
failed_optimistic_starts: HashSet<BatchId>, attempted_optimistic_starts: HashSet<BatchId>,
/// The current state of the chain. /// The current state of the chain.
pub state: ChainSyncingState, pub state: ChainSyncingState,
@ -135,7 +135,7 @@ impl<T: BeaconChainTypes> SyncingChain<T> {
to_be_downloaded: start_epoch, to_be_downloaded: start_epoch,
processing_target: start_epoch, processing_target: start_epoch,
optimistic_start: None, optimistic_start: None,
failed_optimistic_starts: HashSet::default(), attempted_optimistic_starts: HashSet::default(),
state: ChainSyncingState::Stopped, state: ChainSyncingState::Stopped,
current_processing_batch: None, current_processing_batch: None,
beacon_processor_send, beacon_processor_send,
@ -200,7 +200,8 @@ impl<T: BeaconChainTypes> SyncingChain<T> {
&mut self, &mut self,
network: &mut SyncNetworkContext<T::EthSpec>, network: &mut SyncNetworkContext<T::EthSpec>,
batch_id: BatchId, batch_id: BatchId,
peer_id: PeerId, peer_id: &PeerId,
request_id: RequestId,
beacon_block: Option<SignedBeaconBlock<T::EthSpec>>, beacon_block: Option<SignedBeaconBlock<T::EthSpec>>,
) -> ProcessingResult { ) -> ProcessingResult {
// check if we have this batch // check if we have this batch
@ -213,9 +214,14 @@ impl<T: BeaconChainTypes> SyncingChain<T> {
Some(batch) => { Some(batch) => {
// A batch could be retried without the peer failing the request (disconnecting/ // A batch could be retried without the peer failing the request (disconnecting/
// sending an error /timeout) if the peer is removed from the chain for other // sending an error /timeout) if the peer is removed from the chain for other
// reasons. Check that this block belongs to the expected peer // reasons. Check that this block belongs to the expected peer, and that the
if Some(&peer_id) != batch.current_peer() { // request_id matches
return ProcessingResult::KeepChain; if let BatchState::Downloading(expected_peer, _, expected_request_id) =
batch.state()
{
if expected_peer != peer_id || expected_request_id != &request_id {
return ProcessingResult::KeepChain;
}
} }
batch batch
} }
@ -228,11 +234,9 @@ impl<T: BeaconChainTypes> SyncingChain<T> {
} else { } else {
// A stream termination has been sent. This batch has ended. Process a completed batch. // A stream termination has been sent. This batch has ended. Process a completed batch.
// Remove the request from the peer's active batches // Remove the request from the peer's active batches
let peer = batch
.current_peer()
.expect("Batch is downloading from a peer");
self.peers self.peers
.get_mut(peer) .get_mut(peer_id)
.unwrap_or_else(|| panic!("Batch is registered for the peer")) .unwrap_or_else(|| panic!("Batch is registered for the peer"))
.remove(&batch_id); .remove(&batch_id);
@ -442,6 +446,8 @@ impl<T: BeaconChainTypes> SyncingChain<T> {
// blocks. // blocks.
if *was_non_empty { if *was_non_empty {
self.advance_chain(network, batch_id); self.advance_chain(network, batch_id);
// we register so that on chain switching we don't try it again
self.attempted_optimistic_starts.insert(batch_id);
} else if let Some(epoch) = self.optimistic_start { } else if let Some(epoch) = self.optimistic_start {
// check if this batch corresponds to an optimistic batch. In this case, we // check if this batch corresponds to an optimistic batch. In this case, we
// reject it as an optimistic candidate since the batch was empty // reject it as an optimistic candidate since the batch was empty
@ -520,9 +526,8 @@ impl<T: BeaconChainTypes> SyncingChain<T> {
redownload: bool, redownload: bool,
reason: &str, reason: &str,
) -> ProcessingResult { ) -> ProcessingResult {
if let Some(epoch) = self.optimistic_start { if let Some(epoch) = self.optimistic_start.take() {
self.optimistic_start = None; self.attempted_optimistic_starts.insert(epoch);
self.failed_optimistic_starts.insert(epoch);
// if this batch is inside the current processing range, keep it, otherwise drop // if this batch is inside the current processing range, keep it, otherwise drop
// it. NOTE: this is done to prevent non-sequential batches coming from optimistic // it. NOTE: this is done to prevent non-sequential batches coming from optimistic
// starts from filling up the buffer size // starts from filling up the buffer size
@ -628,7 +633,7 @@ impl<T: BeaconChainTypes> SyncingChain<T> {
self.start_epoch = validating_epoch; self.start_epoch = validating_epoch;
self.to_be_downloaded = self.to_be_downloaded.max(validating_epoch); self.to_be_downloaded = self.to_be_downloaded.max(validating_epoch);
if self.batches.contains_key(&self.to_be_downloaded) { if self.batches.contains_key(&self.to_be_downloaded) {
// if a chain is advanced by Range beyond the previous `seld.to_be_downloaded`, we // if a chain is advanced by Range beyond the previous `self.to_be_downloaded`, we
// won't have this batch, so we need to request it. // won't have this batch, so we need to request it.
self.to_be_downloaded += EPOCHS_PER_BATCH; self.to_be_downloaded += EPOCHS_PER_BATCH;
} }
@ -732,8 +737,8 @@ impl<T: BeaconChainTypes> SyncingChain<T> {
// advance the chain to the new validating epoch // advance the chain to the new validating epoch
self.advance_chain(network, validating_epoch); self.advance_chain(network, validating_epoch);
if self.optimistic_start.is_none() if self.optimistic_start.is_none()
&& optimistic_epoch > self.start_epoch && optimistic_epoch > self.processing_target
&& !self.failed_optimistic_starts.contains(&optimistic_epoch) && !self.attempted_optimistic_starts.contains(&optimistic_epoch)
{ {
self.optimistic_start = Some(optimistic_epoch); self.optimistic_start = Some(optimistic_epoch);
} }
@ -782,21 +787,21 @@ impl<T: BeaconChainTypes> SyncingChain<T> {
&mut self, &mut self,
network: &mut SyncNetworkContext<T::EthSpec>, network: &mut SyncNetworkContext<T::EthSpec>,
batch_id: BatchId, batch_id: BatchId,
peer_id: PeerId, peer_id: &PeerId,
request_id: RequestId,
) -> ProcessingResult { ) -> ProcessingResult {
if let Some(batch) = self.batches.get_mut(&batch_id) { if let Some(batch) = self.batches.get_mut(&batch_id) {
// A batch could be retried without the peer failing the request (disconnecting/ // A batch could be retried without the peer failing the request (disconnecting/
// sending an error /timeout) if the peer is removed from the chain for other // sending an error /timeout) if the peer is removed from the chain for other
// reasons. Check that this block belongs to the expected peer // reasons. Check that this block belongs to the expected peer
if Some(&peer_id) != batch.current_peer() { if let BatchState::Downloading(expected_peer, _, expected_request_id) = batch.state() {
return ProcessingResult::KeepChain; if expected_peer != peer_id || expected_request_id != &request_id {
return ProcessingResult::KeepChain;
}
} }
debug!(self.log, "Batch failed. RPC Error"; "batch_epoch" => batch_id); debug!(self.log, "Batch failed. RPC Error"; "batch_epoch" => batch_id);
let failed_peer = batch
.current_peer()
.expect("Batch is downloading from a peer");
self.peers self.peers
.get_mut(failed_peer) .get_mut(peer_id)
.expect("Peer belongs to the chain") .expect("Peer belongs to the chain")
.remove(&batch_id); .remove(&batch_id);
if let BatchState::Failed = batch.download_failed() { if let BatchState::Failed = batch.download_failed() {
@ -851,10 +856,10 @@ impl<T: BeaconChainTypes> SyncingChain<T> {
) -> ProcessingResult { ) -> ProcessingResult {
if let Some(batch) = self.batches.get_mut(&batch_id) { if let Some(batch) = self.batches.get_mut(&batch_id) {
let request = batch.to_blocks_by_range_request(); let request = batch.to_blocks_by_range_request();
// inform the batch about the new request
batch.start_downloading_from_peer(peer.clone());
match network.blocks_by_range_request(peer.clone(), request, self.id, batch_id) { match network.blocks_by_range_request(peer.clone(), request, self.id, batch_id) {
Ok(()) => { Ok(request_id) => {
// inform the batch about the new request
batch.start_downloading_from_peer(peer.clone(), request_id);
if self if self
.optimistic_start .optimistic_start
.map(|epoch| epoch == batch_id) .map(|epoch| epoch == batch_id)
@ -876,6 +881,7 @@ impl<T: BeaconChainTypes> SyncingChain<T> {
warn!(self.log, "Could not send batch request"; warn!(self.log, "Could not send batch request";
"batch_id" => batch_id, "error" => e, &batch); "batch_id" => batch_id, "error" => e, &batch);
// register the failed download and check if the batch can be retried // register the failed download and check if the batch can be retried
batch.start_downloading_from_peer(peer.clone(), 1); // fake request_id is not relevant
self.peers self.peers
.get_mut(&peer) .get_mut(&peer)
.expect("peer belongs to the peer pool") .expect("peer belongs to the peer pool")

View File

@ -214,7 +214,7 @@ impl<T: BeaconChainTypes> RangeSync<T> {
{ {
// check if this chunk removes the chain // check if this chunk removes the chain
match self.chains.call_by_id(chain_id, |chain| { match self.chains.call_by_id(chain_id, |chain| {
chain.on_block_response(network, batch_id, peer_id, beacon_block) chain.on_block_response(network, batch_id, &peer_id, request_id, beacon_block)
}) { }) {
Ok((removed_chain, sync_type)) => { Ok((removed_chain, sync_type)) => {
if let Some(removed_chain) = removed_chain { if let Some(removed_chain) = removed_chain {
@ -228,7 +228,7 @@ impl<T: BeaconChainTypes> RangeSync<T> {
} }
} }
} else { } else {
warn!(self.log, "Response/Error for non registered request"; "request_id" => request_id) debug!(self.log, "Response/Error for non registered request"; "request_id" => request_id)
} }
} }
@ -337,7 +337,7 @@ impl<T: BeaconChainTypes> RangeSync<T> {
if let Some((chain_id, batch_id)) = network.blocks_by_range_response(request_id, true) { if let Some((chain_id, batch_id)) = network.blocks_by_range_response(request_id, true) {
// check that this request is pending // check that this request is pending
match self.chains.call_by_id(chain_id, |chain| { match self.chains.call_by_id(chain_id, |chain| {
chain.inject_error(network, batch_id, peer_id) chain.inject_error(network, batch_id, &peer_id, request_id)
}) { }) {
Ok((removed_chain, sync_type)) => { Ok((removed_chain, sync_type)) => {
if let Some(removed_chain) = removed_chain { if let Some(removed_chain) = removed_chain {