mirror of
https://github.com/vacp2p/status-linea-besu.git
synced 2026-01-09 15:28:09 -05:00
reintroduce checking of block height for certain tasks when we are not PoS (Revert PR#3911) (#5083)
* reintroduce checking of block height for certain tasks when we are not PoS (Revert PR#3911) Signed-off-by: Stefan <stefan.pingel@consensys.net>
This commit is contained in:
@@ -75,6 +75,7 @@ import org.hyperledger.besu.ethereum.eth.transactions.TransactionPool;
|
||||
import org.hyperledger.besu.ethereum.eth.transactions.TransactionPoolConfiguration;
|
||||
import org.hyperledger.besu.ethereum.eth.transactions.TransactionPoolFactory;
|
||||
import org.hyperledger.besu.ethereum.mainnet.ProtocolSchedule;
|
||||
import org.hyperledger.besu.ethereum.mainnet.ProtocolSpec;
|
||||
import org.hyperledger.besu.ethereum.p2p.config.SubProtocolConfiguration;
|
||||
import org.hyperledger.besu.ethereum.storage.StorageProvider;
|
||||
import org.hyperledger.besu.ethereum.storage.keyvalue.KeyValueSegmentIdentifier;
|
||||
@@ -548,9 +549,12 @@ public abstract class BesuControllerBuilder implements MiningParameterOverrides
|
||||
}
|
||||
}
|
||||
final int maxMessageSize = ethereumWireProtocolConfiguration.getMaxMessageSize();
|
||||
final Supplier<ProtocolSpec> currentProtocolSpecSupplier =
|
||||
() -> protocolSchedule.getByBlockHeader(blockchain.getChainHeadHeader());
|
||||
final EthPeers ethPeers =
|
||||
new EthPeers(
|
||||
getSupportedProtocol(),
|
||||
currentProtocolSpecSupplier,
|
||||
clock,
|
||||
metricsSystem,
|
||||
maxPeers,
|
||||
|
||||
@@ -111,6 +111,7 @@ public class Istanbul99ProtocolManagerTest {
|
||||
EthPeers peers =
|
||||
new EthPeers(
|
||||
Istanbul99Protocol.NAME,
|
||||
() -> protocolSchedule.getByBlockHeader(blockchain.getChainHeadHeader()),
|
||||
TestClock.fixed(),
|
||||
new NoOpMetricsSystem(),
|
||||
25,
|
||||
|
||||
@@ -113,7 +113,8 @@ public class EthGetTransactionReceiptTest {
|
||||
null,
|
||||
Optional.of(PoWHasher.ETHASH_LIGHT),
|
||||
null,
|
||||
Optional.empty());
|
||||
Optional.empty(),
|
||||
true);
|
||||
private final ProtocolSpec statusTransactionTypeSpec =
|
||||
new ProtocolSpec(
|
||||
"status",
|
||||
@@ -140,7 +141,8 @@ public class EthGetTransactionReceiptTest {
|
||||
null,
|
||||
Optional.of(PoWHasher.ETHASH_LIGHT),
|
||||
null,
|
||||
Optional.empty());
|
||||
Optional.empty(),
|
||||
true);
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
private final ProtocolSchedule protocolSchedule = mock(ProtocolSchedule.class);
|
||||
|
||||
@@ -638,7 +638,8 @@ public abstract class MainnetProtocolSpecs {
|
||||
.difficultyCalculator(MainnetDifficultyCalculators.PROOF_OF_STAKE_DIFFICULTY)
|
||||
.blockHeaderValidatorBuilder(MainnetBlockHeaderValidator::mergeBlockHeaderValidator)
|
||||
.blockReward(Wei.ZERO)
|
||||
.name("ParisFork");
|
||||
.name("ParisFork")
|
||||
.isPoS(true);
|
||||
}
|
||||
|
||||
static ProtocolSpecBuilder shanghaiDefinition(
|
||||
|
||||
@@ -80,6 +80,7 @@ public class ProtocolSpec {
|
||||
|
||||
private final Optional<WithdrawalsProcessor> withdrawalsProcessor;
|
||||
|
||||
private final boolean isPoS;
|
||||
/**
|
||||
* Creates a new protocol specification instance.
|
||||
*
|
||||
@@ -108,6 +109,7 @@ public class ProtocolSpec {
|
||||
* @param powHasher the proof-of-work hasher
|
||||
* @param withdrawalsValidator the withdrawals validator to use
|
||||
* @param withdrawalsProcessor the Withdrawals processor to use
|
||||
* @param isPoS indicates whether the current spec is PoS
|
||||
*/
|
||||
public ProtocolSpec(
|
||||
final String name,
|
||||
@@ -134,7 +136,8 @@ public class ProtocolSpec {
|
||||
final BadBlockManager badBlockManager,
|
||||
final Optional<PoWHasher> powHasher,
|
||||
final WithdrawalsValidator withdrawalsValidator,
|
||||
final Optional<WithdrawalsProcessor> withdrawalsProcessor) {
|
||||
final Optional<WithdrawalsProcessor> withdrawalsProcessor,
|
||||
final boolean isPoS) {
|
||||
this.name = name;
|
||||
this.evm = evm;
|
||||
this.transactionValidator = transactionValidator;
|
||||
@@ -160,6 +163,7 @@ public class ProtocolSpec {
|
||||
this.powHasher = powHasher;
|
||||
this.withdrawalsValidator = withdrawalsValidator;
|
||||
this.withdrawalsProcessor = withdrawalsProcessor;
|
||||
this.isPoS = isPoS;
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -367,4 +371,13 @@ public class ProtocolSpec {
|
||||
public Optional<WithdrawalsProcessor> getWithdrawalsProcessor() {
|
||||
return withdrawalsProcessor;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns true if the network is running Proof of Stake
|
||||
*
|
||||
* @return true if the network is running Proof of Stake
|
||||
*/
|
||||
public boolean isPoS() {
|
||||
return isPoS;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -79,6 +79,7 @@ public class ProtocolSpecBuilder {
|
||||
private FeeMarket feeMarket = FeeMarket.legacy();
|
||||
private BadBlockManager badBlockManager;
|
||||
private PoWHasher powHasher = PoWHasher.ETHASH_LIGHT;
|
||||
private boolean isPoS = false;
|
||||
|
||||
public ProtocolSpecBuilder gasCalculator(final Supplier<GasCalculator> gasCalculatorBuilder) {
|
||||
this.gasCalculatorBuilder = gasCalculatorBuilder;
|
||||
@@ -257,6 +258,11 @@ public class ProtocolSpecBuilder {
|
||||
return this;
|
||||
}
|
||||
|
||||
public ProtocolSpecBuilder isPoS(final boolean isPoS) {
|
||||
this.isPoS = isPoS;
|
||||
return this;
|
||||
}
|
||||
|
||||
public ProtocolSpec build(final HeaderBasedProtocolSchedule protocolSchedule) {
|
||||
checkNotNull(gasCalculatorBuilder, "Missing gasCalculator");
|
||||
checkNotNull(gasLimitCalculator, "Missing gasLimitCalculator");
|
||||
@@ -363,7 +369,8 @@ public class ProtocolSpecBuilder {
|
||||
badBlockManager,
|
||||
Optional.ofNullable(powHasher),
|
||||
withdrawalsValidator,
|
||||
Optional.ofNullable(withdrawalsProcessor));
|
||||
Optional.ofNullable(withdrawalsProcessor),
|
||||
isPoS);
|
||||
}
|
||||
|
||||
private PrivateTransactionProcessor createPrivateTransactionProcessor(
|
||||
|
||||
@@ -18,6 +18,7 @@ import static org.hyperledger.besu.util.Slf4jLambdaHelper.infoLambda;
|
||||
|
||||
import org.hyperledger.besu.ethereum.eth.manager.EthPeer.DisconnectCallback;
|
||||
import org.hyperledger.besu.ethereum.eth.peervalidation.PeerValidator;
|
||||
import org.hyperledger.besu.ethereum.mainnet.ProtocolSpec;
|
||||
import org.hyperledger.besu.ethereum.p2p.rlpx.connections.PeerConnection;
|
||||
import org.hyperledger.besu.ethereum.p2p.rlpx.wire.messages.DisconnectMessage;
|
||||
import org.hyperledger.besu.metrics.BesuMetricCategory;
|
||||
@@ -36,6 +37,7 @@ import java.util.Optional;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.CopyOnWriteArrayList;
|
||||
import java.util.function.Predicate;
|
||||
import java.util.function.Supplier;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
@@ -67,26 +69,37 @@ public class EthPeers {
|
||||
private final Subscribers<ConnectCallback> connectCallbacks = Subscribers.create();
|
||||
private final Subscribers<DisconnectCallback> disconnectCallbacks = Subscribers.create();
|
||||
private final Collection<PendingPeerRequest> pendingRequests = new CopyOnWriteArrayList<>();
|
||||
private final Supplier<ProtocolSpec> currentProtocolSpecSupplier;
|
||||
|
||||
private Comparator<EthPeer> bestPeerComparator;
|
||||
|
||||
public EthPeers(
|
||||
final String protocolName,
|
||||
final Supplier<ProtocolSpec> currentProtocolSpecSupplier,
|
||||
final Clock clock,
|
||||
final MetricsSystem metricsSystem,
|
||||
final int maxPeers,
|
||||
final int maxMessageSize) {
|
||||
this(protocolName, clock, metricsSystem, maxPeers, maxMessageSize, Collections.emptyList());
|
||||
this(
|
||||
protocolName,
|
||||
currentProtocolSpecSupplier,
|
||||
clock,
|
||||
metricsSystem,
|
||||
maxPeers,
|
||||
maxMessageSize,
|
||||
Collections.emptyList());
|
||||
}
|
||||
|
||||
public EthPeers(
|
||||
final String protocolName,
|
||||
final Supplier<ProtocolSpec> currentProtocolSpecSupplier,
|
||||
final Clock clock,
|
||||
final MetricsSystem metricsSystem,
|
||||
final int maxPeers,
|
||||
final int maxMessageSize,
|
||||
final List<NodeMessagePermissioningProvider> permissioningProviders) {
|
||||
this.protocolName = protocolName;
|
||||
this.currentProtocolSpecSupplier = currentProtocolSpecSupplier;
|
||||
this.clock = clock;
|
||||
this.permissioningProviders = permissioningProviders;
|
||||
this.maxPeers = maxPeers;
|
||||
@@ -148,8 +161,16 @@ public class EthPeers {
|
||||
|
||||
public PendingPeerRequest executePeerRequest(
|
||||
final PeerRequest request, final long minimumBlockNumber, final Optional<EthPeer> peer) {
|
||||
final long actualMinBlockNumber;
|
||||
if (minimumBlockNumber > 0 && currentProtocolSpecSupplier.get().isPoS()) {
|
||||
// if on PoS do not enforce a min block number, since the estimated chain height of the remote
|
||||
// peer is not updated anymore.
|
||||
actualMinBlockNumber = 0;
|
||||
} else {
|
||||
actualMinBlockNumber = minimumBlockNumber;
|
||||
}
|
||||
final PendingPeerRequest pendingPeerRequest =
|
||||
new PendingPeerRequest(this, request, minimumBlockNumber, peer);
|
||||
new PendingPeerRequest(this, request, actualMinBlockNumber, peer);
|
||||
synchronized (this) {
|
||||
if (!pendingPeerRequest.attemptExecution()) {
|
||||
pendingRequests.add(pendingPeerRequest);
|
||||
|
||||
@@ -85,10 +85,6 @@ public abstract class AbstractPeerRequestTask<R> extends AbstractPeerTask<R> {
|
||||
});
|
||||
}
|
||||
|
||||
public PendingPeerRequest sendRequestToPeer(final PeerRequest request) {
|
||||
return sendRequestToPeer(request, 0L);
|
||||
}
|
||||
|
||||
public PendingPeerRequest sendRequestToPeer(
|
||||
final PeerRequest request, final long minimumBlockNumber) {
|
||||
return ethContext.getEthPeers().executePeerRequest(request, minimumBlockNumber, assignedPeer);
|
||||
|
||||
@@ -103,7 +103,7 @@ public class GetBlockFromPeerTask extends AbstractPeerTask<Block> {
|
||||
hash.map(
|
||||
value ->
|
||||
GetHeadersFromPeerByHashTask.forSingleHash(
|
||||
protocolSchedule, ethContext, value, metricsSystem))
|
||||
protocolSchedule, ethContext, value, blockNumber, metricsSystem))
|
||||
.orElseGet(
|
||||
() ->
|
||||
GetHeadersFromPeerByNumberTask.forSingleNumber(
|
||||
|
||||
@@ -83,12 +83,14 @@ public class GetBodiesFromPeerTask extends AbstractPeerRequestTask<List<Block>>
|
||||
protected PendingPeerRequest sendRequest() {
|
||||
final List<Hash> blockHashes =
|
||||
headers.stream().map(BlockHeader::getHash).collect(Collectors.toList());
|
||||
final long minimumRequiredBlockNumber = headers.get(headers.size() - 1).getNumber();
|
||||
|
||||
return sendRequestToPeer(
|
||||
peer -> {
|
||||
LOG.debug("Requesting {} bodies from peer {}.", blockHashes.size(), peer);
|
||||
return peer.getBodies(blockHashes);
|
||||
});
|
||||
},
|
||||
minimumRequiredBlockNumber);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
@@ -32,17 +32,20 @@ public class GetHeadersFromPeerByHashTask extends AbstractGetHeadersFromPeerTask
|
||||
private static final Logger LOG = LoggerFactory.getLogger(GetHeadersFromPeerByHashTask.class);
|
||||
|
||||
private final Hash referenceHash;
|
||||
private final long minimumRequiredBlockNumber;
|
||||
|
||||
@VisibleForTesting
|
||||
GetHeadersFromPeerByHashTask(
|
||||
final ProtocolSchedule protocolSchedule,
|
||||
final EthContext ethContext,
|
||||
final Hash referenceHash,
|
||||
final long minimumRequiredBlockNumber,
|
||||
final int count,
|
||||
final int skip,
|
||||
final boolean reverse,
|
||||
final MetricsSystem metricsSystem) {
|
||||
super(protocolSchedule, ethContext, count, skip, reverse, metricsSystem);
|
||||
this.minimumRequiredBlockNumber = minimumRequiredBlockNumber;
|
||||
checkNotNull(referenceHash);
|
||||
this.referenceHash = referenceHash;
|
||||
}
|
||||
@@ -51,40 +54,65 @@ public class GetHeadersFromPeerByHashTask extends AbstractGetHeadersFromPeerTask
|
||||
final ProtocolSchedule protocolSchedule,
|
||||
final EthContext ethContext,
|
||||
final Hash firstHash,
|
||||
final long firstBlockNumber,
|
||||
final int segmentLength,
|
||||
final MetricsSystem metricsSystem) {
|
||||
return new GetHeadersFromPeerByHashTask(
|
||||
protocolSchedule, ethContext, firstHash, segmentLength, 0, false, metricsSystem);
|
||||
protocolSchedule,
|
||||
ethContext,
|
||||
firstHash,
|
||||
firstBlockNumber,
|
||||
segmentLength,
|
||||
0,
|
||||
false,
|
||||
metricsSystem);
|
||||
}
|
||||
|
||||
public static AbstractGetHeadersFromPeerTask startingAtHash(
|
||||
final ProtocolSchedule protocolSchedule,
|
||||
final EthContext ethContext,
|
||||
final Hash firstHash,
|
||||
final long firstBlockNumber,
|
||||
final int segmentLength,
|
||||
final int skip,
|
||||
final MetricsSystem metricsSystem) {
|
||||
return new GetHeadersFromPeerByHashTask(
|
||||
protocolSchedule, ethContext, firstHash, segmentLength, skip, false, metricsSystem);
|
||||
protocolSchedule,
|
||||
ethContext,
|
||||
firstHash,
|
||||
firstBlockNumber,
|
||||
segmentLength,
|
||||
skip,
|
||||
false,
|
||||
metricsSystem);
|
||||
}
|
||||
|
||||
public static AbstractGetHeadersFromPeerTask endingAtHash(
|
||||
final ProtocolSchedule protocolSchedule,
|
||||
final EthContext ethContext,
|
||||
final Hash lastHash,
|
||||
final long lastBlockNumber,
|
||||
final int segmentLength,
|
||||
final MetricsSystem metricsSystem) {
|
||||
return new GetHeadersFromPeerByHashTask(
|
||||
protocolSchedule, ethContext, lastHash, segmentLength, 0, true, metricsSystem);
|
||||
protocolSchedule,
|
||||
ethContext,
|
||||
lastHash,
|
||||
lastBlockNumber,
|
||||
segmentLength,
|
||||
0,
|
||||
true,
|
||||
metricsSystem);
|
||||
}
|
||||
|
||||
public static AbstractGetHeadersFromPeerTask forSingleHash(
|
||||
final ProtocolSchedule protocolSchedule,
|
||||
final EthContext ethContext,
|
||||
final Hash hash,
|
||||
final long minimumRequiredBlockNumber,
|
||||
final MetricsSystem metricsSystem) {
|
||||
return new GetHeadersFromPeerByHashTask(
|
||||
protocolSchedule, ethContext, hash, 1, 0, false, metricsSystem);
|
||||
protocolSchedule, ethContext, hash, minimumRequiredBlockNumber, 1, 0, false, metricsSystem);
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -97,7 +125,8 @@ public class GetHeadersFromPeerByHashTask extends AbstractGetHeadersFromPeerTask
|
||||
referenceHash.slice(0, 6),
|
||||
peer);
|
||||
return peer.getHeadersByHash(referenceHash, count, skip, reverse);
|
||||
});
|
||||
},
|
||||
minimumRequiredBlockNumber);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
@@ -80,7 +80,8 @@ public class GetHeadersFromPeerByNumberTask extends AbstractGetHeadersFromPeerTa
|
||||
LOG.debug(
|
||||
"Requesting {} headers (blockNumber {}) from peer {}.", count, blockNumber, peer);
|
||||
return peer.getHeadersByNumber(blockNumber, count, skip, reverse);
|
||||
});
|
||||
},
|
||||
blockNumber);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
@@ -39,6 +39,7 @@ public class RetryingGetHeadersEndingAtFromPeerByHashTask
|
||||
|
||||
private final Hash referenceHash;
|
||||
private final ProtocolSchedule protocolSchedule;
|
||||
private final long minimumRequiredBlockNumber;
|
||||
private final int count;
|
||||
|
||||
@VisibleForTesting
|
||||
@@ -46,11 +47,13 @@ public class RetryingGetHeadersEndingAtFromPeerByHashTask
|
||||
final ProtocolSchedule protocolSchedule,
|
||||
final EthContext ethContext,
|
||||
final Hash referenceHash,
|
||||
final long minimumRequiredBlockNumber,
|
||||
final int count,
|
||||
final MetricsSystem metricsSystem,
|
||||
final int maxRetries) {
|
||||
super(ethContext, metricsSystem, List::isEmpty, maxRetries);
|
||||
this.protocolSchedule = protocolSchedule;
|
||||
this.minimumRequiredBlockNumber = minimumRequiredBlockNumber;
|
||||
this.count = count;
|
||||
checkNotNull(referenceHash);
|
||||
this.referenceHash = referenceHash;
|
||||
@@ -60,11 +63,18 @@ public class RetryingGetHeadersEndingAtFromPeerByHashTask
|
||||
final ProtocolSchedule protocolSchedule,
|
||||
final EthContext ethContext,
|
||||
final Hash referenceHash,
|
||||
final long minimumRequiredBlockNumber,
|
||||
final int count,
|
||||
final MetricsSystem metricsSystem,
|
||||
final int maxRetries) {
|
||||
return new RetryingGetHeadersEndingAtFromPeerByHashTask(
|
||||
protocolSchedule, ethContext, referenceHash, count, metricsSystem, maxRetries);
|
||||
protocolSchedule,
|
||||
ethContext,
|
||||
referenceHash,
|
||||
minimumRequiredBlockNumber,
|
||||
count,
|
||||
metricsSystem,
|
||||
maxRetries);
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -72,7 +82,12 @@ public class RetryingGetHeadersEndingAtFromPeerByHashTask
|
||||
final EthPeer currentPeer) {
|
||||
final AbstractGetHeadersFromPeerTask task =
|
||||
GetHeadersFromPeerByHashTask.endingAtHash(
|
||||
protocolSchedule, getEthContext(), referenceHash, count, getMetricsSystem());
|
||||
protocolSchedule,
|
||||
getEthContext(),
|
||||
referenceHash,
|
||||
minimumRequiredBlockNumber,
|
||||
count,
|
||||
getMetricsSystem());
|
||||
task.assignPeer(currentPeer);
|
||||
return executeSubTask(task::run)
|
||||
.thenApply(
|
||||
|
||||
@@ -73,6 +73,7 @@ public class ChainHeadTracker implements ConnectCallback {
|
||||
protocolSchedule,
|
||||
ethContext,
|
||||
Hash.wrap(peer.chainState().getBestBlock().getHash()),
|
||||
0,
|
||||
metricsSystem)
|
||||
.assignPeer(peer)
|
||||
.run()
|
||||
|
||||
@@ -96,6 +96,7 @@ public class DownloadHeadersStep
|
||||
protocolSchedule,
|
||||
ethContext,
|
||||
range.getStart().getHash(),
|
||||
range.getStart().getNumber(),
|
||||
headerRequestSize,
|
||||
metricsSystem)
|
||||
.assignPeer(range.getSyncTarget())
|
||||
|
||||
@@ -76,6 +76,7 @@ public class BackwardSyncStep {
|
||||
context.getProtocolSchedule(),
|
||||
context.getEthContext(),
|
||||
hash,
|
||||
0,
|
||||
batchSize,
|
||||
context.getMetricsSystem(),
|
||||
context.getEthContext().getEthPeers().peerCount());
|
||||
|
||||
@@ -171,7 +171,11 @@ public class FastSyncActions {
|
||||
private CompletableFuture<FastSyncState> downloadPivotBlockHeader(final Hash hash) {
|
||||
LOG.debug("Downloading pivot block header by hash {}", hash);
|
||||
return RetryingGetHeaderFromPeerByHashTask.byHash(
|
||||
protocolSchedule, ethContext, hash, metricsSystem)
|
||||
protocolSchedule,
|
||||
ethContext,
|
||||
hash,
|
||||
pivotBlockSelector.getMinRequiredBlockNumber(),
|
||||
metricsSystem)
|
||||
.getHeader()
|
||||
.whenComplete(
|
||||
(blockHeader, throwable) -> {
|
||||
|
||||
@@ -137,7 +137,7 @@ public class PivotSelectorFromSafeBlock implements PivotBlockSelector {
|
||||
|
||||
private CompletableFuture<BlockHeader> downloadBlockHeader(final Hash hash) {
|
||||
return RetryingGetHeaderFromPeerByHashTask.byHash(
|
||||
protocolSchedule, ethContext, hash, metricsSystem)
|
||||
protocolSchedule, ethContext, hash, 0, metricsSystem)
|
||||
.getHeader()
|
||||
.whenComplete(
|
||||
(blockHeader, throwable) -> {
|
||||
|
||||
@@ -106,6 +106,7 @@ public class RangeHeadersFetcher {
|
||||
protocolSchedule,
|
||||
ethContext,
|
||||
referenceHeader.getHash(),
|
||||
referenceHeader.getNumber(),
|
||||
// + 1 because lastHeader will be returned as well.
|
||||
headerCount + 1,
|
||||
skip,
|
||||
|
||||
@@ -166,7 +166,12 @@ public class DownloadHeaderSequenceTask extends AbstractRetryingPeerTask<List<Bl
|
||||
// Ask for count + 1 because we'll retrieve the previous header as well
|
||||
final AbstractGetHeadersFromPeerTask headersTask =
|
||||
GetHeadersFromPeerByHashTask.endingAtHash(
|
||||
protocolSchedule, ethContext, referenceHash, count + 1, metricsSystem);
|
||||
protocolSchedule,
|
||||
ethContext,
|
||||
referenceHash,
|
||||
referenceHeaderForNextRequest.getNumber(),
|
||||
count + 1,
|
||||
metricsSystem);
|
||||
assignedPeer.ifPresent(headersTask::assignPeer);
|
||||
return headersTask.run();
|
||||
});
|
||||
|
||||
@@ -40,16 +40,19 @@ public class RetryingGetHeaderFromPeerByHashTask
|
||||
LoggerFactory.getLogger(RetryingGetHeaderFromPeerByHashTask.class);
|
||||
private final Hash referenceHash;
|
||||
private final ProtocolSchedule protocolSchedule;
|
||||
private final long minimumRequiredBlockNumber;
|
||||
|
||||
@VisibleForTesting
|
||||
RetryingGetHeaderFromPeerByHashTask(
|
||||
final ProtocolSchedule protocolSchedule,
|
||||
final EthContext ethContext,
|
||||
final Hash referenceHash,
|
||||
final long minimumRequiredBlockNumber,
|
||||
final MetricsSystem metricsSystem,
|
||||
final int maxRetries) {
|
||||
super(ethContext, metricsSystem, List::isEmpty, maxRetries);
|
||||
this.protocolSchedule = protocolSchedule;
|
||||
this.minimumRequiredBlockNumber = minimumRequiredBlockNumber;
|
||||
checkNotNull(referenceHash);
|
||||
this.referenceHash = referenceHash;
|
||||
}
|
||||
@@ -58,11 +61,13 @@ public class RetryingGetHeaderFromPeerByHashTask
|
||||
final ProtocolSchedule protocolSchedule,
|
||||
final EthContext ethContext,
|
||||
final Hash referenceHash,
|
||||
final long minimumRequiredBlockNumber,
|
||||
final MetricsSystem metricsSystem) {
|
||||
return new RetryingGetHeaderFromPeerByHashTask(
|
||||
protocolSchedule,
|
||||
ethContext,
|
||||
referenceHash,
|
||||
minimumRequiredBlockNumber,
|
||||
metricsSystem,
|
||||
ethContext.getEthPeers().peerCount());
|
||||
}
|
||||
@@ -71,7 +76,11 @@ public class RetryingGetHeaderFromPeerByHashTask
|
||||
protected CompletableFuture<List<BlockHeader>> executeTaskOnCurrentPeer(final EthPeer peer) {
|
||||
final AbstractGetHeadersFromPeerTask task =
|
||||
GetHeadersFromPeerByHashTask.forSingleHash(
|
||||
protocolSchedule, getEthContext(), referenceHash, getMetricsSystem());
|
||||
protocolSchedule,
|
||||
getEthContext(),
|
||||
referenceHash,
|
||||
minimumRequiredBlockNumber,
|
||||
getMetricsSystem());
|
||||
task.assignPeer(peer);
|
||||
return executeSubTask(task::run)
|
||||
.thenApply(
|
||||
|
||||
@@ -131,6 +131,7 @@ public final class EthProtocolManagerTest {
|
||||
public void handleMalformedRequestIdMessage() {
|
||||
try (final EthProtocolManager ethManager =
|
||||
EthProtocolManagerTestUtil.create(
|
||||
protocolSchedule,
|
||||
blockchain,
|
||||
() -> false,
|
||||
protocolContext.getWorldStateArchive(),
|
||||
@@ -149,6 +150,7 @@ public final class EthProtocolManagerTest {
|
||||
public void disconnectOnUnsolicitedMessage() {
|
||||
try (final EthProtocolManager ethManager =
|
||||
EthProtocolManagerTestUtil.create(
|
||||
protocolSchedule,
|
||||
blockchain,
|
||||
() -> false,
|
||||
protocolContext.getWorldStateArchive(),
|
||||
@@ -166,6 +168,7 @@ public final class EthProtocolManagerTest {
|
||||
public void disconnectOnFailureToSendStatusMessage() {
|
||||
try (final EthProtocolManager ethManager =
|
||||
EthProtocolManagerTestUtil.create(
|
||||
protocolSchedule,
|
||||
blockchain,
|
||||
() -> false,
|
||||
protocolContext.getWorldStateArchive(),
|
||||
@@ -184,6 +187,7 @@ public final class EthProtocolManagerTest {
|
||||
public void disconnectOnWrongChainId() {
|
||||
try (final EthProtocolManager ethManager =
|
||||
EthProtocolManagerTestUtil.create(
|
||||
protocolSchedule,
|
||||
blockchain,
|
||||
() -> false,
|
||||
protocolContext.getWorldStateArchive(),
|
||||
@@ -214,6 +218,7 @@ public final class EthProtocolManagerTest {
|
||||
final MergePeerFilter mergePeerFilter = new MergePeerFilter();
|
||||
try (final EthProtocolManager ethManager =
|
||||
EthProtocolManagerTestUtil.create(
|
||||
protocolSchedule,
|
||||
blockchain,
|
||||
protocolContext.getWorldStateArchive(),
|
||||
transactionPool,
|
||||
@@ -262,6 +267,7 @@ public final class EthProtocolManagerTest {
|
||||
public void doNotDisconnectOnLargeMessageWithinLimits() {
|
||||
try (final EthProtocolManager ethManager =
|
||||
EthProtocolManagerTestUtil.create(
|
||||
protocolSchedule,
|
||||
blockchain,
|
||||
() -> false,
|
||||
protocolContext.getWorldStateArchive(),
|
||||
@@ -281,6 +287,7 @@ public final class EthProtocolManagerTest {
|
||||
public void disconnectOnWrongGenesisHash() {
|
||||
try (final EthProtocolManager ethManager =
|
||||
EthProtocolManagerTestUtil.create(
|
||||
protocolSchedule,
|
||||
blockchain,
|
||||
() -> false,
|
||||
protocolContext.getWorldStateArchive(),
|
||||
@@ -310,6 +317,7 @@ public final class EthProtocolManagerTest {
|
||||
public void doNotDisconnectOnValidMessage() {
|
||||
try (final EthProtocolManager ethManager =
|
||||
EthProtocolManagerTestUtil.create(
|
||||
protocolSchedule,
|
||||
blockchain,
|
||||
() -> false,
|
||||
protocolContext.getWorldStateArchive(),
|
||||
@@ -331,6 +339,7 @@ public final class EthProtocolManagerTest {
|
||||
final CompletableFuture<Void> done = new CompletableFuture<>();
|
||||
try (final EthProtocolManager ethManager =
|
||||
EthProtocolManagerTestUtil.create(
|
||||
protocolSchedule,
|
||||
blockchain,
|
||||
() -> false,
|
||||
protocolContext.getWorldStateArchive(),
|
||||
@@ -370,6 +379,7 @@ public final class EthProtocolManagerTest {
|
||||
EthProtocolConfiguration.builder().maxGetBlockHeaders(limit).build();
|
||||
try (final EthProtocolManager ethManager =
|
||||
EthProtocolManagerTestUtil.create(
|
||||
protocolSchedule,
|
||||
blockchain,
|
||||
() -> false,
|
||||
protocolContext.getWorldStateArchive(),
|
||||
@@ -406,6 +416,7 @@ public final class EthProtocolManagerTest {
|
||||
final CompletableFuture<Void> done = new CompletableFuture<>();
|
||||
try (final EthProtocolManager ethManager =
|
||||
EthProtocolManagerTestUtil.create(
|
||||
protocolSchedule,
|
||||
blockchain,
|
||||
() -> false,
|
||||
protocolContext.getWorldStateArchive(),
|
||||
@@ -442,6 +453,7 @@ public final class EthProtocolManagerTest {
|
||||
final CompletableFuture<Void> done = new CompletableFuture<>();
|
||||
try (final EthProtocolManager ethManager =
|
||||
EthProtocolManagerTestUtil.create(
|
||||
protocolSchedule,
|
||||
blockchain,
|
||||
() -> false,
|
||||
protocolContext.getWorldStateArchive(),
|
||||
@@ -481,6 +493,7 @@ public final class EthProtocolManagerTest {
|
||||
final CompletableFuture<Void> done = new CompletableFuture<>();
|
||||
try (final EthProtocolManager ethManager =
|
||||
EthProtocolManagerTestUtil.create(
|
||||
protocolSchedule,
|
||||
blockchain,
|
||||
() -> false,
|
||||
protocolContext.getWorldStateArchive(),
|
||||
@@ -541,6 +554,7 @@ public final class EthProtocolManagerTest {
|
||||
final CompletableFuture<Void> done = new CompletableFuture<>();
|
||||
try (final EthProtocolManager ethManager =
|
||||
EthProtocolManagerTestUtil.create(
|
||||
protocolSchedule,
|
||||
blockchain,
|
||||
() -> false,
|
||||
protocolContext.getWorldStateArchive(),
|
||||
@@ -578,6 +592,7 @@ public final class EthProtocolManagerTest {
|
||||
final CompletableFuture<Void> done = new CompletableFuture<>();
|
||||
try (final EthProtocolManager ethManager =
|
||||
EthProtocolManagerTestUtil.create(
|
||||
protocolSchedule,
|
||||
blockchain,
|
||||
() -> false,
|
||||
protocolContext.getWorldStateArchive(),
|
||||
@@ -612,6 +627,7 @@ public final class EthProtocolManagerTest {
|
||||
final CompletableFuture<Void> done = new CompletableFuture<>();
|
||||
try (final EthProtocolManager ethManager =
|
||||
EthProtocolManagerTestUtil.create(
|
||||
protocolSchedule,
|
||||
blockchain,
|
||||
() -> false,
|
||||
protocolContext.getWorldStateArchive(),
|
||||
@@ -664,6 +680,7 @@ public final class EthProtocolManagerTest {
|
||||
EthProtocolConfiguration.builder().maxGetBlockBodies(limit).build();
|
||||
try (final EthProtocolManager ethManager =
|
||||
EthProtocolManagerTestUtil.create(
|
||||
protocolSchedule,
|
||||
blockchain,
|
||||
() -> false,
|
||||
protocolContext.getWorldStateArchive(),
|
||||
@@ -712,6 +729,7 @@ public final class EthProtocolManagerTest {
|
||||
final CompletableFuture<Void> done = new CompletableFuture<>();
|
||||
try (final EthProtocolManager ethManager =
|
||||
EthProtocolManagerTestUtil.create(
|
||||
protocolSchedule,
|
||||
blockchain,
|
||||
() -> false,
|
||||
protocolContext.getWorldStateArchive(),
|
||||
@@ -754,6 +772,7 @@ public final class EthProtocolManagerTest {
|
||||
final CompletableFuture<Void> done = new CompletableFuture<>();
|
||||
try (final EthProtocolManager ethManager =
|
||||
EthProtocolManagerTestUtil.create(
|
||||
protocolSchedule,
|
||||
blockchain,
|
||||
() -> false,
|
||||
protocolContext.getWorldStateArchive(),
|
||||
@@ -804,6 +823,7 @@ public final class EthProtocolManagerTest {
|
||||
EthProtocolConfiguration.builder().maxGetReceipts(limit).build();
|
||||
try (final EthProtocolManager ethManager =
|
||||
EthProtocolManagerTestUtil.create(
|
||||
protocolSchedule,
|
||||
blockchain,
|
||||
() -> false,
|
||||
protocolContext.getWorldStateArchive(),
|
||||
@@ -851,6 +871,7 @@ public final class EthProtocolManagerTest {
|
||||
final CompletableFuture<Void> done = new CompletableFuture<>();
|
||||
try (final EthProtocolManager ethManager =
|
||||
EthProtocolManagerTestUtil.create(
|
||||
protocolSchedule,
|
||||
blockchain,
|
||||
() -> false,
|
||||
protocolContext.getWorldStateArchive(),
|
||||
@@ -895,6 +916,7 @@ public final class EthProtocolManagerTest {
|
||||
|
||||
try (final EthProtocolManager ethManager =
|
||||
EthProtocolManagerTestUtil.create(
|
||||
protocolSchedule,
|
||||
blockchain,
|
||||
() -> false,
|
||||
protocolContext.getWorldStateArchive(),
|
||||
@@ -942,6 +964,7 @@ public final class EthProtocolManagerTest {
|
||||
public void newBlockMinedSendsNewBlockMessageToAllPeers() {
|
||||
try (final EthProtocolManager ethManager =
|
||||
EthProtocolManagerTestUtil.create(
|
||||
protocolSchedule,
|
||||
blockchain,
|
||||
() -> false,
|
||||
protocolContext.getWorldStateArchive(),
|
||||
@@ -1012,6 +1035,7 @@ public final class EthProtocolManagerTest {
|
||||
final CompletableFuture<Void> done = new CompletableFuture<>();
|
||||
try (final EthProtocolManager ethManager =
|
||||
EthProtocolManagerTestUtil.create(
|
||||
protocolSchedule,
|
||||
blockchain,
|
||||
() -> false,
|
||||
protocolContext.getWorldStateArchive(),
|
||||
@@ -1077,6 +1101,7 @@ public final class EthProtocolManagerTest {
|
||||
|
||||
try (final EthProtocolManager ethManager =
|
||||
EthProtocolManagerTestUtil.create(
|
||||
protocolSchedule,
|
||||
blockchain,
|
||||
ethScheduler,
|
||||
protocolContext.getWorldStateArchive(),
|
||||
@@ -1110,6 +1135,7 @@ public final class EthProtocolManagerTest {
|
||||
final EthScheduler ethScheduler = mock(EthScheduler.class);
|
||||
try (final EthProtocolManager ethManager =
|
||||
EthProtocolManagerTestUtil.create(
|
||||
protocolSchedule,
|
||||
blockchain,
|
||||
ethScheduler,
|
||||
protocolContext.getWorldStateArchive(),
|
||||
|
||||
@@ -48,12 +48,14 @@ import java.util.OptionalLong;
|
||||
public class EthProtocolManagerTestUtil {
|
||||
|
||||
public static EthProtocolManager create(
|
||||
final ProtocolSchedule protocolSchedule,
|
||||
final Blockchain blockchain,
|
||||
final TimeoutPolicy timeoutPolicy,
|
||||
final WorldStateArchive worldStateArchive,
|
||||
final TransactionPool transactionPool,
|
||||
final EthProtocolConfiguration ethereumWireProtocolConfiguration) {
|
||||
return create(
|
||||
protocolSchedule,
|
||||
blockchain,
|
||||
new DeterministicEthScheduler(timeoutPolicy),
|
||||
worldStateArchive,
|
||||
@@ -62,6 +64,7 @@ public class EthProtocolManagerTestUtil {
|
||||
}
|
||||
|
||||
public static EthProtocolManager create(
|
||||
final ProtocolSchedule protocolSchedule,
|
||||
final Blockchain blockchain,
|
||||
final WorldStateArchive worldStateArchive,
|
||||
final TransactionPool transactionPool,
|
||||
@@ -71,6 +74,7 @@ public class EthProtocolManagerTestUtil {
|
||||
EthPeers peers =
|
||||
new EthPeers(
|
||||
EthProtocol.NAME,
|
||||
() -> protocolSchedule.getByBlockHeader(blockchain.getChainHeadHeader()),
|
||||
TestClock.fixed(),
|
||||
new NoOpMetricsSystem(),
|
||||
25,
|
||||
@@ -145,15 +149,20 @@ public class EthProtocolManagerTestUtil {
|
||||
}
|
||||
|
||||
public static EthProtocolManager create(final Blockchain blockchain) {
|
||||
return create(blockchain, new DeterministicEthScheduler(TimeoutPolicy.NEVER_TIMEOUT));
|
||||
return create(
|
||||
ProtocolScheduleFixture.MAINNET,
|
||||
blockchain,
|
||||
new DeterministicEthScheduler(TimeoutPolicy.NEVER_TIMEOUT));
|
||||
}
|
||||
|
||||
public static EthProtocolManager create(
|
||||
final ProtocolSchedule protocolSchedule,
|
||||
final Blockchain blockchain,
|
||||
final WorldStateArchive worldStateArchive,
|
||||
final TransactionPool transactionPool,
|
||||
final EthProtocolConfiguration ethProtocolConfiguration) {
|
||||
return create(
|
||||
protocolSchedule,
|
||||
blockchain,
|
||||
new DeterministicEthScheduler(TimeoutPolicy.NEVER_TIMEOUT),
|
||||
worldStateArchive,
|
||||
@@ -166,10 +175,11 @@ public class EthProtocolManagerTestUtil {
|
||||
final GenesisConfigFile config = GenesisConfigFile.mainnet();
|
||||
final GenesisState genesisState = GenesisState.fromConfig(config, protocolSchedule);
|
||||
final Blockchain blockchain = createInMemoryBlockchain(genesisState.getBlock());
|
||||
return create(blockchain, ethScheduler);
|
||||
return create(protocolSchedule, blockchain, ethScheduler);
|
||||
}
|
||||
|
||||
public static EthProtocolManager create(
|
||||
final ProtocolSchedule protocolSchedule,
|
||||
final Blockchain blockchain,
|
||||
final EthScheduler ethScheduler,
|
||||
final WorldStateArchive worldStateArchive,
|
||||
@@ -178,6 +188,7 @@ public class EthProtocolManagerTestUtil {
|
||||
EthPeers peers =
|
||||
new EthPeers(
|
||||
EthProtocol.NAME,
|
||||
() -> protocolSchedule.getByBlockHeader(blockchain.getChainHeadHeader()),
|
||||
TestClock.fixed(),
|
||||
new NoOpMetricsSystem(),
|
||||
25,
|
||||
@@ -196,6 +207,7 @@ public class EthProtocolManagerTestUtil {
|
||||
}
|
||||
|
||||
public static EthProtocolManager create(
|
||||
final ProtocolSchedule protocolSchedule,
|
||||
final Blockchain blockchain,
|
||||
final EthScheduler ethScheduler,
|
||||
final WorldStateArchive worldStateArchive,
|
||||
@@ -205,6 +217,7 @@ public class EthProtocolManagerTestUtil {
|
||||
EthPeers peers =
|
||||
new EthPeers(
|
||||
EthProtocol.NAME,
|
||||
() -> protocolSchedule.getByBlockHeader(blockchain.getChainHeadHeader()),
|
||||
TestClock.fixed(),
|
||||
new NoOpMetricsSystem(),
|
||||
25,
|
||||
@@ -224,10 +237,13 @@ public class EthProtocolManagerTestUtil {
|
||||
}
|
||||
|
||||
public static EthProtocolManager create(
|
||||
final Blockchain blockchain, final EthScheduler ethScheduler) {
|
||||
final ProtocolSchedule protocolSchedule,
|
||||
final Blockchain blockchain,
|
||||
final EthScheduler ethScheduler) {
|
||||
EthPeers peers =
|
||||
new EthPeers(
|
||||
EthProtocol.NAME,
|
||||
() -> protocolSchedule.getByBlockHeader(blockchain.getChainHeadHeader()),
|
||||
TestClock.fixed(),
|
||||
new NoOpMetricsSystem(),
|
||||
25,
|
||||
|
||||
@@ -90,6 +90,7 @@ public abstract class AbstractMessageTaskTest<T, R> {
|
||||
spy(
|
||||
new EthPeers(
|
||||
EthProtocol.NAME,
|
||||
() -> protocolSchedule.getByBlockHeader(blockchain.getChainHeadHeader()),
|
||||
TestClock.fixed(),
|
||||
metricsSystem,
|
||||
MAX_PEERS,
|
||||
|
||||
@@ -69,7 +69,12 @@ public class GetHeadersFromPeerByHashTaskTest extends PeerMessageTaskTest<List<B
|
||||
final List<BlockHeader> requestedData) {
|
||||
final BlockHeader firstHeader = requestedData.get(0);
|
||||
return GetHeadersFromPeerByHashTask.startingAtHash(
|
||||
protocolSchedule, ethContext, firstHeader.getHash(), requestedData.size(), metricsSystem);
|
||||
protocolSchedule,
|
||||
ethContext,
|
||||
firstHeader.getHash(),
|
||||
firstHeader.getNumber(),
|
||||
requestedData.size(),
|
||||
metricsSystem);
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -113,6 +118,7 @@ public class GetHeadersFromPeerByHashTaskTest extends PeerMessageTaskTest<List<B
|
||||
protocolSchedule,
|
||||
ethContext,
|
||||
blockchain.getBlockHashByNumber(startNumber).get(),
|
||||
startNumber,
|
||||
count,
|
||||
skip,
|
||||
reverse,
|
||||
@@ -135,6 +141,8 @@ public class GetHeadersFromPeerByHashTaskTest extends PeerMessageTaskTest<List<B
|
||||
|
||||
@Test
|
||||
public void checkThatSequentialHeadersFormingAChainWorks() {
|
||||
final int startNumber = 1;
|
||||
|
||||
final BlockHeader block1 =
|
||||
new BlockHeaderTestFixture().number(1).parentHash(generateTestBlockHash(0)).buildHeader();
|
||||
final BlockHeader block2 =
|
||||
@@ -145,7 +153,14 @@ public class GetHeadersFromPeerByHashTaskTest extends PeerMessageTaskTest<List<B
|
||||
|
||||
final AbstractGetHeadersFromPeerTask task =
|
||||
new GetHeadersFromPeerByHashTask(
|
||||
protocolSchedule, ethContext, block1.getHash(), 2, 0, false, metricsSystem);
|
||||
protocolSchedule,
|
||||
ethContext,
|
||||
block1.getHash(),
|
||||
startNumber,
|
||||
2,
|
||||
0,
|
||||
false,
|
||||
metricsSystem);
|
||||
final Optional<List<BlockHeader>> optionalBlockHeaders =
|
||||
task.processResponse(false, BlockHeadersMessage.create(headers), peer);
|
||||
assertThat(optionalBlockHeaders).isNotNull();
|
||||
@@ -158,6 +173,7 @@ public class GetHeadersFromPeerByHashTaskTest extends PeerMessageTaskTest<List<B
|
||||
|
||||
@Test
|
||||
public void checkThatSequentialHeadersNotFormingAChainFails() {
|
||||
final int startNumber = 1;
|
||||
final BlockHeader block1 =
|
||||
new BlockHeaderTestFixture().number(1).parentHash(generateTestBlockHash(0)).buildHeader();
|
||||
final BlockHeader block2 =
|
||||
@@ -168,7 +184,14 @@ public class GetHeadersFromPeerByHashTaskTest extends PeerMessageTaskTest<List<B
|
||||
|
||||
final AbstractGetHeadersFromPeerTask task =
|
||||
new GetHeadersFromPeerByHashTask(
|
||||
protocolSchedule, ethContext, block1.getHash(), 2, 0, false, metricsSystem);
|
||||
protocolSchedule,
|
||||
ethContext,
|
||||
block1.getHash(),
|
||||
startNumber,
|
||||
2,
|
||||
0,
|
||||
false,
|
||||
metricsSystem);
|
||||
final Optional<List<BlockHeader>> optionalBlockHeaders =
|
||||
task.processResponse(false, BlockHeadersMessage.create(headers), peer);
|
||||
assertThat(optionalBlockHeaders).isNotNull();
|
||||
|
||||
@@ -106,6 +106,7 @@ public abstract class AbstractBlockPropagationManagerTest {
|
||||
tempProtocolContext.getConsensusContext(ConsensusContext.class));
|
||||
ethProtocolManager =
|
||||
EthProtocolManagerTestUtil.create(
|
||||
protocolSchedule,
|
||||
blockchain,
|
||||
blockchainUtil.getWorldArchive(),
|
||||
blockchainUtil.getTransactionPool(),
|
||||
@@ -617,6 +618,7 @@ public abstract class AbstractBlockPropagationManagerTest {
|
||||
new EthContext(
|
||||
new EthPeers(
|
||||
"eth",
|
||||
() -> protocolSchedule.getByBlockHeader(blockchain.getChainHeadHeader()),
|
||||
TestClock.fixed(),
|
||||
metricsSystem,
|
||||
25,
|
||||
@@ -749,6 +751,7 @@ public abstract class AbstractBlockPropagationManagerTest {
|
||||
new EthContext(
|
||||
new EthPeers(
|
||||
"eth",
|
||||
() -> protocolSchedule.getByBlockHeader(blockchain.getChainHeadHeader()),
|
||||
TestClock.fixed(),
|
||||
metricsSystem,
|
||||
25,
|
||||
|
||||
@@ -74,6 +74,7 @@ public class RangeHeadersFetcherTest {
|
||||
public void setUpTest() {
|
||||
ethProtocolManager =
|
||||
EthProtocolManagerTestUtil.create(
|
||||
protocolSchedule,
|
||||
blockchain,
|
||||
() -> false,
|
||||
protocolContext.getWorldStateArchive(),
|
||||
|
||||
@@ -90,7 +90,9 @@ public class CheckPointSyncChainDownloaderTest {
|
||||
protocolContext = localBlockchainSetup.getProtocolContext();
|
||||
ethProtocolManager =
|
||||
EthProtocolManagerTestUtil.create(
|
||||
localBlockchain, new EthScheduler(1, 1, 1, 1, new NoOpMetricsSystem()));
|
||||
protocolSchedule,
|
||||
localBlockchain,
|
||||
new EthScheduler(1, 1, 1, 1, new NoOpMetricsSystem()));
|
||||
ethContext = ethProtocolManager.ethContext();
|
||||
|
||||
final int blockNumber = 10;
|
||||
|
||||
@@ -24,6 +24,7 @@ import org.hyperledger.besu.ethereum.core.Block;
|
||||
import org.hyperledger.besu.ethereum.core.BlockHeader;
|
||||
import org.hyperledger.besu.ethereum.core.BlockWithReceipts;
|
||||
import org.hyperledger.besu.ethereum.core.BlockchainSetupUtil;
|
||||
import org.hyperledger.besu.ethereum.core.ProtocolScheduleFixture;
|
||||
import org.hyperledger.besu.ethereum.core.TransactionReceipt;
|
||||
import org.hyperledger.besu.ethereum.eth.EthProtocolConfiguration;
|
||||
import org.hyperledger.besu.ethereum.eth.manager.EthProtocolManager;
|
||||
@@ -61,6 +62,7 @@ public class DownloadReceiptsStepTest {
|
||||
TransactionPool transactionPool = mock(TransactionPool.class);
|
||||
ethProtocolManager =
|
||||
EthProtocolManagerTestUtil.create(
|
||||
ProtocolScheduleFixture.MAINNET,
|
||||
blockchain,
|
||||
() -> false,
|
||||
protocolContext.getWorldStateArchive(),
|
||||
|
||||
@@ -26,6 +26,7 @@ import org.hyperledger.besu.ethereum.core.BlockHeader;
|
||||
import org.hyperledger.besu.ethereum.core.BlockHeaderTestFixture;
|
||||
import org.hyperledger.besu.ethereum.core.BlockchainSetupUtil;
|
||||
import org.hyperledger.besu.ethereum.core.Difficulty;
|
||||
import org.hyperledger.besu.ethereum.core.ProtocolScheduleFixture;
|
||||
import org.hyperledger.besu.ethereum.eth.EthProtocolConfiguration;
|
||||
import org.hyperledger.besu.ethereum.eth.manager.EthContext;
|
||||
import org.hyperledger.besu.ethereum.eth.manager.EthPeer;
|
||||
@@ -94,6 +95,7 @@ public class FastSyncActionsTest {
|
||||
blockchain = blockchainSetupUtil.getBlockchain();
|
||||
ethProtocolManager =
|
||||
EthProtocolManagerTestUtil.create(
|
||||
ProtocolScheduleFixture.MAINNET,
|
||||
blockchain,
|
||||
() -> timeoutCount.getAndDecrement() > 0,
|
||||
blockchainSetupUtil.getWorldArchive(),
|
||||
|
||||
@@ -89,7 +89,9 @@ public class FastSyncChainDownloaderTest {
|
||||
protocolContext = localBlockchainSetup.getProtocolContext();
|
||||
ethProtocolManager =
|
||||
EthProtocolManagerTestUtil.create(
|
||||
localBlockchain, new EthScheduler(1, 1, 1, 1, new NoOpMetricsSystem()));
|
||||
protocolSchedule,
|
||||
localBlockchain,
|
||||
new EthScheduler(1, 1, 1, 1, new NoOpMetricsSystem()));
|
||||
|
||||
ethContext = ethProtocolManager.ethContext();
|
||||
syncState = new SyncState(protocolContext.getBlockchain(), ethContext.getEthPeers());
|
||||
|
||||
@@ -85,6 +85,7 @@ public class PivotBlockConfirmerTest {
|
||||
protocolContext = blockchainSetupUtil.getProtocolContext();
|
||||
ethProtocolManager =
|
||||
EthProtocolManagerTestUtil.create(
|
||||
protocolSchedule,
|
||||
blockchain,
|
||||
timeout::get,
|
||||
blockchainSetupUtil.getWorldArchive(),
|
||||
|
||||
@@ -88,6 +88,7 @@ public class PivotBlockRetrieverTest {
|
||||
transactionPool = blockchainSetupUtil.getTransactionPool();
|
||||
ethProtocolManager =
|
||||
EthProtocolManagerTestUtil.create(
|
||||
protocolSchedule,
|
||||
blockchain,
|
||||
timeout::get,
|
||||
blockchainSetupUtil.getWorldArchive(),
|
||||
|
||||
@@ -64,6 +64,7 @@ public class FullSyncChainDownloaderForkTest {
|
||||
protocolContext = localBlockchainSetup.getProtocolContext();
|
||||
ethProtocolManager =
|
||||
EthProtocolManagerTestUtil.create(
|
||||
protocolSchedule,
|
||||
localBlockchain,
|
||||
new EthScheduler(1, 1, 1, 1, new NoOpMetricsSystem()),
|
||||
localBlockchainSetup.getWorldArchive(),
|
||||
|
||||
@@ -101,6 +101,7 @@ public class FullSyncChainDownloaderTest {
|
||||
protocolContext = localBlockchainSetup.getProtocolContext();
|
||||
ethProtocolManager =
|
||||
EthProtocolManagerTestUtil.create(
|
||||
protocolSchedule,
|
||||
localBlockchain,
|
||||
new EthScheduler(1, 1, 1, 1, new NoOpMetricsSystem()),
|
||||
localBlockchainSetup.getWorldArchive(),
|
||||
|
||||
@@ -84,6 +84,7 @@ public class FullSyncChainDownloaderTotalTerminalDifficultyTest {
|
||||
protocolContext = localBlockchainSetup.getProtocolContext();
|
||||
ethProtocolManager =
|
||||
EthProtocolManagerTestUtil.create(
|
||||
protocolSchedule,
|
||||
localBlockchain,
|
||||
new EthScheduler(1, 1, 1, 1, new NoOpMetricsSystem()),
|
||||
localBlockchainSetup.getWorldArchive(),
|
||||
|
||||
@@ -76,6 +76,7 @@ public class FullSyncDownloaderTest {
|
||||
protocolContext = localBlockchainSetup.getProtocolContext();
|
||||
ethProtocolManager =
|
||||
EthProtocolManagerTestUtil.create(
|
||||
protocolSchedule,
|
||||
localBlockchain,
|
||||
new EthScheduler(1, 1, 1, 1, new NoOpMetricsSystem()),
|
||||
localBlockchainSetup.getWorldArchive(),
|
||||
|
||||
@@ -83,6 +83,7 @@ public class FullSyncTargetManagerTest {
|
||||
new ProtocolContext(localBlockchain, localWorldState, null);
|
||||
ethProtocolManager =
|
||||
EthProtocolManagerTestUtil.create(
|
||||
protocolSchedule,
|
||||
localBlockchain,
|
||||
new EthScheduler(1, 1, 1, 1, new NoOpMetricsSystem()),
|
||||
localWorldState,
|
||||
|
||||
@@ -144,6 +144,7 @@ public class DetermineCommonAncestorTaskParameterizedTest {
|
||||
final WorldStateArchive worldStateArchive = createInMemoryWorldStateArchive();
|
||||
final EthProtocolManager ethProtocolManager =
|
||||
EthProtocolManagerTestUtil.create(
|
||||
protocolSchedule,
|
||||
localBlockchain,
|
||||
worldStateArchive,
|
||||
mock(TransactionPool.class),
|
||||
|
||||
@@ -80,6 +80,7 @@ public class DetermineCommonAncestorTaskTest {
|
||||
final WorldStateArchive worldStateArchive = createInMemoryWorldStateArchive();
|
||||
ethProtocolManager =
|
||||
EthProtocolManagerTestUtil.create(
|
||||
protocolSchedule,
|
||||
localBlockchain,
|
||||
worldStateArchive,
|
||||
mock(TransactionPool.class),
|
||||
|
||||
@@ -132,6 +132,7 @@ public class TestNode implements Closeable {
|
||||
final EthPeers ethPeers =
|
||||
new EthPeers(
|
||||
EthProtocol.NAME,
|
||||
() -> protocolSchedule.getByBlockHeader(blockchain.getChainHeadHeader()),
|
||||
TestClock.fixed(),
|
||||
metricsSystem,
|
||||
25,
|
||||
|
||||
@@ -40,6 +40,7 @@ import org.hyperledger.besu.ethereum.eth.sync.state.SyncState;
|
||||
import org.hyperledger.besu.ethereum.eth.transactions.sorter.GasPricePendingTransactionsSorter;
|
||||
import org.hyperledger.besu.ethereum.forkid.ForkIdManager;
|
||||
import org.hyperledger.besu.ethereum.mainnet.ProtocolSchedule;
|
||||
import org.hyperledger.besu.ethereum.mainnet.ProtocolSpec;
|
||||
import org.hyperledger.besu.ethereum.worldstate.WorldStateArchive;
|
||||
import org.hyperledger.besu.metrics.noop.NoOpMetricsSystem;
|
||||
import org.hyperledger.besu.testutil.TestClock;
|
||||
@@ -60,6 +61,7 @@ import org.mockito.junit.MockitoJUnitRunner;
|
||||
public class TransactionPoolFactoryTest {
|
||||
@Mock ProtocolSchedule schedule;
|
||||
@Mock ProtocolContext context;
|
||||
@Mock ProtocolSpec protocolSpec;
|
||||
@Mock MutableBlockchain blockchain;
|
||||
@Mock EthContext ethContext;
|
||||
@Mock EthMessages ethMessages;
|
||||
@@ -85,6 +87,7 @@ public class TransactionPoolFactoryTest {
|
||||
ethPeers =
|
||||
new EthPeers(
|
||||
"ETH",
|
||||
() -> protocolSpec,
|
||||
TestClock.fixed(),
|
||||
new NoOpMetricsSystem(),
|
||||
25,
|
||||
|
||||
@@ -82,7 +82,8 @@ public class NoRewardProtocolScheduleWrapper implements ProtocolSchedule {
|
||||
original.getBadBlocksManager(),
|
||||
Optional.empty(),
|
||||
original.getWithdrawalsValidator(),
|
||||
original.getWithdrawalsProcessor());
|
||||
original.getWithdrawalsProcessor(),
|
||||
original.isPoS());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
@@ -67,6 +67,7 @@ import org.hyperledger.besu.util.Subscribers;
|
||||
|
||||
import java.util.Optional;
|
||||
import java.util.concurrent.locks.ReentrantLock;
|
||||
import java.util.function.Supplier;
|
||||
|
||||
import com.fasterxml.jackson.databind.node.ObjectNode;
|
||||
import org.apache.tuweni.bytes.Bytes;
|
||||
@@ -191,9 +192,12 @@ public class RetestethContext {
|
||||
|
||||
// mining support
|
||||
|
||||
final Supplier<ProtocolSpec> currentProtocolSpecSupplier =
|
||||
() -> protocolSchedule.getByBlockHeader(blockchain.getChainHeadHeader());
|
||||
final EthPeers ethPeers =
|
||||
new EthPeers(
|
||||
"reteseth",
|
||||
currentProtocolSpecSupplier,
|
||||
retestethClock,
|
||||
metricsSystem,
|
||||
0,
|
||||
|
||||
Reference in New Issue
Block a user