Batching backward sync (#3532)

* Backward sync now batches requests

Signed-off-by: Jiri Peinlich <jiri.peinlich@gmail.com>
pull/3617/head
Jiri Peinlich 3 years ago committed by GitHub
parent a8d13c31c9
commit 0fce76cc9f
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
  1. 14
      besu/src/main/java/org/hyperledger/besu/controller/MergeBesuControllerBuilder.java
  2. 17
      consensus/merge/src/main/java/org/hyperledger/besu/consensus/merge/blockcreation/MergeCoordinator.java
  3. 12
      consensus/merge/src/test/java/org/hyperledger/besu/consensus/merge/blockcreation/MergeCoordinatorTest.java
  4. 4
      consensus/merge/src/test/java/org/hyperledger/besu/consensus/merge/blockcreation/MergeReorgTest.java
  5. 3
      ethereum/core/src/main/java/org/hyperledger/besu/ethereum/storage/keyvalue/KeyValueSegmentIdentifier.java
  6. 2
      ethereum/core/src/main/java/org/hyperledger/besu/ethereum/storage/keyvalue/KeyValueStorageProvider.java
  7. 92
      ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/manager/task/RetryingGetHeadersEndingAtFromPeerByHashTask.java
  8. 30
      ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/DefaultSynchronizer.java
  9. 2
      ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/PipelineChainDownloader.java
  10. 204
      ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardChain.java
  11. 151
      ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncContext.java
  12. 19
      ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncException.java
  13. 172
      ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncLookupService.java
  14. 110
      ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncPhase.java
  15. 28
      ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncTask.java
  16. 47
      ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BlocksConvertor.java
  17. 47
      ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BlocksHeadersConvertor.java
  18. 257
      ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/ForwardSyncPhase.java
  19. 147
      ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/ForwardSyncStep.java
  20. 61
      ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/GenericKeyValueStorageFacade.java
  21. 20
      ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/KeyConvertor.java
  22. 22
      ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/ValueConvertor.java
  23. 12
      ethereum/eth/src/main/java/org/hyperledger/besu/ethereum/eth/sync/fullsync/FullSyncDownloader.java
  24. 38
      ethereum/eth/src/test/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncContextTest.java
  25. 132
      ethereum/eth/src/test/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncLookupServiceTest.java
  26. 83
      ethereum/eth/src/test/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncPhaseTest.java
  27. 39
      ethereum/eth/src/test/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncTaskTest.java
  28. 85
      ethereum/eth/src/test/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/ForwardSyncPhaseTest.java
  29. 98
      ethereum/eth/src/test/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/InMemoryBackwardChainTest.java
  30. 21
      util/src/main/java/org/hyperledger/besu/util/Slf4jLambdaHelper.java

@ -28,7 +28,8 @@ import org.hyperledger.besu.ethereum.core.MiningParameters;
import org.hyperledger.besu.ethereum.eth.manager.EthProtocolManager;
import org.hyperledger.besu.ethereum.eth.peervalidation.PeerValidator;
import org.hyperledger.besu.ethereum.eth.peervalidation.RequiredBlocksPeerValidator;
import org.hyperledger.besu.ethereum.eth.sync.backwardsync.BackwardsSyncContext;
import org.hyperledger.besu.ethereum.eth.sync.backwardsync.BackwardSyncContext;
import org.hyperledger.besu.ethereum.eth.sync.backwardsync.BackwardSyncLookupService;
import org.hyperledger.besu.ethereum.eth.sync.state.SyncState;
import org.hyperledger.besu.ethereum.eth.transactions.TransactionPool;
import org.hyperledger.besu.ethereum.mainnet.ProtocolSchedule;
@ -62,8 +63,15 @@ public class MergeBesuControllerBuilder extends BesuControllerBuilder {
protocolSchedule,
transactionPool.getPendingTransactions(),
miningParameters,
new BackwardsSyncContext(
protocolContext, protocolSchedule, metricsSystem, ethProtocolManager.ethContext()));
new BackwardSyncContext(
protocolContext,
protocolSchedule,
metricsSystem,
ethProtocolManager.ethContext(),
syncState,
new BackwardSyncLookupService(
protocolSchedule, ethProtocolManager.ethContext(), metricsSystem, protocolContext),
storageProvider));
}
@Override

@ -16,6 +16,7 @@ package org.hyperledger.besu.consensus.merge.blockcreation;
import static org.hyperledger.besu.consensus.merge.TransitionUtils.isTerminalProofOfWorkBlock;
import static org.hyperledger.besu.util.Slf4jLambdaHelper.debugLambda;
import static org.hyperledger.besu.util.Slf4jLambdaHelper.infoLambda;
import org.hyperledger.besu.consensus.merge.MergeContext;
import org.hyperledger.besu.datatypes.Address;
@ -31,7 +32,7 @@ import org.hyperledger.besu.ethereum.core.BlockHeader;
import org.hyperledger.besu.ethereum.core.Difficulty;
import org.hyperledger.besu.ethereum.core.MiningParameters;
import org.hyperledger.besu.ethereum.core.Transaction;
import org.hyperledger.besu.ethereum.eth.sync.backwardsync.BackwardsSyncContext;
import org.hyperledger.besu.ethereum.eth.sync.backwardsync.BackwardSyncContext;
import org.hyperledger.besu.ethereum.eth.transactions.sorter.AbstractPendingTransactionsSorter;
import org.hyperledger.besu.ethereum.mainnet.AbstractGasLimitSpecification;
import org.hyperledger.besu.ethereum.mainnet.HeaderValidationMode;
@ -59,7 +60,7 @@ public class MergeCoordinator implements MergeMiningCoordinator {
final AtomicReference<Bytes> extraData = new AtomicReference<>(Bytes.fromHexString("0x"));
private final MergeContext mergeContext;
private final ProtocolContext protocolContext;
private final BackwardsSyncContext backwardsSyncContext;
private final BackwardSyncContext backwardSyncContext;
private final ProtocolSchedule protocolSchedule;
public MergeCoordinator(
@ -67,12 +68,12 @@ public class MergeCoordinator implements MergeMiningCoordinator {
final ProtocolSchedule protocolSchedule,
final AbstractPendingTransactionsSorter pendingTransactions,
final MiningParameters miningParams,
final BackwardsSyncContext backwardsSyncContext) {
final BackwardSyncContext backwardSyncContext) {
this.protocolContext = protocolContext;
this.protocolSchedule = protocolSchedule;
this.mergeContext = protocolContext.getConsensusContext(MergeContext.class);
this.miningParameters = miningParams;
this.backwardsSyncContext = backwardsSyncContext;
this.backwardSyncContext = backwardSyncContext;
this.targetGasLimit =
miningParameters
.getTargetGasLimit()
@ -213,8 +214,8 @@ public class MergeCoordinator implements MergeMiningCoordinator {
if (optHeader.isPresent()) {
debugLambda(LOG, "BlockHeader {} is already present", () -> optHeader.get().toLogString());
} else {
debugLambda(LOG, "appending block hash {} to backward sync", blockhash::toHexString);
backwardsSyncContext.syncBackwardsUntil(blockhash);
infoLambda(LOG, "appending block hash {} to backward sync", blockhash::toHexString);
backwardSyncContext.syncBackwardsUntil(blockhash);
}
return optHeader;
}
@ -228,7 +229,7 @@ public class MergeCoordinator implements MergeMiningCoordinator {
.ifPresentOrElse(
blockHeader ->
debugLambda(LOG, "Parent of block {} is already present", block::toLogString),
() -> backwardsSyncContext.syncBackwardsUntil(block));
() -> backwardSyncContext.syncBackwardsUntil(block));
final var validationResult =
protocolSchedule
@ -411,7 +412,7 @@ public class MergeCoordinator implements MergeMiningCoordinator {
@Override
public boolean isBackwardSyncing() {
return backwardsSyncContext.isSyncing();
return backwardSyncContext.isSyncing();
}
@Override

@ -41,7 +41,7 @@ import org.hyperledger.besu.ethereum.core.BlockHeader;
import org.hyperledger.besu.ethereum.core.BlockHeaderTestFixture;
import org.hyperledger.besu.ethereum.core.Difficulty;
import org.hyperledger.besu.ethereum.core.MiningParameters;
import org.hyperledger.besu.ethereum.eth.sync.backwardsync.BackwardsSyncContext;
import org.hyperledger.besu.ethereum.eth.sync.backwardsync.BackwardSyncContext;
import org.hyperledger.besu.ethereum.eth.transactions.sorter.AbstractPendingTransactionsSorter;
import org.hyperledger.besu.ethereum.mainnet.ProtocolSchedule;
import org.hyperledger.besu.ethereum.mainnet.feemarket.BaseFeeMarket;
@ -65,7 +65,7 @@ public class MergeCoordinatorTest implements MergeGenesisConfigHelper {
@Mock AbstractPendingTransactionsSorter mockSorter;
@Mock MergeContext mergeContext;
@Mock BackwardsSyncContext backwardsSyncContext;
@Mock BackwardSyncContext backwardSyncContext;
private MergeCoordinator coordinator;
private ProtocolContext protocolContext;
@ -104,7 +104,7 @@ public class MergeCoordinatorTest implements MergeGenesisConfigHelper {
mockProtocolSchedule,
mockSorter,
new MiningParameters.Builder().coinbase(coinbase).build(),
backwardsSyncContext);
backwardSyncContext);
}
@Test
@ -341,7 +341,7 @@ public class MergeCoordinatorTest implements MergeGenesisConfigHelper {
var res = coordinator.getOrSyncHeaderByHash(mockHeader.getHash());
assertThat(res).isNotPresent();
verify(backwardsSyncContext, times(1)).syncBackwardsUntil(mockHeader.getHash());
verify(backwardSyncContext, times(1)).syncBackwardsUntil(mockHeader.getHash());
}
@Test
@ -394,7 +394,7 @@ public class MergeCoordinatorTest implements MergeGenesisConfigHelper {
mockProtocolSchedule,
mockSorter,
new MiningParameters.Builder().coinbase(coinbase).build(),
mock(BackwardsSyncContext.class));
mock(BackwardSyncContext.class));
var blockZero = mockHeaderBuilder.number(0L).buildHeader();
var blockOne = mockHeaderBuilder.number(1L).parentHash(blockZero.getHash()).buildHeader();
@ -510,7 +510,7 @@ public class MergeCoordinatorTest implements MergeGenesisConfigHelper {
mockProtocolSchedule,
mockSorter,
new MiningParameters.Builder().coinbase(coinbase).build(),
mock(BackwardsSyncContext.class)));
mock(BackwardSyncContext.class)));
return mockCoordinator;
}

