Qbft block type (#8142)

Signed-off-by: Jason Frame <jason.frame@consensys.net>
This commit is contained in:
Jason Frame
2025-02-04 15:40:40 +10:00
committed by GitHub
parent 5534d650fd
commit 4dbb72330f
94 changed files with 3371 additions and 1047 deletions

View File

@@ -1,45 +0,0 @@
/*
* Copyright ConsenSys AG.
*
* 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.controller;
import org.hyperledger.besu.consensus.common.bft.BftBlockInterface;
import org.hyperledger.besu.consensus.common.bft.BftExtraDataCodec;
import java.util.function.Supplier;
import com.google.common.base.Suppliers;
/** Base class for BFT based Besu Controller Builders. */
public abstract class BftBesuControllerBuilder extends BesuControllerBuilder {
/** Default constructor to be used by subclasses. */
protected BftBesuControllerBuilder() {}
/**
* Bft extra data codec supplier.
*
* @return the supplier of type BftExtraDataCodec.
*/
protected abstract Supplier<BftExtraDataCodec> bftExtraDataCodec();
/**
* Bft block interface supplier.
*
* @return the supplier of type BftBlockInterface.
*/
protected Supplier<BftBlockInterface> bftBlockInterface() {
return Suppliers.memoize(() -> new BftBlockInterface(bftExtraDataCodec().get()));
}
}

View File

@@ -19,10 +19,10 @@ import org.hyperledger.besu.config.BftFork;
import org.hyperledger.besu.consensus.common.BftValidatorOverrides;
import org.hyperledger.besu.consensus.common.EpochManager;
import org.hyperledger.besu.consensus.common.ForksSchedule;
import org.hyperledger.besu.consensus.common.bft.BftBlockInterface;
import org.hyperledger.besu.consensus.common.bft.BftContext;
import org.hyperledger.besu.consensus.common.bft.BftEventQueue;
import org.hyperledger.besu.consensus.common.bft.BftExecutors;
import org.hyperledger.besu.consensus.common.bft.BftExtraDataCodec;
import org.hyperledger.besu.consensus.common.bft.BftProcessor;
import org.hyperledger.besu.consensus.common.bft.BftProtocolSchedule;
import org.hyperledger.besu.consensus.common.bft.BlockTimer;
@@ -79,35 +79,32 @@ import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.function.Supplier;
import java.util.stream.Collectors;
import com.google.common.base.Suppliers;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/** The Ibft besu controller builder. */
public class IbftBesuControllerBuilder extends BftBesuControllerBuilder {
public class IbftBesuControllerBuilder extends BesuControllerBuilder {
private static final Logger LOG = LoggerFactory.getLogger(IbftBesuControllerBuilder.class);
private BftEventQueue bftEventQueue;
private BftConfigOptions bftConfig;
private ForksSchedule<BftConfigOptions> forksSchedule;
private ValidatorPeers peers;
private IbftExtraDataCodec bftExtraDataCodec;
private BftBlockInterface bftBlockInterface;
/** Default Constructor */
public IbftBesuControllerBuilder() {}
@Override
protected Supplier<BftExtraDataCodec> bftExtraDataCodec() {
return Suppliers.memoize(IbftExtraDataCodec::new);
}
@Override
protected void prepForBuild() {
bftConfig = genesisConfigOptions.getBftConfigOptions();
bftEventQueue = new BftEventQueue(bftConfig.getMessageQueueLimit());
forksSchedule = IbftForksSchedulesFactory.create(genesisConfigOptions);
bftExtraDataCodec = new IbftExtraDataCodec();
bftBlockInterface = new BftBlockInterface(bftExtraDataCodec);
}
@Override
@@ -158,14 +155,14 @@ public class IbftBesuControllerBuilder extends BftBesuControllerBuilder {
forksSchedule,
miningConfiguration,
localAddress,
bftExtraDataCodec().get(),
bftExtraDataCodec,
ethProtocolManager.ethContext().getScheduler());
final ValidatorProvider validatorProvider =
protocolContext.getConsensusContext(BftContext.class).getValidatorProvider();
final ProposerSelector proposerSelector =
new ProposerSelector(blockchain, bftBlockInterface().get(), true, validatorProvider);
new ProposerSelector(blockchain, bftBlockInterface, true, validatorProvider);
// NOTE: peers should not be used for accessing the network as it does not enforce the
// "only send once" filter applied by the UniqueMessageMulticaster.
@@ -193,7 +190,7 @@ public class IbftBesuControllerBuilder extends BftBesuControllerBuilder {
final MessageValidatorFactory messageValidatorFactory =
new MessageValidatorFactory(
proposerSelector, bftProtocolSchedule, protocolContext, bftExtraDataCodec().get());
proposerSelector, bftProtocolSchedule, protocolContext, bftExtraDataCodec);
final Subscribers<MinedBlockObserver> minedBlockObservers = Subscribers.create();
minedBlockObservers.subscribe(ethProtocolManager);
@@ -222,7 +219,7 @@ public class IbftBesuControllerBuilder extends BftBesuControllerBuilder {
minedBlockObservers,
messageValidatorFactory,
messageFactory,
bftExtraDataCodec().get()),
bftExtraDataCodec),
messageValidatorFactory,
messageFactory),
gossiper,
@@ -291,7 +288,7 @@ public class IbftBesuControllerBuilder extends BftBesuControllerBuilder {
final ValidatorProvider validatorProvider =
protocolContext.getConsensusContext(BftContext.class).getValidatorProvider();
return new IbftQueryPluginServiceFactory(
blockchain, bftBlockInterface().get(), validatorProvider, nodeKey);
blockchain, bftBlockInterface, validatorProvider, nodeKey);
}
@Override
@@ -301,7 +298,7 @@ public class IbftBesuControllerBuilder extends BftBesuControllerBuilder {
forksSchedule,
privacyParameters,
isRevertReasonEnabled,
bftExtraDataCodec().get(),
bftExtraDataCodec,
evmConfiguration,
miningConfiguration,
badBlockManager,
@@ -313,7 +310,7 @@ public class IbftBesuControllerBuilder extends BftBesuControllerBuilder {
protected void validateContext(final ProtocolContext context) {
final BlockHeader genesisBlockHeader = context.getBlockchain().getGenesisBlock().getHeader();
if (bftBlockInterface().get().validatorsInBlock(genesisBlockHeader).isEmpty()) {
if (bftBlockInterface.validatorsInBlock(genesisBlockHeader).isEmpty()) {
LOG.warn("Genesis block contains no signers - chain will not progress.");
}
}
@@ -331,9 +328,9 @@ public class IbftBesuControllerBuilder extends BftBesuControllerBuilder {
return new BftContext(
BlockValidatorProvider.forkingValidatorProvider(
blockchain, epochManager, bftBlockInterface().get(), validatorOverrides),
blockchain, epochManager, bftBlockInterface, validatorOverrides),
epochManager,
bftBlockInterface().get());
bftBlockInterface);
}
private BftValidatorOverrides convertIbftForks(final List<BftFork> bftForks) {

View File

@@ -23,10 +23,10 @@ import org.hyperledger.besu.config.QbftFork;
import org.hyperledger.besu.consensus.common.BftValidatorOverrides;
import org.hyperledger.besu.consensus.common.EpochManager;
import org.hyperledger.besu.consensus.common.ForksSchedule;
import org.hyperledger.besu.consensus.common.bft.BftBlockInterface;
import org.hyperledger.besu.consensus.common.bft.BftContext;
import org.hyperledger.besu.consensus.common.bft.BftEventQueue;
import org.hyperledger.besu.consensus.common.bft.BftExecutors;
import org.hyperledger.besu.consensus.common.bft.BftExtraDataCodec;
import org.hyperledger.besu.consensus.common.bft.BftProcessor;
import org.hyperledger.besu.consensus.common.bft.BftProtocolSchedule;
import org.hyperledger.besu.consensus.common.bft.BlockTimer;
@@ -35,25 +35,36 @@ import org.hyperledger.besu.consensus.common.bft.EventMultiplexer;
import org.hyperledger.besu.consensus.common.bft.MessageTracker;
import org.hyperledger.besu.consensus.common.bft.RoundTimer;
import org.hyperledger.besu.consensus.common.bft.UniqueMessageMulticaster;
import org.hyperledger.besu.consensus.common.bft.blockcreation.BftBlockCreatorFactory;
import org.hyperledger.besu.consensus.common.bft.blockcreation.BftMiningCoordinator;
import org.hyperledger.besu.consensus.common.bft.blockcreation.ProposerSelector;
import org.hyperledger.besu.consensus.common.bft.network.ValidatorPeers;
import org.hyperledger.besu.consensus.common.bft.protocol.BftProtocolManager;
import org.hyperledger.besu.consensus.common.bft.statemachine.BftEventHandler;
import org.hyperledger.besu.consensus.common.bft.statemachine.BftFinalState;
import org.hyperledger.besu.consensus.common.bft.statemachine.FutureMessageBuffer;
import org.hyperledger.besu.consensus.common.validator.ValidatorProvider;
import org.hyperledger.besu.consensus.common.validator.blockbased.BlockValidatorProvider;
import org.hyperledger.besu.consensus.qbft.QbftExtraDataCodec;
import org.hyperledger.besu.consensus.qbft.QbftForksSchedulesFactory;
import org.hyperledger.besu.consensus.qbft.QbftProtocolScheduleBuilder;
import org.hyperledger.besu.consensus.qbft.adaptor.BlockUtil;
import org.hyperledger.besu.consensus.qbft.adaptor.QbftBlockCodecAdaptor;
import org.hyperledger.besu.consensus.qbft.adaptor.QbftBlockCreatorFactoryAdaptor;
import org.hyperledger.besu.consensus.qbft.adaptor.QbftBlockInterfaceAdaptor;
import org.hyperledger.besu.consensus.qbft.adaptor.QbftExtraDataProviderAdaptor;
import org.hyperledger.besu.consensus.qbft.adaptor.QbftFinalStateImpl;
import org.hyperledger.besu.consensus.qbft.adaptor.QbftProtocolScheduleAdaptor;
import org.hyperledger.besu.consensus.qbft.blockcreation.QbftBlockCreatorFactory;
import org.hyperledger.besu.consensus.qbft.core.network.QbftGossip;
import org.hyperledger.besu.consensus.qbft.core.payload.MessageFactory;
import org.hyperledger.besu.consensus.qbft.core.statemachine.QbftBlockHeightManagerFactory;
import org.hyperledger.besu.consensus.qbft.core.statemachine.QbftController;
import org.hyperledger.besu.consensus.qbft.core.statemachine.QbftRoundFactory;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockCodec;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockInterface;
import org.hyperledger.besu.consensus.qbft.core.types.QbftContext;
import org.hyperledger.besu.consensus.qbft.core.types.QbftFinalState;
import org.hyperledger.besu.consensus.qbft.core.types.QbftMinedBlockObserver;
import org.hyperledger.besu.consensus.qbft.core.types.QbftProtocolSchedule;
import org.hyperledger.besu.consensus.qbft.core.validation.MessageValidatorFactory;
import org.hyperledger.besu.consensus.qbft.core.validator.ValidatorModeTransitionLogger;
import org.hyperledger.besu.consensus.qbft.jsonrpc.QbftJsonRpcMethods;
@@ -88,15 +99,13 @@ import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.function.Supplier;
import java.util.stream.Collectors;
import com.google.common.base.Suppliers;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/** The Qbft Besu controller builder. */
public class QbftBesuControllerBuilder extends BftBesuControllerBuilder {
public class QbftBesuControllerBuilder extends BesuControllerBuilder {
private static final Logger LOG = LoggerFactory.getLogger(QbftBesuControllerBuilder.class);
private BftEventQueue bftEventQueue;
@@ -105,21 +114,20 @@ public class QbftBesuControllerBuilder extends BftBesuControllerBuilder {
private ValidatorPeers peers;
private TransactionValidatorProvider transactionValidatorProvider;
private BftConfigOptions bftConfigOptions;
private QbftExtraDataCodec qbftExtraDataCodec;
private BftBlockInterface bftBlockInterface;
/** Default Constructor. */
public QbftBesuControllerBuilder() {}
@Override
protected Supplier<BftExtraDataCodec> bftExtraDataCodec() {
return Suppliers.memoize(QbftExtraDataCodec::new);
}
@Override
protected void prepForBuild() {
qbftConfig = genesisConfigOptions.getQbftConfigOptions();
bftEventQueue = new BftEventQueue(qbftConfig.getMessageQueueLimit());
qbftForksSchedule = QbftForksSchedulesFactory.create(genesisConfigOptions);
bftConfigOptions = qbftConfig;
qbftExtraDataCodec = new QbftExtraDataCodec();
bftBlockInterface = new BftBlockInterface(qbftExtraDataCodec);
}
@Override
@@ -143,7 +151,7 @@ public class QbftBesuControllerBuilder extends BftBesuControllerBuilder {
// Must create our own voteTallyCache as using this would pollute the main voteTallyCache
final BlockValidatorProvider readOnlyBlockValidatorProvider =
BlockValidatorProvider.nonForkingValidatorProvider(
blockchain, epochManager, bftBlockInterface().get());
blockchain, epochManager, bftBlockInterface);
return new ForkingValidatorProvider(
blockchain,
qbftForksSchedule,
@@ -166,9 +174,8 @@ public class QbftBesuControllerBuilder extends BftBesuControllerBuilder {
Istanbul100SubProtocol.ISTANBUL_100,
Istanbul100SubProtocol.get().getName()));
maybeSnapProtocolManager.ifPresent(
snapProtocolManager -> {
subProtocolConfiguration.withSubProtocol(SnapProtocol.get(), snapProtocolManager);
});
snapProtocolManager ->
subProtocolConfiguration.withSubProtocol(SnapProtocol.get(), snapProtocolManager));
return subProtocolConfiguration;
}
@@ -183,10 +190,13 @@ public class QbftBesuControllerBuilder extends BftBesuControllerBuilder {
final MutableBlockchain blockchain = protocolContext.getBlockchain();
final BftExecutors bftExecutors =
BftExecutors.create(metricsSystem, BftExecutors.ConsensusType.QBFT);
final QbftBlockCodec blockEncoder = new QbftBlockCodecAdaptor(qbftExtraDataCodec);
final Address localAddress = Util.publicKeyToAddress(nodeKey.getPublicKey());
final BftProtocolSchedule bftProtocolSchedule = (BftProtocolSchedule) protocolSchedule;
final BftBlockCreatorFactory<?> blockCreatorFactory =
QbftProtocolSchedule qbftProtocolSchedule =
new QbftProtocolScheduleAdaptor(bftProtocolSchedule, protocolContext);
final QbftBlockCreatorFactory blockCreatorFactory =
new QbftBlockCreatorFactory(
transactionPool,
protocolContext,
@@ -194,14 +204,23 @@ public class QbftBesuControllerBuilder extends BftBesuControllerBuilder {
qbftForksSchedule,
miningConfiguration,
localAddress,
bftExtraDataCodec().get(),
qbftExtraDataCodec,
ethProtocolManager.ethContext().getScheduler());
final ValidatorProvider validatorProvider =
protocolContext.getConsensusContext(BftContext.class).getValidatorProvider();
final QbftBlockInterface qbftBlockInterface = new QbftBlockInterfaceAdaptor(bftBlockInterface);
final QbftContext qbftContext = new QbftContext(validatorProvider, qbftBlockInterface);
final ProtocolContext qbftProtocolContext =
new ProtocolContext(
blockchain,
protocolContext.getWorldStateArchive(),
qbftContext,
protocolContext.getBadBlockManager());
final ProposerSelector proposerSelector =
new ProposerSelector(blockchain, bftBlockInterface().get(), true, validatorProvider);
new ProposerSelector(blockchain, bftBlockInterface, true, validatorProvider);
// NOTE: peers should not be used for accessing the network as it does not enforce the
// "only send once" filter applied by the UniqueMessageMulticaster.
@@ -210,10 +229,10 @@ public class QbftBesuControllerBuilder extends BftBesuControllerBuilder {
final UniqueMessageMulticaster uniqueMessageMulticaster =
new UniqueMessageMulticaster(peers, qbftConfig.getGossipedHistoryLimit());
final QbftGossip gossiper = new QbftGossip(uniqueMessageMulticaster, bftExtraDataCodec().get());
final QbftGossip gossiper = new QbftGossip(uniqueMessageMulticaster, blockEncoder);
final BftFinalState finalState =
new BftFinalState(
final QbftFinalState finalState =
new QbftFinalStateImpl(
validatorProvider,
nodeKey,
Util.publicKeyToAddress(nodeKey.getPublicKey()),
@@ -224,16 +243,19 @@ public class QbftBesuControllerBuilder extends BftBesuControllerBuilder {
Duration.ofSeconds(qbftConfig.getRequestTimeoutSeconds()),
bftExecutors),
new BlockTimer(bftEventQueue, qbftForksSchedule, bftExecutors, clock),
blockCreatorFactory,
new QbftBlockCreatorFactoryAdaptor(blockCreatorFactory, qbftExtraDataCodec),
clock);
final MessageValidatorFactory messageValidatorFactory =
new MessageValidatorFactory(
proposerSelector, bftProtocolSchedule, protocolContext, bftExtraDataCodec().get());
new MessageValidatorFactory(proposerSelector, qbftProtocolSchedule, qbftProtocolContext);
final Subscribers<MinedBlockObserver> minedBlockObservers = Subscribers.create();
minedBlockObservers.subscribe(ethProtocolManager);
minedBlockObservers.subscribe(blockLogger(transactionPool, localAddress));
final Subscribers<QbftMinedBlockObserver> minedBlockObservers = Subscribers.create();
minedBlockObservers.subscribe(
qbftBlock -> ethProtocolManager.blockMined(BlockUtil.toBesuBlock(qbftBlock)));
minedBlockObservers.subscribe(
qbftBlock ->
blockLogger(transactionPool, localAddress)
.blockMined(BlockUtil.toBesuBlock(qbftBlock)));
final FutureMessageBuffer futureMessageBuffer =
new FutureMessageBuffer(
@@ -243,19 +265,20 @@ public class QbftBesuControllerBuilder extends BftBesuControllerBuilder {
final MessageTracker duplicateMessageTracker =
new MessageTracker(qbftConfig.getDuplicateMessageLimit());
final MessageFactory messageFactory = new MessageFactory(nodeKey);
final MessageFactory messageFactory = new MessageFactory(nodeKey, blockEncoder);
QbftBlockHeightManagerFactory qbftBlockHeightManagerFactory =
new QbftBlockHeightManagerFactory(
finalState,
new QbftRoundFactory(
finalState,
protocolContext,
bftProtocolSchedule,
qbftProtocolContext,
qbftProtocolSchedule,
minedBlockObservers,
messageValidatorFactory,
messageFactory,
bftExtraDataCodec().get()),
qbftExtraDataCodec,
new QbftExtraDataProviderAdaptor(qbftExtraDataCodec)),
messageValidatorFactory,
messageFactory,
new ValidatorModeTransitionLogger(qbftForksSchedule));
@@ -271,7 +294,7 @@ public class QbftBesuControllerBuilder extends BftBesuControllerBuilder {
duplicateMessageTracker,
futureMessageBuffer,
new EthSynchronizerUpdater(ethProtocolManager.ethContext().getEthPeers()),
bftExtraDataCodec().get());
blockEncoder);
final EventMultiplexer eventMultiplexer = new EventMultiplexer(qbftController);
final BftProcessor bftProcessor = new BftProcessor(bftEventQueue, eventMultiplexer);
@@ -340,7 +363,7 @@ public class QbftBesuControllerBuilder extends BftBesuControllerBuilder {
final ValidatorProvider validatorProvider =
protocolContext.getConsensusContext(BftContext.class).getValidatorProvider();
return new BftQueryPluginServiceFactory(
blockchain, bftExtraDataCodec().get(), validatorProvider, nodeKey, "qbft");
blockchain, qbftExtraDataCodec, validatorProvider, nodeKey, "qbft");
}
@Override
@@ -350,7 +373,7 @@ public class QbftBesuControllerBuilder extends BftBesuControllerBuilder {
qbftForksSchedule,
privacyParameters,
isRevertReasonEnabled,
bftExtraDataCodec().get(),
qbftExtraDataCodec,
evmConfiguration,
miningConfiguration,
badBlockManager,
@@ -387,7 +410,7 @@ public class QbftBesuControllerBuilder extends BftBesuControllerBuilder {
}
private boolean signersExistIn(final BlockHeader genesisBlockHeader) {
return bftBlockInterface().get().validatorsInBlock(genesisBlockHeader).size() > 0;
return !bftBlockInterface.validatorsInBlock(genesisBlockHeader).isEmpty();
}
@Override
@@ -401,7 +424,7 @@ public class QbftBesuControllerBuilder extends BftBesuControllerBuilder {
convertBftForks(genesisConfigOptions.getTransitions().getQbftForks());
final BlockValidatorProvider blockValidatorProvider =
BlockValidatorProvider.forkingValidatorProvider(
blockchain, epochManager, bftBlockInterface().get(), validatorOverrides);
blockchain, epochManager, bftBlockInterface, validatorOverrides);
transactionValidatorProvider =
new TransactionValidatorProvider(
@@ -411,7 +434,7 @@ public class QbftBesuControllerBuilder extends BftBesuControllerBuilder {
new ForkingValidatorProvider(
blockchain, qbftForksSchedule, blockValidatorProvider, transactionValidatorProvider);
return new BftContext(validatorProvider, epochManager, bftBlockInterface().get());
return new BftContext(validatorProvider, epochManager, bftBlockInterface);
}
private BftValidatorOverrides convertBftForks(final List<QbftFork> bftForks) {
@@ -438,7 +461,7 @@ public class QbftBesuControllerBuilder extends BftBesuControllerBuilder {
String.format(
"%s %s #%,d / %d tx / %d pending / %,d (%01.1f%%) gas / (%s)",
block.getHeader().getCoinbase().equals(localAddress) ? "Produced" : "Imported",
block.getBody().getTransactions().size() == 0 ? "empty block" : "block",
block.getBody().getTransactions().isEmpty() ? "empty block" : "block",
block.getHeader().getNumber(),
block.getBody().getTransactions().size(),
transactionPool.count(),

View File

@@ -115,7 +115,7 @@ public class BftFinalState {
* Is local node proposer for round.
*
* @param roundIdentifier the round identifier
* @return the boolean
* @return true if the local node is the proposer for the given round, false otherwise
*/
public boolean isLocalNodeProposerForRound(final ConsensusRoundIdentifier roundIdentifier) {
return getProposerForRound(roundIdentifier).equals(localAddress);
@@ -124,11 +124,10 @@ public class BftFinalState {
/**
* Is local node validator.
*
* @return the boolean
* @return true if the local node is a validator, false otherwise
*/
public boolean isLocalNodeValidator() {
final boolean isValidator = getValidators().contains(localAddress);
return isValidator;
return getValidators().contains(localAddress);
}
/**

View File

@@ -15,38 +15,47 @@
package org.hyperledger.besu.consensus.qbft.core.support;
import org.hyperledger.besu.consensus.common.bft.BftBlockHashing;
import org.hyperledger.besu.consensus.common.bft.BftBlockHeaderFunctions;
import org.hyperledger.besu.consensus.common.bft.BftBlockInterface;
import org.hyperledger.besu.consensus.common.bft.ConsensusRoundIdentifier;
import org.hyperledger.besu.consensus.common.bft.payload.SignedData;
import org.hyperledger.besu.consensus.qbft.QbftExtraDataCodec;
import org.hyperledger.besu.consensus.qbft.adaptor.QbftBlockInterfaceAdaptor;
import org.hyperledger.besu.consensus.qbft.core.payload.CommitPayload;
import org.hyperledger.besu.consensus.qbft.core.payload.MessageFactory;
import org.hyperledger.besu.consensus.qbft.core.statemachine.PreparedCertificate;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlock;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockCodec;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockInterface;
import org.hyperledger.besu.consensus.qbft.core.types.QbftHashMode;
import org.hyperledger.besu.crypto.SECPSignature;
import org.hyperledger.besu.cryptoservices.NodeKey;
import org.hyperledger.besu.ethereum.core.Block;
public class IntegrationTestHelpers {
public static SignedData<CommitPayload> createSignedCommitPayload(
final ConsensusRoundIdentifier roundId, final Block block, final NodeKey nodeKey) {
final ConsensusRoundIdentifier roundId,
final QbftBlock block,
final NodeKey nodeKey,
final QbftBlockCodec blockEncoder) {
final QbftExtraDataCodec qbftExtraDataEncoder = new QbftExtraDataCodec();
final Block commitBlock = createCommitBlockFromProposalBlock(block, roundId.getRoundNumber());
final QbftBlock commitBlock =
createCommitBlockFromProposalBlock(block, roundId.getRoundNumber());
final SECPSignature commitSeal =
nodeKey.sign(
new BftBlockHashing(qbftExtraDataEncoder)
.calculateDataHashForCommittedSeal(commitBlock.getHeader()));
final MessageFactory messageFactory = new MessageFactory(nodeKey);
final MessageFactory messageFactory = new MessageFactory(nodeKey, blockEncoder);
return messageFactory.createCommit(roundId, block.getHash(), commitSeal).getSignedPayload();
}
public static PreparedCertificate createValidPreparedCertificate(
final TestContext context, final ConsensusRoundIdentifier preparedRound, final Block block) {
final TestContext context,
final ConsensusRoundIdentifier preparedRound,
final QbftBlock block) {
final RoundSpecificPeers peers = context.roundSpecificPeers(preparedRound);
return new PreparedCertificate(
@@ -55,11 +64,12 @@ public class IntegrationTestHelpers {
preparedRound.getRoundNumber());
}
public static Block createCommitBlockFromProposalBlock(
final Block proposalBlock, final int round) {
public static QbftBlock createCommitBlockFromProposalBlock(
final QbftBlock proposalBlock, final int round) {
final QbftExtraDataCodec bftExtraDataCodec = new QbftExtraDataCodec();
final BftBlockInterface bftBlockInterface = new BftBlockInterface(bftExtraDataCodec);
return bftBlockInterface.replaceRoundInBlock(
proposalBlock, round, BftBlockHeaderFunctions.forCommittedSeal(bftExtraDataCodec));
final QbftBlockInterface qbftBlockInterface = new QbftBlockInterfaceAdaptor(bftBlockInterface);
return qbftBlockInterface.replaceRoundInBlock(
proposalBlock, round, QbftHashMode.COMMITTED_SEAL);
}
}

View File

@@ -18,7 +18,6 @@ import static java.util.Optional.empty;
import static org.assertj.core.api.Assertions.assertThat;
import static org.assertj.core.api.Fail.fail;
import org.hyperledger.besu.consensus.common.bft.BftExtraDataCodec;
import org.hyperledger.besu.consensus.common.bft.ConsensusRoundIdentifier;
import org.hyperledger.besu.consensus.common.bft.messagewrappers.BftMessage;
import org.hyperledger.besu.consensus.common.bft.payload.Payload;
@@ -32,9 +31,10 @@ import org.hyperledger.besu.consensus.qbft.core.messagewrappers.RoundChange;
import org.hyperledger.besu.consensus.qbft.core.payload.PreparePayload;
import org.hyperledger.besu.consensus.qbft.core.payload.RoundChangePayload;
import org.hyperledger.besu.consensus.qbft.core.statemachine.PreparedCertificate;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlock;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockCodec;
import org.hyperledger.besu.crypto.SECPSignature;
import org.hyperledger.besu.datatypes.Hash;
import org.hyperledger.besu.ethereum.core.Block;
import org.hyperledger.besu.ethereum.p2p.rlpx.wire.MessageData;
import java.util.Arrays;
@@ -52,17 +52,17 @@ public class RoundSpecificPeers {
private final ValidatorPeer proposer;
private final Collection<ValidatorPeer> peers;
private final List<ValidatorPeer> nonProposingPeers;
private final BftExtraDataCodec bftExtraDataCodec;
private final QbftBlockCodec qbftBlockCodec;
public RoundSpecificPeers(
final ValidatorPeer proposer,
final Collection<ValidatorPeer> peers,
final List<ValidatorPeer> nonProposingPeers,
final BftExtraDataCodec bftExtraDataCodec) {
final QbftBlockCodec qbftBlockCodec) {
this.proposer = proposer;
this.peers = peers;
this.nonProposingPeers = nonProposingPeers;
this.bftExtraDataCodec = bftExtraDataCodec;
this.qbftBlockCodec = qbftBlockCodec;
}
public ValidatorPeer getProposer() {
@@ -70,7 +70,7 @@ public class RoundSpecificPeers {
}
public ValidatorPeer getFirstNonProposer() {
return nonProposingPeers.get(0);
return nonProposingPeers.getFirst();
}
public void clearReceivedMessages() {
@@ -92,7 +92,7 @@ public class RoundSpecificPeers {
.collect(Collectors.toList());
}
public void commit(final ConsensusRoundIdentifier roundId, final Block block) {
public void commit(final ConsensusRoundIdentifier roundId, final QbftBlock block) {
peers.forEach(peer -> peer.injectCommit(roundId, block));
}
@@ -128,7 +128,7 @@ public class RoundSpecificPeers {
nonProposingPeers.forEach(peer -> peer.injectPrepare(roundId, hash));
}
public void commitForNonProposing(final ConsensusRoundIdentifier roundId, final Block block) {
public void commitForNonProposing(final ConsensusRoundIdentifier roundId, final QbftBlock block) {
nonProposingPeers.forEach(peer -> peer.injectCommit(roundId, block));
}
@@ -200,7 +200,7 @@ public class RoundSpecificPeers {
switch (expectedMessage.getMessageType()) {
case QbftV1.PROPOSAL:
actualSignedPayload = ProposalMessageData.fromMessageData(actual).decode(bftExtraDataCodec);
actualSignedPayload = ProposalMessageData.fromMessageData(actual).decode(qbftBlockCodec);
break;
case QbftV1.PREPARE:
actualSignedPayload = PrepareMessageData.fromMessageData(actual).decode();
@@ -209,8 +209,7 @@ public class RoundSpecificPeers {
actualSignedPayload = CommitMessageData.fromMessageData(actual).decode();
break;
case QbftV1.ROUND_CHANGE:
actualSignedPayload =
RoundChangeMessageData.fromMessageData(actual).decode(bftExtraDataCodec);
actualSignedPayload = RoundChangeMessageData.fromMessageData(actual).decode(qbftBlockCodec);
break;
default:
fail("Illegal QBFTV1 message type.");

View File

@@ -17,20 +17,30 @@ package org.hyperledger.besu.consensus.qbft.core.support;
import org.hyperledger.besu.consensus.common.bft.BftBlockHeaderFunctions;
import org.hyperledger.besu.consensus.common.bft.BftExecutors;
import org.hyperledger.besu.consensus.common.bft.BftExtraDataCodec;
import org.hyperledger.besu.consensus.common.bft.BftHelpers;
import org.hyperledger.besu.consensus.common.bft.ConsensusRoundIdentifier;
import org.hyperledger.besu.consensus.common.bft.EventMultiplexer;
import org.hyperledger.besu.consensus.common.bft.blockcreation.ProposerSelector;
import org.hyperledger.besu.consensus.common.bft.inttest.NodeParams;
import org.hyperledger.besu.consensus.common.bft.statemachine.BftEventHandler;
import org.hyperledger.besu.consensus.common.bft.statemachine.BftFinalState;
import org.hyperledger.besu.consensus.common.validator.ValidatorProvider;
import org.hyperledger.besu.consensus.qbft.adaptor.BlockUtil;
import org.hyperledger.besu.consensus.qbft.adaptor.QbftBlockAdaptor;
import org.hyperledger.besu.consensus.qbft.core.payload.MessageFactory;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlock;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockCodec;
import org.hyperledger.besu.consensus.qbft.core.types.QbftFinalState;
import org.hyperledger.besu.crypto.SECPSignature;
import org.hyperledger.besu.datatypes.Address;
import org.hyperledger.besu.ethereum.chain.MutableBlockchain;
import org.hyperledger.besu.ethereum.core.Block;
import org.hyperledger.besu.ethereum.core.BlockBody;
import org.hyperledger.besu.ethereum.core.BlockHeader;
import org.hyperledger.besu.ethereum.core.BlockHeaderBuilder;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.Map;
@@ -48,22 +58,26 @@ public class TestContext {
private final MutableBlockchain blockchain;
private final BftExecutors bftExecutors;
private final BftEventHandler controller;
private final BftFinalState finalState;
private final QbftFinalState finalState;
private final EventMultiplexer eventMultiplexer;
private final MessageFactory messageFactory;
private final ValidatorProvider validatorProvider;
private final ProposerSelector proposerSelector;
private final BftExtraDataCodec bftExtraDataCodec;
private final QbftBlockCodec blockEncoder;
public TestContext(
final Map<Address, ValidatorPeer> remotePeers,
final MutableBlockchain blockchain,
final BftExecutors bftExecutors,
final BftEventHandler controller,
final BftFinalState finalState,
final QbftFinalState finalState,
final EventMultiplexer eventMultiplexer,
final MessageFactory messageFactory,
final ValidatorProvider validatorProvider,
final BftExtraDataCodec bftExtraDataCodec) {
final ProposerSelector proposerSelector,
final BftExtraDataCodec bftExtraDataCodec,
final QbftBlockCodec blockEncoder) {
this.remotePeers = remotePeers;
this.blockchain = blockchain;
this.bftExecutors = bftExecutors;
@@ -72,7 +86,9 @@ public class TestContext {
this.eventMultiplexer = eventMultiplexer;
this.messageFactory = messageFactory;
this.validatorProvider = validatorProvider;
this.proposerSelector = proposerSelector;
this.bftExtraDataCodec = bftExtraDataCodec;
this.blockEncoder = blockEncoder;
}
public void start() {
@@ -96,62 +112,76 @@ public class TestContext {
return messageFactory;
}
public Block createBlockForProposalFromChainHead(final long timestamp) {
public QbftBlockCodec getBlockEncoder() {
return blockEncoder;
}
public QbftBlock createBlockForProposalFromChainHead(final long timestamp) {
return createBlockForProposalFromChainHead(timestamp, finalState.getLocalAddress(), 0);
}
public Block createBlockForProposalFromChainHead(final long timestamp, final int roundNumber) {
public QbftBlock createBlockForProposalFromChainHead(
final long timestamp, final int roundNumber) {
return createBlockForProposalFromChainHead(
timestamp, finalState.getLocalAddress(), roundNumber);
}
public Block createBlockForProposalFromChainHead(final long timestamp, final Address proposer) {
public QbftBlock createBlockForProposalFromChainHead(
final long timestamp, final Address proposer) {
// this implies that EVERY block will have this node as the proposer :/
return createBlockForProposal(blockchain.getChainHeadHeader(), timestamp, proposer, 0);
}
public Block createBlockForProposalFromChainHead(
public QbftBlock createBlockForProposalFromChainHead(
final long timestamp, final Address proposer, final int roundNumber) {
// this implies that EVERY block will have this node as the proposer :/
return createBlockForProposal(
blockchain.getChainHeadHeader(), timestamp, proposer, roundNumber);
}
public Block createBlockForProposal(
public QbftBlock createBlockForProposal(
final BlockHeader parent,
final long timestamp,
final Address proposer,
final int roundNumber) {
final Block block =
finalState
.getBlockCreatorFactory()
.create(roundNumber)
.createBlock(timestamp, parent)
.getBlock();
final QbftBlock block =
finalState.getBlockCreatorFactory().create(roundNumber).createBlock(timestamp, parent);
final BlockHeaderBuilder headerBuilder = BlockHeaderBuilder.fromHeader(block.getHeader());
headerBuilder
.coinbase(proposer)
.blockHeaderFunctions(BftBlockHeaderFunctions.forCommittedSeal(bftExtraDataCodec));
final BlockHeader newHeader = headerBuilder.buildBlockHeader();
return new Block(newHeader, block.getBody());
final Block newBlock =
new Block(newHeader, new BlockBody(Collections.emptyList(), Collections.emptyList()));
return new QbftBlockAdaptor(newBlock);
}
public Block createBlockForProposal(
public QbftBlock createBlockForProposal(
final BlockHeader parent, final long timestamp, final Address proposer) {
return createBlockForProposal(parent, timestamp, proposer, 0);
}
public QbftBlock createSealedBlock(
final BftExtraDataCodec bftExtraDataCodec,
final QbftBlock block,
final int roundNumber,
final Collection<SECPSignature> commitSeals) {
final Block sealedBlock =
BftHelpers.createSealedBlock(
bftExtraDataCodec, BlockUtil.toBesuBlock(block), roundNumber, commitSeals);
return new QbftBlockAdaptor(sealedBlock);
}
public RoundSpecificPeers roundSpecificPeers(final ConsensusRoundIdentifier roundId) {
// This will return NULL if the LOCAL node is the proposer for the specified round
final Address proposerAddress = finalState.getProposerForRound(roundId);
final Address proposerAddress = proposerSelector.selectProposerForRound(roundId);
final ValidatorPeer proposer = remotePeers.getOrDefault(proposerAddress, null);
final List<ValidatorPeer> nonProposers = new ArrayList<>(remotePeers.values());
nonProposers.remove(proposer);
return new RoundSpecificPeers(proposer, remotePeers.values(), nonProposers, bftExtraDataCodec);
return new RoundSpecificPeers(proposer, remotePeers.values(), nonProposers, blockEncoder);
}
public NodeParams getLocalNodeParams() {
@@ -165,4 +195,9 @@ public class TestContext {
public ValidatorProvider getValidatorProvider() {
return validatorProvider;
}
public void appendBlock(final QbftBlock signedCurrentHeightBlock) {
blockchain.appendBlock(
BlockUtil.toBesuBlock(signedCurrentHeightBlock), Collections.emptyList());
}
}

View File

@@ -38,7 +38,6 @@ import org.hyperledger.besu.consensus.common.bft.BftContext;
import org.hyperledger.besu.consensus.common.bft.BftEventQueue;
import org.hyperledger.besu.consensus.common.bft.BftExecutors;
import org.hyperledger.besu.consensus.common.bft.BftExtraData;
import org.hyperledger.besu.consensus.common.bft.BftExtraDataCodec;
import org.hyperledger.besu.consensus.common.bft.BftHelpers;
import org.hyperledger.besu.consensus.common.bft.BftProtocolSchedule;
import org.hyperledger.besu.consensus.common.bft.BlockTimer;
@@ -48,7 +47,6 @@ import org.hyperledger.besu.consensus.common.bft.MessageTracker;
import org.hyperledger.besu.consensus.common.bft.RoundTimer;
import org.hyperledger.besu.consensus.common.bft.SynchronizerUpdater;
import org.hyperledger.besu.consensus.common.bft.UniqueMessageMulticaster;
import org.hyperledger.besu.consensus.common.bft.blockcreation.BftBlockCreatorFactory;
import org.hyperledger.besu.consensus.common.bft.blockcreation.ProposerSelector;
import org.hyperledger.besu.consensus.common.bft.inttest.DefaultValidatorPeer;
import org.hyperledger.besu.consensus.common.bft.inttest.NetworkLayout;
@@ -57,7 +55,6 @@ import org.hyperledger.besu.consensus.common.bft.inttest.StubValidatorMulticaste
import org.hyperledger.besu.consensus.common.bft.inttest.StubbedSynchronizerUpdater;
import org.hyperledger.besu.consensus.common.bft.inttest.TestTransitions;
import org.hyperledger.besu.consensus.common.bft.statemachine.BftEventHandler;
import org.hyperledger.besu.consensus.common.bft.statemachine.BftFinalState;
import org.hyperledger.besu.consensus.common.bft.statemachine.FutureMessageBuffer;
import org.hyperledger.besu.consensus.common.validator.ValidatorProvider;
import org.hyperledger.besu.consensus.common.validator.blockbased.BlockValidatorProvider;
@@ -65,12 +62,24 @@ import org.hyperledger.besu.consensus.qbft.MutableQbftConfigOptions;
import org.hyperledger.besu.consensus.qbft.QbftExtraDataCodec;
import org.hyperledger.besu.consensus.qbft.QbftForksSchedulesFactory;
import org.hyperledger.besu.consensus.qbft.QbftProtocolScheduleBuilder;
import org.hyperledger.besu.consensus.qbft.adaptor.QbftBlockCodecAdaptor;
import org.hyperledger.besu.consensus.qbft.adaptor.QbftBlockCreatorFactoryAdaptor;
import org.hyperledger.besu.consensus.qbft.adaptor.QbftBlockInterfaceAdaptor;
import org.hyperledger.besu.consensus.qbft.adaptor.QbftExtraDataProviderAdaptor;
import org.hyperledger.besu.consensus.qbft.adaptor.QbftFinalStateImpl;
import org.hyperledger.besu.consensus.qbft.adaptor.QbftProtocolScheduleAdaptor;
import org.hyperledger.besu.consensus.qbft.blockcreation.QbftBlockCreatorFactory;
import org.hyperledger.besu.consensus.qbft.core.network.QbftGossip;
import org.hyperledger.besu.consensus.qbft.core.payload.MessageFactory;
import org.hyperledger.besu.consensus.qbft.core.statemachine.QbftBlockHeightManagerFactory;
import org.hyperledger.besu.consensus.qbft.core.statemachine.QbftController;
import org.hyperledger.besu.consensus.qbft.core.statemachine.QbftRoundFactory;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockCodec;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockInterface;
import org.hyperledger.besu.consensus.qbft.core.types.QbftContext;
import org.hyperledger.besu.consensus.qbft.core.types.QbftExtraDataProvider;
import org.hyperledger.besu.consensus.qbft.core.types.QbftFinalState;
import org.hyperledger.besu.consensus.qbft.core.types.QbftMinedBlockObserver;
import org.hyperledger.besu.consensus.qbft.core.validation.MessageValidatorFactory;
import org.hyperledger.besu.consensus.qbft.core.validator.ValidatorModeTransitionLogger;
import org.hyperledger.besu.consensus.qbft.validator.ForkingValidatorProvider;
@@ -83,7 +92,6 @@ import org.hyperledger.besu.datatypes.Wei;
import org.hyperledger.besu.ethereum.ProtocolContext;
import org.hyperledger.besu.ethereum.chain.BadBlockManager;
import org.hyperledger.besu.ethereum.chain.GenesisState;
import org.hyperledger.besu.ethereum.chain.MinedBlockObserver;
import org.hyperledger.besu.ethereum.chain.MutableBlockchain;
import org.hyperledger.besu.ethereum.core.AddressHelpers;
import org.hyperledger.besu.ethereum.core.Block;
@@ -141,10 +149,11 @@ public class TestContextBuilder {
private record ControllerAndState(
BftExecutors bftExecutors,
BftEventHandler eventHandler,
BftFinalState finalState,
QbftFinalState finalState,
EventMultiplexer eventMultiplexer,
MessageFactory messageFactory,
ValidatorProvider validatorProvider) {}
ValidatorProvider validatorProvider,
ProposerSelector proposerSelector) {}
private static final MetricsSystem metricsSystem = new NoOpMetricsSystem();
private boolean useValidatorContract;
@@ -162,7 +171,7 @@ public class TestContextBuilder {
public static final int FUTURE_MESSAGES_LIMIT = 1000;
public static final Address VALIDATOR_CONTRACT_ADDRESS =
Address.fromHexString("0x0000000000000000000000000000000000008888");
private static final BftExtraDataCodec BFT_EXTRA_DATA_ENCODER = new QbftExtraDataCodec();
private static final QbftExtraDataCodec BFT_EXTRA_DATA_ENCODER = new QbftExtraDataCodec();
private Clock clock = Clock.fixed(Instant.MIN, ZoneId.of("UTC"));
private BftEventQueue bftEventQueue = new BftEventQueue(MESSAGE_QUEUE_LIMIT);
@@ -254,6 +263,7 @@ public class TestContextBuilder {
networkNodes = new NetworkLayout(localNode, addressKeyMap);
}
final QbftBlockCodec blockEncoder = new QbftBlockCodecAdaptor(BFT_EXTRA_DATA_ENCODER);
final MutableBlockchain blockChain;
final ForestWorldStateArchive worldStateArchive = createInMemoryWorldStateArchive();
@@ -281,9 +291,7 @@ public class TestContextBuilder {
new UniqueMessageMulticaster(multicaster, GOSSIPED_HISTORY_LIMIT);
final Gossiper gossiper =
useGossip
? new QbftGossip(uniqueMulticaster, BFT_EXTRA_DATA_ENCODER)
: mock(Gossiper.class);
useGossip ? new QbftGossip(uniqueMulticaster, blockEncoder) : mock(Gossiper.class);
final StubbedSynchronizerUpdater synchronizerUpdater = new StubbedSynchronizerUpdater();
@@ -302,7 +310,8 @@ public class TestContextBuilder {
useShanghaiMilestone,
useZeroBaseFee,
useFixedBaseFee,
qbftForks);
qbftForks,
blockEncoder);
// Add each networkNode to the Multicaster (such that each can receive msgs from local node).
// NOTE: the remotePeers needs to be ordered based on Address (as this is used to determine
@@ -315,7 +324,7 @@ public class TestContextBuilder {
nodeParams ->
new ValidatorPeer(
nodeParams,
new MessageFactory(nodeParams.getNodeKey()),
new MessageFactory(nodeParams.getNodeKey(), blockEncoder),
controllerAndState.eventMultiplexer()),
(u, v) -> {
throw new IllegalStateException(String.format("Duplicate key %s", u));
@@ -335,7 +344,9 @@ public class TestContextBuilder {
controllerAndState.eventMultiplexer(),
controllerAndState.messageFactory(),
controllerAndState.validatorProvider(),
BFT_EXTRA_DATA_ENCODER);
controllerAndState.proposerSelector(),
BFT_EXTRA_DATA_ENCODER,
blockEncoder);
}
public TestContext buildAndStart() {
@@ -385,7 +396,8 @@ public class TestContextBuilder {
final boolean useShanghaiMilestone,
final boolean useZeroBaseFee,
final boolean useFixedBaseFee,
final List<QbftFork> qbftForks) {
final List<QbftFork> qbftForks,
final QbftBlockCodec blockEncoder) {
final MiningConfiguration miningConfiguration =
ImmutableMiningConfiguration.builder()
@@ -427,7 +439,8 @@ public class TestContextBuilder {
final EpochManager epochManager = new EpochManager(EPOCH_LENGTH);
final BftBlockInterface blockInterface = new BftBlockInterface(BFT_EXTRA_DATA_ENCODER);
final BftBlockInterface bftBlockInterface = new BftBlockInterface(BFT_EXTRA_DATA_ENCODER);
final QbftBlockInterface qbftBlockInterface = new QbftBlockInterfaceAdaptor(bftBlockInterface);
final ForksSchedule<QbftConfigOptions> forksSchedule =
QbftForksSchedulesFactory.create(genesisConfigOptions);
@@ -450,7 +463,7 @@ public class TestContextBuilder {
final BlockValidatorProvider blockValidatorProvider =
BlockValidatorProvider.forkingValidatorProvider(
blockChain, epochManager, blockInterface, validatorOverrides);
blockChain, epochManager, bftBlockInterface, validatorOverrides);
final TransactionValidatorProvider transactionValidatorProvider =
new TransactionValidatorProvider(
blockChain, new ValidatorContractController(transactionSimulator), forksSchedule);
@@ -458,11 +471,17 @@ public class TestContextBuilder {
new ForkingValidatorProvider(
blockChain, forksSchedule, blockValidatorProvider, transactionValidatorProvider);
final ProtocolContext protocolContext =
final ProtocolContext bftProtocolContext =
new ProtocolContext(
blockChain,
worldStateArchive,
new BftContext(validatorProvider, epochManager, blockInterface),
new BftContext(validatorProvider, epochManager, bftBlockInterface),
new BadBlockManager());
final ProtocolContext qbftProtocolContext =
new ProtocolContext(
blockChain,
worldStateArchive,
new QbftContext(validatorProvider, qbftBlockInterface),
new BadBlockManager());
final TransactionPoolConfiguration poolConf =
@@ -479,7 +498,7 @@ public class TestContextBuilder {
new TransactionPool(
() -> pendingTransactions,
protocolSchedule,
protocolContext,
bftProtocolContext,
mock(TransactionBroadcaster.class),
ethContext,
new TransactionPoolMetrics(metricsSystem),
@@ -491,10 +510,10 @@ public class TestContextBuilder {
final EthScheduler ethScheduler = new DeterministicEthScheduler();
final Address localAddress = Util.publicKeyToAddress(nodeKey.getPublicKey());
final BftBlockCreatorFactory<?> blockCreatorFactory =
final QbftBlockCreatorFactory blockCreatorFactory =
new QbftBlockCreatorFactory(
transactionPool, // changed from QbftBesuController
protocolContext,
bftProtocolContext,
protocolSchedule,
forksSchedule,
miningConfiguration,
@@ -503,29 +522,30 @@ public class TestContextBuilder {
ethScheduler);
final ProposerSelector proposerSelector =
new ProposerSelector(blockChain, blockInterface, true, validatorProvider);
new ProposerSelector(blockChain, bftBlockInterface, true, validatorProvider);
final BftExecutors bftExecutors =
BftExecutors.create(new NoOpMetricsSystem(), BftExecutors.ConsensusType.QBFT);
final BftFinalState finalState =
new BftFinalState(
protocolContext.getConsensusContext(BftContext.class).getValidatorProvider(),
final QbftFinalState finalState =
new QbftFinalStateImpl(
validatorProvider,
nodeKey,
Util.publicKeyToAddress(nodeKey.getPublicKey()),
proposerSelector,
multicaster,
new RoundTimer(bftEventQueue, Duration.ofSeconds(ROUND_TIMER_SEC), bftExecutors),
new BlockTimer(bftEventQueue, forksSchedule, bftExecutors, TestClock.fixed()),
blockCreatorFactory,
new QbftBlockCreatorFactoryAdaptor(blockCreatorFactory, BFT_EXTRA_DATA_ENCODER),
clock);
final MessageFactory messageFactory = new MessageFactory(nodeKey);
final MessageFactory messageFactory = new MessageFactory(nodeKey, blockEncoder);
final QbftProtocolScheduleAdaptor qbftProtocolSchedule =
new QbftProtocolScheduleAdaptor(protocolSchedule, bftProtocolContext);
final MessageValidatorFactory messageValidatorFactory =
new MessageValidatorFactory(
proposerSelector, protocolSchedule, protocolContext, BFT_EXTRA_DATA_ENCODER);
new MessageValidatorFactory(proposerSelector, qbftProtocolSchedule, qbftProtocolContext);
final Subscribers<MinedBlockObserver> minedBlockObservers = Subscribers.create();
final Subscribers<QbftMinedBlockObserver> minedBlockObservers = Subscribers.create();
final MessageTracker duplicateMessageTracker = new MessageTracker(DUPLICATE_MESSAGE_LIMIT);
final FutureMessageBuffer futureMessageBuffer =
@@ -533,8 +553,10 @@ public class TestContextBuilder {
FUTURE_MESSAGES_MAX_DISTANCE,
FUTURE_MESSAGES_LIMIT,
blockChain.getChainHeadBlockNumber());
final QbftExtraDataProvider qbftExtraDataProvider =
new QbftExtraDataProviderAdaptor(BFT_EXTRA_DATA_ENCODER);
final BftEventHandler qbftController =
final QbftController qbftController =
new QbftController(
blockChain,
finalState,
@@ -542,12 +564,13 @@ public class TestContextBuilder {
finalState,
new QbftRoundFactory(
finalState,
protocolContext,
protocolSchedule,
qbftProtocolContext,
qbftProtocolSchedule,
minedBlockObservers,
messageValidatorFactory,
messageFactory,
BFT_EXTRA_DATA_ENCODER),
BFT_EXTRA_DATA_ENCODER,
qbftExtraDataProvider),
messageValidatorFactory,
messageFactory,
new ValidatorModeTransitionLogger(forksSchedule)),
@@ -555,7 +578,7 @@ public class TestContextBuilder {
duplicateMessageTracker,
futureMessageBuffer,
synchronizerUpdater,
BFT_EXTRA_DATA_ENCODER);
blockEncoder);
final EventMultiplexer eventMultiplexer = new EventMultiplexer(qbftController);
//////////////////////////// END QBFT BesuController ////////////////////////////
@@ -566,7 +589,8 @@ public class TestContextBuilder {
finalState,
eventMultiplexer,
messageFactory,
validatorProvider);
validatorProvider,
proposerSelector);
}
private static QbftConfigOptions createGenesisConfig(final boolean useValidatorContract) {

View File

@@ -33,9 +33,9 @@ import org.hyperledger.besu.consensus.qbft.core.payload.MessageFactory;
import org.hyperledger.besu.consensus.qbft.core.payload.PreparePayload;
import org.hyperledger.besu.consensus.qbft.core.payload.RoundChangePayload;
import org.hyperledger.besu.consensus.qbft.core.statemachine.PreparedCertificate;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlock;
import org.hyperledger.besu.crypto.SECPSignature;
import org.hyperledger.besu.datatypes.Hash;
import org.hyperledger.besu.ethereum.core.Block;
import java.util.Collections;
import java.util.List;
@@ -54,7 +54,7 @@ public class ValidatorPeer extends DefaultValidatorPeer {
this.messageFactory = messageFactory;
}
public Proposal injectProposal(final ConsensusRoundIdentifier rId, final Block block) {
public Proposal injectProposal(final ConsensusRoundIdentifier rId, final QbftBlock block) {
return injectProposalForFutureRound(
rId, Collections.emptyList(), Collections.emptyList(), block);
}
@@ -65,8 +65,8 @@ public class ValidatorPeer extends DefaultValidatorPeer {
return payload;
}
public Commit injectCommit(final ConsensusRoundIdentifier rId, final Block block) {
final Block commitBlock = createCommitBlockFromProposalBlock(block, rId.getRoundNumber());
public Commit injectCommit(final ConsensusRoundIdentifier rId, final QbftBlock block) {
final QbftBlock commitBlock = createCommitBlockFromProposalBlock(block, rId.getRoundNumber());
final SECPSignature commitSeal = nodeKey.sign(commitBlock.getHash());
return injectCommit(rId, block.getHash(), commitSeal);
}
@@ -82,7 +82,7 @@ public class ValidatorPeer extends DefaultValidatorPeer {
final ConsensusRoundIdentifier rId,
final List<SignedData<RoundChangePayload>> roundChanges,
final List<SignedData<PreparePayload>> prepares,
final Block blockToPropose) {
final QbftBlock blockToPropose) {
final Proposal payload =
messageFactory.createProposal(rId, blockToPropose, roundChanges, prepares);

View File

@@ -15,11 +15,9 @@
package org.hyperledger.besu.consensus.qbft.core.test;
import static org.assertj.core.api.Assertions.assertThat;
import static org.assertj.core.util.Lists.emptyList;
import static org.hyperledger.besu.consensus.qbft.core.support.IntegrationTestHelpers.createSignedCommitPayload;
import org.hyperledger.besu.consensus.common.bft.BftExtraDataCodec;
import org.hyperledger.besu.consensus.common.bft.BftHelpers;
import org.hyperledger.besu.consensus.common.bft.ConsensusRoundIdentifier;
import org.hyperledger.besu.consensus.common.bft.events.NewChainHead;
import org.hyperledger.besu.consensus.qbft.QbftExtraDataCodec;
@@ -29,7 +27,7 @@ import org.hyperledger.besu.consensus.qbft.core.payload.MessageFactory;
import org.hyperledger.besu.consensus.qbft.core.support.RoundSpecificPeers;
import org.hyperledger.besu.consensus.qbft.core.support.TestContext;
import org.hyperledger.besu.consensus.qbft.core.support.TestContextBuilder;
import org.hyperledger.besu.ethereum.core.Block;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlock;
import java.time.Clock;
import java.time.Instant;
@@ -63,12 +61,12 @@ public class FutureHeightTest {
@Test
public void messagesForFutureHeightAreBufferedUntilChainHeightCatchesUp() {
final Block currentHeightBlock = context.createBlockForProposalFromChainHead(30);
final Block signedCurrentHeightBlock =
BftHelpers.createSealedBlock(
final QbftBlock currentHeightBlock = context.createBlockForProposalFromChainHead(30);
final QbftBlock signedCurrentHeightBlock =
context.createSealedBlock(
bftExtraDataCodec, currentHeightBlock, 0, peers.sign(currentHeightBlock.getHash()));
final Block futureHeightBlock =
final QbftBlock futureHeightBlock =
context.createBlockForProposal(
signedCurrentHeightBlock.getHeader(), 60, peers.getProposer().getNodeAddress());
@@ -92,7 +90,7 @@ public class FutureHeightTest {
assertThat(context.getCurrentChainHeight()).isEqualTo(0);
// Add block to chain, and notify system of its arrival.
context.getBlockchain().appendBlock(signedCurrentHeightBlock, emptyList());
context.appendBlock(signedCurrentHeightBlock);
assertThat(context.getCurrentChainHeight()).isEqualTo(1);
context
.getController()
@@ -104,7 +102,10 @@ public class FutureHeightTest {
final Commit expectedCommitMessage =
new Commit(
createSignedCommitPayload(
futureHeightRoundId, futureHeightBlock, context.getLocalNodeParams().getNodeKey()));
futureHeightRoundId,
futureHeightBlock,
context.getLocalNodeParams().getNodeKey(),
context.getBlockEncoder()));
peers.verifyMessagesReceived(expectedPrepareMessage, expectedCommitMessage);
assertThat(context.getCurrentChainHeight()).isEqualTo(2);
@@ -112,10 +113,10 @@ public class FutureHeightTest {
@Test
public void messagesFromPreviousHeightAreDiscarded() {
final Block currentHeightBlock =
final QbftBlock currentHeightBlock =
context.createBlockForProposalFromChainHead(30, peers.getProposer().getNodeAddress());
final Block signedCurrentHeightBlock =
BftHelpers.createSealedBlock(
final QbftBlock signedCurrentHeightBlock =
context.createSealedBlock(
bftExtraDataCodec, currentHeightBlock, 0, peers.sign(currentHeightBlock.getHash()));
peers.getProposer().injectProposal(roundId, currentHeightBlock);
@@ -127,7 +128,7 @@ public class FutureHeightTest {
peers.verifyMessagesReceived(expectedPrepareMessage);
// Add block to chain, and notify system of its arrival.
context.getBlockchain().appendBlock(signedCurrentHeightBlock, emptyList());
context.appendBlock(signedCurrentHeightBlock);
assertThat(context.getCurrentChainHeight()).isEqualTo(1);
context
.getController()
@@ -144,7 +145,7 @@ public class FutureHeightTest {
@Test
public void multipleNewChainHeadEventsDoesNotRestartCurrentHeightManager() {
final Block currentHeightBlock =
final QbftBlock currentHeightBlock =
context.createBlockForProposalFromChainHead(30, peers.getProposer().getNodeAddress());
peers.getProposer().injectProposal(roundId, currentHeightBlock);
@@ -163,25 +164,28 @@ public class FutureHeightTest {
final Commit expectedCommitMessage =
new Commit(
createSignedCommitPayload(
roundId, currentHeightBlock, context.getLocalNodeParams().getNodeKey()));
roundId,
currentHeightBlock,
context.getLocalNodeParams().getNodeKey(),
context.getBlockEncoder()));
peers.verifyMessagesReceived(expectedCommitMessage);
}
@Test
public void correctMessagesAreExtractedFromFutureHeightBuffer() {
final Block currentHeightBlock = context.createBlockForProposalFromChainHead(30);
final Block signedCurrentHeightBlock =
BftHelpers.createSealedBlock(
final QbftBlock currentHeightBlock = context.createBlockForProposalFromChainHead(30);
final QbftBlock signedCurrentHeightBlock =
context.createSealedBlock(
bftExtraDataCodec, currentHeightBlock, 0, peers.sign(currentHeightBlock.getHash()));
final Block nextHeightBlock =
final QbftBlock nextHeightBlock =
context.createBlockForProposal(
signedCurrentHeightBlock.getHeader(), 60, peers.getProposer().getNodeAddress());
final Block signedNextHeightBlock =
BftHelpers.createSealedBlock(
final QbftBlock signedNextHeightBlock =
context.createSealedBlock(
bftExtraDataCodec, nextHeightBlock, 0, peers.sign(nextHeightBlock.getHash()));
final Block futureHeightBlock =
final QbftBlock futureHeightBlock =
context.createBlockForProposal(
signedNextHeightBlock.getHeader(), 90, peers.getNonProposing(0).getNodeAddress());
@@ -193,7 +197,7 @@ public class FutureHeightTest {
peers.commitForNonProposing(futureHeightRoundId, futureHeightBlock);
// Add the "interim" block to chain, and notify system of its arrival.
context.getBlockchain().appendBlock(signedCurrentHeightBlock, emptyList());
context.appendBlock(signedCurrentHeightBlock);
assertThat(context.getCurrentChainHeight()).isEqualTo(1);
context
.getController()
@@ -213,7 +217,7 @@ public class FutureHeightTest {
peers.getNonProposing(0).injectProposal(futureHeightRoundId, futureHeightBlock);
// Change to the FutureRound, and confirm prepare and commit msgs are sent
context.getBlockchain().appendBlock(signedNextHeightBlock, emptyList());
context.appendBlock(signedNextHeightBlock);
assertThat(context.getCurrentChainHeight()).isEqualTo(2);
context
.getController()
@@ -225,7 +229,10 @@ public class FutureHeightTest {
final Commit expectedCommitMessage =
new Commit(
createSignedCommitPayload(
futureHeightRoundId, futureHeightBlock, context.getLocalNodeParams().getNodeKey()));
futureHeightRoundId,
futureHeightBlock,
context.getLocalNodeParams().getNodeKey(),
context.getBlockEncoder()));
// Assert ONLY a prepare message was received, not any commits (i.e. futureHeightRoundId
// messages have not been used.

View File

@@ -28,8 +28,8 @@ import org.hyperledger.besu.consensus.qbft.core.payload.RoundChangePayload;
import org.hyperledger.besu.consensus.qbft.core.support.RoundSpecificPeers;
import org.hyperledger.besu.consensus.qbft.core.support.TestContext;
import org.hyperledger.besu.consensus.qbft.core.support.TestContextBuilder;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlock;
import org.hyperledger.besu.crypto.SECPSignature;
import org.hyperledger.besu.ethereum.core.Block;
import java.time.Clock;
import java.time.Instant;
@@ -64,7 +64,7 @@ public class FutureRoundTest {
@Test
public void messagesForFutureRoundAreNotActionedUntilRoundIsActive() {
final Block futureBlock =
final QbftBlock futureBlock =
context.createBlockForProposalFromChainHead(60, peers.getProposer().getNodeAddress());
final int quorum = BftHelpers.calculateRequiredValidatorQuorum(NETWORK_SIZE);
final ConsensusRoundIdentifier subsequentRoundId = new ConsensusRoundIdentifier(1, 6);
@@ -104,7 +104,7 @@ public class FutureRoundTest {
// following 1 more prepare, a commit msg will be sent
futurePeers.getNonProposing(quorum - 2).injectPrepare(futureRoundId, futureBlock.getHash());
final Block commitBlock =
final QbftBlock commitBlock =
createCommitBlockFromProposalBlock(futureBlock, futureRoundId.getRoundNumber());
final SECPSignature commitSeal =
context.getLocalNodeParams().getNodeKey().sign(commitBlock.getHash());
@@ -120,9 +120,9 @@ public class FutureRoundTest {
@Test
public void priorRoundsCannotBeCompletedAfterReceptionOfNewRound() {
final Block initialBlock =
final QbftBlock initialBlock =
context.createBlockForProposalFromChainHead(30, peers.getProposer().getNodeAddress());
final Block futureBlock =
final QbftBlock futureBlock =
context.createBlockForProposalFromChainHead(60, peers.getProposer().getNodeAddress());
peers.getProposer().injectProposal(roundId, initialBlock);

View File

@@ -16,7 +16,6 @@ package org.hyperledger.besu.consensus.qbft.core.test;
import static java.util.Collections.emptyList;
import org.hyperledger.besu.consensus.common.bft.BftHelpers;
import org.hyperledger.besu.consensus.common.bft.ConsensusRoundIdentifier;
import org.hyperledger.besu.consensus.common.bft.events.NewChainHead;
import org.hyperledger.besu.consensus.qbft.QbftExtraDataCodec;
@@ -30,8 +29,8 @@ import org.hyperledger.besu.consensus.qbft.core.support.RoundSpecificPeers;
import org.hyperledger.besu.consensus.qbft.core.support.TestContext;
import org.hyperledger.besu.consensus.qbft.core.support.TestContextBuilder;
import org.hyperledger.besu.consensus.qbft.core.support.ValidatorPeer;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlock;
import org.hyperledger.besu.cryptoservices.NodeKeyUtils;
import org.hyperledger.besu.ethereum.core.Block;
import java.time.Clock;
import java.time.Instant;
@@ -60,7 +59,7 @@ public class GossipTest {
private final ConsensusRoundIdentifier roundId = new ConsensusRoundIdentifier(1, 0);
private final RoundSpecificPeers peers = context.roundSpecificPeers(roundId);
private Block block;
private QbftBlock block;
private ValidatorPeer sender;
private MessageFactory msgFactory;
@@ -120,7 +119,8 @@ public class GossipTest {
@Test
public void messageWithUnknownValidatorIsNotGossiped() {
final MessageFactory unknownMsgFactory = new MessageFactory(NodeKeyUtils.generate());
final MessageFactory unknownMsgFactory =
new MessageFactory(NodeKeyUtils.generate(), context.getBlockEncoder());
final Proposal unknownProposal =
unknownMsgFactory.createProposal(roundId, block, emptyList(), emptyList());
@@ -159,16 +159,15 @@ public class GossipTest {
@Test
public void futureMessageGetGossipedLater() {
final Block signedCurrentHeightBlock =
BftHelpers.createSealedBlock(
new QbftExtraDataCodec(), block, 0, peers.sign(block.getHash()));
final QbftBlock signedCurrentHeightBlock =
context.createSealedBlock(new QbftExtraDataCodec(), block, 0, peers.sign(block.getHash()));
final ConsensusRoundIdentifier futureRoundId = new ConsensusRoundIdentifier(2, 0);
final Prepare futurePrepare = sender.injectPrepare(futureRoundId, block.getHash());
peers.verifyNoMessagesReceivedNonProposing();
// add block to chain so we can move to next block height
context.getBlockchain().appendBlock(signedCurrentHeightBlock, emptyList());
context.appendBlock(signedCurrentHeightBlock);
context
.getController()
.handleNewBlockEvent(new NewChainHead(signedCurrentHeightBlock.getHeader()));

View File

@@ -28,7 +28,7 @@ import org.hyperledger.besu.consensus.qbft.core.payload.MessageFactory;
import org.hyperledger.besu.consensus.qbft.core.support.RoundSpecificPeers;
import org.hyperledger.besu.consensus.qbft.core.support.TestContext;
import org.hyperledger.besu.consensus.qbft.core.support.TestContextBuilder;
import org.hyperledger.besu.ethereum.core.Block;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlock;
import java.time.Clock;
import java.time.Instant;
@@ -62,7 +62,7 @@ public class LocalNodeIsProposerTest {
private final MessageFactory localNodeMessageFactory = context.getLocalNodeMessageFactory();
private Block expectedProposedBlock;
private QbftBlock expectedProposedBlock;
private Proposal expectedTxProposal;
private Commit expectedTxCommit;
private Prepare expectedTxPrepare;
@@ -80,7 +80,10 @@ public class LocalNodeIsProposerTest {
expectedTxCommit =
new Commit(
createSignedCommitPayload(
roundId, expectedProposedBlock, context.getLocalNodeParams().getNodeKey()));
roundId,
expectedProposedBlock,
context.getLocalNodeParams().getNodeKey(),
context.getBlockEncoder()));
// Trigger "block timer" to send proposal.
context.getController().handleBlockTimerExpiry(new BlockTimerExpiry(roundId));

View File

@@ -24,7 +24,7 @@ import org.hyperledger.besu.consensus.qbft.core.payload.MessageFactory;
import org.hyperledger.besu.consensus.qbft.core.support.RoundSpecificPeers;
import org.hyperledger.besu.consensus.qbft.core.support.TestContext;
import org.hyperledger.besu.consensus.qbft.core.support.TestContextBuilder;
import org.hyperledger.besu.ethereum.core.Block;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlock;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
@@ -45,7 +45,7 @@ public class LocalNodeNotProposerTest {
private final MessageFactory localNodeMessageFactory = context.getLocalNodeMessageFactory();
private final Block blockToPropose =
private final QbftBlock blockToPropose =
context.createBlockForProposalFromChainHead(15, peers.getProposer().getNodeAddress());
private Prepare expectedTxPrepare;
@@ -58,7 +58,10 @@ public class LocalNodeNotProposerTest {
expectedTxCommit =
new Commit(
createSignedCommitPayload(
roundId, blockToPropose, context.getLocalNodeParams().getNodeKey()));
roundId,
blockToPropose,
context.getLocalNodeParams().getNodeKey(),
context.getBlockEncoder()));
}
@Test

View File

@@ -29,7 +29,7 @@ import org.hyperledger.besu.consensus.qbft.core.support.RoundSpecificPeers;
import org.hyperledger.besu.consensus.qbft.core.support.TestContext;
import org.hyperledger.besu.consensus.qbft.core.support.TestContextBuilder;
import org.hyperledger.besu.consensus.qbft.core.support.ValidatorPeer;
import org.hyperledger.besu.ethereum.core.Block;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlock;
import java.util.Collections;
import java.util.List;
@@ -64,7 +64,7 @@ public class ReceivedFutureProposalTest {
peers.createSignedRoundChangePayload(targetRound);
final ValidatorPeer nextProposer = context.roundSpecificPeers(targetRound).getProposer();
final Block blockToPropose =
final QbftBlock blockToPropose =
context.createBlockForProposalFromChainHead(15, nextProposer.getNodeAddress());
nextProposer.injectProposalForFutureRound(
@@ -79,7 +79,7 @@ public class ReceivedFutureProposalTest {
@Test
public void proposalFromIllegalSenderIsDiscardedAndNoPrepareForNewRoundIsSent() {
final ConsensusRoundIdentifier nextRoundId = new ConsensusRoundIdentifier(1, 1);
final Block blockToPropose =
final QbftBlock blockToPropose =
context.createBlockForProposalFromChainHead(15, peers.getProposer().getNodeAddress());
final List<SignedData<RoundChangePayload>> roundChanges =
@@ -96,9 +96,9 @@ public class ReceivedFutureProposalTest {
@Test
public void proposalWithPrepareCertificateResultsInNewRoundStartingWithExpectedBlock() {
final Block initialBlock =
final QbftBlock initialBlock =
context.createBlockForProposalFromChainHead(15, peers.getProposer().getNodeAddress());
final Block reproposedBlock =
final QbftBlock reproposedBlock =
context.createBlockForProposalFromChainHead(15, peers.getProposer().getNodeAddress());
final ConsensusRoundIdentifier nextRoundId = new ConsensusRoundIdentifier(1, 1);
@@ -121,9 +121,9 @@ public class ReceivedFutureProposalTest {
@Test
public void futureProposalWithInsufficientPreparesDoesNotTriggerNextRound() {
final Block initialBlock =
final QbftBlock initialBlock =
context.createBlockForProposalFromChainHead(15, peers.getProposer().getNodeAddress());
final Block reproposedBlock =
final QbftBlock reproposedBlock =
context.createBlockForProposalFromChainHead(15, peers.getProposer().getNodeAddress());
final ConsensusRoundIdentifier nextRoundId = new ConsensusRoundIdentifier(1, 1);
@@ -146,9 +146,9 @@ public class ReceivedFutureProposalTest {
@Test
public void futureProposalWithInvalidPrepareDoesNotTriggerNextRound() {
final Block initialBlock =
final QbftBlock initialBlock =
context.createBlockForProposalFromChainHead(15, peers.getProposer().getNodeAddress());
final Block reproposedBlock = context.createBlockForProposalFromChainHead(15);
final QbftBlock reproposedBlock = context.createBlockForProposalFromChainHead(15);
final ConsensusRoundIdentifier nextRoundId = new ConsensusRoundIdentifier(1, 1);
final PreparedCertificate preparedRoundArtifacts =
@@ -205,7 +205,7 @@ public class ReceivedFutureProposalTest {
@Test
public void receiveRoundStateIsNotLostIfASecondProposalMessageIsReceivedForCurrentRound() {
final Block block =
final QbftBlock block =
context.createBlockForProposalFromChainHead(15, peers.getProposer().getNodeAddress());
final ConsensusRoundIdentifier nextRoundId = new ConsensusRoundIdentifier(1, 1);
@@ -238,7 +238,10 @@ public class ReceivedFutureProposalTest {
final Commit expectedCommit =
new Commit(
IntegrationTestHelpers.createSignedCommitPayload(
nextRoundId, block, context.getLocalNodeParams().getNodeKey()));
nextRoundId,
block,
context.getLocalNodeParams().getNodeKey(),
context.getBlockEncoder()));
peers.verifyMessagesReceived(expectedCommit);
}

View File

@@ -33,7 +33,7 @@ import org.hyperledger.besu.consensus.qbft.core.support.RoundSpecificPeers;
import org.hyperledger.besu.consensus.qbft.core.support.TestContext;
import org.hyperledger.besu.consensus.qbft.core.support.TestContextBuilder;
import org.hyperledger.besu.consensus.qbft.core.support.ValidatorPeer;
import org.hyperledger.besu.ethereum.core.Block;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlock;
import java.time.Clock;
import java.time.Instant;
@@ -41,6 +41,7 @@ import java.time.ZoneId;
import java.util.List;
import java.util.Optional;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import com.google.common.collect.Lists;
import org.junit.jupiter.api.Test;
@@ -65,7 +66,7 @@ public class RoundChangeTest {
private final MessageFactory localNodeMessageFactory = context.getLocalNodeMessageFactory();
private final Block blockToPropose =
private final QbftBlock blockToPropose =
context.createBlockForProposalFromChainHead(15, peers.getProposer().getNodeAddress());
@Test
@@ -131,7 +132,7 @@ public class RoundChangeTest {
Optional.of(
new PreparedCertificate(
blockToPropose,
List.of(localPrepareMessage, p0, p1, p2).stream()
Stream.of(localPrepareMessage, p0, p1, p2)
.map(Prepare::getSignedPayload)
.collect(Collectors.toList()),
roundId.getRoundNumber())));
@@ -144,7 +145,7 @@ public class RoundChangeTest {
public void whenSufficientRoundChangeMessagesAreReceivedForNewRoundLocalNodeCreatesProposalMsg() {
// Note: Round-4 is the next round for which the local node is Proposer
final ConsensusRoundIdentifier targetRound = new ConsensusRoundIdentifier(1, 4);
final Block locallyProposedBlock =
final QbftBlock locallyProposedBlock =
context.createBlockForProposalFromChainHead(blockTimeStamp, 4);
final RoundChange rc1 = peers.getNonProposing(0).injectRoundChange(targetRound, empty());
@@ -205,7 +206,7 @@ public class RoundChangeTest {
// Expected to use the block with "ARBITRARY_BLOCKTIME" (i.e. latter block) but with the target
// round number.
final Block expectedBlockToPropose =
final QbftBlock expectedBlockToPropose =
context.createBlockForProposalFromChainHead(
ARBITRARY_BLOCKTIME, peers.getProposer().getNodeAddress(), 4);
@@ -235,7 +236,7 @@ public class RoundChangeTest {
final ConsensusRoundIdentifier priorRound = new ConsensusRoundIdentifier(1, 4);
peers.roundChange(priorRound);
final Block locallyProposedBlock =
final QbftBlock locallyProposedBlock =
context.createBlockForProposalFromChainHead(blockTimeStamp, 9);
final Proposal expectedProposal =
@@ -288,7 +289,7 @@ public class RoundChangeTest {
roundChangeMessages.addAll(peers.roundChangeForNonProposing(targetRound));
final Block expectedBlockToPropose =
final QbftBlock expectedBlockToPropose =
context.createBlockForProposalFromChainHead(
ARBITRARY_BLOCKTIME, peers.getProposer().getNodeAddress(), 4);

View File

@@ -27,11 +27,11 @@ import org.hyperledger.besu.consensus.qbft.core.support.RoundSpecificPeers;
import org.hyperledger.besu.consensus.qbft.core.support.TestContext;
import org.hyperledger.besu.consensus.qbft.core.support.TestContextBuilder;
import org.hyperledger.besu.consensus.qbft.core.support.ValidatorPeer;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlock;
import org.hyperledger.besu.crypto.SECPSignature;
import org.hyperledger.besu.cryptoservices.NodeKey;
import org.hyperledger.besu.cryptoservices.NodeKeyUtils;
import org.hyperledger.besu.datatypes.Hash;
import org.hyperledger.besu.ethereum.core.Block;
import org.hyperledger.besu.ethereum.core.Util;
import org.hyperledger.besu.ethereum.p2p.rlpx.wire.MessageData;
import org.hyperledger.besu.ethereum.p2p.rlpx.wire.RawMessage;
@@ -63,7 +63,7 @@ public class SpuriousBehaviourTest {
private final ConsensusRoundIdentifier roundId = new ConsensusRoundIdentifier(1, 0);
private final RoundSpecificPeers peers = context.roundSpecificPeers(roundId);
private final Block proposedBlock =
private final QbftBlock proposedBlock =
context.createBlockForProposalFromChainHead(30, peers.getProposer().getNodeAddress());
private Prepare expectedPrepare;
private Commit expectedCommit;
@@ -76,7 +76,10 @@ public class SpuriousBehaviourTest {
expectedCommit =
new Commit(
createSignedCommitPayload(
roundId, proposedBlock, context.getLocalNodeParams().getNodeKey()));
roundId,
proposedBlock,
context.getLocalNodeParams().getNodeKey(),
context.getBlockEncoder()));
}
@Test
@@ -107,7 +110,7 @@ public class SpuriousBehaviourTest {
final ValidatorPeer nonvalidator =
new ValidatorPeer(
nonValidatorParams,
new MessageFactory(nonValidatorParams.getNodeKey()),
new MessageFactory(nonValidatorParams.getNodeKey(), context.getBlockEncoder()),
context.getEventMultiplexer());
nonvalidator.injectProposal(new ConsensusRoundIdentifier(1, 0), proposedBlock);

View File

@@ -31,9 +31,9 @@ import org.hyperledger.besu.consensus.qbft.core.support.RoundSpecificPeers;
import org.hyperledger.besu.consensus.qbft.core.support.TestContext;
import org.hyperledger.besu.consensus.qbft.core.support.TestContextBuilder;
import org.hyperledger.besu.consensus.qbft.core.support.ValidatorPeer;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlock;
import org.hyperledger.besu.cryptoservices.NodeKeyUtils;
import org.hyperledger.besu.datatypes.Address;
import org.hyperledger.besu.ethereum.core.Block;
import org.hyperledger.besu.ethereum.core.BlockHeader;
import org.hyperledger.besu.testutil.TestClock;
@@ -464,7 +464,7 @@ public class ValidatorContractTest {
context.getController().handleBlockTimerExpiry(new BlockTimerExpiry(roundId));
// peers commit proposed block
Block proposedBlock =
QbftBlock proposedBlock =
context.createBlockForProposalFromChainHead(clock.instant().getEpochSecond());
RoundSpecificPeers peers = context.roundSpecificPeers(roundId);
peers.commitForNonProposing(roundId, proposedBlock);
@@ -483,7 +483,7 @@ public class ValidatorContractTest {
context.getController().handleBlockTimerExpiry(new BlockTimerExpiry(roundId));
// peers commit proposed block
Block proposedBlock = context.createBlockForProposalFromChainHead(timestamp);
QbftBlock proposedBlock = context.createBlockForProposalFromChainHead(timestamp);
RoundSpecificPeers peers = context.roundSpecificPeers(roundId);
peers.commitForNonProposing(roundId, proposedBlock);
@@ -498,7 +498,7 @@ public class ValidatorContractTest {
RoundSpecificPeers peers = context.roundSpecificPeers(roundId);
ValidatorPeer remoteProposer = peers.getProposer();
final Block blockToPropose =
final QbftBlock blockToPropose =
context.createBlockForProposalFromChainHead(
clock.instant().getEpochSecond(), remoteProposer.getNodeAddress());
remoteProposer.injectProposal(roundId, blockToPropose);

View File

@@ -17,25 +17,33 @@ package org.hyperledger.besu.consensus.qbft.core.test.round;
import static java.util.Collections.emptyList;
import static java.util.Optional.empty;
import static org.assertj.core.api.Assertions.assertThat;
import static org.hyperledger.besu.consensus.common.bft.BftContextBuilder.setupContextWithBftExtraDataEncoder;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.verifyNoInteractions;
import static org.mockito.Mockito.when;
import org.hyperledger.besu.consensus.common.bft.BftContext;
import org.hyperledger.besu.consensus.common.bft.BftBlockInterface;
import org.hyperledger.besu.consensus.common.bft.BftExtraData;
import org.hyperledger.besu.consensus.common.bft.BftExtraDataCodec;
import org.hyperledger.besu.consensus.common.bft.BftProtocolSchedule;
import org.hyperledger.besu.consensus.common.bft.ConsensusRoundIdentifier;
import org.hyperledger.besu.consensus.common.bft.RoundTimer;
import org.hyperledger.besu.consensus.common.bft.blockcreation.BftBlockCreator;
import org.hyperledger.besu.consensus.common.bft.inttest.StubValidatorMulticaster;
import org.hyperledger.besu.consensus.qbft.QbftExtraDataCodec;
import org.hyperledger.besu.consensus.qbft.adaptor.QbftBlockAdaptor;
import org.hyperledger.besu.consensus.qbft.adaptor.QbftBlockInterfaceAdaptor;
import org.hyperledger.besu.consensus.qbft.core.network.QbftMessageTransmitter;
import org.hyperledger.besu.consensus.qbft.core.payload.MessageFactory;
import org.hyperledger.besu.consensus.qbft.core.statemachine.QbftRound;
import org.hyperledger.besu.consensus.qbft.core.statemachine.RoundState;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlock;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockCodec;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockCreator;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockImporter;
import org.hyperledger.besu.consensus.qbft.core.types.QbftContext;
import org.hyperledger.besu.consensus.qbft.core.types.QbftExtraDataProvider;
import org.hyperledger.besu.consensus.qbft.core.types.QbftMinedBlockObserver;
import org.hyperledger.besu.consensus.qbft.core.types.QbftProtocolSchedule;
import org.hyperledger.besu.consensus.qbft.core.types.QbftProtocolSpec;
import org.hyperledger.besu.consensus.qbft.core.validation.MessageValidator;
import org.hyperledger.besu.crypto.SECPSignature;
import org.hyperledger.besu.crypto.SignatureAlgorithmFactory;
@@ -44,20 +52,17 @@ import org.hyperledger.besu.cryptoservices.NodeKeyUtils;
import org.hyperledger.besu.datatypes.Hash;
import org.hyperledger.besu.ethereum.ProtocolContext;
import org.hyperledger.besu.ethereum.chain.BadBlockManager;
import org.hyperledger.besu.ethereum.chain.MinedBlockObserver;
import org.hyperledger.besu.ethereum.chain.MutableBlockchain;
import org.hyperledger.besu.ethereum.core.Block;
import org.hyperledger.besu.ethereum.core.BlockBody;
import org.hyperledger.besu.ethereum.core.BlockHeader;
import org.hyperledger.besu.ethereum.core.BlockHeaderTestFixture;
import org.hyperledger.besu.ethereum.core.BlockImporter;
import org.hyperledger.besu.ethereum.mainnet.BlockImportResult;
import org.hyperledger.besu.ethereum.mainnet.ProtocolSpec;
import org.hyperledger.besu.ethereum.worldstate.WorldStateArchive;
import org.hyperledger.besu.plugin.services.securitymodule.SecurityModuleException;
import org.hyperledger.besu.util.Subscribers;
import java.math.BigInteger;
import java.util.List;
import org.apache.tuweni.bytes.Bytes;
import org.junit.jupiter.api.BeforeEach;
@@ -72,20 +77,20 @@ import org.mockito.quality.Strictness;
@MockitoSettings(strictness = Strictness.LENIENT)
public class QbftRoundIntegrationTest {
private final MessageFactory peerMessageFactory = new MessageFactory(NodeKeyUtils.generate());
private final MessageFactory peerMessageFactory2 = new MessageFactory(NodeKeyUtils.generate());
private final ConsensusRoundIdentifier roundIdentifier = new ConsensusRoundIdentifier(1, 0);
private final Subscribers<MinedBlockObserver> subscribers = Subscribers.create();
private final Subscribers<QbftMinedBlockObserver> subscribers = Subscribers.create();
private final BftExtraDataCodec bftExtraDataCodec = new QbftExtraDataCodec();
private MessageFactory peerMessageFactory;
private MessageFactory peerMessageFactory2;
private ProtocolContext protocolContext;
@Mock private BftProtocolSchedule protocolSchedule;
@Mock private ProtocolSpec protocolSpec;
@Mock private QbftProtocolSchedule protocolSchedule;
@Mock private QbftProtocolSpec protocolSpec;
@Mock private MutableBlockchain blockChain;
@Mock private WorldStateArchive worldStateArchive;
@Mock private BlockImporter blockImporter;
@Mock private QbftBlockImporter blockImporter;
@Mock private BftBlockCreator blockCreator;
@Mock private QbftBlockCreator blockCreator;
@Mock private MessageValidator messageValidator;
@Mock private RoundTimer roundTimer;
@Mock private NodeKey nodeKey;
@@ -93,8 +98,10 @@ public class QbftRoundIntegrationTest {
private QbftMessageTransmitter transmitter;
@Mock private StubValidatorMulticaster multicaster;
@Mock private BlockHeader parentHeader;
@Mock private QbftBlockCodec blockEncoder;
@Mock private QbftExtraDataProvider qbftExtraDataProvider;
private Block proposedBlock;
private QbftBlock proposedBlock;
private final SECPSignature remoteCommitSeal =
SignatureAlgorithmFactory.getInstance()
@@ -102,6 +109,9 @@ public class QbftRoundIntegrationTest {
@BeforeEach
public void setup() {
peerMessageFactory = new MessageFactory(NodeKeyUtils.generate(), blockEncoder);
peerMessageFactory2 = new MessageFactory(NodeKeyUtils.generate(), blockEncoder);
when(messageValidator.validateProposal(any())).thenReturn(true);
when(messageValidator.validatePrepare(any())).thenReturn(true);
when(messageValidator.validateCommit(any())).thenReturn(true);
@@ -109,7 +119,7 @@ public class QbftRoundIntegrationTest {
when(nodeKey.sign(any())).thenThrow(new SecurityModuleException("Hsm Is Down"));
final QbftExtraDataCodec qbftExtraDataEncoder = new QbftExtraDataCodec();
throwingMessageFactory = new MessageFactory(nodeKey);
throwingMessageFactory = new MessageFactory(nodeKey, blockEncoder);
transmitter = new QbftMessageTransmitter(throwingMessageFactory, multicaster);
final BftExtraData proposedExtraData =
@@ -118,19 +128,21 @@ public class QbftRoundIntegrationTest {
headerTestFixture.extraData(qbftExtraDataEncoder.encode(proposedExtraData));
headerTestFixture.number(1);
final BlockHeader header = headerTestFixture.buildHeader();
proposedBlock = new Block(header, new BlockBody(emptyList(), emptyList()));
final Block block = new Block(header, new BlockBody(emptyList(), emptyList()));
proposedBlock = new QbftBlockAdaptor(block);
when(qbftExtraDataProvider.getExtraData(header)).thenReturn(proposedExtraData);
when(protocolSchedule.getByBlockHeader(any())).thenReturn(protocolSpec);
when(protocolSpec.getBlockImporter()).thenReturn(blockImporter);
when(blockImporter.importBlock(any(), any(), any())).thenReturn(new BlockImportResult(true));
when(blockImporter.importBlock(any())).thenReturn(true);
protocolContext =
new ProtocolContext(
blockChain,
worldStateArchive,
setupContextWithBftExtraDataEncoder(
BftContext.class, emptyList(), qbftExtraDataEncoder),
new QbftContext(
null, new QbftBlockInterfaceAdaptor(new BftBlockInterface(qbftExtraDataEncoder))),
new BadBlockManager());
}
@@ -150,6 +162,7 @@ public class QbftRoundIntegrationTest {
transmitter,
roundTimer,
bftExtraDataCodec,
qbftExtraDataProvider,
parentHeader);
round.handleProposalMessage(
@@ -165,6 +178,16 @@ public class QbftRoundIntegrationTest {
@Test
public void failuresToSignStillAllowBlockToBeImported() {
final BlockHeader header = new BlockHeaderTestFixture().number(1).buildHeader();
final Block sealedBesuBlock = new Block(header, new BlockBody(emptyList(), emptyList()));
final QbftBlock sealedBlock = new QbftBlockAdaptor(sealedBesuBlock);
when(blockCreator.createSealedBlock(
qbftExtraDataProvider,
proposedBlock,
roundIdentifier.getRoundNumber(),
List.of(remoteCommitSeal, remoteCommitSeal)))
.thenReturn(sealedBlock);
final int QUORUM_SIZE = 2;
final RoundState roundState = new RoundState(roundIdentifier, QUORUM_SIZE, messageValidator);
final QbftRound round =
@@ -179,6 +202,7 @@ public class QbftRoundIntegrationTest {
transmitter,
roundTimer,
bftExtraDataCodec,
qbftExtraDataProvider,
parentHeader);
// inject a block first, then a prepare on it.
@@ -207,6 +231,6 @@ public class QbftRoundIntegrationTest {
assertThat(roundState.isCommitted()).isTrue();
verifyNoInteractions(multicaster);
verify(blockImporter).importBlock(any(), any(), any());
verify(blockImporter).importBlock(any());
}
}

View File

@@ -14,9 +14,9 @@
*/
package org.hyperledger.besu.consensus.qbft.core.messagedata;
import org.hyperledger.besu.consensus.common.bft.BftExtraDataCodec;
import org.hyperledger.besu.consensus.common.bft.messagedata.AbstractBftMessageData;
import org.hyperledger.besu.consensus.qbft.core.messagewrappers.Proposal;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockCodec;
import org.hyperledger.besu.ethereum.p2p.rlpx.wire.MessageData;
import org.apache.tuweni.bytes.Bytes;
@@ -44,11 +44,11 @@ public class ProposalMessageData extends AbstractBftMessageData {
/**
* Decode.
*
* @param bftExtraDataCodec the bft extra data codec
* @param blockEncoder the qbft block encoder
* @return the proposal
*/
public Proposal decode(final BftExtraDataCodec bftExtraDataCodec) {
return Proposal.decode(data, bftExtraDataCodec);
public Proposal decode(final QbftBlockCodec blockEncoder) {
return Proposal.decode(data, blockEncoder);
}
/**

View File

@@ -14,9 +14,9 @@
*/
package org.hyperledger.besu.consensus.qbft.core.messagedata;
import org.hyperledger.besu.consensus.common.bft.BftExtraDataCodec;
import org.hyperledger.besu.consensus.common.bft.messagedata.AbstractBftMessageData;
import org.hyperledger.besu.consensus.qbft.core.messagewrappers.RoundChange;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockCodec;
import org.hyperledger.besu.ethereum.p2p.rlpx.wire.MessageData;
import org.apache.tuweni.bytes.Bytes;
@@ -44,11 +44,11 @@ public class RoundChangeMessageData extends AbstractBftMessageData {
/**
* Decode.
*
* @param bftExtraDataCodec the bft extra data codec
* @param blockEncoder the qbft block encoder
* @return the round change
*/
public RoundChange decode(final BftExtraDataCodec bftExtraDataCodec) {
return RoundChange.decode(data, bftExtraDataCodec);
public RoundChange decode(final QbftBlockCodec blockEncoder) {
return RoundChange.decode(data, blockEncoder);
}
/**

View File

@@ -14,13 +14,13 @@
*/
package org.hyperledger.besu.consensus.qbft.core.messagewrappers;
import org.hyperledger.besu.consensus.common.bft.BftExtraDataCodec;
import org.hyperledger.besu.consensus.common.bft.messagewrappers.BftMessage;
import org.hyperledger.besu.consensus.common.bft.payload.SignedData;
import org.hyperledger.besu.consensus.qbft.core.payload.PreparePayload;
import org.hyperledger.besu.consensus.qbft.core.payload.ProposalPayload;
import org.hyperledger.besu.consensus.qbft.core.payload.RoundChangePayload;
import org.hyperledger.besu.ethereum.core.Block;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlock;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockCodec;
import org.hyperledger.besu.ethereum.rlp.BytesValueRLPOutput;
import org.hyperledger.besu.ethereum.rlp.RLP;
import org.hyperledger.besu.ethereum.rlp.RLPInput;
@@ -74,7 +74,7 @@ public class Proposal extends BftMessage<ProposalPayload> {
*
* @return the block
*/
public Block getBlock() {
public QbftBlock getBlock() {
return getPayload().getProposedBlock();
}
@@ -97,14 +97,14 @@ public class Proposal extends BftMessage<ProposalPayload> {
* Decode.
*
* @param data the data
* @param bftExtraDataCodec the bft extra data codec
* @param blockEncoder the qbft block encoder
* @return the proposal
*/
public static Proposal decode(final Bytes data, final BftExtraDataCodec bftExtraDataCodec) {
public static Proposal decode(final Bytes data, final QbftBlockCodec blockEncoder) {
final RLPInput rlpIn = RLP.input(data);
rlpIn.enterList();
final SignedData<ProposalPayload> payload =
readPayload(rlpIn, rlpInput -> ProposalPayload.readFrom(rlpInput, bftExtraDataCodec));
readPayload(rlpIn, rlpInput -> ProposalPayload.readFrom(rlpInput, blockEncoder));
rlpIn.enterList();
final List<SignedData<RoundChangePayload>> roundChanges =

View File

@@ -14,14 +14,14 @@
*/
package org.hyperledger.besu.consensus.qbft.core.messagewrappers;
import org.hyperledger.besu.consensus.common.bft.BftBlockHeaderFunctions;
import org.hyperledger.besu.consensus.common.bft.BftExtraDataCodec;
import org.hyperledger.besu.consensus.common.bft.messagewrappers.BftMessage;
import org.hyperledger.besu.consensus.common.bft.payload.SignedData;
import org.hyperledger.besu.consensus.qbft.core.payload.PreparePayload;
import org.hyperledger.besu.consensus.qbft.core.payload.PreparedRoundMetadata;
import org.hyperledger.besu.consensus.qbft.core.payload.RoundChangePayload;
import org.hyperledger.besu.ethereum.core.Block;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlock;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockCodec;
import org.hyperledger.besu.consensus.qbft.core.types.QbftHashMode;
import org.hyperledger.besu.ethereum.rlp.BytesValueRLPOutput;
import org.hyperledger.besu.ethereum.rlp.RLP;
import org.hyperledger.besu.ethereum.rlp.RLPInput;
@@ -34,7 +34,8 @@ import org.apache.tuweni.bytes.Bytes;
/** The Round change payload message. */
public class RoundChange extends BftMessage<RoundChangePayload> {
private final Optional<Block> proposedBlock;
private final Optional<QbftBlock> proposedBlock;
private final QbftBlockCodec blockEncoder;
private final List<SignedData<PreparePayload>> prepares;
/**
@@ -42,14 +43,17 @@ public class RoundChange extends BftMessage<RoundChangePayload> {
*
* @param payload the payload
* @param proposedBlock the proposed block
* @param blockEncoder the qbft block encoder
* @param prepares the prepares
*/
public RoundChange(
final SignedData<RoundChangePayload> payload,
final Optional<Block> proposedBlock,
final Optional<QbftBlock> proposedBlock,
final QbftBlockCodec blockEncoder,
final List<SignedData<PreparePayload>> prepares) {
super(payload);
this.proposedBlock = proposedBlock;
this.blockEncoder = blockEncoder;
this.prepares = prepares;
}
@@ -58,7 +62,7 @@ public class RoundChange extends BftMessage<RoundChangePayload> {
*
* @return the proposed block
*/
public Optional<Block> getProposedBlock() {
public Optional<QbftBlock> getProposedBlock() {
return proposedBlock;
}
@@ -94,7 +98,7 @@ public class RoundChange extends BftMessage<RoundChangePayload> {
final BytesValueRLPOutput rlpOut = new BytesValueRLPOutput();
rlpOut.startList();
getSignedPayload().writeTo(rlpOut);
proposedBlock.ifPresentOrElse(pb -> pb.writeTo(rlpOut), rlpOut::writeEmptyList);
proposedBlock.ifPresentOrElse(pb -> blockEncoder.writeTo(pb, rlpOut), rlpOut::writeEmptyList);
rlpOut.writeList(prepares, SignedData::writeTo);
rlpOut.endList();
return rlpOut.encoded();
@@ -104,29 +108,27 @@ public class RoundChange extends BftMessage<RoundChangePayload> {
* Decode.
*
* @param data the data
* @param bftExtraDataCodec the bft extra data codec
* @param blockEncoder the qbft block encoder
* @return the round change
*/
public static RoundChange decode(final Bytes data, final BftExtraDataCodec bftExtraDataCodec) {
public static RoundChange decode(final Bytes data, final QbftBlockCodec blockEncoder) {
final RLPInput rlpIn = RLP.input(data);
rlpIn.enterList();
final SignedData<RoundChangePayload> payload = readPayload(rlpIn, RoundChangePayload::readFrom);
final Optional<Block> block;
final Optional<QbftBlock> block;
if (rlpIn.nextIsList() && rlpIn.nextSize() == 0) {
rlpIn.skipNext();
block = Optional.empty();
} else {
block =
Optional.of(
Block.readFrom(rlpIn, BftBlockHeaderFunctions.forCommittedSeal(bftExtraDataCodec)));
block = Optional.of(blockEncoder.readFrom(rlpIn, QbftHashMode.COMMITTED_SEAL));
}
final List<SignedData<PreparePayload>> prepares =
rlpIn.readList(r -> readPayload(r, PreparePayload::readFrom));
rlpIn.leaveList();
return new RoundChange(payload, block, prepares);
return new RoundChange(payload, block, blockEncoder, prepares);
}
}

View File

@@ -14,7 +14,6 @@
*/
package org.hyperledger.besu.consensus.qbft.core.network;
import org.hyperledger.besu.consensus.common.bft.BftExtraDataCodec;
import org.hyperledger.besu.consensus.common.bft.Gossiper;
import org.hyperledger.besu.consensus.common.bft.network.ValidatorMulticaster;
import org.hyperledger.besu.consensus.common.bft.payload.Authored;
@@ -23,6 +22,7 @@ import org.hyperledger.besu.consensus.qbft.core.messagedata.PrepareMessageData;
import org.hyperledger.besu.consensus.qbft.core.messagedata.ProposalMessageData;
import org.hyperledger.besu.consensus.qbft.core.messagedata.QbftV1;
import org.hyperledger.besu.consensus.qbft.core.messagedata.RoundChangeMessageData;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockCodec;
import org.hyperledger.besu.datatypes.Address;
import org.hyperledger.besu.ethereum.p2p.rlpx.wire.Message;
import org.hyperledger.besu.ethereum.p2p.rlpx.wire.MessageData;
@@ -35,18 +35,17 @@ import com.google.common.collect.Lists;
public class QbftGossip implements Gossiper {
private final ValidatorMulticaster multicaster;
private final BftExtraDataCodec bftExtraDataCodec;
private final QbftBlockCodec blockEncoder;
/**
* Constructor that attaches gossip logic to a set of multicaster
*
* @param multicaster Network connections to the remote validators
* @param bftExtraDataCodec Codec used when decoding MessageData
* @param blockEncoder the block encoder
*/
public QbftGossip(
final ValidatorMulticaster multicaster, final BftExtraDataCodec bftExtraDataCodec) {
public QbftGossip(final ValidatorMulticaster multicaster, final QbftBlockCodec blockEncoder) {
this.multicaster = multicaster;
this.bftExtraDataCodec = bftExtraDataCodec;
this.blockEncoder = blockEncoder;
}
/**
@@ -57,25 +56,18 @@ public class QbftGossip implements Gossiper {
@Override
public void send(final Message message) {
final MessageData messageData = message.getData();
final Authored decodedMessage;
switch (messageData.getCode()) {
case QbftV1.PROPOSAL:
decodedMessage = ProposalMessageData.fromMessageData(messageData).decode(bftExtraDataCodec);
break;
case QbftV1.PREPARE:
decodedMessage = PrepareMessageData.fromMessageData(messageData).decode();
break;
case QbftV1.COMMIT:
decodedMessage = CommitMessageData.fromMessageData(messageData).decode();
break;
case QbftV1.ROUND_CHANGE:
decodedMessage =
RoundChangeMessageData.fromMessageData(messageData).decode(bftExtraDataCodec);
break;
default:
throw new IllegalArgumentException(
"Received message does not conform to any recognised QBFT message structure.");
}
final Authored decodedMessage =
switch (messageData.getCode()) {
case QbftV1.PROPOSAL ->
ProposalMessageData.fromMessageData(messageData).decode(blockEncoder);
case QbftV1.PREPARE -> PrepareMessageData.fromMessageData(messageData).decode();
case QbftV1.COMMIT -> CommitMessageData.fromMessageData(messageData).decode();
case QbftV1.ROUND_CHANGE ->
RoundChangeMessageData.fromMessageData(messageData).decode(blockEncoder);
default ->
throw new IllegalArgumentException(
"Received message does not conform to any recognised QBFT message structure.");
};
final List<Address> excludeAddressesList =
Lists.newArrayList(
message.getConnection().getPeerInfo().getAddress(), decodedMessage.getAuthor());

View File

@@ -29,9 +29,9 @@ import org.hyperledger.besu.consensus.qbft.core.payload.MessageFactory;
import org.hyperledger.besu.consensus.qbft.core.payload.PreparePayload;
import org.hyperledger.besu.consensus.qbft.core.payload.RoundChangePayload;
import org.hyperledger.besu.consensus.qbft.core.statemachine.PreparedCertificate;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlock;
import org.hyperledger.besu.crypto.SECPSignature;
import org.hyperledger.besu.datatypes.Hash;
import org.hyperledger.besu.ethereum.core.Block;
import org.hyperledger.besu.plugin.services.securitymodule.SecurityModuleException;
import java.util.List;
@@ -70,7 +70,7 @@ public class QbftMessageTransmitter {
*/
public void multicastProposal(
final ConsensusRoundIdentifier roundIdentifier,
final Block block,
final QbftBlock block,
final List<SignedData<RoundChangePayload>> roundChanges,
final List<SignedData<PreparePayload>> prepares) {
try {

View File

@@ -22,10 +22,11 @@ import org.hyperledger.besu.consensus.qbft.core.messagewrappers.Prepare;
import org.hyperledger.besu.consensus.qbft.core.messagewrappers.Proposal;
import org.hyperledger.besu.consensus.qbft.core.messagewrappers.RoundChange;
import org.hyperledger.besu.consensus.qbft.core.statemachine.PreparedCertificate;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlock;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockCodec;
import org.hyperledger.besu.crypto.SECPSignature;
import org.hyperledger.besu.cryptoservices.NodeKey;
import org.hyperledger.besu.datatypes.Hash;
import org.hyperledger.besu.ethereum.core.Block;
import java.util.Collections;
import java.util.List;
@@ -35,14 +36,17 @@ import java.util.Optional;
public class MessageFactory {
private final NodeKey nodeKey;
private final QbftBlockCodec blockEncoder;
/**
* Instantiates a new Message factory.
*
* @param nodeKey the node key
* @param blockEncoder the block encoder
*/
public MessageFactory(final NodeKey nodeKey) {
public MessageFactory(final NodeKey nodeKey, final QbftBlockCodec blockEncoder) {
this.nodeKey = nodeKey;
this.blockEncoder = blockEncoder;
}
/**
@@ -56,11 +60,11 @@ public class MessageFactory {
*/
public Proposal createProposal(
final ConsensusRoundIdentifier roundIdentifier,
final Block block,
final QbftBlock block,
final List<SignedData<RoundChangePayload>> roundChanges,
final List<SignedData<PreparePayload>> prepares) {
final ProposalPayload payload = new ProposalPayload(roundIdentifier, block);
final ProposalPayload payload = new ProposalPayload(roundIdentifier, block, blockEncoder);
return new Proposal(createSignedMessage(payload), roundChanges, prepares);
}
@@ -107,7 +111,7 @@ public class MessageFactory {
final RoundChangePayload payload;
if (preparedRoundData.isPresent()) {
final Block preparedBlock = preparedRoundData.get().getBlock();
final QbftBlock preparedBlock = preparedRoundData.get().getBlock();
payload =
new RoundChangePayload(
roundIdentifier,
@@ -118,12 +122,13 @@ public class MessageFactory {
return new RoundChange(
createSignedMessage(payload),
Optional.of(preparedBlock),
blockEncoder,
preparedRoundData.get().getPrepares());
} else {
payload = new RoundChangePayload(roundIdentifier, Optional.empty());
return new RoundChange(
createSignedMessage(payload), Optional.empty(), Collections.emptyList());
createSignedMessage(payload), Optional.empty(), blockEncoder, Collections.emptyList());
}
}

View File

@@ -14,11 +14,11 @@
*/
package org.hyperledger.besu.consensus.qbft.core.payload;
import org.hyperledger.besu.consensus.common.bft.BftBlockHeaderFunctions;
import org.hyperledger.besu.consensus.common.bft.BftExtraDataCodec;
import org.hyperledger.besu.consensus.common.bft.ConsensusRoundIdentifier;
import org.hyperledger.besu.consensus.qbft.core.messagedata.QbftV1;
import org.hyperledger.besu.ethereum.core.Block;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlock;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockCodec;
import org.hyperledger.besu.consensus.qbft.core.types.QbftHashMode;
import org.hyperledger.besu.ethereum.rlp.RLPInput;
import org.hyperledger.besu.ethereum.rlp.RLPOutput;
@@ -31,43 +31,47 @@ public class ProposalPayload extends QbftPayload {
private static final int TYPE = QbftV1.PROPOSAL;
private final ConsensusRoundIdentifier roundIdentifier;
private final Block proposedBlock;
private final QbftBlock proposedBlock;
private final QbftBlockCodec blockEncoder;
/**
* Instantiates a new Proposal payload.
*
* @param roundIdentifier the round identifier
* @param proposedBlock the proposed block
* @param blockEncoder the qbft block encoder
*/
public ProposalPayload(
final ConsensusRoundIdentifier roundIdentifier, final Block proposedBlock) {
final ConsensusRoundIdentifier roundIdentifier,
final QbftBlock proposedBlock,
final QbftBlockCodec blockEncoder) {
this.roundIdentifier = roundIdentifier;
this.proposedBlock = proposedBlock;
this.blockEncoder = blockEncoder;
}
/**
* Read from rlp input and return proposal payload.
*
* @param rlpInput the rlp input
* @param bftExtraDataCodec the bft extra data codec
* @param blockEncoder the qbft block encoder
* @return the proposal payload
*/
public static ProposalPayload readFrom(
final RLPInput rlpInput, final BftExtraDataCodec bftExtraDataCodec) {
final RLPInput rlpInput, final QbftBlockCodec blockEncoder) {
rlpInput.enterList();
final ConsensusRoundIdentifier roundIdentifier = readConsensusRound(rlpInput);
final Block proposedBlock =
Block.readFrom(rlpInput, BftBlockHeaderFunctions.forCommittedSeal(bftExtraDataCodec));
final QbftBlock proposedBlock = blockEncoder.readFrom(rlpInput, QbftHashMode.COMMITTED_SEAL);
rlpInput.leaveList();
return new ProposalPayload(roundIdentifier, proposedBlock);
return new ProposalPayload(roundIdentifier, proposedBlock, blockEncoder);
}
@Override
public void writeTo(final RLPOutput rlpOutput) {
rlpOutput.startList();
writeConsensusRound(rlpOutput);
proposedBlock.writeTo(rlpOutput);
blockEncoder.writeTo(proposedBlock, rlpOutput);
rlpOutput.endList();
}
@@ -76,7 +80,7 @@ public class ProposalPayload extends QbftPayload {
*
* @return the proposed block
*/
public Block getProposedBlock() {
public QbftBlock getProposedBlock() {
return proposedBlock;
}

View File

@@ -16,14 +16,14 @@ package org.hyperledger.besu.consensus.qbft.core.statemachine;
import org.hyperledger.besu.consensus.common.bft.payload.SignedData;
import org.hyperledger.besu.consensus.qbft.core.payload.PreparePayload;
import org.hyperledger.besu.ethereum.core.Block;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlock;
import java.util.List;
/** The Prepared certificate. */
public class PreparedCertificate {
private final Block block;
private final QbftBlock block;
private final List<SignedData<PreparePayload>> prepares;
private final int round;
@@ -35,7 +35,7 @@ public class PreparedCertificate {
* @param round the round
*/
public PreparedCertificate(
final Block block, final List<SignedData<PreparePayload>> prepares, final int round) {
final QbftBlock block, final List<SignedData<PreparePayload>> prepares, final int round) {
this.block = block;
this.prepares = prepares;
this.round = round;
@@ -55,7 +55,7 @@ public class PreparedCertificate {
*
* @return the block
*/
public Block getBlock() {
public QbftBlock getBlock() {
return block;
}

View File

@@ -18,17 +18,17 @@ import org.hyperledger.besu.consensus.common.bft.ConsensusRoundIdentifier;
import org.hyperledger.besu.consensus.common.bft.events.RoundExpiry;
import org.hyperledger.besu.consensus.common.bft.messagewrappers.BftMessage;
import org.hyperledger.besu.consensus.common.bft.payload.Payload;
import org.hyperledger.besu.consensus.common.bft.statemachine.BftFinalState;
import org.hyperledger.besu.consensus.qbft.core.messagewrappers.Commit;
import org.hyperledger.besu.consensus.qbft.core.messagewrappers.Prepare;
import org.hyperledger.besu.consensus.qbft.core.messagewrappers.Proposal;
import org.hyperledger.besu.consensus.qbft.core.messagewrappers.RoundChange;
import org.hyperledger.besu.consensus.qbft.core.network.QbftMessageTransmitter;
import org.hyperledger.besu.consensus.qbft.core.payload.MessageFactory;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlock;
import org.hyperledger.besu.consensus.qbft.core.types.QbftFinalState;
import org.hyperledger.besu.consensus.qbft.core.validation.FutureRoundProposalMessageValidator;
import org.hyperledger.besu.consensus.qbft.core.validation.MessageValidatorFactory;
import org.hyperledger.besu.datatypes.Address;
import org.hyperledger.besu.ethereum.core.Block;
import org.hyperledger.besu.ethereum.core.BlockHeader;
import org.hyperledger.besu.plugin.services.securitymodule.SecurityModuleException;
@@ -64,7 +64,7 @@ public class QbftBlockHeightManager implements BaseQbftBlockHeightManager {
private final FutureRoundProposalMessageValidator futureRoundProposalMessageValidator;
private final Clock clock;
private final Function<ConsensusRoundIdentifier, RoundState> roundStateCreator;
private final BftFinalState finalState;
private final QbftFinalState finalState;
private Optional<PreparedCertificate> latestPreparedCertificate = Optional.empty();
private Optional<QbftRound> currentRound = Optional.empty();
@@ -83,7 +83,7 @@ public class QbftBlockHeightManager implements BaseQbftBlockHeightManager {
*/
public QbftBlockHeightManager(
final BlockHeader parentHeader,
final BftFinalState finalState,
final QbftFinalState finalState,
final RoundChangeManager roundChangeManager,
final QbftRoundFactory qbftRoundFactory,
final Clock clock,
@@ -131,7 +131,7 @@ public class QbftBlockHeightManager implements BaseQbftBlockHeightManager {
*/
public QbftBlockHeightManager(
final BlockHeader parentHeader,
final BftFinalState finalState,
final QbftFinalState finalState,
final RoundChangeManager roundChangeManager,
final QbftRoundFactory qbftRoundFactory,
final Clock clock,
@@ -189,14 +189,12 @@ public class QbftBlockHeightManager implements BaseQbftBlockHeightManager {
}
final long headerTimeStampSeconds = Math.round(clock.millis() / 1000D);
final Block block = qbftRound.createBlock(headerTimeStampSeconds);
final boolean blockHasTransactions = !block.getBody().getTransactions().isEmpty();
if (blockHasTransactions) {
final QbftBlock block = qbftRound.createBlock(headerTimeStampSeconds);
if (!block.isEmpty()) {
LOG.trace(
"Block has transactions and this node is a proposer so it will send a proposal: "
"Block is not empty and this node is a proposer so it will send a proposal: "
+ roundIdentifier);
qbftRound.updateStateWithProposalAndTransmit(block);
} else {
// handle the block times period
final long currentTimeInMillis = finalState.getClock().millis();

View File

@@ -15,8 +15,8 @@
package org.hyperledger.besu.consensus.qbft.core.statemachine;
import org.hyperledger.besu.consensus.common.bft.BftHelpers;
import org.hyperledger.besu.consensus.common.bft.statemachine.BftFinalState;
import org.hyperledger.besu.consensus.qbft.core.payload.MessageFactory;
import org.hyperledger.besu.consensus.qbft.core.types.QbftFinalState;
import org.hyperledger.besu.consensus.qbft.core.validation.MessageValidatorFactory;
import org.hyperledger.besu.consensus.qbft.core.validator.ValidatorModeTransitionLogger;
import org.hyperledger.besu.ethereum.core.BlockHeader;
@@ -30,7 +30,7 @@ public class QbftBlockHeightManagerFactory {
private static final Logger LOG = LoggerFactory.getLogger(QbftBlockHeightManagerFactory.class);
private final QbftRoundFactory roundFactory;
private final BftFinalState finalState;
private final QbftFinalState finalState;
private final MessageValidatorFactory messageValidatorFactory;
private final MessageFactory messageFactory;
private final ValidatorModeTransitionLogger validatorModeTransitionLogger;
@@ -46,7 +46,7 @@ public class QbftBlockHeightManagerFactory {
* @param validatorModeTransitionLogger the validator mode transition logger
*/
public QbftBlockHeightManagerFactory(
final BftFinalState finalState,
final QbftFinalState finalState,
final QbftRoundFactory roundFactory,
final MessageValidatorFactory messageValidatorFactory,
final MessageFactory messageFactory,

View File

@@ -1,5 +1,5 @@
/*
* Copyright ConsenSys AG.
* Copyright contributors to Besu.
*
* Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
@@ -14,73 +14,91 @@
*/
package org.hyperledger.besu.consensus.qbft.core.statemachine;
import org.hyperledger.besu.consensus.common.bft.BftExtraDataCodec;
import org.hyperledger.besu.consensus.common.bft.ConsensusRoundIdentifier;
import org.hyperledger.besu.consensus.common.bft.Gossiper;
import org.hyperledger.besu.consensus.common.bft.MessageTracker;
import org.hyperledger.besu.consensus.common.bft.SynchronizerUpdater;
import org.hyperledger.besu.consensus.common.bft.statemachine.BaseBftController;
import org.hyperledger.besu.consensus.common.bft.statemachine.BaseBlockHeightManager;
import org.hyperledger.besu.consensus.common.bft.statemachine.BftFinalState;
import org.hyperledger.besu.consensus.common.bft.events.BftReceivedMessageEvent;
import org.hyperledger.besu.consensus.common.bft.events.BlockTimerExpiry;
import org.hyperledger.besu.consensus.common.bft.events.NewChainHead;
import org.hyperledger.besu.consensus.common.bft.events.RoundExpiry;
import org.hyperledger.besu.consensus.common.bft.messagewrappers.BftMessage;
import org.hyperledger.besu.consensus.common.bft.payload.Authored;
import org.hyperledger.besu.consensus.common.bft.statemachine.BftEventHandler;
import org.hyperledger.besu.consensus.common.bft.statemachine.FutureMessageBuffer;
import org.hyperledger.besu.consensus.qbft.core.messagedata.CommitMessageData;
import org.hyperledger.besu.consensus.qbft.core.messagedata.PrepareMessageData;
import org.hyperledger.besu.consensus.qbft.core.messagedata.ProposalMessageData;
import org.hyperledger.besu.consensus.qbft.core.messagedata.QbftV1;
import org.hyperledger.besu.consensus.qbft.core.messagedata.RoundChangeMessageData;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockCodec;
import org.hyperledger.besu.consensus.qbft.core.types.QbftFinalState;
import org.hyperledger.besu.ethereum.chain.Blockchain;
import org.hyperledger.besu.ethereum.core.BlockHeader;
import org.hyperledger.besu.ethereum.p2p.rlpx.wire.Message;
import org.hyperledger.besu.ethereum.p2p.rlpx.wire.MessageData;
/** The Qbft controller. */
public class QbftController extends BaseBftController {
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.function.Consumer;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/** The Qbft controller. */
public class QbftController implements BftEventHandler {
private static final Logger LOG = LoggerFactory.getLogger(QbftController.class);
private final Blockchain blockchain;
private final QbftFinalState finalState;
private final FutureMessageBuffer futureMessageBuffer;
private final Gossiper gossiper;
private final MessageTracker duplicateMessageTracker;
private final SynchronizerUpdater synchronizerUpdater;
private final AtomicBoolean started = new AtomicBoolean(false);
private final QbftBlockCodec blockEncoder;
private BaseQbftBlockHeightManager currentHeightManager;
private final QbftBlockHeightManagerFactory qbftBlockHeightManagerFactory;
private final BftExtraDataCodec bftExtraDataCodec;
/**
* Instantiates a new Qbft controller.
*
* @param blockchain the blockchain
* @param bftFinalState the bft final state
* @param finalState the qbft final state
* @param qbftBlockHeightManagerFactory the qbft block height manager factory
* @param gossiper the gossiper
* @param duplicateMessageTracker the duplicate message tracker
* @param futureMessageBuffer the future message buffer
* @param synchronizerUpdater the synchronizer updater
* @param bftExtraDataCodec the bft extra data codec
* @param blockEncoder the block encoder
*/
public QbftController(
final Blockchain blockchain,
final BftFinalState bftFinalState,
final QbftFinalState finalState,
final QbftBlockHeightManagerFactory qbftBlockHeightManagerFactory,
final Gossiper gossiper,
final MessageTracker duplicateMessageTracker,
final FutureMessageBuffer futureMessageBuffer,
final SynchronizerUpdater synchronizerUpdater,
final BftExtraDataCodec bftExtraDataCodec) {
final QbftBlockCodec blockEncoder) {
super(
blockchain,
bftFinalState,
gossiper,
duplicateMessageTracker,
futureMessageBuffer,
synchronizerUpdater);
this.blockchain = blockchain;
this.finalState = finalState;
this.futureMessageBuffer = futureMessageBuffer;
this.gossiper = gossiper;
this.duplicateMessageTracker = duplicateMessageTracker;
this.synchronizerUpdater = synchronizerUpdater;
this.qbftBlockHeightManagerFactory = qbftBlockHeightManagerFactory;
this.bftExtraDataCodec = bftExtraDataCodec;
this.blockEncoder = blockEncoder;
}
@Override
protected void handleMessage(final Message message) {
private void handleMessage(final Message message) {
final MessageData messageData = message.getData();
switch (messageData.getCode()) {
case QbftV1.PROPOSAL:
consumeMessage(
message,
ProposalMessageData.fromMessageData(messageData).decode(bftExtraDataCodec),
ProposalMessageData.fromMessageData(messageData).decode(blockEncoder),
currentHeightManager::handleProposalPayload);
break;
@@ -101,7 +119,7 @@ public class QbftController extends BaseBftController {
case QbftV1.ROUND_CHANGE:
consumeMessage(
message,
RoundChangeMessageData.fromMessageData(messageData).decode(bftExtraDataCodec),
RoundChangeMessageData.fromMessageData(messageData).decode(blockEncoder),
currentHeightManager::handleRoundChangePayload);
break;
@@ -113,13 +131,162 @@ public class QbftController extends BaseBftController {
}
}
@Override
protected void createNewHeightManager(final BlockHeader parentHeader) {
private void createNewHeightManager(final BlockHeader parentHeader) {
currentHeightManager = qbftBlockHeightManagerFactory.create(parentHeader);
}
@Override
protected BaseBlockHeightManager getCurrentHeightManager() {
private BaseQbftBlockHeightManager getCurrentHeightManager() {
return currentHeightManager;
}
@Override
public void start() {
if (started.compareAndSet(false, true)) {
startNewHeightManager(blockchain.getChainHeadHeader());
}
}
@Override
public void handleMessageEvent(final BftReceivedMessageEvent msg) {
final MessageData data = msg.getMessage().getData();
if (!duplicateMessageTracker.hasSeenMessage(data)) {
duplicateMessageTracker.addSeenMessage(data);
handleMessage(msg.getMessage());
} else {
LOG.trace("Discarded duplicate message");
}
}
/**
* Consume message.
*
* @param <P> the type parameter of BftMessage
* @param message the message
* @param bftMessage the bft message
* @param handleMessage the handle message
*/
protected <P extends BftMessage<?>> void consumeMessage(
final Message message, final P bftMessage, final Consumer<P> handleMessage) {
LOG.trace("Received BFT {} message", bftMessage.getClass().getSimpleName());
// Discard all messages which target the BLOCKCHAIN height (which SHOULD be 1 less than
// the currentHeightManager, but CAN be the same directly following import).
if (bftMessage.getRoundIdentifier().getSequenceNumber()
<= blockchain.getChainHeadBlockNumber()) {
LOG.debug(
"Discarding a message which targets a height {} not above current chain height {}.",
bftMessage.getRoundIdentifier().getSequenceNumber(),
blockchain.getChainHeadBlockNumber());
return;
}
if (processMessage(bftMessage, message)) {
gossiper.send(message);
handleMessage.accept(bftMessage);
}
}
@Override
public void handleNewBlockEvent(final NewChainHead newChainHead) {
final BlockHeader newBlockHeader = newChainHead.getNewChainHeadHeader();
final BlockHeader currentMiningParent = getCurrentHeightManager().getParentBlockHeader();
LOG.debug(
"New chain head detected (block number={})," + " currently mining on top of {}.",
newBlockHeader.getNumber(),
currentMiningParent.getNumber());
if (newBlockHeader.getNumber() < currentMiningParent.getNumber()) {
LOG.trace(
"Discarding NewChainHead event, was for previous block height. chainHeight={} eventHeight={}",
currentMiningParent.getNumber(),
newBlockHeader.getNumber());
return;
}
if (newBlockHeader.getNumber() == currentMiningParent.getNumber()) {
if (newBlockHeader.getHash().equals(currentMiningParent.getHash())) {
LOG.trace(
"Discarding duplicate NewChainHead event. chainHeight={} newBlockHash={} parentBlockHash={}",
newBlockHeader.getNumber(),
newBlockHeader.getHash(),
currentMiningParent.getHash());
} else {
LOG.error(
"Subsequent NewChainHead event at same block height indicates chain fork. chainHeight={}",
currentMiningParent.getNumber());
}
return;
}
startNewHeightManager(newBlockHeader);
}
@Override
public void handleBlockTimerExpiry(final BlockTimerExpiry blockTimerExpiry) {
final ConsensusRoundIdentifier roundIdentifier = blockTimerExpiry.getRoundIdentifier();
if (isMsgForCurrentHeight(roundIdentifier)) {
getCurrentHeightManager().handleBlockTimerExpiry(roundIdentifier);
} else {
LOG.trace(
"Block timer event discarded as it is not for current block height chainHeight={} eventHeight={}",
getCurrentHeightManager().getChainHeight(),
roundIdentifier.getSequenceNumber());
}
}
@Override
public void handleRoundExpiry(final RoundExpiry roundExpiry) {
// Discard all messages which target the BLOCKCHAIN height (which SHOULD be 1 less than
// the currentHeightManager, but CAN be the same directly following import).
if (roundExpiry.getView().getSequenceNumber() <= blockchain.getChainHeadBlockNumber()) {
LOG.debug("Discarding a round-expiry which targets a height not above current chain height.");
return;
}
if (isMsgForCurrentHeight(roundExpiry.getView())) {
getCurrentHeightManager().roundExpired(roundExpiry);
} else {
LOG.trace(
"Round expiry event discarded as it is not for current block height chainHeight={} eventHeight={}",
getCurrentHeightManager().getChainHeight(),
roundExpiry.getView().getSequenceNumber());
}
}
private void startNewHeightManager(final BlockHeader parentHeader) {
createNewHeightManager(parentHeader);
final long newChainHeight = getCurrentHeightManager().getChainHeight();
futureMessageBuffer.retrieveMessagesForHeight(newChainHeight).forEach(this::handleMessage);
}
private boolean processMessage(final BftMessage<?> msg, final Message rawMsg) {
final ConsensusRoundIdentifier msgRoundIdentifier = msg.getRoundIdentifier();
if (isMsgForCurrentHeight(msgRoundIdentifier)) {
return isMsgFromKnownValidator(msg) && finalState.isLocalNodeValidator();
} else if (isMsgForFutureChainHeight(msgRoundIdentifier)) {
LOG.trace("Received message for future block height round={}", msgRoundIdentifier);
futureMessageBuffer.addMessage(msgRoundIdentifier.getSequenceNumber(), rawMsg);
// Notify the synchronizer the transmitting peer must have the parent block to the received
// messages's target height.
synchronizerUpdater.updatePeerChainState(
msgRoundIdentifier.getSequenceNumber() - 1L, rawMsg.getConnection());
} else {
LOG.trace(
"BFT message discarded as it is from a previous block height messageType={} chainHeight={} eventHeight={}",
msg.getMessageType(),
getCurrentHeightManager().getChainHeight(),
msgRoundIdentifier.getSequenceNumber());
}
return false;
}
private boolean isMsgFromKnownValidator(final Authored msg) {
return finalState.getValidators().contains(msg.getAuthor());
}
private boolean isMsgForCurrentHeight(final ConsensusRoundIdentifier roundIdentifier) {
return roundIdentifier.getSequenceNumber() == getCurrentHeightManager().getChainHeight();
}
private boolean isMsgForFutureChainHeight(final ConsensusRoundIdentifier roundIdentifier) {
return roundIdentifier.getSequenceNumber() > getCurrentHeightManager().getChainHeight();
}
}

View File

@@ -17,12 +17,8 @@ package org.hyperledger.besu.consensus.qbft.core.statemachine;
import static java.util.Collections.emptyList;
import org.hyperledger.besu.consensus.common.bft.BftBlockHashing;
import org.hyperledger.besu.consensus.common.bft.BftBlockHeaderFunctions;
import org.hyperledger.besu.consensus.common.bft.BftBlockInterface;
import org.hyperledger.besu.consensus.common.bft.BftContext;
import org.hyperledger.besu.consensus.common.bft.BftExtraData;
import org.hyperledger.besu.consensus.common.bft.BftExtraDataCodec;
import org.hyperledger.besu.consensus.common.bft.BftHelpers;
import org.hyperledger.besu.consensus.common.bft.ConsensusRoundIdentifier;
import org.hyperledger.besu.consensus.common.bft.RoundTimer;
import org.hyperledger.besu.consensus.common.bft.payload.SignedData;
@@ -33,18 +29,20 @@ import org.hyperledger.besu.consensus.qbft.core.network.QbftMessageTransmitter;
import org.hyperledger.besu.consensus.qbft.core.payload.MessageFactory;
import org.hyperledger.besu.consensus.qbft.core.payload.PreparePayload;
import org.hyperledger.besu.consensus.qbft.core.payload.RoundChangePayload;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlock;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockCreator;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockImporter;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockInterface;
import org.hyperledger.besu.consensus.qbft.core.types.QbftContext;
import org.hyperledger.besu.consensus.qbft.core.types.QbftExtraDataProvider;
import org.hyperledger.besu.consensus.qbft.core.types.QbftHashMode;
import org.hyperledger.besu.consensus.qbft.core.types.QbftMinedBlockObserver;
import org.hyperledger.besu.consensus.qbft.core.types.QbftProtocolSchedule;
import org.hyperledger.besu.crypto.SECPSignature;
import org.hyperledger.besu.cryptoservices.NodeKey;
import org.hyperledger.besu.datatypes.Hash;
import org.hyperledger.besu.ethereum.ProtocolContext;
import org.hyperledger.besu.ethereum.blockcreation.BlockCreator;
import org.hyperledger.besu.ethereum.chain.MinedBlockObserver;
import org.hyperledger.besu.ethereum.core.Block;
import org.hyperledger.besu.ethereum.core.BlockHeader;
import org.hyperledger.besu.ethereum.core.BlockImporter;
import org.hyperledger.besu.ethereum.mainnet.BlockImportResult;
import org.hyperledger.besu.ethereum.mainnet.HeaderValidationMode;
import org.hyperledger.besu.ethereum.mainnet.ProtocolSchedule;
import org.hyperledger.besu.plugin.services.securitymodule.SecurityModuleException;
import org.hyperledger.besu.util.Subscribers;
@@ -59,19 +57,19 @@ public class QbftRound {
private static final Logger LOG = LoggerFactory.getLogger(QbftRound.class);
private final Subscribers<MinedBlockObserver> observers;
private final Subscribers<QbftMinedBlockObserver> observers;
/** The Round state. */
protected final RoundState roundState;
/** The Block creator. */
protected final BlockCreator blockCreator;
protected final QbftBlockCreator blockCreator;
/** The Protocol context. */
protected final ProtocolContext protocolContext;
/** The Protocol schedule. */
protected final ProtocolSchedule protocolSchedule;
protected final QbftProtocolSchedule protocolSchedule;
private final NodeKey nodeKey;
private final MessageFactory messageFactory; // used only to create stored local msgs
@@ -80,6 +78,9 @@ public class QbftRound {
/** The Bft extra data codec. */
protected final BftExtraDataCodec bftExtraDataCodec;
/** The Bft extra data provider */
protected final QbftExtraDataProvider qbftExtraDataProvider;
private final BlockHeader parentHeader;
/**
@@ -95,19 +96,21 @@ public class QbftRound {
* @param transmitter the transmitter
* @param roundTimer the round timer
* @param bftExtraDataCodec the bft extra data codec
* @param qbftExtraDataProvider the qbft extra data provider
* @param parentHeader the parent header
*/
public QbftRound(
final RoundState roundState,
final BlockCreator blockCreator,
final QbftBlockCreator blockCreator,
final ProtocolContext protocolContext,
final ProtocolSchedule protocolSchedule,
final Subscribers<MinedBlockObserver> observers,
final QbftProtocolSchedule protocolSchedule,
final Subscribers<QbftMinedBlockObserver> observers,
final NodeKey nodeKey,
final MessageFactory messageFactory,
final QbftMessageTransmitter transmitter,
final RoundTimer roundTimer,
final BftExtraDataCodec bftExtraDataCodec,
final QbftExtraDataProvider qbftExtraDataProvider,
final BlockHeader parentHeader) {
this.roundState = roundState;
this.blockCreator = blockCreator;
@@ -118,6 +121,7 @@ public class QbftRound {
this.messageFactory = messageFactory;
this.transmitter = transmitter;
this.bftExtraDataCodec = bftExtraDataCodec;
this.qbftExtraDataProvider = qbftExtraDataProvider;
this.parentHeader = parentHeader;
roundTimer.startTimer(getRoundIdentifier());
}
@@ -137,9 +141,9 @@ public class QbftRound {
* @param headerTimeStampSeconds of the block
* @return a Block
*/
public Block createBlock(final long headerTimeStampSeconds) {
public QbftBlock createBlock(final long headerTimeStampSeconds) {
LOG.debug("Creating proposed block. round={}", roundState.getRoundIdentifier());
return blockCreator.createBlock(headerTimeStampSeconds, this.parentHeader).getBlock();
return blockCreator.createBlock(headerTimeStampSeconds, this.parentHeader);
}
/**
@@ -153,21 +157,21 @@ public class QbftRound {
final Optional<PreparedCertificate> bestPreparedCertificate =
roundChangeArtifacts.getBestPreparedPeer();
final Block blockToPublish;
final QbftBlock blockToPublish;
if (bestPreparedCertificate.isEmpty()) {
LOG.debug("Sending proposal with new block. round={}", roundState.getRoundIdentifier());
blockToPublish = blockCreator.createBlock(headerTimestamp, this.parentHeader).getBlock();
blockToPublish = blockCreator.createBlock(headerTimestamp, this.parentHeader);
} else {
LOG.debug(
"Sending proposal from PreparedCertificate. round={}", roundState.getRoundIdentifier());
Block preparedBlock = bestPreparedCertificate.get().getBlock();
final BftBlockInterface bftBlockInterface =
protocolContext.getConsensusContext(BftContext.class).getBlockInterface();
QbftBlock preparedBlock = bestPreparedCertificate.get().getBlock();
final QbftBlockInterface bftBlockInterface =
protocolContext.getConsensusContext(QbftContext.class).getBlockInterface();
blockToPublish =
bftBlockInterface.replaceRoundInBlock(
preparedBlock,
roundState.getRoundIdentifier().getRoundNumber(),
BftBlockHeaderFunctions.forCommittedSeal(bftExtraDataCodec));
QbftHashMode.COMMITTED_SEAL);
}
LOG.debug(" proposal - new/prepared block hash : {}", blockToPublish.getHash());
@@ -183,7 +187,7 @@ public class QbftRound {
*
* @param block the block
*/
protected void updateStateWithProposalAndTransmit(final Block block) {
protected void updateStateWithProposalAndTransmit(final QbftBlock block) {
updateStateWithProposalAndTransmit(block, emptyList(), emptyList());
}
@@ -195,7 +199,7 @@ public class QbftRound {
* @param prepares the prepares
*/
protected void updateStateWithProposalAndTransmit(
final Block block,
final QbftBlock block,
final List<SignedData<RoundChangePayload>> roundChanges,
final List<SignedData<PreparePayload>> prepares) {
final Proposal proposal;
@@ -226,13 +230,13 @@ public class QbftRound {
"Received a proposal message. round={}. author={}",
roundState.getRoundIdentifier(),
msg.getAuthor());
final Block block = msg.getSignedPayload().getPayload().getProposedBlock();
final QbftBlock block = msg.getSignedPayload().getPayload().getProposedBlock();
if (updateStateWithProposedBlock(msg)) {
sendPrepare(block);
}
}
private void sendPrepare(final Block block) {
private void sendPrepare(final QbftBlock block) {
LOG.debug("Sending prepare message. round={}", roundState.getRoundIdentifier());
try {
final Prepare localPrepareMessage =
@@ -286,7 +290,7 @@ public class QbftRound {
final boolean blockAccepted = roundState.setProposedBlock(msg);
if (blockAccepted) {
final Block block = roundState.getProposedBlock().get();
final QbftBlock block = roundState.getProposedBlock().get();
final SECPSignature commitSeal;
try {
commitSeal = createCommitSeal(block);
@@ -328,7 +332,7 @@ public class QbftRound {
roundState.addPrepareMessage(msg);
if (wasPrepared != roundState.isPrepared()) {
LOG.debug("Sending commit message. round={}", roundState.getRoundIdentifier());
final Block block = roundState.getProposedBlock().get();
final QbftBlock block = roundState.getProposedBlock().get();
try {
transmitter.multicastCommit(getRoundIdentifier(), block.getHash(), createCommitSeal(block));
// Note: the local-node's commit message was added to RoundState on block acceptance
@@ -349,15 +353,15 @@ public class QbftRound {
private void importBlockToChain() {
final Block blockToImport =
BftHelpers.createSealedBlock(
bftExtraDataCodec,
final QbftBlock blockToImport =
blockCreator.createSealedBlock(
qbftExtraDataProvider,
roundState.getProposedBlock().get(),
roundState.getRoundIdentifier().getRoundNumber(),
roundState.getCommitSeals());
final long blockNumber = blockToImport.getHeader().getNumber();
final BftExtraData extraData = bftExtraDataCodec.decode(blockToImport.getHeader());
final BftExtraData extraData = qbftExtraDataProvider.getExtraData(blockToImport.getHeader());
if (getRoundIdentifier().getRoundNumber() > 0) {
LOG.info(
"Importing proposed block to chain. round={}, hash={}",
@@ -371,11 +375,10 @@ public class QbftRound {
}
LOG.trace("Importing proposed block with extraData={}", extraData);
final BlockImporter blockImporter =
final QbftBlockImporter blockImporter =
protocolSchedule.getByBlockHeader(blockToImport.getHeader()).getBlockImporter();
final BlockImportResult result =
blockImporter.importBlock(protocolContext, blockToImport, HeaderValidationMode.FULL);
if (!result.isImported()) {
final boolean result = blockImporter.importBlock(blockToImport);
if (!result) {
LOG.error(
"Failed to import proposed block to chain. block={} extraData={} blockHeader={}",
blockNumber,
@@ -386,26 +389,24 @@ public class QbftRound {
}
}
private SECPSignature createCommitSeal(final Block block) {
final Block commitBlock = createCommitBlock(block);
private SECPSignature createCommitSeal(final QbftBlock block) {
final QbftBlock commitBlock = createCommitBlock(block);
final BlockHeader proposedHeader = commitBlock.getHeader();
final BftExtraData extraData = bftExtraDataCodec.decode(proposedHeader);
final BftExtraData extraData = qbftExtraDataProvider.getExtraData(proposedHeader);
final Hash commitHash =
new BftBlockHashing(bftExtraDataCodec)
.calculateDataHashForCommittedSeal(proposedHeader, extraData);
return nodeKey.sign(commitHash);
}
private Block createCommitBlock(final Block block) {
final BftBlockInterface bftBlockInterface =
protocolContext.getConsensusContext(BftContext.class).getBlockInterface();
private QbftBlock createCommitBlock(final QbftBlock block) {
final QbftBlockInterface bftBlockInterface =
protocolContext.getConsensusContext(QbftContext.class).getBlockInterface();
return bftBlockInterface.replaceRoundInBlock(
block,
getRoundIdentifier().getRoundNumber(),
BftBlockHeaderFunctions.forCommittedSeal(bftExtraDataCodec));
block, getRoundIdentifier().getRoundNumber(), QbftHashMode.COMMITTED_SEAL);
}
private void notifyNewBlockListeners(final Block block) {
private void notifyNewBlockListeners(final QbftBlock block) {
observers.forEach(obs -> obs.blockMined(block));
}
}

View File

@@ -15,30 +15,32 @@
package org.hyperledger.besu.consensus.qbft.core.statemachine;
import org.hyperledger.besu.consensus.common.bft.BftExtraDataCodec;
import org.hyperledger.besu.consensus.common.bft.BftProtocolSchedule;
import org.hyperledger.besu.consensus.common.bft.ConsensusRoundIdentifier;
import org.hyperledger.besu.consensus.common.bft.blockcreation.BftBlockCreatorFactory;
import org.hyperledger.besu.consensus.common.bft.statemachine.BftFinalState;
import org.hyperledger.besu.consensus.qbft.core.network.QbftMessageTransmitter;
import org.hyperledger.besu.consensus.qbft.core.payload.MessageFactory;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockCreator;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockCreatorFactory;
import org.hyperledger.besu.consensus.qbft.core.types.QbftExtraDataProvider;
import org.hyperledger.besu.consensus.qbft.core.types.QbftFinalState;
import org.hyperledger.besu.consensus.qbft.core.types.QbftMinedBlockObserver;
import org.hyperledger.besu.consensus.qbft.core.types.QbftProtocolSchedule;
import org.hyperledger.besu.consensus.qbft.core.validation.MessageValidatorFactory;
import org.hyperledger.besu.ethereum.ProtocolContext;
import org.hyperledger.besu.ethereum.blockcreation.BlockCreator;
import org.hyperledger.besu.ethereum.chain.MinedBlockObserver;
import org.hyperledger.besu.ethereum.core.BlockHeader;
import org.hyperledger.besu.util.Subscribers;
/** The Qbft round factory. */
public class QbftRoundFactory {
private final BftFinalState finalState;
private final BftBlockCreatorFactory<?> blockCreatorFactory;
private final QbftFinalState finalState;
private final QbftBlockCreatorFactory blockCreatorFactory;
private final ProtocolContext protocolContext;
private final BftProtocolSchedule protocolSchedule;
private final Subscribers<MinedBlockObserver> minedBlockObservers;
private final QbftProtocolSchedule protocolSchedule;
private final Subscribers<QbftMinedBlockObserver> minedBlockObservers;
private final MessageValidatorFactory messageValidatorFactory;
private final MessageFactory messageFactory;
private final BftExtraDataCodec bftExtraDataCodec;
private final QbftExtraDataProvider qbftExtraDataProvider;
/**
* Instantiates a new Qbft round factory.
@@ -50,15 +52,17 @@ public class QbftRoundFactory {
* @param messageValidatorFactory the message validator factory
* @param messageFactory the message factory
* @param bftExtraDataCodec the bft extra data codec
* @param qbftExtraDataProvider the bft extra data codec
*/
public QbftRoundFactory(
final BftFinalState finalState,
final QbftFinalState finalState,
final ProtocolContext protocolContext,
final BftProtocolSchedule protocolSchedule,
final Subscribers<MinedBlockObserver> minedBlockObservers,
final QbftProtocolSchedule protocolSchedule,
final Subscribers<QbftMinedBlockObserver> minedBlockObservers,
final MessageValidatorFactory messageValidatorFactory,
final MessageFactory messageFactory,
final BftExtraDataCodec bftExtraDataCodec) {
final BftExtraDataCodec bftExtraDataCodec,
final QbftExtraDataProvider qbftExtraDataProvider) {
this.finalState = finalState;
this.blockCreatorFactory = finalState.getBlockCreatorFactory();
this.protocolContext = protocolContext;
@@ -67,6 +71,7 @@ public class QbftRoundFactory {
this.messageValidatorFactory = messageValidatorFactory;
this.messageFactory = messageFactory;
this.bftExtraDataCodec = bftExtraDataCodec;
this.qbftExtraDataProvider = qbftExtraDataProvider;
}
/**
@@ -99,7 +104,7 @@ public class QbftRoundFactory {
*/
public QbftRound createNewRoundWithState(
final BlockHeader parentHeader, final RoundState roundState) {
final BlockCreator blockCreator =
final QbftBlockCreator blockCreator =
blockCreatorFactory.create(roundState.getRoundIdentifier().getRoundNumber());
// TODO(tmm): Why is this created everytime?!
@@ -117,6 +122,7 @@ public class QbftRoundFactory {
messageTransmitter,
finalState.getRoundTimer(),
bftExtraDataCodec,
qbftExtraDataProvider,
parentHeader);
}
}

View File

@@ -18,9 +18,9 @@ import org.hyperledger.besu.consensus.common.bft.ConsensusRoundIdentifier;
import org.hyperledger.besu.consensus.qbft.core.messagewrappers.Commit;
import org.hyperledger.besu.consensus.qbft.core.messagewrappers.Prepare;
import org.hyperledger.besu.consensus.qbft.core.messagewrappers.Proposal;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlock;
import org.hyperledger.besu.consensus.qbft.core.validation.MessageValidator;
import org.hyperledger.besu.crypto.SECPSignature;
import org.hyperledger.besu.ethereum.core.Block;
import java.util.Collection;
import java.util.Optional;
@@ -83,7 +83,7 @@ public class RoundState {
*/
public boolean setProposedBlock(final Proposal msg) {
if (!proposalMessage.isPresent()) {
if (proposalMessage.isEmpty()) {
if (validator.validateProposal(msg)) {
proposalMessage = Optional.of(msg);
prepareMessages.removeIf(p -> !validator.validatePrepare(p));
@@ -102,7 +102,7 @@ public class RoundState {
* @param msg the msg
*/
public void addPrepareMessage(final Prepare msg) {
if (!proposalMessage.isPresent() || validator.validatePrepare(msg)) {
if (proposalMessage.isEmpty() || validator.validatePrepare(msg)) {
prepareMessages.add(msg);
LOG.trace("Round state added prepare message prepare={}", msg);
}
@@ -115,7 +115,7 @@ public class RoundState {
* @param msg the msg
*/
public void addCommitMessage(final Commit msg) {
if (!proposalMessage.isPresent() || validator.validateCommit(msg)) {
if (proposalMessage.isEmpty() || validator.validateCommit(msg)) {
commitMessages.add(msg);
LOG.trace("Round state added commit message commit={}", msg);
}
@@ -141,7 +141,7 @@ public class RoundState {
*
* @return the proposed block
*/
public Optional<Block> getProposedBlock() {
public Optional<QbftBlock> getProposedBlock() {
return proposalMessage.map(p -> p.getSignedPayload().getPayload().getProposedBlock());
}

View File

@@ -0,0 +1,46 @@
/*
* Copyright contributors to Besu.
*
* Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
* an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
* specific language governing permissions and limitations under the License.
*
* SPDX-License-Identifier: Apache-2.0
*/
package org.hyperledger.besu.consensus.qbft.core.types;
import org.hyperledger.besu.datatypes.Hash;
import org.hyperledger.besu.ethereum.core.BlockHeader;
/** Represents a block in the context of the QBFT consensus mechanism. */
public interface QbftBlock {
/**
* Returns the block header of the block.
*
* @return the block header.
*/
BlockHeader getHeader();
/**
* Whether the block is considered empty, generally this means that the block has no transactions.
*
* @return true if the block is empty, false otherwise.
*/
boolean isEmpty();
/**
* Returns the hash of the header. A convenience method to avoid having to call
* getHeader().getHash().
*
* @return the hash of the block.
*/
default Hash getHash() {
return getHeader().getHash();
}
}

View File

@@ -0,0 +1,38 @@
/*
* Copyright contributors to Besu.
*
* Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
* an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
* specific language governing permissions and limitations under the License.
*
* SPDX-License-Identifier: Apache-2.0
*/
package org.hyperledger.besu.consensus.qbft.core.types;
import org.hyperledger.besu.ethereum.rlp.RLPInput;
import org.hyperledger.besu.ethereum.rlp.RLPOutput;
/** Block encoding and decoding to and from RLP */
public interface QbftBlockCodec {
/**
* Read a block from RLP
*
* @param rlpInput RLP input
* @param hashMode Hash mode to ensure the block hash is calculated correctly
* @return The block
*/
QbftBlock readFrom(RLPInput rlpInput, QbftHashMode hashMode);
/**
* Write a block to RLP
*
* @param block The block to write
* @param rlpOutput RLP output to write the block to
*/
void writeTo(QbftBlock block, RLPOutput rlpOutput);
}

View File

@@ -0,0 +1,48 @@
/*
* Copyright contributors to Besu.
*
* Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
* an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
* specific language governing permissions and limitations under the License.
*
* SPDX-License-Identifier: Apache-2.0
*/
package org.hyperledger.besu.consensus.qbft.core.types;
import org.hyperledger.besu.crypto.SECPSignature;
import org.hyperledger.besu.ethereum.core.BlockHeader;
import java.util.Collection;
/** Responsible for creating a block. */
public interface QbftBlockCreator {
/**
* Create a block.
*
* @param headerTimeStampSeconds the header timestamp
* @param parentHeader the parent header
* @return the block
*/
QbftBlock createBlock(long headerTimeStampSeconds, BlockHeader parentHeader);
/**
* Create sealed block.
*
* @param qbftExtraDataProvider the extra data provider
* @param block the block
* @param roundNumber the round number
* @param commitSeals the commit seals
* @return the block
*/
QbftBlock createSealedBlock(
final QbftExtraDataProvider qbftExtraDataProvider,
final QbftBlock block,
final int roundNumber,
final Collection<SECPSignature> commitSeals);
}

View File

@@ -0,0 +1,27 @@
/*
* Copyright contributors to Besu.
*
* Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
* an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
* specific language governing permissions and limitations under the License.
*
* SPDX-License-Identifier: Apache-2.0
*/
package org.hyperledger.besu.consensus.qbft.core.types;
/** Factory for creating a {@link QbftBlockCreator} for a specific round. */
public interface QbftBlockCreatorFactory {
/**
* Create a {@link QbftBlockCreator} for the specified round.
*
* @param roundNumber The round number for which to create a block creator.
* @return A block creator for the specified round.
*/
QbftBlockCreator create(int roundNumber);
}

View File

@@ -0,0 +1,27 @@
/*
* Copyright contributors to Besu.
*
* Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
* an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
* specific language governing permissions and limitations under the License.
*
* SPDX-License-Identifier: Apache-2.0
*/
package org.hyperledger.besu.consensus.qbft.core.types;
/** Imports a block into the chain. */
public interface QbftBlockImporter {
/**
* Import a block into the chain.
*
* @param block to import
* @return true if the block was successfully imported, false otherwise
*/
boolean importBlock(QbftBlock block);
}

View File

@@ -0,0 +1,29 @@
/*
* Copyright ConsenSys AG.
*
* 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.consensus.qbft.core.types;
/** Utility functions for QBFT blocks */
public interface QbftBlockInterface {
/**
* Create a new block using the supplied block with round number replaced.
*
* @param proposalBlock the proposal block
* @param roundNumber the round number
* @param hashMode the hash mode
* @return the new qbft block with updated round number
*/
QbftBlock replaceRoundInBlock(QbftBlock proposalBlock, int roundNumber, QbftHashMode hashMode);
}

View File

@@ -0,0 +1,37 @@
/*
* Copyright contributors to Besu.
*
* Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
* an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
* specific language governing permissions and limitations under the License.
*
* SPDX-License-Identifier: Apache-2.0
*/
package org.hyperledger.besu.consensus.qbft.core.types;
import java.util.Optional;
/** Validates a block. */
public interface QbftBlockValidator {
/**
* Validates a block.
*
* @param block the block to validate
* @return the validation result
*/
ValidationResult validateBlock(QbftBlock block);
/**
* The result of a block validation.
*
* @param success whether the validation was successful
* @param errorMessage the error message if the validation was not successful
*/
record ValidationResult(boolean success, Optional<String> errorMessage) {}
}

View File

@@ -0,0 +1,60 @@
/*
* Copyright ConsenSys AG.
*
* 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.consensus.qbft.core.types;
import org.hyperledger.besu.consensus.common.validator.ValidatorProvider;
import org.hyperledger.besu.ethereum.ConsensusContext;
/** Holds the QBFT specific mutable state. */
public class QbftContext implements ConsensusContext {
private final ValidatorProvider validatorProvider;
private final QbftBlockInterface blockInterface;
/**
* Instantiates a new Bft context.
*
* @param validatorProvider the validator provider
* @param blockInterface the block interface
*/
public QbftContext(
final ValidatorProvider validatorProvider, final QbftBlockInterface blockInterface) {
this.validatorProvider = validatorProvider;
this.blockInterface = blockInterface;
}
/**
* Gets validator provider.
*
* @return the validator provider
*/
public ValidatorProvider getValidatorProvider() {
return validatorProvider;
}
/**
* Gets block interface.
*
* @return the block interface
*/
public QbftBlockInterface getBlockInterface() {
return blockInterface;
}
@Override
public <C extends ConsensusContext> C as(final Class<C> klass) {
return klass.cast(this);
}
}

View File

@@ -0,0 +1,30 @@
/*
* Copyright contributors to Besu.
*
* Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
* an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
* specific language governing permissions and limitations under the License.
*
* SPDX-License-Identifier: Apache-2.0
*/
package org.hyperledger.besu.consensus.qbft.core.types;
import org.hyperledger.besu.consensus.common.bft.BftExtraData;
import org.hyperledger.besu.ethereum.core.BlockHeader;
/** Provides the extra data for a block. */
public interface QbftExtraDataProvider {
/**
* Returns the extra data for the supplied block header.
*
* @param header the block header to retrieve the extra data from
* @return the extra data
*/
BftExtraData getExtraData(BlockHeader header);
}

View File

@@ -0,0 +1,107 @@
/*
* Copyright contributors to Besu.
*
* Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
* an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
* specific language governing permissions and limitations under the License.
*
* SPDX-License-Identifier: Apache-2.0
*/
package org.hyperledger.besu.consensus.qbft.core.types;
import org.hyperledger.besu.consensus.common.bft.BlockTimer;
import org.hyperledger.besu.consensus.common.bft.ConsensusRoundIdentifier;
import org.hyperledger.besu.consensus.common.bft.RoundTimer;
import org.hyperledger.besu.consensus.common.bft.network.ValidatorMulticaster;
import org.hyperledger.besu.cryptoservices.NodeKey;
import org.hyperledger.besu.datatypes.Address;
import java.time.Clock;
import java.util.Collection;
/** This is the full data set, or context, required for many of the aspects of QBFT workflows. */
public interface QbftFinalState {
/**
* Gets validator multicaster.
*
* @return the validator multicaster
*/
ValidatorMulticaster getValidatorMulticaster();
/**
* Gets node key.
*
* @return the node key
*/
NodeKey getNodeKey();
/**
* Gets round timer.
*
* @return the round timer
*/
RoundTimer getRoundTimer();
/**
* Is local node validator.
*
* @return true if the local node is a validator, false otherwise
*/
boolean isLocalNodeValidator();
/**
* Gets validators.
*
* @return the validators
*/
Collection<Address> getValidators();
/**
* Gets local address.
*
* @return the local address
*/
Address getLocalAddress();
/**
* Gets clock.
*
* @return the clock
*/
Clock getClock();
/**
* Gets block creator factory.
*
* @return the block creator factory
*/
QbftBlockCreatorFactory getBlockCreatorFactory();
/**
* Gets quorum.
*
* @return the quorum
*/
int getQuorum();
/**
* Gets block timer.
*
* @return the block timer
*/
BlockTimer getBlockTimer();
/**
* Is local node proposer for round.
*
* @param roundIdentifier the round identifier
* @return true if the local node is the proposer for the given round, false otherwise
*/
boolean isLocalNodeProposerForRound(ConsensusRoundIdentifier roundIdentifier);
}

View File

@@ -0,0 +1,35 @@
/*
* Copyright contributors to Besu.
*
* Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
* an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
* specific language governing permissions and limitations under the License.
*
* SPDX-License-Identifier: Apache-2.0
*/
package org.hyperledger.besu.consensus.qbft.core.types;
/**
* The mode in which the block hash is calculated for a QBFT block.
*
* <p>When a block is hashed, the hash may be calculated in different ways depending on the context
* in which the hash is being used.
*/
public enum QbftHashMode {
/**
* Hash the block for the committed seal. This typically means the block hash excludes the commit
* seal from the hashing.
*/
COMMITTED_SEAL,
/**
* Hash the block for onchain block. This typically means the block hash exclude the commit seals
* and round number from the hashing as each node may have a different value for these fields.
*/
ONCHAIN
}

View File

@@ -0,0 +1,25 @@
/*
* Copyright contributors to Besu.
*
* Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
* an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
* specific language governing permissions and limitations under the License.
*
* SPDX-License-Identifier: Apache-2.0
*/
package org.hyperledger.besu.consensus.qbft.core.types;
/** Observer for mined blocks in QBFT */
public interface QbftMinedBlockObserver {
/**
* Called when a block is mined
*
* @param block the mined block
*/
void blockMined(QbftBlock block);
}

View File

@@ -0,0 +1,32 @@
/*
* Copyright contributors to Besu.
*
* Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
* an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
* specific language governing permissions and limitations under the License.
*
* SPDX-License-Identifier: Apache-2.0
*/
package org.hyperledger.besu.consensus.qbft.core.types;
import org.hyperledger.besu.ethereum.core.BlockHeader;
/**
* Provides the ability to select the appropriate QbftProtocolSpec containing the validation and
* import for the supplied block header.
*/
public interface QbftProtocolSchedule {
/**
* Returns the QbftProtocolSpec for the supplied block header.
*
* @param header The block header to select the appropriate QbftProtocolSpec for
* @return The QbftProtocolSpec for the supplied block header
*/
QbftProtocolSpec getByBlockHeader(BlockHeader header);
}

View File

@@ -0,0 +1,35 @@
/*
* Copyright contributors to Besu.
*
* Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
* an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
* specific language governing permissions and limitations under the License.
*
* SPDX-License-Identifier: Apache-2.0
*/
package org.hyperledger.besu.consensus.qbft.core.types;
/**
* Provides the ability to select the appropriate QbftProtocolSpec containing the validation and
* import for the supplied block header.
*/
public interface QbftProtocolSpec {
/**
* Gets the block importer.
*
* @return the block importer
*/
QbftBlockImporter getBlockImporter();
/**
* Gets the block validator.
*
* @return the block validator
*/
QbftBlockValidator getBlockValidator();
}

View File

@@ -14,15 +14,14 @@
*/
package org.hyperledger.besu.consensus.qbft.core.validation;
import org.hyperledger.besu.consensus.common.bft.BftBlockHeaderFunctions;
import org.hyperledger.besu.consensus.common.bft.BftBlockInterface;
import org.hyperledger.besu.consensus.common.bft.BftExtraDataCodec;
import org.hyperledger.besu.consensus.common.bft.ConsensusRoundIdentifier;
import org.hyperledger.besu.consensus.qbft.core.messagewrappers.Commit;
import org.hyperledger.besu.consensus.qbft.core.messagewrappers.Prepare;
import org.hyperledger.besu.consensus.qbft.core.messagewrappers.Proposal;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlock;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockInterface;
import org.hyperledger.besu.consensus.qbft.core.types.QbftHashMode;
import org.hyperledger.besu.datatypes.Address;
import org.hyperledger.besu.ethereum.core.Block;
import java.util.Collection;
import java.util.Optional;
@@ -48,19 +47,15 @@ public class MessageValidator {
* @param targetRound the target round
* @param proposalBlock the proposal block
* @param blockInterface the block interface
* @param bftExtraDataCodec the bft extra data codec
*/
public SubsequentMessageValidator(
final Collection<Address> validators,
final ConsensusRoundIdentifier targetRound,
final Block proposalBlock,
final BftBlockInterface blockInterface,
final BftExtraDataCodec bftExtraDataCodec) {
final Block commitBlock =
final QbftBlock proposalBlock,
final QbftBlockInterface blockInterface) {
final QbftBlock commitBlock =
blockInterface.replaceRoundInBlock(
proposalBlock,
targetRound.getRoundNumber(),
BftBlockHeaderFunctions.forCommittedSeal(bftExtraDataCodec));
proposalBlock, targetRound.getRoundNumber(), QbftHashMode.COMMITTED_SEAL);
prepareValidator = new PrepareValidator(validators, targetRound, proposalBlock.getHash());
commitValidator =
new CommitValidator(
@@ -97,7 +92,7 @@ public class MessageValidator {
* @param proposalBlock the proposal block
* @return the subsequent message validator
*/
SubsequentMessageValidator create(Block proposalBlock);
SubsequentMessageValidator create(QbftBlock proposalBlock);
}
private final SubsequentMessageValidatorFactory subsequentMessageValidatorFactory;

View File

@@ -14,13 +14,12 @@
*/
package org.hyperledger.besu.consensus.qbft.core.validation;
import org.hyperledger.besu.consensus.common.bft.BftBlockInterface;
import org.hyperledger.besu.consensus.common.bft.BftContext;
import org.hyperledger.besu.consensus.common.bft.BftExtraDataCodec;
import org.hyperledger.besu.consensus.common.bft.BftHelpers;
import org.hyperledger.besu.consensus.common.bft.BftProtocolSchedule;
import org.hyperledger.besu.consensus.common.bft.ConsensusRoundIdentifier;
import org.hyperledger.besu.consensus.common.bft.blockcreation.ProposerSelector;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockInterface;
import org.hyperledger.besu.consensus.qbft.core.types.QbftContext;
import org.hyperledger.besu.consensus.qbft.core.types.QbftProtocolSchedule;
import org.hyperledger.besu.consensus.qbft.core.validation.MessageValidator.SubsequentMessageValidator;
import org.hyperledger.besu.datatypes.Address;
import org.hyperledger.besu.ethereum.ProtocolContext;
@@ -32,9 +31,8 @@ import java.util.Collection;
public class MessageValidatorFactory {
private final ProposerSelector proposerSelector;
private final BftProtocolSchedule protocolSchedule;
private final QbftProtocolSchedule protocolSchedule;
private final ProtocolContext protocolContext;
private final BftExtraDataCodec bftExtraDataCodec;
/**
* Instantiates a new Message validator factory.
@@ -42,17 +40,14 @@ public class MessageValidatorFactory {
* @param proposerSelector the proposer selector
* @param protocolSchedule the protocol schedule
* @param protocolContext the protocol context
* @param bftExtraDataCodec the bft extra data codec
*/
public MessageValidatorFactory(
final ProposerSelector proposerSelector,
final BftProtocolSchedule protocolSchedule,
final ProtocolContext protocolContext,
final BftExtraDataCodec bftExtraDataCodec) {
final QbftProtocolSchedule protocolSchedule,
final ProtocolContext protocolContext) {
this.proposerSelector = proposerSelector;
this.protocolSchedule = protocolSchedule;
this.protocolContext = protocolContext;
this.bftExtraDataCodec = bftExtraDataCodec;
}
/**
@@ -65,7 +60,7 @@ public class MessageValidatorFactory {
public static Collection<Address> getValidatorsAfterBlock(
final ProtocolContext protocolContext, final BlockHeader parentHeader) {
return protocolContext
.getConsensusContext(BftContext.class)
.getConsensusContext(QbftContext.class)
.getValidatorProvider()
.getValidatorsAfterBlock(parentHeader);
}
@@ -80,7 +75,7 @@ public class MessageValidatorFactory {
public static Collection<Address> getValidatorsForBlock(
final ProtocolContext protocolContext, final BlockHeader parentHeader) {
return protocolContext
.getConsensusContext(BftContext.class)
.getConsensusContext(QbftContext.class)
.getValidatorProvider()
.getValidatorsForBlock(parentHeader);
}
@@ -106,7 +101,6 @@ public class MessageValidatorFactory {
BftHelpers.calculateRequiredValidatorQuorum(validatorsForHeight.size()),
chainHeight,
validatorsForHeight,
protocolContext,
protocolSchedule);
}
@@ -129,15 +123,14 @@ public class MessageValidatorFactory {
BftHelpers.calculateRequiredValidatorQuorum(validatorsForHeight.size()),
validatorsForHeight,
roundIdentifier,
proposerSelector.selectProposerForRound(roundIdentifier),
bftExtraDataCodec);
proposerSelector.selectProposerForRound(roundIdentifier));
final BftBlockInterface blockInterface =
protocolContext.getConsensusContext(BftContext.class).getBlockInterface();
final QbftBlockInterface blockInterface =
protocolContext.getConsensusContext(QbftContext.class).getBlockInterface();
return new MessageValidator(
block ->
new SubsequentMessageValidator(
validatorsForHeight, roundIdentifier, block, blockInterface, bftExtraDataCodec),
validatorsForHeight, roundIdentifier, block, blockInterface),
proposalValidator);
}

View File

@@ -19,11 +19,9 @@ import static com.google.common.base.Preconditions.checkState;
import org.hyperledger.besu.consensus.common.bft.ConsensusRoundIdentifier;
import org.hyperledger.besu.consensus.common.bft.payload.SignedData;
import org.hyperledger.besu.consensus.qbft.core.payload.ProposalPayload;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlock;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockValidator;
import org.hyperledger.besu.datatypes.Address;
import org.hyperledger.besu.ethereum.BlockValidator;
import org.hyperledger.besu.ethereum.ProtocolContext;
import org.hyperledger.besu.ethereum.core.Block;
import org.hyperledger.besu.ethereum.mainnet.HeaderValidationMode;
import com.google.common.annotations.VisibleForTesting;
import org.slf4j.Logger;
@@ -37,8 +35,7 @@ public class ProposalPayloadValidator {
private static final Logger LOG = LoggerFactory.getLogger(ProposalPayloadValidator.class);
private final Address expectedProposer;
private final ConsensusRoundIdentifier targetRound;
private final BlockValidator blockValidator;
private final ProtocolContext protocolContext;
private final QbftBlockValidator blockValidator;
/**
* Instantiates a new Proposal payload validator.
@@ -46,18 +43,15 @@ public class ProposalPayloadValidator {
* @param expectedProposer the expected proposer
* @param targetRound the target round
* @param blockValidator the block validator
* @param protocolContext the protocol context
*/
@VisibleForTesting
public ProposalPayloadValidator(
final Address expectedProposer,
final ConsensusRoundIdentifier targetRound,
final BlockValidator blockValidator,
final ProtocolContext protocolContext) {
final QbftBlockValidator blockValidator) {
this.expectedProposer = expectedProposer;
this.targetRound = targetRound;
this.blockValidator = blockValidator;
this.protocolContext = protocolContext;
}
/**
@@ -80,7 +74,7 @@ public class ProposalPayloadValidator {
return false;
}
final Block block = payload.getProposedBlock();
final QbftBlock block = payload.getProposedBlock();
if (!validateBlock(block)) {
return false;
}
@@ -93,18 +87,16 @@ public class ProposalPayloadValidator {
return true;
}
private boolean validateBlock(final Block block) {
private boolean validateBlock(final QbftBlock block) {
checkState(blockValidator != null, "block validation not possible, no block validator.");
final var validationResult =
blockValidator.validateAndProcessBlock(
protocolContext, block, HeaderValidationMode.LIGHT, HeaderValidationMode.FULL, false);
final var validationResult = blockValidator.validateBlock(block);
if (!validationResult.isSuccessful()) {
if (!validationResult.success()) {
LOG.info(
"{}: block did not pass validation. Reason {}",
ERROR_PREFIX,
validationResult.errorMessage);
validationResult.errorMessage());
return false;
}

View File

@@ -17,10 +17,6 @@ package org.hyperledger.besu.consensus.qbft.core.validation;
import static org.hyperledger.besu.consensus.common.bft.validation.ValidationHelpers.hasDuplicateAuthors;
import static org.hyperledger.besu.consensus.common.bft.validation.ValidationHelpers.hasSufficientEntries;
import org.hyperledger.besu.consensus.common.bft.BftBlockHeaderFunctions;
import org.hyperledger.besu.consensus.common.bft.BftBlockInterface;
import org.hyperledger.besu.consensus.common.bft.BftContext;
import org.hyperledger.besu.consensus.common.bft.BftExtraDataCodec;
import org.hyperledger.besu.consensus.common.bft.ConsensusRoundIdentifier;
import org.hyperledger.besu.consensus.common.bft.payload.Payload;
import org.hyperledger.besu.consensus.common.bft.payload.SignedData;
@@ -28,18 +24,20 @@ import org.hyperledger.besu.consensus.qbft.core.messagewrappers.Proposal;
import org.hyperledger.besu.consensus.qbft.core.payload.PreparePayload;
import org.hyperledger.besu.consensus.qbft.core.payload.PreparedRoundMetadata;
import org.hyperledger.besu.consensus.qbft.core.payload.RoundChangePayload;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlock;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockInterface;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockValidator;
import org.hyperledger.besu.consensus.qbft.core.types.QbftContext;
import org.hyperledger.besu.consensus.qbft.core.types.QbftHashMode;
import org.hyperledger.besu.consensus.qbft.core.types.QbftProtocolSchedule;
import org.hyperledger.besu.datatypes.Address;
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.mainnet.ProtocolSchedule;
import java.util.Collection;
import java.util.Comparator;
import java.util.List;
import java.util.Optional;
import java.util.stream.Collectors;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -51,12 +49,11 @@ public class ProposalValidator {
private static final String ERROR_PREFIX = "Invalid Proposal Payload";
private final ProtocolContext protocolContext;
private final ProtocolSchedule protocolSchedule;
private final QbftProtocolSchedule protocolSchedule;
private final int quorumMessageCount;
private final Collection<Address> validators;
private final ConsensusRoundIdentifier roundIdentifier;
private final Address expectedProposer;
private final BftExtraDataCodec bftExtraDataCodec;
/**
* Instantiates a new Proposal validator.
@@ -67,23 +64,20 @@ public class ProposalValidator {
* @param validators the validators
* @param roundIdentifier the round identifier
* @param expectedProposer the expected proposer
* @param bftExtraDataCodec the bft extra data codec
*/
public ProposalValidator(
final ProtocolContext protocolContext,
final ProtocolSchedule protocolSchedule,
final QbftProtocolSchedule protocolSchedule,
final int quorumMessageCount,
final Collection<Address> validators,
final ConsensusRoundIdentifier roundIdentifier,
final Address expectedProposer,
final BftExtraDataCodec bftExtraDataCodec) {
final Address expectedProposer) {
this.protocolContext = protocolContext;
this.protocolSchedule = protocolSchedule;
this.quorumMessageCount = quorumMessageCount;
this.validators = validators;
this.roundIdentifier = roundIdentifier;
this.expectedProposer = expectedProposer;
this.bftExtraDataCodec = bftExtraDataCodec;
}
/**
@@ -93,12 +87,11 @@ public class ProposalValidator {
* @return the boolean
*/
public boolean validate(final Proposal msg) {
final BlockValidator blockValidator =
final QbftBlockValidator blockValidator =
protocolSchedule.getByBlockHeader(msg.getBlock().getHeader()).getBlockValidator();
final ProposalPayloadValidator payloadValidator =
new ProposalPayloadValidator(
expectedProposer, roundIdentifier, blockValidator, protocolContext);
new ProposalPayloadValidator(expectedProposer, roundIdentifier, blockValidator);
if (!payloadValidator.validate(msg.getSignedPayload())) {
LOG.info("{}: invalid proposal payload in proposal message", ERROR_PREFIX);
@@ -150,13 +143,11 @@ public class ProposalValidator {
// to create a block with the old round in it, then re-calc expected hash
// Need to check that if we substitute the LatestPrepareCert round number into the supplied
// block that we get the SAME hash as PreparedCert.
final BftBlockInterface bftBlockInterface =
protocolContext.getConsensusContext(BftContext.class).getBlockInterface();
final Block currentBlockWithOldRound =
final QbftBlockInterface bftBlockInterface =
protocolContext.getConsensusContext(QbftContext.class).getBlockInterface();
final QbftBlock currentBlockWithOldRound =
bftBlockInterface.replaceRoundInBlock(
proposal.getBlock(),
metadata.getPreparedRound(),
BftBlockHeaderFunctions.forCommittedSeal(bftExtraDataCodec));
proposal.getBlock(), metadata.getPreparedRound(), QbftHashMode.COMMITTED_SEAL);
final Hash expectedPriorBlockHash = currentBlockWithOldRound.getHash();
@@ -190,7 +181,7 @@ public class ProposalValidator {
}
private boolean validateRoundZeroProposalHasNoRoundChangesOrPrepares(final Proposal proposal) {
if ((proposal.getRoundChanges().size() != 0) || proposal.getPrepares().size() != 0) {
if ((!proposal.getRoundChanges().isEmpty()) || !proposal.getPrepares().isEmpty()) {
LOG.info("{}: round-0 proposal must not contain any prepares or roundchanges", ERROR_PREFIX);
return false;
}
@@ -292,12 +283,10 @@ public class ProposalValidator {
.filter(Optional::isPresent)
.map(Optional::get)
.distinct()
.collect(Collectors.toList());
.toList();
final List<Integer> preparedRounds =
distinctMetadatas.stream()
.map(PreparedRoundMetadata::getPreparedRound)
.collect(Collectors.toList());
distinctMetadatas.stream().map(PreparedRoundMetadata::getPreparedRound).toList();
for (final Integer preparedRound : preparedRounds) {
if (distinctMetadatas.stream().filter(dm -> dm.getPreparedRound() == preparedRound).count()

View File

@@ -22,12 +22,10 @@ import org.hyperledger.besu.consensus.common.bft.payload.SignedData;
import org.hyperledger.besu.consensus.qbft.core.messagewrappers.RoundChange;
import org.hyperledger.besu.consensus.qbft.core.payload.PreparePayload;
import org.hyperledger.besu.consensus.qbft.core.payload.PreparedRoundMetadata;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlock;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockValidator;
import org.hyperledger.besu.consensus.qbft.core.types.QbftProtocolSchedule;
import org.hyperledger.besu.datatypes.Address;
import org.hyperledger.besu.ethereum.BlockValidator;
import org.hyperledger.besu.ethereum.ProtocolContext;
import org.hyperledger.besu.ethereum.core.Block;
import org.hyperledger.besu.ethereum.mainnet.HeaderValidationMode;
import org.hyperledger.besu.ethereum.mainnet.ProtocolSchedule;
import java.util.Collection;
import java.util.List;
@@ -46,8 +44,7 @@ public class RoundChangeMessageValidator {
private final long quorumMessageCount;
private final long chainHeight;
private final Collection<Address> validators;
private final ProtocolContext protocolContext;
private final ProtocolSchedule protocolSchedule;
private final QbftProtocolSchedule protocolSchedule;
/**
* Instantiates a new Round change message validator.
@@ -56,7 +53,6 @@ public class RoundChangeMessageValidator {
* @param quorumMessageCount the quorum message count
* @param chainHeight the chain height
* @param validators the validators
* @param protocolContext the protocol context
* @param protocolSchedule the protocol context
*/
public RoundChangeMessageValidator(
@@ -64,13 +60,11 @@ public class RoundChangeMessageValidator {
final long quorumMessageCount,
final long chainHeight,
final Collection<Address> validators,
final ProtocolContext protocolContext,
final ProtocolSchedule protocolSchedule) {
final QbftProtocolSchedule protocolSchedule) {
this.roundChangePayloadValidator = roundChangePayloadValidator;
this.quorumMessageCount = quorumMessageCount;
this.chainHeight = chainHeight;
this.validators = validators;
this.protocolContext = protocolContext;
this.protocolSchedule = protocolSchedule;
}
@@ -94,20 +88,18 @@ public class RoundChangeMessageValidator {
return msg.getPreparedRoundMetadata().isEmpty();
}
private boolean validateBlock(final Block block) {
private boolean validateBlock(final QbftBlock block) {
final BlockValidator blockValidator =
final QbftBlockValidator blockValidator =
protocolSchedule.getByBlockHeader(block.getHeader()).getBlockValidator();
final var validationResult =
blockValidator.validateAndProcessBlock(
protocolContext, block, HeaderValidationMode.LIGHT, HeaderValidationMode.FULL);
final var validationResult = blockValidator.validateBlock(block);
if (!validationResult.isSuccessful()) {
if (!validationResult.success()) {
LOG.info(
"{}: block did not pass validation. Reason {}",
ERROR_PREFIX,
validationResult.errorMessage);
validationResult.errorMessage());
return false;
}
@@ -115,7 +107,7 @@ public class RoundChangeMessageValidator {
}
private boolean validateWithBlock(final RoundChange msg) {
final Block block = msg.getProposedBlock().get();
final QbftBlock block = msg.getProposedBlock().get();
if (!validateBlock(block)) {
return false;

View File

@@ -0,0 +1,52 @@
/*
* Copyright contributors to Besu.
*
* Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
* an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
* specific language governing permissions and limitations under the License.
*
* SPDX-License-Identifier: Apache-2.0
*/
package org.hyperledger.besu.consensus.qbft.core;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlock;
import org.hyperledger.besu.ethereum.core.BlockDataGenerator;
import org.hyperledger.besu.ethereum.core.BlockHeader;
public class QbftBlockTestFixture {
private BlockHeader blockHeader = new BlockDataGenerator().header();
private boolean isEmpty = true;
public QbftBlockTestFixture blockHeader(final BlockHeader blockHeader) {
this.blockHeader = blockHeader;
return this;
}
public QbftBlockTestFixture isEmpty(final boolean isEmpty) {
this.isEmpty = isEmpty;
return this;
}
public QbftBlock build() {
return new TestQbftBlock();
}
class TestQbftBlock implements QbftBlock {
@Override
public BlockHeader getHeader() {
return blockHeader;
}
@Override
public boolean isEmpty() {
return isEmpty;
}
}
}

View File

@@ -0,0 +1,47 @@
/*
* Copyright ConsenSys AG.
*
* 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.consensus.qbft.core;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
import static org.mockito.Mockito.withSettings;
import org.hyperledger.besu.consensus.common.validator.ValidatorProvider;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockInterface;
import org.hyperledger.besu.consensus.qbft.core.types.QbftContext;
import org.hyperledger.besu.datatypes.Address;
import java.util.Collection;
import org.mockito.quality.Strictness;
public class QbftContextBuilder {
public static <T extends QbftContext> T setupContextWithBftBlockInterface(
final Class<T> contextClazz,
final Collection<Address> validators,
final QbftBlockInterface bftBlockInterface) {
final T bftContext = mock(contextClazz, withSettings().strictness(Strictness.LENIENT));
final ValidatorProvider mockValidatorProvider =
mock(ValidatorProvider.class, withSettings().strictness(Strictness.LENIENT));
when(bftContext.getValidatorProvider()).thenReturn(mockValidatorProvider);
when(mockValidatorProvider.getValidatorsAfterBlock(any())).thenReturn(validators);
when(bftContext.getBlockInterface()).thenReturn(bftBlockInterface);
when(bftContext.as(any())).thenReturn(bftContext);
return bftContext;
}
}

View File

@@ -15,24 +15,26 @@
package org.hyperledger.besu.consensus.qbft.core.messagewrappers;
import static org.assertj.core.api.Assertions.assertThat;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.ArgumentMatchers.eq;
import static org.mockito.Mockito.when;
import org.hyperledger.besu.consensus.common.bft.BftExtraDataCodec;
import org.hyperledger.besu.consensus.common.bft.ConsensusRoundIdentifier;
import org.hyperledger.besu.consensus.common.bft.payload.SignedData;
import org.hyperledger.besu.consensus.qbft.core.QbftBlockTestFixture;
import org.hyperledger.besu.consensus.qbft.core.messagedata.QbftV1;
import org.hyperledger.besu.consensus.qbft.core.payload.PreparePayload;
import org.hyperledger.besu.consensus.qbft.core.payload.PreparedRoundMetadata;
import org.hyperledger.besu.consensus.qbft.core.payload.ProposalPayload;
import org.hyperledger.besu.consensus.qbft.core.payload.RoundChangePayload;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlock;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockCodec;
import org.hyperledger.besu.consensus.qbft.core.types.QbftHashMode;
import org.hyperledger.besu.cryptoservices.NodeKey;
import org.hyperledger.besu.cryptoservices.NodeKeyUtils;
import org.hyperledger.besu.datatypes.Address;
import org.hyperledger.besu.ethereum.core.Block;
import org.hyperledger.besu.ethereum.core.BlockBody;
import org.hyperledger.besu.ethereum.core.BlockHeaderTestFixture;
import org.hyperledger.besu.ethereum.core.Util;
import java.util.Collections;
import java.util.List;
import java.util.Optional;
@@ -43,22 +45,19 @@ import org.mockito.junit.jupiter.MockitoExtension;
@ExtendWith(MockitoExtension.class)
public class ProposalTest {
@Mock private BftExtraDataCodec bftExtraDataCodec;
@Mock private QbftBlockCodec blockEncoder;
private static final Block BLOCK =
new Block(
new BlockHeaderTestFixture().buildHeader(),
new BlockBody(
Collections.emptyList(),
Collections.emptyList(),
Optional.of(Collections.emptyList())));
private static final QbftBlock BLOCK = new QbftBlockTestFixture().build();
@Test
public void canRoundTripProposalMessage() {
when(blockEncoder.readFrom(any(), eq(QbftHashMode.COMMITTED_SEAL))).thenReturn(BLOCK);
final NodeKey nodeKey = NodeKeyUtils.generate();
final Address addr = Util.publicKeyToAddress(nodeKey.getPublicKey());
final ProposalPayload payload = new ProposalPayload(new ConsensusRoundIdentifier(1, 1), BLOCK);
final ProposalPayload payload =
new ProposalPayload(new ConsensusRoundIdentifier(1, 1), BLOCK, blockEncoder);
final SignedData<ProposalPayload> signedPayload =
SignedData.create(payload, nodeKey.sign(payload.hashForSignature()));
@@ -78,14 +77,14 @@ public class ProposalTest {
final Proposal proposal = new Proposal(signedPayload, List.of(roundChange), List.of(prepare));
final Proposal decodedProposal = Proposal.decode(proposal.encode(), bftExtraDataCodec);
final Proposal decodedProposal = Proposal.decode(proposal.encode(), blockEncoder);
assertThat(decodedProposal.getAuthor()).isEqualTo(addr);
assertThat(decodedProposal.getMessageType()).isEqualTo(QbftV1.PROPOSAL);
assertThat(decodedProposal.getPrepares()).hasSize(1);
assertThat(decodedProposal.getPrepares().get(0)).isEqualToComparingFieldByField(prepare);
assertThat(decodedProposal.getPrepares().getFirst()).isEqualToComparingFieldByField(prepare);
assertThat(decodedProposal.getRoundChanges()).hasSize(1);
assertThat(decodedProposal.getRoundChanges().get(0))
assertThat(decodedProposal.getRoundChanges().getFirst())
.isEqualToComparingFieldByField(roundChange);
assertThat(decodedProposal.getSignedPayload().getPayload().getProposedBlock()).isEqualTo(BLOCK);
assertThat(decodedProposal.getSignedPayload().getPayload().getRoundIdentifier())

View File

@@ -15,20 +15,23 @@
package org.hyperledger.besu.consensus.qbft.core.messagewrappers;
import static org.assertj.core.api.Assertions.assertThat;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.ArgumentMatchers.eq;
import static org.mockito.Mockito.when;
import org.hyperledger.besu.consensus.common.bft.BftExtraDataCodec;
import org.hyperledger.besu.consensus.common.bft.ConsensusRoundIdentifier;
import org.hyperledger.besu.consensus.common.bft.payload.SignedData;
import org.hyperledger.besu.consensus.qbft.core.QbftBlockTestFixture;
import org.hyperledger.besu.consensus.qbft.core.messagedata.QbftV1;
import org.hyperledger.besu.consensus.qbft.core.payload.PreparePayload;
import org.hyperledger.besu.consensus.qbft.core.payload.PreparedRoundMetadata;
import org.hyperledger.besu.consensus.qbft.core.payload.RoundChangePayload;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlock;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockCodec;
import org.hyperledger.besu.consensus.qbft.core.types.QbftHashMode;
import org.hyperledger.besu.cryptoservices.NodeKey;
import org.hyperledger.besu.cryptoservices.NodeKeyUtils;
import org.hyperledger.besu.datatypes.Address;
import org.hyperledger.besu.ethereum.core.Block;
import org.hyperledger.besu.ethereum.core.BlockBody;
import org.hyperledger.besu.ethereum.core.BlockHeaderTestFixture;
import org.hyperledger.besu.ethereum.core.Util;
import java.util.Collections;
@@ -42,15 +45,14 @@ import org.mockito.junit.jupiter.MockitoExtension;
@ExtendWith(MockitoExtension.class)
public class RoundChangeTest {
@Mock private BftExtraDataCodec bftExtraDataCodec;
@Mock private QbftBlockCodec blockEncoder;
private static final Block BLOCK =
new Block(
new BlockHeaderTestFixture().buildHeader(),
new BlockBody(Collections.emptyList(), Collections.emptyList()));
private static final QbftBlock BLOCK = new QbftBlockTestFixture().build();
@Test
public void canRoundTripARoundChangeMessage() {
when(blockEncoder.readFrom(any(), eq(QbftHashMode.COMMITTED_SEAL))).thenReturn(BLOCK);
final NodeKey nodeKey = NodeKeyUtils.generate();
final Address addr = Util.publicKeyToAddress(nodeKey.getPublicKey());
@@ -69,10 +71,12 @@ public class RoundChangeTest {
final RoundChange roundChange =
new RoundChange(
signedRoundChangePayload, Optional.of(BLOCK), List.of(signedPreparePayload));
signedRoundChangePayload,
Optional.of(BLOCK),
blockEncoder,
List.of(signedPreparePayload));
final RoundChange decodedRoundChange =
RoundChange.decode(roundChange.encode(), bftExtraDataCodec);
final RoundChange decodedRoundChange = RoundChange.decode(roundChange.encode(), blockEncoder);
assertThat(decodedRoundChange.getMessageType()).isEqualTo(QbftV1.ROUND_CHANGE);
assertThat(decodedRoundChange.getAuthor()).isEqualTo(addr);
@@ -81,7 +85,7 @@ public class RoundChangeTest {
assertThat(decodedRoundChange.getProposedBlock()).isNotEmpty();
assertThat(decodedRoundChange.getProposedBlock().get()).isEqualToComparingFieldByField(BLOCK);
assertThat(decodedRoundChange.getPrepares()).hasSize(1);
assertThat(decodedRoundChange.getPrepares().get(0))
assertThat(decodedRoundChange.getPrepares().getFirst())
.isEqualToComparingFieldByField(signedPreparePayload);
}
@@ -97,10 +101,10 @@ public class RoundChangeTest {
SignedData.create(payload, nodeKey.sign(payload.hashForSignature()));
final RoundChange roundChange =
new RoundChange(signedRoundChangePayload, Optional.empty(), Collections.emptyList());
new RoundChange(
signedRoundChangePayload, Optional.empty(), blockEncoder, Collections.emptyList());
final RoundChange decodedRoundChange =
RoundChange.decode(roundChange.encode(), bftExtraDataCodec);
final RoundChange decodedRoundChange = RoundChange.decode(roundChange.encode(), blockEncoder);
assertThat(decodedRoundChange.getMessageType()).isEqualTo(QbftV1.ROUND_CHANGE);
assertThat(decodedRoundChange.getAuthor()).isEqualTo(addr);

View File

@@ -17,7 +17,6 @@ package org.hyperledger.besu.consensus.qbft.core.statemachine;
import static java.util.Collections.emptyList;
import static java.util.Collections.singletonList;
import static org.assertj.core.api.Assertions.assertThat;
import static org.hyperledger.besu.consensus.common.bft.BftContextBuilder.setupContextWithBftExtraDataEncoder;
import static org.hyperledger.besu.consensus.common.bft.ConsensusRoundHelpers.createFrom;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.ArgumentMatchers.anyInt;
@@ -31,18 +30,15 @@ import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;
import org.hyperledger.besu.config.StubGenesisConfigOptions;
import org.hyperledger.besu.consensus.common.bft.BftContext;
import org.hyperledger.besu.consensus.common.bft.BftExtraData;
import org.hyperledger.besu.consensus.common.bft.BftExtraDataCodec;
import org.hyperledger.besu.consensus.common.bft.BftProtocolSchedule;
import org.hyperledger.besu.consensus.common.bft.BlockTimer;
import org.hyperledger.besu.consensus.common.bft.ConsensusRoundIdentifier;
import org.hyperledger.besu.consensus.common.bft.RoundTimer;
import org.hyperledger.besu.consensus.common.bft.blockcreation.BftBlockCreator;
import org.hyperledger.besu.consensus.common.bft.events.RoundExpiry;
import org.hyperledger.besu.consensus.common.bft.network.ValidatorMulticaster;
import org.hyperledger.besu.consensus.common.bft.statemachine.BftFinalState;
import org.hyperledger.besu.consensus.common.validator.ValidatorProvider;
import org.hyperledger.besu.consensus.qbft.core.QbftBlockTestFixture;
import org.hyperledger.besu.consensus.qbft.core.messagedata.RoundChangeMessageData;
import org.hyperledger.besu.consensus.qbft.core.messagewrappers.Commit;
import org.hyperledger.besu.consensus.qbft.core.messagewrappers.Prepare;
@@ -50,6 +46,16 @@ import org.hyperledger.besu.consensus.qbft.core.messagewrappers.Proposal;
import org.hyperledger.besu.consensus.qbft.core.messagewrappers.RoundChange;
import org.hyperledger.besu.consensus.qbft.core.network.QbftMessageTransmitter;
import org.hyperledger.besu.consensus.qbft.core.payload.MessageFactory;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlock;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockCodec;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockCreator;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockImporter;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockInterface;
import org.hyperledger.besu.consensus.qbft.core.types.QbftContext;
import org.hyperledger.besu.consensus.qbft.core.types.QbftExtraDataProvider;
import org.hyperledger.besu.consensus.qbft.core.types.QbftFinalState;
import org.hyperledger.besu.consensus.qbft.core.types.QbftProtocolSchedule;
import org.hyperledger.besu.consensus.qbft.core.types.QbftProtocolSpec;
import org.hyperledger.besu.consensus.qbft.core.validation.FutureRoundProposalMessageValidator;
import org.hyperledger.besu.consensus.qbft.core.validation.MessageValidator;
import org.hyperledger.besu.consensus.qbft.core.validation.MessageValidatorFactory;
@@ -60,32 +66,16 @@ import org.hyperledger.besu.cryptoservices.NodeKeyUtils;
import org.hyperledger.besu.datatypes.Address;
import org.hyperledger.besu.datatypes.Hash;
import org.hyperledger.besu.ethereum.ProtocolContext;
import org.hyperledger.besu.ethereum.blockcreation.BlockCreationTiming;
import org.hyperledger.besu.ethereum.blockcreation.BlockCreator.BlockCreationResult;
import org.hyperledger.besu.ethereum.blockcreation.txselection.TransactionSelectionResults;
import org.hyperledger.besu.ethereum.chain.BadBlockManager;
import org.hyperledger.besu.ethereum.chain.DefaultBlockchain;
import org.hyperledger.besu.ethereum.core.Block;
import org.hyperledger.besu.ethereum.core.BlockBody;
import org.hyperledger.besu.ethereum.core.BlockHeader;
import org.hyperledger.besu.ethereum.core.BlockHeaderTestFixture;
import org.hyperledger.besu.ethereum.core.MiningConfiguration;
import org.hyperledger.besu.ethereum.core.PrivacyParameters;
import org.hyperledger.besu.ethereum.core.Util;
import org.hyperledger.besu.ethereum.mainnet.DefaultProtocolSchedule;
import org.hyperledger.besu.ethereum.mainnet.ProtocolSchedule;
import org.hyperledger.besu.ethereum.mainnet.ProtocolScheduleBuilder;
import org.hyperledger.besu.ethereum.mainnet.ProtocolSpecAdapters;
import org.hyperledger.besu.ethereum.p2p.rlpx.wire.MessageData;
import org.hyperledger.besu.evm.internal.EvmConfiguration;
import org.hyperledger.besu.metrics.noop.NoOpMetricsSystem;
import org.hyperledger.besu.util.Subscribers;
import java.math.BigInteger;
import java.time.Clock;
import java.util.List;
import java.util.Optional;
import java.util.function.Function;
import com.google.common.collect.Lists;
import org.apache.tuweni.bytes.Bytes;
@@ -105,48 +95,49 @@ import org.mockito.quality.Strictness;
public class QbftBlockHeightManagerTest {
private final NodeKey nodeKey = NodeKeyUtils.generate();
private final MessageFactory messageFactory = new MessageFactory(nodeKey);
private final BlockHeaderTestFixture headerTestFixture = new BlockHeaderTestFixture();
private MessageFactory messageFactory;
@Mock private BftFinalState finalState;
@Mock private QbftFinalState finalState;
@Mock private QbftMessageTransmitter messageTransmitter;
@Mock private RoundChangeManager roundChangeManager;
@Mock private QbftRoundFactory roundFactory;
@Mock private Clock clock;
@Mock private MessageValidatorFactory messageValidatorFactory;
@Mock private BftBlockCreator blockCreator;
@Mock private QbftBlockCreator blockCreator;
@Mock private BlockTimer blockTimer;
@Mock private RoundTimer roundTimer;
@Mock private DefaultBlockchain blockchain;
@Mock private FutureRoundProposalMessageValidator futureRoundProposalMessageValidator;
@Mock private ValidatorMulticaster validatorMulticaster;
@Mock private ProtocolContext protocolContext;
@Mock private QbftProtocolSchedule protocolSchedule;
@Mock private BlockHeader parentHeader;
@Mock private BftExtraDataCodec bftExtraDataCodec;
@Mock private QbftBlockCodec blockEncoder;
@Mock private QbftExtraDataProvider qbftExtraDataProvider;
@Mock private QbftBlockInterface blockInterface;
@Mock private ValidatorProvider validatorProvider;
@Mock private QbftProtocolSpec protocolSpec;
@Mock private QbftBlockImporter blockImporter;
@Captor private ArgumentCaptor<MessageData> sentMessageArgCaptor;
private final List<Address> validators = Lists.newArrayList();
private final List<MessageFactory> validatorMessageFactory = Lists.newArrayList();
private ProtocolContext protocolContext;
private final ConsensusRoundIdentifier roundIdentifier = new ConsensusRoundIdentifier(1, 0);
private Block createdBlock;
private void buildCreatedBlock() {
final BlockHeader header = headerTestFixture.buildHeader();
createdBlock = new Block(header, new BlockBody(emptyList(), emptyList()));
}
private final QbftBlock createdBlock = new QbftBlockTestFixture().build();
@BeforeEach
public void setup() {
for (int i = 0; i <= 3; i++) {
messageFactory = new MessageFactory(nodeKey, blockEncoder);
for (int i = 0; i < 3; i++) {
final NodeKey nodeKey = NodeKeyUtils.generate();
validators.add(Util.publicKeyToAddress(nodeKey.getPublicKey()));
validatorMessageFactory.add(new MessageFactory(nodeKey));
validatorMessageFactory.add(new MessageFactory(nodeKey, blockEncoder));
}
buildCreatedBlock();
final MessageValidator messageValidator = mock(MessageValidator.class);
when(messageValidator.validateProposal(any())).thenReturn(true);
when(messageValidator.validateCommit(any())).thenReturn(true);
@@ -156,40 +147,16 @@ public class QbftBlockHeightManagerTest {
when(finalState.getQuorum()).thenReturn(3);
when(finalState.getValidatorMulticaster()).thenReturn(validatorMulticaster);
when(finalState.getClock()).thenReturn(clock);
when(blockCreator.createBlock(anyLong(), any()))
.thenReturn(
new BlockCreationResult(
createdBlock, new TransactionSelectionResults(), new BlockCreationTiming()));
when(blockCreator.createBlock(anyLong(), any())).thenReturn(createdBlock);
QbftContext qbftContext = new QbftContext(validatorProvider, blockInterface);
when(protocolContext.getConsensusContext(QbftContext.class)).thenReturn(qbftContext);
when(futureRoundProposalMessageValidator.validateProposalMessage(any())).thenReturn(true);
when(messageValidatorFactory.createFutureRoundProposalMessageValidator(anyLong(), any()))
.thenReturn(futureRoundProposalMessageValidator);
when(messageValidatorFactory.createMessageValidator(any(), any())).thenReturn(messageValidator);
protocolContext =
new ProtocolContext(
blockchain,
null,
setupContextWithBftExtraDataEncoder(BftContext.class, validators, bftExtraDataCodec),
new BadBlockManager());
final ProtocolScheduleBuilder protocolScheduleBuilder =
new ProtocolScheduleBuilder(
new StubGenesisConfigOptions(),
Optional.of(BigInteger.ONE),
ProtocolSpecAdapters.create(0, Function.identity()),
new PrivacyParameters(),
false,
EvmConfiguration.DEFAULT,
MiningConfiguration.MINING_DISABLED,
new BadBlockManager(),
false,
new NoOpMetricsSystem());
ProtocolSchedule protocolSchedule =
new BftProtocolSchedule(
(DefaultProtocolSchedule) protocolScheduleBuilder.createProtocolSchedule());
// Ensure the created QbftRound has the valid ConsensusRoundIdentifier;
when(roundFactory.createNewRound(any(), anyInt()))
.thenAnswer(
@@ -208,6 +175,7 @@ public class QbftBlockHeightManagerTest {
messageTransmitter,
roundTimer,
bftExtraDataCodec,
qbftExtraDataProvider,
parentHeader);
});
@@ -226,14 +194,20 @@ public class QbftBlockHeightManagerTest {
messageTransmitter,
roundTimer,
bftExtraDataCodec,
qbftExtraDataProvider,
parentHeader);
});
when(qbftExtraDataProvider.getExtraData(any()))
.thenReturn(
new BftExtraData(
Bytes.wrap(new byte[32]), emptyList(), Optional.empty(), 0, validators));
when(bftExtraDataCodec.decode(any()))
.thenReturn(
new BftExtraData(
Bytes.wrap(new byte[32]), emptyList(), Optional.empty(), 0, validators));
when(bftExtraDataCodec.encode(any())).thenReturn(Bytes.EMPTY);
when(bftExtraDataCodec.encodeWithoutCommitSeals(any())).thenReturn(Bytes.EMPTY);
}
@Test
@@ -269,7 +243,9 @@ public class QbftBlockHeightManagerTest {
@Test
public void onBlockTimerExpiryRoundTimerIsStartedAndProposalMessageIsTransmitted() {
when(finalState.isLocalNodeProposerForRound(roundIdentifier)).thenReturn(true);
when(blockTimer.checkEmptyBlockExpired(any(), eq(0l))).thenReturn(true);
when(blockTimer.checkEmptyBlockExpired(any(), eq(0L))).thenReturn(true);
when(blockInterface.replaceRoundInBlock(eq(createdBlock), eq(0), any()))
.thenReturn(createdBlock);
final QbftBlockHeightManager manager =
new QbftBlockHeightManager(
@@ -293,7 +269,7 @@ public class QbftBlockHeightManagerTest {
public void
onBlockTimerExpiryForNonProposerRoundTimerIsStartedAndNoProposalMessageIsTransmitted() {
when(finalState.isLocalNodeProposerForRound(roundIdentifier)).thenReturn(false);
when(blockTimer.checkEmptyBlockExpired(any(), eq(0l))).thenReturn(true);
when(blockTimer.checkEmptyBlockExpired(any(), eq(0L))).thenReturn(true);
final QbftBlockHeightManager manager =
new QbftBlockHeightManager(
@@ -324,6 +300,9 @@ public class QbftBlockHeightManagerTest {
messageValidatorFactory,
messageFactory);
when(blockInterface.replaceRoundInBlock(eq(createdBlock), eq(2), any()))
.thenReturn(createdBlock);
// Force a new round to be started at new round number.
final ConsensusRoundIdentifier futureRoundIdentifier = createFrom(roundIdentifier, 0, +2);
final Proposal futureRoundProposal =
@@ -398,6 +377,8 @@ public class QbftBlockHeightManagerTest {
when(roundChangeManager.appendRoundChangeMessage(any()))
.thenReturn(Optional.of(singletonList(roundChange)));
when(finalState.isLocalNodeProposerForRound(any())).thenReturn(true);
when(blockInterface.replaceRoundInBlock(eq(createdBlock), eq(2), any()))
.thenReturn(createdBlock);
final QbftBlockHeightManager manager =
new QbftBlockHeightManager(
@@ -423,6 +404,11 @@ public class QbftBlockHeightManagerTest {
@Test
public void messagesForFutureRoundsAreBufferedAndUsedToPreloadNewRoundWhenItIsStarted() {
when(finalState.getQuorum()).thenReturn(1);
when(blockInterface.replaceRoundInBlock(eq(createdBlock), eq(2), any()))
.thenReturn(createdBlock);
when(blockCreator.createSealedBlock(any(), any(), anyInt(), any())).thenReturn(createdBlock);
when(protocolSchedule.getByBlockHeader(any())).thenReturn(protocolSpec);
when(protocolSpec.getBlockImporter()).thenReturn(blockImporter);
final ConsensusRoundIdentifier futureRoundIdentifier = createFrom(roundIdentifier, 0, +2);
@@ -467,7 +453,12 @@ public class QbftBlockHeightManagerTest {
public void messagesForCurrentRoundAreBufferedAndUsedToPreloadRoundWhenItIsStarted() {
when(finalState.getQuorum()).thenReturn(1);
when(finalState.isLocalNodeProposerForRound(roundIdentifier)).thenReturn(true);
when(blockTimer.checkEmptyBlockExpired(any(), eq(0l))).thenReturn(true);
when(blockTimer.checkEmptyBlockExpired(any(), eq(0L))).thenReturn(true);
when(blockInterface.replaceRoundInBlock(eq(createdBlock), eq(0), any()))
.thenReturn(createdBlock);
when(blockCreator.createSealedBlock(any(), any(), anyInt(), any())).thenReturn(createdBlock);
when(protocolSchedule.getByBlockHeader(any())).thenReturn(protocolSpec);
when(protocolSpec.getBlockImporter()).thenReturn(blockImporter);
final QbftBlockHeightManager manager =
new QbftBlockHeightManager(
@@ -505,7 +496,10 @@ public class QbftBlockHeightManagerTest {
@Test
public void preparedCertificateIncludedInRoundChangeMessageOnRoundTimeoutExpired() {
when(finalState.isLocalNodeProposerForRound(any())).thenReturn(true);
when(blockTimer.checkEmptyBlockExpired(any(), eq(0l))).thenReturn(true);
when(blockTimer.checkEmptyBlockExpired(any(), eq(0L))).thenReturn(true);
when(blockInterface.replaceRoundInBlock(eq(createdBlock), eq(0), any()))
.thenReturn(createdBlock);
when(blockEncoder.readFrom(any(), any())).thenReturn(createdBlock);
final QbftBlockHeightManager manager =
new QbftBlockHeightManager(
@@ -546,7 +540,7 @@ public class QbftBlockHeightManagerTest {
assertThat(capturedMessageData).isInstanceOf(RoundChangeMessageData.class);
final RoundChangeMessageData roundChange = (RoundChangeMessageData) capturedMessageData;
final RoundChange receivedRoundChange = roundChange.decode(bftExtraDataCodec);
final RoundChange receivedRoundChange = roundChange.decode(blockEncoder);
Assertions.assertThat(receivedRoundChange.getPreparedRoundMetadata()).isNotEmpty();

View File

@@ -25,7 +25,6 @@ import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.verifyNoMoreInteractions;
import static org.mockito.Mockito.when;
import org.hyperledger.besu.consensus.common.bft.BftExtraDataCodec;
import org.hyperledger.besu.consensus.common.bft.ConsensusRoundIdentifier;
import org.hyperledger.besu.consensus.common.bft.EthSynchronizerUpdater;
import org.hyperledger.besu.consensus.common.bft.MessageTracker;
@@ -33,7 +32,6 @@ import org.hyperledger.besu.consensus.common.bft.events.BftReceivedMessageEvent;
import org.hyperledger.besu.consensus.common.bft.events.BlockTimerExpiry;
import org.hyperledger.besu.consensus.common.bft.events.NewChainHead;
import org.hyperledger.besu.consensus.common.bft.events.RoundExpiry;
import org.hyperledger.besu.consensus.common.bft.statemachine.BftFinalState;
import org.hyperledger.besu.consensus.common.bft.statemachine.FutureMessageBuffer;
import org.hyperledger.besu.consensus.qbft.core.messagedata.CommitMessageData;
import org.hyperledger.besu.consensus.qbft.core.messagedata.PrepareMessageData;
@@ -45,6 +43,8 @@ import org.hyperledger.besu.consensus.qbft.core.messagewrappers.Prepare;
import org.hyperledger.besu.consensus.qbft.core.messagewrappers.Proposal;
import org.hyperledger.besu.consensus.qbft.core.messagewrappers.RoundChange;
import org.hyperledger.besu.consensus.qbft.core.network.QbftGossip;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockCodec;
import org.hyperledger.besu.consensus.qbft.core.types.QbftFinalState;
import org.hyperledger.besu.datatypes.Address;
import org.hyperledger.besu.datatypes.Hash;
import org.hyperledger.besu.ethereum.chain.Blockchain;
@@ -68,12 +68,11 @@ import org.mockito.quality.Strictness;
@MockitoSettings(strictness = Strictness.LENIENT)
public class QbftControllerTest {
@Mock private Blockchain blockChain;
@Mock private BftFinalState bftFinalState;
@Mock private QbftFinalState qbftFinalState;
@Mock private QbftBlockHeightManagerFactory blockHeightManagerFactory;
@Mock private BlockHeader chainHeadBlockHeader;
@Mock private BlockHeader nextBlock;
@Mock private BaseQbftBlockHeightManager blockHeightManager;
@Mock private BftExtraDataCodec bftExtraDataCodec;
@Mock private Proposal proposal;
private Message proposalMessage;
@@ -99,6 +98,7 @@ public class QbftControllerTest {
private final ConsensusRoundIdentifier pastRoundIdentifier = new ConsensusRoundIdentifier(3, 0);
@Mock private QbftGossip qbftGossip;
@Mock private FutureMessageBuffer futureMessageBuffer;
@Mock private QbftBlockCodec blockEncoder;
private QbftController qbftController;
@BeforeEach
@@ -106,7 +106,7 @@ public class QbftControllerTest {
when(blockChain.getChainHeadHeader()).thenReturn(chainHeadBlockHeader);
when(blockChain.getChainHeadBlockNumber()).thenReturn(3L);
when(blockHeightManagerFactory.create(any())).thenReturn(blockHeightManager);
when(bftFinalState.getValidators()).thenReturn(ImmutableList.of(validator));
when(qbftFinalState.getValidators()).thenReturn(ImmutableList.of(validator));
when(chainHeadBlockHeader.getNumber()).thenReturn(3L);
when(chainHeadBlockHeader.getHash()).thenReturn(Hash.ZERO);
@@ -116,7 +116,7 @@ public class QbftControllerTest {
when(nextBlock.getNumber()).thenReturn(5L);
when(bftFinalState.isLocalNodeValidator()).thenReturn(true);
when(qbftFinalState.isLocalNodeValidator()).thenReturn(true);
when(messageTracker.hasSeenMessage(any())).thenReturn(false);
}
@@ -124,13 +124,13 @@ public class QbftControllerTest {
qbftController =
new QbftController(
blockChain,
bftFinalState,
qbftFinalState,
blockHeightManagerFactory,
qbftGossip,
messageTracker,
futureMessageBuffer,
mock(EthSynchronizerUpdater.class),
bftExtraDataCodec);
blockEncoder);
}
@Test
@@ -454,7 +454,7 @@ public class QbftControllerTest {
when(proposal.getAuthor()).thenReturn(validator);
when(proposal.getRoundIdentifier()).thenReturn(roundIdentifier);
when(proposalMessageData.getCode()).thenReturn(QbftV1.PROPOSAL);
when(proposalMessageData.decode(bftExtraDataCodec)).thenReturn(proposal);
when(proposalMessageData.decode(blockEncoder)).thenReturn(proposal);
proposalMessage = new DefaultMessage(null, proposalMessageData);
}
@@ -481,7 +481,7 @@ public class QbftControllerTest {
when(roundChange.getAuthor()).thenReturn(validator);
when(roundChange.getRoundIdentifier()).thenReturn(roundIdentifier);
when(roundChangeMessageData.getCode()).thenReturn(QbftV1.ROUND_CHANGE);
when(roundChangeMessageData.decode(bftExtraDataCodec)).thenReturn(roundChange);
when(roundChangeMessageData.decode(blockEncoder)).thenReturn(roundChange);
roundChangeMessage = new DefaultMessage(null, roundChangeMessageData);
}
}

View File

@@ -18,7 +18,7 @@ import static java.util.Collections.emptyList;
import static java.util.Collections.singletonList;
import static java.util.Optional.empty;
import static org.assertj.core.api.Assertions.assertThat;
import static org.hyperledger.besu.consensus.common.bft.BftContextBuilder.setupContextWithBftBlockInterface;
import static org.hyperledger.besu.consensus.qbft.core.QbftContextBuilder.setupContextWithBftBlockInterface;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.ArgumentMatchers.anyLong;
import static org.mockito.ArgumentMatchers.eq;
@@ -29,38 +29,37 @@ import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.verifyNoInteractions;
import static org.mockito.Mockito.when;
import org.hyperledger.besu.consensus.common.bft.BftBlockInterface;
import org.hyperledger.besu.consensus.common.bft.BftContext;
import org.hyperledger.besu.consensus.common.bft.BftExtraData;
import org.hyperledger.besu.consensus.common.bft.BftExtraDataCodec;
import org.hyperledger.besu.consensus.common.bft.BftProtocolSchedule;
import org.hyperledger.besu.consensus.common.bft.ConsensusRoundIdentifier;
import org.hyperledger.besu.consensus.common.bft.RoundTimer;
import org.hyperledger.besu.consensus.common.bft.blockcreation.BftBlockCreator;
import org.hyperledger.besu.consensus.common.bft.payload.SignedData;
import org.hyperledger.besu.consensus.qbft.core.QbftBlockTestFixture;
import org.hyperledger.besu.consensus.qbft.core.messagewrappers.RoundChange;
import org.hyperledger.besu.consensus.qbft.core.network.QbftMessageTransmitter;
import org.hyperledger.besu.consensus.qbft.core.payload.MessageFactory;
import org.hyperledger.besu.consensus.qbft.core.payload.PreparePayload;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlock;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockCodec;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockCreator;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockImporter;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockInterface;
import org.hyperledger.besu.consensus.qbft.core.types.QbftContext;
import org.hyperledger.besu.consensus.qbft.core.types.QbftExtraDataProvider;
import org.hyperledger.besu.consensus.qbft.core.types.QbftHashMode;
import org.hyperledger.besu.consensus.qbft.core.types.QbftMinedBlockObserver;
import org.hyperledger.besu.consensus.qbft.core.types.QbftProtocolSchedule;
import org.hyperledger.besu.consensus.qbft.core.types.QbftProtocolSpec;
import org.hyperledger.besu.consensus.qbft.core.validation.MessageValidator;
import org.hyperledger.besu.crypto.SECPSignature;
import org.hyperledger.besu.crypto.SignatureAlgorithmFactory;
import org.hyperledger.besu.cryptoservices.NodeKey;
import org.hyperledger.besu.cryptoservices.NodeKeyUtils;
import org.hyperledger.besu.ethereum.ProtocolContext;
import org.hyperledger.besu.ethereum.blockcreation.BlockCreationTiming;
import org.hyperledger.besu.ethereum.blockcreation.BlockCreator.BlockCreationResult;
import org.hyperledger.besu.ethereum.blockcreation.txselection.TransactionSelectionResults;
import org.hyperledger.besu.ethereum.chain.BadBlockManager;
import org.hyperledger.besu.ethereum.chain.MinedBlockObserver;
import org.hyperledger.besu.ethereum.chain.MutableBlockchain;
import org.hyperledger.besu.ethereum.core.Block;
import org.hyperledger.besu.ethereum.core.BlockBody;
import org.hyperledger.besu.ethereum.core.BlockHeader;
import org.hyperledger.besu.ethereum.core.BlockHeaderTestFixture;
import org.hyperledger.besu.ethereum.core.BlockImporter;
import org.hyperledger.besu.ethereum.mainnet.BlockImportResult;
import org.hyperledger.besu.ethereum.mainnet.ProtocolSpec;
import org.hyperledger.besu.ethereum.worldstate.WorldStateArchive;
import org.hyperledger.besu.plugin.services.securitymodule.SecurityModuleException;
import org.hyperledger.besu.util.Subscribers;
@@ -88,28 +87,30 @@ public class QbftRoundTest {
private final NodeKey nodeKey = NodeKeyUtils.generate();
private final NodeKey nodeKey2 = NodeKeyUtils.generate();
private final ConsensusRoundIdentifier roundIdentifier = new ConsensusRoundIdentifier(1, 0);
private final MessageFactory messageFactory = new MessageFactory(nodeKey);
private final MessageFactory messageFactory2 = new MessageFactory(nodeKey2);
private final Subscribers<MinedBlockObserver> subscribers = Subscribers.create();
private final Subscribers<QbftMinedBlockObserver> subscribers = Subscribers.create();
private ProtocolContext protocolContext;
private MessageFactory messageFactory;
private MessageFactory messageFactory2;
@Mock private BftProtocolSchedule protocolSchedule;
@Mock private QbftProtocolSchedule protocolSchedule;
@Mock private MutableBlockchain blockChain;
@Mock private WorldStateArchive worldStateArchive;
@Mock private QbftMessageTransmitter transmitter;
@Mock private MinedBlockObserver minedBlockObserver;
@Mock private BftBlockCreator blockCreator;
@Mock private QbftMinedBlockObserver minedBlockObserver;
@Mock private QbftBlockCreator blockCreator;
@Mock private MessageValidator messageValidator;
@Mock private RoundTimer roundTimer;
@Mock private ProtocolSpec protocolSpec;
@Mock private BlockImporter blockImporter;
@Mock private BlockHeader parentHeader;
@Mock private QbftProtocolSpec protocolSpec;
@Mock private QbftBlockImporter blockImporter;
@Mock private org.hyperledger.besu.ethereum.core.BlockHeader parentHeader;
@Mock private BftExtraDataCodec bftExtraDataCodec;
@Mock private BftBlockInterface bftBlockInteface;
@Mock private QbftBlockInterface blockInteface;
@Mock private QbftBlockCodec blockEncoder;
@Mock private QbftExtraDataProvider qbftExtraDataProvider;
@Captor private ArgumentCaptor<Block> blockCaptor;
@Captor private ArgumentCaptor<QbftBlock> blockCaptor;
private Block proposedBlock;
private QbftBlock proposedBlock;
private final SECPSignature remoteCommitSeal =
SignatureAlgorithmFactory.getInstance()
@@ -117,33 +118,30 @@ public class QbftRoundTest {
@BeforeEach
public void setup() {
messageFactory = new MessageFactory(nodeKey, blockEncoder);
messageFactory2 = new MessageFactory(nodeKey2, blockEncoder);
protocolContext =
new ProtocolContext(
blockChain,
worldStateArchive,
setupContextWithBftBlockInterface(BftContext.class, emptyList(), bftBlockInteface),
setupContextWithBftBlockInterface(QbftContext.class, emptyList(), blockInteface),
new BadBlockManager());
when(messageValidator.validateProposal(any())).thenReturn(true);
when(messageValidator.validatePrepare(any())).thenReturn(true);
when(messageValidator.validateCommit(any())).thenReturn(true);
final BlockHeaderTestFixture headerTestFixture = new BlockHeaderTestFixture();
headerTestFixture.number(1);
final BlockHeader header = new BlockHeaderTestFixture().number(1).buildHeader();
final BlockHeader header = headerTestFixture.buildHeader();
proposedBlock = new Block(header, new BlockBody(emptyList(), emptyList()));
proposedBlock = new QbftBlockTestFixture().blockHeader(header).build();
when(blockCreator.createBlock(anyLong(), any()))
.thenReturn(
new BlockCreationResult(
proposedBlock, new TransactionSelectionResults(), new BlockCreationTiming()));
when(blockCreator.createBlock(anyLong(), any())).thenReturn(proposedBlock);
when(protocolSchedule.getByBlockHeader(any())).thenReturn(protocolSpec);
when(protocolSpec.getBlockImporter()).thenReturn(blockImporter);
when(blockImporter.importBlock(any(), any(), any()))
.thenReturn(new BlockImportResult(BlockImportResult.BlockImportStatus.IMPORTED));
when(blockImporter.importBlock(any())).thenReturn(true);
BftExtraData bftExtraData =
new BftExtraData(Bytes.wrap(new byte[32]), emptyList(), empty(), 0, emptyList());
@@ -151,9 +149,6 @@ public class QbftRoundTest {
when(bftExtraDataCodec.encode(any())).thenReturn(Bytes.EMPTY);
when(bftExtraDataCodec.encodeWithoutCommitSeals(any())).thenReturn(Bytes.EMPTY);
when(bftExtraDataCodec.encodeWithoutCommitSealsAndRoundNumber(any())).thenReturn(Bytes.EMPTY);
when(bftBlockInteface.replaceRoundInBlock(
eq(proposedBlock), eq(roundIdentifier.getRoundNumber()), any()))
.thenReturn(proposedBlock);
subscribers.subscribe(minedBlockObserver);
}
@@ -172,6 +167,7 @@ public class QbftRoundTest {
transmitter,
roundTimer,
bftExtraDataCodec,
qbftExtraDataProvider,
parentHeader);
verify(roundTimer, times(1)).startTimer(roundIdentifier);
}
@@ -191,9 +187,10 @@ public class QbftRoundTest {
transmitter,
roundTimer,
bftExtraDataCodec,
qbftExtraDataProvider,
parentHeader);
when(bftBlockInteface.replaceRoundInBlock(eq(proposedBlock), eq(0), any()))
when(blockInteface.replaceRoundInBlock(eq(proposedBlock), eq(0), any()))
.thenReturn(proposedBlock);
round.handleProposalMessage(
@@ -205,6 +202,12 @@ public class QbftRoundTest {
@Test
public void aProposalWithAnewBlockIsSentUponReceptionOfARoundChangeWithNoCertificate() {
final BlockHeader header = new BlockHeaderTestFixture().number(0).buildHeader();
final QbftBlock commitBlock = new QbftBlockTestFixture().blockHeader(header).build();
when(blockInteface.replaceRoundInBlock(proposedBlock, 0, QbftHashMode.COMMITTED_SEAL))
.thenReturn(commitBlock);
final RoundState roundState = new RoundState(roundIdentifier, 2, messageValidator);
final QbftRound round =
new QbftRound(
@@ -218,6 +221,7 @@ public class QbftRoundTest {
transmitter,
roundTimer,
bftExtraDataCodec,
qbftExtraDataProvider,
parentHeader);
round.startRoundWith(new RoundChangeArtifacts(emptyList(), Optional.empty()), 15);
@@ -228,6 +232,19 @@ public class QbftRoundTest {
@Test
public void aProposalMessageWithTheSameBlockIsSentUponReceptionOfARoundChangeWithCertificate() {
final QbftBlock publishBlock =
new QbftBlockTestFixture()
.blockHeader(new BlockHeaderTestFixture().number(0).buildHeader())
.build();
final QbftBlock commitBlock =
new QbftBlockTestFixture()
.blockHeader(new BlockHeaderTestFixture().number(0).buildHeader())
.build();
when(blockInteface.replaceRoundInBlock(proposedBlock, 0, QbftHashMode.COMMITTED_SEAL))
.thenReturn(publishBlock);
when(blockInteface.replaceRoundInBlock(publishBlock, 0, QbftHashMode.COMMITTED_SEAL))
.thenReturn(commitBlock);
final ConsensusRoundIdentifier priorRoundChange = new ConsensusRoundIdentifier(1, 0);
final RoundState roundState = new RoundState(roundIdentifier, 2, messageValidator);
final QbftRound round =
@@ -242,6 +259,7 @@ public class QbftRoundTest {
transmitter,
roundTimer,
bftExtraDataCodec,
qbftExtraDataProvider,
parentHeader);
final SignedData<PreparePayload> preparedPayload =
@@ -274,6 +292,13 @@ public class QbftRoundTest {
@Test
public void creatingNewBlockFromEmptyPreparedCertificateUpdatesInternalState() {
final QbftBlock commitBlock =
new QbftBlockTestFixture()
.blockHeader(new BlockHeaderTestFixture().number(0).buildHeader())
.build();
when(blockInteface.replaceRoundInBlock(proposedBlock, 0, QbftHashMode.COMMITTED_SEAL))
.thenReturn(commitBlock);
final RoundState roundState = new RoundState(roundIdentifier, 2, messageValidator);
final QbftRound round =
new QbftRound(
@@ -287,6 +312,7 @@ public class QbftRoundTest {
transmitter,
roundTimer,
bftExtraDataCodec,
qbftExtraDataProvider,
parentHeader);
final RoundChange roundChange =
@@ -329,17 +355,22 @@ public class QbftRoundTest {
transmitter,
roundTimer,
bftExtraDataCodec,
qbftExtraDataProvider,
parentHeader);
when(bftBlockInteface.replaceRoundInBlock(eq(proposedBlock), eq(0), any()))
when(blockInteface.replaceRoundInBlock(proposedBlock, 0, QbftHashMode.COMMITTED_SEAL))
.thenReturn(proposedBlock);
when(blockCreator.createSealedBlock(eq(qbftExtraDataProvider), eq(proposedBlock), eq(0), any()))
.thenReturn(proposedBlock);
round.handleCommitMessage(
messageFactory.createCommit(roundIdentifier, proposedBlock.getHash(), remoteCommitSeal));
verify(blockImporter, never()).importBlock(any());
round.handleProposalMessage(
messageFactory.createProposal(
roundIdentifier, proposedBlock, Collections.emptyList(), Collections.emptyList()));
verify(blockImporter).importBlock(proposedBlock);
}
@Test
@@ -358,17 +389,22 @@ public class QbftRoundTest {
transmitter,
roundTimer,
bftExtraDataCodec,
qbftExtraDataProvider,
parentHeader);
when(bftBlockInteface.replaceRoundInBlock(eq(proposedBlock), eq(0), any()))
when(blockInteface.replaceRoundInBlock(eq(proposedBlock), eq(0), any()))
.thenReturn(proposedBlock);
when(blockCreator.createSealedBlock(eq(qbftExtraDataProvider), eq(proposedBlock), eq(0), any()))
.thenReturn(proposedBlock);
round.handleCommitMessage(
messageFactory.createCommit(roundIdentifier, proposedBlock.getHash(), remoteCommitSeal));
verify(blockImporter, never()).importBlock(any());
round.handleProposalMessage(
messageFactory.createProposal(
roundIdentifier, proposedBlock, Collections.emptyList(), Collections.emptyList()));
verify(blockImporter).importBlock(proposedBlock);
}
@Test
@@ -376,7 +412,7 @@ public class QbftRoundTest {
final int QUORUM_SIZE = 1;
final RoundState roundState = new RoundState(roundIdentifier, QUORUM_SIZE, messageValidator);
final NodeKey throwingNodeKey = mock(NodeKey.class);
final MessageFactory throwingMessageFactory = new MessageFactory(throwingNodeKey);
final MessageFactory throwingMessageFactory = new MessageFactory(throwingNodeKey, blockEncoder);
when(throwingNodeKey.sign(any())).thenThrow(new SecurityModuleException("Hsm is Offline"));
final QbftRound round =
@@ -391,9 +427,10 @@ public class QbftRoundTest {
transmitter,
roundTimer,
bftExtraDataCodec,
qbftExtraDataProvider,
parentHeader);
when(bftBlockInteface.replaceRoundInBlock(eq(proposedBlock), eq(0), any()))
when(blockInteface.replaceRoundInBlock(eq(proposedBlock), eq(0), any()))
.thenReturn(proposedBlock);
round.handleProposalMessage(

View File

@@ -28,21 +28,21 @@ import org.hyperledger.besu.consensus.qbft.core.messagewrappers.Prepare;
import org.hyperledger.besu.consensus.qbft.core.messagewrappers.Proposal;
import org.hyperledger.besu.consensus.qbft.core.payload.MessageFactory;
import org.hyperledger.besu.consensus.qbft.core.payload.PreparePayload;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlock;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockCodec;
import org.hyperledger.besu.consensus.qbft.core.validation.MessageValidator;
import org.hyperledger.besu.crypto.SignatureAlgorithm;
import org.hyperledger.besu.crypto.SignatureAlgorithmFactory;
import org.hyperledger.besu.cryptoservices.NodeKey;
import org.hyperledger.besu.cryptoservices.NodeKeyUtils;
import org.hyperledger.besu.datatypes.Address;
import org.hyperledger.besu.datatypes.Hash;
import org.hyperledger.besu.ethereum.core.Block;
import org.hyperledger.besu.ethereum.core.Util;
import java.math.BigInteger;
import java.util.Collections;
import java.util.List;
import java.util.Optional;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import com.google.common.base.Supplier;
import com.google.common.base.Suppliers;
@@ -59,24 +59,19 @@ public class RoundStateTest {
private static final Supplier<SignatureAlgorithm> SIGNATURE_ALGORITHM =
Suppliers.memoize(SignatureAlgorithmFactory::getInstance);
private final List<NodeKey> validatorKeys = Lists.newArrayList();
private final List<MessageFactory> validatorMessageFactories = Lists.newArrayList();
private final ConsensusRoundIdentifier roundIdentifier = new ConsensusRoundIdentifier(1, 1);
private final List<Address> validators = Lists.newArrayList();
private final Hash blockHash = Hash.fromHexStringLenient("1");
@Mock private MessageValidator messageValidator;
@Mock private Block block;
@Mock private QbftBlock block;
@Mock private QbftBlockCodec blockEncoder;
@BeforeEach
public void setup() {
for (int i = 0; i < 3; i++) {
final NodeKey newNodeKey = NodeKeyUtils.generate();
validatorKeys.add(newNodeKey);
validators.add(Util.publicKeyToAddress(newNodeKey.getPublicKey()));
validatorMessageFactories.add(new MessageFactory(newNodeKey));
validatorMessageFactories.add(new MessageFactory(newNodeKey, blockEncoder));
}
}
@@ -96,7 +91,7 @@ public class RoundStateTest {
final Proposal proposal =
validatorMessageFactories
.get(0)
.getFirst()
.createProposal(
roundIdentifier, block, Collections.emptyList(), Collections.emptyList());
@@ -115,7 +110,7 @@ public class RoundStateTest {
final Proposal proposal =
validatorMessageFactories
.get(0)
.getFirst()
.createProposal(
roundIdentifier, block, Collections.emptyList(), Collections.emptyList());
@@ -125,7 +120,7 @@ public class RoundStateTest {
final Commit commit =
validatorMessageFactories
.get(0)
.getFirst()
.createCommit(
roundIdentifier,
blockHash,
@@ -172,7 +167,7 @@ public class RoundStateTest {
final Proposal proposal =
validatorMessageFactories
.get(0)
.getFirst()
.createProposal(
roundIdentifier, block, Collections.emptyList(), Collections.emptyList());
assertThat(roundState.setProposedBlock(proposal)).isTrue();
@@ -203,7 +198,7 @@ public class RoundStateTest {
final Proposal proposal =
validatorMessageFactories
.get(0)
.getFirst()
.createProposal(
roundIdentifier, block, Collections.emptyList(), Collections.emptyList());
@@ -330,7 +325,7 @@ public class RoundStateTest {
assertThat(preparedCertificate.get().getBlock()).isEqualTo(block);
final List<SignedData<PreparePayload>> expectedPrepares =
List.of(firstPrepare, secondPrepare).stream()
Stream.of(firstPrepare, secondPrepare)
.map(BftMessage::getSignedPayload)
.collect(Collectors.toList());
assertThat(preparedCertificate.get().getPrepares()).isEqualTo(expectedPrepares);

View File

@@ -19,21 +19,34 @@ import static org.assertj.core.api.Assertions.assertThat;
import org.hyperledger.besu.consensus.common.bft.ConsensusRoundHelpers;
import org.hyperledger.besu.consensus.common.bft.ConsensusRoundIdentifier;
import org.hyperledger.besu.consensus.qbft.core.messagewrappers.Commit;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockCodec;
import org.hyperledger.besu.crypto.SECPSignature;
import org.hyperledger.besu.datatypes.Hash;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.extension.ExtendWith;
import org.mockito.Mock;
import org.mockito.junit.jupiter.MockitoExtension;
@ExtendWith(MockitoExtension.class)
public class CommitValidatorTest {
private static final int VALIDATOR_COUNT = 4;
private final QbftNodeList validators = QbftNodeList.createNodes(VALIDATOR_COUNT);
final ConsensusRoundIdentifier round = new ConsensusRoundIdentifier(1, 0);
final Hash expectedHash = Hash.fromHexStringLenient("0x1");
final Hash expectedCommitHash = Hash.fromHexStringLenient("0x1");
final CommitValidator validator =
new CommitValidator(validators.getNodeAddresses(), round, expectedHash, expectedCommitHash);
private final ConsensusRoundIdentifier round = new ConsensusRoundIdentifier(1, 0);
private final Hash expectedHash = Hash.fromHexStringLenient("0x1");
private final Hash expectedCommitHash = Hash.fromHexStringLenient("0x1");
private QbftNodeList validators;
private CommitValidator validator;
private @Mock QbftBlockCodec qbftBlockCodec;
@BeforeEach
public void setup() {
validators = QbftNodeList.createNodes(VALIDATOR_COUNT, qbftBlockCodec);
validator =
new CommitValidator(validators.getNodeAddresses(), round, expectedHash, expectedCommitHash);
}
@Test
public void commitIsValidIfItMatchesExpectedValues() {
@@ -47,7 +60,7 @@ public class CommitValidatorTest {
@Test
public void commitSignedByANonValidatorFails() {
final QbftNode nonValidator = QbftNode.create();
final QbftNode nonValidator = QbftNode.create(qbftBlockCodec);
final SECPSignature commitSeal = nonValidator.getNodeKey().sign(expectedHash);
final Commit msg =
nonValidator.getMessageFactory().createCommit(round, expectedHash, commitSeal);

View File

@@ -19,19 +19,28 @@ import static org.assertj.core.api.Assertions.assertThat;
import org.hyperledger.besu.consensus.common.bft.ConsensusRoundHelpers;
import org.hyperledger.besu.consensus.common.bft.ConsensusRoundIdentifier;
import org.hyperledger.besu.consensus.qbft.core.messagewrappers.Prepare;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockCodec;
import org.hyperledger.besu.datatypes.Hash;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.mockito.Mock;
public class PrepareValidatorTest {
private static final int VALIDATOR_COUNT = 4;
private final QbftNodeList validators = QbftNodeList.createNodes(VALIDATOR_COUNT);
private final ConsensusRoundIdentifier round = new ConsensusRoundIdentifier(1, 0);
private final Hash expectedHash = Hash.fromHexStringLenient("0x1");
private final PrepareValidator validator =
new PrepareValidator(validators.getNodeAddresses(), round, expectedHash);
@Mock private QbftBlockCodec blockEncoder;
private QbftNodeList validators;
private PrepareValidator validator;
@BeforeEach
public void setup() {
validators = QbftNodeList.createNodes(VALIDATOR_COUNT, blockEncoder);
validator = new PrepareValidator(validators.getNodeAddresses(), round, expectedHash);
}
@Test
public void prepareIsValidIfItMatchesExpectedValues() {
@@ -43,7 +52,7 @@ public class PrepareValidatorTest {
@Test
public void prepareSignedByANonValidatorFails() {
final QbftNode nonValidator = QbftNode.create();
final QbftNode nonValidator = QbftNode.create(blockEncoder);
final Prepare msg = nonValidator.getMessageFactory().createPrepare(round, expectedHash);
assertThat(validator.validate(msg)).isFalse();

View File

@@ -16,35 +16,27 @@ package org.hyperledger.besu.consensus.qbft.core.validation;
import static java.util.Collections.emptyList;
import static org.assertj.core.api.Assertions.assertThat;
import static org.hyperledger.besu.consensus.common.bft.BftContextBuilder.setupContextWithBftExtraDataEncoder;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.ArgumentMatchers.eq;
import static org.mockito.Mockito.verifyNoMoreInteractions;
import static org.mockito.Mockito.when;
import org.hyperledger.besu.consensus.common.bft.BftContext;
import org.hyperledger.besu.consensus.common.bft.BftExtraDataCodec;
import org.hyperledger.besu.consensus.common.bft.ConsensusRoundHelpers;
import org.hyperledger.besu.consensus.common.bft.ConsensusRoundIdentifier;
import org.hyperledger.besu.consensus.common.bft.ProposedBlockHelpers;
import org.hyperledger.besu.consensus.qbft.core.QbftBlockTestFixture;
import org.hyperledger.besu.consensus.qbft.core.messagewrappers.Proposal;
import org.hyperledger.besu.consensus.qbft.core.payload.MessageFactory;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlock;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockCodec;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockValidator;
import org.hyperledger.besu.cryptoservices.NodeKey;
import org.hyperledger.besu.cryptoservices.NodeKeyUtils;
import org.hyperledger.besu.datatypes.Address;
import org.hyperledger.besu.ethereum.BlockProcessingResult;
import org.hyperledger.besu.ethereum.BlockValidator;
import org.hyperledger.besu.ethereum.ProtocolContext;
import org.hyperledger.besu.ethereum.chain.BadBlockManager;
import org.hyperledger.besu.ethereum.chain.MutableBlockchain;
import org.hyperledger.besu.ethereum.core.Block;
import org.hyperledger.besu.ethereum.core.BlockHeader;
import org.hyperledger.besu.ethereum.core.BlockHeaderTestFixture;
import org.hyperledger.besu.ethereum.core.Util;
import org.hyperledger.besu.ethereum.mainnet.HeaderValidationMode;
import org.hyperledger.besu.ethereum.worldstate.WorldStateArchive;
import java.util.Optional;
import org.apache.tuweni.bytes.Bytes;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.extension.ExtendWith;
@@ -54,11 +46,8 @@ import org.mockito.junit.jupiter.MockitoExtension;
@ExtendWith(MockitoExtension.class)
public class ProposalPayloadValidatorTest {
@Mock private BlockValidator blockValidator;
@Mock private MutableBlockchain blockChain;
@Mock private WorldStateArchive worldStateArchive;
@Mock private BftExtraDataCodec bftExtraDataCodec;
private ProtocolContext protocolContext;
@Mock private QbftBlockValidator blockValidator;
@Mock private QbftBlockCodec blockEncoder;
private static final int CHAIN_HEIGHT = 3;
private final ConsensusRoundIdentifier targetRound =
@@ -66,89 +55,63 @@ public class ProposalPayloadValidatorTest {
private final NodeKey nodeKey = NodeKeyUtils.generate();
private final Address expectedProposer = Util.publicKeyToAddress(nodeKey.getPublicKey());
private final MessageFactory messageFactory = new MessageFactory(nodeKey);
private MessageFactory messageFactory;
final ConsensusRoundIdentifier roundIdentifier =
ConsensusRoundHelpers.createFrom(targetRound, 1, 0);
@BeforeEach
public void setup() {
protocolContext =
new ProtocolContext(
blockChain,
worldStateArchive,
setupContextWithBftExtraDataEncoder(BftContext.class, emptyList(), bftExtraDataCodec),
new BadBlockManager());
messageFactory = new MessageFactory(nodeKey, blockEncoder);
}
@Test
public void validationPassesWhenProposerAndRoundMatchAndBlockIsValid() {
when(bftExtraDataCodec.encode(any())).thenReturn(Bytes.EMPTY);
final ProposalPayloadValidator payloadValidator =
new ProposalPayloadValidator(
expectedProposer, roundIdentifier, blockValidator, protocolContext);
final Block block =
ProposedBlockHelpers.createProposalBlock(emptyList(), roundIdentifier, bftExtraDataCodec);
new ProposalPayloadValidator(expectedProposer, roundIdentifier, blockValidator);
final BlockHeader header =
new BlockHeaderTestFixture().number(roundIdentifier.getSequenceNumber()).buildHeader();
final QbftBlock block = new QbftBlockTestFixture().blockHeader(header).build();
final Proposal proposal =
messageFactory.createProposal(roundIdentifier, block, emptyList(), emptyList());
when(blockValidator.validateAndProcessBlock(
eq(protocolContext),
eq(block),
eq(HeaderValidationMode.LIGHT),
eq(HeaderValidationMode.FULL),
eq(false)))
.thenReturn(new BlockProcessingResult(Optional.empty()));
when(blockValidator.validateBlock(eq(block)))
.thenReturn(new QbftBlockValidator.ValidationResult(true, Optional.empty()));
assertThat(payloadValidator.validate(proposal.getSignedPayload())).isTrue();
}
@Test
public void validationPassesWhenBlockRoundDoesNotMatchProposalRound() {
when(bftExtraDataCodec.encode(any())).thenReturn(Bytes.EMPTY);
final ProposalPayloadValidator payloadValidator =
new ProposalPayloadValidator(
expectedProposer, roundIdentifier, blockValidator, protocolContext);
new ProposalPayloadValidator(expectedProposer, roundIdentifier, blockValidator);
final Block block =
ProposedBlockHelpers.createProposalBlock(
emptyList(),
ConsensusRoundHelpers.createFrom(roundIdentifier, 0, +1),
bftExtraDataCodec);
final BlockHeader header =
new BlockHeaderTestFixture().number(roundIdentifier.getSequenceNumber()).buildHeader();
final QbftBlock block = new QbftBlockTestFixture().blockHeader(header).build();
final Proposal proposal =
messageFactory.createProposal(roundIdentifier, block, emptyList(), emptyList());
when(blockValidator.validateAndProcessBlock(
eq(protocolContext),
eq(block),
eq(HeaderValidationMode.LIGHT),
eq(HeaderValidationMode.FULL),
eq(false)))
.thenReturn(new BlockProcessingResult(Optional.empty()));
when(blockValidator.validateBlock(eq(block)))
.thenReturn(new QbftBlockValidator.ValidationResult(true, Optional.empty()));
assertThat(payloadValidator.validate(proposal.getSignedPayload())).isTrue();
}
@Test
public void validationFailsWhenBlockFailsValidation() {
when(bftExtraDataCodec.encode(any())).thenReturn(Bytes.EMPTY);
final ConsensusRoundIdentifier roundIdentifier =
ConsensusRoundHelpers.createFrom(targetRound, 1, 0);
final ProposalPayloadValidator payloadValidator =
new ProposalPayloadValidator(
expectedProposer, roundIdentifier, blockValidator, protocolContext);
final Block block =
ProposedBlockHelpers.createProposalBlock(emptyList(), roundIdentifier, bftExtraDataCodec);
new ProposalPayloadValidator(expectedProposer, roundIdentifier, blockValidator);
final BlockHeader header =
new BlockHeaderTestFixture().number(roundIdentifier.getSequenceNumber()).buildHeader();
final QbftBlock block = new QbftBlockTestFixture().blockHeader(header).build();
final Proposal proposal =
messageFactory.createProposal(roundIdentifier, block, emptyList(), emptyList());
when(blockValidator.validateAndProcessBlock(
eq(protocolContext),
eq(block),
eq(HeaderValidationMode.LIGHT),
eq(HeaderValidationMode.FULL),
eq(false)))
.thenReturn(new BlockProcessingResult("Failed"));
when(blockValidator.validateBlock(eq(block)))
.thenReturn(new QbftBlockValidator.ValidationResult(false, Optional.empty()));
assertThat(payloadValidator.validate(proposal.getSignedPayload())).isFalse();
}
@@ -156,9 +119,10 @@ public class ProposalPayloadValidatorTest {
@Test
public void validationFailsWhenExpectedProposerDoesNotMatchPayloadsAuthor() {
final ProposalPayloadValidator payloadValidator =
new ProposalPayloadValidator(
Address.fromHexString("0x1"), roundIdentifier, blockValidator, protocolContext);
final Block block = ProposedBlockHelpers.createProposalBlock(emptyList(), roundIdentifier);
new ProposalPayloadValidator(Address.fromHexString("0x1"), roundIdentifier, blockValidator);
final BlockHeader header =
new BlockHeaderTestFixture().number(roundIdentifier.getSequenceNumber()).buildHeader();
final QbftBlock block = new QbftBlockTestFixture().blockHeader(header).build();
final Proposal proposal =
messageFactory.createProposal(roundIdentifier, block, emptyList(), emptyList());
@@ -169,10 +133,11 @@ public class ProposalPayloadValidatorTest {
@Test
public void validationFailsWhenMessageMismatchesExpectedRound() {
final ProposalPayloadValidator payloadValidator =
new ProposalPayloadValidator(
expectedProposer, roundIdentifier, blockValidator, protocolContext);
new ProposalPayloadValidator(expectedProposer, roundIdentifier, blockValidator);
final Block block = ProposedBlockHelpers.createProposalBlock(emptyList(), roundIdentifier);
final BlockHeader header =
new BlockHeaderTestFixture().number(roundIdentifier.getSequenceNumber()).buildHeader();
final QbftBlock block = new QbftBlockTestFixture().blockHeader(header).build();
final Proposal proposal =
messageFactory.createProposal(
ConsensusRoundHelpers.createFrom(roundIdentifier, 0, +1),
@@ -187,10 +152,11 @@ public class ProposalPayloadValidatorTest {
@Test
public void validationFailsWhenMessageMismatchesExpectedHeight() {
final ProposalPayloadValidator payloadValidator =
new ProposalPayloadValidator(
expectedProposer, roundIdentifier, blockValidator, protocolContext);
new ProposalPayloadValidator(expectedProposer, roundIdentifier, blockValidator);
final Block block = ProposedBlockHelpers.createProposalBlock(emptyList(), roundIdentifier);
final BlockHeader header =
new BlockHeaderTestFixture().number(roundIdentifier.getSequenceNumber()).buildHeader();
final QbftBlock block = new QbftBlockTestFixture().blockHeader(header).build();
final Proposal proposal =
messageFactory.createProposal(
ConsensusRoundHelpers.createFrom(roundIdentifier, +1, 0),
@@ -204,26 +170,16 @@ public class ProposalPayloadValidatorTest {
@Test
public void validationFailsForBlockWithIncorrectHeight() {
when(bftExtraDataCodec.encode(any())).thenReturn(Bytes.EMPTY);
final ProposalPayloadValidator payloadValidator =
new ProposalPayloadValidator(
expectedProposer, roundIdentifier, blockValidator, protocolContext);
final Block block =
ProposedBlockHelpers.createProposalBlock(
emptyList(),
ConsensusRoundHelpers.createFrom(roundIdentifier, +1, 0),
bftExtraDataCodec);
new ProposalPayloadValidator(expectedProposer, roundIdentifier, blockValidator);
final BlockHeader header =
new BlockHeaderTestFixture().number(roundIdentifier.getSequenceNumber() + 1).buildHeader();
final QbftBlock block = new QbftBlockTestFixture().blockHeader(header).build();
final Proposal proposal =
messageFactory.createProposal(roundIdentifier, block, emptyList(), emptyList());
when(blockValidator.validateAndProcessBlock(
eq(protocolContext),
eq(block),
eq(HeaderValidationMode.LIGHT),
eq(HeaderValidationMode.FULL),
eq(false)))
.thenReturn(new BlockProcessingResult(Optional.empty()));
when(blockValidator.validateBlock(eq(block)))
.thenReturn(new QbftBlockValidator.ValidationResult(true, Optional.empty()));
assertThat(payloadValidator.validate(proposal.getSignedPayload())).isFalse();
}

View File

@@ -16,46 +16,43 @@ package org.hyperledger.besu.consensus.qbft.core.validation;
import static java.util.Collections.emptyList;
import static org.assertj.core.api.Assertions.assertThat;
import static org.hyperledger.besu.consensus.common.bft.BftContextBuilder.setupContextWithBftExtraDataEncoder;
import static org.hyperledger.besu.consensus.qbft.core.validation.ValidationTestHelpers.createEmptyRoundChangePayloads;
import static org.hyperledger.besu.consensus.qbft.core.validation.ValidationTestHelpers.createPreparePayloads;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.ArgumentMatchers.eq;
import static org.mockito.Mockito.lenient;
import static org.mockito.Mockito.reset;
import static org.mockito.Mockito.when;
import org.hyperledger.besu.consensus.common.bft.BftContext;
import org.hyperledger.besu.consensus.common.bft.BftExtraData;
import org.hyperledger.besu.consensus.common.bft.BftExtraDataCodec;
import org.hyperledger.besu.consensus.common.bft.BftHelpers;
import org.hyperledger.besu.consensus.common.bft.BftProtocolSchedule;
import org.hyperledger.besu.consensus.common.bft.ConsensusRoundHelpers;
import org.hyperledger.besu.consensus.common.bft.ConsensusRoundIdentifier;
import org.hyperledger.besu.consensus.common.bft.ProposedBlockHelpers;
import org.hyperledger.besu.consensus.common.bft.payload.SignedData;
import org.hyperledger.besu.consensus.common.validator.ValidatorProvider;
import org.hyperledger.besu.consensus.qbft.core.QbftBlockTestFixture;
import org.hyperledger.besu.consensus.qbft.core.messagewrappers.Prepare;
import org.hyperledger.besu.consensus.qbft.core.messagewrappers.Proposal;
import org.hyperledger.besu.consensus.qbft.core.messagewrappers.RoundChange;
import org.hyperledger.besu.consensus.qbft.core.payload.PreparePayload;
import org.hyperledger.besu.consensus.qbft.core.payload.PreparedRoundMetadata;
import org.hyperledger.besu.consensus.qbft.core.payload.RoundChangePayload;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlock;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockCodec;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockInterface;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockValidator;
import org.hyperledger.besu.consensus.qbft.core.types.QbftContext;
import org.hyperledger.besu.consensus.qbft.core.types.QbftHashMode;
import org.hyperledger.besu.consensus.qbft.core.types.QbftProtocolSchedule;
import org.hyperledger.besu.consensus.qbft.core.types.QbftProtocolSpec;
import org.hyperledger.besu.datatypes.Hash;
import org.hyperledger.besu.ethereum.BlockProcessingResult;
import org.hyperledger.besu.ethereum.BlockValidator;
import org.hyperledger.besu.ethereum.ProtocolContext;
import org.hyperledger.besu.ethereum.chain.BadBlockManager;
import org.hyperledger.besu.ethereum.chain.MutableBlockchain;
import org.hyperledger.besu.ethereum.core.Block;
import org.hyperledger.besu.ethereum.mainnet.HeaderValidationMode;
import org.hyperledger.besu.ethereum.mainnet.ProtocolSpec;
import org.hyperledger.besu.ethereum.worldstate.WorldStateArchive;
import org.hyperledger.besu.ethereum.core.BlockHeader;
import org.hyperledger.besu.ethereum.core.BlockHeaderTestFixture;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import org.apache.tuweni.bytes.Bytes;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.extension.ExtendWith;
@@ -72,12 +69,12 @@ public class ProposalValidatorTest {
private static class RoundSpecificItems {
public final Block block;
public final QbftBlock block;
public final ConsensusRoundIdentifier roundIdentifier;
public final ProposalValidator messageValidator;
public RoundSpecificItems(
final Block block,
final QbftBlock block,
final ConsensusRoundIdentifier roundIdentifier,
final ProposalValidator messageValidator) {
this.block = block;
@@ -87,50 +84,45 @@ public class ProposalValidatorTest {
}
private static final int VALIDATOR_COUNT = 4;
private static final QbftNodeList validators = QbftNodeList.createNodes(VALIDATOR_COUNT);
@Mock private BlockValidator blockValidator;
@Mock private MutableBlockchain blockChain;
@Mock private WorldStateArchive worldStateArchive;
@Mock private BftProtocolSchedule protocolSchedule;
@Mock private ProtocolSpec protocolSpec;
@Mock private BftExtraDataCodec bftExtraDataCodec;
private ProtocolContext protocolContext;
@Mock private QbftBlockValidator blockValidator;
@Mock private QbftProtocolSchedule protocolSchedule;
@Mock private QbftProtocolSpec protocolSpec;
@Mock private QbftBlockCodec blockEncoder;
@Mock private QbftBlockInterface blockInterface;
@Mock private ValidatorProvider validatorProvider;
@Mock private ProtocolContext protocolContext;
private QbftNodeList validators;
private final Map<ROUND_ID, RoundSpecificItems> roundItems = new HashMap<>();
@BeforeEach
public void setup() {
protocolContext =
new ProtocolContext(
blockChain,
worldStateArchive,
setupContextWithBftExtraDataEncoder(BftContext.class, emptyList(), bftExtraDataCodec),
new BadBlockManager());
validators = QbftNodeList.createNodes(VALIDATOR_COUNT, blockEncoder);
// typically tests require the blockValidation to be successful
when(blockValidator.validateAndProcessBlock(
eq(protocolContext),
any(),
eq(HeaderValidationMode.LIGHT),
eq(HeaderValidationMode.FULL),
eq(false)))
.thenReturn(new BlockProcessingResult(Optional.empty()));
when(blockValidator.validateBlock(any()))
.thenReturn(new QbftBlockValidator.ValidationResult(true, Optional.empty()));
QbftContext qbftContext = new QbftContext(validatorProvider, blockInterface);
lenient().when(protocolContext.getConsensusContext(QbftContext.class)).thenReturn(qbftContext);
when(protocolSchedule.getByBlockHeader(any())).thenReturn(protocolSpec);
when(protocolSpec.getBlockValidator()).thenReturn(blockValidator);
when(bftExtraDataCodec.encode(any())).thenReturn(Bytes.EMPTY);
roundItems.put(ROUND_ID.ZERO, createRoundSpecificItems(0));
roundItems.put(ROUND_ID.ONE, createRoundSpecificItems(1));
}
private RoundSpecificItems createRoundSpecificItems(final int roundNumber) {
final ConsensusRoundIdentifier roundIdentifier = new ConsensusRoundIdentifier(1, roundNumber);
final BlockHeader blockHeader =
new BlockHeaderTestFixture()
.number(roundIdentifier.getSequenceNumber())
.coinbase(validators.getNodeAddresses().getFirst())
.buildHeader();
final QbftBlock block = new QbftBlockTestFixture().blockHeader(blockHeader).build();
return new RoundSpecificItems(
ProposedBlockHelpers.createProposalBlock(
validators.getNodeAddresses(), roundIdentifier, bftExtraDataCodec),
block,
roundIdentifier,
new ProposalValidator(
protocolContext,
@@ -138,8 +130,7 @@ public class ProposalValidatorTest {
BftHelpers.calculateRequiredValidatorQuorum(VALIDATOR_COUNT),
validators.getNodeAddresses(),
roundIdentifier,
validators.getNode(0).getAddress(),
bftExtraDataCodec));
validators.getNode(0).getAddress()));
}
// NOTE: tests herein assume the ProposalPayloadValidator works as expected, so other than
@@ -167,23 +158,15 @@ public class ProposalValidatorTest {
final Proposal proposal = createProposal(roundItem, emptyList(), emptyList());
reset(blockValidator);
when(blockValidator.validateAndProcessBlock(
eq(protocolContext),
any(),
eq(HeaderValidationMode.LIGHT),
eq(HeaderValidationMode.FULL),
eq(false)))
.thenReturn(new BlockProcessingResult("Failed"));
when(blockValidator.validateBlock(any()))
.thenReturn(new QbftBlockValidator.ValidationResult(false, Optional.of("Failed")));
assertThat(roundItem.messageValidator.validate(proposal)).isFalse();
}
@Test
public void validationFailsIfRoundZeroHasNonEmptyPrepares() {
when(bftExtraDataCodec.encodeWithoutCommitSeals(any())).thenReturn(Bytes.EMPTY);
when(bftExtraDataCodec.decode(any()))
.thenReturn(new BftExtraData(Bytes.EMPTY, emptyList(), Optional.empty(), 0, emptyList()));
final Prepare prepareMsg =
validators
.getMessageFactory(1)
@@ -261,7 +244,7 @@ public class ProposalValidatorTest {
@Test
public void validationFailsIfPiggybackedRoundChangePayloadIsFromNonValidation() {
final RoundSpecificItems roundItem = roundItems.get(ROUND_ID.ONE);
final QbftNode nonValidatorNode = QbftNode.create();
final QbftNode nonValidatorNode = QbftNode.create(blockEncoder);
final List<SignedData<RoundChangePayload>> roundChanges =
createEmptyRoundChangePayloads(
@@ -341,6 +324,8 @@ public class ProposalValidatorTest {
final List<SignedData<RoundChangePayload>> roundChanges =
createEmptyRoundChangePayloads(
roundItem.roundIdentifier, validators.getNode(0), validators.getNode(1));
when(blockInterface.replaceRoundInBlock(roundItem.block, 0, QbftHashMode.COMMITTED_SEAL))
.thenReturn(roundItems.get(ROUND_ID.ZERO).block);
final RoundChangePayload illegalPayload =
new RoundChangePayload(
@@ -357,10 +342,6 @@ public class ProposalValidatorTest {
roundChanges.add(preparedRoundChange);
when(bftExtraDataCodec.encodeWithoutCommitSeals(any())).thenReturn(Bytes.EMPTY);
when(bftExtraDataCodec.decode(any()))
.thenReturn(new BftExtraData(Bytes.EMPTY, emptyList(), Optional.empty(), 0, emptyList()));
final Proposal proposal =
validators
.getMessageFactory(0)
@@ -406,14 +387,12 @@ public class ProposalValidatorTest {
@Test
public void validationFailsIfPiggybackedPreparePayloadIsFromNonValidator() {
when(bftExtraDataCodec.encodeWithoutCommitSeals(any())).thenReturn(Bytes.EMPTY);
when(bftExtraDataCodec.decode(any()))
.thenReturn(new BftExtraData(Bytes.EMPTY, emptyList(), Optional.empty(), 0, emptyList()));
final RoundSpecificItems roundItem = roundItems.get(ROUND_ID.ONE);
final List<SignedData<RoundChangePayload>> roundChanges = createPreparedRoundZeroRoundChanges();
when(blockInterface.replaceRoundInBlock(roundItem.block, 0, QbftHashMode.COMMITTED_SEAL))
.thenReturn(roundItems.get(ROUND_ID.ZERO).block);
final QbftNode nonValidator = QbftNode.create();
final QbftNode nonValidator = QbftNode.create(blockEncoder);
final Proposal proposal =
validators
.getMessageFactory(0)
@@ -433,12 +412,10 @@ public class ProposalValidatorTest {
@Test
public void validationFailsIfPiggybackedPreparePayloadHasDuplicatedAuthors() {
when(bftExtraDataCodec.encodeWithoutCommitSeals(any())).thenReturn(Bytes.EMPTY);
when(bftExtraDataCodec.decode(any()))
.thenReturn(new BftExtraData(Bytes.EMPTY, emptyList(), Optional.empty(), 0, emptyList()));
final RoundSpecificItems roundItem = roundItems.get(ROUND_ID.ONE);
final List<SignedData<RoundChangePayload>> roundChanges = createPreparedRoundZeroRoundChanges();
when(blockInterface.replaceRoundInBlock(roundItem.block, 0, QbftHashMode.COMMITTED_SEAL))
.thenReturn(roundItems.get(ROUND_ID.ZERO).block);
final Proposal proposal =
validators
@@ -459,12 +436,10 @@ public class ProposalValidatorTest {
@Test
public void validationFailsIfInsufficientPiggybackedPreparePayloads() {
when(bftExtraDataCodec.encodeWithoutCommitSeals(any())).thenReturn(Bytes.EMPTY);
when(bftExtraDataCodec.decode(any()))
.thenReturn(new BftExtraData(Bytes.EMPTY, emptyList(), Optional.empty(), 0, emptyList()));
final RoundSpecificItems roundItem = roundItems.get(ROUND_ID.ONE);
final List<SignedData<RoundChangePayload>> roundChanges = createPreparedRoundZeroRoundChanges();
when(blockInterface.replaceRoundInBlock(roundItem.block, 0, QbftHashMode.COMMITTED_SEAL))
.thenReturn(roundItems.get(ROUND_ID.ZERO).block);
final Proposal proposal =
validators
@@ -484,12 +459,10 @@ public class ProposalValidatorTest {
@Test
public void validationFailsIfPreparePayloadsDoNotMatchMetadataInRoundChanges() {
when(bftExtraDataCodec.encodeWithoutCommitSeals(any())).thenReturn(Bytes.EMPTY);
when(bftExtraDataCodec.decode(any()))
.thenReturn(new BftExtraData(Bytes.EMPTY, emptyList(), Optional.empty(), 0, emptyList()));
final RoundSpecificItems roundItem = roundItems.get(ROUND_ID.ONE);
final List<SignedData<RoundChangePayload>> roundChanges = createPreparedRoundZeroRoundChanges();
when(blockInterface.replaceRoundInBlock(roundItem.block, 0, QbftHashMode.COMMITTED_SEAL))
.thenReturn(roundItems.get(ROUND_ID.ZERO).block);
final Proposal proposal =
validators
@@ -510,12 +483,10 @@ public class ProposalValidatorTest {
@Test
public void validationFailsIfPreparePayloadsDoNotMatchBlockHashInRoundChanges() {
when(bftExtraDataCodec.encodeWithoutCommitSeals(any())).thenReturn(Bytes.EMPTY);
when(bftExtraDataCodec.decode(any()))
.thenReturn(new BftExtraData(Bytes.EMPTY, emptyList(), Optional.empty(), 0, emptyList()));
final RoundSpecificItems roundItem = roundItems.get(ROUND_ID.ONE);
final List<SignedData<RoundChangePayload>> roundChanges = createPreparedRoundZeroRoundChanges();
when(blockInterface.replaceRoundInBlock(roundItem.block, 0, QbftHashMode.COMMITTED_SEAL))
.thenReturn(roundItems.get(ROUND_ID.ZERO).block);
final Proposal proposal =
validators
@@ -536,12 +507,10 @@ public class ProposalValidatorTest {
@Test
public void validationFailsIfTwoRoundChangesArePreparedOnSameRoundDifferentBlock() {
when(bftExtraDataCodec.encodeWithoutCommitSeals(any())).thenReturn(Bytes.EMPTY);
when(bftExtraDataCodec.decode(any()))
.thenReturn(new BftExtraData(Bytes.EMPTY, emptyList(), Optional.empty(), 0, emptyList()));
final RoundSpecificItems roundItem = roundItems.get(ROUND_ID.ONE);
final List<SignedData<RoundChangePayload>> roundChanges = createPreparedRoundZeroRoundChanges();
when(blockInterface.replaceRoundInBlock(roundItem.block, 0, QbftHashMode.COMMITTED_SEAL))
.thenReturn(roundItems.get(ROUND_ID.ZERO).block);
final RoundChangePayload illegalPreparedRoundChangePayload =
new RoundChangePayload(

View File

@@ -15,6 +15,7 @@
package org.hyperledger.besu.consensus.qbft.core.validation;
import org.hyperledger.besu.consensus.qbft.core.payload.MessageFactory;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockCodec;
import org.hyperledger.besu.cryptoservices.NodeKey;
import org.hyperledger.besu.cryptoservices.NodeKeyUtils;
import org.hyperledger.besu.datatypes.Address;
@@ -42,9 +43,9 @@ public class QbftNode {
return nodeKey;
}
public static QbftNode create() {
public static QbftNode create(final QbftBlockCodec qbftBlockCodec) {
final NodeKey nodeKey = NodeKeyUtils.generate();
final MessageFactory factory = new MessageFactory(nodeKey);
final MessageFactory factory = new MessageFactory(nodeKey, qbftBlockCodec);
return new QbftNode(nodeKey, factory);
}

View File

@@ -15,6 +15,7 @@
package org.hyperledger.besu.consensus.qbft.core.validation;
import org.hyperledger.besu.consensus.qbft.core.payload.MessageFactory;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockCodec;
import org.hyperledger.besu.datatypes.Address;
import java.util.Collection;
@@ -25,11 +26,11 @@ import com.google.common.collect.Lists;
public class QbftNodeList {
public static QbftNodeList createNodes(final int count) {
public static QbftNodeList createNodes(final int count, final QbftBlockCodec blockEncoder) {
final List<QbftNode> nodes = Lists.newArrayList();
for (int i = 0; i < count; i++) {
nodes.add(QbftNode.create());
nodes.add(QbftNode.create(blockEncoder));
}
return new QbftNodeList(nodes);
}

View File

@@ -17,41 +17,35 @@ package org.hyperledger.besu.consensus.qbft.core.validation;
import static com.google.common.collect.Iterables.toArray;
import static java.util.Collections.emptyList;
import static org.assertj.core.api.Assertions.assertThat;
import static org.hyperledger.besu.consensus.common.bft.BftContextBuilder.setupContextWithBftExtraDataEncoder;
import static org.hyperledger.besu.consensus.qbft.core.validation.ValidationTestHelpers.createPreparePayloads;
import static org.hyperledger.besu.consensus.qbft.core.validation.ValidationTestHelpers.createPreparedCertificate;
import static org.mockito.Mockito.any;
import static org.mockito.Mockito.lenient;
import static org.mockito.Mockito.when;
import org.hyperledger.besu.consensus.common.bft.BftContext;
import org.hyperledger.besu.consensus.common.bft.BftExtraDataCodec;
import org.hyperledger.besu.consensus.common.bft.BftHelpers;
import org.hyperledger.besu.consensus.common.bft.BftProtocolSchedule;
import org.hyperledger.besu.consensus.common.bft.ConsensusRoundHelpers;
import org.hyperledger.besu.consensus.common.bft.ConsensusRoundIdentifier;
import org.hyperledger.besu.consensus.common.bft.ProposedBlockHelpers;
import org.hyperledger.besu.consensus.common.bft.payload.SignedData;
import org.hyperledger.besu.consensus.qbft.core.QbftBlockTestFixture;
import org.hyperledger.besu.consensus.qbft.core.messagewrappers.RoundChange;
import org.hyperledger.besu.consensus.qbft.core.payload.PreparedRoundMetadata;
import org.hyperledger.besu.consensus.qbft.core.payload.RoundChangePayload;
import org.hyperledger.besu.consensus.qbft.core.statemachine.PreparedCertificate;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlock;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockCodec;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockValidator;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockValidator.ValidationResult;
import org.hyperledger.besu.consensus.qbft.core.types.QbftProtocolSchedule;
import org.hyperledger.besu.consensus.qbft.core.types.QbftProtocolSpec;
import org.hyperledger.besu.crypto.SECPSignature;
import org.hyperledger.besu.datatypes.Hash;
import org.hyperledger.besu.ethereum.BlockProcessingResult;
import org.hyperledger.besu.ethereum.BlockValidator;
import org.hyperledger.besu.ethereum.ProtocolContext;
import org.hyperledger.besu.ethereum.chain.BadBlockManager;
import org.hyperledger.besu.ethereum.chain.MutableBlockchain;
import org.hyperledger.besu.ethereum.core.Block;
import org.hyperledger.besu.ethereum.mainnet.ProtocolSpec;
import org.hyperledger.besu.ethereum.worldstate.WorldStateArchive;
import org.hyperledger.besu.ethereum.core.BlockHeader;
import org.hyperledger.besu.ethereum.core.BlockHeaderTestFixture;
import java.util.Collections;
import java.util.Optional;
import java.util.stream.Collectors;
import org.apache.tuweni.bytes.Bytes;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.extension.ExtendWith;
@@ -62,17 +56,14 @@ import org.mockito.junit.jupiter.MockitoExtension;
public class RoundChangeMessageValidatorTest {
@Mock private RoundChangePayloadValidator payloadValidator;
@Mock private MutableBlockchain blockChain;
@Mock private WorldStateArchive worldStateArchive;
@Mock private BftProtocolSchedule protocolSchedule;
@Mock private BlockValidator blockValidator;
@Mock private ProtocolSpec protocolSpec;
@Mock private BftExtraDataCodec bftExtraDataCodec;
private ProtocolContext protocolContext;
@Mock private QbftProtocolSchedule protocolSchedule;
@Mock private QbftBlockValidator blockValidator;
@Mock private QbftProtocolSpec protocolSpec;
@Mock private QbftBlockCodec blockEncoder;
private RoundChangeMessageValidator messageValidator;
private QbftNodeList validators;
private static final int VALIDATOR_COUNT = 4;
private final QbftNodeList validators = QbftNodeList.createNodes(VALIDATOR_COUNT);
private static final int CHAIN_HEIGHT = 3;
private final ConsensusRoundIdentifier targetRound =
new ConsensusRoundIdentifier(CHAIN_HEIGHT, 3);
@@ -81,15 +72,9 @@ public class RoundChangeMessageValidatorTest {
@BeforeEach
public void setup() {
protocolContext =
new ProtocolContext(
blockChain,
worldStateArchive,
setupContextWithBftExtraDataEncoder(BftContext.class, emptyList(), bftExtraDataCodec),
new BadBlockManager());
validators = QbftNodeList.createNodes(VALIDATOR_COUNT, blockEncoder);
lenient().when(protocolSchedule.getByBlockHeader(any())).thenReturn(protocolSpec);
lenient().when(protocolSpec.getBlockValidator()).thenReturn(blockValidator);
}
@@ -102,7 +87,6 @@ public class RoundChangeMessageValidatorTest {
BftHelpers.calculateRequiredValidatorQuorum(VALIDATOR_COUNT),
CHAIN_HEIGHT,
validators.getNodeAddresses(),
protocolContext,
protocolSchedule);
for (int i = 0; i < VALIDATOR_COUNT; i++) {
@@ -115,22 +99,19 @@ public class RoundChangeMessageValidatorTest {
@Test
public void roundChangeWithValidPiggyBackDataIsValid() {
when(payloadValidator.validate(any())).thenReturn(true);
when(blockValidator.validateAndProcessBlock(any(), any(), any(), any()))
.thenReturn(new BlockProcessingResult(Optional.empty()));
when(bftExtraDataCodec.encode(any())).thenReturn(Bytes.EMPTY);
when(bftExtraDataCodec.encodeWithoutCommitSeals(any())).thenReturn(Bytes.EMPTY);
when(blockValidator.validateBlock(any()))
.thenReturn(new ValidationResult(true, Optional.empty()));
messageValidator =
new RoundChangeMessageValidator(
payloadValidator,
BftHelpers.calculateRequiredValidatorQuorum(VALIDATOR_COUNT),
CHAIN_HEIGHT,
validators.getNodeAddresses(),
protocolContext,
protocolSchedule);
final Block block =
ProposedBlockHelpers.createProposalBlock(
Collections.emptyList(), roundIdentifier, bftExtraDataCodec);
final BlockHeader header =
new BlockHeaderTestFixture().number(roundIdentifier.getSequenceNumber()).buildHeader();
final QbftBlock block = new QbftBlockTestFixture().blockHeader(header).build();
final PreparedCertificate prepCert =
createPreparedCertificate(
block, roundIdentifier, toArray(validators.getNodes(), QbftNode.class));
@@ -143,22 +124,19 @@ public class RoundChangeMessageValidatorTest {
@Test
public void roundChangeWithBlockRoundMismatchingPreparesIsValid() {
when(payloadValidator.validate(any())).thenReturn(true);
when(blockValidator.validateAndProcessBlock(any(), any(), any(), any()))
.thenReturn(new BlockProcessingResult(Optional.empty()));
when(bftExtraDataCodec.encode(any())).thenReturn(Bytes.EMPTY);
when(bftExtraDataCodec.encodeWithoutCommitSeals(any())).thenReturn(Bytes.EMPTY);
when(blockValidator.validateBlock(any()))
.thenReturn(new ValidationResult(true, Optional.empty()));
messageValidator =
new RoundChangeMessageValidator(
payloadValidator,
BftHelpers.calculateRequiredValidatorQuorum(VALIDATOR_COUNT),
CHAIN_HEIGHT,
validators.getNodeAddresses(),
protocolContext,
protocolSchedule);
final Block block =
ProposedBlockHelpers.createProposalBlock(
Collections.emptyList(), roundIdentifier, bftExtraDataCodec);
final BlockHeader header =
new BlockHeaderTestFixture().number(roundIdentifier.getSequenceNumber()).buildHeader();
final QbftBlock block = new QbftBlockTestFixture().blockHeader(header).build();
final PreparedCertificate prepCert =
createPreparedCertificate(
block,
@@ -173,19 +151,19 @@ public class RoundChangeMessageValidatorTest {
@Test
public void blockIsInvalidFailsValidation() {
when(payloadValidator.validate(any())).thenReturn(true);
when(blockValidator.validateAndProcessBlock(any(), any(), any(), any()))
.thenReturn(BlockProcessingResult.FAILED);
when(blockValidator.validateBlock(any()))
.thenReturn(new ValidationResult(false, Optional.empty()));
messageValidator =
new RoundChangeMessageValidator(
payloadValidator,
BftHelpers.calculateRequiredValidatorQuorum(VALIDATOR_COUNT),
CHAIN_HEIGHT,
validators.getNodeAddresses(),
protocolContext,
protocolSchedule);
final Block block =
ProposedBlockHelpers.createProposalBlock(Collections.emptyList(), roundIdentifier);
final BlockHeader header =
new BlockHeaderTestFixture().number(roundIdentifier.getSequenceNumber()).buildHeader();
final QbftBlock block = new QbftBlockTestFixture().blockHeader(header).build();
final PreparedCertificate prepCert =
createPreparedCertificate(
block, roundIdentifier, toArray(validators.getNodes(), QbftNode.class));
@@ -204,7 +182,6 @@ public class RoundChangeMessageValidatorTest {
BftHelpers.calculateRequiredValidatorQuorum(VALIDATOR_COUNT),
CHAIN_HEIGHT,
validators.getNodeAddresses(),
protocolContext,
protocolSchedule);
final RoundChange message =
@@ -215,22 +192,19 @@ public class RoundChangeMessageValidatorTest {
@Test
public void insufficientPiggyBackedPrepareMessagesIsInvalid() {
when(payloadValidator.validate(any())).thenReturn(true);
when(blockValidator.validateAndProcessBlock(any(), any(), any(), any()))
.thenReturn(new BlockProcessingResult(Optional.empty()));
when(bftExtraDataCodec.encode(any())).thenReturn(Bytes.EMPTY);
when(bftExtraDataCodec.encodeWithoutCommitSeals(any())).thenReturn(Bytes.EMPTY);
when(blockValidator.validateBlock(any()))
.thenReturn(new ValidationResult(true, Optional.empty()));
messageValidator =
new RoundChangeMessageValidator(
payloadValidator,
BftHelpers.calculateRequiredValidatorQuorum(VALIDATOR_COUNT),
CHAIN_HEIGHT,
validators.getNodeAddresses(),
protocolContext,
protocolSchedule);
final Block block =
ProposedBlockHelpers.createProposalBlock(
Collections.emptyList(), roundIdentifier, bftExtraDataCodec);
final BlockHeader header =
new BlockHeaderTestFixture().number(roundIdentifier.getSequenceNumber()).buildHeader();
final QbftBlock block = new QbftBlockTestFixture().blockHeader(header).build();
final PreparedCertificate prepCert =
createPreparedCertificate(
block, roundIdentifier, validators.getNode(0), validators.getNode(1));
@@ -243,24 +217,21 @@ public class RoundChangeMessageValidatorTest {
@Test
public void prepareFromNonValidatorFails() {
when(payloadValidator.validate(any())).thenReturn(true);
when(blockValidator.validateAndProcessBlock(any(), any(), any(), any()))
.thenReturn(new BlockProcessingResult(Optional.empty()));
when(bftExtraDataCodec.encode(any())).thenReturn(Bytes.EMPTY);
when(bftExtraDataCodec.encodeWithoutCommitSeals(any())).thenReturn(Bytes.EMPTY);
when(blockValidator.validateBlock(any()))
.thenReturn(new ValidationResult(true, Optional.empty()));
messageValidator =
new RoundChangeMessageValidator(
payloadValidator,
BftHelpers.calculateRequiredValidatorQuorum(VALIDATOR_COUNT),
CHAIN_HEIGHT,
validators.getNodeAddresses(),
protocolContext,
protocolSchedule);
final QbftNode nonValidator = QbftNode.create();
final QbftNode nonValidator = QbftNode.create(blockEncoder);
final Block block =
ProposedBlockHelpers.createProposalBlock(
Collections.emptyList(), roundIdentifier, bftExtraDataCodec);
final BlockHeader header =
new BlockHeaderTestFixture().number(roundIdentifier.getSequenceNumber()).buildHeader();
final QbftBlock block = new QbftBlockTestFixture().blockHeader(header).build();
final PreparedCertificate prepCert =
createPreparedCertificate(
block, roundIdentifier, validators.getNode(0), validators.getNode(1), nonValidator);
@@ -273,22 +244,19 @@ public class RoundChangeMessageValidatorTest {
@Test
public void validationFailsIfPreparedMetadataContainsDifferentRoundToBlock() {
when(payloadValidator.validate(any())).thenReturn(true);
when(blockValidator.validateAndProcessBlock(any(), any(), any(), any()))
.thenReturn(new BlockProcessingResult(Optional.empty()));
when(bftExtraDataCodec.encode(any())).thenReturn(Bytes.EMPTY);
when(bftExtraDataCodec.encodeWithoutCommitSeals(any())).thenReturn(Bytes.EMPTY);
when(blockValidator.validateBlock(any()))
.thenReturn(new ValidationResult(true, Optional.empty()));
messageValidator =
new RoundChangeMessageValidator(
payloadValidator,
BftHelpers.calculateRequiredValidatorQuorum(VALIDATOR_COUNT),
CHAIN_HEIGHT,
validators.getNodeAddresses(),
protocolContext,
protocolSchedule);
final Block block =
ProposedBlockHelpers.createProposalBlock(
Collections.emptyList(), roundIdentifier, bftExtraDataCodec);
final BlockHeader header =
new BlockHeaderTestFixture().number(roundIdentifier.getSequenceNumber()).buildHeader();
final QbftBlock block = new QbftBlockTestFixture().blockHeader(header).build();
final PreparedCertificate prepCert =
new PreparedCertificate(
block,
@@ -309,22 +277,19 @@ public class RoundChangeMessageValidatorTest {
@Test
public void validationFailsIfPreparesContainsDifferentRoundToBlock() {
when(payloadValidator.validate(any())).thenReturn(true);
when(blockValidator.validateAndProcessBlock(any(), any(), any(), any()))
.thenReturn(new BlockProcessingResult(Optional.empty()));
when(bftExtraDataCodec.encode(any())).thenReturn(Bytes.EMPTY);
when(bftExtraDataCodec.encodeWithoutCommitSeals(any())).thenReturn(Bytes.EMPTY);
when(blockValidator.validateBlock(any()))
.thenReturn(new ValidationResult(true, Optional.empty()));
messageValidator =
new RoundChangeMessageValidator(
payloadValidator,
BftHelpers.calculateRequiredValidatorQuorum(VALIDATOR_COUNT),
CHAIN_HEIGHT,
validators.getNodeAddresses(),
protocolContext,
protocolSchedule);
final Block block =
ProposedBlockHelpers.createProposalBlock(
Collections.emptyList(), roundIdentifier, bftExtraDataCodec);
final BlockHeader header =
new BlockHeaderTestFixture().number(roundIdentifier.getSequenceNumber()).buildHeader();
final QbftBlock block = new QbftBlockTestFixture().blockHeader(header).build();
final PreparedCertificate prepCert =
new PreparedCertificate(
block,
@@ -347,22 +312,19 @@ public class RoundChangeMessageValidatorTest {
@Test
public void validationFailsIfPreparesContainsWrongHeight() {
when(payloadValidator.validate(any())).thenReturn(true);
when(blockValidator.validateAndProcessBlock(any(), any(), any(), any()))
.thenReturn(new BlockProcessingResult(Optional.empty()));
when(bftExtraDataCodec.encode(any())).thenReturn(Bytes.EMPTY);
when(bftExtraDataCodec.encodeWithoutCommitSeals(any())).thenReturn(Bytes.EMPTY);
when(blockValidator.validateBlock(any()))
.thenReturn(new ValidationResult(true, Optional.empty()));
messageValidator =
new RoundChangeMessageValidator(
payloadValidator,
BftHelpers.calculateRequiredValidatorQuorum(VALIDATOR_COUNT),
CHAIN_HEIGHT,
validators.getNodeAddresses(),
protocolContext,
protocolSchedule);
final Block block =
ProposedBlockHelpers.createProposalBlock(
Collections.emptyList(), roundIdentifier, bftExtraDataCodec);
final BlockHeader header =
new BlockHeaderTestFixture().number(roundIdentifier.getSequenceNumber()).buildHeader();
final QbftBlock block = new QbftBlockTestFixture().blockHeader(header).build();
final PreparedCertificate prepCert =
new PreparedCertificate(
block,
@@ -385,22 +347,19 @@ public class RoundChangeMessageValidatorTest {
@Test
public void validationFailsIfPreparesHaveDuplicateAuthors() {
when(payloadValidator.validate(any())).thenReturn(true);
when(blockValidator.validateAndProcessBlock(any(), any(), any(), any()))
.thenReturn(new BlockProcessingResult(Optional.empty()));
when(bftExtraDataCodec.encode(any())).thenReturn(Bytes.EMPTY);
when(bftExtraDataCodec.encodeWithoutCommitSeals(any())).thenReturn(Bytes.EMPTY);
when(blockValidator.validateBlock(any()))
.thenReturn(new ValidationResult(true, Optional.empty()));
messageValidator =
new RoundChangeMessageValidator(
payloadValidator,
BftHelpers.calculateRequiredValidatorQuorum(VALIDATOR_COUNT),
CHAIN_HEIGHT,
validators.getNodeAddresses(),
protocolContext,
protocolSchedule);
final Block block =
ProposedBlockHelpers.createProposalBlock(
Collections.emptyList(), roundIdentifier, bftExtraDataCodec);
final BlockHeader header =
new BlockHeaderTestFixture().number(roundIdentifier.getSequenceNumber()).buildHeader();
final QbftBlock block = new QbftBlockTestFixture().blockHeader(header).build();
final PreparedCertificate prepCert =
createPreparedCertificate(
block,
@@ -418,26 +377,27 @@ public class RoundChangeMessageValidatorTest {
@Test
public void validationFailsIfBlockExistsButNotPreparedMetadata() {
when(payloadValidator.validate(any())).thenReturn(true);
when(blockValidator.validateAndProcessBlock(any(), any(), any(), any()))
.thenReturn(new BlockProcessingResult(Optional.empty()));
when(blockValidator.validateBlock(any()))
.thenReturn(new ValidationResult(true, Optional.empty()));
messageValidator =
new RoundChangeMessageValidator(
payloadValidator,
BftHelpers.calculateRequiredValidatorQuorum(VALIDATOR_COUNT),
CHAIN_HEIGHT,
validators.getNodeAddresses(),
protocolContext,
protocolSchedule);
final Block block =
ProposedBlockHelpers.createProposalBlock(Collections.emptyList(), roundIdentifier);
final BlockHeader header =
new BlockHeaderTestFixture().number(roundIdentifier.getSequenceNumber()).buildHeader();
final QbftBlock block = new QbftBlockTestFixture().blockHeader(header).build();
final RoundChangePayload payload = new RoundChangePayload(targetRound, Optional.empty());
final SECPSignature signature =
validators.getNode(0).getNodeKey().sign(payload.hashForSignature());
final RoundChange message =
new RoundChange(SignedData.create(payload, signature), Optional.of(block), emptyList());
new RoundChange(
SignedData.create(payload, signature), Optional.of(block), blockEncoder, emptyList());
assertThat(messageValidator.validate(message)).isFalse();
}
@@ -445,19 +405,19 @@ public class RoundChangeMessageValidatorTest {
@Test
public void validationFailsIfBlockHashDoesNotMatchPreparedMetadata() {
when(payloadValidator.validate(any())).thenReturn(true);
when(blockValidator.validateAndProcessBlock(any(), any(), any(), any()))
.thenReturn(new BlockProcessingResult(Optional.empty()));
when(blockValidator.validateBlock(any()))
.thenReturn(new ValidationResult(true, Optional.empty()));
messageValidator =
new RoundChangeMessageValidator(
payloadValidator,
BftHelpers.calculateRequiredValidatorQuorum(VALIDATOR_COUNT),
CHAIN_HEIGHT,
validators.getNodeAddresses(),
protocolContext,
protocolSchedule);
final Block block =
ProposedBlockHelpers.createProposalBlock(Collections.emptyList(), roundIdentifier);
final BlockHeader header =
new BlockHeaderTestFixture().number(roundIdentifier.getSequenceNumber()).buildHeader();
final QbftBlock block = new QbftBlockTestFixture().blockHeader(header).build();
final RoundChangePayload payload =
new RoundChangePayload(
@@ -472,6 +432,7 @@ public class RoundChangeMessageValidatorTest {
new RoundChange(
SignedData.create(payload, signature),
Optional.of(block),
blockEncoder,
createPreparePayloads(
roundIdentifier, block.getHash(), toArray(validators.getNodes(), QbftNode.class)));

View File

@@ -20,6 +20,7 @@ import org.hyperledger.besu.consensus.common.bft.ConsensusRoundIdentifier;
import org.hyperledger.besu.consensus.common.bft.payload.SignedData;
import org.hyperledger.besu.consensus.qbft.core.payload.PreparedRoundMetadata;
import org.hyperledger.besu.consensus.qbft.core.payload.RoundChangePayload;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockCodec;
import org.hyperledger.besu.crypto.SECPSignature;
import org.hyperledger.besu.cryptoservices.NodeKey;
import org.hyperledger.besu.cryptoservices.NodeKeyUtils;
@@ -27,17 +28,25 @@ import org.hyperledger.besu.datatypes.Hash;
import java.util.Optional;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.mockito.Mock;
public class RoundChangePayloadValidatorTest {
private static final int VALIDATOR_COUNT = 4;
private final QbftNodeList validators = QbftNodeList.createNodes(VALIDATOR_COUNT);
private final long chainHeight = 5L;
private final Hash preparedBlockHash = Hash.fromHexStringLenient("0x1");
final RoundChangePayloadValidator messageValidator =
new RoundChangePayloadValidator(validators.getNodeAddresses(), chainHeight);
private RoundChangePayloadValidator messageValidator;
private QbftNodeList validators;
@Mock QbftBlockCodec qbftBlockCodec;
@BeforeEach
public void setup() {
validators = QbftNodeList.createNodes(VALIDATOR_COUNT, qbftBlockCodec);
messageValidator = new RoundChangePayloadValidator(validators.getNodeAddresses(), chainHeight);
}
@Test
public void roundChangeIsValidIfItMatchesExpectedValues() {

View File

@@ -19,8 +19,8 @@ import org.hyperledger.besu.consensus.common.bft.payload.SignedData;
import org.hyperledger.besu.consensus.qbft.core.payload.PreparePayload;
import org.hyperledger.besu.consensus.qbft.core.payload.RoundChangePayload;
import org.hyperledger.besu.consensus.qbft.core.statemachine.PreparedCertificate;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlock;
import org.hyperledger.besu.datatypes.Hash;
import org.hyperledger.besu.ethereum.core.Block;
import java.util.List;
import java.util.Optional;
@@ -30,7 +30,7 @@ import java.util.stream.Stream;
public class ValidationTestHelpers {
public static PreparedCertificate createPreparedCertificate(
final Block block,
final QbftBlock block,
final ConsensusRoundIdentifier reportedRound,
final QbftNode... preparedNodes) {

View File

@@ -0,0 +1,44 @@
/*
* Copyright contributors to Besu.
*
* Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
* an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
* specific language governing permissions and limitations under the License.
*
* SPDX-License-Identifier: Apache-2.0
*/
package org.hyperledger.besu.consensus.qbft.adaptor;
import org.hyperledger.besu.consensus.common.bft.BftBlockHeaderFunctions;
import org.hyperledger.besu.consensus.qbft.QbftExtraDataCodec;
import org.hyperledger.besu.consensus.qbft.core.types.QbftHashMode;
import org.hyperledger.besu.ethereum.core.BlockHeaderFunctions;
/** Utility class to get the correct BlockHeaderFunctions based on the QbftHashMode. */
public class BlockHeaderFunctionsUtil {
// Private constructor to prevent instantiation
private BlockHeaderFunctionsUtil() {}
/**
* Get the correct BlockHeaderFunctions based on the QbftHashMode.
*
* @param extraDataCodec the extra data codec
* @param hashMode the hash mode
* @return the block header functions
*/
public static BlockHeaderFunctions getBlockHeaderFunctions(
final QbftExtraDataCodec extraDataCodec, final QbftHashMode hashMode) {
if (hashMode == QbftHashMode.ONCHAIN) {
return BftBlockHeaderFunctions.forOnchainBlock(extraDataCodec);
} else if (hashMode == QbftHashMode.COMMITTED_SEAL) {
return BftBlockHeaderFunctions.forCommittedSeal(extraDataCodec);
} else {
throw new IllegalStateException("Invalid HashMode");
}
}
}

View File

@@ -0,0 +1,39 @@
/*
* Copyright contributors to Besu.
*
* Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
* an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
* specific language governing permissions and limitations under the License.
*
* SPDX-License-Identifier: Apache-2.0
*/
package org.hyperledger.besu.consensus.qbft.adaptor;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlock;
import org.hyperledger.besu.ethereum.core.Block;
/** Utility class to convert between Besu and QBFT blocks. */
public class BlockUtil {
/** Private constructor to prevent instantiation. */
private BlockUtil() {}
/**
* Convert a QBFT block to a Besu block.
*
* @param block the QBFT block
* @return the Besu block
*/
public static Block toBesuBlock(final QbftBlock block) {
if (block instanceof QbftBlockAdaptor) {
return ((QbftBlockAdaptor) block).getBesuBlock();
} else {
throw new IllegalArgumentException("Unsupported block type");
}
}
}

View File

@@ -0,0 +1,69 @@
/*
* Copyright contributors to Besu.
*
* Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
* an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
* specific language governing permissions and limitations under the License.
*
* SPDX-License-Identifier: Apache-2.0
*/
package org.hyperledger.besu.consensus.qbft.adaptor;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlock;
import org.hyperledger.besu.ethereum.core.Block;
import org.hyperledger.besu.ethereum.core.BlockHeader;
import java.util.Objects;
/** Adaptor class to allow a {@link Block} to be used as a {@link QbftBlock}. */
public class QbftBlockAdaptor implements QbftBlock {
private final BlockHeader header;
private final Block block;
/**
* Constructs a QbftBlock from a Besu Block.
*
* @param block the Besu Block
*/
public QbftBlockAdaptor(final Block block) {
this.block = block;
this.header = block.getHeader();
}
@Override
public BlockHeader getHeader() {
return header;
}
@Override
public boolean isEmpty() {
return block.getBody().getTransactions().isEmpty();
}
/**
* Returns the Besu Block associated with this QbftBlock. Used to convert a QbftBlock back to a
* Besu block.
*
* @return the Besu Block
*/
public Block getBesuBlock() {
return block;
}
@Override
public boolean equals(final Object o) {
if (!(o instanceof QbftBlockAdaptor qbftBlock)) return false;
return Objects.equals(header, qbftBlock.header) && Objects.equals(block, qbftBlock.block);
}
@Override
public int hashCode() {
return Objects.hash(header, block);
}
}

View File

@@ -0,0 +1,54 @@
/*
* Copyright contributors to Besu.
*
* Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
* an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
* specific language governing permissions and limitations under the License.
*
* SPDX-License-Identifier: Apache-2.0
*/
package org.hyperledger.besu.consensus.qbft.adaptor;
import static org.hyperledger.besu.consensus.qbft.adaptor.BlockHeaderFunctionsUtil.getBlockHeaderFunctions;
import org.hyperledger.besu.consensus.qbft.QbftExtraDataCodec;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlock;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockCodec;
import org.hyperledger.besu.consensus.qbft.core.types.QbftHashMode;
import org.hyperledger.besu.ethereum.core.Block;
import org.hyperledger.besu.ethereum.core.BlockHeaderFunctions;
import org.hyperledger.besu.ethereum.rlp.RLPInput;
import org.hyperledger.besu.ethereum.rlp.RLPOutput;
/** Adaptor class to allow a {@link QbftExtraDataCodec} to be used as a {@link QbftBlockCodec}. */
public class QbftBlockCodecAdaptor implements QbftBlockCodec {
private final QbftExtraDataCodec qbftExtraDataCodec;
/**
* Constructs a new Qbft block codec.
*
* @param qbftExtraDataCodec the QBFT extra data codec
*/
public QbftBlockCodecAdaptor(final QbftExtraDataCodec qbftExtraDataCodec) {
this.qbftExtraDataCodec = qbftExtraDataCodec;
}
@Override
public QbftBlock readFrom(final RLPInput rlpInput, final QbftHashMode hashMode) {
final BlockHeaderFunctions blockHeaderFunctions =
getBlockHeaderFunctions(qbftExtraDataCodec, hashMode);
Block besuBlock = Block.readFrom(rlpInput, blockHeaderFunctions);
return new QbftBlockAdaptor(besuBlock);
}
@Override
public void writeTo(final QbftBlock block, final RLPOutput rlpOutput) {
BlockUtil.toBesuBlock(block).writeTo(rlpOutput);
}
}

View File

@@ -0,0 +1,81 @@
/*
* Copyright contributors to Besu.
*
* Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
* an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
* specific language governing permissions and limitations under the License.
*
* SPDX-License-Identifier: Apache-2.0
*/
package org.hyperledger.besu.consensus.qbft.adaptor;
import org.hyperledger.besu.consensus.common.bft.BftBlockHeaderFunctions;
import org.hyperledger.besu.consensus.common.bft.BftExtraData;
import org.hyperledger.besu.consensus.common.bft.BftExtraDataCodec;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlock;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockCreator;
import org.hyperledger.besu.consensus.qbft.core.types.QbftExtraDataProvider;
import org.hyperledger.besu.crypto.SECPSignature;
import org.hyperledger.besu.ethereum.blockcreation.BlockCreator;
import org.hyperledger.besu.ethereum.core.Block;
import org.hyperledger.besu.ethereum.core.BlockHeader;
import org.hyperledger.besu.ethereum.core.BlockHeaderBuilder;
import java.util.Collection;
/** Adaptor class to allow a {@link BlockCreator} to be used as a {@link QbftBlockCreator}. */
public class QbftBlockCreatorAdaptor implements QbftBlockCreator {
private final BlockCreator besuBlockCreator;
private final BftExtraDataCodec bftExtraDataCodec;
/**
* Constructs a new QbftBlockCreator
*
* @param besuBftBlockCreator the Besu BFT block creator
* @param bftExtraDataCodec the bftExtraDataCodec used to encode extra data for the new header
*/
public QbftBlockCreatorAdaptor(
final BlockCreator besuBftBlockCreator, final BftExtraDataCodec bftExtraDataCodec) {
this.besuBlockCreator = besuBftBlockCreator;
this.bftExtraDataCodec = bftExtraDataCodec;
}
@Override
public QbftBlock createBlock(final long headerTimeStampSeconds, final BlockHeader parentHeader) {
var blockResult = besuBlockCreator.createBlock(headerTimeStampSeconds, parentHeader);
return new QbftBlockAdaptor(blockResult.getBlock());
}
@Override
public QbftBlock createSealedBlock(
final QbftExtraDataProvider bftQbftExtraDataProvider,
final QbftBlock block,
final int roundNumber,
final Collection<SECPSignature> commitSeals) {
final Block besuBlock = BlockUtil.toBesuBlock(block);
final BlockHeader initialBesuHeader = besuBlock.getHeader();
final BftExtraData initialExtraData = bftQbftExtraDataProvider.getExtraData(initialBesuHeader);
final BftExtraData sealedExtraData =
new BftExtraData(
initialExtraData.getVanityData(),
commitSeals,
initialExtraData.getVote(),
roundNumber,
initialExtraData.getValidators());
final BlockHeader sealedHeader =
BlockHeaderBuilder.fromHeader(initialBesuHeader)
.extraData(bftExtraDataCodec.encode(sealedExtraData))
.blockHeaderFunctions(BftBlockHeaderFunctions.forOnchainBlock(bftExtraDataCodec))
.buildBlockHeader();
final Block sealedBesuBlock = new Block(sealedHeader, besuBlock.getBody());
return new QbftBlockAdaptor(sealedBesuBlock);
}
}

View File

@@ -0,0 +1,50 @@
/*
* Copyright contributors to Besu.
*
* Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
* an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
* specific language governing permissions and limitations under the License.
*
* SPDX-License-Identifier: Apache-2.0
*/
package org.hyperledger.besu.consensus.qbft.adaptor;
import org.hyperledger.besu.config.QbftConfigOptions;
import org.hyperledger.besu.consensus.common.bft.BftExtraDataCodec;
import org.hyperledger.besu.consensus.common.bft.blockcreation.BftBlockCreatorFactory;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockCreator;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockCreatorFactory;
/**
* Adaptor class to allow a {@link BftBlockCreatorFactory} to be used as a {@link
* QbftBlockCreatorFactory}.
*/
public class QbftBlockCreatorFactoryAdaptor implements QbftBlockCreatorFactory {
private final BftBlockCreatorFactory<QbftConfigOptions> qbftBlockCreatorFactory;
private final BftExtraDataCodec bftExtraDataCodec;
/**
* Constructs a new QbftBlockCreatorFactory
*
* @param bftBlockCreatorFactory The Besu QBFT block creator factory
* @param bftExtraDataCodec the bftExtraDataCodec used to encode extra data for the new header
*/
public QbftBlockCreatorFactoryAdaptor(
final BftBlockCreatorFactory<QbftConfigOptions> bftBlockCreatorFactory,
final BftExtraDataCodec bftExtraDataCodec) {
this.qbftBlockCreatorFactory = bftBlockCreatorFactory;
this.bftExtraDataCodec = bftExtraDataCodec;
}
@Override
public QbftBlockCreator create(final int roundNumber) {
return new QbftBlockCreatorAdaptor(
qbftBlockCreatorFactory.create(roundNumber), bftExtraDataCodec);
}
}

View File

@@ -0,0 +1,48 @@
/*
* Copyright contributors to Besu.
*
* Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
* an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
* specific language governing permissions and limitations under the License.
*
* SPDX-License-Identifier: Apache-2.0
*/
package org.hyperledger.besu.consensus.qbft.adaptor;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlock;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockImporter;
import org.hyperledger.besu.ethereum.ProtocolContext;
import org.hyperledger.besu.ethereum.core.BlockImporter;
import org.hyperledger.besu.ethereum.mainnet.BlockImportResult;
import org.hyperledger.besu.ethereum.mainnet.HeaderValidationMode;
/** Adaptor class to allow a {@link BlockImporter} to be used as a {@link QbftBlockImporter}. */
public class QbftBlockImporterAdaptor implements QbftBlockImporter {
private final BlockImporter blockImporter;
private final ProtocolContext context;
/**
* Constructs a new Qbft block importer.
*
* @param blockImporter The Besu block importer
* @param context The protocol context
*/
public QbftBlockImporterAdaptor(
final BlockImporter blockImporter, final ProtocolContext context) {
this.blockImporter = blockImporter;
this.context = context;
}
@Override
public boolean importBlock(final QbftBlock block) {
final BlockImportResult blockImportResult =
blockImporter.importBlock(context, BlockUtil.toBesuBlock(block), HeaderValidationMode.FULL);
return blockImportResult.isImported();
}
}

View File

@@ -0,0 +1,54 @@
/*
* Copyright contributors to Besu.
*
* Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
* an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
* specific language governing permissions and limitations under the License.
*
* SPDX-License-Identifier: Apache-2.0
*/
package org.hyperledger.besu.consensus.qbft.adaptor;
import static org.hyperledger.besu.consensus.qbft.adaptor.BlockHeaderFunctionsUtil.getBlockHeaderFunctions;
import static org.hyperledger.besu.consensus.qbft.adaptor.BlockUtil.toBesuBlock;
import org.hyperledger.besu.consensus.common.bft.BftBlockInterface;
import org.hyperledger.besu.consensus.qbft.QbftExtraDataCodec;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlock;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockInterface;
import org.hyperledger.besu.consensus.qbft.core.types.QbftHashMode;
import org.hyperledger.besu.ethereum.core.Block;
import org.hyperledger.besu.ethereum.core.BlockHeaderFunctions;
/**
* Adaptor class to allow a {@link BftBlockInterface} to be used as a {@link QbftBlockInterface}.
*/
public class QbftBlockInterfaceAdaptor implements QbftBlockInterface {
private final QbftExtraDataCodec bftExtraDataCodec = new QbftExtraDataCodec();
private final BftBlockInterface bftBlockInterface;
/**
* Constructs a new QbftBlockInterface
*
* @param bftBlockInterface the BFT block interface
*/
public QbftBlockInterfaceAdaptor(final BftBlockInterface bftBlockInterface) {
this.bftBlockInterface = bftBlockInterface;
}
@Override
public QbftBlock replaceRoundInBlock(
final QbftBlock proposalBlock, final int roundNumber, final QbftHashMode hashMode) {
final Block besuBlock = toBesuBlock(proposalBlock);
final BlockHeaderFunctions blockHeaderFunctions =
getBlockHeaderFunctions(bftExtraDataCodec, hashMode);
final Block updatedRoundBlock =
bftBlockInterface.replaceRoundInBlock(besuBlock, roundNumber, blockHeaderFunctions);
return new QbftBlockAdaptor(updatedRoundBlock);
}
}

View File

@@ -0,0 +1,54 @@
/*
* Copyright contributors to Besu.
*
* Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
* an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
* specific language governing permissions and limitations under the License.
*
* SPDX-License-Identifier: Apache-2.0
*/
package org.hyperledger.besu.consensus.qbft.adaptor;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlock;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockValidator;
import org.hyperledger.besu.ethereum.BlockProcessingResult;
import org.hyperledger.besu.ethereum.BlockValidator;
import org.hyperledger.besu.ethereum.ProtocolContext;
import org.hyperledger.besu.ethereum.mainnet.HeaderValidationMode;
/** Adaptor class to allow a {@link BlockValidator} to be used as a {@link QbftBlockValidator}. */
public class QbftBlockValidatorAdaptor implements QbftBlockValidator {
private final BlockValidator blockValidator;
private final ProtocolContext protocolContext;
/**
* Constructs a new Qbft block validator
*
* @param blockValidator The Besu block validator
* @param protocolContext The protocol context
*/
public QbftBlockValidatorAdaptor(
final BlockValidator blockValidator, final ProtocolContext protocolContext) {
this.blockValidator = blockValidator;
this.protocolContext = protocolContext;
}
@Override
public ValidationResult validateBlock(final QbftBlock block) {
final BlockProcessingResult blockProcessingResult =
blockValidator.validateAndProcessBlock(
protocolContext,
BlockUtil.toBesuBlock(block),
HeaderValidationMode.LIGHT,
HeaderValidationMode.FULL,
false);
return new ValidationResult(
blockProcessingResult.isSuccessful(), blockProcessingResult.errorMessage);
}
}

View File

@@ -0,0 +1,41 @@
/*
* Copyright contributors to Besu.
*
* Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
* an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
* specific language governing permissions and limitations under the License.
*
* SPDX-License-Identifier: Apache-2.0
*/
package org.hyperledger.besu.consensus.qbft.adaptor;
import org.hyperledger.besu.consensus.common.bft.BftExtraData;
import org.hyperledger.besu.consensus.common.bft.BftExtraDataCodec;
import org.hyperledger.besu.consensus.qbft.core.types.QbftExtraDataProvider;
import org.hyperledger.besu.ethereum.core.BlockHeader;
/**
* Adaptor class to allow a {@link BftExtraDataCodec} to be used as a {@link QbftExtraDataProvider}.
*/
public class QbftExtraDataProviderAdaptor implements QbftExtraDataProvider {
private final BftExtraDataCodec bftExtraDataCodec;
/**
* Constructs a new QbftExtraDataProvider
*
* @param bftExtraDataCodec the bftExtraDataCodec used to decode the extra data from the header
*/
public QbftExtraDataProviderAdaptor(final BftExtraDataCodec bftExtraDataCodec) {
this.bftExtraDataCodec = bftExtraDataCodec;
}
@Override
public BftExtraData getExtraData(final BlockHeader header) {
return bftExtraDataCodec.decode(header);
}
}

View File

@@ -0,0 +1,188 @@
/*
* Copyright contributors to Besu.
*
* Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
* an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
* specific language governing permissions and limitations under the License.
*
* SPDX-License-Identifier: Apache-2.0
*/
package org.hyperledger.besu.consensus.qbft.adaptor;
import org.hyperledger.besu.consensus.common.bft.BftHelpers;
import org.hyperledger.besu.consensus.common.bft.BlockTimer;
import org.hyperledger.besu.consensus.common.bft.ConsensusRoundIdentifier;
import org.hyperledger.besu.consensus.common.bft.RoundTimer;
import org.hyperledger.besu.consensus.common.bft.blockcreation.ProposerSelector;
import org.hyperledger.besu.consensus.common.bft.network.ValidatorMulticaster;
import org.hyperledger.besu.consensus.common.validator.ValidatorProvider;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockCreatorFactory;
import org.hyperledger.besu.consensus.qbft.core.types.QbftFinalState;
import org.hyperledger.besu.cryptoservices.NodeKey;
import org.hyperledger.besu.datatypes.Address;
import java.time.Clock;
import java.util.Collection;
/** Besu implementation of QbftFinalState for maintaining the state of a QBFT network. */
public class QbftFinalStateImpl implements QbftFinalState {
private final ValidatorProvider validatorProvider;
private final NodeKey nodeKey;
private final Address localAddress;
private final ProposerSelector proposerSelector;
private final ValidatorMulticaster validatorMulticaster;
private final RoundTimer roundTimer;
private final BlockTimer blockTimer;
private final QbftBlockCreatorFactory blockCreatorFactory;
private final Clock clock;
/**
* Constructs a new QBFT final state.
*
* @param validatorProvider the validator provider
* @param nodeKey the node key
* @param localAddress the local address
* @param proposerSelector the proposer selector
* @param validatorMulticaster the validator multicaster
* @param roundTimer the round timer
* @param blockTimer the block timer
* @param blockCreatorFactory the block creator factory
* @param clock the clock
*/
public QbftFinalStateImpl(
final ValidatorProvider validatorProvider,
final NodeKey nodeKey,
final Address localAddress,
final ProposerSelector proposerSelector,
final ValidatorMulticaster validatorMulticaster,
final RoundTimer roundTimer,
final BlockTimer blockTimer,
final QbftBlockCreatorFactory blockCreatorFactory,
final Clock clock) {
this.validatorProvider = validatorProvider;
this.nodeKey = nodeKey;
this.localAddress = localAddress;
this.proposerSelector = proposerSelector;
this.validatorMulticaster = validatorMulticaster;
this.roundTimer = roundTimer;
this.blockTimer = blockTimer;
this.blockCreatorFactory = blockCreatorFactory;
this.clock = clock;
}
/**
* Gets validators.
*
* @return the validators
*/
@Override
public Collection<Address> getValidators() {
return validatorProvider.getValidatorsAtHead();
}
/**
* Gets the validator multicaster.
*
* @return the validator multicaster
*/
@Override
public ValidatorMulticaster getValidatorMulticaster() {
return validatorMulticaster;
}
/**
* Gets node key.
*
* @return the node key
*/
@Override
public NodeKey getNodeKey() {
return nodeKey;
}
/**
* Gets local address.
*
* @return the local address
*/
@Override
public Address getLocalAddress() {
return localAddress;
}
/**
* Is local node validator.
*
* @return the boolean
*/
@Override
public boolean isLocalNodeValidator() {
return getValidators().contains(localAddress);
}
/**
* Gets round timer.
*
* @return the round timer
*/
@Override
public RoundTimer getRoundTimer() {
return roundTimer;
}
/**
* Gets block creator factory.
*
* @return the block creator factory
*/
@Override
public QbftBlockCreatorFactory getBlockCreatorFactory() {
return blockCreatorFactory;
}
@Override
public int getQuorum() {
return BftHelpers.calculateRequiredValidatorQuorum(getValidators().size());
}
@Override
public BlockTimer getBlockTimer() {
return blockTimer;
}
/**
* Is local node proposer for round.
*
* @param roundIdentifier the round identifier
* @return the boolean
*/
@Override
public boolean isLocalNodeProposerForRound(final ConsensusRoundIdentifier roundIdentifier) {
return getProposerForRound(roundIdentifier).equals(localAddress);
}
/**
* Gets proposer for round.
*
* @param roundIdentifier the round identifier
* @return the proposer for round
*/
public Address getProposerForRound(final ConsensusRoundIdentifier roundIdentifier) {
return proposerSelector.selectProposerForRound(roundIdentifier);
}
/**
* Gets clock.
*
* @return the clock
*/
@Override
public Clock getClock() {
return clock;
}
}

View File

@@ -0,0 +1,49 @@
/*
* Copyright contributors to Besu.
*
* Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
* an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
* specific language governing permissions and limitations under the License.
*
* SPDX-License-Identifier: Apache-2.0
*/
package org.hyperledger.besu.consensus.qbft.adaptor;
import org.hyperledger.besu.consensus.qbft.core.types.QbftProtocolSchedule;
import org.hyperledger.besu.consensus.qbft.core.types.QbftProtocolSpec;
import org.hyperledger.besu.ethereum.ProtocolContext;
import org.hyperledger.besu.ethereum.core.BlockHeader;
import org.hyperledger.besu.ethereum.mainnet.ProtocolSchedule;
import org.hyperledger.besu.ethereum.mainnet.ProtocolSpec;
/**
* Adaptor class to allow a {@link ProtocolSchedule} to be used as a {@link QbftProtocolSchedule}.
*/
public class QbftProtocolScheduleAdaptor implements QbftProtocolSchedule {
private final ProtocolSchedule besuProtocolSchedule;
private final ProtocolContext context;
/**
* Constructs a new Qbft protocol schedule.
*
* @param besuProtocolSchedule The Besu protocol schedule.
* @param context The protocol context.
*/
public QbftProtocolScheduleAdaptor(
final ProtocolSchedule besuProtocolSchedule, final ProtocolContext context) {
this.besuProtocolSchedule = besuProtocolSchedule;
this.context = context;
}
@Override
public QbftProtocolSpec getByBlockHeader(final BlockHeader header) {
final ProtocolSpec protocolSpec = besuProtocolSchedule.getByBlockHeader(header);
return new QbftProtocolSpecAdaptor(protocolSpec, context);
}
}

View File

@@ -0,0 +1,49 @@
/*
* Copyright contributors to Besu.
*
* Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
* an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
* specific language governing permissions and limitations under the License.
*
* SPDX-License-Identifier: Apache-2.0
*/
package org.hyperledger.besu.consensus.qbft.adaptor;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockImporter;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockValidator;
import org.hyperledger.besu.consensus.qbft.core.types.QbftProtocolSpec;
import org.hyperledger.besu.ethereum.ProtocolContext;
import org.hyperledger.besu.ethereum.mainnet.ProtocolSpec;
/** Adaptor class to allow a {@link ProtocolSpec} to be used as a {@link QbftProtocolSpec}. */
public class QbftProtocolSpecAdaptor implements QbftProtocolSpec {
private final ProtocolSpec besuProtocolSpec;
private final ProtocolContext context;
/**
* Constructs a new Qbft protocol spec.
*
* @param besuProtocolSpec The Besu protocol spec.
* @param context The protocol context.
*/
public QbftProtocolSpecAdaptor(
final ProtocolSpec besuProtocolSpec, final ProtocolContext context) {
this.besuProtocolSpec = besuProtocolSpec;
this.context = context;
}
@Override
public QbftBlockImporter getBlockImporter() {
return new QbftBlockImporterAdaptor(besuProtocolSpec.getBlockImporter(), context);
}
@Override
public QbftBlockValidator getBlockValidator() {
return new QbftBlockValidatorAdaptor(besuProtocolSpec.getBlockValidator(), context);
}
}

View File

@@ -0,0 +1,69 @@
/*
* Copyright contributors to Besu.
*
* Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
* an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
* specific language governing permissions and limitations under the License.
*
* SPDX-License-Identifier: Apache-2.0
*/
package org.hyperledger.besu.consensus.qbft.adaptor;
import static java.util.Collections.emptyList;
import static org.assertj.core.api.Assertions.assertThat;
import org.hyperledger.besu.consensus.common.bft.BftBlockHeaderFunctions;
import org.hyperledger.besu.consensus.common.bft.BftExtraData;
import org.hyperledger.besu.consensus.qbft.QbftExtraDataCodec;
import org.hyperledger.besu.consensus.qbft.core.types.QbftHashMode;
import org.hyperledger.besu.crypto.SECPSignature;
import org.hyperledger.besu.datatypes.Hash;
import org.hyperledger.besu.ethereum.core.BlockHeader;
import org.hyperledger.besu.ethereum.core.BlockHeaderTestFixture;
import java.math.BigInteger;
import java.util.List;
import java.util.Optional;
import org.apache.tuweni.bytes.Bytes;
import org.junit.jupiter.api.Test;
class BlockHeaderFunctionsUtilTest {
private final QbftExtraDataCodec qbftExtraDataCodec = new QbftExtraDataCodec();
private final BftExtraData bftExtraData =
new BftExtraData(
Bytes.wrap(new byte[32]),
List.of(new SECPSignature(BigInteger.ONE, BigInteger.ONE, (byte) 0)),
Optional.empty(),
2,
emptyList());
private final BlockHeader header =
new BlockHeaderTestFixture()
.number(1)
.extraData(new QbftExtraDataCodec().encode(bftExtraData))
.buildHeader();
@Test
void returnBlockHeaderFunctionThatHashesForCommitedSeal() {
Hash expectedHash = BftBlockHeaderFunctions.forCommittedSeal(qbftExtraDataCodec).hash(header);
Hash hash =
BlockHeaderFunctionsUtil.getBlockHeaderFunctions(
qbftExtraDataCodec, QbftHashMode.COMMITTED_SEAL)
.hash(header);
assertThat(hash).isEqualTo(expectedHash);
}
@Test
void returnBlockHeaderFunctionThatHashesForOnChain() {
Hash expectedHash = BftBlockHeaderFunctions.forOnchainBlock(qbftExtraDataCodec).hash(header);
Hash hash =
BlockHeaderFunctionsUtil.getBlockHeaderFunctions(qbftExtraDataCodec, QbftHashMode.ONCHAIN)
.hash(header);
assertThat(hash).isEqualTo(expectedHash);
}
}

View File

@@ -0,0 +1,37 @@
/*
* Copyright contributors to Besu.
*
* Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
* an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
* specific language governing permissions and limitations under the License.
*
* SPDX-License-Identifier: Apache-2.0
*/
package org.hyperledger.besu.consensus.qbft.adaptor;
import static org.assertj.core.api.Assertions.assertThat;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlock;
import org.hyperledger.besu.ethereum.core.Block;
import org.hyperledger.besu.ethereum.core.BlockBody;
import org.hyperledger.besu.ethereum.core.BlockHeader;
import org.hyperledger.besu.ethereum.core.BlockHeaderTestFixture;
import org.junit.jupiter.api.Test;
class BlockUtilTest {
@Test
void canConvertQbftBlockToBesuBlock() {
BlockHeader header = new BlockHeaderTestFixture().buildHeader();
Block besuBlock = new Block(header, BlockBody.empty());
QbftBlock qbftBlock = new QbftBlockAdaptor(besuBlock);
assertThat(BlockUtil.toBesuBlock(qbftBlock)).isSameAs(besuBlock);
}
}

View File

@@ -0,0 +1,58 @@
/*
* Copyright contributors to Besu.
*
* Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
* an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
* specific language governing permissions and limitations under the License.
*
* SPDX-License-Identifier: Apache-2.0
*/
package org.hyperledger.besu.consensus.qbft.adaptor;
import static java.util.Collections.emptyList;
import static org.assertj.core.api.Assertions.assertThat;
import org.hyperledger.besu.consensus.common.bft.BftExtraData;
import org.hyperledger.besu.consensus.qbft.QbftExtraDataCodec;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlock;
import org.hyperledger.besu.consensus.qbft.core.types.QbftHashMode;
import org.hyperledger.besu.ethereum.core.Block;
import org.hyperledger.besu.ethereum.core.BlockBody;
import org.hyperledger.besu.ethereum.core.BlockHeader;
import org.hyperledger.besu.ethereum.core.BlockHeaderTestFixture;
import org.hyperledger.besu.ethereum.rlp.BytesValueRLPInput;
import org.hyperledger.besu.ethereum.rlp.BytesValueRLPOutput;
import org.hyperledger.besu.ethereum.rlp.RLPInput;
import java.util.Optional;
import org.apache.tuweni.bytes.Bytes;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.EnumSource;
class QbftBlockCodecAdaptorTest {
@ParameterizedTest
@EnumSource(QbftHashMode.class)
void canRoundTripBlock(final QbftHashMode hashMode) {
BftExtraData bftExtraData =
new BftExtraData(Bytes.wrap(new byte[32]), emptyList(), Optional.empty(), 0, emptyList());
Bytes encodedExtraData = new QbftExtraDataCodec().encode(bftExtraData);
BlockHeader header = new BlockHeaderTestFixture().extraData(encodedExtraData).buildHeader();
Block besuBlock = new Block(header, BlockBody.empty());
QbftBlock block = new QbftBlockAdaptor(besuBlock);
BytesValueRLPOutput rlpOutput = new BytesValueRLPOutput();
QbftBlockCodecAdaptor qbftBlockCodec = new QbftBlockCodecAdaptor(new QbftExtraDataCodec());
qbftBlockCodec.writeTo(block, rlpOutput);
RLPInput rlpInput = new BytesValueRLPInput(rlpOutput.encoded(), false);
QbftBlock decodedBlock = qbftBlockCodec.readFrom(rlpInput, hashMode);
assertThat(decodedBlock).isEqualTo(block);
}
}

View File

@@ -0,0 +1,89 @@
/*
* Copyright contributors to Besu.
*
* Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
* an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
* specific language governing permissions and limitations under the License.
*
* SPDX-License-Identifier: Apache-2.0
*/
package org.hyperledger.besu.consensus.qbft.adaptor;
import static java.util.Collections.emptyList;
import static org.assertj.core.api.Assertions.assertThat;
import static org.mockito.Mockito.when;
import org.hyperledger.besu.consensus.common.bft.BftExtraData;
import org.hyperledger.besu.consensus.common.bft.Vote;
import org.hyperledger.besu.consensus.qbft.QbftExtraDataCodec;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlock;
import org.hyperledger.besu.consensus.qbft.core.types.QbftExtraDataProvider;
import org.hyperledger.besu.crypto.SECPSignature;
import org.hyperledger.besu.datatypes.Address;
import org.hyperledger.besu.ethereum.blockcreation.BlockCreator;
import org.hyperledger.besu.ethereum.core.Block;
import org.hyperledger.besu.ethereum.core.BlockBody;
import org.hyperledger.besu.ethereum.core.BlockHeader;
import org.hyperledger.besu.ethereum.core.BlockHeaderTestFixture;
import java.math.BigInteger;
import java.util.List;
import java.util.Optional;
import org.apache.tuweni.bytes.Bytes;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.extension.ExtendWith;
import org.mockito.Mock;
import org.mockito.junit.jupiter.MockitoExtension;
@ExtendWith(MockitoExtension.class)
class QbftBlockCreatorAdaptorTest {
@Mock private BlockCreator blockCreator;
@Mock private BlockHeader parentHeader;
@Mock private Block besuBlock;
@Mock private QbftExtraDataProvider qbftExtraDataProvider;
private final QbftExtraDataCodec qbftExtraDataCodec = new QbftExtraDataCodec();
@Test
void createsBlockUsingBesuBlockCreator() {
when(blockCreator.createBlock(10, parentHeader))
.thenReturn(new BlockCreator.BlockCreationResult(besuBlock, null, null));
QbftBlockCreatorAdaptor qbftBlockCreator =
new QbftBlockCreatorAdaptor(blockCreator, qbftExtraDataCodec);
QbftBlock qbftBlock = qbftBlockCreator.createBlock(10, parentHeader);
assertThat(((QbftBlockAdaptor) qbftBlock).getBesuBlock()).isEqualTo(besuBlock);
}
@Test
void createsSealedBlockWithRoundAndSeals() {
BftExtraData bftExtraData =
new BftExtraData(
Bytes.wrap(new byte[32]),
emptyList(),
Optional.of(Vote.authVote(Address.ZERO)),
0,
List.of(Address.ZERO));
BlockHeader header = new BlockHeaderTestFixture().buildHeader();
Block besuBlock = new Block(header, BlockBody.empty());
QbftBlock block = new QbftBlockAdaptor(besuBlock);
SECPSignature seal = new SECPSignature(BigInteger.ONE, BigInteger.ONE, (byte) 1);
when(qbftExtraDataProvider.getExtraData(header)).thenReturn(bftExtraData);
QbftBlockCreatorAdaptor qbftBlockCreator =
new QbftBlockCreatorAdaptor(blockCreator, qbftExtraDataCodec);
QbftBlock sealedBlock =
qbftBlockCreator.createSealedBlock(qbftExtraDataProvider, block, 1, List.of(seal));
BftExtraData sealedExtraData = qbftExtraDataCodec.decode(sealedBlock.getHeader());
assertThat(sealedExtraData.getVanityData()).isEqualTo(Bytes.wrap(new byte[32]));
assertThat(sealedExtraData.getVote()).contains(Vote.authVote(Address.ZERO));
assertThat(sealedExtraData.getValidators()).isEqualTo(List.of(Address.ZERO));
assertThat(sealedExtraData.getRound()).isEqualTo(1);
assertThat(sealedExtraData.getSeals()).isEqualTo(List.of(seal));
}
}

View File

@@ -0,0 +1,44 @@
/*
* Copyright contributors to Besu.
*
* Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
* an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
* specific language governing permissions and limitations under the License.
*
* SPDX-License-Identifier: Apache-2.0
*/
package org.hyperledger.besu.consensus.qbft.adaptor;
import static org.assertj.core.api.Assertions.assertThat;
import static org.mockito.Mockito.when;
import org.hyperledger.besu.consensus.qbft.QbftExtraDataCodec;
import org.hyperledger.besu.consensus.qbft.blockcreation.QbftBlockCreatorFactory;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockCreator;
import org.hyperledger.besu.ethereum.blockcreation.BlockCreator;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.extension.ExtendWith;
import org.mockito.Mock;
import org.mockito.junit.jupiter.MockitoExtension;
@ExtendWith(MockitoExtension.class)
class QbftBlockCreatorFactoryAdaptorTest {
@Mock private QbftBlockCreatorFactory qbftBlockCreatorFactory;
@Mock private BlockCreator blockCreator;
@Test
void createsQbftBlockCreatorFactory() {
when(qbftBlockCreatorFactory.create(1)).thenReturn(blockCreator);
QbftBlockCreatorFactoryAdaptor qbftBlockCreatorFactoryAdaptor =
new QbftBlockCreatorFactoryAdaptor(qbftBlockCreatorFactory, new QbftExtraDataCodec());
QbftBlockCreator qbftBlockCreator = qbftBlockCreatorFactoryAdaptor.create(1);
assertThat(qbftBlockCreator).hasFieldOrPropertyWithValue("besuBlockCreator", blockCreator);
}
}

View File

@@ -0,0 +1,72 @@
/*
* Copyright contributors to Besu.
*
* Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
* an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
* specific language governing permissions and limitations under the License.
*
* SPDX-License-Identifier: Apache-2.0
*/
package org.hyperledger.besu.consensus.qbft.adaptor;
import static org.assertj.core.api.Assertions.assertThat;
import static org.hyperledger.besu.ethereum.mainnet.BlockImportResult.BlockImportStatus.ALREADY_IMPORTED;
import static org.hyperledger.besu.ethereum.mainnet.BlockImportResult.BlockImportStatus.IMPORTED;
import static org.hyperledger.besu.ethereum.mainnet.BlockImportResult.BlockImportStatus.NOT_IMPORTED;
import static org.mockito.Mockito.when;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlock;
import org.hyperledger.besu.ethereum.ProtocolContext;
import org.hyperledger.besu.ethereum.core.Block;
import org.hyperledger.besu.ethereum.core.BlockDataGenerator;
import org.hyperledger.besu.ethereum.core.BlockImporter;
import org.hyperledger.besu.ethereum.mainnet.BlockImportResult;
import org.hyperledger.besu.ethereum.mainnet.HeaderValidationMode;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.extension.ExtendWith;
import org.mockito.Mock;
import org.mockito.junit.jupiter.MockitoExtension;
@ExtendWith(MockitoExtension.class)
class QbftBlockImporterAdaptorTest {
@Mock private BlockImporter blockImporter;
@Mock private ProtocolContext protocolContext;
private final Block besuBlock = new BlockDataGenerator().block();
private final QbftBlock block = new QbftBlockAdaptor(besuBlock);
@Test
void importsBlockSuccessfullyWhenBesuBlockImports() {
when(blockImporter.importBlock(protocolContext, besuBlock, HeaderValidationMode.FULL))
.thenReturn(new BlockImportResult(IMPORTED));
QbftBlockImporterAdaptor qbftBlockImporter =
new QbftBlockImporterAdaptor(blockImporter, protocolContext);
assertThat(qbftBlockImporter.importBlock(block)).isEqualTo(true);
}
@Test
void importsBlockSuccessfullyWhenBesuBlockAlreadyImported() {
when(blockImporter.importBlock(protocolContext, besuBlock, HeaderValidationMode.FULL))
.thenReturn(new BlockImportResult(ALREADY_IMPORTED));
QbftBlockImporterAdaptor qbftBlockImporter =
new QbftBlockImporterAdaptor(blockImporter, protocolContext);
assertThat(qbftBlockImporter.importBlock(block)).isEqualTo(true);
}
@Test
void importsBlockFailsWhenBesuBlockNotImported() {
when(blockImporter.importBlock(protocolContext, besuBlock, HeaderValidationMode.FULL))
.thenReturn(new BlockImportResult(NOT_IMPORTED));
QbftBlockImporterAdaptor qbftBlockImporter =
new QbftBlockImporterAdaptor(blockImporter, protocolContext);
assertThat(qbftBlockImporter.importBlock(block)).isEqualTo(false);
}
}

View File

@@ -0,0 +1,59 @@
/*
* Copyright contributors to Besu.
*
* Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
* an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
* specific language governing permissions and limitations under the License.
*
* SPDX-License-Identifier: Apache-2.0
*/
package org.hyperledger.besu.consensus.qbft.adaptor;
import static java.util.Collections.emptyList;
import static org.assertj.core.api.Assertions.assertThat;
import org.hyperledger.besu.consensus.common.bft.BftBlockInterface;
import org.hyperledger.besu.consensus.common.bft.BftExtraData;
import org.hyperledger.besu.consensus.qbft.QbftExtraDataCodec;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlock;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockInterface;
import org.hyperledger.besu.consensus.qbft.core.types.QbftHashMode;
import org.hyperledger.besu.ethereum.core.Block;
import org.hyperledger.besu.ethereum.core.BlockBody;
import org.hyperledger.besu.ethereum.core.BlockHeader;
import org.hyperledger.besu.ethereum.core.BlockHeaderTestFixture;
import java.util.Optional;
import org.apache.tuweni.bytes.Bytes;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.extension.ExtendWith;
import org.mockito.junit.jupiter.MockitoExtension;
@ExtendWith(MockitoExtension.class)
class QbftBlockInterfaceAdaptorTest {
private final BftBlockInterface bftBlockInterface =
new BftBlockInterface(new QbftExtraDataCodec());
@Test
void replacesRoundInBlockHeader() {
QbftExtraDataCodec qbftExtraDataCodec = new QbftExtraDataCodec();
BftExtraData bftExtraData =
new BftExtraData(Bytes.wrap(new byte[32]), emptyList(), Optional.empty(), 0, emptyList());
Bytes encodedExtraData = qbftExtraDataCodec.encode(bftExtraData);
BlockHeader header = new BlockHeaderTestFixture().extraData(encodedExtraData).buildHeader();
Block besuBlock = new Block(header, BlockBody.empty());
QbftBlock block = new QbftBlockAdaptor(besuBlock);
QbftBlockInterface qbftBlockInterface = new QbftBlockInterfaceAdaptor(bftBlockInterface);
QbftBlock updatedBlock =
qbftBlockInterface.replaceRoundInBlock(block, 1, QbftHashMode.COMMITTED_SEAL);
BftExtraData extraData = qbftExtraDataCodec.decode(updatedBlock.getHeader());
assertThat(extraData.getRound()).isEqualTo(1);
}
}

View File

@@ -0,0 +1,78 @@
/*
* Copyright contributors to Besu.
*
* Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
* an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
* specific language governing permissions and limitations under the License.
*
* SPDX-License-Identifier: Apache-2.0
*/
package org.hyperledger.besu.consensus.qbft.adaptor;
import static org.assertj.core.api.Assertions.assertThat;
import static org.mockito.Mockito.when;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockValidator;
import org.hyperledger.besu.ethereum.BlockProcessingResult;
import org.hyperledger.besu.ethereum.BlockValidator;
import org.hyperledger.besu.ethereum.ProtocolContext;
import org.hyperledger.besu.ethereum.core.Block;
import org.hyperledger.besu.ethereum.mainnet.HeaderValidationMode;
import java.util.Optional;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.extension.ExtendWith;
import org.mockito.Mock;
import org.mockito.junit.jupiter.MockitoExtension;
@ExtendWith(MockitoExtension.class)
class QbftBlockValidatorAdaptorTest {
@Mock private BlockValidator blockValidator;
@Mock private ProtocolContext protocolContext;
@Mock private Block besuBlock;
@Mock private QbftBlockAdaptor qbftBlock;
@Test
void validateSuccessfullyWhenBesuValidatorSuccessful() {
when(qbftBlock.getBesuBlock()).thenReturn(besuBlock);
when(blockValidator.validateAndProcessBlock(
protocolContext,
besuBlock,
HeaderValidationMode.LIGHT,
HeaderValidationMode.FULL,
false))
.thenReturn(new BlockProcessingResult(Optional.empty()));
QbftBlockValidatorAdaptor qbftBlockValidator =
new QbftBlockValidatorAdaptor(blockValidator, protocolContext);
QbftBlockValidator.ValidationResult validationResult =
qbftBlockValidator.validateBlock(qbftBlock);
assertThat(validationResult.success()).isTrue();
assertThat(validationResult.errorMessage()).isEmpty();
}
@Test
void validateFailsWhenBesuValidatorFails() {
when(qbftBlock.getBesuBlock()).thenReturn(besuBlock);
when(blockValidator.validateAndProcessBlock(
protocolContext,
besuBlock,
HeaderValidationMode.LIGHT,
HeaderValidationMode.FULL,
false))
.thenReturn(new BlockProcessingResult("failed"));
QbftBlockValidatorAdaptor qbftBlockValidator =
new QbftBlockValidatorAdaptor(blockValidator, protocolContext);
QbftBlockValidator.ValidationResult validationResult =
qbftBlockValidator.validateBlock(qbftBlock);
assertThat(validationResult.success()).isFalse();
assertThat(validationResult.errorMessage()).contains("failed");
}
}

View File

@@ -0,0 +1,46 @@
/*
* Copyright contributors to Besu.
*
* Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
* an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
* specific language governing permissions and limitations under the License.
*
* SPDX-License-Identifier: Apache-2.0
*/
package org.hyperledger.besu.consensus.qbft.adaptor;
import static java.util.Collections.emptyList;
import static org.assertj.core.api.Assertions.assertThat;
import org.hyperledger.besu.consensus.common.bft.BftExtraData;
import org.hyperledger.besu.consensus.qbft.QbftExtraDataCodec;
import org.hyperledger.besu.ethereum.core.BlockHeader;
import org.hyperledger.besu.ethereum.core.BlockHeaderTestFixture;
import java.util.Optional;
import org.apache.tuweni.bytes.Bytes;
import org.junit.jupiter.api.Test;
class QbftExtraDataProviderAdaptorTest {
@Test
void retrievesExtraDataFromBlockHeader() {
final QbftExtraDataCodec qbftExtraDataCodec = new QbftExtraDataCodec();
final BftExtraData bftExtraData =
new BftExtraData(Bytes.wrap(new byte[32]), emptyList(), Optional.empty(), 0, emptyList());
final Bytes encoded = qbftExtraDataCodec.encode(bftExtraData);
final BlockHeader header =
new BlockHeaderTestFixture().number(1).extraData(encoded).buildHeader();
final QbftExtraDataProviderAdaptor qbftExtraDataProvider =
new QbftExtraDataProviderAdaptor(new QbftExtraDataCodec());
final BftExtraData retrievedExtraData = qbftExtraDataProvider.getExtraData(header);
assertThat(retrievedExtraData).isEqualToComparingFieldByField(bftExtraData);
}
}

View File

@@ -0,0 +1,48 @@
/*
* Copyright contributors to Besu.
*
* Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
* an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
* specific language governing permissions and limitations under the License.
*
* SPDX-License-Identifier: Apache-2.0
*/
package org.hyperledger.besu.consensus.qbft.adaptor;
import static org.assertj.core.api.Assertions.assertThat;
import static org.mockito.Mockito.when;
import org.hyperledger.besu.consensus.qbft.core.types.QbftProtocolSchedule;
import org.hyperledger.besu.consensus.qbft.core.types.QbftProtocolSpec;
import org.hyperledger.besu.ethereum.ProtocolContext;
import org.hyperledger.besu.ethereum.core.BlockHeader;
import org.hyperledger.besu.ethereum.mainnet.ProtocolSchedule;
import org.hyperledger.besu.ethereum.mainnet.ProtocolSpec;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.extension.ExtendWith;
import org.mockito.Mock;
import org.mockito.junit.jupiter.MockitoExtension;
@ExtendWith(MockitoExtension.class)
class QbftProtocolScheduleAdaptorTest {
@Mock private ProtocolSchedule besuProtocolSchedule;
@Mock private ProtocolSpec besuProtocolSpec;
@Mock private ProtocolContext besuProtocolContext;
@Mock private BlockHeader blockHeader;
@Test
void createsAProtocolSpecUsingBesuProtocolSpec() {
when(besuProtocolSchedule.getByBlockHeader(blockHeader)).thenReturn(besuProtocolSpec);
final QbftProtocolSchedule qbftProtocolSchedule =
new QbftProtocolScheduleAdaptor(besuProtocolSchedule, besuProtocolContext);
final QbftProtocolSpec protocolSpec = qbftProtocolSchedule.getByBlockHeader(blockHeader);
assertThat(protocolSpec).hasFieldOrPropertyWithValue("besuProtocolSpec", besuProtocolSpec);
}
}

View File

@@ -0,0 +1,60 @@
/*
* Copyright contributors to Besu.
*
* Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
* an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
* specific language governing permissions and limitations under the License.
*
* SPDX-License-Identifier: Apache-2.0
*/
package org.hyperledger.besu.consensus.qbft.adaptor;
import static org.assertj.core.api.Assertions.assertThat;
import static org.mockito.Mockito.when;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockImporter;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockValidator;
import org.hyperledger.besu.ethereum.BlockValidator;
import org.hyperledger.besu.ethereum.ProtocolContext;
import org.hyperledger.besu.ethereum.core.BlockImporter;
import org.hyperledger.besu.ethereum.mainnet.ProtocolSpec;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.extension.ExtendWith;
import org.mockito.Mock;
import org.mockito.Mockito;
import org.mockito.junit.jupiter.MockitoExtension;
@ExtendWith(MockitoExtension.class)
class QbftProtocolSpecAdaptorTest {
@Mock private ProtocolSpec besuProtocolSpec;
@Mock private ProtocolContext besuProtocolContext;
@Test
void createsBlockImporterUsingBesuBlockImporter() {
final BlockImporter besuBlockImporter = Mockito.mock(BlockImporter.class);
when(besuProtocolSpec.getBlockImporter()).thenReturn(besuBlockImporter);
final QbftProtocolSpecAdaptor qbftProtocolSpec =
new QbftProtocolSpecAdaptor(besuProtocolSpec, besuProtocolContext);
final QbftBlockImporter qbftBlockImporter = qbftProtocolSpec.getBlockImporter();
assertThat(qbftBlockImporter).hasFieldOrPropertyWithValue("blockImporter", besuBlockImporter);
}
@Test
void createsBlockValidatorUsingBesuBlockValidator() {
final BlockValidator besuBlockValidator = Mockito.mock(BlockValidator.class);
when(besuProtocolSpec.getBlockValidator()).thenReturn(besuBlockValidator);
final QbftProtocolSpecAdaptor qbftProtocolSpec =
new QbftProtocolSpecAdaptor(besuProtocolSpec, besuProtocolContext);
final QbftBlockValidator qbftBlockValidator = qbftProtocolSpec.getBlockValidator();
assertThat(qbftBlockValidator)
.hasFieldOrPropertyWithValue("blockValidator", besuBlockValidator);
}
}