From a74b9421fa56a39673faed985a366348fc234007 Mon Sep 17 00:00:00 2001 From: Fabio Di Fabio Date: Thu, 10 Nov 2022 16:00:36 +0100 Subject: [PATCH 01/32] Log UX, retry switching peer and dead code removal Signed-off-by: Fabio Di Fabio --- .../merge/blockcreation/MergeCoordinator.java | 36 ++--- .../blockcreation/MergeMiningCoordinator.java | 4 +- .../blockcreation/TransitionCoordinator.java | 10 +- .../blockcreation/MergeCoordinatorTest.java | 4 +- .../engine/EngineForkchoiceUpdated.java | 18 +-- .../engine/EngineForkchoiceUpdatedTest.java | 30 ++-- .../manager/task/GetBodiesFromPeerTask.java | 4 +- .../task/RetryingGetBlockFromPeersTask.java | 7 +- .../task/RetryingGetBlocksFromPeersTask.java | 112 +++++++++++++++ ...gGetHeadersEndingAtFromPeerByHashTask.java | 43 ++++-- ...GetHeadersEndingAtFromPeerByHashTask2.java | 78 +++++++++++ .../backwardsync/BackwardSyncContext.java | 131 +++++++----------- .../sync/backwardsync/BackwardSyncStep.java | 37 +++-- .../backwardsync/BackwardsSyncAlgorithm.java | 98 +++++-------- .../sync/backwardsync/ForwardSyncStep.java | 51 +++++-- .../eth/sync/backwardsync/SyncStepStep.java | 13 +- .../backwardsync/BackwardSyncAlgSpec.java | 90 +----------- .../backwardsync/BackwardSyncContextTest.java | 17 --- .../backwardsync/BackwardSyncStepTest.java | 8 +- 19 files changed, 428 insertions(+), 363 deletions(-) create mode 100644 ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/manager/task/RetryingGetBlocksFromPeersTask.java create mode 100644 ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/manager/task/RetryingGetHeadersEndingAtFromPeerByHashTask2.java diff --git a/consensus/merge/src/main/java/org/hyperledger/besu/consensus/merge/blockcreation/MergeCoordinator.java b/consensus/merge/src/main/java/org/hyperledger/besu/consensus/merge/blockcreation/MergeCoordinator.java index 4ca248546fa..de8c702ca3e 100644 --- a/consensus/merge/src/main/java/org/hyperledger/besu/consensus/merge/blockcreation/MergeCoordinator.java +++ b/consensus/merge/src/main/java/org/hyperledger/besu/consensus/merge/blockcreation/MergeCoordinator.java @@ -356,37 +356,19 @@ private boolean canRetryBlockCreation(final Throwable throwable) { } @Override - public Optional getOrSyncHeaderByHash(final Hash blockHash) { + public Optional getOrSyncHeadByHash(final Hash headHash) { final var chain = protocolContext.getBlockchain(); - final var optHeader = chain.getBlockHeader(blockHash); + final var maybeHeadHeader = chain.getBlockHeader(headHash); - if (optHeader.isPresent()) { - debugLambda(LOG, "BlockHeader {} is already present", () -> optHeader.get().toLogString()); + if (maybeHeadHeader.isPresent()) { + debugLambda(LOG, "BlockHeader {} is already present", maybeHeadHeader.get()::toLogString); } else { - debugLambda(LOG, "appending block hash {} to backward sync", blockHash::toHexString); + debugLambda(LOG, "appending block hash {} to backward sync", headHash::toHexString); backwardSyncContext - .syncBackwardsUntil(blockHash) - .exceptionally(e -> logSyncException(blockHash, e)); + .syncBackwardsUntil(headHash) + .exceptionally(e -> logSyncException(headHash, e)); } - return optHeader; - } - - @Override - public Optional getOrSyncHeaderByHash( - final Hash blockHash, final Hash finalizedBlockHash) { - final var chain = protocolContext.getBlockchain(); - final var optHeader = chain.getBlockHeader(blockHash); - - if (optHeader.isPresent()) { - debugLambda(LOG, "BlockHeader {} is already present", () -> optHeader.get().toLogString()); - } else { - debugLambda(LOG, "appending block hash {} to backward sync", blockHash::toHexString); - backwardSyncContext.updateHeads(blockHash, finalizedBlockHash); - backwardSyncContext - .syncBackwardsUntil(blockHash) - .exceptionally(e -> logSyncException(blockHash, e)); - } - return optHeader; + return maybeHeadHeader; } private Void logSyncException(final Hash blockHash, final Throwable exception) { @@ -439,7 +421,7 @@ public ForkchoiceResult updateForkChoice( if (newHead.getNumber() < blockchain.getChainHeadBlockNumber() && isDescendantOf(newHead, blockchain.getChainHeadHeader())) { - LOG.info("Ignoring update to old head"); + debugLambda(LOG, "Ignoring update to old head {}", newHead::toLogString); return ForkchoiceResult.withIgnoreUpdateToOldHead(newHead); } diff --git a/consensus/merge/src/main/java/org/hyperledger/besu/consensus/merge/blockcreation/MergeMiningCoordinator.java b/consensus/merge/src/main/java/org/hyperledger/besu/consensus/merge/blockcreation/MergeMiningCoordinator.java index b3f7931a2cc..c1f6467d805 100644 --- a/consensus/merge/src/main/java/org/hyperledger/besu/consensus/merge/blockcreation/MergeMiningCoordinator.java +++ b/consensus/merge/src/main/java/org/hyperledger/besu/consensus/merge/blockcreation/MergeMiningCoordinator.java @@ -63,12 +63,10 @@ ForkchoiceResult updateForkChoice( CompletableFuture appendNewPayloadToSync(Block newPayload); - Optional getOrSyncHeaderByHash(Hash blockHash); + Optional getOrSyncHeadByHash(Hash blockHash); boolean isMiningBeforeMerge(); - Optional getOrSyncHeaderByHash(Hash blockHash, Hash finalizedBlockHash); - void addBadBlock(final Block block); boolean isBadBlock(Hash blockHash); diff --git a/consensus/merge/src/main/java/org/hyperledger/besu/consensus/merge/blockcreation/TransitionCoordinator.java b/consensus/merge/src/main/java/org/hyperledger/besu/consensus/merge/blockcreation/TransitionCoordinator.java index 69d0514a708..bcc81807158 100644 --- a/consensus/merge/src/main/java/org/hyperledger/besu/consensus/merge/blockcreation/TransitionCoordinator.java +++ b/consensus/merge/src/main/java/org/hyperledger/besu/consensus/merge/blockcreation/TransitionCoordinator.java @@ -183,14 +183,8 @@ public CompletableFuture appendNewPayloadToSync(final Block newPayload) { } @Override - public Optional getOrSyncHeaderByHash(final Hash blockHash) { - return mergeCoordinator.getOrSyncHeaderByHash(blockHash); - } - - @Override - public Optional getOrSyncHeaderByHash( - final Hash blockHash, final Hash finalizedBlockHash) { - return mergeCoordinator.getOrSyncHeaderByHash(blockHash, finalizedBlockHash); + public Optional getOrSyncHeadByHash(final Hash blockHash) { + return mergeCoordinator.getOrSyncHeadByHash(blockHash); } @Override diff --git a/consensus/merge/src/test/java/org/hyperledger/besu/consensus/merge/blockcreation/MergeCoordinatorTest.java b/consensus/merge/src/test/java/org/hyperledger/besu/consensus/merge/blockcreation/MergeCoordinatorTest.java index d0d7567291d..45126103524 100644 --- a/consensus/merge/src/test/java/org/hyperledger/besu/consensus/merge/blockcreation/MergeCoordinatorTest.java +++ b/consensus/merge/src/test/java/org/hyperledger/besu/consensus/merge/blockcreation/MergeCoordinatorTest.java @@ -598,7 +598,7 @@ public void assertGetOrSyncForBlockAlreadyPresent() { BlockHeader mockHeader = headerGenerator.parentHash(Hash.fromHexStringLenient("0xdead")).buildHeader(); when(blockchain.getBlockHeader(mockHeader.getHash())).thenReturn(Optional.of(mockHeader)); - var res = coordinator.getOrSyncHeaderByHash(mockHeader.getHash()); + var res = coordinator.getOrSyncHeadByHash(mockHeader.getHash()); assertThat(res).isPresent(); } @@ -610,7 +610,7 @@ public void assertGetOrSyncForBlockNotPresent() { when(backwardSyncContext.syncBackwardsUntil(mockHeader.getBlockHash())) .thenReturn(CompletableFuture.completedFuture(null)); - var res = coordinator.getOrSyncHeaderByHash(mockHeader.getHash()); + var res = coordinator.getOrSyncHeadByHash(mockHeader.getHash()); assertThat(res).isNotPresent(); } diff --git a/ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/methods/engine/EngineForkchoiceUpdated.java b/ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/methods/engine/EngineForkchoiceUpdated.java index bc88647ed40..9789f4e2f12 100644 --- a/ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/methods/engine/EngineForkchoiceUpdated.java +++ b/ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/methods/engine/EngineForkchoiceUpdated.java @@ -99,24 +99,26 @@ public JsonRpcResponse syncResponse(final JsonRpcRequestContext requestContext) Optional.of(forkChoice.getHeadBlockHash() + " is an invalid block"))); } - Optional newHead = - mergeCoordinator.getOrSyncHeaderByHash(forkChoice.getHeadBlockHash()); + final Optional maybeNewHead = + mergeCoordinator.getOrSyncHeadByHash(forkChoice.getHeadBlockHash()); - if (newHead.isEmpty()) { + if (maybeNewHead.isEmpty()) { return syncingResponse(requestId, forkChoice); } + final BlockHeader newHead = maybeNewHead.get(); + maybePayloadAttributes.ifPresentOrElse( this::logPayload, () -> LOG.debug("Payload attributes are null")); if (!isValidForkchoiceState( - forkChoice.getSafeBlockHash(), forkChoice.getFinalizedBlockHash(), newHead.get())) { + forkChoice.getSafeBlockHash(), forkChoice.getFinalizedBlockHash(), newHead)) { logForkchoiceUpdatedCall(INVALID, forkChoice); return new JsonRpcErrorResponse(requestId, JsonRpcError.INVALID_FORKCHOICE_STATE); } // TODO: post-merge cleanup, this should be unnecessary after merge - if (!mergeCoordinator.latestValidAncestorDescendsFromTerminal(newHead.get())) { + if (!mergeCoordinator.latestValidAncestorDescendsFromTerminal(newHead)) { logForkchoiceUpdatedCall(INVALID, forkChoice); return new JsonRpcSuccessResponse( requestId, @@ -124,12 +126,12 @@ public JsonRpcResponse syncResponse(final JsonRpcRequestContext requestContext) INVALID, Hash.ZERO, null, - Optional.of(newHead.get() + " did not descend from terminal block"))); + Optional.of(newHead + " did not descend from terminal block"))); } ForkchoiceResult result = mergeCoordinator.updateForkChoice( - newHead.get(), + newHead, forkChoice.getFinalizedBlockHash(), forkChoice.getSafeBlockHash(), maybePayloadAttributes.map( @@ -149,7 +151,7 @@ public JsonRpcResponse syncResponse(final JsonRpcRequestContext requestContext) maybePayloadAttributes.map( payloadAttributes -> mergeCoordinator.preparePayload( - newHead.get(), + newHead, payloadAttributes.getTimestamp(), payloadAttributes.getPrevRandao(), payloadAttributes.getSuggestedFeeRecipient())); diff --git a/ethereum/api/src/test/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/methods/engine/EngineForkchoiceUpdatedTest.java b/ethereum/api/src/test/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/methods/engine/EngineForkchoiceUpdatedTest.java index 3b53a4f8d68..019e63940ea 100644 --- a/ethereum/api/src/test/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/methods/engine/EngineForkchoiceUpdatedTest.java +++ b/ethereum/api/src/test/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/methods/engine/EngineForkchoiceUpdatedTest.java @@ -111,7 +111,7 @@ public void shouldReturnSyncingIfMissingNewHead() { public void shouldReturnInvalidOnBadTerminalBlock() { BlockHeader mockHeader = new BlockHeaderTestFixture().baseFeePerGas(Wei.ONE).buildHeader(); - when(mergeCoordinator.getOrSyncHeaderByHash(mockHeader.getHash())) + when(mergeCoordinator.getOrSyncHeadByHash(mockHeader.getHash())) .thenReturn(Optional.of(mockHeader)); when(mergeCoordinator.latestValidAncestorDescendsFromTerminal(mockHeader)).thenReturn(false); assertSuccessWithPayloadForForkchoiceResult( @@ -148,7 +148,7 @@ public void shouldReturnSyncingOnHeadNotFound() { @Test public void shouldReturnValidWithoutFinalizedOrPayload() { BlockHeader mockHeader = new BlockHeaderTestFixture().baseFeePerGas(Wei.ONE).buildHeader(); - when(mergeCoordinator.getOrSyncHeaderByHash(mockHeader.getHash())) + when(mergeCoordinator.getOrSyncHeadByHash(mockHeader.getHash())) .thenReturn(Optional.of(mockHeader)); when(mergeCoordinator.latestValidAncestorDescendsFromTerminal(mockHeader)).thenReturn(true); @@ -172,7 +172,7 @@ public void shouldReturnInvalidOnOldTimestamp() { when(mergeCoordinator.latestValidAncestorDescendsFromTerminal(mockHeader)).thenReturn(true); when(mergeCoordinator.isDescendantOf(any(), any())).thenReturn(true); when(mergeContext.isSyncing()).thenReturn(false); - when(mergeCoordinator.getOrSyncHeaderByHash(mockHeader.getHash())) + when(mergeCoordinator.getOrSyncHeadByHash(mockHeader.getHash())) .thenReturn(Optional.of(mockHeader)); when(mergeCoordinator.updateForkChoice( mockHeader, parent.getHash(), parent.getHash(), Optional.empty())) @@ -202,7 +202,7 @@ public void shouldReturnValidWithNewHeadAndFinalizedNoPayload() { BlockHeader mockParent = builder.number(9L).buildHeader(); BlockHeader mockHeader = builder.number(10L).parentHash(mockParent.getHash()).buildHeader(); when(blockchain.getBlockHeader(any())).thenReturn(Optional.of(mockHeader)); - when(mergeCoordinator.getOrSyncHeaderByHash(mockHeader.getHash())) + when(mergeCoordinator.getOrSyncHeadByHash(mockHeader.getHash())) .thenReturn(Optional.of(mockHeader)); when(mergeCoordinator.latestValidAncestorDescendsFromTerminal(mockHeader)).thenReturn(true); when(mergeCoordinator.isDescendantOf(any(), any())).thenReturn(true); @@ -217,7 +217,7 @@ public void shouldReturnValidWithNewHeadAndFinalizedNoPayload() { @Test public void shouldReturnValidWithoutFinalizedWithPayload() { BlockHeader mockHeader = new BlockHeaderTestFixture().baseFeePerGas(Wei.ONE).buildHeader(); - when(mergeCoordinator.getOrSyncHeaderByHash(mockHeader.getHash())) + when(mergeCoordinator.getOrSyncHeadByHash(mockHeader.getHash())) .thenReturn(Optional.of(mockHeader)); when(mergeCoordinator.latestValidAncestorDescendsFromTerminal(mockHeader)).thenReturn(true); @@ -254,8 +254,7 @@ public void shouldReturnInvalidForkchoiceStateIfFinalizedBlockIsUnknown() { when(blockchain.getBlockHeader(finalizedBlockHash)).thenReturn(Optional.empty()); when(mergeContext.isSyncing()).thenReturn(false); - when(mergeCoordinator.getOrSyncHeaderByHash(newHead.getHash())) - .thenReturn(Optional.of(newHead)); + when(mergeCoordinator.getOrSyncHeadByHash(newHead.getHash())).thenReturn(Optional.of(newHead)); var resp = resp( @@ -275,8 +274,7 @@ public void shouldReturnInvalidForkchoiceStateIfFinalizedBlockIsNotAnAncestorOfN when(blockchain.getBlockHeader(newHead.getHash())).thenReturn(Optional.of(newHead)); when(blockchain.getBlockHeader(finalized.getHash())).thenReturn(Optional.of(finalized)); when(mergeContext.isSyncing()).thenReturn(false); - when(mergeCoordinator.getOrSyncHeaderByHash(newHead.getHash())) - .thenReturn(Optional.of(newHead)); + when(mergeCoordinator.getOrSyncHeadByHash(newHead.getHash())).thenReturn(Optional.of(newHead)); when(mergeCoordinator.isDescendantOf(finalized, newHead)).thenReturn(false); var resp = @@ -301,8 +299,7 @@ public void shouldReturnInvalidForkchoiceStateIfSafeHeadZeroWithFinalizedBlock() when(blockchain.getBlockHeader(parent.getHash())).thenReturn(Optional.of(parent)); when(mergeContext.isSyncing()).thenReturn(false); - when(mergeCoordinator.getOrSyncHeaderByHash(newHead.getHash())) - .thenReturn(Optional.of(newHead)); + when(mergeCoordinator.getOrSyncHeadByHash(newHead.getHash())).thenReturn(Optional.of(newHead)); var resp = resp( @@ -328,8 +325,7 @@ public void shouldReturnInvalidForkchoiceStateIfSafeBlockIsUnknown() { when(blockchain.getBlockHeader(finalized.getHash())).thenReturn(Optional.of(finalized)); when(blockchain.getBlockHeader(safeBlockBlockHash)).thenReturn(Optional.empty()); when(mergeContext.isSyncing()).thenReturn(false); - when(mergeCoordinator.getOrSyncHeaderByHash(newHead.getHash())) - .thenReturn(Optional.of(newHead)); + when(mergeCoordinator.getOrSyncHeadByHash(newHead.getHash())).thenReturn(Optional.of(newHead)); when(mergeCoordinator.isDescendantOf(finalized, newHead)).thenReturn(true); var resp = @@ -352,8 +348,7 @@ public void shouldReturnInvalidForkchoiceStateIfSafeBlockIsNotADescendantOfFinal when(blockchain.getBlockHeader(finalized.getHash())).thenReturn(Optional.of(finalized)); when(blockchain.getBlockHeader(safeBlock.getHash())).thenReturn(Optional.of(safeBlock)); when(mergeContext.isSyncing()).thenReturn(false); - when(mergeCoordinator.getOrSyncHeaderByHash(newHead.getHash())) - .thenReturn(Optional.of(newHead)); + when(mergeCoordinator.getOrSyncHeadByHash(newHead.getHash())).thenReturn(Optional.of(newHead)); when(mergeCoordinator.isDescendantOf(finalized, newHead)).thenReturn(true); when(mergeCoordinator.isDescendantOf(finalized, safeBlock)).thenReturn(false); @@ -377,8 +372,7 @@ public void shouldReturnInvalidForkchoiceStateIfSafeBlockIsNotAnAncestorOfNewHea when(blockchain.getBlockHeader(finalized.getHash())).thenReturn(Optional.of(finalized)); when(blockchain.getBlockHeader(safeBlock.getHash())).thenReturn(Optional.of(safeBlock)); when(mergeContext.isSyncing()).thenReturn(false); - when(mergeCoordinator.getOrSyncHeaderByHash(newHead.getHash())) - .thenReturn(Optional.of(newHead)); + when(mergeCoordinator.getOrSyncHeadByHash(newHead.getHash())).thenReturn(Optional.of(newHead)); when(mergeCoordinator.isDescendantOf(finalized, newHead)).thenReturn(true); when(mergeCoordinator.isDescendantOf(finalized, safeBlock)).thenReturn(true); when(mergeCoordinator.isDescendantOf(safeBlock, newHead)).thenReturn(false); @@ -397,7 +391,7 @@ public void shouldReturnInvalidForkchoiceStateIfSafeBlockIsNotAnAncestorOfNewHea public void shouldIgnoreUpdateToOldHeadAndNotPreparePayload() { BlockHeader mockHeader = new BlockHeaderTestFixture().baseFeePerGas(Wei.ONE).buildHeader(); - when(mergeCoordinator.getOrSyncHeaderByHash(mockHeader.getHash())) + when(mergeCoordinator.getOrSyncHeadByHash(mockHeader.getHash())) .thenReturn(Optional.of(mockHeader)); when(mergeCoordinator.latestValidAncestorDescendsFromTerminal(mockHeader)).thenReturn(true); diff --git a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/manager/task/GetBodiesFromPeerTask.java b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/manager/task/GetBodiesFromPeerTask.java index dd4f2792f54..9eb0f77f5d4 100644 --- a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/manager/task/GetBodiesFromPeerTask.java +++ b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/manager/task/GetBodiesFromPeerTask.java @@ -65,7 +65,7 @@ private GetBodiesFromPeerTask( headers.forEach( (header) -> { final BodyIdentifier bodyId = new BodyIdentifier(header); - bodyToHeaders.putIfAbsent(bodyId, new ArrayList<>()); + bodyToHeaders.putIfAbsent(bodyId, new ArrayList<>(headers.size())); bodyToHeaders.get(bodyId).add(header); }); } @@ -112,7 +112,7 @@ protected Optional> processResponse( return Optional.empty(); } - final List blocks = new ArrayList<>(); + final List blocks = new ArrayList<>(headers.size()); for (final BlockBody body : bodies) { final List headers = bodyToHeaders.get(new BodyIdentifier(body)); if (headers == null) { diff --git a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/manager/task/RetryingGetBlockFromPeersTask.java b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/manager/task/RetryingGetBlockFromPeersTask.java index c1894189065..7d5062ec74d 100644 --- a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/manager/task/RetryingGetBlockFromPeersTask.java +++ b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/manager/task/RetryingGetBlockFromPeersTask.java @@ -102,8 +102,11 @@ protected void handleTaskError(final Throwable error) { this::getAssignedPeer, this::getRetryCount); } else { - LOG.warn( - "Failed to get block {} after {} retries", logBlockNumberMaybeHash(), getRetryCount()); + debugLambda( + LOG, + "Failed to get block {} after {} retries", + this::logBlockNumberMaybeHash, + this::getRetryCount); } super.handleTaskError(error); } diff --git a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/manager/task/RetryingGetBlocksFromPeersTask.java b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/manager/task/RetryingGetBlocksFromPeersTask.java new file mode 100644 index 00000000000..85a663f6230 --- /dev/null +++ b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/manager/task/RetryingGetBlocksFromPeersTask.java @@ -0,0 +1,112 @@ +/* + * Copyright contributors to Hyperledger Besu + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + * + * SPDX-License-Identifier: Apache-2.0 + */ +package org.hyperledger.besu.ethereum.eth.manager.task; + +import static org.hyperledger.besu.util.Slf4jLambdaHelper.debugLambda; + +import org.hyperledger.besu.ethereum.core.Block; +import org.hyperledger.besu.ethereum.core.BlockHeader; +import org.hyperledger.besu.ethereum.eth.manager.EthContext; +import org.hyperledger.besu.ethereum.eth.manager.EthPeer; +import org.hyperledger.besu.ethereum.eth.manager.exceptions.IncompleteResultsException; +import org.hyperledger.besu.ethereum.eth.manager.task.AbstractPeerTask.PeerTaskResult; +import org.hyperledger.besu.ethereum.mainnet.ProtocolSchedule; +import org.hyperledger.besu.plugin.services.MetricsSystem; + +import java.util.List; +import java.util.Objects; +import java.util.concurrent.CompletableFuture; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class RetryingGetBlocksFromPeersTask + extends AbstractRetryingSwitchingPeerTask>> { + + private static final Logger LOG = LoggerFactory.getLogger(RetryingGetBlocksFromPeersTask.class); + + private final ProtocolSchedule protocolSchedule; + private final List headers; + + protected RetryingGetBlocksFromPeersTask( + final EthContext ethContext, + final ProtocolSchedule protocolSchedule, + final MetricsSystem metricsSystem, + final int maxRetries, + final List headers) { + super(ethContext, metricsSystem, Objects::isNull, maxRetries); + this.protocolSchedule = protocolSchedule; + this.headers = headers; + } + + public static RetryingGetBlocksFromPeersTask forHeaders( + final ProtocolSchedule protocolSchedule, + final EthContext ethContext, + final MetricsSystem metricsSystem, + final int maxRetries, + final List headers) { + return new RetryingGetBlocksFromPeersTask( + ethContext, protocolSchedule, metricsSystem, maxRetries, headers); + } + + @Override + protected CompletableFuture>> executeTaskOnCurrentPeer( + final EthPeer currentPeer) { + final GetBodiesFromPeerTask getBodiesTask = + GetBodiesFromPeerTask.forHeaders( + protocolSchedule, getEthContext(), headers, getMetricsSystem()); + getBodiesTask.assignPeer(currentPeer); + + return executeSubTask(getBodiesTask::run) + .thenApply( + peerResult -> { + debugLambda( + LOG, + "Got {} blocks {} from peer {}, attempt {}", + peerResult.getResult()::size, + peerResult.getPeer()::toString, + this::getRetryCount); + + if (peerResult.getResult().isEmpty()) { + currentPeer.recordUselessResponse("GetBodiesFromPeerTask"); + throw new IncompleteResultsException( + "No blocks returned by peer " + currentPeer.getShortNodeId()); + } + + result.complete(peerResult); + return peerResult; + }); + } + + @Override + protected boolean isRetryableError(final Throwable error) { + return super.isRetryableError(error) || error instanceof IncompleteResultsException; + } + + @Override + protected void handleTaskError(final Throwable error) { + if (getRetryCount() < getMaxRetries()) { + debugLambda( + LOG, + "Failed to get {} blocks from peer {}, attempt {}, retrying later", + headers::size, + this::getAssignedPeer, + this::getRetryCount); + } else { + LOG.warn("Failed to get {} blocks after {} retries", headers.size(), getRetryCount()); + } + super.handleTaskError(error); + } +} diff --git a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/manager/task/RetryingGetHeadersEndingAtFromPeerByHashTask.java b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/manager/task/RetryingGetHeadersEndingAtFromPeerByHashTask.java index bc758e6b177..2d1119c4aca 100644 --- a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/manager/task/RetryingGetHeadersEndingAtFromPeerByHashTask.java +++ b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/manager/task/RetryingGetHeadersEndingAtFromPeerByHashTask.java @@ -21,17 +21,21 @@ import org.hyperledger.besu.ethereum.core.BlockHeader; import org.hyperledger.besu.ethereum.eth.manager.EthContext; import org.hyperledger.besu.ethereum.eth.manager.EthPeer; +import org.hyperledger.besu.ethereum.eth.manager.exceptions.IncompleteResultsException; import org.hyperledger.besu.ethereum.mainnet.ProtocolSchedule; import org.hyperledger.besu.plugin.services.MetricsSystem; import java.util.List; -import java.util.Optional; import java.util.concurrent.CompletableFuture; import com.google.common.annotations.VisibleForTesting; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class RetryingGetHeadersEndingAtFromPeerByHashTask - extends AbstractRetryingPeerTask> { + extends AbstractRetryingSwitchingPeerTask> { + private static final Logger LOG = + LoggerFactory.getLogger(RetryingGetHeadersEndingAtFromPeerByHashTask.class); private final Hash referenceHash; private final ProtocolSchedule protocolSchedule; @@ -43,8 +47,9 @@ public class RetryingGetHeadersEndingAtFromPeerByHashTask final EthContext ethContext, final Hash referenceHash, final int count, - final MetricsSystem metricsSystem) { - super(ethContext, 4, List::isEmpty, metricsSystem); + final MetricsSystem metricsSystem, + final int maxRetries) { + super(ethContext, metricsSystem, List::isEmpty, maxRetries); this.protocolSchedule = protocolSchedule; this.count = count; checkNotNull(referenceHash); @@ -56,23 +61,43 @@ public static RetryingGetHeadersEndingAtFromPeerByHashTask endingAtHash( final EthContext ethContext, final Hash referenceHash, final int count, - final MetricsSystem metricsSystem) { + final MetricsSystem metricsSystem, + final int maxRetries) { return new RetryingGetHeadersEndingAtFromPeerByHashTask( - protocolSchedule, ethContext, referenceHash, count, metricsSystem); + protocolSchedule, ethContext, referenceHash, count, metricsSystem, maxRetries); } @Override - protected CompletableFuture> executePeerTask( - final Optional assignedPeer) { + protected CompletableFuture> executeTaskOnCurrentPeer( + final EthPeer currentPeer) { final AbstractGetHeadersFromPeerTask task = GetHeadersFromPeerByHashTask.endingAtHash( protocolSchedule, getEthContext(), referenceHash, count, getMetricsSystem()); - assignedPeer.ifPresent(task::assignPeer); + task.assignPeer(currentPeer); return executeSubTask(task::run) .thenApply( peerResult -> { + LOG.debug( + "Get {} block headers by hash {} from peer {} has result {}", + count, + referenceHash, + currentPeer, + peerResult.getResult()); + if (peerResult.getResult().isEmpty()) { + currentPeer.recordUselessResponse("GetHeadersFromPeerByHashTask"); + throw new IncompleteResultsException( + "No block headers for hash " + + referenceHash + + " returned by peer " + + currentPeer.getShortNodeId()); + } result.complete(peerResult.getResult()); return peerResult.getResult(); }); } + + @Override + protected boolean isRetryableError(final Throwable error) { + return super.isRetryableError(error) || error instanceof IncompleteResultsException; + } } diff --git a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/manager/task/RetryingGetHeadersEndingAtFromPeerByHashTask2.java b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/manager/task/RetryingGetHeadersEndingAtFromPeerByHashTask2.java new file mode 100644 index 00000000000..ce4128146f4 --- /dev/null +++ b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/manager/task/RetryingGetHeadersEndingAtFromPeerByHashTask2.java @@ -0,0 +1,78 @@ +/* + * Copyright Hyperledger Besu Contributors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + * + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.hyperledger.besu.ethereum.eth.manager.task; + +import static com.google.common.base.Preconditions.checkNotNull; + +import org.hyperledger.besu.datatypes.Hash; +import org.hyperledger.besu.ethereum.core.BlockHeader; +import org.hyperledger.besu.ethereum.eth.manager.EthContext; +import org.hyperledger.besu.ethereum.eth.manager.EthPeer; +import org.hyperledger.besu.ethereum.mainnet.ProtocolSchedule; +import org.hyperledger.besu.plugin.services.MetricsSystem; + +import java.util.List; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; + +import com.google.common.annotations.VisibleForTesting; + +public class RetryingGetHeadersEndingAtFromPeerByHashTask2 + extends AbstractRetryingPeerTask> { + + private final Hash referenceHash; + private final ProtocolSchedule protocolSchedule; + private final int count; + + @VisibleForTesting + RetryingGetHeadersEndingAtFromPeerByHashTask2( + final ProtocolSchedule protocolSchedule, + final EthContext ethContext, + final Hash referenceHash, + final int count, + final MetricsSystem metricsSystem) { + super(ethContext, 4, List::isEmpty, metricsSystem); + this.protocolSchedule = protocolSchedule; + this.count = count; + checkNotNull(referenceHash); + this.referenceHash = referenceHash; + } + + public static RetryingGetHeadersEndingAtFromPeerByHashTask2 endingAtHash( + final ProtocolSchedule protocolSchedule, + final EthContext ethContext, + final Hash referenceHash, + final int count, + final MetricsSystem metricsSystem) { + return new RetryingGetHeadersEndingAtFromPeerByHashTask2( + protocolSchedule, ethContext, referenceHash, count, metricsSystem); + } + + @Override + protected CompletableFuture> executePeerTask( + final Optional assignedPeer) { + final AbstractGetHeadersFromPeerTask task = + GetHeadersFromPeerByHashTask.endingAtHash( + protocolSchedule, getEthContext(), referenceHash, count, getMetricsSystem()); + assignedPeer.ifPresent(task::assignPeer); + return executeSubTask(task::run) + .thenApply( + peerResult -> { + result.complete(peerResult.getResult()); + return peerResult.getResult(); + }); + } +} diff --git a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncContext.java b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncContext.java index deb94063441..d6466c3c8fd 100644 --- a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncContext.java +++ b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncContext.java @@ -37,7 +37,7 @@ import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicReference; -import java.util.stream.Stream; +import java.util.function.Supplier; import com.google.common.annotations.VisibleForTesting; import org.slf4j.Logger; @@ -47,7 +47,6 @@ public class BackwardSyncContext { private static final Logger LOG = LoggerFactory.getLogger(BackwardSyncContext.class); public static final int BATCH_SIZE = 200; private static final int DEFAULT_MAX_RETRIES = 20; - private static final long DEFAULT_MILLIS_BETWEEN_RETRIES = 5000; protected final ProtocolContext protocolContext; @@ -55,18 +54,11 @@ public class BackwardSyncContext { private final EthContext ethContext; private final MetricsSystem metricsSystem; private final SyncState syncState; - - private final AtomicReference> currentBackwardSyncFuture = - new AtomicReference<>(); + private final AtomicReference currentBackwardSyncStatus = new AtomicReference<>(); private final BackwardChain backwardChain; private int batchSize = BATCH_SIZE; - private Optional maybeFinalized = Optional.empty(); - private Optional maybeHead = Optional.empty(); - private final int maxRetries; - private final long millisBetweenRetries = DEFAULT_MILLIS_BETWEEN_RETRIES; - private final Subscribers badChainListeners = Subscribers.create(); public BackwardSyncContext( @@ -105,51 +97,33 @@ public BackwardSyncContext( } public synchronized boolean isSyncing() { - return Optional.ofNullable(currentBackwardSyncFuture.get()) - .map(CompletableFuture::isDone) + return Optional.ofNullable(currentBackwardSyncStatus.get()) + .map(status -> status.currentFuture.isDone()) .orElse(Boolean.FALSE); } - public synchronized void updateHeads(final Hash head, final Hash finalizedBlockHash) { - if (Hash.ZERO.equals(finalizedBlockHash)) { - this.maybeFinalized = Optional.empty(); - } else { - this.maybeFinalized = Optional.ofNullable(finalizedBlockHash); - } - if (Hash.ZERO.equals(head)) { - this.maybeHead = Optional.empty(); - } else { - this.maybeHead = Optional.ofNullable(head); - } - } - public synchronized CompletableFuture syncBackwardsUntil(final Hash newBlockHash) { - Optional> maybeFuture = - Optional.ofNullable(this.currentBackwardSyncFuture.get()); - if (isTrusted(newBlockHash)) { - return maybeFuture.orElseGet(() -> CompletableFuture.completedFuture(null)); - } - backwardChain.addNewHash(newBlockHash); - return maybeFuture.orElseGet( - () -> { - CompletableFuture future = prepareBackwardSyncFutureWithRetry(); - this.currentBackwardSyncFuture.set(future); - return future; - }); + return syncBackwardsUntil(() -> newBlockHash, () -> backwardChain.addNewHash(newBlockHash)); } public synchronized CompletableFuture syncBackwardsUntil(final Block newPivot) { + return syncBackwardsUntil(newPivot::getHash, () -> backwardChain.appendTrustedBlock(newPivot)); + } + + private CompletableFuture syncBackwardsUntil( + final Supplier hashProvider, final Runnable saveAction) { Optional> maybeFuture = - Optional.ofNullable(this.currentBackwardSyncFuture.get()); - if (isTrusted(newPivot.getHash())) { + Optional.ofNullable(this.currentBackwardSyncStatus.get()) + .map(status -> status.currentFuture); + if (isTrusted(hashProvider.get())) { return maybeFuture.orElseGet(() -> CompletableFuture.completedFuture(null)); } - backwardChain.appendTrustedBlock(newPivot); + saveAction.run(); return maybeFuture.orElseGet( () -> { - CompletableFuture future = prepareBackwardSyncFutureWithRetry(); - this.currentBackwardSyncFuture.set(future); - return future; + Status status = new Status(prepareBackwardSyncFutureWithRetry()); + this.currentBackwardSyncStatus.set(status); + return status.currentFuture; }); } @@ -168,7 +142,7 @@ private CompletableFuture prepareBackwardSyncFutureWithRetry() { return prepareBackwardSyncFutureWithRetry(maxRetries) .handle( (unused, throwable) -> { - this.currentBackwardSyncFuture.set(null); + this.currentBackwardSyncStatus.set(null); if (throwable != null) { throw extractBackwardSyncException(throwable) .orElse(new BackwardSyncException(throwable)); @@ -201,8 +175,8 @@ protected void processException(final Throwable throwable) { .ifPresentOrElse( backwardSyncException -> { if (backwardSyncException.shouldRestart()) { - LOG.info( - "Backward sync failed ({}). Current Peers: {}. Retrying in {} milliseconds...", + LOG.debug( + "Backward sync failed ({}). Current Peers: {}. Retrying in {} milliseconds", throwable.getMessage(), ethContext.getEthPeers().peerCount(), millisBetweenRetries); @@ -213,8 +187,8 @@ protected void processException(final Throwable throwable) { } }, () -> { - LOG.warn( - "Backward sync failed ({}). Current Peers: {}. Retrying in {} milliseconds...", + LOG.debug( + "Backward sync failed ({}). Current Peers: {}. Retrying in {} milliseconds", throwable.getMessage(), ethContext.getEthPeers().peerCount(), millisBetweenRetries); @@ -278,10 +252,6 @@ public boolean isReady() { && syncState.isInitialSyncPhaseDone(); } - public CompletableFuture stop() { - return currentBackwardSyncFuture.get(); - } - public void subscribeBadChainListener(final BadChainListener badChainListener) { badChainListeners.subscribe(badChainListener); } @@ -315,7 +285,6 @@ protected Void saveBlock(final Block block) { this.getProtocolContext() .getBlockchain() .appendBlock(block, optResult.getYield().get().getReceipts()); - possiblyMoveHead(block); } else { emitBadChainEvent(block); throw new BackwardSyncException( @@ -328,28 +297,6 @@ protected Void saveBlock(final Block block) { return null; } - @VisibleForTesting - protected void possiblyMoveHead(final Block lastSavedBlock) { - final MutableBlockchain blockchain = getProtocolContext().getBlockchain(); - if (maybeHead.isEmpty()) { - LOG.debug("Nothing to do with the head"); - return; - } - if (blockchain.getChainHead().getHash().equals(maybeHead.get())) { - LOG.debug("Head is already properly set"); - return; - } - if (blockchain.contains(maybeHead.get())) { - LOG.debug("Changing head to {}", maybeHead.get().toHexString()); - blockchain.rewindToBlock(maybeHead.get()); - return; - } - if (blockchain.getChainHead().getHash().equals(lastSavedBlock.getHash())) { - LOG.debug("Rewinding head to lastSavedBlock {}", lastSavedBlock.getHash()); - blockchain.rewindToBlock(lastSavedBlock.getHash()); - } - } - public SyncState getSyncState() { return syncState; } @@ -358,11 +305,8 @@ public synchronized BackwardChain getBackwardChain() { return backwardChain; } - public Optional findMaybeFinalized() { - return Stream.of(maybeFinalized, getProtocolContext().getBlockchain().getFinalized()) - .filter(Optional::isPresent) - .map(Optional::get) - .findFirst(); + public Status getStatus() { + return currentBackwardSyncStatus.get(); } private void emitBadChainEvent(final Block badBlock) { @@ -385,4 +329,31 @@ private void emitBadChainEvent(final Block badBlock) { badChainListeners.forEach( listener -> listener.onBadChain(badBlock, badBlockDescendants, badBlockHeaderDescendants)); } + + static class Status { + private final CompletableFuture currentFuture; + private long targetChainHeight; + private long initialChainHeight; + + public Status(final CompletableFuture currentFuture) { + this.currentFuture = currentFuture; + } + + public void setSyncRange(final long initialHeight, final long targetHeight) { + initialChainHeight = initialHeight; + targetChainHeight = targetHeight; + } + + public long getTargetChainHeight() { + return targetChainHeight; + } + + public long getInitialChainHeight() { + return initialChainHeight; + } + + public long getBlockCount() { + return targetChainHeight - initialChainHeight; + } + } } diff --git a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncStep.java b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncStep.java index acdfc823c6d..99d643de236 100644 --- a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncStep.java +++ b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncStep.java @@ -15,7 +15,6 @@ package org.hyperledger.besu.ethereum.eth.sync.backwardsync; import static org.hyperledger.besu.util.Slf4jLambdaHelper.debugLambda; -import static org.hyperledger.besu.util.Slf4jLambdaHelper.infoLambda; import org.hyperledger.besu.datatypes.Hash; import org.hyperledger.besu.ethereum.core.BlockHeader; @@ -61,7 +60,7 @@ protected Hash possibleRestoreOldNodes(final BlockHeader firstAncestor) { @VisibleForTesting protected CompletableFuture> requestHeaders(final Hash hash) { final int batchSize = context.getBatchSize(); - debugLambda(LOG, "Requesting header for hash {}", hash::toHexString); + LOG.debug("Requesting headers for hash {}, with batch size {}", hash, batchSize); final RetryingGetHeadersEndingAtFromPeerByHashTask retryingGetHeadersEndingAtFromPeerByHashTask = @@ -70,17 +69,14 @@ protected CompletableFuture> requestHeaders(final Hash hash) { context.getEthContext(), hash, batchSize, - context.getMetricsSystem()); + context.getMetricsSystem(), + context.getEthContext().getEthPeers().peerCount()); return context .getEthContext() .getScheduler() .scheduleSyncWorkerTask(retryingGetHeadersEndingAtFromPeerByHashTask::run) .thenApply( blockHeaders -> { - if (blockHeaders.isEmpty()) { - throw new BackwardSyncException( - "Did not receive a headers for hash " + hash.toHexString(), true); - } debugLambda( LOG, "Got headers {} -> {}", @@ -101,12 +97,27 @@ protected Void saveHeaders(final List blockHeaders) { for (BlockHeader blockHeader : blockHeaders) { saveHeader(blockHeader); } - infoLambda( - LOG, - "Saved headers {} -> {} (head: {})", - () -> blockHeaders.get(0).getNumber(), - () -> blockHeaders.get(blockHeaders.size() - 1).getNumber(), - () -> context.getProtocolContext().getBlockchain().getChainHead().getHeight()); + + logProgress(blockHeaders.size(), blockHeaders.get(blockHeaders.size() - 1).getNumber()); + return null; } + + private void logProgress(final int downloadedBatchSize, final long currLowestDownloadedHeight) { + final long targetHeight = context.getStatus().getTargetChainHeight(); + final long initialHeight = context.getStatus().getInitialChainHeight(); + final long estimatedTotal = targetHeight - Math.min(initialHeight, currLowestDownloadedHeight); + final long downloaded = targetHeight - currLowestDownloadedHeight; + + final float completedPercentage = 100.0f * downloaded / estimatedTotal; + + LOG.info( + String.format( + "Backward sync phase 1 of 2, %.2f%% completed, downloaded %d (+%d) headers (estimated total %d). Peers: %d", + completedPercentage, + downloaded, + downloadedBatchSize, + estimatedTotal, + context.getEthContext().getEthPeers().peerCount())); + } } diff --git a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardsSyncAlgorithm.java b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardsSyncAlgorithm.java index 8ff1402f282..910adcc6466 100644 --- a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardsSyncAlgorithm.java +++ b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardsSyncAlgorithm.java @@ -14,7 +14,6 @@ * SPDX-License-Identifier: Apache-2.0 * */ - package org.hyperledger.besu.ethereum.eth.sync.backwardsync; import static org.hyperledger.besu.util.Slf4jLambdaHelper.debugLambda; @@ -22,11 +21,14 @@ import org.hyperledger.besu.datatypes.Hash; import org.hyperledger.besu.ethereum.chain.MutableBlockchain; +import org.hyperledger.besu.ethereum.core.Block; import org.hyperledger.besu.ethereum.core.BlockHeader; +import org.hyperledger.besu.ethereum.eth.manager.task.WaitForPeersTask; import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import com.google.common.annotations.VisibleForTesting; @@ -57,41 +59,49 @@ public CompletableFuture pickNextStep() { final Optional firstHash = context.getBackwardChain().getFirstHashToAppend(); if (firstHash.isPresent()) { return executeSyncStep(firstHash.get()) - .whenComplete( - (result, throwable) -> { - if (throwable == null) { - context.getBackwardChain().removeFromHashToAppend(firstHash.get()); - } + .thenAccept( + result -> { + LOG.info("Backward sync target block is {}", result.toLogString()); + context.getBackwardChain().removeFromHashToAppend(firstHash.get()); + context + .getStatus() + .setSyncRange( + context.getProtocolContext().getBlockchain().getChainHeadBlockNumber(), + result.getHeader().getNumber()); }); } if (!context.isReady()) { return waitForReady(); } - runFinalizedSuccessionRule( - context.getProtocolContext().getBlockchain(), context.findMaybeFinalized()); - final Optional possibleFirstAncestorHeader = + + final Optional maybeFirstAncestorHeader = context.getBackwardChain().getFirstAncestorHeader(); - if (possibleFirstAncestorHeader.isEmpty()) { + if (maybeFirstAncestorHeader.isEmpty()) { this.finished = true; - LOG.info("The Backward sync is done..."); + LOG.info("The Backward sync is done"); context.getBackwardChain().clear(); return CompletableFuture.completedFuture(null); } + final MutableBlockchain blockchain = context.getProtocolContext().getBlockchain(); - final BlockHeader firstAncestorHeader = possibleFirstAncestorHeader.get(); + final BlockHeader firstAncestorHeader = maybeFirstAncestorHeader.get(); if (blockchain.contains(firstAncestorHeader.getHash())) { return executeProcessKnownAncestors(); } + if (blockchain.getChainHead().getHeight() > firstAncestorHeader.getNumber()) { debugLambda( LOG, - "Backward reached below previous head {} : {}", + "Backward reached below current chain head {} : {}", () -> blockchain.getChainHead().toLogString(), firstAncestorHeader::toLogString); } if (finalBlockConfirmation.ancestorHeaderReached(firstAncestorHeader)) { - LOG.info("Backward sync reached ancestor header, starting Forward sync"); + debugLambda( + LOG, + "Backward sync reached ancestor header with {}, starting Forward sync", + firstAncestorHeader::toLogString); return executeForwardAsync(); } @@ -104,7 +114,7 @@ public CompletableFuture executeProcessKnownAncestors() { } @VisibleForTesting - public CompletableFuture executeSyncStep(final Hash hash) { + public CompletableFuture executeSyncStep(final Hash hash) { return new SyncStepStep(context, context.getBackwardChain()).executeAsync(hash); } @@ -134,12 +144,18 @@ private void checkReadiness(final CountDownLatch latch, final long idTTD, final LOG.debug("Waiting for preconditions..."); final boolean await = latch.await(2, TimeUnit.MINUTES); if (await) { - LOG.debug("Preconditions meet..."); + LOG.debug("Preconditions meet, ensure at least one peer is connected"); + // ensure at least one peer is connected + waitForPeers(1).get(); } } } catch (InterruptedException e) { Thread.currentThread().interrupt(); - throw new BackwardSyncException("Wait for TTD preconditions interrupted"); + throw new BackwardSyncException( + "Wait for TTD preconditions interrupted (" + e.getMessage() + ")"); + } catch (ExecutionException e) { + throw new BackwardSyncException( + "Error while waiting for at least one connected peer (" + e.getMessage() + ")"); } finally { context.getSyncState().unsubscribeTTDReached(idTTD); context.getSyncState().unsubscribeInitialConditionReached(idIS); @@ -152,49 +168,9 @@ private void countDownIfReady(final CountDownLatch latch) { } } - @VisibleForTesting - protected void runFinalizedSuccessionRule( - final MutableBlockchain blockchain, final Optional maybeFinalized) { - if (maybeFinalized.isEmpty()) { - LOG.debug("Nothing to validate yet, consensus layer did not provide a new finalized block"); - return; - } - final Hash newFinalized = maybeFinalized.get(); - if (!blockchain.contains(newFinalized)) { - LOG.debug("New finalized block {} is not imported yet", newFinalized); - return; - } - - final Optional maybeOldFinalized = blockchain.getFinalized(); - if (maybeOldFinalized.isPresent()) { - final Hash oldFinalized = maybeOldFinalized.get(); - if (newFinalized.equals(oldFinalized)) { - LOG.debug("We already have this block as finalized"); - return; - } - BlockHeader newFinalizedHeader = - blockchain - .getBlockHeader(newFinalized) - .orElseThrow( - () -> - new BackwardSyncException( - "The header " + newFinalized.toHexString() + "not found")); - BlockHeader oldFinalizedHeader = - blockchain - .getBlockHeader(oldFinalized) - .orElseThrow( - () -> - new BackwardSyncException( - "The header " + oldFinalized.toHexString() + "not found")); - LOG.info( - "Updating finalized {} block to new finalized block {}", - oldFinalizedHeader.toLogString(), - newFinalizedHeader.toLogString()); - } else { - // Todo: should TTD test be here? - LOG.info("Setting new finalized block to {}", newFinalized); - } - - blockchain.setFinalized(newFinalized); + private CompletableFuture waitForPeers(final int count) { + final WaitForPeersTask waitForPeersTask = + WaitForPeersTask.create(context.getEthContext(), count, context.getMetricsSystem()); + return waitForPeersTask.run(); } } diff --git a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/ForwardSyncStep.java b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/ForwardSyncStep.java index 71c0aa26cd8..fded485ae56 100644 --- a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/ForwardSyncStep.java +++ b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/ForwardSyncStep.java @@ -15,12 +15,11 @@ package org.hyperledger.besu.ethereum.eth.sync.backwardsync; import static org.hyperledger.besu.util.Slf4jLambdaHelper.debugLambda; -import static org.hyperledger.besu.util.Slf4jLambdaHelper.infoLambda; import org.hyperledger.besu.ethereum.core.Block; import org.hyperledger.besu.ethereum.core.BlockHeader; import org.hyperledger.besu.ethereum.eth.manager.task.AbstractPeerTask; -import org.hyperledger.besu.ethereum.eth.manager.task.GetBodiesFromPeerTask; +import org.hyperledger.besu.ethereum.eth.manager.task.RetryingGetBlocksFromPeersTask; import java.util.Comparator; import java.util.List; @@ -66,18 +65,20 @@ public CompletableFuture possibleRequestBodies(final List blo @VisibleForTesting protected CompletableFuture> requestBodies(final List blockHeaders) { - final GetBodiesFromPeerTask getBodiesFromPeerTask = - GetBodiesFromPeerTask.forHeaders( + final RetryingGetBlocksFromPeersTask getBodiesFromPeerTask = + RetryingGetBlocksFromPeersTask.forHeaders( context.getProtocolSchedule(), context.getEthContext(), - blockHeaders, - context.getMetricsSystem()); + context.getMetricsSystem(), + context.getEthContext().getEthPeers().peerCount(), + blockHeaders); final CompletableFuture>> run = getBodiesFromPeerTask.run(); return run.thenApply(AbstractPeerTask.PeerTaskResult::getResult) .thenApply( blocks -> { + LOG.debug("Got {} blocks from peers", blocks.size()); blocks.sort(Comparator.comparing(block -> block.getHeader().getNumber())); return blocks; }); @@ -86,8 +87,8 @@ protected CompletableFuture> requestBodies(final List b @VisibleForTesting protected Void saveBlocks(final List blocks) { if (blocks.isEmpty()) { - LOG.info("No blocks to save..."); context.halveBatchSize(); + LOG.debug("No blocks to save, reducing batch size to {}", context.getBatchSize()); return null; } @@ -97,21 +98,43 @@ protected Void saveBlocks(final List blocks) { .getProtocolContext() .getBlockchain() .getBlockByHash(block.getHeader().getParentHash()); + if (parent.isEmpty()) { context.halveBatchSize(); + debugLambda( + LOG, + "Parent block {} not found, while saving block {}, reducing batch size to {}", + block.getHeader().getParentHash()::toString, + block::toLogString, + context::getBatchSize); + logProgress(blocks.size(), blocks.get(blocks.size() - 1).getHeader().getNumber()); return null; } else { context.saveBlock(block); } } - infoLambda( - LOG, - "Saved blocks {} -> {} (target: {})", - () -> blocks.get(0).getHeader().getNumber(), - () -> blocks.get(blocks.size() - 1).getHeader().getNumber(), - () -> - backwardChain.getPivot().orElse(blocks.get(blocks.size() - 1)).getHeader().getNumber()); + + logProgress(blocks.size(), blocks.get(blocks.size() - 1).getHeader().getNumber()); + context.resetBatchSize(); return null; } + + private void logProgress(final int importedBatchSize, final long currImportedHeight) { + final long targetHeight = context.getStatus().getTargetChainHeight(); + final long initialHeight = context.getStatus().getInitialChainHeight(); + final long estimatedTotal = Math.max(targetHeight, currImportedHeight) - initialHeight; + final long imported = currImportedHeight - initialHeight; + + final float completedPercentage = 100.0f * imported / estimatedTotal; + + LOG.info( + String.format( + "Backward sync phase 2 of 2, %.2f%% completed, imported %d (+%d) blocks (estimated total %d). Peers: %d", + completedPercentage, + imported, + importedBatchSize, + estimatedTotal, + context.getEthContext().getEthPeers().peerCount())); + } } diff --git a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/SyncStepStep.java b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/SyncStepStep.java index 88ddc18b2c6..2dc8b1d4351 100644 --- a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/SyncStepStep.java +++ b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/SyncStepStep.java @@ -17,6 +17,7 @@ package org.hyperledger.besu.ethereum.eth.sync.backwardsync; +import static org.hyperledger.besu.util.Slf4jLambdaHelper.debugLambda; import static org.slf4j.LoggerFactory.getLogger; import org.hyperledger.besu.datatypes.Hash; @@ -41,19 +42,20 @@ public SyncStepStep(final BackwardSyncContext context, final BackwardChain backw this.backwardChain = backwardChain; } - public CompletableFuture executeAsync(final Hash hash) { + public CompletableFuture executeAsync(final Hash hash) { return CompletableFuture.supplyAsync(() -> hash) .thenCompose(this::requestBlock) .thenApply(this::saveBlock); } private CompletableFuture requestBlock(final Hash targetHash) { + debugLambda(LOG, "Fetching block by hash {} from peers", targetHash::toString); final RetryingGetBlockFromPeersTask getBlockTask = RetryingGetBlockFromPeersTask.create( context.getProtocolSchedule(), context.getEthContext(), context.getMetricsSystem(), - context.getEthContext().getEthPeers().getMaxPeers(), + context.getEthContext().getEthPeers().peerCount(), Optional.of(targetHash), UNUSED); return context @@ -63,10 +65,9 @@ private CompletableFuture requestBlock(final Hash targetHash) { .thenApply(AbstractPeerTask.PeerTaskResult::getResult); } - private Void saveBlock(final Block block) { - LOG.debug( - "Appending block {}({})", block.getHeader().getNumber(), block.getHash().toHexString()); + private Block saveBlock(final Block block) { + debugLambda(LOG, "Appending fetched block {}", block::toLogString); backwardChain.appendTrustedBlock(block); - return null; + return block; } } diff --git a/ethereum/eth/src/test/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncAlgSpec.java b/ethereum/eth/src/test/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncAlgSpec.java index 3d4782bef2d..69ab5470f3a 100644 --- a/ethereum/eth/src/test/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncAlgSpec.java +++ b/ethereum/eth/src/test/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncAlgSpec.java @@ -36,7 +36,6 @@ import java.nio.charset.StandardCharsets; import java.util.List; -import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import javax.annotation.Nonnull; @@ -149,6 +148,7 @@ public void shouldAwokeWhenTTDReachedAndReady() throws Exception { when(context.getSyncState().subscribeTTDReached(any())).thenReturn(88L); when(context.getSyncState().subscribeCompletionReached(any())).thenReturn(99L); + when(context.getEthContext().getEthPeers().peerCount()).thenReturn(1); final CompletableFuture voidCompletableFuture = algorithm.waitForReady(); @@ -175,6 +175,7 @@ public void shouldAwokeWhenConditionReachedAndReady() throws Exception { when(context.getSyncState().subscribeTTDReached(any())).thenReturn(88L); when(context.getSyncState().subscribeCompletionReached(any())).thenReturn(99L); + when(context.getEthContext().getEthPeers().peerCount()).thenReturn(1); final CompletableFuture voidCompletableFuture = algorithm.waitForReady(); Thread.sleep(50); @@ -260,93 +261,6 @@ public void shouldRunBackwardStepWhenNotOnLocalHeight() { verify(algorithm).executeBackwardAsync(any()); } - @Test - public void successionShouldIgnoreEmptyFinalized() { - final BackwardsSyncAlgorithm backwardsSyncAlgorithm = - new BackwardsSyncAlgorithm(context, firstHeader -> false); - - Optional finalized = localBlockchain.getFinalized(); - assertThat(finalized).isEmpty(); - - backwardsSyncAlgorithm.runFinalizedSuccessionRule(localBlockchain, Optional.empty()); - - finalized = localBlockchain.getFinalized(); - assertThat(finalized).isEmpty(); - } - - @Test - public void successionShouldSetFinalizedFromEmpty() { - final BackwardsSyncAlgorithm backwardsSyncAlgorithm = - new BackwardsSyncAlgorithm(context, firstHeader -> false); - - Optional finalized = localBlockchain.getFinalized(); - assertThat(finalized).isEmpty(); - - backwardsSyncAlgorithm.runFinalizedSuccessionRule( - localBlockchain, Optional.of(localBlockchain.getChainHead().getHash())); - - finalized = localBlockchain.getFinalized(); - assertThat(finalized).isPresent(); - assertThat(finalized).contains(localBlockchain.getChainHead().getHash()); - } - - @Test - public void successionShouldIgnoreFinalisedWhenNotImportedYet() { - final BackwardsSyncAlgorithm backwardsSyncAlgorithm = - new BackwardsSyncAlgorithm(context, firstHeader -> false); - - Optional finalized = localBlockchain.getFinalized(); - assertThat(finalized).isEmpty(); - - backwardsSyncAlgorithm.runFinalizedSuccessionRule( - localBlockchain, Optional.of(remoteBlockchain.getChainHead().getHash())); - - finalized = localBlockchain.getFinalized(); - assertThat(finalized).isEmpty(); - } - - @Test - public void successionShouldKeepFinalizedWhenNotChanged() { - final BackwardsSyncAlgorithm backwardsSyncAlgorithm = - new BackwardsSyncAlgorithm(context, firstHeader -> false); - - Optional finalized = localBlockchain.getFinalized(); - assertThat(finalized).isEmpty(); - - backwardsSyncAlgorithm.runFinalizedSuccessionRule( - localBlockchain, Optional.of(localBlockchain.getChainHead().getHash())); - backwardsSyncAlgorithm.runFinalizedSuccessionRule( - localBlockchain, Optional.of(localBlockchain.getChainHead().getHash())); - - finalized = localBlockchain.getFinalized(); - assertThat(finalized).isPresent(); - assertThat(finalized).contains(localBlockchain.getChainHead().getHash()); - } - - @Test - public void successionShouldUpdateOldFinalizedToNewFinalized() { - final BackwardsSyncAlgorithm backwardsSyncAlgorithm = - new BackwardsSyncAlgorithm(context, firstHeader -> false); - - Optional finalized = localBlockchain.getFinalized(); - assertThat(finalized).isEmpty(); - - final Hash fin1 = localBlockchain.getBlockByNumber(LOCAL_HEIGHT - 5).orElseThrow().getHash(); - backwardsSyncAlgorithm.runFinalizedSuccessionRule(localBlockchain, Optional.of(fin1)); - - finalized = localBlockchain.getFinalized(); - assertThat(finalized).isPresent(); - assertThat(finalized).contains(fin1); - - final Hash fin2 = localBlockchain.getBlockByNumber(LOCAL_HEIGHT - 3).orElseThrow().getHash(); - - backwardsSyncAlgorithm.runFinalizedSuccessionRule(localBlockchain, Optional.of(fin2)); - - finalized = localBlockchain.getFinalized(); - assertThat(finalized).isPresent(); - assertThat(finalized).contains(fin2); - } - @Test public void shouldStartForwardSyncIfGenesisIsReached() { doReturn(true).when(context).isReady(); diff --git a/ethereum/eth/src/test/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncContextTest.java b/ethereum/eth/src/test/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncContextTest.java index cf2c4eb92f8..897786ce94b 100644 --- a/ethereum/eth/src/test/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncContextTest.java +++ b/ethereum/eth/src/test/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncContextTest.java @@ -247,23 +247,6 @@ private Block getBlockByNumber(final int number) { return remoteBlockchain.getBlockByNumber(number).orElseThrow(); } - @Test - public void testUpdatingHead() { - context.updateHeads(null, null); - context.possiblyMoveHead(null); - assertThat(localBlockchain.getChainHeadBlock().getHeader().getNumber()).isEqualTo(LOCAL_HEIGHT); - - context.updateHeads(Hash.ZERO, null); - context.possiblyMoveHead(null); - - assertThat(localBlockchain.getChainHeadBlock().getHeader().getNumber()).isEqualTo(LOCAL_HEIGHT); - - context.updateHeads(localBlockchain.getBlockByNumber(4).orElseThrow().getHash(), null); - context.possiblyMoveHead(null); - - assertThat(localBlockchain.getChainHeadBlock().getHeader().getNumber()).isEqualTo(4); - } - @Test public void shouldProcessExceptionsCorrectly() { assertThatThrownBy( diff --git a/ethereum/eth/src/test/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncStepTest.java b/ethereum/eth/src/test/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncStepTest.java index ada5b5494f7..7f9e5b7c5b5 100644 --- a/ethereum/eth/src/test/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncStepTest.java +++ b/ethereum/eth/src/test/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncStepTest.java @@ -33,6 +33,7 @@ import org.hyperledger.besu.ethereum.eth.manager.EthProtocolManager; import org.hyperledger.besu.ethereum.eth.manager.EthProtocolManagerTestUtil; import org.hyperledger.besu.ethereum.eth.manager.RespondingEthPeer; +import org.hyperledger.besu.ethereum.eth.manager.exceptions.MaxRetriesReachedException; import org.hyperledger.besu.ethereum.mainnet.MainnetBlockHeaderFunctions; import org.hyperledger.besu.ethereum.mainnet.MainnetProtocolSchedule; import org.hyperledger.besu.ethereum.mainnet.ProtocolSchedule; @@ -181,7 +182,7 @@ public void shouldRequestHeaderBeforeCurrentHeight() throws Exception { } @Test - public void shouldThrowWhenResponseIsEmptyWhenRequestingHeader() throws Exception { + public void shouldThrowWhenResponseIsEmptyWhenRequestingHeader() { BackwardSyncStep step = new BackwardSyncStep(context, createBackwardChain(REMOTE_HEIGHT - 1)); final Block lookingForBlock = getBlockByNumber(REMOTE_HEIGHT - 2); @@ -191,10 +192,7 @@ public void shouldThrowWhenResponseIsEmptyWhenRequestingHeader() throws Exceptio step.requestHeaders(lookingForBlock.getHeader().getHash()); peer.respondWhileOtherThreadsWork(responder, () -> !future.isDone()); - assertThatThrownBy(future::get) - .getCause() - .isInstanceOf(BackwardSyncException.class) - .hasMessageContaining("Did not receive a headers for hash"); + assertThatThrownBy(future::get).cause().isInstanceOf(MaxRetriesReachedException.class); } @Test From b22d1a00f35dd52306870491350fa19708b2a375 Mon Sep 17 00:00:00 2001 From: Fabio Di Fabio Date: Fri, 11 Nov 2022 12:58:23 +0100 Subject: [PATCH 02/32] Backward sync log UX improvements Signed-off-by: Fabio Di Fabio --- .../merge/blockcreation/MergeCoordinator.java | 2 +- .../task/RetryingGetBlockFromPeersTask.java | 7 +- .../backwardsync/BackwardSyncContext.java | 71 +++++++++++++------ .../sync/backwardsync/BackwardSyncStep.java | 42 ++++++++--- .../backwardsync/BackwardsSyncAlgorithm.java | 26 ++++--- .../sync/backwardsync/ForwardSyncStep.java | 52 +++++++++++--- .../eth/sync/backwardsync/SyncStepStep.java | 11 +-- 7 files changed, 155 insertions(+), 56 deletions(-) diff --git a/consensus/merge/src/main/java/org/hyperledger/besu/consensus/merge/blockcreation/MergeCoordinator.java b/consensus/merge/src/main/java/org/hyperledger/besu/consensus/merge/blockcreation/MergeCoordinator.java index 4ca248546fa..438d294602c 100644 --- a/consensus/merge/src/main/java/org/hyperledger/besu/consensus/merge/blockcreation/MergeCoordinator.java +++ b/consensus/merge/src/main/java/org/hyperledger/besu/consensus/merge/blockcreation/MergeCoordinator.java @@ -439,7 +439,7 @@ public ForkchoiceResult updateForkChoice( if (newHead.getNumber() < blockchain.getChainHeadBlockNumber() && isDescendantOf(newHead, blockchain.getChainHeadHeader())) { - LOG.info("Ignoring update to old head"); + debugLambda(LOG, "Ignoring update to old head {}", newHead::toLogString); return ForkchoiceResult.withIgnoreUpdateToOldHead(newHead); } diff --git a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/manager/task/RetryingGetBlockFromPeersTask.java b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/manager/task/RetryingGetBlockFromPeersTask.java index c1894189065..7d5062ec74d 100644 --- a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/manager/task/RetryingGetBlockFromPeersTask.java +++ b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/manager/task/RetryingGetBlockFromPeersTask.java @@ -102,8 +102,11 @@ protected void handleTaskError(final Throwable error) { this::getAssignedPeer, this::getRetryCount); } else { - LOG.warn( - "Failed to get block {} after {} retries", logBlockNumberMaybeHash(), getRetryCount()); + debugLambda( + LOG, + "Failed to get block {} after {} retries", + this::logBlockNumberMaybeHash, + this::getRetryCount); } super.handleTaskError(error); } diff --git a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncContext.java b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncContext.java index deb94063441..fb941722733 100644 --- a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncContext.java +++ b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncContext.java @@ -55,9 +55,7 @@ public class BackwardSyncContext { private final EthContext ethContext; private final MetricsSystem metricsSystem; private final SyncState syncState; - - private final AtomicReference> currentBackwardSyncFuture = - new AtomicReference<>(); + private final AtomicReference currentBackwardSyncStatus = new AtomicReference<>(); private final BackwardChain backwardChain; private int batchSize = BATCH_SIZE; private Optional maybeFinalized = Optional.empty(); @@ -105,8 +103,8 @@ public BackwardSyncContext( } public synchronized boolean isSyncing() { - return Optional.ofNullable(currentBackwardSyncFuture.get()) - .map(CompletableFuture::isDone) + return Optional.ofNullable(currentBackwardSyncStatus.get()) + .map(status -> status.currentFuture.isDone()) .orElse(Boolean.FALSE); } @@ -125,31 +123,33 @@ public synchronized void updateHeads(final Hash head, final Hash finalizedBlockH public synchronized CompletableFuture syncBackwardsUntil(final Hash newBlockHash) { Optional> maybeFuture = - Optional.ofNullable(this.currentBackwardSyncFuture.get()); + Optional.ofNullable(this.currentBackwardSyncStatus.get()) + .map(status -> status.currentFuture); if (isTrusted(newBlockHash)) { return maybeFuture.orElseGet(() -> CompletableFuture.completedFuture(null)); } backwardChain.addNewHash(newBlockHash); return maybeFuture.orElseGet( () -> { - CompletableFuture future = prepareBackwardSyncFutureWithRetry(); - this.currentBackwardSyncFuture.set(future); - return future; + Status status = new Status(prepareBackwardSyncFutureWithRetry()); + this.currentBackwardSyncStatus.set(status); + return status.currentFuture; }); } public synchronized CompletableFuture syncBackwardsUntil(final Block newPivot) { Optional> maybeFuture = - Optional.ofNullable(this.currentBackwardSyncFuture.get()); + Optional.ofNullable(this.currentBackwardSyncStatus.get()) + .map(status -> status.currentFuture); if (isTrusted(newPivot.getHash())) { return maybeFuture.orElseGet(() -> CompletableFuture.completedFuture(null)); } backwardChain.appendTrustedBlock(newPivot); return maybeFuture.orElseGet( () -> { - CompletableFuture future = prepareBackwardSyncFutureWithRetry(); - this.currentBackwardSyncFuture.set(future); - return future; + Status status = new Status(prepareBackwardSyncFutureWithRetry()); + this.currentBackwardSyncStatus.set(status); + return status.currentFuture; }); } @@ -168,7 +168,7 @@ private CompletableFuture prepareBackwardSyncFutureWithRetry() { return prepareBackwardSyncFutureWithRetry(maxRetries) .handle( (unused, throwable) -> { - this.currentBackwardSyncFuture.set(null); + this.currentBackwardSyncStatus.set(null); if (throwable != null) { throw extractBackwardSyncException(throwable) .orElse(new BackwardSyncException(throwable)); @@ -201,8 +201,8 @@ protected void processException(final Throwable throwable) { .ifPresentOrElse( backwardSyncException -> { if (backwardSyncException.shouldRestart()) { - LOG.info( - "Backward sync failed ({}). Current Peers: {}. Retrying in {} milliseconds...", + LOG.debug( + "Backward sync failed ({}). Current Peers: {}. Retrying in {} milliseconds", throwable.getMessage(), ethContext.getEthPeers().peerCount(), millisBetweenRetries); @@ -213,8 +213,8 @@ protected void processException(final Throwable throwable) { } }, () -> { - LOG.warn( - "Backward sync failed ({}). Current Peers: {}. Retrying in {} milliseconds...", + LOG.debug( + "Backward sync failed ({}). Current Peers: {}. Retrying in {} milliseconds", throwable.getMessage(), ethContext.getEthPeers().peerCount(), millisBetweenRetries); @@ -278,10 +278,6 @@ public boolean isReady() { && syncState.isInitialSyncPhaseDone(); } - public CompletableFuture stop() { - return currentBackwardSyncFuture.get(); - } - public void subscribeBadChainListener(final BadChainListener badChainListener) { badChainListeners.subscribe(badChainListener); } @@ -365,6 +361,10 @@ public Optional findMaybeFinalized() { .findFirst(); } + public Status getStatus() { + return currentBackwardSyncStatus.get(); + } + private void emitBadChainEvent(final Block badBlock) { final List badBlockDescendants = new ArrayList<>(); final List badBlockHeaderDescendants = new ArrayList<>(); @@ -385,4 +385,31 @@ private void emitBadChainEvent(final Block badBlock) { badChainListeners.forEach( listener -> listener.onBadChain(badBlock, badBlockDescendants, badBlockHeaderDescendants)); } + + static class Status { + private final CompletableFuture currentFuture; + private long targetChainHeight; + private long initialChainHeight; + + public Status(final CompletableFuture currentFuture) { + this.currentFuture = currentFuture; + } + + public void setSyncRange(final long initialHeight, final long targetHeight) { + initialChainHeight = initialHeight; + targetChainHeight = targetHeight; + } + + public long getTargetChainHeight() { + return targetChainHeight; + } + + public long getInitialChainHeight() { + return initialChainHeight; + } + + public long getBlockCount() { + return targetChainHeight - initialChainHeight; + } + } } diff --git a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncStep.java b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncStep.java index acdfc823c6d..e22e4bbd698 100644 --- a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncStep.java +++ b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncStep.java @@ -15,7 +15,6 @@ package org.hyperledger.besu.ethereum.eth.sync.backwardsync; import static org.hyperledger.besu.util.Slf4jLambdaHelper.debugLambda; -import static org.hyperledger.besu.util.Slf4jLambdaHelper.infoLambda; import org.hyperledger.besu.datatypes.Hash; import org.hyperledger.besu.ethereum.core.BlockHeader; @@ -31,6 +30,8 @@ public class BackwardSyncStep { private static final Logger LOG = LoggerFactory.getLogger(BackwardSyncStep.class); + private static final long MILLIS_DELAY_BETWEEN_PROGRESS_LOG = 10_000L; + private static long lastLogAt = 0; private final BackwardSyncContext context; private final BackwardChain backwardChain; @@ -61,7 +62,7 @@ protected Hash possibleRestoreOldNodes(final BlockHeader firstAncestor) { @VisibleForTesting protected CompletableFuture> requestHeaders(final Hash hash) { final int batchSize = context.getBatchSize(); - debugLambda(LOG, "Requesting header for hash {}", hash::toHexString); + LOG.debug("Requesting headers for hash {}, with batch size {}", hash, batchSize); final RetryingGetHeadersEndingAtFromPeerByHashTask retryingGetHeadersEndingAtFromPeerByHashTask = @@ -101,12 +102,37 @@ protected Void saveHeaders(final List blockHeaders) { for (BlockHeader blockHeader : blockHeaders) { saveHeader(blockHeader); } - infoLambda( - LOG, - "Saved headers {} -> {} (head: {})", - () -> blockHeaders.get(0).getNumber(), - () -> blockHeaders.get(blockHeaders.size() - 1).getNumber(), - () -> context.getProtocolContext().getBlockchain().getChainHead().getHeight()); + + logProgress(blockHeaders.get(blockHeaders.size() - 1).getNumber()); + return null; } + + private void logProgress(final long currLowestDownloadedHeight) { + final long targetHeight = context.getStatus().getTargetChainHeight(); + final long initialHeight = context.getStatus().getInitialChainHeight(); + final long estimatedTotal = targetHeight - initialHeight; + final long downloaded = targetHeight - currLowestDownloadedHeight; + + final float completedPercentage = 100.0f * downloaded / estimatedTotal; + + if (currLowestDownloadedHeight > initialHeight) { + final long now = System.currentTimeMillis(); + if (now - lastLogAt > MILLIS_DELAY_BETWEEN_PROGRESS_LOG) { + LOG.info( + String.format( + "Backward sync phase 1 of 2, %.2f%% completed, downloaded %d headers of at least %d. Peers: %d", + completedPercentage, + downloaded, + estimatedTotal, + context.getEthContext().getEthPeers().peerCount())); + lastLogAt = now; + } + } else { + LOG.info( + String.format( + "Backward sync phase 1 of 2 completed, downloaded a total of %d headers. Peers: %d", + downloaded, context.getEthContext().getEthPeers().peerCount())); + } + } } diff --git a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardsSyncAlgorithm.java b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardsSyncAlgorithm.java index 8ff1402f282..35c3f90ec21 100644 --- a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardsSyncAlgorithm.java +++ b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardsSyncAlgorithm.java @@ -22,6 +22,7 @@ import org.hyperledger.besu.datatypes.Hash; import org.hyperledger.besu.ethereum.chain.MutableBlockchain; +import org.hyperledger.besu.ethereum.core.Block; import org.hyperledger.besu.ethereum.core.BlockHeader; import java.util.Optional; @@ -57,11 +58,15 @@ public CompletableFuture pickNextStep() { final Optional firstHash = context.getBackwardChain().getFirstHashToAppend(); if (firstHash.isPresent()) { return executeSyncStep(firstHash.get()) - .whenComplete( - (result, throwable) -> { - if (throwable == null) { - context.getBackwardChain().removeFromHashToAppend(firstHash.get()); - } + .thenAccept( + result -> { + LOG.info("Backward sync target block is {}", result.toLogString()); + context.getBackwardChain().removeFromHashToAppend(firstHash.get()); + context + .getStatus() + .setSyncRange( + context.getProtocolContext().getBlockchain().getChainHeadBlockNumber(), + result.getHeader().getNumber()); }); } if (!context.isReady()) { @@ -73,7 +78,7 @@ public CompletableFuture pickNextStep() { context.getBackwardChain().getFirstAncestorHeader(); if (possibleFirstAncestorHeader.isEmpty()) { this.finished = true; - LOG.info("The Backward sync is done..."); + LOG.info("The Backward sync is done"); context.getBackwardChain().clear(); return CompletableFuture.completedFuture(null); } @@ -85,13 +90,16 @@ public CompletableFuture pickNextStep() { if (blockchain.getChainHead().getHeight() > firstAncestorHeader.getNumber()) { debugLambda( LOG, - "Backward reached below previous head {} : {}", + "Backward reached below current chain head {} : {}", () -> blockchain.getChainHead().toLogString(), firstAncestorHeader::toLogString); } if (finalBlockConfirmation.ancestorHeaderReached(firstAncestorHeader)) { - LOG.info("Backward sync reached ancestor header, starting Forward sync"); + debugLambda( + LOG, + "Backward sync reached ancestor header with {}, starting Forward sync", + firstAncestorHeader::toLogString); return executeForwardAsync(); } @@ -104,7 +112,7 @@ public CompletableFuture executeProcessKnownAncestors() { } @VisibleForTesting - public CompletableFuture executeSyncStep(final Hash hash) { + public CompletableFuture executeSyncStep(final Hash hash) { return new SyncStepStep(context, context.getBackwardChain()).executeAsync(hash); } diff --git a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/ForwardSyncStep.java b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/ForwardSyncStep.java index 71c0aa26cd8..00748dd5234 100644 --- a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/ForwardSyncStep.java +++ b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/ForwardSyncStep.java @@ -15,7 +15,6 @@ package org.hyperledger.besu.ethereum.eth.sync.backwardsync; import static org.hyperledger.besu.util.Slf4jLambdaHelper.debugLambda; -import static org.hyperledger.besu.util.Slf4jLambdaHelper.infoLambda; import org.hyperledger.besu.ethereum.core.Block; import org.hyperledger.besu.ethereum.core.BlockHeader; @@ -34,6 +33,8 @@ public class ForwardSyncStep { private static final Logger LOG = LoggerFactory.getLogger(ForwardSyncStep.class); + private static final long MILLIS_DELAY_BETWEEN_PROGRESS_LOG = 10_000L; + private static long lastLogAt = 0; private final BackwardSyncContext context; private final BackwardChain backwardChain; @@ -78,6 +79,7 @@ protected CompletableFuture> requestBodies(final List b return run.thenApply(AbstractPeerTask.PeerTaskResult::getResult) .thenApply( blocks -> { + LOG.debug("Got {} blocks from peers", blocks.size()); blocks.sort(Comparator.comparing(block -> block.getHeader().getNumber())); return blocks; }); @@ -86,8 +88,8 @@ protected CompletableFuture> requestBodies(final List b @VisibleForTesting protected Void saveBlocks(final List blocks) { if (blocks.isEmpty()) { - LOG.info("No blocks to save..."); context.halveBatchSize(); + LOG.debug("No blocks to save, reducing batch size to {}", context.getBatchSize()); return null; } @@ -97,21 +99,53 @@ protected Void saveBlocks(final List blocks) { .getProtocolContext() .getBlockchain() .getBlockByHash(block.getHeader().getParentHash()); + if (parent.isEmpty()) { context.halveBatchSize(); + debugLambda( + LOG, + "Parent block {} not found, while saving block {}, reducing batch size to {}", + block.getHeader().getParentHash()::toString, + block::toLogString, + context::getBatchSize); + logProgress(blocks.get(blocks.size() - 1).getHeader().getNumber()); return null; } else { context.saveBlock(block); } } - infoLambda( - LOG, - "Saved blocks {} -> {} (target: {})", - () -> blocks.get(0).getHeader().getNumber(), - () -> blocks.get(blocks.size() - 1).getHeader().getNumber(), - () -> - backwardChain.getPivot().orElse(blocks.get(blocks.size() - 1)).getHeader().getNumber()); + + logProgress(blocks.get(blocks.size() - 1).getHeader().getNumber()); + context.resetBatchSize(); return null; } + + private void logProgress(final long currImportedHeight) { + final long targetHeight = context.getStatus().getTargetChainHeight(); + final long initialHeight = context.getStatus().getInitialChainHeight(); + final long estimatedTotal = targetHeight - initialHeight; + final long imported = currImportedHeight - initialHeight; + + final float completedPercentage = 100.0f * imported / estimatedTotal; + + if (currImportedHeight < targetHeight) { + final long now = System.currentTimeMillis(); + if (now - lastLogAt > MILLIS_DELAY_BETWEEN_PROGRESS_LOG) { + LOG.info( + String.format( + "Backward sync phase 2 of 2, %.2f%% completed, imported %d blocks of at least %d. Peers: %d", + completedPercentage, + imported, + estimatedTotal, + context.getEthContext().getEthPeers().peerCount())); + lastLogAt = now; + } + } else { + LOG.info( + String.format( + "Backward sync phase 2 of 2 completed, imported a total of %d blocks. Peers: %d", + imported, context.getEthContext().getEthPeers().peerCount())); + } + } } diff --git a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/SyncStepStep.java b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/SyncStepStep.java index 88ddc18b2c6..f15a4d2b6d8 100644 --- a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/SyncStepStep.java +++ b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/SyncStepStep.java @@ -17,6 +17,7 @@ package org.hyperledger.besu.ethereum.eth.sync.backwardsync; +import static org.hyperledger.besu.util.Slf4jLambdaHelper.debugLambda; import static org.slf4j.LoggerFactory.getLogger; import org.hyperledger.besu.datatypes.Hash; @@ -41,13 +42,14 @@ public SyncStepStep(final BackwardSyncContext context, final BackwardChain backw this.backwardChain = backwardChain; } - public CompletableFuture executeAsync(final Hash hash) { + public CompletableFuture executeAsync(final Hash hash) { return CompletableFuture.supplyAsync(() -> hash) .thenCompose(this::requestBlock) .thenApply(this::saveBlock); } private CompletableFuture requestBlock(final Hash targetHash) { + debugLambda(LOG, "Fetching block by hash {} from peers", targetHash::toString); final RetryingGetBlockFromPeersTask getBlockTask = RetryingGetBlockFromPeersTask.create( context.getProtocolSchedule(), @@ -63,10 +65,9 @@ private CompletableFuture requestBlock(final Hash targetHash) { .thenApply(AbstractPeerTask.PeerTaskResult::getResult); } - private Void saveBlock(final Block block) { - LOG.debug( - "Appending block {}({})", block.getHeader().getNumber(), block.getHash().toHexString()); + private Block saveBlock(final Block block) { + debugLambda(LOG, "Appending fetched block {}", block::toLogString); backwardChain.appendTrustedBlock(block); - return null; + return block; } } From 65af5ef215c2995aedc3797f4b489a38c0161d7a Mon Sep 17 00:00:00 2001 From: Fabio Di Fabio Date: Fri, 11 Nov 2022 12:59:50 +0100 Subject: [PATCH 03/32] Add CHANGELOG entry Signed-off-by: Fabio Di Fabio --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index c6cf27dc902..b6e5aeeb9d7 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -8,6 +8,7 @@ - Explain and improve price validation for London and local transactions during block proposal selection [#4602](https://github.com/hyperledger/besu/pull/4602) - Support for ephemeral testnet Shandong. EIPs are still in flux, besu does not fully sync yet, and the network is subject to restarts. [#//FIXME](https://github.com/hyperledger/besu/pull///FIXME) - Improve performance of block processing by parallelizing some parts during the "commit" step [#4635](https://github.com/hyperledger/besu/pull/4635) +- Backward sync log UX improvements [#4655](https://github.com/hyperledger/besu/pull/4655) ### Bug Fixes From af2e8fc1e788e3425119847bc61ebdae50b74963 Mon Sep 17 00:00:00 2001 From: Fabio Di Fabio Date: Fri, 11 Nov 2022 14:28:31 +0100 Subject: [PATCH 04/32] Backward sync, use retry switching peer when fetching data from peers Signed-off-by: Fabio Di Fabio --- .../manager/task/GetBodiesFromPeerTask.java | 4 +- .../task/RetryingGetBlocksFromPeersTask.java | 112 ++++++++++++++++++ ...gGetHeadersEndingAtFromPeerByHashTask.java | 43 +++++-- .../sync/backwardsync/BackwardSyncStep.java | 7 +- .../backwardsync/BackwardsSyncAlgorithm.java | 18 ++- .../sync/backwardsync/ForwardSyncStep.java | 11 +- .../eth/sync/backwardsync/SyncStepStep.java | 2 +- .../backwardsync/BackwardSyncAlgSpec.java | 2 + .../backwardsync/BackwardSyncStepTest.java | 8 +- 9 files changed, 177 insertions(+), 30 deletions(-) create mode 100644 ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/manager/task/RetryingGetBlocksFromPeersTask.java diff --git a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/manager/task/GetBodiesFromPeerTask.java b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/manager/task/GetBodiesFromPeerTask.java index dd4f2792f54..9eb0f77f5d4 100644 --- a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/manager/task/GetBodiesFromPeerTask.java +++ b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/manager/task/GetBodiesFromPeerTask.java @@ -65,7 +65,7 @@ private GetBodiesFromPeerTask( headers.forEach( (header) -> { final BodyIdentifier bodyId = new BodyIdentifier(header); - bodyToHeaders.putIfAbsent(bodyId, new ArrayList<>()); + bodyToHeaders.putIfAbsent(bodyId, new ArrayList<>(headers.size())); bodyToHeaders.get(bodyId).add(header); }); } @@ -112,7 +112,7 @@ protected Optional> processResponse( return Optional.empty(); } - final List blocks = new ArrayList<>(); + final List blocks = new ArrayList<>(headers.size()); for (final BlockBody body : bodies) { final List headers = bodyToHeaders.get(new BodyIdentifier(body)); if (headers == null) { diff --git a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/manager/task/RetryingGetBlocksFromPeersTask.java b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/manager/task/RetryingGetBlocksFromPeersTask.java new file mode 100644 index 00000000000..85a663f6230 --- /dev/null +++ b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/manager/task/RetryingGetBlocksFromPeersTask.java @@ -0,0 +1,112 @@ +/* + * Copyright contributors to Hyperledger Besu + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + * + * SPDX-License-Identifier: Apache-2.0 + */ +package org.hyperledger.besu.ethereum.eth.manager.task; + +import static org.hyperledger.besu.util.Slf4jLambdaHelper.debugLambda; + +import org.hyperledger.besu.ethereum.core.Block; +import org.hyperledger.besu.ethereum.core.BlockHeader; +import org.hyperledger.besu.ethereum.eth.manager.EthContext; +import org.hyperledger.besu.ethereum.eth.manager.EthPeer; +import org.hyperledger.besu.ethereum.eth.manager.exceptions.IncompleteResultsException; +import org.hyperledger.besu.ethereum.eth.manager.task.AbstractPeerTask.PeerTaskResult; +import org.hyperledger.besu.ethereum.mainnet.ProtocolSchedule; +import org.hyperledger.besu.plugin.services.MetricsSystem; + +import java.util.List; +import java.util.Objects; +import java.util.concurrent.CompletableFuture; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class RetryingGetBlocksFromPeersTask + extends AbstractRetryingSwitchingPeerTask>> { + + private static final Logger LOG = LoggerFactory.getLogger(RetryingGetBlocksFromPeersTask.class); + + private final ProtocolSchedule protocolSchedule; + private final List headers; + + protected RetryingGetBlocksFromPeersTask( + final EthContext ethContext, + final ProtocolSchedule protocolSchedule, + final MetricsSystem metricsSystem, + final int maxRetries, + final List headers) { + super(ethContext, metricsSystem, Objects::isNull, maxRetries); + this.protocolSchedule = protocolSchedule; + this.headers = headers; + } + + public static RetryingGetBlocksFromPeersTask forHeaders( + final ProtocolSchedule protocolSchedule, + final EthContext ethContext, + final MetricsSystem metricsSystem, + final int maxRetries, + final List headers) { + return new RetryingGetBlocksFromPeersTask( + ethContext, protocolSchedule, metricsSystem, maxRetries, headers); + } + + @Override + protected CompletableFuture>> executeTaskOnCurrentPeer( + final EthPeer currentPeer) { + final GetBodiesFromPeerTask getBodiesTask = + GetBodiesFromPeerTask.forHeaders( + protocolSchedule, getEthContext(), headers, getMetricsSystem()); + getBodiesTask.assignPeer(currentPeer); + + return executeSubTask(getBodiesTask::run) + .thenApply( + peerResult -> { + debugLambda( + LOG, + "Got {} blocks {} from peer {}, attempt {}", + peerResult.getResult()::size, + peerResult.getPeer()::toString, + this::getRetryCount); + + if (peerResult.getResult().isEmpty()) { + currentPeer.recordUselessResponse("GetBodiesFromPeerTask"); + throw new IncompleteResultsException( + "No blocks returned by peer " + currentPeer.getShortNodeId()); + } + + result.complete(peerResult); + return peerResult; + }); + } + + @Override + protected boolean isRetryableError(final Throwable error) { + return super.isRetryableError(error) || error instanceof IncompleteResultsException; + } + + @Override + protected void handleTaskError(final Throwable error) { + if (getRetryCount() < getMaxRetries()) { + debugLambda( + LOG, + "Failed to get {} blocks from peer {}, attempt {}, retrying later", + headers::size, + this::getAssignedPeer, + this::getRetryCount); + } else { + LOG.warn("Failed to get {} blocks after {} retries", headers.size(), getRetryCount()); + } + super.handleTaskError(error); + } +} diff --git a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/manager/task/RetryingGetHeadersEndingAtFromPeerByHashTask.java b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/manager/task/RetryingGetHeadersEndingAtFromPeerByHashTask.java index bc758e6b177..2d1119c4aca 100644 --- a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/manager/task/RetryingGetHeadersEndingAtFromPeerByHashTask.java +++ b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/manager/task/RetryingGetHeadersEndingAtFromPeerByHashTask.java @@ -21,17 +21,21 @@ import org.hyperledger.besu.ethereum.core.BlockHeader; import org.hyperledger.besu.ethereum.eth.manager.EthContext; import org.hyperledger.besu.ethereum.eth.manager.EthPeer; +import org.hyperledger.besu.ethereum.eth.manager.exceptions.IncompleteResultsException; import org.hyperledger.besu.ethereum.mainnet.ProtocolSchedule; import org.hyperledger.besu.plugin.services.MetricsSystem; import java.util.List; -import java.util.Optional; import java.util.concurrent.CompletableFuture; import com.google.common.annotations.VisibleForTesting; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class RetryingGetHeadersEndingAtFromPeerByHashTask - extends AbstractRetryingPeerTask> { + extends AbstractRetryingSwitchingPeerTask> { + private static final Logger LOG = + LoggerFactory.getLogger(RetryingGetHeadersEndingAtFromPeerByHashTask.class); private final Hash referenceHash; private final ProtocolSchedule protocolSchedule; @@ -43,8 +47,9 @@ public class RetryingGetHeadersEndingAtFromPeerByHashTask final EthContext ethContext, final Hash referenceHash, final int count, - final MetricsSystem metricsSystem) { - super(ethContext, 4, List::isEmpty, metricsSystem); + final MetricsSystem metricsSystem, + final int maxRetries) { + super(ethContext, metricsSystem, List::isEmpty, maxRetries); this.protocolSchedule = protocolSchedule; this.count = count; checkNotNull(referenceHash); @@ -56,23 +61,43 @@ public static RetryingGetHeadersEndingAtFromPeerByHashTask endingAtHash( final EthContext ethContext, final Hash referenceHash, final int count, - final MetricsSystem metricsSystem) { + final MetricsSystem metricsSystem, + final int maxRetries) { return new RetryingGetHeadersEndingAtFromPeerByHashTask( - protocolSchedule, ethContext, referenceHash, count, metricsSystem); + protocolSchedule, ethContext, referenceHash, count, metricsSystem, maxRetries); } @Override - protected CompletableFuture> executePeerTask( - final Optional assignedPeer) { + protected CompletableFuture> executeTaskOnCurrentPeer( + final EthPeer currentPeer) { final AbstractGetHeadersFromPeerTask task = GetHeadersFromPeerByHashTask.endingAtHash( protocolSchedule, getEthContext(), referenceHash, count, getMetricsSystem()); - assignedPeer.ifPresent(task::assignPeer); + task.assignPeer(currentPeer); return executeSubTask(task::run) .thenApply( peerResult -> { + LOG.debug( + "Get {} block headers by hash {} from peer {} has result {}", + count, + referenceHash, + currentPeer, + peerResult.getResult()); + if (peerResult.getResult().isEmpty()) { + currentPeer.recordUselessResponse("GetHeadersFromPeerByHashTask"); + throw new IncompleteResultsException( + "No block headers for hash " + + referenceHash + + " returned by peer " + + currentPeer.getShortNodeId()); + } result.complete(peerResult.getResult()); return peerResult.getResult(); }); } + + @Override + protected boolean isRetryableError(final Throwable error) { + return super.isRetryableError(error) || error instanceof IncompleteResultsException; + } } diff --git a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncStep.java b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncStep.java index e22e4bbd698..357d596eff4 100644 --- a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncStep.java +++ b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncStep.java @@ -71,17 +71,14 @@ protected CompletableFuture> requestHeaders(final Hash hash) { context.getEthContext(), hash, batchSize, - context.getMetricsSystem()); + context.getMetricsSystem(), + context.getEthContext().getEthPeers().peerCount()); return context .getEthContext() .getScheduler() .scheduleSyncWorkerTask(retryingGetHeadersEndingAtFromPeerByHashTask::run) .thenApply( blockHeaders -> { - if (blockHeaders.isEmpty()) { - throw new BackwardSyncException( - "Did not receive a headers for hash " + hash.toHexString(), true); - } debugLambda( LOG, "Got headers {} -> {}", diff --git a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardsSyncAlgorithm.java b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardsSyncAlgorithm.java index 35c3f90ec21..f362c9ec3ad 100644 --- a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardsSyncAlgorithm.java +++ b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardsSyncAlgorithm.java @@ -14,7 +14,6 @@ * SPDX-License-Identifier: Apache-2.0 * */ - package org.hyperledger.besu.ethereum.eth.sync.backwardsync; import static org.hyperledger.besu.util.Slf4jLambdaHelper.debugLambda; @@ -24,10 +23,12 @@ import org.hyperledger.besu.ethereum.chain.MutableBlockchain; import org.hyperledger.besu.ethereum.core.Block; import org.hyperledger.besu.ethereum.core.BlockHeader; +import org.hyperledger.besu.ethereum.eth.manager.task.WaitForPeersTask; import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import com.google.common.annotations.VisibleForTesting; @@ -142,12 +143,17 @@ private void checkReadiness(final CountDownLatch latch, final long idTTD, final LOG.debug("Waiting for preconditions..."); final boolean await = latch.await(2, TimeUnit.MINUTES); if (await) { - LOG.debug("Preconditions meet..."); + LOG.debug("Preconditions meet, ensure at least one peer is connected"); + waitForPeers(1).get(); } } } catch (InterruptedException e) { Thread.currentThread().interrupt(); - throw new BackwardSyncException("Wait for TTD preconditions interrupted"); + throw new BackwardSyncException( + "Wait for TTD preconditions interrupted (" + e.getMessage() + ")"); + } catch (ExecutionException e) { + throw new BackwardSyncException( + "Error while waiting for at least one connected peer (" + e.getMessage() + ")", true); } finally { context.getSyncState().unsubscribeTTDReached(idTTD); context.getSyncState().unsubscribeInitialConditionReached(idIS); @@ -205,4 +211,10 @@ protected void runFinalizedSuccessionRule( blockchain.setFinalized(newFinalized); } + + private CompletableFuture waitForPeers(final int count) { + final WaitForPeersTask waitForPeersTask = + WaitForPeersTask.create(context.getEthContext(), count, context.getMetricsSystem()); + return waitForPeersTask.run(); + } } diff --git a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/ForwardSyncStep.java b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/ForwardSyncStep.java index 00748dd5234..d76c5b27f10 100644 --- a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/ForwardSyncStep.java +++ b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/ForwardSyncStep.java @@ -19,7 +19,7 @@ import org.hyperledger.besu.ethereum.core.Block; import org.hyperledger.besu.ethereum.core.BlockHeader; import org.hyperledger.besu.ethereum.eth.manager.task.AbstractPeerTask; -import org.hyperledger.besu.ethereum.eth.manager.task.GetBodiesFromPeerTask; +import org.hyperledger.besu.ethereum.eth.manager.task.RetryingGetBlocksFromPeersTask; import java.util.Comparator; import java.util.List; @@ -67,12 +67,13 @@ public CompletableFuture possibleRequestBodies(final List blo @VisibleForTesting protected CompletableFuture> requestBodies(final List blockHeaders) { - final GetBodiesFromPeerTask getBodiesFromPeerTask = - GetBodiesFromPeerTask.forHeaders( + final RetryingGetBlocksFromPeersTask getBodiesFromPeerTask = + RetryingGetBlocksFromPeersTask.forHeaders( context.getProtocolSchedule(), context.getEthContext(), - blockHeaders, - context.getMetricsSystem()); + context.getMetricsSystem(), + context.getEthContext().getEthPeers().peerCount(), + blockHeaders); final CompletableFuture>> run = getBodiesFromPeerTask.run(); diff --git a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/SyncStepStep.java b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/SyncStepStep.java index f15a4d2b6d8..2dc8b1d4351 100644 --- a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/SyncStepStep.java +++ b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/SyncStepStep.java @@ -55,7 +55,7 @@ private CompletableFuture requestBlock(final Hash targetHash) { context.getProtocolSchedule(), context.getEthContext(), context.getMetricsSystem(), - context.getEthContext().getEthPeers().getMaxPeers(), + context.getEthContext().getEthPeers().peerCount(), Optional.of(targetHash), UNUSED); return context diff --git a/ethereum/eth/src/test/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncAlgSpec.java b/ethereum/eth/src/test/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncAlgSpec.java index 3d4782bef2d..91a57bb8636 100644 --- a/ethereum/eth/src/test/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncAlgSpec.java +++ b/ethereum/eth/src/test/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncAlgSpec.java @@ -149,6 +149,7 @@ public void shouldAwokeWhenTTDReachedAndReady() throws Exception { when(context.getSyncState().subscribeTTDReached(any())).thenReturn(88L); when(context.getSyncState().subscribeCompletionReached(any())).thenReturn(99L); + when(context.getEthContext().getEthPeers().peerCount()).thenReturn(1); final CompletableFuture voidCompletableFuture = algorithm.waitForReady(); @@ -175,6 +176,7 @@ public void shouldAwokeWhenConditionReachedAndReady() throws Exception { when(context.getSyncState().subscribeTTDReached(any())).thenReturn(88L); when(context.getSyncState().subscribeCompletionReached(any())).thenReturn(99L); + when(context.getEthContext().getEthPeers().peerCount()).thenReturn(1); final CompletableFuture voidCompletableFuture = algorithm.waitForReady(); Thread.sleep(50); diff --git a/ethereum/eth/src/test/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncStepTest.java b/ethereum/eth/src/test/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncStepTest.java index ada5b5494f7..7f9e5b7c5b5 100644 --- a/ethereum/eth/src/test/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncStepTest.java +++ b/ethereum/eth/src/test/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncStepTest.java @@ -33,6 +33,7 @@ import org.hyperledger.besu.ethereum.eth.manager.EthProtocolManager; import org.hyperledger.besu.ethereum.eth.manager.EthProtocolManagerTestUtil; import org.hyperledger.besu.ethereum.eth.manager.RespondingEthPeer; +import org.hyperledger.besu.ethereum.eth.manager.exceptions.MaxRetriesReachedException; import org.hyperledger.besu.ethereum.mainnet.MainnetBlockHeaderFunctions; import org.hyperledger.besu.ethereum.mainnet.MainnetProtocolSchedule; import org.hyperledger.besu.ethereum.mainnet.ProtocolSchedule; @@ -181,7 +182,7 @@ public void shouldRequestHeaderBeforeCurrentHeight() throws Exception { } @Test - public void shouldThrowWhenResponseIsEmptyWhenRequestingHeader() throws Exception { + public void shouldThrowWhenResponseIsEmptyWhenRequestingHeader() { BackwardSyncStep step = new BackwardSyncStep(context, createBackwardChain(REMOTE_HEIGHT - 1)); final Block lookingForBlock = getBlockByNumber(REMOTE_HEIGHT - 2); @@ -191,10 +192,7 @@ public void shouldThrowWhenResponseIsEmptyWhenRequestingHeader() throws Exceptio step.requestHeaders(lookingForBlock.getHeader().getHash()); peer.respondWhileOtherThreadsWork(responder, () -> !future.isDone()); - assertThatThrownBy(future::get) - .getCause() - .isInstanceOf(BackwardSyncException.class) - .hasMessageContaining("Did not receive a headers for hash"); + assertThatThrownBy(future::get).cause().isInstanceOf(MaxRetriesReachedException.class); } @Test From cbff279c33c9bbaafc35ac8f2f455b9a961202c2 Mon Sep 17 00:00:00 2001 From: Fabio Di Fabio Date: Fri, 11 Nov 2022 14:30:14 +0100 Subject: [PATCH 05/32] Add CHANGELOG entry Signed-off-by: Fabio Di Fabio --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index b6e5aeeb9d7..626780a3c42 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -9,6 +9,7 @@ - Support for ephemeral testnet Shandong. EIPs are still in flux, besu does not fully sync yet, and the network is subject to restarts. [#//FIXME](https://github.com/hyperledger/besu/pull///FIXME) - Improve performance of block processing by parallelizing some parts during the "commit" step [#4635](https://github.com/hyperledger/besu/pull/4635) - Backward sync log UX improvements [#4655](https://github.com/hyperledger/besu/pull/4655) +- Backward sync: use retry switching peer when fetching data from peers [#4656](https://github.com/hyperledger/besu/pull/4656) ### Bug Fixes From 5003ad75c1c46b646038720960d2114b14dd4047 Mon Sep 17 00:00:00 2001 From: Fabio Di Fabio Date: Fri, 11 Nov 2022 16:25:52 +0100 Subject: [PATCH 06/32] Fix log in case of sync failures Signed-off-by: Fabio Di Fabio --- .../merge/blockcreation/MergeCoordinator.java | 6 +++++- .../internal/methods/engine/EngineNewPayload.java | 7 +++++-- .../eth/sync/backwardsync/BackwardSyncContext.java | 11 +++++++---- .../eth/sync/backwardsync/BackwardSyncStep.java | 2 +- .../eth/sync/backwardsync/ForwardSyncStep.java | 10 +++++++--- 5 files changed, 25 insertions(+), 11 deletions(-) diff --git a/consensus/merge/src/main/java/org/hyperledger/besu/consensus/merge/blockcreation/MergeCoordinator.java b/consensus/merge/src/main/java/org/hyperledger/besu/consensus/merge/blockcreation/MergeCoordinator.java index 438d294602c..d907ccdc7b4 100644 --- a/consensus/merge/src/main/java/org/hyperledger/besu/consensus/merge/blockcreation/MergeCoordinator.java +++ b/consensus/merge/src/main/java/org/hyperledger/besu/consensus/merge/blockcreation/MergeCoordinator.java @@ -390,7 +390,11 @@ public Optional getOrSyncHeaderByHash( } private Void logSyncException(final Hash blockHash, final Throwable exception) { - LOG.warn("Sync to block hash " + blockHash.toHexString() + " failed", exception.getMessage()); + debugLambda( + LOG, + "Sync to block hash {} failed, reason {}", + blockHash::toHexString, + exception::getMessage); return null; } diff --git a/ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/methods/engine/EngineNewPayload.java b/ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/methods/engine/EngineNewPayload.java index 7390b9d80f0..f0636c02d1b 100644 --- a/ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/methods/engine/EngineNewPayload.java +++ b/ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/methods/engine/EngineNewPayload.java @@ -180,7 +180,6 @@ public JsonRpcResponse syncResponse(final JsonRpcRequestContext requestContext) final var block = new Block(newBlockHeader, new BlockBody(transactions, Collections.emptyList())); - final String warningMessage = "Sync to block " + block.toLogString() + " failed"; if (mergeContext.get().isSyncing() || parentHeader.isEmpty()) { LOG.debug( @@ -192,7 +191,11 @@ public JsonRpcResponse syncResponse(final JsonRpcRequestContext requestContext) .appendNewPayloadToSync(block) .exceptionally( exception -> { - LOG.warn(warningMessage, exception.getMessage()); + debugLambda( + LOG, + "Sync to block {} failed, reason {}", + block::toLogString, + exception::getMessage); return null; }); return respondWith(reqId, blockParam, null, SYNCING); diff --git a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncContext.java b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncContext.java index fb941722733..f34cf5561bb 100644 --- a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncContext.java +++ b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncContext.java @@ -131,6 +131,7 @@ public synchronized CompletableFuture syncBackwardsUntil(final Hash newBlo backwardChain.addNewHash(newBlockHash); return maybeFuture.orElseGet( () -> { + LOG.info("Starting a new backward sync session"); Status status = new Status(prepareBackwardSyncFutureWithRetry()); this.currentBackwardSyncStatus.set(status); return status.currentFuture; @@ -147,7 +148,12 @@ public synchronized CompletableFuture syncBackwardsUntil(final Block newPi backwardChain.appendTrustedBlock(newPivot); return maybeFuture.orElseGet( () -> { + LOG.info("Starting a new backward sync session"); + LOG.info("Backward sync target block is {}", newPivot.toLogString()); Status status = new Status(prepareBackwardSyncFutureWithRetry()); + status.setSyncRange( + protocolContext.getBlockchain().getChainHeadBlockNumber(), + newPivot.getHeader().getNumber()); this.currentBackwardSyncStatus.set(status); return status.currentFuture; }); @@ -170,6 +176,7 @@ private CompletableFuture prepareBackwardSyncFutureWithRetry() { (unused, throwable) -> { this.currentBackwardSyncStatus.set(null); if (throwable != null) { + LOG.info("Current backward sync session failed, it will be restarted"); throw extractBackwardSyncException(throwable) .orElse(new BackwardSyncException(throwable)); } @@ -407,9 +414,5 @@ public long getTargetChainHeight() { public long getInitialChainHeight() { return initialChainHeight; } - - public long getBlockCount() { - return targetChainHeight - initialChainHeight; - } } } diff --git a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncStep.java b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncStep.java index e22e4bbd698..2b15940eacd 100644 --- a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncStep.java +++ b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncStep.java @@ -116,7 +116,7 @@ private void logProgress(final long currLowestDownloadedHeight) { final float completedPercentage = 100.0f * downloaded / estimatedTotal; - if (currLowestDownloadedHeight > initialHeight) { + if (completedPercentage < 100.0f) { final long now = System.currentTimeMillis(); if (now - lastLogAt > MILLIS_DELAY_BETWEEN_PROGRESS_LOG) { LOG.info( diff --git a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/ForwardSyncStep.java b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/ForwardSyncStep.java index 00748dd5234..18a83ecebe2 100644 --- a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/ForwardSyncStep.java +++ b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/ForwardSyncStep.java @@ -93,6 +93,9 @@ protected Void saveBlocks(final List blocks) { return null; } + long lastImportedHeight = + context.getProtocolContext().getBlockchain().getChainHeadBlockNumber(); + for (Block block : blocks) { final Optional parent = context @@ -108,14 +111,15 @@ protected Void saveBlocks(final List blocks) { block.getHeader().getParentHash()::toString, block::toLogString, context::getBatchSize); - logProgress(blocks.get(blocks.size() - 1).getHeader().getNumber()); + logProgress(lastImportedHeight); return null; } else { context.saveBlock(block); + lastImportedHeight = block.getHeader().getNumber(); } } - logProgress(blocks.get(blocks.size() - 1).getHeader().getNumber()); + logProgress(lastImportedHeight); context.resetBatchSize(); return null; @@ -129,7 +133,7 @@ private void logProgress(final long currImportedHeight) { final float completedPercentage = 100.0f * imported / estimatedTotal; - if (currImportedHeight < targetHeight) { + if (completedPercentage < 100.0f) { final long now = System.currentTimeMillis(); if (now - lastLogAt > MILLIS_DELAY_BETWEEN_PROGRESS_LOG) { LOG.info( From 8568ccd47422ac94a8f8954cd1f7971c8691bcc8 Mon Sep 17 00:00:00 2001 From: Fabio Di Fabio Date: Fri, 11 Nov 2022 16:46:49 +0100 Subject: [PATCH 07/32] Delete unneeded file Signed-off-by: Fabio Di Fabio --- ...GetHeadersEndingAtFromPeerByHashTask2.java | 78 ------------------- 1 file changed, 78 deletions(-) delete mode 100644 ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/manager/task/RetryingGetHeadersEndingAtFromPeerByHashTask2.java diff --git a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/manager/task/RetryingGetHeadersEndingAtFromPeerByHashTask2.java b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/manager/task/RetryingGetHeadersEndingAtFromPeerByHashTask2.java deleted file mode 100644 index ce4128146f4..00000000000 --- a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/manager/task/RetryingGetHeadersEndingAtFromPeerByHashTask2.java +++ /dev/null @@ -1,78 +0,0 @@ -/* - * Copyright Hyperledger Besu Contributors. - * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on - * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the - * specific language governing permissions and limitations under the License. - * - * SPDX-License-Identifier: Apache-2.0 - */ - -package org.hyperledger.besu.ethereum.eth.manager.task; - -import static com.google.common.base.Preconditions.checkNotNull; - -import org.hyperledger.besu.datatypes.Hash; -import org.hyperledger.besu.ethereum.core.BlockHeader; -import org.hyperledger.besu.ethereum.eth.manager.EthContext; -import org.hyperledger.besu.ethereum.eth.manager.EthPeer; -import org.hyperledger.besu.ethereum.mainnet.ProtocolSchedule; -import org.hyperledger.besu.plugin.services.MetricsSystem; - -import java.util.List; -import java.util.Optional; -import java.util.concurrent.CompletableFuture; - -import com.google.common.annotations.VisibleForTesting; - -public class RetryingGetHeadersEndingAtFromPeerByHashTask2 - extends AbstractRetryingPeerTask> { - - private final Hash referenceHash; - private final ProtocolSchedule protocolSchedule; - private final int count; - - @VisibleForTesting - RetryingGetHeadersEndingAtFromPeerByHashTask2( - final ProtocolSchedule protocolSchedule, - final EthContext ethContext, - final Hash referenceHash, - final int count, - final MetricsSystem metricsSystem) { - super(ethContext, 4, List::isEmpty, metricsSystem); - this.protocolSchedule = protocolSchedule; - this.count = count; - checkNotNull(referenceHash); - this.referenceHash = referenceHash; - } - - public static RetryingGetHeadersEndingAtFromPeerByHashTask2 endingAtHash( - final ProtocolSchedule protocolSchedule, - final EthContext ethContext, - final Hash referenceHash, - final int count, - final MetricsSystem metricsSystem) { - return new RetryingGetHeadersEndingAtFromPeerByHashTask2( - protocolSchedule, ethContext, referenceHash, count, metricsSystem); - } - - @Override - protected CompletableFuture> executePeerTask( - final Optional assignedPeer) { - final AbstractGetHeadersFromPeerTask task = - GetHeadersFromPeerByHashTask.endingAtHash( - protocolSchedule, getEthContext(), referenceHash, count, getMetricsSystem()); - assignedPeer.ifPresent(task::assignPeer); - return executeSubTask(task::run) - .thenApply( - peerResult -> { - result.complete(peerResult.getResult()); - return peerResult.getResult(); - }); - } -} From 5ebd45f247596462e605e2a44305271cfd8a7ae9 Mon Sep 17 00:00:00 2001 From: Fabio Di Fabio Date: Fri, 11 Nov 2022 16:48:34 +0100 Subject: [PATCH 08/32] Log transient error at debug level Signed-off-by: Fabio Di Fabio --- .../eth/manager/task/RetryingGetBlocksFromPeersTask.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/manager/task/RetryingGetBlocksFromPeersTask.java b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/manager/task/RetryingGetBlocksFromPeersTask.java index 85a663f6230..7d238586f2e 100644 --- a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/manager/task/RetryingGetBlocksFromPeersTask.java +++ b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/manager/task/RetryingGetBlocksFromPeersTask.java @@ -105,7 +105,7 @@ protected void handleTaskError(final Throwable error) { this::getAssignedPeer, this::getRetryCount); } else { - LOG.warn("Failed to get {} blocks after {} retries", headers.size(), getRetryCount()); + LOG.debug("Failed to get {} blocks after {} retries", headers.size(), getRetryCount()); } super.handleTaskError(error); } From ca71f6e0caaaf9e9e6960598cfa2fe7bc0e4ddb3 Mon Sep 17 00:00:00 2001 From: Fabio Di Fabio Date: Fri, 11 Nov 2022 12:58:23 +0100 Subject: [PATCH 09/32] Backward sync log UX improvements Signed-off-by: Fabio Di Fabio --- .../merge/blockcreation/MergeCoordinator.java | 2 +- .../task/RetryingGetBlockFromPeersTask.java | 7 +- .../backwardsync/BackwardSyncContext.java | 71 +++++++++++++------ .../sync/backwardsync/BackwardSyncStep.java | 42 ++++++++--- .../backwardsync/BackwardsSyncAlgorithm.java | 26 ++++--- .../sync/backwardsync/ForwardSyncStep.java | 52 +++++++++++--- .../eth/sync/backwardsync/SyncStepStep.java | 11 +-- 7 files changed, 155 insertions(+), 56 deletions(-) diff --git a/consensus/merge/src/main/java/org/hyperledger/besu/consensus/merge/blockcreation/MergeCoordinator.java b/consensus/merge/src/main/java/org/hyperledger/besu/consensus/merge/blockcreation/MergeCoordinator.java index 4ca248546fa..438d294602c 100644 --- a/consensus/merge/src/main/java/org/hyperledger/besu/consensus/merge/blockcreation/MergeCoordinator.java +++ b/consensus/merge/src/main/java/org/hyperledger/besu/consensus/merge/blockcreation/MergeCoordinator.java @@ -439,7 +439,7 @@ public ForkchoiceResult updateForkChoice( if (newHead.getNumber() < blockchain.getChainHeadBlockNumber() && isDescendantOf(newHead, blockchain.getChainHeadHeader())) { - LOG.info("Ignoring update to old head"); + debugLambda(LOG, "Ignoring update to old head {}", newHead::toLogString); return ForkchoiceResult.withIgnoreUpdateToOldHead(newHead); } diff --git a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/manager/task/RetryingGetBlockFromPeersTask.java b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/manager/task/RetryingGetBlockFromPeersTask.java index c1894189065..7d5062ec74d 100644 --- a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/manager/task/RetryingGetBlockFromPeersTask.java +++ b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/manager/task/RetryingGetBlockFromPeersTask.java @@ -102,8 +102,11 @@ protected void handleTaskError(final Throwable error) { this::getAssignedPeer, this::getRetryCount); } else { - LOG.warn( - "Failed to get block {} after {} retries", logBlockNumberMaybeHash(), getRetryCount()); + debugLambda( + LOG, + "Failed to get block {} after {} retries", + this::logBlockNumberMaybeHash, + this::getRetryCount); } super.handleTaskError(error); } diff --git a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncContext.java b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncContext.java index deb94063441..fb941722733 100644 --- a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncContext.java +++ b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncContext.java @@ -55,9 +55,7 @@ public class BackwardSyncContext { private final EthContext ethContext; private final MetricsSystem metricsSystem; private final SyncState syncState; - - private final AtomicReference> currentBackwardSyncFuture = - new AtomicReference<>(); + private final AtomicReference currentBackwardSyncStatus = new AtomicReference<>(); private final BackwardChain backwardChain; private int batchSize = BATCH_SIZE; private Optional maybeFinalized = Optional.empty(); @@ -105,8 +103,8 @@ public BackwardSyncContext( } public synchronized boolean isSyncing() { - return Optional.ofNullable(currentBackwardSyncFuture.get()) - .map(CompletableFuture::isDone) + return Optional.ofNullable(currentBackwardSyncStatus.get()) + .map(status -> status.currentFuture.isDone()) .orElse(Boolean.FALSE); } @@ -125,31 +123,33 @@ public synchronized void updateHeads(final Hash head, final Hash finalizedBlockH public synchronized CompletableFuture syncBackwardsUntil(final Hash newBlockHash) { Optional> maybeFuture = - Optional.ofNullable(this.currentBackwardSyncFuture.get()); + Optional.ofNullable(this.currentBackwardSyncStatus.get()) + .map(status -> status.currentFuture); if (isTrusted(newBlockHash)) { return maybeFuture.orElseGet(() -> CompletableFuture.completedFuture(null)); } backwardChain.addNewHash(newBlockHash); return maybeFuture.orElseGet( () -> { - CompletableFuture future = prepareBackwardSyncFutureWithRetry(); - this.currentBackwardSyncFuture.set(future); - return future; + Status status = new Status(prepareBackwardSyncFutureWithRetry()); + this.currentBackwardSyncStatus.set(status); + return status.currentFuture; }); } public synchronized CompletableFuture syncBackwardsUntil(final Block newPivot) { Optional> maybeFuture = - Optional.ofNullable(this.currentBackwardSyncFuture.get()); + Optional.ofNullable(this.currentBackwardSyncStatus.get()) + .map(status -> status.currentFuture); if (isTrusted(newPivot.getHash())) { return maybeFuture.orElseGet(() -> CompletableFuture.completedFuture(null)); } backwardChain.appendTrustedBlock(newPivot); return maybeFuture.orElseGet( () -> { - CompletableFuture future = prepareBackwardSyncFutureWithRetry(); - this.currentBackwardSyncFuture.set(future); - return future; + Status status = new Status(prepareBackwardSyncFutureWithRetry()); + this.currentBackwardSyncStatus.set(status); + return status.currentFuture; }); } @@ -168,7 +168,7 @@ private CompletableFuture prepareBackwardSyncFutureWithRetry() { return prepareBackwardSyncFutureWithRetry(maxRetries) .handle( (unused, throwable) -> { - this.currentBackwardSyncFuture.set(null); + this.currentBackwardSyncStatus.set(null); if (throwable != null) { throw extractBackwardSyncException(throwable) .orElse(new BackwardSyncException(throwable)); @@ -201,8 +201,8 @@ protected void processException(final Throwable throwable) { .ifPresentOrElse( backwardSyncException -> { if (backwardSyncException.shouldRestart()) { - LOG.info( - "Backward sync failed ({}). Current Peers: {}. Retrying in {} milliseconds...", + LOG.debug( + "Backward sync failed ({}). Current Peers: {}. Retrying in {} milliseconds", throwable.getMessage(), ethContext.getEthPeers().peerCount(), millisBetweenRetries); @@ -213,8 +213,8 @@ protected void processException(final Throwable throwable) { } }, () -> { - LOG.warn( - "Backward sync failed ({}). Current Peers: {}. Retrying in {} milliseconds...", + LOG.debug( + "Backward sync failed ({}). Current Peers: {}. Retrying in {} milliseconds", throwable.getMessage(), ethContext.getEthPeers().peerCount(), millisBetweenRetries); @@ -278,10 +278,6 @@ public boolean isReady() { && syncState.isInitialSyncPhaseDone(); } - public CompletableFuture stop() { - return currentBackwardSyncFuture.get(); - } - public void subscribeBadChainListener(final BadChainListener badChainListener) { badChainListeners.subscribe(badChainListener); } @@ -365,6 +361,10 @@ public Optional findMaybeFinalized() { .findFirst(); } + public Status getStatus() { + return currentBackwardSyncStatus.get(); + } + private void emitBadChainEvent(final Block badBlock) { final List badBlockDescendants = new ArrayList<>(); final List badBlockHeaderDescendants = new ArrayList<>(); @@ -385,4 +385,31 @@ private void emitBadChainEvent(final Block badBlock) { badChainListeners.forEach( listener -> listener.onBadChain(badBlock, badBlockDescendants, badBlockHeaderDescendants)); } + + static class Status { + private final CompletableFuture currentFuture; + private long targetChainHeight; + private long initialChainHeight; + + public Status(final CompletableFuture currentFuture) { + this.currentFuture = currentFuture; + } + + public void setSyncRange(final long initialHeight, final long targetHeight) { + initialChainHeight = initialHeight; + targetChainHeight = targetHeight; + } + + public long getTargetChainHeight() { + return targetChainHeight; + } + + public long getInitialChainHeight() { + return initialChainHeight; + } + + public long getBlockCount() { + return targetChainHeight - initialChainHeight; + } + } } diff --git a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncStep.java b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncStep.java index acdfc823c6d..e22e4bbd698 100644 --- a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncStep.java +++ b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncStep.java @@ -15,7 +15,6 @@ package org.hyperledger.besu.ethereum.eth.sync.backwardsync; import static org.hyperledger.besu.util.Slf4jLambdaHelper.debugLambda; -import static org.hyperledger.besu.util.Slf4jLambdaHelper.infoLambda; import org.hyperledger.besu.datatypes.Hash; import org.hyperledger.besu.ethereum.core.BlockHeader; @@ -31,6 +30,8 @@ public class BackwardSyncStep { private static final Logger LOG = LoggerFactory.getLogger(BackwardSyncStep.class); + private static final long MILLIS_DELAY_BETWEEN_PROGRESS_LOG = 10_000L; + private static long lastLogAt = 0; private final BackwardSyncContext context; private final BackwardChain backwardChain; @@ -61,7 +62,7 @@ protected Hash possibleRestoreOldNodes(final BlockHeader firstAncestor) { @VisibleForTesting protected CompletableFuture> requestHeaders(final Hash hash) { final int batchSize = context.getBatchSize(); - debugLambda(LOG, "Requesting header for hash {}", hash::toHexString); + LOG.debug("Requesting headers for hash {}, with batch size {}", hash, batchSize); final RetryingGetHeadersEndingAtFromPeerByHashTask retryingGetHeadersEndingAtFromPeerByHashTask = @@ -101,12 +102,37 @@ protected Void saveHeaders(final List blockHeaders) { for (BlockHeader blockHeader : blockHeaders) { saveHeader(blockHeader); } - infoLambda( - LOG, - "Saved headers {} -> {} (head: {})", - () -> blockHeaders.get(0).getNumber(), - () -> blockHeaders.get(blockHeaders.size() - 1).getNumber(), - () -> context.getProtocolContext().getBlockchain().getChainHead().getHeight()); + + logProgress(blockHeaders.get(blockHeaders.size() - 1).getNumber()); + return null; } + + private void logProgress(final long currLowestDownloadedHeight) { + final long targetHeight = context.getStatus().getTargetChainHeight(); + final long initialHeight = context.getStatus().getInitialChainHeight(); + final long estimatedTotal = targetHeight - initialHeight; + final long downloaded = targetHeight - currLowestDownloadedHeight; + + final float completedPercentage = 100.0f * downloaded / estimatedTotal; + + if (currLowestDownloadedHeight > initialHeight) { + final long now = System.currentTimeMillis(); + if (now - lastLogAt > MILLIS_DELAY_BETWEEN_PROGRESS_LOG) { + LOG.info( + String.format( + "Backward sync phase 1 of 2, %.2f%% completed, downloaded %d headers of at least %d. Peers: %d", + completedPercentage, + downloaded, + estimatedTotal, + context.getEthContext().getEthPeers().peerCount())); + lastLogAt = now; + } + } else { + LOG.info( + String.format( + "Backward sync phase 1 of 2 completed, downloaded a total of %d headers. Peers: %d", + downloaded, context.getEthContext().getEthPeers().peerCount())); + } + } } diff --git a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardsSyncAlgorithm.java b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardsSyncAlgorithm.java index 8ff1402f282..35c3f90ec21 100644 --- a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardsSyncAlgorithm.java +++ b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardsSyncAlgorithm.java @@ -22,6 +22,7 @@ import org.hyperledger.besu.datatypes.Hash; import org.hyperledger.besu.ethereum.chain.MutableBlockchain; +import org.hyperledger.besu.ethereum.core.Block; import org.hyperledger.besu.ethereum.core.BlockHeader; import java.util.Optional; @@ -57,11 +58,15 @@ public CompletableFuture pickNextStep() { final Optional firstHash = context.getBackwardChain().getFirstHashToAppend(); if (firstHash.isPresent()) { return executeSyncStep(firstHash.get()) - .whenComplete( - (result, throwable) -> { - if (throwable == null) { - context.getBackwardChain().removeFromHashToAppend(firstHash.get()); - } + .thenAccept( + result -> { + LOG.info("Backward sync target block is {}", result.toLogString()); + context.getBackwardChain().removeFromHashToAppend(firstHash.get()); + context + .getStatus() + .setSyncRange( + context.getProtocolContext().getBlockchain().getChainHeadBlockNumber(), + result.getHeader().getNumber()); }); } if (!context.isReady()) { @@ -73,7 +78,7 @@ public CompletableFuture pickNextStep() { context.getBackwardChain().getFirstAncestorHeader(); if (possibleFirstAncestorHeader.isEmpty()) { this.finished = true; - LOG.info("The Backward sync is done..."); + LOG.info("The Backward sync is done"); context.getBackwardChain().clear(); return CompletableFuture.completedFuture(null); } @@ -85,13 +90,16 @@ public CompletableFuture pickNextStep() { if (blockchain.getChainHead().getHeight() > firstAncestorHeader.getNumber()) { debugLambda( LOG, - "Backward reached below previous head {} : {}", + "Backward reached below current chain head {} : {}", () -> blockchain.getChainHead().toLogString(), firstAncestorHeader::toLogString); } if (finalBlockConfirmation.ancestorHeaderReached(firstAncestorHeader)) { - LOG.info("Backward sync reached ancestor header, starting Forward sync"); + debugLambda( + LOG, + "Backward sync reached ancestor header with {}, starting Forward sync", + firstAncestorHeader::toLogString); return executeForwardAsync(); } @@ -104,7 +112,7 @@ public CompletableFuture executeProcessKnownAncestors() { } @VisibleForTesting - public CompletableFuture executeSyncStep(final Hash hash) { + public CompletableFuture executeSyncStep(final Hash hash) { return new SyncStepStep(context, context.getBackwardChain()).executeAsync(hash); } diff --git a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/ForwardSyncStep.java b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/ForwardSyncStep.java index 71c0aa26cd8..00748dd5234 100644 --- a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/ForwardSyncStep.java +++ b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/ForwardSyncStep.java @@ -15,7 +15,6 @@ package org.hyperledger.besu.ethereum.eth.sync.backwardsync; import static org.hyperledger.besu.util.Slf4jLambdaHelper.debugLambda; -import static org.hyperledger.besu.util.Slf4jLambdaHelper.infoLambda; import org.hyperledger.besu.ethereum.core.Block; import org.hyperledger.besu.ethereum.core.BlockHeader; @@ -34,6 +33,8 @@ public class ForwardSyncStep { private static final Logger LOG = LoggerFactory.getLogger(ForwardSyncStep.class); + private static final long MILLIS_DELAY_BETWEEN_PROGRESS_LOG = 10_000L; + private static long lastLogAt = 0; private final BackwardSyncContext context; private final BackwardChain backwardChain; @@ -78,6 +79,7 @@ protected CompletableFuture> requestBodies(final List b return run.thenApply(AbstractPeerTask.PeerTaskResult::getResult) .thenApply( blocks -> { + LOG.debug("Got {} blocks from peers", blocks.size()); blocks.sort(Comparator.comparing(block -> block.getHeader().getNumber())); return blocks; }); @@ -86,8 +88,8 @@ protected CompletableFuture> requestBodies(final List b @VisibleForTesting protected Void saveBlocks(final List blocks) { if (blocks.isEmpty()) { - LOG.info("No blocks to save..."); context.halveBatchSize(); + LOG.debug("No blocks to save, reducing batch size to {}", context.getBatchSize()); return null; } @@ -97,21 +99,53 @@ protected Void saveBlocks(final List blocks) { .getProtocolContext() .getBlockchain() .getBlockByHash(block.getHeader().getParentHash()); + if (parent.isEmpty()) { context.halveBatchSize(); + debugLambda( + LOG, + "Parent block {} not found, while saving block {}, reducing batch size to {}", + block.getHeader().getParentHash()::toString, + block::toLogString, + context::getBatchSize); + logProgress(blocks.get(blocks.size() - 1).getHeader().getNumber()); return null; } else { context.saveBlock(block); } } - infoLambda( - LOG, - "Saved blocks {} -> {} (target: {})", - () -> blocks.get(0).getHeader().getNumber(), - () -> blocks.get(blocks.size() - 1).getHeader().getNumber(), - () -> - backwardChain.getPivot().orElse(blocks.get(blocks.size() - 1)).getHeader().getNumber()); + + logProgress(blocks.get(blocks.size() - 1).getHeader().getNumber()); + context.resetBatchSize(); return null; } + + private void logProgress(final long currImportedHeight) { + final long targetHeight = context.getStatus().getTargetChainHeight(); + final long initialHeight = context.getStatus().getInitialChainHeight(); + final long estimatedTotal = targetHeight - initialHeight; + final long imported = currImportedHeight - initialHeight; + + final float completedPercentage = 100.0f * imported / estimatedTotal; + + if (currImportedHeight < targetHeight) { + final long now = System.currentTimeMillis(); + if (now - lastLogAt > MILLIS_DELAY_BETWEEN_PROGRESS_LOG) { + LOG.info( + String.format( + "Backward sync phase 2 of 2, %.2f%% completed, imported %d blocks of at least %d. Peers: %d", + completedPercentage, + imported, + estimatedTotal, + context.getEthContext().getEthPeers().peerCount())); + lastLogAt = now; + } + } else { + LOG.info( + String.format( + "Backward sync phase 2 of 2 completed, imported a total of %d blocks. Peers: %d", + imported, context.getEthContext().getEthPeers().peerCount())); + } + } } diff --git a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/SyncStepStep.java b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/SyncStepStep.java index 88ddc18b2c6..f15a4d2b6d8 100644 --- a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/SyncStepStep.java +++ b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/SyncStepStep.java @@ -17,6 +17,7 @@ package org.hyperledger.besu.ethereum.eth.sync.backwardsync; +import static org.hyperledger.besu.util.Slf4jLambdaHelper.debugLambda; import static org.slf4j.LoggerFactory.getLogger; import org.hyperledger.besu.datatypes.Hash; @@ -41,13 +42,14 @@ public SyncStepStep(final BackwardSyncContext context, final BackwardChain backw this.backwardChain = backwardChain; } - public CompletableFuture executeAsync(final Hash hash) { + public CompletableFuture executeAsync(final Hash hash) { return CompletableFuture.supplyAsync(() -> hash) .thenCompose(this::requestBlock) .thenApply(this::saveBlock); } private CompletableFuture requestBlock(final Hash targetHash) { + debugLambda(LOG, "Fetching block by hash {} from peers", targetHash::toString); final RetryingGetBlockFromPeersTask getBlockTask = RetryingGetBlockFromPeersTask.create( context.getProtocolSchedule(), @@ -63,10 +65,9 @@ private CompletableFuture requestBlock(final Hash targetHash) { .thenApply(AbstractPeerTask.PeerTaskResult::getResult); } - private Void saveBlock(final Block block) { - LOG.debug( - "Appending block {}({})", block.getHeader().getNumber(), block.getHash().toHexString()); + private Block saveBlock(final Block block) { + debugLambda(LOG, "Appending fetched block {}", block::toLogString); backwardChain.appendTrustedBlock(block); - return null; + return block; } } From 00a54686fbcc6a98ebc7598ec472cf7aaa5acb5d Mon Sep 17 00:00:00 2001 From: Fabio Di Fabio Date: Fri, 11 Nov 2022 12:59:50 +0100 Subject: [PATCH 10/32] Add CHANGELOG entry Signed-off-by: Fabio Di Fabio --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index c0e56539154..815136aa0b3 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -9,6 +9,7 @@ - Support for ephemeral testnet Shandong. EIPs are still in flux, besu does not fully sync yet, and the network is subject to restarts. [#//FIXME](https://github.com/hyperledger/besu/pull///FIXME) - Improve performance of block processing by parallelizing some parts during the "commit" step [#4635](https://github.com/hyperledger/besu/pull/4635) - Upgrade RocksDB version from 7.6.0 to 7.7.3 +- Backward sync log UX improvements [#4655](https://github.com/hyperledger/besu/pull/4655) ### Bug Fixes From 0ad7d31bd6438db98092d32e3b70ca3f3c759bf3 Mon Sep 17 00:00:00 2001 From: Fabio Di Fabio Date: Fri, 11 Nov 2022 16:25:52 +0100 Subject: [PATCH 11/32] Fix log in case of sync failures Signed-off-by: Fabio Di Fabio --- .../merge/blockcreation/MergeCoordinator.java | 6 +++++- .../internal/methods/engine/EngineNewPayload.java | 7 +++++-- .../eth/sync/backwardsync/BackwardSyncContext.java | 11 +++++++---- .../eth/sync/backwardsync/BackwardSyncStep.java | 2 +- .../eth/sync/backwardsync/ForwardSyncStep.java | 10 +++++++--- 5 files changed, 25 insertions(+), 11 deletions(-) diff --git a/consensus/merge/src/main/java/org/hyperledger/besu/consensus/merge/blockcreation/MergeCoordinator.java b/consensus/merge/src/main/java/org/hyperledger/besu/consensus/merge/blockcreation/MergeCoordinator.java index 438d294602c..d907ccdc7b4 100644 --- a/consensus/merge/src/main/java/org/hyperledger/besu/consensus/merge/blockcreation/MergeCoordinator.java +++ b/consensus/merge/src/main/java/org/hyperledger/besu/consensus/merge/blockcreation/MergeCoordinator.java @@ -390,7 +390,11 @@ public Optional getOrSyncHeaderByHash( } private Void logSyncException(final Hash blockHash, final Throwable exception) { - LOG.warn("Sync to block hash " + blockHash.toHexString() + " failed", exception.getMessage()); + debugLambda( + LOG, + "Sync to block hash {} failed, reason {}", + blockHash::toHexString, + exception::getMessage); return null; } diff --git a/ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/methods/engine/EngineNewPayload.java b/ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/methods/engine/EngineNewPayload.java index 7390b9d80f0..f0636c02d1b 100644 --- a/ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/methods/engine/EngineNewPayload.java +++ b/ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/methods/engine/EngineNewPayload.java @@ -180,7 +180,6 @@ public JsonRpcResponse syncResponse(final JsonRpcRequestContext requestContext) final var block = new Block(newBlockHeader, new BlockBody(transactions, Collections.emptyList())); - final String warningMessage = "Sync to block " + block.toLogString() + " failed"; if (mergeContext.get().isSyncing() || parentHeader.isEmpty()) { LOG.debug( @@ -192,7 +191,11 @@ public JsonRpcResponse syncResponse(final JsonRpcRequestContext requestContext) .appendNewPayloadToSync(block) .exceptionally( exception -> { - LOG.warn(warningMessage, exception.getMessage()); + debugLambda( + LOG, + "Sync to block {} failed, reason {}", + block::toLogString, + exception::getMessage); return null; }); return respondWith(reqId, blockParam, null, SYNCING); diff --git a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncContext.java b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncContext.java index fb941722733..f34cf5561bb 100644 --- a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncContext.java +++ b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncContext.java @@ -131,6 +131,7 @@ public synchronized CompletableFuture syncBackwardsUntil(final Hash newBlo backwardChain.addNewHash(newBlockHash); return maybeFuture.orElseGet( () -> { + LOG.info("Starting a new backward sync session"); Status status = new Status(prepareBackwardSyncFutureWithRetry()); this.currentBackwardSyncStatus.set(status); return status.currentFuture; @@ -147,7 +148,12 @@ public synchronized CompletableFuture syncBackwardsUntil(final Block newPi backwardChain.appendTrustedBlock(newPivot); return maybeFuture.orElseGet( () -> { + LOG.info("Starting a new backward sync session"); + LOG.info("Backward sync target block is {}", newPivot.toLogString()); Status status = new Status(prepareBackwardSyncFutureWithRetry()); + status.setSyncRange( + protocolContext.getBlockchain().getChainHeadBlockNumber(), + newPivot.getHeader().getNumber()); this.currentBackwardSyncStatus.set(status); return status.currentFuture; }); @@ -170,6 +176,7 @@ private CompletableFuture prepareBackwardSyncFutureWithRetry() { (unused, throwable) -> { this.currentBackwardSyncStatus.set(null); if (throwable != null) { + LOG.info("Current backward sync session failed, it will be restarted"); throw extractBackwardSyncException(throwable) .orElse(new BackwardSyncException(throwable)); } @@ -407,9 +414,5 @@ public long getTargetChainHeight() { public long getInitialChainHeight() { return initialChainHeight; } - - public long getBlockCount() { - return targetChainHeight - initialChainHeight; - } } } diff --git a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncStep.java b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncStep.java index e22e4bbd698..2b15940eacd 100644 --- a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncStep.java +++ b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncStep.java @@ -116,7 +116,7 @@ private void logProgress(final long currLowestDownloadedHeight) { final float completedPercentage = 100.0f * downloaded / estimatedTotal; - if (currLowestDownloadedHeight > initialHeight) { + if (completedPercentage < 100.0f) { final long now = System.currentTimeMillis(); if (now - lastLogAt > MILLIS_DELAY_BETWEEN_PROGRESS_LOG) { LOG.info( diff --git a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/ForwardSyncStep.java b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/ForwardSyncStep.java index 00748dd5234..18a83ecebe2 100644 --- a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/ForwardSyncStep.java +++ b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/ForwardSyncStep.java @@ -93,6 +93,9 @@ protected Void saveBlocks(final List blocks) { return null; } + long lastImportedHeight = + context.getProtocolContext().getBlockchain().getChainHeadBlockNumber(); + for (Block block : blocks) { final Optional parent = context @@ -108,14 +111,15 @@ protected Void saveBlocks(final List blocks) { block.getHeader().getParentHash()::toString, block::toLogString, context::getBatchSize); - logProgress(blocks.get(blocks.size() - 1).getHeader().getNumber()); + logProgress(lastImportedHeight); return null; } else { context.saveBlock(block); + lastImportedHeight = block.getHeader().getNumber(); } } - logProgress(blocks.get(blocks.size() - 1).getHeader().getNumber()); + logProgress(lastImportedHeight); context.resetBatchSize(); return null; @@ -129,7 +133,7 @@ private void logProgress(final long currImportedHeight) { final float completedPercentage = 100.0f * imported / estimatedTotal; - if (currImportedHeight < targetHeight) { + if (completedPercentage < 100.0f) { final long now = System.currentTimeMillis(); if (now - lastLogAt > MILLIS_DELAY_BETWEEN_PROGRESS_LOG) { LOG.info( From 8f783896d61dc4c44e90f9d5baed1f2979b02b43 Mon Sep 17 00:00:00 2001 From: Fabio Di Fabio Date: Fri, 11 Nov 2022 12:58:23 +0100 Subject: [PATCH 12/32] Backward sync log UX improvements Signed-off-by: Fabio Di Fabio --- .../merge/blockcreation/MergeCoordinator.java | 2 +- .../task/RetryingGetBlockFromPeersTask.java | 7 +- .../backwardsync/BackwardSyncContext.java | 71 +++++++++++++------ .../sync/backwardsync/BackwardSyncStep.java | 42 ++++++++--- .../backwardsync/BackwardsSyncAlgorithm.java | 26 ++++--- .../sync/backwardsync/ForwardSyncStep.java | 52 +++++++++++--- .../eth/sync/backwardsync/SyncStepStep.java | 11 +-- 7 files changed, 155 insertions(+), 56 deletions(-) diff --git a/consensus/merge/src/main/java/org/hyperledger/besu/consensus/merge/blockcreation/MergeCoordinator.java b/consensus/merge/src/main/java/org/hyperledger/besu/consensus/merge/blockcreation/MergeCoordinator.java index 4ca248546fa..438d294602c 100644 --- a/consensus/merge/src/main/java/org/hyperledger/besu/consensus/merge/blockcreation/MergeCoordinator.java +++ b/consensus/merge/src/main/java/org/hyperledger/besu/consensus/merge/blockcreation/MergeCoordinator.java @@ -439,7 +439,7 @@ public ForkchoiceResult updateForkChoice( if (newHead.getNumber() < blockchain.getChainHeadBlockNumber() && isDescendantOf(newHead, blockchain.getChainHeadHeader())) { - LOG.info("Ignoring update to old head"); + debugLambda(LOG, "Ignoring update to old head {}", newHead::toLogString); return ForkchoiceResult.withIgnoreUpdateToOldHead(newHead); } diff --git a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/manager/task/RetryingGetBlockFromPeersTask.java b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/manager/task/RetryingGetBlockFromPeersTask.java index c1894189065..7d5062ec74d 100644 --- a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/manager/task/RetryingGetBlockFromPeersTask.java +++ b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/manager/task/RetryingGetBlockFromPeersTask.java @@ -102,8 +102,11 @@ protected void handleTaskError(final Throwable error) { this::getAssignedPeer, this::getRetryCount); } else { - LOG.warn( - "Failed to get block {} after {} retries", logBlockNumberMaybeHash(), getRetryCount()); + debugLambda( + LOG, + "Failed to get block {} after {} retries", + this::logBlockNumberMaybeHash, + this::getRetryCount); } super.handleTaskError(error); } diff --git a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncContext.java b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncContext.java index deb94063441..fb941722733 100644 --- a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncContext.java +++ b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncContext.java @@ -55,9 +55,7 @@ public class BackwardSyncContext { private final EthContext ethContext; private final MetricsSystem metricsSystem; private final SyncState syncState; - - private final AtomicReference> currentBackwardSyncFuture = - new AtomicReference<>(); + private final AtomicReference currentBackwardSyncStatus = new AtomicReference<>(); private final BackwardChain backwardChain; private int batchSize = BATCH_SIZE; private Optional maybeFinalized = Optional.empty(); @@ -105,8 +103,8 @@ public BackwardSyncContext( } public synchronized boolean isSyncing() { - return Optional.ofNullable(currentBackwardSyncFuture.get()) - .map(CompletableFuture::isDone) + return Optional.ofNullable(currentBackwardSyncStatus.get()) + .map(status -> status.currentFuture.isDone()) .orElse(Boolean.FALSE); } @@ -125,31 +123,33 @@ public synchronized void updateHeads(final Hash head, final Hash finalizedBlockH public synchronized CompletableFuture syncBackwardsUntil(final Hash newBlockHash) { Optional> maybeFuture = - Optional.ofNullable(this.currentBackwardSyncFuture.get()); + Optional.ofNullable(this.currentBackwardSyncStatus.get()) + .map(status -> status.currentFuture); if (isTrusted(newBlockHash)) { return maybeFuture.orElseGet(() -> CompletableFuture.completedFuture(null)); } backwardChain.addNewHash(newBlockHash); return maybeFuture.orElseGet( () -> { - CompletableFuture future = prepareBackwardSyncFutureWithRetry(); - this.currentBackwardSyncFuture.set(future); - return future; + Status status = new Status(prepareBackwardSyncFutureWithRetry()); + this.currentBackwardSyncStatus.set(status); + return status.currentFuture; }); } public synchronized CompletableFuture syncBackwardsUntil(final Block newPivot) { Optional> maybeFuture = - Optional.ofNullable(this.currentBackwardSyncFuture.get()); + Optional.ofNullable(this.currentBackwardSyncStatus.get()) + .map(status -> status.currentFuture); if (isTrusted(newPivot.getHash())) { return maybeFuture.orElseGet(() -> CompletableFuture.completedFuture(null)); } backwardChain.appendTrustedBlock(newPivot); return maybeFuture.orElseGet( () -> { - CompletableFuture future = prepareBackwardSyncFutureWithRetry(); - this.currentBackwardSyncFuture.set(future); - return future; + Status status = new Status(prepareBackwardSyncFutureWithRetry()); + this.currentBackwardSyncStatus.set(status); + return status.currentFuture; }); } @@ -168,7 +168,7 @@ private CompletableFuture prepareBackwardSyncFutureWithRetry() { return prepareBackwardSyncFutureWithRetry(maxRetries) .handle( (unused, throwable) -> { - this.currentBackwardSyncFuture.set(null); + this.currentBackwardSyncStatus.set(null); if (throwable != null) { throw extractBackwardSyncException(throwable) .orElse(new BackwardSyncException(throwable)); @@ -201,8 +201,8 @@ protected void processException(final Throwable throwable) { .ifPresentOrElse( backwardSyncException -> { if (backwardSyncException.shouldRestart()) { - LOG.info( - "Backward sync failed ({}). Current Peers: {}. Retrying in {} milliseconds...", + LOG.debug( + "Backward sync failed ({}). Current Peers: {}. Retrying in {} milliseconds", throwable.getMessage(), ethContext.getEthPeers().peerCount(), millisBetweenRetries); @@ -213,8 +213,8 @@ protected void processException(final Throwable throwable) { } }, () -> { - LOG.warn( - "Backward sync failed ({}). Current Peers: {}. Retrying in {} milliseconds...", + LOG.debug( + "Backward sync failed ({}). Current Peers: {}. Retrying in {} milliseconds", throwable.getMessage(), ethContext.getEthPeers().peerCount(), millisBetweenRetries); @@ -278,10 +278,6 @@ public boolean isReady() { && syncState.isInitialSyncPhaseDone(); } - public CompletableFuture stop() { - return currentBackwardSyncFuture.get(); - } - public void subscribeBadChainListener(final BadChainListener badChainListener) { badChainListeners.subscribe(badChainListener); } @@ -365,6 +361,10 @@ public Optional findMaybeFinalized() { .findFirst(); } + public Status getStatus() { + return currentBackwardSyncStatus.get(); + } + private void emitBadChainEvent(final Block badBlock) { final List badBlockDescendants = new ArrayList<>(); final List badBlockHeaderDescendants = new ArrayList<>(); @@ -385,4 +385,31 @@ private void emitBadChainEvent(final Block badBlock) { badChainListeners.forEach( listener -> listener.onBadChain(badBlock, badBlockDescendants, badBlockHeaderDescendants)); } + + static class Status { + private final CompletableFuture currentFuture; + private long targetChainHeight; + private long initialChainHeight; + + public Status(final CompletableFuture currentFuture) { + this.currentFuture = currentFuture; + } + + public void setSyncRange(final long initialHeight, final long targetHeight) { + initialChainHeight = initialHeight; + targetChainHeight = targetHeight; + } + + public long getTargetChainHeight() { + return targetChainHeight; + } + + public long getInitialChainHeight() { + return initialChainHeight; + } + + public long getBlockCount() { + return targetChainHeight - initialChainHeight; + } + } } diff --git a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncStep.java b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncStep.java index acdfc823c6d..e22e4bbd698 100644 --- a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncStep.java +++ b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncStep.java @@ -15,7 +15,6 @@ package org.hyperledger.besu.ethereum.eth.sync.backwardsync; import static org.hyperledger.besu.util.Slf4jLambdaHelper.debugLambda; -import static org.hyperledger.besu.util.Slf4jLambdaHelper.infoLambda; import org.hyperledger.besu.datatypes.Hash; import org.hyperledger.besu.ethereum.core.BlockHeader; @@ -31,6 +30,8 @@ public class BackwardSyncStep { private static final Logger LOG = LoggerFactory.getLogger(BackwardSyncStep.class); + private static final long MILLIS_DELAY_BETWEEN_PROGRESS_LOG = 10_000L; + private static long lastLogAt = 0; private final BackwardSyncContext context; private final BackwardChain backwardChain; @@ -61,7 +62,7 @@ protected Hash possibleRestoreOldNodes(final BlockHeader firstAncestor) { @VisibleForTesting protected CompletableFuture> requestHeaders(final Hash hash) { final int batchSize = context.getBatchSize(); - debugLambda(LOG, "Requesting header for hash {}", hash::toHexString); + LOG.debug("Requesting headers for hash {}, with batch size {}", hash, batchSize); final RetryingGetHeadersEndingAtFromPeerByHashTask retryingGetHeadersEndingAtFromPeerByHashTask = @@ -101,12 +102,37 @@ protected Void saveHeaders(final List blockHeaders) { for (BlockHeader blockHeader : blockHeaders) { saveHeader(blockHeader); } - infoLambda( - LOG, - "Saved headers {} -> {} (head: {})", - () -> blockHeaders.get(0).getNumber(), - () -> blockHeaders.get(blockHeaders.size() - 1).getNumber(), - () -> context.getProtocolContext().getBlockchain().getChainHead().getHeight()); + + logProgress(blockHeaders.get(blockHeaders.size() - 1).getNumber()); + return null; } + + private void logProgress(final long currLowestDownloadedHeight) { + final long targetHeight = context.getStatus().getTargetChainHeight(); + final long initialHeight = context.getStatus().getInitialChainHeight(); + final long estimatedTotal = targetHeight - initialHeight; + final long downloaded = targetHeight - currLowestDownloadedHeight; + + final float completedPercentage = 100.0f * downloaded / estimatedTotal; + + if (currLowestDownloadedHeight > initialHeight) { + final long now = System.currentTimeMillis(); + if (now - lastLogAt > MILLIS_DELAY_BETWEEN_PROGRESS_LOG) { + LOG.info( + String.format( + "Backward sync phase 1 of 2, %.2f%% completed, downloaded %d headers of at least %d. Peers: %d", + completedPercentage, + downloaded, + estimatedTotal, + context.getEthContext().getEthPeers().peerCount())); + lastLogAt = now; + } + } else { + LOG.info( + String.format( + "Backward sync phase 1 of 2 completed, downloaded a total of %d headers. Peers: %d", + downloaded, context.getEthContext().getEthPeers().peerCount())); + } + } } diff --git a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardsSyncAlgorithm.java b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardsSyncAlgorithm.java index 8ff1402f282..35c3f90ec21 100644 --- a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardsSyncAlgorithm.java +++ b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardsSyncAlgorithm.java @@ -22,6 +22,7 @@ import org.hyperledger.besu.datatypes.Hash; import org.hyperledger.besu.ethereum.chain.MutableBlockchain; +import org.hyperledger.besu.ethereum.core.Block; import org.hyperledger.besu.ethereum.core.BlockHeader; import java.util.Optional; @@ -57,11 +58,15 @@ public CompletableFuture pickNextStep() { final Optional firstHash = context.getBackwardChain().getFirstHashToAppend(); if (firstHash.isPresent()) { return executeSyncStep(firstHash.get()) - .whenComplete( - (result, throwable) -> { - if (throwable == null) { - context.getBackwardChain().removeFromHashToAppend(firstHash.get()); - } + .thenAccept( + result -> { + LOG.info("Backward sync target block is {}", result.toLogString()); + context.getBackwardChain().removeFromHashToAppend(firstHash.get()); + context + .getStatus() + .setSyncRange( + context.getProtocolContext().getBlockchain().getChainHeadBlockNumber(), + result.getHeader().getNumber()); }); } if (!context.isReady()) { @@ -73,7 +78,7 @@ public CompletableFuture pickNextStep() { context.getBackwardChain().getFirstAncestorHeader(); if (possibleFirstAncestorHeader.isEmpty()) { this.finished = true; - LOG.info("The Backward sync is done..."); + LOG.info("The Backward sync is done"); context.getBackwardChain().clear(); return CompletableFuture.completedFuture(null); } @@ -85,13 +90,16 @@ public CompletableFuture pickNextStep() { if (blockchain.getChainHead().getHeight() > firstAncestorHeader.getNumber()) { debugLambda( LOG, - "Backward reached below previous head {} : {}", + "Backward reached below current chain head {} : {}", () -> blockchain.getChainHead().toLogString(), firstAncestorHeader::toLogString); } if (finalBlockConfirmation.ancestorHeaderReached(firstAncestorHeader)) { - LOG.info("Backward sync reached ancestor header, starting Forward sync"); + debugLambda( + LOG, + "Backward sync reached ancestor header with {}, starting Forward sync", + firstAncestorHeader::toLogString); return executeForwardAsync(); } @@ -104,7 +112,7 @@ public CompletableFuture executeProcessKnownAncestors() { } @VisibleForTesting - public CompletableFuture executeSyncStep(final Hash hash) { + public CompletableFuture executeSyncStep(final Hash hash) { return new SyncStepStep(context, context.getBackwardChain()).executeAsync(hash); } diff --git a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/ForwardSyncStep.java b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/ForwardSyncStep.java index 71c0aa26cd8..00748dd5234 100644 --- a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/ForwardSyncStep.java +++ b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/ForwardSyncStep.java @@ -15,7 +15,6 @@ package org.hyperledger.besu.ethereum.eth.sync.backwardsync; import static org.hyperledger.besu.util.Slf4jLambdaHelper.debugLambda; -import static org.hyperledger.besu.util.Slf4jLambdaHelper.infoLambda; import org.hyperledger.besu.ethereum.core.Block; import org.hyperledger.besu.ethereum.core.BlockHeader; @@ -34,6 +33,8 @@ public class ForwardSyncStep { private static final Logger LOG = LoggerFactory.getLogger(ForwardSyncStep.class); + private static final long MILLIS_DELAY_BETWEEN_PROGRESS_LOG = 10_000L; + private static long lastLogAt = 0; private final BackwardSyncContext context; private final BackwardChain backwardChain; @@ -78,6 +79,7 @@ protected CompletableFuture> requestBodies(final List b return run.thenApply(AbstractPeerTask.PeerTaskResult::getResult) .thenApply( blocks -> { + LOG.debug("Got {} blocks from peers", blocks.size()); blocks.sort(Comparator.comparing(block -> block.getHeader().getNumber())); return blocks; }); @@ -86,8 +88,8 @@ protected CompletableFuture> requestBodies(final List b @VisibleForTesting protected Void saveBlocks(final List blocks) { if (blocks.isEmpty()) { - LOG.info("No blocks to save..."); context.halveBatchSize(); + LOG.debug("No blocks to save, reducing batch size to {}", context.getBatchSize()); return null; } @@ -97,21 +99,53 @@ protected Void saveBlocks(final List blocks) { .getProtocolContext() .getBlockchain() .getBlockByHash(block.getHeader().getParentHash()); + if (parent.isEmpty()) { context.halveBatchSize(); + debugLambda( + LOG, + "Parent block {} not found, while saving block {}, reducing batch size to {}", + block.getHeader().getParentHash()::toString, + block::toLogString, + context::getBatchSize); + logProgress(blocks.get(blocks.size() - 1).getHeader().getNumber()); return null; } else { context.saveBlock(block); } } - infoLambda( - LOG, - "Saved blocks {} -> {} (target: {})", - () -> blocks.get(0).getHeader().getNumber(), - () -> blocks.get(blocks.size() - 1).getHeader().getNumber(), - () -> - backwardChain.getPivot().orElse(blocks.get(blocks.size() - 1)).getHeader().getNumber()); + + logProgress(blocks.get(blocks.size() - 1).getHeader().getNumber()); + context.resetBatchSize(); return null; } + + private void logProgress(final long currImportedHeight) { + final long targetHeight = context.getStatus().getTargetChainHeight(); + final long initialHeight = context.getStatus().getInitialChainHeight(); + final long estimatedTotal = targetHeight - initialHeight; + final long imported = currImportedHeight - initialHeight; + + final float completedPercentage = 100.0f * imported / estimatedTotal; + + if (currImportedHeight < targetHeight) { + final long now = System.currentTimeMillis(); + if (now - lastLogAt > MILLIS_DELAY_BETWEEN_PROGRESS_LOG) { + LOG.info( + String.format( + "Backward sync phase 2 of 2, %.2f%% completed, imported %d blocks of at least %d. Peers: %d", + completedPercentage, + imported, + estimatedTotal, + context.getEthContext().getEthPeers().peerCount())); + lastLogAt = now; + } + } else { + LOG.info( + String.format( + "Backward sync phase 2 of 2 completed, imported a total of %d blocks. Peers: %d", + imported, context.getEthContext().getEthPeers().peerCount())); + } + } } diff --git a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/SyncStepStep.java b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/SyncStepStep.java index 88ddc18b2c6..f15a4d2b6d8 100644 --- a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/SyncStepStep.java +++ b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/SyncStepStep.java @@ -17,6 +17,7 @@ package org.hyperledger.besu.ethereum.eth.sync.backwardsync; +import static org.hyperledger.besu.util.Slf4jLambdaHelper.debugLambda; import static org.slf4j.LoggerFactory.getLogger; import org.hyperledger.besu.datatypes.Hash; @@ -41,13 +42,14 @@ public SyncStepStep(final BackwardSyncContext context, final BackwardChain backw this.backwardChain = backwardChain; } - public CompletableFuture executeAsync(final Hash hash) { + public CompletableFuture executeAsync(final Hash hash) { return CompletableFuture.supplyAsync(() -> hash) .thenCompose(this::requestBlock) .thenApply(this::saveBlock); } private CompletableFuture requestBlock(final Hash targetHash) { + debugLambda(LOG, "Fetching block by hash {} from peers", targetHash::toString); final RetryingGetBlockFromPeersTask getBlockTask = RetryingGetBlockFromPeersTask.create( context.getProtocolSchedule(), @@ -63,10 +65,9 @@ private CompletableFuture requestBlock(final Hash targetHash) { .thenApply(AbstractPeerTask.PeerTaskResult::getResult); } - private Void saveBlock(final Block block) { - LOG.debug( - "Appending block {}({})", block.getHeader().getNumber(), block.getHash().toHexString()); + private Block saveBlock(final Block block) { + debugLambda(LOG, "Appending fetched block {}", block::toLogString); backwardChain.appendTrustedBlock(block); - return null; + return block; } } From 91d5d3aaee8a6f19ceccac785e9e54f19d19fda7 Mon Sep 17 00:00:00 2001 From: Fabio Di Fabio Date: Fri, 11 Nov 2022 12:59:50 +0100 Subject: [PATCH 13/32] Add CHANGELOG entry Signed-off-by: Fabio Di Fabio --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index c0e56539154..815136aa0b3 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -9,6 +9,7 @@ - Support for ephemeral testnet Shandong. EIPs are still in flux, besu does not fully sync yet, and the network is subject to restarts. [#//FIXME](https://github.com/hyperledger/besu/pull///FIXME) - Improve performance of block processing by parallelizing some parts during the "commit" step [#4635](https://github.com/hyperledger/besu/pull/4635) - Upgrade RocksDB version from 7.6.0 to 7.7.3 +- Backward sync log UX improvements [#4655](https://github.com/hyperledger/besu/pull/4655) ### Bug Fixes From 3c202709892d07597e4c7ce34232037b9406e09b Mon Sep 17 00:00:00 2001 From: Fabio Di Fabio Date: Fri, 11 Nov 2022 14:28:31 +0100 Subject: [PATCH 14/32] Backward sync, use retry switching peer when fetching data from peers Signed-off-by: Fabio Di Fabio --- .../manager/task/GetBodiesFromPeerTask.java | 4 +- .../task/RetryingGetBlocksFromPeersTask.java | 112 ++++++++++++++++++ ...gGetHeadersEndingAtFromPeerByHashTask.java | 43 +++++-- .../sync/backwardsync/BackwardSyncStep.java | 7 +- .../backwardsync/BackwardsSyncAlgorithm.java | 18 ++- .../sync/backwardsync/ForwardSyncStep.java | 11 +- .../eth/sync/backwardsync/SyncStepStep.java | 2 +- .../backwardsync/BackwardSyncAlgSpec.java | 2 + .../backwardsync/BackwardSyncStepTest.java | 8 +- 9 files changed, 177 insertions(+), 30 deletions(-) create mode 100644 ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/manager/task/RetryingGetBlocksFromPeersTask.java diff --git a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/manager/task/GetBodiesFromPeerTask.java b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/manager/task/GetBodiesFromPeerTask.java index dd4f2792f54..9eb0f77f5d4 100644 --- a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/manager/task/GetBodiesFromPeerTask.java +++ b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/manager/task/GetBodiesFromPeerTask.java @@ -65,7 +65,7 @@ private GetBodiesFromPeerTask( headers.forEach( (header) -> { final BodyIdentifier bodyId = new BodyIdentifier(header); - bodyToHeaders.putIfAbsent(bodyId, new ArrayList<>()); + bodyToHeaders.putIfAbsent(bodyId, new ArrayList<>(headers.size())); bodyToHeaders.get(bodyId).add(header); }); } @@ -112,7 +112,7 @@ protected Optional> processResponse( return Optional.empty(); } - final List blocks = new ArrayList<>(); + final List blocks = new ArrayList<>(headers.size()); for (final BlockBody body : bodies) { final List headers = bodyToHeaders.get(new BodyIdentifier(body)); if (headers == null) { diff --git a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/manager/task/RetryingGetBlocksFromPeersTask.java b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/manager/task/RetryingGetBlocksFromPeersTask.java new file mode 100644 index 00000000000..85a663f6230 --- /dev/null +++ b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/manager/task/RetryingGetBlocksFromPeersTask.java @@ -0,0 +1,112 @@ +/* + * Copyright contributors to Hyperledger Besu + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + * + * SPDX-License-Identifier: Apache-2.0 + */ +package org.hyperledger.besu.ethereum.eth.manager.task; + +import static org.hyperledger.besu.util.Slf4jLambdaHelper.debugLambda; + +import org.hyperledger.besu.ethereum.core.Block; +import org.hyperledger.besu.ethereum.core.BlockHeader; +import org.hyperledger.besu.ethereum.eth.manager.EthContext; +import org.hyperledger.besu.ethereum.eth.manager.EthPeer; +import org.hyperledger.besu.ethereum.eth.manager.exceptions.IncompleteResultsException; +import org.hyperledger.besu.ethereum.eth.manager.task.AbstractPeerTask.PeerTaskResult; +import org.hyperledger.besu.ethereum.mainnet.ProtocolSchedule; +import org.hyperledger.besu.plugin.services.MetricsSystem; + +import java.util.List; +import java.util.Objects; +import java.util.concurrent.CompletableFuture; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class RetryingGetBlocksFromPeersTask + extends AbstractRetryingSwitchingPeerTask>> { + + private static final Logger LOG = LoggerFactory.getLogger(RetryingGetBlocksFromPeersTask.class); + + private final ProtocolSchedule protocolSchedule; + private final List headers; + + protected RetryingGetBlocksFromPeersTask( + final EthContext ethContext, + final ProtocolSchedule protocolSchedule, + final MetricsSystem metricsSystem, + final int maxRetries, + final List headers) { + super(ethContext, metricsSystem, Objects::isNull, maxRetries); + this.protocolSchedule = protocolSchedule; + this.headers = headers; + } + + public static RetryingGetBlocksFromPeersTask forHeaders( + final ProtocolSchedule protocolSchedule, + final EthContext ethContext, + final MetricsSystem metricsSystem, + final int maxRetries, + final List headers) { + return new RetryingGetBlocksFromPeersTask( + ethContext, protocolSchedule, metricsSystem, maxRetries, headers); + } + + @Override + protected CompletableFuture>> executeTaskOnCurrentPeer( + final EthPeer currentPeer) { + final GetBodiesFromPeerTask getBodiesTask = + GetBodiesFromPeerTask.forHeaders( + protocolSchedule, getEthContext(), headers, getMetricsSystem()); + getBodiesTask.assignPeer(currentPeer); + + return executeSubTask(getBodiesTask::run) + .thenApply( + peerResult -> { + debugLambda( + LOG, + "Got {} blocks {} from peer {}, attempt {}", + peerResult.getResult()::size, + peerResult.getPeer()::toString, + this::getRetryCount); + + if (peerResult.getResult().isEmpty()) { + currentPeer.recordUselessResponse("GetBodiesFromPeerTask"); + throw new IncompleteResultsException( + "No blocks returned by peer " + currentPeer.getShortNodeId()); + } + + result.complete(peerResult); + return peerResult; + }); + } + + @Override + protected boolean isRetryableError(final Throwable error) { + return super.isRetryableError(error) || error instanceof IncompleteResultsException; + } + + @Override + protected void handleTaskError(final Throwable error) { + if (getRetryCount() < getMaxRetries()) { + debugLambda( + LOG, + "Failed to get {} blocks from peer {}, attempt {}, retrying later", + headers::size, + this::getAssignedPeer, + this::getRetryCount); + } else { + LOG.warn("Failed to get {} blocks after {} retries", headers.size(), getRetryCount()); + } + super.handleTaskError(error); + } +} diff --git a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/manager/task/RetryingGetHeadersEndingAtFromPeerByHashTask.java b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/manager/task/RetryingGetHeadersEndingAtFromPeerByHashTask.java index bc758e6b177..2d1119c4aca 100644 --- a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/manager/task/RetryingGetHeadersEndingAtFromPeerByHashTask.java +++ b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/manager/task/RetryingGetHeadersEndingAtFromPeerByHashTask.java @@ -21,17 +21,21 @@ import org.hyperledger.besu.ethereum.core.BlockHeader; import org.hyperledger.besu.ethereum.eth.manager.EthContext; import org.hyperledger.besu.ethereum.eth.manager.EthPeer; +import org.hyperledger.besu.ethereum.eth.manager.exceptions.IncompleteResultsException; import org.hyperledger.besu.ethereum.mainnet.ProtocolSchedule; import org.hyperledger.besu.plugin.services.MetricsSystem; import java.util.List; -import java.util.Optional; import java.util.concurrent.CompletableFuture; import com.google.common.annotations.VisibleForTesting; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class RetryingGetHeadersEndingAtFromPeerByHashTask - extends AbstractRetryingPeerTask> { + extends AbstractRetryingSwitchingPeerTask> { + private static final Logger LOG = + LoggerFactory.getLogger(RetryingGetHeadersEndingAtFromPeerByHashTask.class); private final Hash referenceHash; private final ProtocolSchedule protocolSchedule; @@ -43,8 +47,9 @@ public class RetryingGetHeadersEndingAtFromPeerByHashTask final EthContext ethContext, final Hash referenceHash, final int count, - final MetricsSystem metricsSystem) { - super(ethContext, 4, List::isEmpty, metricsSystem); + final MetricsSystem metricsSystem, + final int maxRetries) { + super(ethContext, metricsSystem, List::isEmpty, maxRetries); this.protocolSchedule = protocolSchedule; this.count = count; checkNotNull(referenceHash); @@ -56,23 +61,43 @@ public static RetryingGetHeadersEndingAtFromPeerByHashTask endingAtHash( final EthContext ethContext, final Hash referenceHash, final int count, - final MetricsSystem metricsSystem) { + final MetricsSystem metricsSystem, + final int maxRetries) { return new RetryingGetHeadersEndingAtFromPeerByHashTask( - protocolSchedule, ethContext, referenceHash, count, metricsSystem); + protocolSchedule, ethContext, referenceHash, count, metricsSystem, maxRetries); } @Override - protected CompletableFuture> executePeerTask( - final Optional assignedPeer) { + protected CompletableFuture> executeTaskOnCurrentPeer( + final EthPeer currentPeer) { final AbstractGetHeadersFromPeerTask task = GetHeadersFromPeerByHashTask.endingAtHash( protocolSchedule, getEthContext(), referenceHash, count, getMetricsSystem()); - assignedPeer.ifPresent(task::assignPeer); + task.assignPeer(currentPeer); return executeSubTask(task::run) .thenApply( peerResult -> { + LOG.debug( + "Get {} block headers by hash {} from peer {} has result {}", + count, + referenceHash, + currentPeer, + peerResult.getResult()); + if (peerResult.getResult().isEmpty()) { + currentPeer.recordUselessResponse("GetHeadersFromPeerByHashTask"); + throw new IncompleteResultsException( + "No block headers for hash " + + referenceHash + + " returned by peer " + + currentPeer.getShortNodeId()); + } result.complete(peerResult.getResult()); return peerResult.getResult(); }); } + + @Override + protected boolean isRetryableError(final Throwable error) { + return super.isRetryableError(error) || error instanceof IncompleteResultsException; + } } diff --git a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncStep.java b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncStep.java index e22e4bbd698..357d596eff4 100644 --- a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncStep.java +++ b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncStep.java @@ -71,17 +71,14 @@ protected CompletableFuture> requestHeaders(final Hash hash) { context.getEthContext(), hash, batchSize, - context.getMetricsSystem()); + context.getMetricsSystem(), + context.getEthContext().getEthPeers().peerCount()); return context .getEthContext() .getScheduler() .scheduleSyncWorkerTask(retryingGetHeadersEndingAtFromPeerByHashTask::run) .thenApply( blockHeaders -> { - if (blockHeaders.isEmpty()) { - throw new BackwardSyncException( - "Did not receive a headers for hash " + hash.toHexString(), true); - } debugLambda( LOG, "Got headers {} -> {}", diff --git a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardsSyncAlgorithm.java b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardsSyncAlgorithm.java index 35c3f90ec21..f362c9ec3ad 100644 --- a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardsSyncAlgorithm.java +++ b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardsSyncAlgorithm.java @@ -14,7 +14,6 @@ * SPDX-License-Identifier: Apache-2.0 * */ - package org.hyperledger.besu.ethereum.eth.sync.backwardsync; import static org.hyperledger.besu.util.Slf4jLambdaHelper.debugLambda; @@ -24,10 +23,12 @@ import org.hyperledger.besu.ethereum.chain.MutableBlockchain; import org.hyperledger.besu.ethereum.core.Block; import org.hyperledger.besu.ethereum.core.BlockHeader; +import org.hyperledger.besu.ethereum.eth.manager.task.WaitForPeersTask; import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import com.google.common.annotations.VisibleForTesting; @@ -142,12 +143,17 @@ private void checkReadiness(final CountDownLatch latch, final long idTTD, final LOG.debug("Waiting for preconditions..."); final boolean await = latch.await(2, TimeUnit.MINUTES); if (await) { - LOG.debug("Preconditions meet..."); + LOG.debug("Preconditions meet, ensure at least one peer is connected"); + waitForPeers(1).get(); } } } catch (InterruptedException e) { Thread.currentThread().interrupt(); - throw new BackwardSyncException("Wait for TTD preconditions interrupted"); + throw new BackwardSyncException( + "Wait for TTD preconditions interrupted (" + e.getMessage() + ")"); + } catch (ExecutionException e) { + throw new BackwardSyncException( + "Error while waiting for at least one connected peer (" + e.getMessage() + ")", true); } finally { context.getSyncState().unsubscribeTTDReached(idTTD); context.getSyncState().unsubscribeInitialConditionReached(idIS); @@ -205,4 +211,10 @@ protected void runFinalizedSuccessionRule( blockchain.setFinalized(newFinalized); } + + private CompletableFuture waitForPeers(final int count) { + final WaitForPeersTask waitForPeersTask = + WaitForPeersTask.create(context.getEthContext(), count, context.getMetricsSystem()); + return waitForPeersTask.run(); + } } diff --git a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/ForwardSyncStep.java b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/ForwardSyncStep.java index 00748dd5234..d76c5b27f10 100644 --- a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/ForwardSyncStep.java +++ b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/ForwardSyncStep.java @@ -19,7 +19,7 @@ import org.hyperledger.besu.ethereum.core.Block; import org.hyperledger.besu.ethereum.core.BlockHeader; import org.hyperledger.besu.ethereum.eth.manager.task.AbstractPeerTask; -import org.hyperledger.besu.ethereum.eth.manager.task.GetBodiesFromPeerTask; +import org.hyperledger.besu.ethereum.eth.manager.task.RetryingGetBlocksFromPeersTask; import java.util.Comparator; import java.util.List; @@ -67,12 +67,13 @@ public CompletableFuture possibleRequestBodies(final List blo @VisibleForTesting protected CompletableFuture> requestBodies(final List blockHeaders) { - final GetBodiesFromPeerTask getBodiesFromPeerTask = - GetBodiesFromPeerTask.forHeaders( + final RetryingGetBlocksFromPeersTask getBodiesFromPeerTask = + RetryingGetBlocksFromPeersTask.forHeaders( context.getProtocolSchedule(), context.getEthContext(), - blockHeaders, - context.getMetricsSystem()); + context.getMetricsSystem(), + context.getEthContext().getEthPeers().peerCount(), + blockHeaders); final CompletableFuture>> run = getBodiesFromPeerTask.run(); diff --git a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/SyncStepStep.java b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/SyncStepStep.java index f15a4d2b6d8..2dc8b1d4351 100644 --- a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/SyncStepStep.java +++ b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/SyncStepStep.java @@ -55,7 +55,7 @@ private CompletableFuture requestBlock(final Hash targetHash) { context.getProtocolSchedule(), context.getEthContext(), context.getMetricsSystem(), - context.getEthContext().getEthPeers().getMaxPeers(), + context.getEthContext().getEthPeers().peerCount(), Optional.of(targetHash), UNUSED); return context diff --git a/ethereum/eth/src/test/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncAlgSpec.java b/ethereum/eth/src/test/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncAlgSpec.java index 3d4782bef2d..91a57bb8636 100644 --- a/ethereum/eth/src/test/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncAlgSpec.java +++ b/ethereum/eth/src/test/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncAlgSpec.java @@ -149,6 +149,7 @@ public void shouldAwokeWhenTTDReachedAndReady() throws Exception { when(context.getSyncState().subscribeTTDReached(any())).thenReturn(88L); when(context.getSyncState().subscribeCompletionReached(any())).thenReturn(99L); + when(context.getEthContext().getEthPeers().peerCount()).thenReturn(1); final CompletableFuture voidCompletableFuture = algorithm.waitForReady(); @@ -175,6 +176,7 @@ public void shouldAwokeWhenConditionReachedAndReady() throws Exception { when(context.getSyncState().subscribeTTDReached(any())).thenReturn(88L); when(context.getSyncState().subscribeCompletionReached(any())).thenReturn(99L); + when(context.getEthContext().getEthPeers().peerCount()).thenReturn(1); final CompletableFuture voidCompletableFuture = algorithm.waitForReady(); Thread.sleep(50); diff --git a/ethereum/eth/src/test/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncStepTest.java b/ethereum/eth/src/test/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncStepTest.java index ada5b5494f7..7f9e5b7c5b5 100644 --- a/ethereum/eth/src/test/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncStepTest.java +++ b/ethereum/eth/src/test/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncStepTest.java @@ -33,6 +33,7 @@ import org.hyperledger.besu.ethereum.eth.manager.EthProtocolManager; import org.hyperledger.besu.ethereum.eth.manager.EthProtocolManagerTestUtil; import org.hyperledger.besu.ethereum.eth.manager.RespondingEthPeer; +import org.hyperledger.besu.ethereum.eth.manager.exceptions.MaxRetriesReachedException; import org.hyperledger.besu.ethereum.mainnet.MainnetBlockHeaderFunctions; import org.hyperledger.besu.ethereum.mainnet.MainnetProtocolSchedule; import org.hyperledger.besu.ethereum.mainnet.ProtocolSchedule; @@ -181,7 +182,7 @@ public void shouldRequestHeaderBeforeCurrentHeight() throws Exception { } @Test - public void shouldThrowWhenResponseIsEmptyWhenRequestingHeader() throws Exception { + public void shouldThrowWhenResponseIsEmptyWhenRequestingHeader() { BackwardSyncStep step = new BackwardSyncStep(context, createBackwardChain(REMOTE_HEIGHT - 1)); final Block lookingForBlock = getBlockByNumber(REMOTE_HEIGHT - 2); @@ -191,10 +192,7 @@ public void shouldThrowWhenResponseIsEmptyWhenRequestingHeader() throws Exceptio step.requestHeaders(lookingForBlock.getHeader().getHash()); peer.respondWhileOtherThreadsWork(responder, () -> !future.isDone()); - assertThatThrownBy(future::get) - .getCause() - .isInstanceOf(BackwardSyncException.class) - .hasMessageContaining("Did not receive a headers for hash"); + assertThatThrownBy(future::get).cause().isInstanceOf(MaxRetriesReachedException.class); } @Test From 816e4f2cc4b36f44f848c24d13732096e0db65bd Mon Sep 17 00:00:00 2001 From: Fabio Di Fabio Date: Fri, 11 Nov 2022 14:30:14 +0100 Subject: [PATCH 15/32] Add CHANGELOG entry Signed-off-by: Fabio Di Fabio --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 815136aa0b3..f4c9d3f7cbc 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -10,6 +10,7 @@ - Improve performance of block processing by parallelizing some parts during the "commit" step [#4635](https://github.com/hyperledger/besu/pull/4635) - Upgrade RocksDB version from 7.6.0 to 7.7.3 - Backward sync log UX improvements [#4655](https://github.com/hyperledger/besu/pull/4655) +- Backward sync: use retry switching peer when fetching data from peers [#4656](https://github.com/hyperledger/besu/pull/4656) ### Bug Fixes From 869f2c6457d7d28e18c2e03235afb56273d6f6b4 Mon Sep 17 00:00:00 2001 From: Fabio Di Fabio Date: Fri, 11 Nov 2022 16:25:52 +0100 Subject: [PATCH 16/32] Fix log in case of sync failures Signed-off-by: Fabio Di Fabio --- .../merge/blockcreation/MergeCoordinator.java | 6 +++++- .../internal/methods/engine/EngineNewPayload.java | 7 +++++-- .../eth/sync/backwardsync/BackwardSyncContext.java | 11 +++++++---- .../eth/sync/backwardsync/BackwardSyncStep.java | 2 +- .../eth/sync/backwardsync/ForwardSyncStep.java | 10 +++++++--- 5 files changed, 25 insertions(+), 11 deletions(-) diff --git a/consensus/merge/src/main/java/org/hyperledger/besu/consensus/merge/blockcreation/MergeCoordinator.java b/consensus/merge/src/main/java/org/hyperledger/besu/consensus/merge/blockcreation/MergeCoordinator.java index 438d294602c..d907ccdc7b4 100644 --- a/consensus/merge/src/main/java/org/hyperledger/besu/consensus/merge/blockcreation/MergeCoordinator.java +++ b/consensus/merge/src/main/java/org/hyperledger/besu/consensus/merge/blockcreation/MergeCoordinator.java @@ -390,7 +390,11 @@ public Optional getOrSyncHeaderByHash( } private Void logSyncException(final Hash blockHash, final Throwable exception) { - LOG.warn("Sync to block hash " + blockHash.toHexString() + " failed", exception.getMessage()); + debugLambda( + LOG, + "Sync to block hash {} failed, reason {}", + blockHash::toHexString, + exception::getMessage); return null; } diff --git a/ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/methods/engine/EngineNewPayload.java b/ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/methods/engine/EngineNewPayload.java index 7390b9d80f0..f0636c02d1b 100644 --- a/ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/methods/engine/EngineNewPayload.java +++ b/ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/methods/engine/EngineNewPayload.java @@ -180,7 +180,6 @@ public JsonRpcResponse syncResponse(final JsonRpcRequestContext requestContext) final var block = new Block(newBlockHeader, new BlockBody(transactions, Collections.emptyList())); - final String warningMessage = "Sync to block " + block.toLogString() + " failed"; if (mergeContext.get().isSyncing() || parentHeader.isEmpty()) { LOG.debug( @@ -192,7 +191,11 @@ public JsonRpcResponse syncResponse(final JsonRpcRequestContext requestContext) .appendNewPayloadToSync(block) .exceptionally( exception -> { - LOG.warn(warningMessage, exception.getMessage()); + debugLambda( + LOG, + "Sync to block {} failed, reason {}", + block::toLogString, + exception::getMessage); return null; }); return respondWith(reqId, blockParam, null, SYNCING); diff --git a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncContext.java b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncContext.java index fb941722733..f34cf5561bb 100644 --- a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncContext.java +++ b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncContext.java @@ -131,6 +131,7 @@ public synchronized CompletableFuture syncBackwardsUntil(final Hash newBlo backwardChain.addNewHash(newBlockHash); return maybeFuture.orElseGet( () -> { + LOG.info("Starting a new backward sync session"); Status status = new Status(prepareBackwardSyncFutureWithRetry()); this.currentBackwardSyncStatus.set(status); return status.currentFuture; @@ -147,7 +148,12 @@ public synchronized CompletableFuture syncBackwardsUntil(final Block newPi backwardChain.appendTrustedBlock(newPivot); return maybeFuture.orElseGet( () -> { + LOG.info("Starting a new backward sync session"); + LOG.info("Backward sync target block is {}", newPivot.toLogString()); Status status = new Status(prepareBackwardSyncFutureWithRetry()); + status.setSyncRange( + protocolContext.getBlockchain().getChainHeadBlockNumber(), + newPivot.getHeader().getNumber()); this.currentBackwardSyncStatus.set(status); return status.currentFuture; }); @@ -170,6 +176,7 @@ private CompletableFuture prepareBackwardSyncFutureWithRetry() { (unused, throwable) -> { this.currentBackwardSyncStatus.set(null); if (throwable != null) { + LOG.info("Current backward sync session failed, it will be restarted"); throw extractBackwardSyncException(throwable) .orElse(new BackwardSyncException(throwable)); } @@ -407,9 +414,5 @@ public long getTargetChainHeight() { public long getInitialChainHeight() { return initialChainHeight; } - - public long getBlockCount() { - return targetChainHeight - initialChainHeight; - } } } diff --git a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncStep.java b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncStep.java index 357d596eff4..4c747708cc5 100644 --- a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncStep.java +++ b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncStep.java @@ -113,7 +113,7 @@ private void logProgress(final long currLowestDownloadedHeight) { final float completedPercentage = 100.0f * downloaded / estimatedTotal; - if (currLowestDownloadedHeight > initialHeight) { + if (completedPercentage < 100.0f) { final long now = System.currentTimeMillis(); if (now - lastLogAt > MILLIS_DELAY_BETWEEN_PROGRESS_LOG) { LOG.info( diff --git a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/ForwardSyncStep.java b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/ForwardSyncStep.java index d76c5b27f10..1518cc2b2d1 100644 --- a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/ForwardSyncStep.java +++ b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/ForwardSyncStep.java @@ -94,6 +94,9 @@ protected Void saveBlocks(final List blocks) { return null; } + long lastImportedHeight = + context.getProtocolContext().getBlockchain().getChainHeadBlockNumber(); + for (Block block : blocks) { final Optional parent = context @@ -109,14 +112,15 @@ protected Void saveBlocks(final List blocks) { block.getHeader().getParentHash()::toString, block::toLogString, context::getBatchSize); - logProgress(blocks.get(blocks.size() - 1).getHeader().getNumber()); + logProgress(lastImportedHeight); return null; } else { context.saveBlock(block); + lastImportedHeight = block.getHeader().getNumber(); } } - logProgress(blocks.get(blocks.size() - 1).getHeader().getNumber()); + logProgress(lastImportedHeight); context.resetBatchSize(); return null; @@ -130,7 +134,7 @@ private void logProgress(final long currImportedHeight) { final float completedPercentage = 100.0f * imported / estimatedTotal; - if (currImportedHeight < targetHeight) { + if (completedPercentage < 100.0f) { final long now = System.currentTimeMillis(); if (now - lastLogAt > MILLIS_DELAY_BETWEEN_PROGRESS_LOG) { LOG.info( From e8190d1dc903abce1372ffc3bc6d15e189d73f2e Mon Sep 17 00:00:00 2001 From: Fabio Di Fabio Date: Fri, 11 Nov 2022 16:48:34 +0100 Subject: [PATCH 17/32] Log transient error at debug level Signed-off-by: Fabio Di Fabio --- .../eth/manager/task/RetryingGetBlocksFromPeersTask.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/manager/task/RetryingGetBlocksFromPeersTask.java b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/manager/task/RetryingGetBlocksFromPeersTask.java index 85a663f6230..7d238586f2e 100644 --- a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/manager/task/RetryingGetBlocksFromPeersTask.java +++ b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/manager/task/RetryingGetBlocksFromPeersTask.java @@ -105,7 +105,7 @@ protected void handleTaskError(final Throwable error) { this::getAssignedPeer, this::getRetryCount); } else { - LOG.warn("Failed to get {} blocks after {} retries", headers.size(), getRetryCount()); + LOG.debug("Failed to get {} blocks after {} retries", headers.size(), getRetryCount()); } super.handleTaskError(error); } From 6c0c302fc79ebbf9c8d317c19d351e3522a3b315 Mon Sep 17 00:00:00 2001 From: Fabio Di Fabio Date: Mon, 14 Nov 2022 14:41:38 +0100 Subject: [PATCH 18/32] Rework log of imported block to support also ProcessKnownAncestorsStep Signed-off-by: Fabio Di Fabio --- .../backwardsync/BackwardSyncContext.java | 113 +++++++++++++----- .../sync/backwardsync/BackwardSyncStep.java | 6 +- .../sync/backwardsync/ForwardSyncStep.java | 37 ------ 3 files changed, 87 insertions(+), 69 deletions(-) diff --git a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncContext.java b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncContext.java index f34cf5561bb..f52ac4cdc28 100644 --- a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncContext.java +++ b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncContext.java @@ -47,7 +47,7 @@ public class BackwardSyncContext { private static final Logger LOG = LoggerFactory.getLogger(BackwardSyncContext.class); public static final int BATCH_SIZE = 200; private static final int DEFAULT_MAX_RETRIES = 20; - + private static final long MILLIS_DELAY_BETWEEN_PROGRESS_LOG = 10_000L; private static final long DEFAULT_MILLIS_BETWEEN_RETRIES = 5000; protected final ProtocolContext protocolContext; @@ -122,41 +122,51 @@ public synchronized void updateHeads(final Hash head, final Hash finalizedBlockH } public synchronized CompletableFuture syncBackwardsUntil(final Hash newBlockHash) { - Optional> maybeFuture = - Optional.ofNullable(this.currentBackwardSyncStatus.get()) - .map(status -> status.currentFuture); + Optional maybeCurrentStatus = Optional.ofNullable(this.currentBackwardSyncStatus.get()); if (isTrusted(newBlockHash)) { - return maybeFuture.orElseGet(() -> CompletableFuture.completedFuture(null)); + return maybeCurrentStatus + .map( + status -> { + backwardChain + .getBlock(newBlockHash) + .ifPresent(block -> status.updateTargetHeight(block.getHeader().getNumber())); + return status.currentFuture; + }) + .orElseGet(() -> CompletableFuture.completedFuture(null)); } backwardChain.addNewHash(newBlockHash); - return maybeFuture.orElseGet( - () -> { - LOG.info("Starting a new backward sync session"); - Status status = new Status(prepareBackwardSyncFutureWithRetry()); - this.currentBackwardSyncStatus.set(status); - return status.currentFuture; - }); + return maybeCurrentStatus + .map(Status::getCurrentFuture) + .orElseGet( + () -> { + LOG.info("Starting a new backward sync session"); + Status status = new Status(prepareBackwardSyncFutureWithRetry()); + this.currentBackwardSyncStatus.set(status); + return status.currentFuture; + }); } public synchronized CompletableFuture syncBackwardsUntil(final Block newPivot) { - Optional> maybeFuture = - Optional.ofNullable(this.currentBackwardSyncStatus.get()) - .map(status -> status.currentFuture); + Optional maybeCurrentStatus = Optional.ofNullable(this.currentBackwardSyncStatus.get()); if (isTrusted(newPivot.getHash())) { - return maybeFuture.orElseGet(() -> CompletableFuture.completedFuture(null)); + return maybeCurrentStatus + .map(Status::getCurrentFuture) + .orElseGet(() -> CompletableFuture.completedFuture(null)); } backwardChain.appendTrustedBlock(newPivot); - return maybeFuture.orElseGet( - () -> { - LOG.info("Starting a new backward sync session"); - LOG.info("Backward sync target block is {}", newPivot.toLogString()); - Status status = new Status(prepareBackwardSyncFutureWithRetry()); - status.setSyncRange( - protocolContext.getBlockchain().getChainHeadBlockNumber(), - newPivot.getHeader().getNumber()); - this.currentBackwardSyncStatus.set(status); - return status.currentFuture; - }); + return maybeCurrentStatus + .map(Status::getCurrentFuture) + .orElseGet( + () -> { + LOG.info("Starting a new backward sync session"); + LOG.info("Backward sync target block is {}", newPivot.toLogString()); + Status status = new Status(prepareBackwardSyncFutureWithRetry()); + status.setSyncRange( + getProtocolContext().getBlockchain().getChainHeadBlockNumber(), + newPivot.getHeader().getNumber()); + this.currentBackwardSyncStatus.set(status); + return status.currentFuture; + }); } private boolean isTrusted(final Hash hash) { @@ -319,6 +329,7 @@ protected Void saveBlock(final Block block) { .getBlockchain() .appendBlock(block, optResult.getYield().get().getReceipts()); possiblyMoveHead(block); + logBlockImportProgress(block.getHeader().getNumber()); } else { emitBadChainEvent(block); throw new BackwardSyncException( @@ -393,11 +404,42 @@ private void emitBadChainEvent(final Block badBlock) { listener -> listener.onBadChain(badBlock, badBlockDescendants, badBlockHeaderDescendants)); } + private void logBlockImportProgress(final long currImportedHeight) { + final Status currentStatus = getStatus(); + final long targetHeight = currentStatus.getTargetChainHeight(); + final long initialHeight = currentStatus.getInitialChainHeight(); + final long estimatedTotal = targetHeight - initialHeight; + final long imported = currImportedHeight - initialHeight; + + final float completedPercentage = 100.0f * imported / estimatedTotal; + + if (completedPercentage < 100.0f) { + if (currentStatus.couldLogProgress()) { + LOG.info( + String.format( + "Backward sync phase 2 of 2, %.2f%% completed, imported %d blocks of at least %d (current head %d, target head %d). Peers: %d", + completedPercentage, + imported, + estimatedTotal, + currImportedHeight, + currentStatus.getTargetChainHeight(), + getEthContext().getEthPeers().peerCount())); + } + } else { + LOG.info( + String.format( + "Backward sync phase 2 of 2 completed, imported a total of %d blocks. Peers: %d", + imported, getEthContext().getEthPeers().peerCount())); + } + } + static class Status { private final CompletableFuture currentFuture; private long targetChainHeight; private long initialChainHeight; + private static long lastLogAt = 0; + public Status(final CompletableFuture currentFuture) { this.currentFuture = currentFuture; } @@ -407,6 +449,23 @@ public void setSyncRange(final long initialHeight, final long targetHeight) { targetChainHeight = targetHeight; } + public void updateTargetHeight(final long newTargetHeight) { + targetChainHeight = newTargetHeight; + } + + public boolean couldLogProgress() { + final long now = System.currentTimeMillis(); + if (now - lastLogAt > MILLIS_DELAY_BETWEEN_PROGRESS_LOG) { + lastLogAt = now; + return true; + } + return false; + } + + public CompletableFuture getCurrentFuture() { + return currentFuture; + } + public long getTargetChainHeight() { return targetChainHeight; } diff --git a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncStep.java b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncStep.java index 2b15940eacd..c13276d013f 100644 --- a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncStep.java +++ b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncStep.java @@ -30,8 +30,6 @@ public class BackwardSyncStep { private static final Logger LOG = LoggerFactory.getLogger(BackwardSyncStep.class); - private static final long MILLIS_DELAY_BETWEEN_PROGRESS_LOG = 10_000L; - private static long lastLogAt = 0; private final BackwardSyncContext context; private final BackwardChain backwardChain; @@ -117,8 +115,7 @@ private void logProgress(final long currLowestDownloadedHeight) { final float completedPercentage = 100.0f * downloaded / estimatedTotal; if (completedPercentage < 100.0f) { - final long now = System.currentTimeMillis(); - if (now - lastLogAt > MILLIS_DELAY_BETWEEN_PROGRESS_LOG) { + if (context.getStatus().couldLogProgress()) { LOG.info( String.format( "Backward sync phase 1 of 2, %.2f%% completed, downloaded %d headers of at least %d. Peers: %d", @@ -126,7 +123,6 @@ private void logProgress(final long currLowestDownloadedHeight) { downloaded, estimatedTotal, context.getEthContext().getEthPeers().peerCount())); - lastLogAt = now; } } else { LOG.info( diff --git a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/ForwardSyncStep.java b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/ForwardSyncStep.java index 18a83ecebe2..37035d5ed4f 100644 --- a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/ForwardSyncStep.java +++ b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/ForwardSyncStep.java @@ -33,8 +33,6 @@ public class ForwardSyncStep { private static final Logger LOG = LoggerFactory.getLogger(ForwardSyncStep.class); - private static final long MILLIS_DELAY_BETWEEN_PROGRESS_LOG = 10_000L; - private static long lastLogAt = 0; private final BackwardSyncContext context; private final BackwardChain backwardChain; @@ -93,9 +91,6 @@ protected Void saveBlocks(final List blocks) { return null; } - long lastImportedHeight = - context.getProtocolContext().getBlockchain().getChainHeadBlockNumber(); - for (Block block : blocks) { final Optional parent = context @@ -111,45 +106,13 @@ protected Void saveBlocks(final List blocks) { block.getHeader().getParentHash()::toString, block::toLogString, context::getBatchSize); - logProgress(lastImportedHeight); return null; } else { context.saveBlock(block); - lastImportedHeight = block.getHeader().getNumber(); } } - logProgress(lastImportedHeight); - context.resetBatchSize(); return null; } - - private void logProgress(final long currImportedHeight) { - final long targetHeight = context.getStatus().getTargetChainHeight(); - final long initialHeight = context.getStatus().getInitialChainHeight(); - final long estimatedTotal = targetHeight - initialHeight; - final long imported = currImportedHeight - initialHeight; - - final float completedPercentage = 100.0f * imported / estimatedTotal; - - if (completedPercentage < 100.0f) { - final long now = System.currentTimeMillis(); - if (now - lastLogAt > MILLIS_DELAY_BETWEEN_PROGRESS_LOG) { - LOG.info( - String.format( - "Backward sync phase 2 of 2, %.2f%% completed, imported %d blocks of at least %d. Peers: %d", - completedPercentage, - imported, - estimatedTotal, - context.getEthContext().getEthPeers().peerCount())); - lastLogAt = now; - } - } else { - LOG.info( - String.format( - "Backward sync phase 2 of 2 completed, imported a total of %d blocks. Peers: %d", - imported, context.getEthContext().getEthPeers().peerCount())); - } - } } From 7ed79a02b8af3bd8f196a57270c5f1fad83ac222 Mon Sep 17 00:00:00 2001 From: Fabio Di Fabio Date: Mon, 14 Nov 2022 16:09:35 +0100 Subject: [PATCH 19/32] Reset the batch size only if we got a full batch Signed-off-by: Fabio Di Fabio --- .../besu/ethereum/eth/sync/backwardsync/ForwardSyncStep.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/ForwardSyncStep.java b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/ForwardSyncStep.java index 11dce332306..46fc13754a9 100644 --- a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/ForwardSyncStep.java +++ b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/ForwardSyncStep.java @@ -113,7 +113,10 @@ protected Void saveBlocks(final List blocks) { } } - context.resetBatchSize(); + if (blocks.size() == context.getBatchSize()) { + // reset the batch size only if we got a full batch + context.resetBatchSize(); + } return null; } } From d98618a5f39929b9108867ea78ac8c847af15a1b Mon Sep 17 00:00:00 2001 From: Fabio Di Fabio Date: Mon, 14 Nov 2022 16:14:27 +0100 Subject: [PATCH 20/32] Fix log Signed-off-by: Fabio Di Fabio --- .../eth/manager/task/RetryingGetBlocksFromPeersTask.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/manager/task/RetryingGetBlocksFromPeersTask.java b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/manager/task/RetryingGetBlocksFromPeersTask.java index 7d238586f2e..72f2c236cf6 100644 --- a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/manager/task/RetryingGetBlocksFromPeersTask.java +++ b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/manager/task/RetryingGetBlocksFromPeersTask.java @@ -74,7 +74,7 @@ protected CompletableFuture>> executeTaskOnCurrentPee peerResult -> { debugLambda( LOG, - "Got {} blocks {} from peer {}, attempt {}", + "Got {} blocks from peer {}, attempt {}", peerResult.getResult()::size, peerResult.getPeer()::toString, this::getRetryCount); From d8d0d677c94346f1e449f05d561f41deea9c5c59 Mon Sep 17 00:00:00 2001 From: Fabio Di Fabio Date: Mon, 14 Nov 2022 16:09:35 +0100 Subject: [PATCH 21/32] Reset the batch size only if we got a full batch Signed-off-by: Fabio Di Fabio --- .../besu/ethereum/eth/sync/backwardsync/ForwardSyncStep.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/ForwardSyncStep.java b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/ForwardSyncStep.java index 11dce332306..46fc13754a9 100644 --- a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/ForwardSyncStep.java +++ b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/ForwardSyncStep.java @@ -113,7 +113,10 @@ protected Void saveBlocks(final List blocks) { } } - context.resetBatchSize(); + if (blocks.size() == context.getBatchSize()) { + // reset the batch size only if we got a full batch + context.resetBatchSize(); + } return null; } } From 06515873df9432ff67aba9f04f52b0752a7f0034 Mon Sep 17 00:00:00 2001 From: Fabio Di Fabio Date: Mon, 14 Nov 2022 17:04:08 +0100 Subject: [PATCH 22/32] Halve the batch size in case the request failed Signed-off-by: Fabio Di Fabio --- .../eth/sync/backwardsync/ForwardSyncStep.java | 14 +++++++++++++- 1 file changed, 13 insertions(+), 1 deletion(-) diff --git a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/ForwardSyncStep.java b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/ForwardSyncStep.java index 46fc13754a9..05b8e37b63d 100644 --- a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/ForwardSyncStep.java +++ b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/ForwardSyncStep.java @@ -59,7 +59,19 @@ public CompletableFuture possibleRequestBodies(final List blo () -> blockHeaders.get(0).getNumber(), () -> blockHeaders.get(blockHeaders.size() - 1).getNumber(), () -> blockHeaders.get(0).getHash().toHexString()); - return requestBodies(blockHeaders).thenApply(this::saveBlocks); + return requestBodies(blockHeaders) + .thenApply(this::saveBlocks) + .exceptionally( + throwable -> { + context.halveBatchSize(); + debugLambda( + LOG, + "Getting {} blocks from peers failed with reason {}, reducing batch size to {}", + blockHeaders::size, + throwable::getMessage, + context::getBatchSize); + return null; + }); } } From 4409b6248053eb99b33a95ef744dfe618fb8dfe3 Mon Sep 17 00:00:00 2001 From: Fabio Di Fabio Date: Wed, 16 Nov 2022 12:40:25 +0100 Subject: [PATCH 23/32] Update backward sync head during FcU Signed-off-by: Fabio Di Fabio --- .../merge/blockcreation/MergeCoordinator.java | 3 +- .../backwardsync/BackwardSyncContext.java | 36 +++++++++---------- .../backwardsync/BackwardSyncContextTest.java | 8 +++++ 3 files changed, 27 insertions(+), 20 deletions(-) diff --git a/consensus/merge/src/main/java/org/hyperledger/besu/consensus/merge/blockcreation/MergeCoordinator.java b/consensus/merge/src/main/java/org/hyperledger/besu/consensus/merge/blockcreation/MergeCoordinator.java index 286c9204393..70d5ac83dc3 100644 --- a/consensus/merge/src/main/java/org/hyperledger/besu/consensus/merge/blockcreation/MergeCoordinator.java +++ b/consensus/merge/src/main/java/org/hyperledger/besu/consensus/merge/blockcreation/MergeCoordinator.java @@ -363,7 +363,8 @@ public Optional getOrSyncHeadByHash(final Hash headHash) { if (maybeHeadHeader.isPresent()) { debugLambda(LOG, "BlockHeader {} is already present", maybeHeadHeader.get()::toLogString); } else { - debugLambda(LOG, "appending block hash {} to backward sync", headHash::toHexString); + debugLambda(LOG, "Appending new head block hash {} to backward sync", headHash::toHexString); + backwardSyncContext.updateHead(headHash); backwardSyncContext .syncBackwardsUntil(headHash) .exceptionally(e -> logSyncException(headHash, e)); diff --git a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncContext.java b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncContext.java index b4f37a6e861..f9e3b14a5a7 100644 --- a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncContext.java +++ b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncContext.java @@ -103,25 +103,21 @@ public synchronized boolean isSyncing() { .orElse(Boolean.FALSE); } - public synchronized void updateHeads(final Hash head) { - if (Hash.ZERO.equals(head)) { - this.maybeHead = Optional.empty(); - } else { - this.maybeHead = Optional.of(head); - } + public synchronized void updateHead(final Hash head) { + Optional maybeCurrentStatus = Optional.ofNullable(this.currentBackwardSyncStatus.get()); + maybeCurrentStatus.ifPresent( + status -> + backwardChain + .getBlock(head) + .ifPresent(block -> status.updateTargetHeight(block.getHeader().getNumber()))); + this.maybeHead = Optional.of(head); } public synchronized CompletableFuture syncBackwardsUntil(final Hash newBlockHash) { Optional maybeCurrentStatus = Optional.ofNullable(this.currentBackwardSyncStatus.get()); if (isTrusted(newBlockHash)) { return maybeCurrentStatus - .map( - status -> { - backwardChain - .getBlock(newBlockHash) - .ifPresent(block -> status.updateTargetHeight(block.getHeader().getNumber())); - return status.currentFuture; - }) + .map(Status::getCurrentFuture) .orElseGet(() -> CompletableFuture.completedFuture(null)); } backwardChain.addNewHash(newBlockHash); @@ -290,7 +286,7 @@ public void subscribeBadChainListener(final BadChainListener badChainListener) { } // In rare case when we request too many headers/blocks we get response that does not contain all - // data and we might want to retry with smaller batch size + // data, and we might want to retry with smaller batch size public int getBatchSize() { return batchSize; } @@ -339,17 +335,19 @@ protected void possiblyMoveHead(final Block lastSavedBlock) { LOG.debug("Nothing to do with the head"); return; } - if (blockchain.getChainHead().getHash().equals(maybeHead.get())) { + + final Hash head = maybeHead.get(); + if (blockchain.getChainHead().getHash().equals(head)) { LOG.debug("Head is already properly set"); return; } - if (blockchain.contains(maybeHead.get())) { - LOG.debug("Changing head to {}", maybeHead.get().toHexString()); - blockchain.rewindToBlock(maybeHead.get()); + if (blockchain.contains(head)) { + LOG.debug("Changing head to {}", head); + blockchain.rewindToBlock(head); return; } if (blockchain.getChainHead().getHash().equals(lastSavedBlock.getHash())) { - LOG.debug("Rewinding head to lastSavedBlock {}", lastSavedBlock.getHash()); + debugLambda(LOG, "Rewinding head to lastSavedBlock {}", lastSavedBlock::toLogString); blockchain.rewindToBlock(lastSavedBlock.getHash()); } } diff --git a/ethereum/eth/src/test/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncContextTest.java b/ethereum/eth/src/test/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncContextTest.java index 897786ce94b..61b5c235bdb 100644 --- a/ethereum/eth/src/test/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncContextTest.java +++ b/ethereum/eth/src/test/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncContextTest.java @@ -247,6 +247,14 @@ private Block getBlockByNumber(final int number) { return remoteBlockchain.getBlockByNumber(number).orElseThrow(); } + @Test + public void testUpdatingHead() { + context.updateHead(localBlockchain.getBlockByNumber(4).orElseThrow().getHash()); + context.possiblyMoveHead(null); + + assertThat(localBlockchain.getChainHeadBlock().getHeader().getNumber()).isEqualTo(4); + } + @Test public void shouldProcessExceptionsCorrectly() { assertThatThrownBy( From 14bd4c4f4ec30a0a008e4dac37051aafa8448759 Mon Sep 17 00:00:00 2001 From: Fabio Di Fabio Date: Wed, 16 Nov 2022 13:44:29 +0100 Subject: [PATCH 24/32] Add CHANGELOG entry Signed-off-by: Fabio Di Fabio --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 272b4bbbd17..99b5a735869 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -18,6 +18,7 @@ - Shanghai implementation of EIP-3540 and EIP-3670 Ethereum Object Format and Code Validation [#4644](https://github.com/hyperledger/besu/pull/4644) ### Bug Fixes +- Restore updating chain head during backward sync [#4688](https://github.com/hyperledger/besu/pull/4688) ### Download Links From f05d1fb75fc276415258a9c8aeadea3beedc1cd7 Mon Sep 17 00:00:00 2001 From: Fabio Di Fabio Date: Thu, 17 Nov 2022 15:00:34 +0100 Subject: [PATCH 25/32] Update the target height whenever a new sync target of head is set Signed-off-by: Fabio Di Fabio --- .../backwardsync/BackwardSyncContext.java | 28 ++++++++++++++----- 1 file changed, 21 insertions(+), 7 deletions(-) diff --git a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncContext.java b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncContext.java index f9e3b14a5a7..b7b80de3a07 100644 --- a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncContext.java +++ b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncContext.java @@ -105,11 +105,7 @@ public synchronized boolean isSyncing() { public synchronized void updateHead(final Hash head) { Optional maybeCurrentStatus = Optional.ofNullable(this.currentBackwardSyncStatus.get()); - maybeCurrentStatus.ifPresent( - status -> - backwardChain - .getBlock(head) - .ifPresent(block -> status.updateTargetHeight(block.getHeader().getNumber()))); + maybeCurrentStatus.ifPresent(status -> updateTargetHeight(status, head)); this.maybeHead = Optional.of(head); } @@ -117,7 +113,11 @@ public synchronized CompletableFuture syncBackwardsUntil(final Hash newBlo Optional maybeCurrentStatus = Optional.ofNullable(this.currentBackwardSyncStatus.get()); if (isTrusted(newBlockHash)) { return maybeCurrentStatus - .map(Status::getCurrentFuture) + .map( + status -> { + updateTargetHeight(status, newBlockHash); + return status.currentFuture; + }) .orElseGet(() -> CompletableFuture.completedFuture(null)); } backwardChain.addNewHash(newBlockHash); @@ -141,7 +141,11 @@ public synchronized CompletableFuture syncBackwardsUntil(final Block newPi } backwardChain.appendTrustedBlock(newPivot); return maybeCurrentStatus - .map(Status::getCurrentFuture) + .map( + status -> { + updateTargetHeight(status, newPivot); + return status.currentFuture; + }) .orElseGet( () -> { LOG.info("Starting a new backward sync session"); @@ -155,6 +159,16 @@ public synchronized CompletableFuture syncBackwardsUntil(final Block newPi }); } + private void updateTargetHeight(final Status status, final Hash targetHash) { + backwardChain + .getBlock(targetHash) + .ifPresent(targetBlock -> updateTargetHeight(status, targetBlock)); + } + + private void updateTargetHeight(final Status status, final Block targetBlock) { + status.updateTargetHeight(targetBlock.getHeader().getNumber()); + } + private boolean isTrusted(final Hash hash) { if (backwardChain.isTrusted(hash)) { debugLambda( From a73590bb66a4a6473f375943102ccadaa255dbfc Mon Sep 17 00:00:00 2001 From: Fabio Di Fabio Date: Thu, 17 Nov 2022 15:54:55 +0100 Subject: [PATCH 26/32] remove logs that keep objects live until backward sync is done Signed-off-by: Fabio Di Fabio --- .../merge/blockcreation/MergeCoordinator.java | 13 +------------ .../internal/methods/engine/EngineNewPayload.java | 13 ++----------- 2 files changed, 3 insertions(+), 23 deletions(-) diff --git a/consensus/merge/src/main/java/org/hyperledger/besu/consensus/merge/blockcreation/MergeCoordinator.java b/consensus/merge/src/main/java/org/hyperledger/besu/consensus/merge/blockcreation/MergeCoordinator.java index 70d5ac83dc3..153e2e76c4d 100644 --- a/consensus/merge/src/main/java/org/hyperledger/besu/consensus/merge/blockcreation/MergeCoordinator.java +++ b/consensus/merge/src/main/java/org/hyperledger/besu/consensus/merge/blockcreation/MergeCoordinator.java @@ -365,22 +365,11 @@ public Optional getOrSyncHeadByHash(final Hash headHash) { } else { debugLambda(LOG, "Appending new head block hash {} to backward sync", headHash::toHexString); backwardSyncContext.updateHead(headHash); - backwardSyncContext - .syncBackwardsUntil(headHash) - .exceptionally(e -> logSyncException(headHash, e)); + backwardSyncContext.syncBackwardsUntil(headHash); } return maybeHeadHeader; } - private Void logSyncException(final Hash blockHash, final Throwable exception) { - debugLambda( - LOG, - "Sync to block hash {} failed, reason {}", - blockHash::toHexString, - exception::getMessage); - return null; - } - @Override public BlockProcessingResult validateBlock(final Block block) { diff --git a/ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/methods/engine/EngineNewPayload.java b/ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/methods/engine/EngineNewPayload.java index f0636c02d1b..0b8bd914539 100644 --- a/ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/methods/engine/EngineNewPayload.java +++ b/ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/methods/engine/EngineNewPayload.java @@ -187,17 +187,8 @@ public JsonRpcResponse syncResponse(final JsonRpcRequestContext requestContext) mergeContext.get().isSyncing(), parentHeader.isEmpty(), block.getHash()); - mergeCoordinator - .appendNewPayloadToSync(block) - .exceptionally( - exception -> { - debugLambda( - LOG, - "Sync to block {} failed, reason {}", - block::toLogString, - exception::getMessage); - return null; - }); + mergeCoordinator.appendNewPayloadToSync(block); + return respondWith(reqId, blockParam, null, SYNCING); } From 3f0d3fb0ae43f848295ae04d8e005690b63cb741 Mon Sep 17 00:00:00 2001 From: Fabio Di Fabio Date: Thu, 17 Nov 2022 15:57:45 +0100 Subject: [PATCH 27/32] Revert "Update the target height whenever a new sync target of head is set" This reverts commit f05d1fb75fc276415258a9c8aeadea3beedc1cd7. Signed-off-by: Fabio Di Fabio --- .../backwardsync/BackwardSyncContext.java | 28 +++++-------------- 1 file changed, 7 insertions(+), 21 deletions(-) diff --git a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncContext.java b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncContext.java index b7b80de3a07..f9e3b14a5a7 100644 --- a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncContext.java +++ b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncContext.java @@ -105,7 +105,11 @@ public synchronized boolean isSyncing() { public synchronized void updateHead(final Hash head) { Optional maybeCurrentStatus = Optional.ofNullable(this.currentBackwardSyncStatus.get()); - maybeCurrentStatus.ifPresent(status -> updateTargetHeight(status, head)); + maybeCurrentStatus.ifPresent( + status -> + backwardChain + .getBlock(head) + .ifPresent(block -> status.updateTargetHeight(block.getHeader().getNumber()))); this.maybeHead = Optional.of(head); } @@ -113,11 +117,7 @@ public synchronized CompletableFuture syncBackwardsUntil(final Hash newBlo Optional maybeCurrentStatus = Optional.ofNullable(this.currentBackwardSyncStatus.get()); if (isTrusted(newBlockHash)) { return maybeCurrentStatus - .map( - status -> { - updateTargetHeight(status, newBlockHash); - return status.currentFuture; - }) + .map(Status::getCurrentFuture) .orElseGet(() -> CompletableFuture.completedFuture(null)); } backwardChain.addNewHash(newBlockHash); @@ -141,11 +141,7 @@ public synchronized CompletableFuture syncBackwardsUntil(final Block newPi } backwardChain.appendTrustedBlock(newPivot); return maybeCurrentStatus - .map( - status -> { - updateTargetHeight(status, newPivot); - return status.currentFuture; - }) + .map(Status::getCurrentFuture) .orElseGet( () -> { LOG.info("Starting a new backward sync session"); @@ -159,16 +155,6 @@ public synchronized CompletableFuture syncBackwardsUntil(final Block newPi }); } - private void updateTargetHeight(final Status status, final Hash targetHash) { - backwardChain - .getBlock(targetHash) - .ifPresent(targetBlock -> updateTargetHeight(status, targetBlock)); - } - - private void updateTargetHeight(final Status status, final Block targetBlock) { - status.updateTargetHeight(targetBlock.getHeader().getNumber()); - } - private boolean isTrusted(final Hash hash) { if (backwardChain.isTrusted(hash)) { debugLambda( From d8a6a29e80aee9a937c93b28227831a3b8404a38 Mon Sep 17 00:00:00 2001 From: Fabio Di Fabio Date: Tue, 22 Nov 2022 11:20:36 +0100 Subject: [PATCH 28/32] Set finalized block in merge context when backward sync is done Signed-off-by: Fabio Di Fabio --- .../merge/blockcreation/MergeCoordinator.java | 49 ++++++++++++++----- .../blockcreation/MergeMiningCoordinator.java | 2 +- .../blockcreation/TransitionCoordinator.java | 4 +- .../blockcreation/MergeCoordinatorTest.java | 4 +- .../engine/EngineForkchoiceUpdated.java | 3 +- .../engine/EngineForkchoiceUpdatedTest.java | 30 +++++++----- .../eth/sync/backwardsync/BackwardChain.java | 15 ++++-- .../backwardsync/BackwardSyncContext.java | 20 +++++--- .../backwardsync/BackwardsSyncAlgorithm.java | 5 +- 9 files changed, 87 insertions(+), 45 deletions(-) diff --git a/consensus/merge/src/main/java/org/hyperledger/besu/consensus/merge/blockcreation/MergeCoordinator.java b/consensus/merge/src/main/java/org/hyperledger/besu/consensus/merge/blockcreation/MergeCoordinator.java index a178f971a76..8e17715bf75 100644 --- a/consensus/merge/src/main/java/org/hyperledger/besu/consensus/merge/blockcreation/MergeCoordinator.java +++ b/consensus/merge/src/main/java/org/hyperledger/besu/consensus/merge/blockcreation/MergeCoordinator.java @@ -390,7 +390,7 @@ private boolean canRetryBlockCreation(final Throwable throwable) { } @Override - public Optional getOrSyncHeadByHash(final Hash headHash) { + public Optional getOrSyncHeadByHash(final Hash headHash, final Hash finalizedHash) { final var chain = protocolContext.getBlockchain(); final var maybeHeadHeader = chain.getBlockHeader(headHash); @@ -399,11 +399,38 @@ public Optional getOrSyncHeadByHash(final Hash headHash) { } else { debugLambda(LOG, "Appending new head block hash {} to backward sync", headHash::toHexString); backwardSyncContext.updateHead(headHash); - backwardSyncContext.syncBackwardsUntil(headHash); + backwardSyncContext + .syncBackwardsUntil(headHash) + .thenRun(() -> updateFinalized(finalizedHash)); } return maybeHeadHeader; } + private void updateFinalized(final Hash finalizedHash) { + if (mergeContext + .getFinalized() + .map(BlockHeader::getHash) + .map(finalizedHash::equals) + .orElse(Boolean.FALSE)) { + LOG.debug("Finalized block already set to {}, nothing to do", finalizedHash); + return; + } + + protocolContext + .getBlockchain() + .getBlockHeader(finalizedHash) + .ifPresentOrElse( + finalizedHeader -> { + debugLambda( + LOG, "Setting finalized block header to {}", finalizedHeader::toLogString); + mergeContext.setFinalized(finalizedHeader); + }, + () -> + LOG.warn( + "Internal error, backward sync completed but failed to import finalized block {}", + finalizedHash)); + } + @Override public BlockProcessingResult validateBlock(final Block block) { @@ -678,12 +705,11 @@ private Optional findValidAncestor( @Override public boolean isDescendantOf(final BlockHeader ancestorBlock, final BlockHeader newBlock) { - LOG.debug( - "checking if block {}:{} is ancestor of {}:{}", - ancestorBlock.getNumber(), - ancestorBlock.getBlockHash(), - newBlock.getNumber(), - newBlock.getBlockHash()); + debugLambda( + LOG, + "checking if block {} is ancestor of {}", + ancestorBlock::toLogString, + newBlock::toLogString); // start with self, because descending from yourself is valid Optional parentOf = Optional.of(newBlock); @@ -699,10 +725,11 @@ public boolean isDescendantOf(final BlockHeader ancestorBlock, final BlockHeader && ancestorBlock.getBlockHash().equals(parentOf.get().getBlockHash())) { return true; } else { - LOG.debug( + debugLambda( + LOG, "looped all the way back, did not find ancestor {} of child {}", - ancestorBlock.getBlockHash(), - newBlock.getBlockHash()); + ancestorBlock::toLogString, + newBlock::toLogString); return false; } } diff --git a/consensus/merge/src/main/java/org/hyperledger/besu/consensus/merge/blockcreation/MergeMiningCoordinator.java b/consensus/merge/src/main/java/org/hyperledger/besu/consensus/merge/blockcreation/MergeMiningCoordinator.java index 918bd0b235a..79fedf90734 100644 --- a/consensus/merge/src/main/java/org/hyperledger/besu/consensus/merge/blockcreation/MergeMiningCoordinator.java +++ b/consensus/merge/src/main/java/org/hyperledger/besu/consensus/merge/blockcreation/MergeMiningCoordinator.java @@ -63,7 +63,7 @@ ForkchoiceResult updateForkChoice( CompletableFuture appendNewPayloadToSync(Block newPayload); - Optional getOrSyncHeadByHash(Hash blockHash); + Optional getOrSyncHeadByHash(Hash headHash, Hash finalizedHash); boolean isMiningBeforeMerge(); diff --git a/consensus/merge/src/main/java/org/hyperledger/besu/consensus/merge/blockcreation/TransitionCoordinator.java b/consensus/merge/src/main/java/org/hyperledger/besu/consensus/merge/blockcreation/TransitionCoordinator.java index 6b89d4f9688..ce9e84e7dd2 100644 --- a/consensus/merge/src/main/java/org/hyperledger/besu/consensus/merge/blockcreation/TransitionCoordinator.java +++ b/consensus/merge/src/main/java/org/hyperledger/besu/consensus/merge/blockcreation/TransitionCoordinator.java @@ -183,8 +183,8 @@ public CompletableFuture appendNewPayloadToSync(final Block newPayload) { } @Override - public Optional getOrSyncHeadByHash(final Hash blockHash) { - return mergeCoordinator.getOrSyncHeadByHash(blockHash); + public Optional getOrSyncHeadByHash(final Hash headHash, final Hash finalizedHash) { + return mergeCoordinator.getOrSyncHeadByHash(headHash, finalizedHash); } @Override diff --git a/consensus/merge/src/test/java/org/hyperledger/besu/consensus/merge/blockcreation/MergeCoordinatorTest.java b/consensus/merge/src/test/java/org/hyperledger/besu/consensus/merge/blockcreation/MergeCoordinatorTest.java index e906b52ab07..074002ea2ba 100644 --- a/consensus/merge/src/test/java/org/hyperledger/besu/consensus/merge/blockcreation/MergeCoordinatorTest.java +++ b/consensus/merge/src/test/java/org/hyperledger/besu/consensus/merge/blockcreation/MergeCoordinatorTest.java @@ -698,7 +698,7 @@ public void assertGetOrSyncForBlockAlreadyPresent() { BlockHeader mockHeader = headerGenerator.parentHash(Hash.fromHexStringLenient("0xdead")).buildHeader(); when(blockchain.getBlockHeader(mockHeader.getHash())).thenReturn(Optional.of(mockHeader)); - var res = coordinator.getOrSyncHeadByHash(mockHeader.getHash()); + var res = coordinator.getOrSyncHeadByHash(mockHeader.getHash(), Hash.ZERO); assertThat(res).isPresent(); } @@ -710,7 +710,7 @@ public void assertGetOrSyncForBlockNotPresent() { when(backwardSyncContext.syncBackwardsUntil(mockHeader.getBlockHash())) .thenReturn(CompletableFuture.completedFuture(null)); - var res = coordinator.getOrSyncHeadByHash(mockHeader.getHash()); + var res = coordinator.getOrSyncHeadByHash(mockHeader.getHash(), Hash.ZERO); assertThat(res).isNotPresent(); } diff --git a/ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/methods/engine/EngineForkchoiceUpdated.java b/ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/methods/engine/EngineForkchoiceUpdated.java index 9789f4e2f12..64c427da1e8 100644 --- a/ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/methods/engine/EngineForkchoiceUpdated.java +++ b/ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/methods/engine/EngineForkchoiceUpdated.java @@ -100,7 +100,8 @@ public JsonRpcResponse syncResponse(final JsonRpcRequestContext requestContext) } final Optional maybeNewHead = - mergeCoordinator.getOrSyncHeadByHash(forkChoice.getHeadBlockHash()); + mergeCoordinator.getOrSyncHeadByHash( + forkChoice.getHeadBlockHash(), forkChoice.getFinalizedBlockHash()); if (maybeNewHead.isEmpty()) { return syncingResponse(requestId, forkChoice); diff --git a/ethereum/api/src/test/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/methods/engine/EngineForkchoiceUpdatedTest.java b/ethereum/api/src/test/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/methods/engine/EngineForkchoiceUpdatedTest.java index 019e63940ea..6ad01828fb4 100644 --- a/ethereum/api/src/test/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/methods/engine/EngineForkchoiceUpdatedTest.java +++ b/ethereum/api/src/test/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/methods/engine/EngineForkchoiceUpdatedTest.java @@ -111,7 +111,7 @@ public void shouldReturnSyncingIfMissingNewHead() { public void shouldReturnInvalidOnBadTerminalBlock() { BlockHeader mockHeader = new BlockHeaderTestFixture().baseFeePerGas(Wei.ONE).buildHeader(); - when(mergeCoordinator.getOrSyncHeadByHash(mockHeader.getHash())) + when(mergeCoordinator.getOrSyncHeadByHash(mockHeader.getHash(), Hash.ZERO)) .thenReturn(Optional.of(mockHeader)); when(mergeCoordinator.latestValidAncestorDescendsFromTerminal(mockHeader)).thenReturn(false); assertSuccessWithPayloadForForkchoiceResult( @@ -148,7 +148,7 @@ public void shouldReturnSyncingOnHeadNotFound() { @Test public void shouldReturnValidWithoutFinalizedOrPayload() { BlockHeader mockHeader = new BlockHeaderTestFixture().baseFeePerGas(Wei.ONE).buildHeader(); - when(mergeCoordinator.getOrSyncHeadByHash(mockHeader.getHash())) + when(mergeCoordinator.getOrSyncHeadByHash(mockHeader.getHash(), Hash.ZERO)) .thenReturn(Optional.of(mockHeader)); when(mergeCoordinator.latestValidAncestorDescendsFromTerminal(mockHeader)).thenReturn(true); @@ -172,7 +172,7 @@ public void shouldReturnInvalidOnOldTimestamp() { when(mergeCoordinator.latestValidAncestorDescendsFromTerminal(mockHeader)).thenReturn(true); when(mergeCoordinator.isDescendantOf(any(), any())).thenReturn(true); when(mergeContext.isSyncing()).thenReturn(false); - when(mergeCoordinator.getOrSyncHeadByHash(mockHeader.getHash())) + when(mergeCoordinator.getOrSyncHeadByHash(mockHeader.getHash(), parent.getHash())) .thenReturn(Optional.of(mockHeader)); when(mergeCoordinator.updateForkChoice( mockHeader, parent.getHash(), parent.getHash(), Optional.empty())) @@ -202,7 +202,7 @@ public void shouldReturnValidWithNewHeadAndFinalizedNoPayload() { BlockHeader mockParent = builder.number(9L).buildHeader(); BlockHeader mockHeader = builder.number(10L).parentHash(mockParent.getHash()).buildHeader(); when(blockchain.getBlockHeader(any())).thenReturn(Optional.of(mockHeader)); - when(mergeCoordinator.getOrSyncHeadByHash(mockHeader.getHash())) + when(mergeCoordinator.getOrSyncHeadByHash(mockHeader.getHash(), Hash.ZERO)) .thenReturn(Optional.of(mockHeader)); when(mergeCoordinator.latestValidAncestorDescendsFromTerminal(mockHeader)).thenReturn(true); when(mergeCoordinator.isDescendantOf(any(), any())).thenReturn(true); @@ -217,7 +217,7 @@ public void shouldReturnValidWithNewHeadAndFinalizedNoPayload() { @Test public void shouldReturnValidWithoutFinalizedWithPayload() { BlockHeader mockHeader = new BlockHeaderTestFixture().baseFeePerGas(Wei.ONE).buildHeader(); - when(mergeCoordinator.getOrSyncHeadByHash(mockHeader.getHash())) + when(mergeCoordinator.getOrSyncHeadByHash(mockHeader.getHash(), Hash.ZERO)) .thenReturn(Optional.of(mockHeader)); when(mergeCoordinator.latestValidAncestorDescendsFromTerminal(mockHeader)).thenReturn(true); @@ -254,7 +254,8 @@ public void shouldReturnInvalidForkchoiceStateIfFinalizedBlockIsUnknown() { when(blockchain.getBlockHeader(finalizedBlockHash)).thenReturn(Optional.empty()); when(mergeContext.isSyncing()).thenReturn(false); - when(mergeCoordinator.getOrSyncHeadByHash(newHead.getHash())).thenReturn(Optional.of(newHead)); + when(mergeCoordinator.getOrSyncHeadByHash(newHead.getHash(), finalizedBlockHash)) + .thenReturn(Optional.of(newHead)); var resp = resp( @@ -274,7 +275,8 @@ public void shouldReturnInvalidForkchoiceStateIfFinalizedBlockIsNotAnAncestorOfN when(blockchain.getBlockHeader(newHead.getHash())).thenReturn(Optional.of(newHead)); when(blockchain.getBlockHeader(finalized.getHash())).thenReturn(Optional.of(finalized)); when(mergeContext.isSyncing()).thenReturn(false); - when(mergeCoordinator.getOrSyncHeadByHash(newHead.getHash())).thenReturn(Optional.of(newHead)); + when(mergeCoordinator.getOrSyncHeadByHash(newHead.getHash(), finalized.getBlockHash())) + .thenReturn(Optional.of(newHead)); when(mergeCoordinator.isDescendantOf(finalized, newHead)).thenReturn(false); var resp = @@ -299,7 +301,8 @@ public void shouldReturnInvalidForkchoiceStateIfSafeHeadZeroWithFinalizedBlock() when(blockchain.getBlockHeader(parent.getHash())).thenReturn(Optional.of(parent)); when(mergeContext.isSyncing()).thenReturn(false); - when(mergeCoordinator.getOrSyncHeadByHash(newHead.getHash())).thenReturn(Optional.of(newHead)); + when(mergeCoordinator.getOrSyncHeadByHash(newHead.getHash(), parent.getBlockHash())) + .thenReturn(Optional.of(newHead)); var resp = resp( @@ -325,7 +328,8 @@ public void shouldReturnInvalidForkchoiceStateIfSafeBlockIsUnknown() { when(blockchain.getBlockHeader(finalized.getHash())).thenReturn(Optional.of(finalized)); when(blockchain.getBlockHeader(safeBlockBlockHash)).thenReturn(Optional.empty()); when(mergeContext.isSyncing()).thenReturn(false); - when(mergeCoordinator.getOrSyncHeadByHash(newHead.getHash())).thenReturn(Optional.of(newHead)); + when(mergeCoordinator.getOrSyncHeadByHash(newHead.getHash(), finalized.getBlockHash())) + .thenReturn(Optional.of(newHead)); when(mergeCoordinator.isDescendantOf(finalized, newHead)).thenReturn(true); var resp = @@ -348,7 +352,8 @@ public void shouldReturnInvalidForkchoiceStateIfSafeBlockIsNotADescendantOfFinal when(blockchain.getBlockHeader(finalized.getHash())).thenReturn(Optional.of(finalized)); when(blockchain.getBlockHeader(safeBlock.getHash())).thenReturn(Optional.of(safeBlock)); when(mergeContext.isSyncing()).thenReturn(false); - when(mergeCoordinator.getOrSyncHeadByHash(newHead.getHash())).thenReturn(Optional.of(newHead)); + when(mergeCoordinator.getOrSyncHeadByHash(newHead.getHash(), finalized.getBlockHash())) + .thenReturn(Optional.of(newHead)); when(mergeCoordinator.isDescendantOf(finalized, newHead)).thenReturn(true); when(mergeCoordinator.isDescendantOf(finalized, safeBlock)).thenReturn(false); @@ -372,7 +377,8 @@ public void shouldReturnInvalidForkchoiceStateIfSafeBlockIsNotAnAncestorOfNewHea when(blockchain.getBlockHeader(finalized.getHash())).thenReturn(Optional.of(finalized)); when(blockchain.getBlockHeader(safeBlock.getHash())).thenReturn(Optional.of(safeBlock)); when(mergeContext.isSyncing()).thenReturn(false); - when(mergeCoordinator.getOrSyncHeadByHash(newHead.getHash())).thenReturn(Optional.of(newHead)); + when(mergeCoordinator.getOrSyncHeadByHash(newHead.getHash(), finalized.getBlockHash())) + .thenReturn(Optional.of(newHead)); when(mergeCoordinator.isDescendantOf(finalized, newHead)).thenReturn(true); when(mergeCoordinator.isDescendantOf(finalized, safeBlock)).thenReturn(true); when(mergeCoordinator.isDescendantOf(safeBlock, newHead)).thenReturn(false); @@ -391,7 +397,7 @@ public void shouldReturnInvalidForkchoiceStateIfSafeBlockIsNotAnAncestorOfNewHea public void shouldIgnoreUpdateToOldHeadAndNotPreparePayload() { BlockHeader mockHeader = new BlockHeaderTestFixture().baseFeePerGas(Wei.ONE).buildHeader(); - when(mergeCoordinator.getOrSyncHeadByHash(mockHeader.getHash())) + when(mergeCoordinator.getOrSyncHeadByHash(mockHeader.getHash(), Hash.ZERO)) .thenReturn(Optional.of(mockHeader)); when(mergeCoordinator.latestValidAncestorDescendsFromTerminal(mockHeader)).thenReturn(true); diff --git a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardChain.java b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardChain.java index 94dfa89ef66..893c9547a38 100644 --- a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardChain.java +++ b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardChain.java @@ -93,15 +93,14 @@ public synchronized void prependAncestorsHeader(final BlockHeader blockHeader) { headers.put(blockHeader.getHash(), blockHeader); return; } - BlockHeader firstHeader = firstStoredAncestor.get(); + final BlockHeader firstHeader = firstStoredAncestor.get(); headers.put(blockHeader.getHash(), blockHeader); - chainStorage.put(blockHeader.getHash(), firstStoredAncestor.get().getHash()); + chainStorage.put(blockHeader.getHash(), firstHeader.getHash()); firstStoredAncestor = Optional.of(blockHeader); debugLambda( LOG, - "Added header {} on height {} to backward chain led by pivot {} on height {}", + "Added header {} to backward chain led by pivot {} on height {}", blockHeader::toLogString, - blockHeader::getNumber, () -> lastStoredPivot.orElseThrow().toLogString(), firstHeader::getNumber); } @@ -127,13 +126,19 @@ public synchronized void dropFirstHeader() { } public synchronized void appendTrustedBlock(final Block newPivot) { - debugLambda(LOG, "appending trusted block {}", newPivot::toLogString); + debugLambda(LOG, "Appending trusted block {}", newPivot::toLogString); headers.put(newPivot.getHash(), newPivot.getHeader()); blocks.put(newPivot.getHash(), newPivot); if (lastStoredPivot.isEmpty()) { firstStoredAncestor = Optional.of(newPivot.getHeader()); } else { if (newPivot.getHeader().getParentHash().equals(lastStoredPivot.get().getHash())) { + debugLambda( + LOG, + "Added block {} to backward chain led by pivot {} on height {}", + newPivot::toLogString, + lastStoredPivot.get()::toLogString, + firstStoredAncestor.get()::getNumber); chainStorage.put(lastStoredPivot.get().getHash(), newPivot.getHash()); } else { firstStoredAncestor = Optional.of(newPivot.getHeader()); diff --git a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncContext.java b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncContext.java index dc1a33fa8fd..c351f93ee1b 100644 --- a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncContext.java +++ b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncContext.java @@ -103,14 +103,18 @@ public synchronized boolean isSyncing() { .orElse(Boolean.FALSE); } - public synchronized void updateHead(final Hash head) { - Optional maybeCurrentStatus = Optional.ofNullable(this.currentBackwardSyncStatus.get()); - maybeCurrentStatus.ifPresent( - status -> - backwardChain - .getBlock(head) - .ifPresent(block -> status.updateTargetHeight(block.getHeader().getNumber()))); - this.maybeHead = Optional.of(head); + public synchronized void updateHead(final Hash headHash) { + if (Hash.ZERO.equals(headHash)) { + maybeHead = Optional.empty(); + } else { + maybeHead = Optional.of(headHash); + Optional maybeCurrentStatus = Optional.ofNullable(currentBackwardSyncStatus.get()); + maybeCurrentStatus.ifPresent( + status -> + backwardChain + .getBlock(headHash) + .ifPresent(block -> status.updateTargetHeight(block.getHeader().getNumber()))); + } } public synchronized CompletableFuture syncBackwardsUntil(final Hash newBlockHash) { diff --git a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardsSyncAlgorithm.java b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardsSyncAlgorithm.java index 687b0652dc0..a87af03bbb3 100644 --- a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardsSyncAlgorithm.java +++ b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardsSyncAlgorithm.java @@ -73,12 +73,11 @@ public CompletableFuture pickNextStep() { if (!context.isReady()) { return waitForReady(); } - final Optional maybeFirstAncestorHeader = context.getBackwardChain().getFirstAncestorHeader(); if (maybeFirstAncestorHeader.isEmpty()) { this.finished = true; - LOG.info("The Backward sync is done"); + LOG.info("Current backward sync session is done"); context.getBackwardChain().clear(); return CompletableFuture.completedFuture(null); } @@ -100,7 +99,7 @@ public CompletableFuture pickNextStep() { if (finalBlockConfirmation.ancestorHeaderReached(firstAncestorHeader)) { debugLambda( LOG, - "Backward sync reached ancestor header with {}, starting Forward sync", + "Backward sync reached ancestor header with {}, starting forward sync", firstAncestorHeader::toLogString); return executeForwardAsync(); } From bf62808c18a4d4712872ff2379b174c33ae24c68 Mon Sep 17 00:00:00 2001 From: Fabio Di Fabio Date: Tue, 22 Nov 2022 11:27:41 +0100 Subject: [PATCH 29/32] Update CHANGELOG Signed-off-by: Fabio Di Fabio --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 0886db06b56..ddeba958584 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -21,7 +21,7 @@ - Add field `type` to Transaction receipt object (eth_getTransactionReceipt) [#4505](https://github.com/hyperledger/besu/issues/4505) ### Bug Fixes -- Restore updating chain head during backward sync [#4688](https://github.com/hyperledger/besu/pull/4688) +- Restore updating chain head and finalized block during backward sync [#4718](https://github.com/hyperledger/besu/pull/4718) ### Download Links From f67c4f116444568882c7090e724c4469405395f0 Mon Sep 17 00:00:00 2001 From: Fabio Di Fabio Date: Wed, 23 Nov 2022 12:25:40 +0100 Subject: [PATCH 30/32] Fix: move the head to the last imported block Signed-off-by: Fabio Di Fabio --- .../eth/sync/backwardsync/BackwardSyncContext.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncContext.java b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncContext.java index c351f93ee1b..05724c5d67f 100644 --- a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncContext.java +++ b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncContext.java @@ -345,15 +345,15 @@ protected void possiblyMoveHead(final Block lastSavedBlock) { LOG.debug("Head is already properly set"); return; } + if (blockchain.contains(head)) { LOG.debug("Changing head to {}", head); blockchain.rewindToBlock(head); return; } - if (blockchain.getChainHead().getHash().equals(lastSavedBlock.getHash())) { - debugLambda(LOG, "Rewinding head to lastSavedBlock {}", lastSavedBlock::toLogString); - blockchain.rewindToBlock(lastSavedBlock.getHash()); - } + + debugLambda(LOG, "Rewinding head to last saved block {}", lastSavedBlock::toLogString); + blockchain.rewindToBlock(lastSavedBlock.getHash()); } public SyncState getSyncState() { From 1763a8aaa191739e093366a02930eae3728fe1a6 Mon Sep 17 00:00:00 2001 From: Fabio Di Fabio Date: Thu, 24 Nov 2022 10:34:35 +0100 Subject: [PATCH 31/32] Optimize edge case when CL sends new payload out of order It could happens that CL sends new payload out of order, creating a temporary gap in the chain, this could cause to redownload some block headers that are already present, and create glitches in the progession log. This changes, just check if the headers are already present before trying to download them. Signed-off-by: Fabio Di Fabio --- .../besu/ethereum/eth/sync/backwardsync/BackwardChain.java | 1 + .../ethereum/eth/sync/backwardsync/BackwardSyncStep.java | 6 ++++++ 2 files changed, 7 insertions(+) diff --git a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardChain.java b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardChain.java index 893c9547a38..2844c461b04 100644 --- a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardChain.java +++ b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardChain.java @@ -142,6 +142,7 @@ public synchronized void appendTrustedBlock(final Block newPivot) { chainStorage.put(lastStoredPivot.get().getHash(), newPivot.getHash()); } else { firstStoredAncestor = Optional.of(newPivot.getHeader()); + debugLambda(LOG, "Re-pivoting to new target block {}", newPivot::toLogString); } } lastStoredPivot = Optional.of(newPivot.getHeader()); diff --git a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncStep.java b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncStep.java index 73ac964b530..d36a4d6814c 100644 --- a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncStep.java +++ b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncStep.java @@ -59,6 +59,12 @@ protected Hash possibleRestoreOldNodes(final BlockHeader firstAncestor) { @VisibleForTesting protected CompletableFuture> requestHeaders(final Hash hash) { + if (context.protocolContext.getBlockchain().contains(hash)) { + LOG.debug( + "Hash {} already present in local blockchain no need to request headers to peers", hash); + return CompletableFuture.completedFuture(List.of()); + } + final int batchSize = context.getBatchSize(); LOG.debug("Requesting headers for hash {}, with batch size {}", hash, batchSize); From 5185dc0526ac923cdfa88c43244b2545abb2576a Mon Sep 17 00:00:00 2001 From: Fabio Di Fabio Date: Thu, 24 Nov 2022 11:27:26 +0100 Subject: [PATCH 32/32] Fix and more tests Signed-off-by: Fabio Di Fabio --- .../eth/sync/backwardsync/BackwardSyncStep.java | 2 +- .../eth/sync/backwardsync/BackwardSyncStepTest.java | 11 +++++++++++ 2 files changed, 12 insertions(+), 1 deletion(-) diff --git a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncStep.java b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncStep.java index d36a4d6814c..dd05a02dd0f 100644 --- a/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncStep.java +++ b/ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncStep.java @@ -59,7 +59,7 @@ protected Hash possibleRestoreOldNodes(final BlockHeader firstAncestor) { @VisibleForTesting protected CompletableFuture> requestHeaders(final Hash hash) { - if (context.protocolContext.getBlockchain().contains(hash)) { + if (context.getProtocolContext().getBlockchain().contains(hash)) { LOG.debug( "Hash {} already present in local blockchain no need to request headers to peers", hash); return CompletableFuture.completedFuture(List.of()); diff --git a/ethereum/eth/src/test/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncStepTest.java b/ethereum/eth/src/test/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncStepTest.java index 7f9e5b7c5b5..17eb2e45a92 100644 --- a/ethereum/eth/src/test/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncStepTest.java +++ b/ethereum/eth/src/test/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncStepTest.java @@ -163,6 +163,17 @@ public void shouldRequestHeaderWhenAsked() throws Exception { assertThat(blockHeader).isEqualTo(lookingForBlock.getHeader()); } + @Test + public void shouldNotRequestHeaderIfAlreadyPresent() throws Exception { + BackwardSyncStep step = new BackwardSyncStep(context, createBackwardChain(REMOTE_HEIGHT - 1)); + final Block lookingForBlock = getBlockByNumber(LOCAL_HEIGHT); + + final CompletableFuture> future = + step.requestHeaders(lookingForBlock.getHeader().getHash()); + + assertThat(future.get().isEmpty()).isTrue(); + } + @Test public void shouldRequestHeaderBeforeCurrentHeight() throws Exception { extendBlockchain(REMOTE_HEIGHT + 1, context.getProtocolContext().getBlockchain());