@ -33,7 +33,7 @@ import org.hyperledger.besu.ethereum.core.BlockHeader;
import org.hyperledger.besu.ethereum.core.BlockHeaderTestFixture;
import org.hyperledger.besu.ethereum.core.Difficulty;
import org.hyperledger.besu.ethereum.core.MiningParameters;
import org.hyperledger.besu.ethereum.eth.sync.backwardsync.BackwardsSyncContext;
import org.hyperledger.besu.ethereum.eth.sync.backwardsync.BackwardSyncContext;
import org.hyperledger.besu.ethereum.eth.transactions.sorter.AbstractPendingTransactionsSorter;
import org.hyperledger.besu.ethereum.mainnet.BlockHeaderValidator;
import org.hyperledger.besu.ethereum.mainnet.ProtocolSchedule;
@ -87,7 +87,7 @@ public class MergeReorgTest implements MergeGenesisConfigHelper {
mockProtocolSchedule,
mockSorter,
new MiningParameters.Builder().coinbase(coinbase).build(),
mock(BackwardsSyncContext.class));
mock(BackwardSyncContext.class));
mergeContext.setIsPostMerge(genesisState.getBlock().getHeader().getDifficulty());
blockchain.observeBlockAdded(
blockAddedEvent ->

@ -30,7 +30,8 @@ public enum KeyValueSegmentIdentifier implements SegmentIdentifier {
TRIE_BRANCH_STORAGE(new byte[] {9}, new int[] {2}),
TRIE_LOG_STORAGE(new byte[] {10}, new int[] {2}),
GOQUORUM_PRIVATE_WORLD_STATE(new byte[] {11}),
GOQUORUM_PRIVATE_STORAGE(new byte[] {12});
BACKWARD_SYNC_HEADERS(new byte[] {13}),
BACKWARD_SYNC_BLOCKS(new byte[] {14});
private final byte[] id;
private final int[] versionList;

@ -102,7 +102,7 @@ public class KeyValueStorageProvider implements StorageProvider {
@Override
public GoQuorumPrivateStorage createGoQuorumPrivateStorage() {
return new GoQuorumPrivateKeyValueStorage(
getStorageBySegmentIdentifier(KeyValueSegmentIdentifier.GOQUORUM_PRIVATE_STORAGE));
getStorageBySegmentIdentifier(KeyValueSegmentIdentifier.BACKWARD_SYNC_HEADERS));
}
@Override

@ -0,0 +1,92 @@
/*
* 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 RetryingGetHeadersEndingAtFromPeerByHashTask
extends AbstractRetryingPeerTask<List<BlockHeader>> {
private final Hash referenceHash;
private final ProtocolSchedule protocolSchedule;
private final long minimumRequiredBlockNumber;
private final int count;
@VisibleForTesting
RetryingGetHeadersEndingAtFromPeerByHashTask(
final ProtocolSchedule protocolSchedule,
final EthContext ethContext,
final Hash referenceHash,
final long minimumRequiredBlockNumber,
final int count,
final MetricsSystem metricsSystem) {
super(ethContext, 3, List::isEmpty, metricsSystem);
this.protocolSchedule = protocolSchedule;
this.minimumRequiredBlockNumber = minimumRequiredBlockNumber;
this.count = count;
checkNotNull(referenceHash);
this.referenceHash = referenceHash;
}
public static RetryingGetHeadersEndingAtFromPeerByHashTask endingAtHash(
final ProtocolSchedule protocolSchedule,
final EthContext ethContext,
final Hash referenceHash,
final long minimumRequiredBlockNumber,
final int count,
final MetricsSystem metricsSystem) {
return new RetryingGetHeadersEndingAtFromPeerByHashTask(
protocolSchedule,
ethContext,
referenceHash,
minimumRequiredBlockNumber,
count,
metricsSystem);
}
@Override
protected CompletableFuture<List<BlockHeader>> executePeerTask(
final Optional<EthPeer> assignedPeer) {
final AbstractGetHeadersFromPeerTask task =
GetHeadersFromPeerByHashTask.endingAtHash(
protocolSchedule,
getEthContext(),
referenceHash,
minimumRequiredBlockNumber,
count,
getMetricsSystem());
assignedPeer.ifPresent(task::assignPeer);
return executeSubTask(task::run)
.thenApply(
peerResult -> {
result.complete(peerResult.getResult());
return peerResult.getResult();
});
}
}

@ -141,13 +141,7 @@ public class DefaultSynchronizer implements Synchronizer {
} else {
future = startFullSync();
}
future =
future.thenApply(
unused -> {
blockPropagationManager.stop();
running.set(false);
return null;
});
future = future.thenApply(this::finalizeSync);
return future;
} else {
throw new IllegalStateException("Attempt to start an already started synchronizer.");
@ -191,18 +185,7 @@ public class DefaultSynchronizer implements Synchronizer {
private CompletableFuture<Void> startFullSync() {
maybePruner.ifPresent(Pruner::start);
return fullSyncDownloader
.start()
.thenCompose(
unused -> {
maybePruner.ifPresent(Pruner::stop);
return null;
})
.thenApply(
o -> {
maybePruner.ifPresent(Pruner::stop);
return null;
});
return fullSyncDownloader.start();
}
@Override
@ -238,4 +221,13 @@ public class DefaultSynchronizer implements Synchronizer {
public boolean unsubscribeInSync(final long listenerId) {
return syncState.unsubscribeSyncStatus(listenerId);
}
private Void finalizeSync(final Void unused) {
LOG.info("Stopping block propagation.");
blockPropagationManager.stop();
LOG.info("Stopping the pruner.");
maybePruner.ifPresent(Pruner::stop);
running.set(false);
return null;
}
}

@ -110,7 +110,7 @@ public class PipelineChainDownloader implements ChainDownloader {
&& !syncState.hasReachedTerminalDifficulty().orElse(false)) {
return performDownload();
} else {
LOG.info("Chain download complete");
LOG.info("PipelineChain download complete");
return completedFuture(null);
}
}

@ -12,54 +12,99 @@
*
* SPDX-License-Identifier: Apache-2.0
*/
package org.hyperledger.besu.ethereum.eth.sync.backwardsync;
import static org.hyperledger.besu.util.Slf4jLambdaHelper.debugLambda;
import static org.hyperledger.besu.util.Slf4jLambdaHelper.warnLambda;
import static org.slf4j.LoggerFactory.getLogger;
import org.hyperledger.besu.datatypes.Hash;
import org.hyperledger.besu.ethereum.core.Block;
import org.hyperledger.besu.ethereum.core.BlockHeader;
import org.hyperledger.besu.ethereum.core.BlockHeaderFunctions;
import org.hyperledger.besu.ethereum.storage.StorageProvider;
import org.hyperledger.besu.ethereum.storage.keyvalue.KeyValueSegmentIdentifier;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.stream.Collectors;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class BackwardChain { // TODO: this class now stores everything in memory...
private static final Logger LOG = LoggerFactory.getLogger(BackwardChain.class);
private final List<BlockHeader> ancestors = new ArrayList<>();
private final List<Block> successors = new ArrayList<>();
private final Map<Hash, Block> trustedBlocks = new HashMap<>();
public class BackwardChain {
private static final Logger LOG = getLogger(BackwardChain.class);
private final List<Hash> ancestors = new ArrayList<>();
private final List<Hash> successors = new ArrayList<>();
protected final GenericKeyValueStorageFacade<Hash, BlockHeader> headers;
protected final GenericKeyValueStorageFacade<Hash, Block> blocks;
public BackwardChain(
final StorageProvider provider,
final BlockHeaderFunctions blockHeaderFunctions,
final Block pivot) {
this(
new GenericKeyValueStorageFacade<>(
Hash::toArrayUnsafe,
BlocksHeadersConvertor.of(blockHeaderFunctions),
provider.getStorageBySegmentIdentifier(
KeyValueSegmentIdentifier.BACKWARD_SYNC_HEADERS)),
new GenericKeyValueStorageFacade<>(
Hash::toArrayUnsafe,
BlocksConvertor.of(blockHeaderFunctions),
provider.getStorageBySegmentIdentifier(KeyValueSegmentIdentifier.BACKWARD_SYNC_BLOCKS)),
pivot);
}
public BackwardChain(final Block pivot) {
ancestors.add(pivot.getHeader());
successors.add(pivot);
public BackwardChain(
final GenericKeyValueStorageFacade<Hash, BlockHeader> headersStorage,
final GenericKeyValueStorageFacade<Hash, Block> blocksStorage,
final Block pivot) {
this.headers = headersStorage;
this.blocks = blocksStorage;
headersStorage.put(pivot.getHeader().getHash(), pivot.getHeader());
blocksStorage.put(pivot.getHash(), pivot);
ancestors.add(pivot.getHeader().getHash());
successors.add(pivot.getHash());
}
public Optional<BlockHeader> getFirstAncestorHeader() {
if (ancestors.isEmpty()) {
return Optional.empty();
}
return Optional.of(ancestors.get(ancestors.size() - 1));
return headers.get(ancestors.get(ancestors.size() - 1));
}
public List<BlockHeader> getFirstNAncestorHeaders(final int size) {
List<Hash> resultList = new ArrayList<>(size);
for (int i = Math.min(size, ancestors.size()); i > 0; --i) {
resultList.add(ancestors.get(ancestors.size() - i));
}
return resultList.stream()
.map(h -> this.headers.get(h).orElseThrow())
.collect(Collectors.toList());
}
public List<BlockHeader> getAllAncestors() {
return getFirstNAncestorHeaders(ancestors.size());
}
public void saveHeader(final BlockHeader blockHeader) {
public void prependAncestorsHeader(final BlockHeader blockHeader) {
BlockHeader firstHeader =
getFirstAncestorHeader()
.orElseThrow(
() -> new BackwardSyncException("Cannot save more headers during forward sync"));
() ->
new BackwardSyncException(
"Cannot save more headers during forward sync", true));
if (firstHeader.getNumber() != blockHeader.getNumber() + 1) {
throw new BackwardSyncException(
"Wrong height of header "
+ blockHeader.getHash().toString().substring(0, 20)
+ blockHeader.getHash().toHexString()
+ " is "
+ blockHeader.getNumber()
+ " when we were expecting "
@ -68,72 +113,139 @@ public class BackwardChain { // TODO: this class now stores everything in memory
if (!firstHeader.getParentHash().equals(blockHeader.getHash())) {
throw new BackwardSyncException(
"Hash of header does not match our expectations, was "
+ blockHeader.getHash().toString().substring(0, 20)
+ blockHeader.getHash().toHexString()
+ " when we expected "
+ firstHeader.getParentHash().toString().substring(0, 20));
+ firstHeader.getParentHash().toHexString());
}
ancestors.add(blockHeader);
headers.put(blockHeader.getHash(), blockHeader);
ancestors.add(blockHeader.getHash());
debugLambda(
LOG,
"Added header {} on height {} to backward chain led by pivot {} on height {}",
() -> blockHeader.getHash().toString().substring(0, 20),
() -> blockHeader.getHash().toHexString(),
blockHeader::getNumber,
() -> firstHeader.getHash().toString().substring(0, 20),
() -> firstHeader.getHash().toHexString(),
firstHeader::getNumber);
}
public void merge(final BackwardChain historicalBackwardChain) {
public void prependChain(final BackwardChain historicalBackwardChain) {
BlockHeader firstHeader =
getFirstAncestorHeader()
.orElseThrow(() -> new BackwardSyncException("Cannot merge when syncing forward..."));
Block historicalPivot = historicalBackwardChain.getPivot();
Block pivot = getPivot();
if (firstHeader.getParentHash().equals(historicalPivot.getHash())) {
Collections.reverse(historicalBackwardChain.successors);
this.ancestors.addAll(
historicalBackwardChain.successors.stream()
.map(Block::getHeader)
.collect(Collectors.toList()));
this.ancestors.addAll(historicalBackwardChain.ancestors);
.orElseThrow(
() -> new BackwardSyncException("Cannot merge when syncing forward...", true));
Optional<BlockHeader> historicalHeader =
historicalBackwardChain.getHeaderOnHeight(firstHeader.getNumber() - 1);
if (historicalHeader.isEmpty()) {
return;
}
if (firstHeader.getParentHash().equals(historicalHeader.orElseThrow().getHash())) {
for (Block successor : historicalBackwardChain.getSuccessors()) {
if (successor.getHeader().getNumber() > getPivot().getHeader().getNumber()) {
this.successors.add(successor.getHeader().getHash());
}
}
Collections.reverse(historicalBackwardChain.getSuccessors());
for (Block successor : historicalBackwardChain.getSuccessors()) {
if (successor.getHeader().getNumber()
< getFirstAncestorHeader().orElseThrow().getNumber()) {
this.ancestors.add(successor.getHeader().getHash());
}
}
for (BlockHeader ancestor : historicalBackwardChain.getAllAncestors()) {
if (ancestor.getNumber() < getFirstAncestorHeader().orElseThrow().getNumber()) {
this.ancestors.add(ancestor.getHash());
}
}
debugLambda(
LOG,
"Merged backward chain led by block {} into chain led by block {}, new backward chain starts at height {} and ends at height {}",
() -> historicalPivot.getHash().toString().substring(0, 20),
() -> pivot.getHash().toString().substring(0, 20),
() -> pivot.getHeader().getNumber(),
"Merged backward chain. New chain starts at height {} and ends at height {}",
() -> getPivot().getHeader().getNumber(),
() -> getFirstAncestorHeader().orElseThrow().getNumber());
trustedBlocks.putAll(historicalBackwardChain.trustedBlocks);
} else {
warnLambda(
LOG,
"Cannot merge previous historical run because headers of {} and {} do not equal. Ignoring previous run. Did someone lie to us?",
() -> firstHeader.getHash().toString().substring(0, 20),
() -> historicalPivot.getHash().toString().substring(0, 20));
"Cannot merge previous historical run because headers on height {} ({}) of {} and {} are not equal. Ignoring previous run. Did someone lie to us?",
() -> firstHeader.getNumber() - 1,
() -> historicalHeader.orElseThrow().getNumber(),
() -> firstHeader.getParentHash().toHexString(),
() -> historicalHeader.orElseThrow().getHash().toHexString());
}
}
public Block getPivot() {
return successors.get(successors.size() - 1);
return blocks.get(successors.get(successors.size() - 1)).orElseThrow();
}
public void dropFirstHeader() {
headers.drop(ancestors.get(ancestors.size() - 1));
ancestors.remove(ancestors.size() - 1);
}
public void appendExpectedBlock(final Block newPivot) {
successors.add(newPivot);
trustedBlocks.put(newPivot.getHash(), newPivot);
successors.add(newPivot.getHash());
headers.put(newPivot.getHash(), newPivot.getHeader());
blocks.put(newPivot.getHash(), newPivot);
}
public List<Block> getSuccessors() {
return successors;
return successors.stream()
.map(hash -> blocks.get(hash).orElseThrow())
.collect(Collectors.toList());
}
public boolean isTrusted(final Hash hash) {
return trustedBlocks.containsKey(hash);
return blocks.get(hash).isPresent();
}
public Block getTrustedBlock(final Hash hash) {
return trustedBlocks.get(hash);
return blocks.get(hash).orElseThrow();
}
public void clear() {
ancestors.clear();
successors.clear();
blocks.clear();
headers.clear();
}
public void commit() {}
public Optional<BlockHeader> getHeaderOnHeight(final long height) {
if (ancestors.isEmpty()) {
return Optional.empty();
}
final long firstAncestor = headers.get(ancestors.get(0)).orElseThrow().getNumber();
if (firstAncestor >= height) {
if (firstAncestor - height < ancestors.size()) {
final Optional<BlockHeader> blockHeader =
headers.get(ancestors.get((int) (firstAncestor - height)));
blockHeader.ifPresent(
blockHeader1 ->
LOG.debug(
"First: {} Height: {}, result: {}",
firstAncestor,
height,
blockHeader.orElseThrow().getNumber()));
return blockHeader;
} else {
return Optional.empty();
}
} else {
if (successors.isEmpty()) {
return Optional.empty();
}
final long firstSuccessor = headers.get(successors.get(0)).orElseThrow().getNumber();
if (height - firstSuccessor < successors.size()) {
LOG.debug(
"First: {} Height: {}, result: {}",
firstSuccessor,
height,
headers.get(successors.get((int) (height - firstSuccessor))).orElseThrow().getNumber());
return headers.get(successors.get((int) (height - firstSuccessor)));
} else {
return Optional.empty();
}
}
}
}

@ -15,49 +15,67 @@
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.BlockValidator;
import org.hyperledger.besu.ethereum.ProtocolContext;
import org.hyperledger.besu.ethereum.core.Block;
import org.hyperledger.besu.ethereum.eth.manager.EthContext;
import org.hyperledger.besu.ethereum.eth.manager.task.GetBodiesFromPeerTask;
import org.hyperledger.besu.ethereum.eth.manager.task.GetHeadersFromPeerByHashTask;
import org.hyperledger.besu.ethereum.eth.sync.state.SyncState;
import org.hyperledger.besu.ethereum.mainnet.ProtocolSchedule;
import org.hyperledger.besu.ethereum.mainnet.ScheduleBasedBlockHeaderFunctions;
import org.hyperledger.besu.ethereum.storage.StorageProvider;
import org.hyperledger.besu.plugin.services.MetricsSystem;
import java.time.Duration;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CompletionStage;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.atomic.AtomicReference;
import java.util.function.Function;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class BackwardsSyncContext {
private static final Logger LOG = LoggerFactory.getLogger(BackwardsSyncContext.class);
public class BackwardSyncContext {
private static final Logger LOG = LoggerFactory.getLogger(BackwardSyncContext.class);
public static final int BATCH_SIZE = 200;
private static final int MAX_RETRIES = 100;
private final ProtocolContext protocolContext;
private final ProtocolSchedule protocolSchedule;
private final EthContext ethContext;
private final MetricsSystem metricsSystem;
private final SyncState syncState;
private final Map<Long, BackwardChain> backwardChainMap = new ConcurrentHashMap<>();
private final AtomicReference<BackwardChain> currentChain = new AtomicReference<>();
private final AtomicReference<CompletableFuture<Void>> currentBackwardSyncFuture =
new AtomicReference<>();
private final BackwardSyncLookupService service;
private final StorageProvider storageProvider;
public BackwardsSyncContext(
public BackwardSyncContext(
final ProtocolContext protocolContext,
final ProtocolSchedule protocolSchedule,
final MetricsSystem metricsSystem,
final EthContext ethContext) {
final EthContext ethContext,
final SyncState syncState,
final BackwardSyncLookupService backwardSyncLookupService,
final StorageProvider storageProvider) {
this.protocolContext = protocolContext;
this.protocolSchedule = protocolSchedule;
this.ethContext = ethContext;
this.metricsSystem = metricsSystem;
this.syncState = syncState;
this.service = backwardSyncLookupService;
this.storageProvider = storageProvider;
}
public boolean isSyncing() {
@ -67,36 +85,36 @@ public class BackwardsSyncContext {
}
public CompletableFuture<Void> syncBackwardsUntil(final Hash newBlockhash) {
if (getCurrentChain()
.flatMap(
chain ->
chain.getSuccessors().stream()
.map(Block::getHash)
.filter(hash -> hash.equals(newBlockhash))
.findAny())
.isPresent()) {
debugLambda(
final Optional<BackwardChain> chain = getCurrentChain();
CompletableFuture<List<Block>> completableFuture;
if (chain.isPresent() && chain.get().isTrusted(newBlockhash)) {
infoLambda(
LOG,
"not fetching and appending hash {} to backwards sync since it is present in successors",
() -> newBlockhash.toHexString());
return CompletableFuture.completedFuture(null);
newBlockhash::toHexString);
completableFuture = CompletableFuture.completedFuture(Collections.emptyList());
} else {
completableFuture = service.lookup(newBlockhash);
}
// kick off async process to fetch this block by hash then delegate to syncBackwardsUntil
return GetHeadersFromPeerByHashTask.forSingleHash(
protocolSchedule, ethContext, newBlockhash, 0L, metricsSystem)
.run()
.thenCompose(
headers ->
GetBodiesFromPeerTask.forHeaders(
protocolSchedule, ethContext, headers.getResult(), metricsSystem)
.run()
.thenCompose(blocks -> syncBackwardsUntil(blocks.getResult().get(0))))
.exceptionally(
ex -> {
LOG.error("Failed to fetch block by hash " + newBlockhash.toHexString(), ex);
throw new BackwardSyncException(ex);
final CompletableFuture<Void> future =
completableFuture.thenCompose(
blocks -> {
if (blocks.isEmpty()) {
return CompletableFuture.completedFuture(null);
} else return this.syncBackwardsUntil(blocks);
});
this.currentBackwardSyncFuture.set(future);
return future;
}
private CompletionStage<Void> syncBackwardsUntil(final List<Block> blocks) {
CompletableFuture<Void> future = CompletableFuture.completedFuture(null);
for (Block block : blocks) {
future = future.thenCompose(unused -> syncBackwardsUntil(block));
}
return future;
}
public CompletableFuture<Void> syncBackwardsUntil(final Block newPivot) {
@ -107,10 +125,14 @@ public class BackwardsSyncContext {
"Starting new backward sync towards a pivot {} at height {}",
() -> newPivot.getHash().toString().substring(0, 20),
() -> newPivot.getHeader().getNumber());
final BackwardChain newChain = new BackwardChain(newPivot);
final BackwardChain newChain =
new BackwardChain(
storageProvider,
ScheduleBasedBlockHeaderFunctions.create(protocolSchedule),
newPivot);
this.currentChain.set(newChain);
backwardChainMap.put(newPivot.getHeader().getNumber(), newChain);
this.currentBackwardSyncFuture.set(prepareBackwardSyncFuture(newChain));
currentBackwardSyncFuture.set(prepareBackwardSyncFutureWithRetry(newChain));
return currentBackwardSyncFuture.get();
}
if (newPivot.getHeader().getParentHash().equals(currentChain.get().getPivot().getHash())) {
@ -128,7 +150,9 @@ public class BackwardsSyncContext {
() -> newPivot.getHash().toString().substring(0, 20),
() -> newPivot.getHeader().getNumber());
BackwardChain newBackwardChain = new BackwardChain(newPivot);
BackwardChain newBackwardChain =
new BackwardChain(
storageProvider, ScheduleBasedBlockHeaderFunctions.create(protocolSchedule), newPivot);
backwardChainMap.put(newPivot.getHeader().getNumber(), newBackwardChain);
this.currentChain.set(
newBackwardChain); // the current ongoing backward sync will finish its current step and end
@ -141,11 +165,11 @@ public class BackwardsSyncContext {
if (error != null) {
if ((error.getCause() != null)
&& (error.getCause() instanceof BackwardSyncException)) {
LOG.debug(
LOG.info(
"Previous Backward sync ended exceptionally with message {}",
error.getMessage());
} else {
LOG.debug(
LOG.info(
"Previous Backward sync ended exceptionally with message {}",
error.getMessage());
if (error instanceof RuntimeException) {
@ -155,18 +179,59 @@ public class BackwardsSyncContext {
}
}
} else {
LOG.debug("The previous backward sync finished without and exception");
LOG.info("The previous backward sync finished without and exception");
}
return newBackwardChain;
})
.thenCompose(this::prepareBackwardSyncFuture));
.thenCompose(this::prepareBackwardSyncFutureWithRetry));
return currentBackwardSyncFuture.get();
}
private CompletableFuture<Void> prepareBackwardSyncFutureWithRetry(
final BackwardChain backwardChain) {
CompletableFuture<Void> f = prepareBackwardSyncFuture(backwardChain);
for (int i = 0; i < MAX_RETRIES; i++) {
f =
f.thenApply(CompletableFuture::completedFuture)
.exceptionally(
ex -> {
if (ex instanceof BackwardSyncException && ex.getCause() == null) {
LOG.info(
"Backward sync failed ({}). Current Peers: {}. Retrying in few seconds... ",
ex.getMessage(),
ethContext.getEthPeers().peerCount());
} else {
LOG.warn("there was an uncaught exception during backward sync", ex);
}
return ethContext
.getScheduler()
.scheduleFutureTask(
() -> prepareBackwardSyncFuture(backwardChain), Duration.ofSeconds(5));
})
.thenCompose(Function.identity());
}
return f.handle(
(unused, throwable) -> {
this.cleanup(backwardChain);
if (throwable != null) {
throw new BackwardSyncException(throwable);
}
return null;
});
}
private CompletableFuture<Void> prepareBackwardSyncFuture(final BackwardChain backwardChain) {
return new BackwardSyncStep(this, backwardChain)
return new BackwardSyncPhase(this, backwardChain)
.executeAsync(null)
.thenCompose(new ForwardSyncStep(this, backwardChain)::executeAsync);
.thenCompose(new ForwardSyncPhase(this, backwardChain)::executeAsync);
}
private void cleanup(final BackwardChain chain) {
if (currentChain.compareAndSet(chain, null)) {
this.currentBackwardSyncFuture.set(null);
}
}
public Optional<BackwardChain> getCurrentChain() {
@ -193,11 +258,15 @@ public class BackwardsSyncContext {
return protocolSchedule.getByBlockNumber(blockNumber).getBlockValidator();
}
public BackwardChain findCorrectChainFromPivot(final long number) {
return backwardChainMap.get(number);
public Optional<BackwardChain> findCorrectChainFromPivot(final long number) {
return Optional.ofNullable(backwardChainMap.get(number));
}
public void putCurrentChainToHeight(final long height, final BackwardChain backwardChain) {
backwardChainMap.put(height, backwardChain);
}
public boolean isOnTTD() {
return syncState.hasReachedTerminalDifficulty().orElse(false);
}
}

@ -15,11 +15,28 @@
package org.hyperledger.besu.ethereum.eth.sync.backwardsync;
public class BackwardSyncException extends RuntimeException {
private final boolean restartable;
public BackwardSyncException(final String message) {
super(message);
this(message, false);
}
public BackwardSyncException(final Throwable error) {
this(error, false);
}
public BackwardSyncException(final String message, final boolean restartable) {
super(message);
this.restartable = restartable;
}
public BackwardSyncException(final Throwable error, final boolean restartable) {
super(error);
this.restartable = restartable;
}
public boolean shouldRestart() {
return restartable;
}
}

@ -0,0 +1,172 @@
/*
* 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.sync.backwardsync;
import static org.slf4j.LoggerFactory.getLogger;
import org.hyperledger.besu.datatypes.Hash;
import org.hyperledger.besu.ethereum.ProtocolContext;
import org.hyperledger.besu.ethereum.core.Block;
import org.hyperledger.besu.ethereum.eth.manager.EthContext;
import org.hyperledger.besu.ethereum.eth.manager.task.AbstractPeerTask;
import org.hyperledger.besu.ethereum.eth.manager.task.RetryingGetBlockFromPeersTask;
import org.hyperledger.besu.ethereum.mainnet.ProtocolSchedule;
import org.hyperledger.besu.plugin.services.MetricsSystem;
import java.time.Duration;
import java.util.ArrayDeque;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Optional;
import java.util.Queue;
import java.util.concurrent.CompletableFuture;
import java.util.function.Function;
import javax.annotation.concurrent.GuardedBy;
import javax.annotation.concurrent.ThreadSafe;
import org.slf4j.Logger;
@ThreadSafe
public class BackwardSyncLookupService {
private static final Logger LOG = getLogger(BackwardSyncLookupService.class);
private static final int MAX_RETRIES = 100;
public static final int UNUSED = -1;
@GuardedBy("this")
private final Queue<Hash> hashes = new ArrayDeque<>();
@GuardedBy("this")
boolean running = false;
private final ProtocolSchedule protocolSchedule;
private final EthContext ethContext;
private final MetricsSystem metricsSystem;
private List<Block> results = new ArrayList<>();
private final ProtocolContext protocolContext;
public BackwardSyncLookupService(
final ProtocolSchedule protocolSchedule,
final EthContext ethContext,
final MetricsSystem metricsSystem,
final ProtocolContext protocolContext) {
this.protocolSchedule = protocolSchedule;
this.ethContext = ethContext;
this.metricsSystem = metricsSystem;
this.protocolContext = protocolContext;
}
public CompletableFuture<List<Block>> lookup(final Hash newBlockhash) {
synchronized (this) {
hashes.add(newBlockhash);
if (running) {
LOG.info(
"some other future is already running and will process our hash {} when time comes...",
newBlockhash.toHexString());
return CompletableFuture.completedFuture(Collections.emptyList());
}
running = true;
}
return findBlocksWithRetries()
.handle(
(blocks, throwable) -> {
synchronized (this) {
running = false;
}
if (throwable != null) {
throw new BackwardSyncException(throwable);
}
return blocks;
});
}
private CompletableFuture<List<Block>> findBlocksWithRetries() {
CompletableFuture<List<Block>> f = tryToFindBlocks();
for (int i = 0; i < MAX_RETRIES; i++) {
f =
f.thenApply(CompletableFuture::completedFuture)
.exceptionally(
ex -> {
synchronized (this) {
if (!results.isEmpty()) {
List<Block> copy = new ArrayList<>(results);
results = new ArrayList<>();
return CompletableFuture.completedFuture(copy);
}
}
LOG.error(
"Failed to fetch blocks because {} Current peers: {}. Waiting for few seconds ...",
ex.getMessage(),
ethContext.getEthPeers().peerCount());
return ethContext
.getScheduler()
.scheduleFutureTask(this::tryToFindBlocks, Duration.ofSeconds(5));
})
.thenCompose(Function.identity());
}
return f.thenApply(this::rememberResults).thenCompose(this::possibleNextHash);
}
private CompletableFuture<List<Block>> tryToFindBlocks() {
return CompletableFuture.supplyAsync(this::getNextHash)
.thenCompose(this::tryToFindBlock)
.thenApply(this::rememberResult)
.thenCompose(this::possibleNextHash);
}
private CompletableFuture<List<Block>> possibleNextHash(final List<Block> blocks) {
synchronized (this) {
hashes.poll();
if (hashes.isEmpty()) {
results = new ArrayList<>();
running = false;
return CompletableFuture.completedFuture(blocks);
}
}
return tryToFindBlocks();
}
private List<Block> rememberResult(final Block block) {
this.results.add(block);
return results;
}
private List<Block> rememberResults(final List<Block> blocks) {
this.results.addAll(blocks);
return results;
}
private synchronized Hash getNextHash() {
return hashes.peek();
}
private CompletableFuture<Block> tryToFindBlock(final Hash targetHash) {
final RetryingGetBlockFromPeersTask getBlockTask =
RetryingGetBlockFromPeersTask.create(
protocolContext,
protocolSchedule,
ethContext,
metricsSystem,
ethContext.getEthPeers().getMaxPeers(),
Optional.of(targetHash),
UNUSED);
return ethContext
.getScheduler()
.scheduleSyncWorkerTask(getBlockTask::run)
.thenApply(AbstractPeerTask.PeerTaskResult::getResult);
}
}

@ -15,56 +15,75 @@
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;
import org.hyperledger.besu.ethereum.eth.manager.task.GetHeadersFromPeerByHashTask;
import org.hyperledger.besu.ethereum.eth.manager.task.RetryingGetHeadersEndingAtFromPeerByHashTask;
import java.time.Duration;
import java.util.List;
import java.util.Optional;
import java.util.concurrent.CompletableFuture;
import java.util.function.Function;
import com.google.common.annotations.VisibleForTesting;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class BackwardSyncStep extends BackwardSyncTask {
private static final Logger LOG = LoggerFactory.getLogger(BackwardSyncStep.class);
public class BackwardSyncPhase extends BackwardSyncTask {
private static final Logger LOG = LoggerFactory.getLogger(BackwardSyncPhase.class);
public BackwardSyncStep(final BackwardsSyncContext context, final BackwardChain backwardChain) {
public BackwardSyncPhase(final BackwardSyncContext context, final BackwardChain backwardChain) {
super(context, backwardChain);
}
@VisibleForTesting
protected CompletableFuture<Void> waitForTTD() {
if (context.isOnTTD()) {
return CompletableFuture.completedFuture(null);
}
LOG.info("Did not reach TTD yet, falling asleep...");
return context
.getEthContext()
.getScheduler()
.scheduleFutureTask(this::waitForTTD, Duration.ofSeconds(5));
}
@Override
public CompletableFuture<Void> executeStep() {
return CompletableFuture.supplyAsync(this::earliestUnprocessedHash)
.thenCompose(this::requestHeader)
.thenApply(this::saveHeader)
return CompletableFuture.supplyAsync(this::waitForTTD)
.thenCompose(Function.identity())
.thenApply(this::earliestUnprocessedHash)
.thenCompose(this::requestHeaders)
.thenApply(this::saveHeaders)
.thenApply(this::possibleMerge)
.thenCompose(this::possiblyMoreBackwardSteps);
}
@VisibleForTesting
protected Hash earliestUnprocessedHash() {
protected Hash earliestUnprocessedHash(final Void unused) {
BlockHeader firstHeader =
backwardChain
.getFirstAncestorHeader()
.orElseThrow(
() ->
new BackwardSyncException(
"No unprocessed hashes during backward sync. that is probably a bug."));
"No unprocessed hashes during backward sync. that is probably a bug.",
true));
Hash parentHash = firstHeader.getParentHash();
debugLambda(
LOG,
"First unprocessed hash for current pivot is {} expected on height {}",
() -> parentHash.toString().substring(0, 20),
parentHash::toHexString,
() -> firstHeader.getNumber() - 1);
return parentHash;
}
@VisibleForTesting
protected CompletableFuture<BlockHeader> requestHeader(final Hash hash) {
debugLambda(LOG, "Requesting header for hash {}", () -> hash.toString().substring(0, 20));
debugLambda(LOG, "Requesting header for hash {}", hash::toHexString);
return GetHeadersFromPeerByHashTask.forSingleHash(
context.getProtocolSchedule(),
context.getEthContext(),
@ -77,32 +96,75 @@ public class BackwardSyncStep extends BackwardSyncTask {
final List<BlockHeader> result = peerResult.getResult();
if (result.isEmpty()) {
throw new BackwardSyncException(
"Did not receive a header for hash {}" + hash.toString().substring(0, 20));
"Did not receive a header for hash {}" + hash.toHexString(), true);
}
BlockHeader blockHeader = result.get(0);
debugLambda(
LOG,
"Got header {} with height {}",
() -> blockHeader.getHash().toString().substring(0, 20),
() -> blockHeader.getHash().toHexString(),
blockHeader::getNumber);
return blockHeader;
});
}
@VisibleForTesting
protected CompletableFuture<List<BlockHeader>> requestHeaders(final Hash hash) {
debugLambda(LOG, "Requesting header for hash {}", hash::toHexString);
final RetryingGetHeadersEndingAtFromPeerByHashTask
retryingGetHeadersEndingAtFromPeerByHashTask =
RetryingGetHeadersEndingAtFromPeerByHashTask.endingAtHash(
context.getProtocolSchedule(),
context.getEthContext(),
hash,
context.getProtocolContext().getBlockchain().getChainHead().getHeight(),
BackwardSyncContext.BATCH_SIZE,
context.getMetricsSystem());
return context
.getEthContext()
.getScheduler()
.scheduleSyncWorkerTask(retryingGetHeadersEndingAtFromPeerByHashTask::run)
.thenApply(
blockHeaders -> {
if (blockHeaders.isEmpty()) {
throw new BackwardSyncException(
"Did not receive a header for hash {}" + hash.toHexString(), true);
}
debugLambda(
LOG,
"Got headers {} -> {}",
blockHeaders.get(0)::getNumber,
blockHeaders.get(blockHeaders.size() - 1)::getNumber);
return blockHeaders;
});
}
@VisibleForTesting
protected Void saveHeader(final BlockHeader blockHeader) {
backwardChain.saveHeader(blockHeader);
backwardChain.prependAncestorsHeader(blockHeader);
context.putCurrentChainToHeight(blockHeader.getNumber(), backwardChain);
return null;
}
@VisibleForTesting
protected Void saveHeaders(final List<BlockHeader> blockHeaders) {
for (BlockHeader blockHeader : blockHeaders) {
saveHeader(blockHeader);
}
infoLambda(
LOG,
"Saved headers {} -> {}",
() -> blockHeaders.get(0).getNumber(),
() -> blockHeaders.get(blockHeaders.size() - 1).getNumber());
return null;
}
@VisibleForTesting
protected BlockHeader possibleMerge(final Void unused) {
Optional<BackwardChain> maybeHistoricalBackwardChain =
Optional.ofNullable(
context.findCorrectChainFromPivot(
backwardChain.getFirstAncestorHeader().orElseThrow().getNumber() - 1));
maybeHistoricalBackwardChain.ifPresent(backwardChain::merge);
context.findCorrectChainFromPivot(
backwardChain.getFirstAncestorHeader().orElseThrow().getNumber() - 1);
maybeHistoricalBackwardChain.ifPresent(backwardChain::prependChain);
return backwardChain.getFirstAncestorHeader().orElseThrow();
}
@ -111,16 +173,18 @@ public class BackwardSyncStep extends BackwardSyncTask {
protected CompletableFuture<Void> possiblyMoreBackwardSteps(final BlockHeader blockHeader) {
CompletableFuture<Void> completableFuture = new CompletableFuture<>();
if (context.getProtocolContext().getBlockchain().contains(blockHeader.getHash())) {
LOG.debug(
"The backward sync let us to a block that we already know... We will init forward sync...");
completableFuture.complete(null); // we finished backward sync
LOG.info("Backward Phase finished.");
completableFuture.complete(null);
return completableFuture;
}
if (context.getProtocolContext().getBlockchain().getChainHead().getHeight()
> blockHeader.getNumber() - 1) {
completableFuture.completeExceptionally(
new RuntimeException("Backward sync would reach under know head of blockchain"));
return completableFuture;
LOG.warn(
"Backward sync is following unknown branch {} ({}) and reached bellow previous head {}({})",
blockHeader.getNumber(),
blockHeader.getHash(),
context.getProtocolContext().getBlockchain().getChainHead().getHeight(),
context.getProtocolContext().getBlockchain().getChainHead().getHash().toHexString());
}
LOG.debug("Backward sync did not reach a know block, need to go deeper");
completableFuture.complete(null);

@ -14,33 +14,39 @@
*/
package org.hyperledger.besu.ethereum.eth.sync.backwardsync;
import static org.slf4j.LoggerFactory.getLogger;
import java.util.Optional;
import java.util.concurrent.CompletableFuture;
import org.slf4j.Logger;
public abstract class BackwardSyncTask {
protected BackwardsSyncContext context;
protected BackwardSyncContext context;
protected BackwardChain backwardChain;
private static final Logger LOG = getLogger(BackwardSyncTask.class);
protected BackwardSyncTask(
final BackwardsSyncContext context, final BackwardChain backwardChain) {
protected BackwardSyncTask(final BackwardSyncContext context, final BackwardChain backwardChain) {
this.context = context;
this.backwardChain = backwardChain;
}
CompletableFuture<Void> executeAsync(final Void unused) {
CompletableFuture<Void> result = new CompletableFuture<>();
Optional<BackwardChain> currentChain = context.getCurrentChain();
if (currentChain.isPresent()) {
if (!backwardChain.equals(currentChain.get())) {
result.completeExceptionally(
new BackwardSyncException(
"The pivot changed, we should stop current flow, some new flow is waiting to take over..."));
return result;
} else {
result.complete(null);
return executeStep();
LOG.debug(
"The pivot changed, we should stop current flow, some new flow is waiting to take over...");
return CompletableFuture.completedFuture(null);
}
if (backwardChain.getFirstAncestorHeader().isEmpty()) {
LOG.info("The Backwards sync is already finished...");
return CompletableFuture.completedFuture(null);
}
return executeStep();
} else {
CompletableFuture<Void> result = new CompletableFuture<>();
result.completeExceptionally(
new BackwardSyncException(
"No pivot... that is weird and should not have happened. This method should have been called after the pivot was set..."));

@ -0,0 +1,47 @@
/*
* 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.sync.backwardsync;
import org.hyperledger.besu.ethereum.core.Block;
import org.hyperledger.besu.ethereum.core.BlockHeaderFunctions;
import org.hyperledger.besu.ethereum.rlp.RLP;
import org.hyperledger.besu.ethereum.rlp.RLPInput;
import org.apache.tuweni.bytes.Bytes;
public class BlocksConvertor implements ValueConvertor<Block> {
private final BlockHeaderFunctions blockHeaderFunctions;
public BlocksConvertor(final BlockHeaderFunctions blockHeaderFunctions) {
this.blockHeaderFunctions = blockHeaderFunctions;
}
public static ValueConvertor<Block> of(final BlockHeaderFunctions blockHeaderFunctions) {
return new BlocksConvertor(blockHeaderFunctions);
}
@Override
public Block fromBytes(final byte[] bytes) {
final RLPInput input = RLP.input(Bytes.wrap(bytes));
return Block.readFrom(input, blockHeaderFunctions);
}
@Override
public byte[] toBytes(final Block value) {
return value.toRlp().toArrayUnsafe();
}
}

@ -0,0 +1,47 @@
/*
* 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.sync.backwardsync;
import org.hyperledger.besu.ethereum.core.BlockHeader;
import org.hyperledger.besu.ethereum.core.BlockHeaderFunctions;
import org.hyperledger.besu.ethereum.rlp.BytesValueRLPOutput;
import org.hyperledger.besu.ethereum.rlp.RLP;
import org.apache.tuweni.bytes.Bytes;
public class BlocksHeadersConvertor implements ValueConvertor<BlockHeader> {
private final BlockHeaderFunctions blockHeaderFunctions;
public BlocksHeadersConvertor(final BlockHeaderFunctions blockHeaderFunctions) {
this.blockHeaderFunctions = blockHeaderFunctions;
}
public static ValueConvertor<BlockHeader> of(final BlockHeaderFunctions blockHeaderFunctions) {
return new BlocksHeadersConvertor(blockHeaderFunctions);
}
@Override
public BlockHeader fromBytes(final byte[] bytes) {
return BlockHeader.readFrom(RLP.input(Bytes.wrap(bytes)), blockHeaderFunctions);
}
@Override
public byte[] toBytes(final BlockHeader value) {
BytesValueRLPOutput output = new BytesValueRLPOutput();
value.writeTo(output);
return output.encoded().toArrayUnsafe();
}
}

@ -0,0 +1,257 @@
/*
* 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.sync.backwardsync;
import static org.hyperledger.besu.util.Slf4jLambdaHelper.debugLambda;
import static org.hyperledger.besu.util.Slf4jLambdaHelper.infoLambda;
import static org.hyperledger.besu.util.Slf4jLambdaHelper.warnLambda;
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.GetBlockFromPeerTask;
import org.hyperledger.besu.ethereum.eth.manager.task.GetBodiesFromPeerTask;
import org.hyperledger.besu.ethereum.mainnet.HeaderValidationMode;
import java.util.Collections;
import java.util.Comparator;
import java.util.List;
import java.util.Optional;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CompletionStage;
import com.google.common.annotations.VisibleForTesting;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class ForwardSyncPhase extends BackwardSyncTask {
private static final Logger LOG = LoggerFactory.getLogger(ForwardSyncPhase.class);
private int batchSize = BackwardSyncContext.BATCH_SIZE;
public ForwardSyncPhase(final BackwardSyncContext context, final BackwardChain backwardChain) {
super(context, backwardChain);
}
@Override
public CompletableFuture<Void> executeStep() {
return CompletableFuture.supplyAsync(() -> returnFirstNUnknownHeaders(null))
.thenCompose(this::possibleRequestBodies)
.thenApply(this::processKnownAncestors)
.thenCompose(this::possiblyMoreForwardSteps);
}
@VisibleForTesting
protected BlockHeader processKnownAncestors(final Void unused) {
while (backwardChain.getFirstAncestorHeader().isPresent()) {
BlockHeader header = backwardChain.getFirstAncestorHeader().orElseThrow();
if (context.getProtocolContext().getBlockchain().contains(header.getHash())) {
debugLambda(
LOG,
"Block {} is already imported, we can ignore it for the sync process",
() -> header.getHash().toHexString());
backwardChain.dropFirstHeader();
} else if (backwardChain.isTrusted(header.getHash())) {
debugLambda(
LOG,
"Importing trusted block {}({})",
header::getNumber,
() -> header.getHash().toHexString());
saveBlock(backwardChain.getTrustedBlock(header.getHash()));
} else {
debugLambda(LOG, "First unprocessed header is {}", header::getNumber);
return header;
}
}
return null;
}
@VisibleForTesting
protected List<BlockHeader> returnFirstNUnknownHeaders(final Void unused) {
while (backwardChain.getFirstAncestorHeader().isPresent()) {
BlockHeader header = backwardChain.getFirstAncestorHeader().orElseThrow();
if (context.getProtocolContext().getBlockchain().contains(header.getHash())) {
debugLambda(
LOG,
"Block {}({}) is already imported, we can ignore it for the sync process",
() -> header.getNumber(),
() -> header.getHash().toHexString());
backwardChain.dropFirstHeader();
} else if (backwardChain.isTrusted(header.getHash())) {
debugLambda(
LOG,
"Block {} was added by consensus layer, we can trust it and should therefore import it.",
() -> header.getHash().toHexString());
saveBlock(backwardChain.getTrustedBlock(header.getHash()));
} else {
return backwardChain.getFirstNAncestorHeaders(batchSize);
}
}
return Collections.emptyList();
}
@VisibleForTesting
public CompletableFuture<Void> possibleRequestBlock(final BlockHeader blockHeader) {
if (blockHeader == null) {
return CompletableFuture.completedFuture(null);
} else {
debugLambda(
LOG,
"Requesting body for {} ({})",
blockHeader::getNumber,
() -> blockHeader.getHash().toHexString());
return requestBlock(blockHeader).thenApply(this::saveBlock);
}
}
@VisibleForTesting
public CompletableFuture<Void> possibleRequestBodies(final List<BlockHeader> blockHeaders) {
if (blockHeaders.isEmpty()) {
return CompletableFuture.completedFuture(null);
} else {
debugLambda(
LOG,
"Requesting {} blocks {}->{} ({})",
blockHeaders::size,
() -> blockHeaders.get(0).getNumber(),
() -> blockHeaders.get(blockHeaders.size() - 1).getNumber(),
() -> blockHeaders.get(0).getHash().toHexString());
return requestBodies(blockHeaders).thenApply(this::saveBlocks);
}
}
@VisibleForTesting
protected CompletableFuture<Block> requestBlock(final BlockHeader blockHeader) {
final GetBlockFromPeerTask getBlockFromPeerTask =
GetBlockFromPeerTask.create(
context.getProtocolSchedule(),
context.getEthContext(),
Optional.of(blockHeader.getHash()),
blockHeader.getNumber(),
context.getMetricsSystem());
final CompletableFuture<AbstractPeerTask.PeerTaskResult<Block>> run =
getBlockFromPeerTask.run();
return run.thenApply(AbstractPeerTask.PeerTaskResult::getResult);
}
@VisibleForTesting
protected CompletableFuture<List<Block>> requestBodies(final List<BlockHeader> blockHeaders) {
final GetBodiesFromPeerTask getBodiesFromPeerTask =
GetBodiesFromPeerTask.forHeaders(
context.getProtocolSchedule(),
context.getEthContext(),
blockHeaders,
context.getMetricsSystem());
final CompletableFuture<AbstractPeerTask.PeerTaskResult<List<Block>>> run =
getBodiesFromPeerTask.run();
return run.thenApply(AbstractPeerTask.PeerTaskResult::getResult)
.thenApply(
blocks -> {
blocks.sort(Comparator.comparing(block -> block.getHeader().getNumber()));
return blocks;
});
}
@VisibleForTesting
protected Void saveBlock(final Block block) {
debugLambda(LOG, "Going to validate block {}", () -> block.getHeader().getHash().toHexString());
var optResult =
context
.getBlockValidator(block.getHeader().getNumber())
.validateAndProcessBlock(
context.getProtocolContext(),
block,
HeaderValidationMode.FULL,
HeaderValidationMode.NONE);
optResult.blockProcessingOutputs.ifPresent(
result -> {
debugLambda(
LOG,
"Block {} was validated, going to import it",
() -> block.getHeader().getHash().toHexString());
result.worldState.persist(block.getHeader());
context.getProtocolContext().getBlockchain().appendBlock(block, result.receipts);
});
return null;
}
@VisibleForTesting
protected Void saveBlocks(final List<Block> blocks) {
for (Block block : blocks) {
final Optional<Block> parent =
context
.getProtocolContext()
.getBlockchain()
.getBlockByHash(block.getHeader().getParentHash());
if (parent.isEmpty()) {
batchSize = batchSize / 2 + 1;
return null;
} else {
batchSize = BackwardSyncContext.BATCH_SIZE;
saveBlock(block);
}
}
backwardChain.commit();
infoLambda(
LOG,
"Saved blocks {}->{}",
() -> blocks.get(0).getHeader().getNumber(),
() -> blocks.get(blocks.size() - 1).getHeader().getNumber());
return null;
}
@VisibleForTesting
protected CompletableFuture<Void> possiblyMoreForwardSteps(final BlockHeader firstNotSynced) {
CompletableFuture<Void> completableFuture = CompletableFuture.completedFuture(null);
if (firstNotSynced == null) {
final List<Block> successors = backwardChain.getSuccessors();
LOG.info(
"Forward Sync Phase is finished. Importing {} block(s) provided by consensus layer...",
successors.size());
successors.forEach(
block -> {
if (!context.getProtocolContext().getBlockchain().contains(block.getHash())) {
saveBlock(block);
}
});
LOG.info("The Backward sync is done...");
backwardChain.clear();
return CompletableFuture.completedFuture(null);
}
if (context.getProtocolContext().getBlockchain().contains(firstNotSynced.getParentHash())) {
debugLambda(
LOG,
"Block {} is not yet imported, we need to run another step of ForwardSync",
firstNotSynced::toLogString);
return completableFuture.thenCompose(this::executeAsync);
}
warnLambda(
LOG,
"Block {} is not yet imported but its parent {} is not imported either... "
+ "This should not normally happen and indicates a wrong behaviour somewhere...",
firstNotSynced::toLogString,
() -> firstNotSynced.getParentHash().toHexString());
return completableFuture.thenCompose(this::executeBackwardAsync);
}
@VisibleForTesting
protected CompletionStage<Void> executeBackwardAsync(final Void unused) {
return new BackwardSyncPhase(context, backwardChain).executeAsync(unused);
}
}

@ -1,147 +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.sync.backwardsync;
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.task.AbstractPeerTask;
import org.hyperledger.besu.ethereum.eth.manager.task.GetBlockFromPeerTask;
import org.hyperledger.besu.ethereum.mainnet.HeaderValidationMode;
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 ForwardSyncStep extends BackwardSyncTask {
private static final Logger LOG = LoggerFactory.getLogger(ForwardSyncStep.class);
public ForwardSyncStep(final BackwardsSyncContext context, final BackwardChain backwardChain) {
super(context, backwardChain);
}
@Override
public CompletableFuture<Void> executeStep() {
return CompletableFuture.supplyAsync(() -> processKnownAncestors(null))
.thenCompose(this::possibleRequestBlock)
.thenApply(this::processKnownAncestors)
.thenCompose(this::possiblyMoreForwardSteps);
}
@VisibleForTesting
protected BlockHeader processKnownAncestors(final Void unused) {
while (backwardChain.getFirstAncestorHeader().isPresent()) {
BlockHeader header = backwardChain.getFirstAncestorHeader().orElseThrow();
if (context.getProtocolContext().getBlockchain().contains(header.getHash())) {
debugLambda(
LOG,
"Block {} is already imported, we can ignore it for the sync process",
() -> header.getHash().toString().substring(0, 20));
backwardChain.dropFirstHeader();
} else if (backwardChain.isTrusted(header.getHash())) {
debugLambda(
LOG,
"Block {} was added by consensus layer, we can trust it and should therefore import it.",
() -> header.getHash().toString().substring(0, 20));
saveBlock(backwardChain.getTrustedBlock(header.getHash()));
} else {
return header;
}
}
return null;
}
@VisibleForTesting
public CompletableFuture<Void> possibleRequestBlock(final BlockHeader blockHeader) {
if (blockHeader == null) {
return CompletableFuture.completedFuture(null);
} else {
debugLambda(
LOG,
"We don't have body of block {}, going to request it",
() -> blockHeader.getHash().toString().substring(0, 20));
return requestBlock(blockHeader).thenApply(this::saveBlock);
}
}
@VisibleForTesting
protected CompletableFuture<Block> requestBlock(final BlockHeader blockHeader) {
final GetBlockFromPeerTask getBlockFromPeerTask =
GetBlockFromPeerTask.create(
context.getProtocolSchedule(),
context.getEthContext(),
Optional.of(blockHeader.getHash()),
blockHeader.getNumber(),
context.getMetricsSystem());
final CompletableFuture<AbstractPeerTask.PeerTaskResult<Block>> run =
getBlockFromPeerTask.run();
return run.thenApply(AbstractPeerTask.PeerTaskResult::getResult);
}
@VisibleForTesting
protected Void saveBlock(final Block block) {
debugLambda(
LOG,
"Going to validate block {}",
() -> block.getHeader().getHash().toString().substring(0, 20));
var optResult =
context
.getBlockValidator(block.getHeader().getNumber())
.validateAndProcessBlock(
context.getProtocolContext(),
block,
HeaderValidationMode.FULL,
HeaderValidationMode.NONE);
optResult.blockProcessingOutputs.ifPresent(
result -> {
debugLambda(
LOG,
"Block {} was validated, going to import it",
() -> block.getHeader().getHash().toString().substring(0, 20));
result.worldState.persist(block.getHeader());
context.getProtocolContext().getBlockchain().appendBlock(block, result.receipts);
});
return null;
}
@VisibleForTesting
protected CompletableFuture<Void> possiblyMoreForwardSteps(final BlockHeader firstUnsynced) {
CompletableFuture<Void> completableFuture = CompletableFuture.completedFuture(null);
if (firstUnsynced == null) {
LOG.debug("The only work left is to import blocks provided by consensus layer...");
backwardChain
.getSuccessors()
.forEach(
block -> {
if (!context.getProtocolContext().getBlockchain().contains(block.getHash())) {
saveBlock(block);
}
});
LOG.debug("The sync is done...");
return CompletableFuture.completedFuture(null);
}
debugLambda(
LOG,
"Block {} is not yet imported, we need to run another step of ForwardSync",
() -> firstUnsynced.getHash().toString().substring(0, 20));
return completableFuture.thenCompose(this::executeAsync);
}
}

@ -0,0 +1,61 @@
/*
* 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.sync.backwardsync;
import org.hyperledger.besu.plugin.services.storage.KeyValueStorage;
import org.hyperledger.besu.plugin.services.storage.KeyValueStorageTransaction;
import java.io.Closeable;
import java.io.IOException;
import java.util.Optional;
public class GenericKeyValueStorageFacade<K, V> implements Closeable {
protected final KeyValueStorage storage;
private final KeyConvertor<K> keyConvertor;
private final ValueConvertor<V> valueConvertor;
public GenericKeyValueStorageFacade(
final KeyConvertor<K> keyConvertor,
final ValueConvertor<V> valueConvertor,
final KeyValueStorage storageBySegmentIdentifier) {
this.keyConvertor = keyConvertor;
this.valueConvertor = valueConvertor;
this.storage = storageBySegmentIdentifier;
}
public Optional<V> get(final K key) {
return storage.get(keyConvertor.toBytes(key)).map(valueConvertor::fromBytes);
}
public void put(final K key, final V value) {
final KeyValueStorageTransaction keyValueStorageTransaction = storage.startTransaction();
keyValueStorageTransaction.put(keyConvertor.toBytes(key), valueConvertor.toBytes(value));
keyValueStorageTransaction.commit();
}
public void drop(final K key) {
storage.tryDelete(keyConvertor.toBytes(key));
}
public void clear() {
storage.clear();
}
@Override
public void close() throws IOException {
storage.close();
}
}

@ -0,0 +1,20 @@
/*
* 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.sync.backwardsync;
public interface KeyConvertor<T> {
byte[] toBytes(final T key);
}

@ -0,0 +1,22 @@
/*
* 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.sync.backwardsync;
public interface ValueConvertor<V> {
V fromBytes(final byte[] bytes);
byte[] toBytes(final V value);
}

@ -35,6 +35,7 @@ public class FullSyncDownloader {
private final SynchronizerConfiguration syncConfig;
private final ProtocolContext protocolContext;
private final SyncState syncState;
private final SyncTerminationCondition terminationCondition;
public FullSyncDownloader(
final SynchronizerConfiguration syncConfig,
@ -47,6 +48,7 @@ public class FullSyncDownloader {
this.syncConfig = syncConfig;
this.protocolContext = protocolContext;
this.syncState = syncState;
this.terminationCondition = terminationCondition;
this.chainDownloader =
FullSyncChainDownloader.create(
@ -61,7 +63,15 @@ public class FullSyncDownloader {
public CompletableFuture<Void> start() {
LOG.info("Starting full sync.");
return chainDownloader.start();
return chainDownloader
.start()
.thenApply(
unused -> {
if (terminationCondition.shouldStopDownload()) {
syncState.setReachedTerminalDifficulty(true);
}
return null;
});
}
public void stop() {

@ -25,23 +25,27 @@ import static org.mockito.Mockito.spy;
import static org.mockito.Mockito.when;
import org.hyperledger.besu.config.StubGenesisConfigOptions;
import org.hyperledger.besu.datatypes.Hash;
import org.hyperledger.besu.ethereum.BlockValidator;
import org.hyperledger.besu.ethereum.BlockValidator.Result;
import org.hyperledger.besu.ethereum.ProtocolContext;
import org.hyperledger.besu.ethereum.chain.MutableBlockchain;
import org.hyperledger.besu.ethereum.core.Block;
import org.hyperledger.besu.ethereum.core.BlockDataGenerator;
import org.hyperledger.besu.ethereum.core.InMemoryKeyValueStorageProvider;
import org.hyperledger.besu.ethereum.core.TransactionReceipt;
import org.hyperledger.besu.ethereum.eth.manager.EthContext;
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.sync.state.SyncState;
import org.hyperledger.besu.ethereum.mainnet.MainnetProtocolSchedule;
import org.hyperledger.besu.ethereum.mainnet.ProtocolSchedule;
import org.hyperledger.besu.ethereum.mainnet.ProtocolSpec;
import org.hyperledger.besu.ethereum.referencetests.ReferenceTestWorldState;
import org.hyperledger.besu.plugin.services.MetricsSystem;
import java.util.Collections;
import java.util.List;
import java.util.Optional;
import java.util.concurrent.CompletableFuture;
@ -56,13 +60,13 @@ import org.mockito.Spy;
import org.mockito.junit.MockitoJUnitRunner;
@RunWith(MockitoJUnitRunner.class)
public class BackwardsSyncContextTest {
public class BackwardSyncContextTest {
public static final int REMOTE_HEIGHT = 50;
public static final int LOCAL_HEIGHT = 25;
private static final BlockDataGenerator blockDataGenerator = new BlockDataGenerator();
private BackwardsSyncContext context;
private BackwardSyncContext context;
private MutableBlockchain remoteBlockchain;
private RespondingEthPeer peer;
@ -80,11 +84,15 @@ public class BackwardsSyncContextTest {
private MetricsSystem metricsSystem;
@Mock private BlockValidator blockValidator;
@Mock private SyncState syncState;
@Mock private BackwardSyncLookupService backwardSyncLookupService;
@Before
public void setup() {
when(mockProtocolSpec.getBlockValidator()).thenReturn(blockValidator);
when(protocolSchedule.getByBlockNumber(anyLong())).thenReturn(mockProtocolSpec);
when(syncState.hasReachedTerminalDifficulty()).thenReturn(Optional.of(true));
Block genesisBlock = blockDataGenerator.genesisBlock();
remoteBlockchain = createInMemoryBlockchain(genesisBlock);
localBlockchain = createInMemoryBlockchain(genesisBlock);
@ -119,13 +127,23 @@ public class BackwardsSyncContextTest {
});
context =
spy(new BackwardsSyncContext(protocolContext, protocolSchedule, metricsSystem, ethContext));
spy(
new BackwardSyncContext(
protocolContext,
protocolSchedule,
metricsSystem,
ethContext,
syncState,
backwardSyncLookupService,
new InMemoryKeyValueStorageProvider()));
}
@Test
public void shouldSyncUntilHash() throws Exception {
final CompletableFuture<Void> future =
context.syncBackwardsUntil(getBlockByNumber(REMOTE_HEIGHT).getHash());
final Hash hash = getBlockByNumber(REMOTE_HEIGHT).getHash();
when(backwardSyncLookupService.lookup(hash))
.thenReturn(CompletableFuture.completedFuture(List.of(getBlockByNumber(REMOTE_HEIGHT))));
final CompletableFuture<Void> future = context.syncBackwardsUntil(hash);
respondUntilFutureIsDone(future);
@ -135,13 +153,11 @@ public class BackwardsSyncContextTest {
@Test
public void shouldNotAppendWhenAlreadySyncingHash() throws Exception {
when(context.getCurrentChain())
.thenReturn(Optional.of(new BackwardChain(getBlockByNumber(REMOTE_HEIGHT))));
final CompletableFuture<Void> fut2 =
context.syncBackwardsUntil(getBlockByNumber(REMOTE_HEIGHT).getHash());
final Hash hash = getBlockByNumber(REMOTE_HEIGHT).getHash();
when(backwardSyncLookupService.lookup(hash))
.thenReturn(CompletableFuture.completedFuture(Collections.emptyList()));
final CompletableFuture<Void> fut2 = context.syncBackwardsUntil(hash);
assertThat(fut2).isCompleted();
assertThat(fut2.get()).isNull();
}
@Test

@ -0,0 +1,132 @@
/*
* 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.sync.backwardsync;
import static org.assertj.core.api.AssertionsForClassTypes.assertThat;
import static org.hyperledger.besu.ethereum.core.InMemoryKeyValueStorageProvider.createInMemoryBlockchain;
import static org.mockito.ArgumentMatchers.anyLong;
import static org.mockito.Mockito.spy;
import static org.mockito.Mockito.when;
import org.hyperledger.besu.config.StubGenesisConfigOptions;
import org.hyperledger.besu.datatypes.Hash;
import org.hyperledger.besu.ethereum.ProtocolContext;
import org.hyperledger.besu.ethereum.chain.MutableBlockchain;
import org.hyperledger.besu.ethereum.core.Block;
import org.hyperledger.besu.ethereum.core.BlockDataGenerator;
import org.hyperledger.besu.ethereum.core.TransactionReceipt;
import org.hyperledger.besu.ethereum.eth.manager.EthContext;
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.mainnet.MainnetProtocolSchedule;
import org.hyperledger.besu.ethereum.mainnet.ProtocolSchedule;
import org.hyperledger.besu.ethereum.mainnet.ProtocolSpec;
import org.hyperledger.besu.plugin.services.MetricsSystem;
import java.lang.reflect.Field;
import java.lang.reflect.Modifier;
import java.util.List;
import java.util.concurrent.CompletableFuture;
import org.jetbrains.annotations.NotNull;
import org.junit.Before;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.mockito.Answers;
import org.mockito.Mock;
import org.mockito.Spy;
import org.mockito.junit.MockitoJUnitRunner;
@RunWith(MockitoJUnitRunner.class)
public class BackwardSyncLookupServiceTest {
public static final int REMOTE_HEIGHT = 50;
private static final BlockDataGenerator blockDataGenerator = new BlockDataGenerator();
@Spy
private final ProtocolSchedule protocolSchedule =
MainnetProtocolSchedule.fromConfig(new StubGenesisConfigOptions());
@Spy private final ProtocolSpec mockProtocolSpec = protocolSchedule.getByBlockNumber(0L);
private MutableBlockchain remoteBlockchain;
private RespondingEthPeer peer;
@Mock(answer = Answers.RETURNS_DEEP_STUBS)
private MetricsSystem metricsSystem;
@Mock private ProtocolContext protocolContext;
private BackwardSyncLookupService backwardSyncLookupService;
@Before
public void setup() throws NoSuchFieldException, IllegalAccessException {
when(protocolSchedule.getByBlockNumber(anyLong())).thenReturn(mockProtocolSpec);
Block genesisBlock = blockDataGenerator.genesisBlock();
remoteBlockchain = createInMemoryBlockchain(genesisBlock);
final Field max_retries = BackwardSyncLookupService.class.getDeclaredField("MAX_RETRIES");
Field modifiersField = Field.class.getDeclaredField("modifiers");
modifiersField.setAccessible(true);
modifiersField.setInt(max_retries, max_retries.getModifiers() & ~Modifier.FINAL);
max_retries.setAccessible(true);
max_retries.set(null, 1);
for (int i = 1; i <= REMOTE_HEIGHT; i++) {
final BlockDataGenerator.BlockOptions options =
new BlockDataGenerator.BlockOptions()
.setBlockNumber(i)
.setParentHash(remoteBlockchain.getBlockHashByNumber(i - 1).orElseThrow());
final Block block = blockDataGenerator.block(options);
final List<TransactionReceipt> receipts = blockDataGenerator.receipts(block);
remoteBlockchain.appendBlock(block, receipts);
}
EthProtocolManager ethProtocolManager = EthProtocolManagerTestUtil.create();
peer = EthProtocolManagerTestUtil.createPeer(ethProtocolManager);
EthContext ethContext = ethProtocolManager.ethContext();
backwardSyncLookupService =
spy(
new BackwardSyncLookupService(
protocolSchedule, ethContext, metricsSystem, protocolContext));
}
@Test
public void shouldFindABlockWhenResponding() throws Exception {
final Hash hash = getBlockByNumber(23).getHash();
final CompletableFuture<List<Block>> future = backwardSyncLookupService.lookup(hash);
respondUntilFutureIsDone(future);
final List<Block> blocks = future.get();
assertThat(blocks.get(0)).isEqualTo(getBlockByNumber(23));
}
private void respondUntilFutureIsDone(final CompletableFuture<?> future) {
final RespondingEthPeer.Responder responder =
RespondingEthPeer.blockchainResponder(remoteBlockchain);
peer.respondWhileOtherThreadsWork(responder, () -> !future.isDone());
}
@NotNull
private Block getBlockByNumber(final int number) {
return remoteBlockchain.getBlockByNumber(number).orElseThrow();
}
}

@ -34,12 +34,16 @@ import org.hyperledger.besu.ethereum.eth.manager.EthContext;
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.mainnet.MainnetBlockHeaderFunctions;
import org.hyperledger.besu.ethereum.mainnet.MainnetProtocolSchedule;
import org.hyperledger.besu.ethereum.mainnet.ProtocolSchedule;
import org.hyperledger.besu.services.kvstore.InMemoryKeyValueStorage;
import java.util.List;
import java.util.Optional;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeoutException;
import org.jetbrains.annotations.NotNull;
import org.junit.Before;
@ -51,23 +55,36 @@ import org.mockito.Mockito;
import org.mockito.junit.MockitoJUnitRunner;
@RunWith(MockitoJUnitRunner.class)
public class BackwardSyncStepTest {
public class BackwardSyncPhaseTest {
public static final int REMOTE_HEIGHT = 50;
public static final int LOCAL_HEIGHT = 25;
private static final BlockDataGenerator blockDataGenerator = new BlockDataGenerator();
@Mock(answer = Answers.RETURNS_DEEP_STUBS)
private BackwardsSyncContext context;
private MutableBlockchain remoteBlockchain;
private RespondingEthPeer peer;
private BackwardSyncContext context;
private final ProtocolSchedule protocolSchedule =
MainnetProtocolSchedule.fromConfig(new StubGenesisConfigOptions());
private MutableBlockchain remoteBlockchain;
private RespondingEthPeer peer;
GenericKeyValueStorageFacade<Hash, BlockHeader> headersStorage;
GenericKeyValueStorageFacade<Hash, Block> blocksStorage;
@Before
public void setup() {
headersStorage =
new GenericKeyValueStorageFacade<>(
Hash::toArrayUnsafe,
new BlocksHeadersConvertor(new MainnetBlockHeaderFunctions()),
new InMemoryKeyValueStorage());
blocksStorage =
new GenericKeyValueStorageFacade<>(
Hash::toArrayUnsafe,
new BlocksConvertor(new MainnetBlockHeaderFunctions()),
new InMemoryKeyValueStorage());
Block genesisBlock = blockDataGenerator.genesisBlock();
remoteBlockchain = createInMemoryBlockchain(genesisBlock);
MutableBlockchain localBlockchain = createInMemoryBlockchain(genesisBlock);
@ -94,12 +111,22 @@ public class BackwardSyncStepTest {
when(context.getEthContext()).thenReturn(ethContext);
}
@Test
public void shouldWaitWhenTTDNotReached()
throws ExecutionException, InterruptedException, TimeoutException {
final BackwardChain backwardChain = createBackwardChain(LOCAL_HEIGHT + 3);
when(context.isOnTTD()).thenReturn(false).thenReturn(false).thenReturn(true);
BackwardSyncPhase step = new BackwardSyncPhase(context, backwardChain);
step.waitForTTD();
verify(context, Mockito.times(2)).getEthContext();
}
@Test
public void shouldFindHeaderWhenRequested() throws Exception {
final BackwardChain backwardChain = createBackwardChain(LOCAL_HEIGHT + 3);
BackwardSyncStep step = new BackwardSyncStep(context, backwardChain);
when(context.isOnTTD()).thenReturn(true);
BackwardSyncPhase step = new BackwardSyncPhase(context, backwardChain);
when(context.getCurrentChain()).thenReturn(Optional.of(backwardChain));
final RespondingEthPeer.Responder responder =
RespondingEthPeer.blockchainResponder(remoteBlockchain);
@ -111,10 +138,10 @@ public class BackwardSyncStepTest {
@Test
public void shouldFindHashToSync() {
BackwardSyncStep step =
new BackwardSyncStep(context, createBackwardChain(REMOTE_HEIGHT - 4, REMOTE_HEIGHT));
BackwardSyncPhase step =
new BackwardSyncPhase(context, createBackwardChain(REMOTE_HEIGHT - 4, REMOTE_HEIGHT));
final Hash hash = step.earliestUnprocessedHash();
final Hash hash = step.earliestUnprocessedHash(null);
assertThat(hash).isEqualTo(getBlockByNumber(REMOTE_HEIGHT - 4).getHeader().getParentHash());
}
@ -123,15 +150,15 @@ public class BackwardSyncStepTest {
public void shouldFailWhenNothingToSync() {
final BackwardChain chain = createBackwardChain(REMOTE_HEIGHT);
chain.dropFirstHeader();
BackwardSyncStep step = new BackwardSyncStep(context, chain);
assertThatThrownBy(step::earliestUnprocessedHash)
BackwardSyncPhase step = new BackwardSyncPhase(context, chain);
assertThatThrownBy(() -> step.earliestUnprocessedHash(null))
.isInstanceOf(BackwardSyncException.class)
.hasMessageContaining("No unprocessed hashes during backward sync");
}
@Test
public void shouldRequestHeaderWhenAsked() throws Exception {
BackwardSyncStep step = new BackwardSyncStep(context, createBackwardChain(REMOTE_HEIGHT - 1));
BackwardSyncPhase step = new BackwardSyncPhase(context, createBackwardChain(REMOTE_HEIGHT - 1));
final Block lookingForBlock = getBlockByNumber(REMOTE_HEIGHT - 2);
final RespondingEthPeer.Responder responder =
@ -147,7 +174,7 @@ public class BackwardSyncStepTest {
@Test
public void shouldThrowWhenResponseIsEmptyWhenRequestingHeader() throws Exception {
BackwardSyncStep step = new BackwardSyncStep(context, createBackwardChain(REMOTE_HEIGHT - 1));
BackwardSyncPhase step = new BackwardSyncPhase(context, createBackwardChain(REMOTE_HEIGHT - 1));
final Block lookingForBlock = getBlockByNumber(REMOTE_HEIGHT - 2);
final RespondingEthPeer.Responder responder = RespondingEthPeer.emptyResponder();
@ -169,11 +196,11 @@ public class BackwardSyncStepTest {
when(header.getNumber()).thenReturn(12345L);
BackwardSyncStep step = new BackwardSyncStep(context, chain);
BackwardSyncPhase step = new BackwardSyncPhase(context, chain);
step.saveHeader(header);
verify(chain).saveHeader(header);
verify(chain).prependAncestorsHeader(header);
verify(context).putCurrentChainToHeight(12345L, chain);
}
@ -181,11 +208,12 @@ public class BackwardSyncStepTest {
public void shouldMergeWhenPossible() {
BackwardChain backwardChain = createBackwardChain(REMOTE_HEIGHT - 3, REMOTE_HEIGHT);
backwardChain = spy(backwardChain);
BackwardSyncStep step = new BackwardSyncStep(context, backwardChain);
BackwardSyncPhase step = new BackwardSyncPhase(context, backwardChain);
final BackwardChain historicalChain =
createBackwardChain(REMOTE_HEIGHT - 10, REMOTE_HEIGHT - 4);
when(context.findCorrectChainFromPivot(REMOTE_HEIGHT - 4)).thenReturn(historicalChain);
when(context.findCorrectChainFromPivot(REMOTE_HEIGHT - 4))
.thenReturn(Optional.of(historicalChain));
assertThat(backwardChain.getFirstAncestorHeader().orElseThrow())
.isEqualTo(getBlockByNumber(REMOTE_HEIGHT - 3).getHeader());
@ -193,25 +221,25 @@ public class BackwardSyncStepTest {
assertThat(backwardChain.getFirstAncestorHeader().orElseThrow())
.isEqualTo(getBlockByNumber(REMOTE_HEIGHT - 10).getHeader());
verify(backwardChain).merge(historicalChain);
verify(backwardChain).prependChain(historicalChain);
}
@Test
public void shouldNotMergeWhenNotPossible() {
BackwardChain backwardChain = createBackwardChain(REMOTE_HEIGHT - 5, REMOTE_HEIGHT);
backwardChain = spy(backwardChain);
when(context.findCorrectChainFromPivot(any(Long.class))).thenReturn(null);
BackwardSyncStep step = new BackwardSyncStep(context, backwardChain);
when(context.findCorrectChainFromPivot(any(Long.class))).thenReturn(Optional.empty());
BackwardSyncPhase step = new BackwardSyncPhase(context, backwardChain);
step.possibleMerge(null);
verify(backwardChain, never()).merge(any());
verify(backwardChain, never()).prependChain(any());
}
@Test
public void shouldFinishWhenNoMoreSteps() {
BackwardChain backwardChain = createBackwardChain(LOCAL_HEIGHT, LOCAL_HEIGHT + 10);
BackwardSyncStep step = new BackwardSyncStep(context, backwardChain);
BackwardSyncPhase step = new BackwardSyncPhase(context, backwardChain);
final CompletableFuture<Void> completableFuture =
step.possiblyMoreBackwardSteps(getBlockByNumber(LOCAL_HEIGHT).getHeader());
@ -223,7 +251,7 @@ public class BackwardSyncStepTest {
@Test
public void shouldFinishExceptionallyWhenHeaderIsBellowBlockchainHeightButUnknown() {
BackwardChain backwardChain = createBackwardChain(LOCAL_HEIGHT, LOCAL_HEIGHT + 10);
BackwardSyncStep step = new BackwardSyncStep(context, backwardChain);
BackwardSyncPhase step = new BackwardSyncPhase(context, backwardChain);
final CompletableFuture<Void> completableFuture =
step.possiblyMoreBackwardSteps(
@ -235,7 +263,7 @@ public class BackwardSyncStepTest {
@Test
public void shouldCreateAnotherStepWhenThereIsWorkToBeDone() {
BackwardChain backwardChain = createBackwardChain(LOCAL_HEIGHT + 3, LOCAL_HEIGHT + 10);
BackwardSyncStep step = spy(new BackwardSyncStep(context, backwardChain));
BackwardSyncPhase step = spy(new BackwardSyncPhase(context, backwardChain));
step.possiblyMoreBackwardSteps(backwardChain.getFirstAncestorHeader().orElseThrow());
@ -245,14 +273,15 @@ public class BackwardSyncStepTest {
private BackwardChain createBackwardChain(final int from, final int until) {
BackwardChain chain = createBackwardChain(until);
for (int i = until; i > from; --i) {
chain.saveHeader(getBlockByNumber(i - 1).getHeader());
chain.prependAncestorsHeader(getBlockByNumber(i - 1).getHeader());
}
return chain;
}
@NotNull
private BackwardChain createBackwardChain(final int number) {
return new BackwardChain(remoteBlockchain.getBlockByNumber(number).orElseThrow());
return new BackwardChain(
headersStorage, blocksStorage, remoteBlockchain.getBlockByNumber(number).orElseThrow());
}
@NotNull

@ -18,11 +18,16 @@ import static org.assertj.core.api.AssertionsForClassTypes.assertThat;
import static org.assertj.core.api.AssertionsForClassTypes.assertThatThrownBy;
import static org.mockito.Mockito.when;
import org.hyperledger.besu.datatypes.Hash;
import org.hyperledger.besu.ethereum.core.Block;
import org.hyperledger.besu.ethereum.core.BlockHeader;
import org.hyperledger.besu.ethereum.mainnet.MainnetBlockHeaderFunctions;
import org.hyperledger.besu.services.kvstore.InMemoryKeyValueStorage;
import java.util.List;
import java.util.Optional;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutionException;
import org.jetbrains.annotations.NotNull;
import org.junit.Before;
@ -36,13 +41,25 @@ public class BackwardSyncTaskTest {
public static final int HEIGHT = 20_000;
@Mock private BackwardsSyncContext context;
@Mock private BackwardSyncContext context;
private List<Block> blocks;
GenericKeyValueStorageFacade<Hash, BlockHeader> headersStorage;
GenericKeyValueStorageFacade<Hash, Block> blocksStorage;
@Before
public void initContextAndChain() {
blocks = ChainForTestCreator.prepareChain(2, HEIGHT);
headersStorage =
new GenericKeyValueStorageFacade<>(
Hash::toArrayUnsafe,
new BlocksHeadersConvertor(new MainnetBlockHeaderFunctions()),
new InMemoryKeyValueStorage());
blocksStorage =
new GenericKeyValueStorageFacade<>(
Hash::toArrayUnsafe,
new BlocksConvertor(new MainnetBlockHeaderFunctions()),
new InMemoryKeyValueStorage());
}
@Test
@ -58,7 +75,8 @@ public class BackwardSyncTaskTest {
@NotNull
private BackwardSyncTask createBackwardSyncTask() {
final BackwardChain backwardChain = new BackwardChain(blocks.get(1));
final BackwardChain backwardChain =
new BackwardChain(headersStorage, blocksStorage, blocks.get(1));
return createBackwardSyncTask(backwardChain);
}
@ -73,19 +91,20 @@ public class BackwardSyncTaskTest {
}
@Test
public void shouldFailWhenPivotIsDifferent() {
when(context.getCurrentChain()).thenReturn(Optional.of(new BackwardChain(blocks.get(0))));
public void shouldFinishImmediatellyFailWhenPivotIsDifferent()
throws ExecutionException, InterruptedException {
final BackwardChain backwardChain =
new BackwardChain(headersStorage, blocksStorage, blocks.get(0));
when(context.getCurrentChain()).thenReturn(Optional.of(backwardChain));
BackwardSyncTask step = createBackwardSyncTask();
CompletableFuture<Void> completableFuture = step.executeAsync(null);
assertThatThrownBy(completableFuture::get)
.getCause()
.isInstanceOf(BackwardSyncException.class)
.hasMessageContaining("The pivot changed");
assertThat(completableFuture.isDone()).isTrue();
}
@Test
public void shouldExecuteWhenPivotIsCorrect() {
final BackwardChain backwardChain = new BackwardChain(blocks.get(1));
final BackwardChain backwardChain =
new BackwardChain(headersStorage, blocksStorage, blocks.get(1));
BackwardSyncTask step = createBackwardSyncTask();
when(context.getCurrentChain()).thenReturn(Optional.of(backwardChain));
CompletableFuture<Void> completableFuture = step.executeAsync(null);

@ -26,6 +26,7 @@ import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;
import org.hyperledger.besu.config.StubGenesisConfigOptions;
import org.hyperledger.besu.datatypes.Hash;
import org.hyperledger.besu.ethereum.BlockValidator;
import org.hyperledger.besu.ethereum.BlockValidator.Result;
import org.hyperledger.besu.ethereum.chain.MutableBlockchain;
@ -37,12 +38,14 @@ import org.hyperledger.besu.ethereum.eth.manager.EthContext;
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.mainnet.MainnetBlockHeaderFunctions;
import org.hyperledger.besu.ethereum.mainnet.MainnetProtocolSchedule;
import org.hyperledger.besu.ethereum.mainnet.ProtocolSchedule;
import org.hyperledger.besu.ethereum.referencetests.ReferenceTestWorldState;
import org.hyperledger.besu.services.kvstore.InMemoryKeyValueStorage;
import java.util.Collections;
import java.util.List;
import java.util.Optional;
import java.util.concurrent.CompletableFuture;
import org.jetbrains.annotations.NotNull;
@ -54,14 +57,14 @@ import org.mockito.Mock;
import org.mockito.junit.MockitoJUnitRunner;
@RunWith(MockitoJUnitRunner.class)
public class ForwardSyncStepTest {
public class ForwardSyncPhaseTest {
public static final int REMOTE_HEIGHT = 50;
public static final int LOCAL_HEIGHT = 25;
private static final BlockDataGenerator blockDataGenerator = new BlockDataGenerator();
@Mock(answer = Answers.RETURNS_DEEP_STUBS)
private BackwardsSyncContext context;
private BackwardSyncContext context;
private MutableBlockchain remoteBlockchain;
private RespondingEthPeer peer;
@ -69,9 +72,22 @@ public class ForwardSyncStepTest {
private final ProtocolSchedule protocolSchedule =
MainnetProtocolSchedule.fromConfig(new StubGenesisConfigOptions());
private MutableBlockchain localBlockchain;
GenericKeyValueStorageFacade<Hash, BlockHeader> headersStorage;
GenericKeyValueStorageFacade<Hash, Block> blocksStorage;
@Before
public void setup() {
headersStorage =
new GenericKeyValueStorageFacade<>(
Hash::toArrayUnsafe,
new BlocksHeadersConvertor(new MainnetBlockHeaderFunctions()),
new InMemoryKeyValueStorage());
blocksStorage =
new GenericKeyValueStorageFacade<>(
Hash::toArrayUnsafe,
new BlocksConvertor(new MainnetBlockHeaderFunctions()),
new InMemoryKeyValueStorage());
Block genesisBlock = blockDataGenerator.genesisBlock();
remoteBlockchain = createInMemoryBlockchain(genesisBlock);
localBlockchain = createInMemoryBlockchain(genesisBlock);
@ -112,8 +128,7 @@ public class ForwardSyncStepTest {
@Test
public void shouldExecuteForwardSyncWhenPossible() throws Exception {
final BackwardChain backwardChain = createBackwardChain(LOCAL_HEIGHT, LOCAL_HEIGHT + 3);
ForwardSyncStep step = new ForwardSyncStep(context, backwardChain);
when(context.getCurrentChain()).thenReturn(Optional.of(backwardChain));
ForwardSyncPhase step = new ForwardSyncPhase(context, backwardChain);
final RespondingEthPeer.Responder responder =
RespondingEthPeer.blockchainResponder(remoteBlockchain);
@ -137,11 +152,11 @@ public class ForwardSyncStepTest {
@Test
public void shouldDropHeadersAsLongAsWeKnowThem() {
final BackwardChain backwardChain = createBackwardChain(LOCAL_HEIGHT - 5, LOCAL_HEIGHT + 3);
ForwardSyncStep step = new ForwardSyncStep(context, backwardChain);
ForwardSyncPhase step = new ForwardSyncPhase(context, backwardChain);
assertThat(backwardChain.getFirstAncestorHeader().orElseThrow())
.isEqualTo(getBlockByNumber(LOCAL_HEIGHT - 5).getHeader());
step.processKnownAncestors(null);
step.returnFirstNUnknownHeaders(null);
assertThat(backwardChain.getFirstAncestorHeader().orElseThrow())
.isEqualTo(getBlockByNumber(LOCAL_HEIGHT + 1).getHeader());
}
@ -151,9 +166,9 @@ public class ForwardSyncStepTest {
final BackwardChain backwardChain = createBackwardChain(LOCAL_HEIGHT - 5, LOCAL_HEIGHT);
backwardChain.appendExpectedBlock(getBlockByNumber(LOCAL_HEIGHT + 1));
final BackwardChain finalChain = createBackwardChain(LOCAL_HEIGHT + 2, LOCAL_HEIGHT + 5);
finalChain.merge(backwardChain);
finalChain.prependChain(backwardChain);
ForwardSyncStep step = new ForwardSyncStep(context, finalChain);
ForwardSyncPhase step = new ForwardSyncPhase(context, finalChain);
assertThat(finalChain.getFirstAncestorHeader().orElseThrow())
.isEqualTo(getBlockByNumber(LOCAL_HEIGHT - 5).getHeader());
@ -162,10 +177,37 @@ public class ForwardSyncStepTest {
.isEqualTo(getBlockByNumber(LOCAL_HEIGHT + 2).getHeader());
}
@Test
public void shouldMergeEvenLongerChains() {
final BackwardChain backwardChain = createBackwardChain(LOCAL_HEIGHT - 5, LOCAL_HEIGHT + 7);
backwardChain.appendExpectedBlock(getBlockByNumber(LOCAL_HEIGHT + 1));
final BackwardChain finalChain = createBackwardChain(LOCAL_HEIGHT + 2, LOCAL_HEIGHT + 5);
finalChain.prependChain(backwardChain);
ForwardSyncPhase step = new ForwardSyncPhase(context, finalChain);
assertThat(finalChain.getFirstAncestorHeader().orElseThrow())
.isEqualTo(getBlockByNumber(LOCAL_HEIGHT - 5).getHeader());
step.processKnownAncestors(null);
assertThat(finalChain.getFirstAncestorHeader().orElseThrow())
.isEqualTo(getBlockByNumber(LOCAL_HEIGHT + 2).getHeader());
}
@Test
public void shouldNotRequestWhenNull() {
ForwardSyncPhase phase = new ForwardSyncPhase(null, null);
final CompletableFuture<Void> completableFuture = phase.possibleRequestBlock(null);
assertThat(completableFuture.isDone()).isTrue();
final CompletableFuture<Void> completableFuture1 =
phase.possibleRequestBodies(Collections.emptyList());
assertThat(completableFuture1.isDone()).isTrue();
}
@Test
public void shouldFindBlockWhenRequested() throws Exception {
ForwardSyncStep step =
new ForwardSyncStep(context, createBackwardChain(LOCAL_HEIGHT + 1, LOCAL_HEIGHT + 3));
ForwardSyncPhase step =
new ForwardSyncPhase(context, createBackwardChain(LOCAL_HEIGHT + 1, LOCAL_HEIGHT + 3));
final RespondingEthPeer.Responder responder =
RespondingEthPeer.blockchainResponder(remoteBlockchain);
@ -179,14 +221,24 @@ public class ForwardSyncStepTest {
@Test
public void shouldCreateAnotherStepWhenThereIsWorkToBeDone() {
BackwardChain backwardChain = createBackwardChain(LOCAL_HEIGHT + 3, LOCAL_HEIGHT + 10);
ForwardSyncStep step = spy(new ForwardSyncStep(context, backwardChain));
BackwardChain backwardChain = createBackwardChain(LOCAL_HEIGHT + 1, LOCAL_HEIGHT + 10);
ForwardSyncPhase step = spy(new ForwardSyncPhase(context, backwardChain));
step.possiblyMoreForwardSteps(backwardChain.getFirstAncestorHeader().orElseThrow());
verify(step).executeAsync(any());
}
@Test
public void shouldCreateBackwardStepWhenParentOfWorkIsNotImportedYet() {
BackwardChain backwardChain = createBackwardChain(LOCAL_HEIGHT + 3, LOCAL_HEIGHT + 10);
ForwardSyncPhase step = spy(new ForwardSyncPhase(context, backwardChain));
step.possiblyMoreForwardSteps(backwardChain.getFirstAncestorHeader().orElseThrow());
verify(step).executeBackwardAsync(any());
}
@Test
public void shouldAddSuccessorsWhenNoUnknownBlockSet() throws Exception {
BackwardChain backwardChain = createBackwardChain(LOCAL_HEIGHT - 3, LOCAL_HEIGHT);
@ -194,7 +246,7 @@ public class ForwardSyncStepTest {
backwardChain.appendExpectedBlock(getBlockByNumber(LOCAL_HEIGHT + 2));
backwardChain.appendExpectedBlock(getBlockByNumber(LOCAL_HEIGHT + 3));
ForwardSyncStep step = new ForwardSyncStep(context, backwardChain);
ForwardSyncPhase step = new ForwardSyncPhase(context, backwardChain);
final BlockHeader header = step.processKnownAncestors(null);
assertThat(header).isNull();
@ -206,14 +258,15 @@ public class ForwardSyncStepTest {
private BackwardChain createBackwardChain(final int from, final int until) {
BackwardChain chain = backwardChainFromBlock(until);
for (int i = until; i > from; --i) {
chain.saveHeader(getBlockByNumber(i - 1).getHeader());
chain.prependAncestorsHeader(getBlockByNumber(i - 1).getHeader());
}
return chain;
}
@NotNull
private BackwardChain backwardChainFromBlock(final int number) {
return new BackwardChain(remoteBlockchain.getBlockByNumber(number).orElseThrow());
return new BackwardChain(
headersStorage, blocksStorage, remoteBlockchain.getBlockByNumber(number).orElseThrow());
}
@NotNull

@ -19,51 +19,75 @@ import static org.assertj.core.api.AssertionsForClassTypes.assertThatThrownBy;
import static org.hyperledger.besu.ethereum.eth.sync.backwardsync.ChainForTestCreator.prepareChain;
import static org.hyperledger.besu.ethereum.eth.sync.backwardsync.ChainForTestCreator.prepareWrongParentHash;
import org.hyperledger.besu.datatypes.Hash;
import org.hyperledger.besu.ethereum.core.Block;
import org.hyperledger.besu.ethereum.core.BlockHeader;
import org.hyperledger.besu.ethereum.mainnet.MainnetBlockHeaderFunctions;
import org.hyperledger.besu.services.kvstore.InMemoryKeyValueStorage;
import java.util.List;
import org.junit.Before;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.mockito.junit.MockitoJUnitRunner;
public class BackwardChainTest {
@RunWith(MockitoJUnitRunner.class)
public class InMemoryBackwardChainTest {
public static final int HEIGHT = 20_000;
public static final int ELEMENTS = 20;
private List<Block> blocks;
GenericKeyValueStorageFacade<Hash, BlockHeader> headersStorage;
GenericKeyValueStorageFacade<Hash, Block> blocksStorage;
@Before
public void prepareData() {
headersStorage =
new GenericKeyValueStorageFacade<>(
Hash::toArrayUnsafe,
new BlocksHeadersConvertor(new MainnetBlockHeaderFunctions()),
new InMemoryKeyValueStorage());
blocksStorage =
new GenericKeyValueStorageFacade<>(
Hash::toArrayUnsafe,
new BlocksConvertor(new MainnetBlockHeaderFunctions()),
new InMemoryKeyValueStorage());
blocks = prepareChain(ELEMENTS, HEIGHT);
}
@Test
public void shouldReturnFirstHeaderCorrectly() {
BackwardChain backwardChain = new BackwardChain(blocks.get(blocks.size() - 1));
backwardChain.saveHeader(blocks.get(blocks.size() - 2).getHeader());
backwardChain.saveHeader(blocks.get(blocks.size() - 3).getHeader());
backwardChain.saveHeader(blocks.get(blocks.size() - 4).getHeader());
BackwardChain backwardChain =
new BackwardChain(headersStorage, blocksStorage, blocks.get(blocks.size() - 1));
backwardChain.prependAncestorsHeader(blocks.get(blocks.size() - 2).getHeader());
backwardChain.prependAncestorsHeader(blocks.get(blocks.size() - 3).getHeader());
backwardChain.prependAncestorsHeader(blocks.get(blocks.size() - 4).getHeader());
BlockHeader firstHeader = backwardChain.getFirstAncestorHeader().orElseThrow();
assertThat(firstHeader).isEqualTo(blocks.get(blocks.size() - 4).getHeader());
}
@Test
public void shouldSaveHeadersWhenHeightAndHashMatches() {
BackwardChain backwardChain = new BackwardChain(blocks.get(blocks.size() - 1));
backwardChain.saveHeader(blocks.get(blocks.size() - 2).getHeader());
backwardChain.saveHeader(blocks.get(blocks.size() - 3).getHeader());
backwardChain.saveHeader(blocks.get(blocks.size() - 4).getHeader());
BackwardChain backwardChain =
new BackwardChain(headersStorage, blocksStorage, blocks.get(blocks.size() - 1));
backwardChain.prependAncestorsHeader(blocks.get(blocks.size() - 2).getHeader());
backwardChain.prependAncestorsHeader(blocks.get(blocks.size() - 3).getHeader());
backwardChain.prependAncestorsHeader(blocks.get(blocks.size() - 4).getHeader());
BlockHeader firstHeader = backwardChain.getFirstAncestorHeader().orElseThrow();
assertThat(firstHeader).isEqualTo(blocks.get(blocks.size() - 4).getHeader());
}
@Test
public void shouldNotSaveHeadersWhenWrongHeight() {
BackwardChain backwardChain = new BackwardChain(blocks.get(blocks.size() - 1));
backwardChain.saveHeader(blocks.get(blocks.size() - 2).getHeader());
backwardChain.saveHeader(blocks.get(blocks.size() - 3).getHeader());
assertThatThrownBy(() -> backwardChain.saveHeader(blocks.get(blocks.size() - 5).getHeader()))
BackwardChain backwardChain =
new BackwardChain(headersStorage, blocksStorage, blocks.get(blocks.size() - 1));
backwardChain.prependAncestorsHeader(blocks.get(blocks.size() - 2).getHeader());
backwardChain.prependAncestorsHeader(blocks.get(blocks.size() - 3).getHeader());
assertThatThrownBy(
() -> backwardChain.prependAncestorsHeader(blocks.get(blocks.size() - 5).getHeader()))
.isInstanceOf(BackwardSyncException.class)
.hasMessageContaining("Wrong height of header");
BlockHeader firstHeader = backwardChain.getFirstAncestorHeader().orElseThrow();
@ -72,11 +96,12 @@ public class BackwardChainTest {
@Test
public void shouldNotSaveHeadersWhenWrongHash() {
BackwardChain backwardChain = new BackwardChain(blocks.get(blocks.size() - 1));
backwardChain.saveHeader(blocks.get(blocks.size() - 2).getHeader());
backwardChain.saveHeader(blocks.get(blocks.size() - 3).getHeader());
BackwardChain backwardChain =
new BackwardChain(headersStorage, blocksStorage, blocks.get(blocks.size() - 1));
backwardChain.prependAncestorsHeader(blocks.get(blocks.size() - 2).getHeader());
backwardChain.prependAncestorsHeader(blocks.get(blocks.size() - 3).getHeader());
BlockHeader wrongHashHeader = prepareWrongParentHash(blocks.get(blocks.size() - 4).getHeader());
assertThatThrownBy(() -> backwardChain.saveHeader(wrongHashHeader))
assertThatThrownBy(() -> backwardChain.prependAncestorsHeader(wrongHashHeader))
.isInstanceOf(BackwardSyncException.class)
.hasMessageContaining("Hash of header does not match our expectations");
BlockHeader firstHeader = backwardChain.getFirstAncestorHeader().orElseThrow();
@ -86,18 +111,20 @@ public class BackwardChainTest {
@Test
public void shouldMergeConnectedChains() {
BackwardChain firstChain = new BackwardChain(blocks.get(blocks.size() - 1));
firstChain.saveHeader(blocks.get(blocks.size() - 2).getHeader());
firstChain.saveHeader(blocks.get(blocks.size() - 3).getHeader());
BackwardChain firstChain =
new BackwardChain(headersStorage, blocksStorage, blocks.get(blocks.size() - 1));
firstChain.prependAncestorsHeader(blocks.get(blocks.size() - 2).getHeader());
firstChain.prependAncestorsHeader(blocks.get(blocks.size() - 3).getHeader());
BackwardChain secondChain = new BackwardChain(blocks.get(blocks.size() - 4));
secondChain.saveHeader(blocks.get(blocks.size() - 5).getHeader());
secondChain.saveHeader(blocks.get(blocks.size() - 6).getHeader());
BackwardChain secondChain =
new BackwardChain(headersStorage, blocksStorage, blocks.get(blocks.size() - 4));
secondChain.prependAncestorsHeader(blocks.get(blocks.size() - 5).getHeader());
secondChain.prependAncestorsHeader(blocks.get(blocks.size() - 6).getHeader());
BlockHeader firstHeader = firstChain.getFirstAncestorHeader().orElseThrow();
assertThat(firstHeader).isEqualTo(blocks.get(blocks.size() - 3).getHeader());
firstChain.merge(secondChain);
firstChain.prependChain(secondChain);
firstHeader = firstChain.getFirstAncestorHeader().orElseThrow();
assertThat(firstHeader).isEqualTo(blocks.get(blocks.size() - 6).getHeader());
@ -106,18 +133,20 @@ public class BackwardChainTest {
@Test
public void shouldNotMergeNotConnectedChains() {
BackwardChain firstChain = new BackwardChain(blocks.get(blocks.size() - 1));
firstChain.saveHeader(blocks.get(blocks.size() - 2).getHeader());
firstChain.saveHeader(blocks.get(blocks.size() - 3).getHeader());
BackwardChain firstChain =
new BackwardChain(headersStorage, blocksStorage, blocks.get(blocks.size() - 1));
firstChain.prependAncestorsHeader(blocks.get(blocks.size() - 2).getHeader());
firstChain.prependAncestorsHeader(blocks.get(blocks.size() - 3).getHeader());
BackwardChain secondChain = new BackwardChain(blocks.get(blocks.size() - 5));
secondChain.saveHeader(blocks.get(blocks.size() - 6).getHeader());
secondChain.saveHeader(blocks.get(blocks.size() - 7).getHeader());
BackwardChain secondChain =
new BackwardChain(headersStorage, blocksStorage, blocks.get(blocks.size() - 5));
secondChain.prependAncestorsHeader(blocks.get(blocks.size() - 6).getHeader());
secondChain.prependAncestorsHeader(blocks.get(blocks.size() - 7).getHeader());
BlockHeader firstHeader = firstChain.getFirstAncestorHeader().orElseThrow();
assertThat(firstHeader).isEqualTo(blocks.get(blocks.size() - 3).getHeader());
firstChain.merge(secondChain);
firstChain.prependChain(secondChain);
firstHeader = firstChain.getFirstAncestorHeader().orElseThrow();
assertThat(firstHeader).isEqualTo(blocks.get(blocks.size() - 3).getHeader());
@ -126,9 +155,10 @@ public class BackwardChainTest {
@Test
public void shouldDropFromTheEnd() {
BackwardChain backwardChain = new BackwardChain(blocks.get(blocks.size() - 1));
backwardChain.saveHeader(blocks.get(blocks.size() - 2).getHeader());
backwardChain.saveHeader(blocks.get(blocks.size() - 3).getHeader());
BackwardChain backwardChain =
new BackwardChain(headersStorage, blocksStorage, blocks.get(blocks.size() - 1));
backwardChain.prependAncestorsHeader(blocks.get(blocks.size() - 2).getHeader());
backwardChain.prependAncestorsHeader(blocks.get(blocks.size() - 3).getHeader());
BlockHeader firstHeader = backwardChain.getFirstAncestorHeader().orElseThrow();
assertThat(firstHeader).isEqualTo(blocks.get(blocks.size() - 3).getHeader());

@ -28,6 +28,20 @@ public class Slf4jLambdaHelper {
// sonar code smell
private Slf4jLambdaHelper() {}
public static void warnLambda(
final Logger log, final String message, final Supplier<?>... params) {
if (log.isWarnEnabled()) {
log.warn(message, Arrays.stream(params).map(Supplier::get).toArray());
}
}
public static void infoLambda(
final Logger log, final String message, final Supplier<?>... params) {
if (log.isInfoEnabled()) {
log.info(message, Arrays.stream(params).map(Supplier::get).toArray());
}
}
public static void debugLambda(
final Logger log, final String message, final Supplier<?>... params) {
if (log.isDebugEnabled()) {
@ -41,11 +55,4 @@ public class Slf4jLambdaHelper {
log.trace(message, Arrays.stream(params).map(Supplier::get).toArray());
}
}
public static void warnLambda(
final Logger log, final String message, final Supplier<?>... params) {
if (log.isWarnEnabled()) {
log.warn(message, Arrays.stream(params).map(Supplier::get).toArray());
}
}
}

Loading…
Cancel
Save