Change Qbft to use its own QbftBlockHeader type (#8244)

Signed-off-by: Jason Frame <jason.frame@consensys.net>
This commit is contained in:
Jason Frame
2025-02-07 12:37:32 +10:00
committed by GitHub
parent e835cd07d2
commit a021feae03
69 changed files with 1572 additions and 345 deletions

View File

@@ -95,7 +95,7 @@ public class BftMiningSoakTest extends ParameterizedBftTestBase {
} catch (RuntimeException e) { } catch (RuntimeException e) {
assertThat(e.getMessage()) assertThat(e.getMessage())
.contains( .contains(
"Revert reason: 'Transaction processing could not be completed due to an exception'"); "Revert reason: 'Transaction processing could not be completed due to an exception (Invalid opcode: 0x5f)'");
} }
// Should initially be set to 0 // Should initially be set to 0

View File

@@ -23,6 +23,7 @@ import org.hyperledger.besu.config.QbftFork;
import org.hyperledger.besu.consensus.common.BftValidatorOverrides; import org.hyperledger.besu.consensus.common.BftValidatorOverrides;
import org.hyperledger.besu.consensus.common.EpochManager; import org.hyperledger.besu.consensus.common.EpochManager;
import org.hyperledger.besu.consensus.common.ForksSchedule; import org.hyperledger.besu.consensus.common.ForksSchedule;
import org.hyperledger.besu.consensus.common.bft.BftBlockHashing;
import org.hyperledger.besu.consensus.common.bft.BftBlockInterface; import org.hyperledger.besu.consensus.common.bft.BftBlockInterface;
import org.hyperledger.besu.consensus.common.bft.BftContext; import org.hyperledger.besu.consensus.common.bft.BftContext;
import org.hyperledger.besu.consensus.common.bft.BftEventQueue; import org.hyperledger.besu.consensus.common.bft.BftEventQueue;
@@ -46,13 +47,18 @@ import org.hyperledger.besu.consensus.common.validator.blockbased.BlockValidator
import org.hyperledger.besu.consensus.qbft.QbftExtraDataCodec; import org.hyperledger.besu.consensus.qbft.QbftExtraDataCodec;
import org.hyperledger.besu.consensus.qbft.QbftForksSchedulesFactory; import org.hyperledger.besu.consensus.qbft.QbftForksSchedulesFactory;
import org.hyperledger.besu.consensus.qbft.QbftProtocolScheduleBuilder; import org.hyperledger.besu.consensus.qbft.QbftProtocolScheduleBuilder;
import org.hyperledger.besu.consensus.qbft.adaptor.BftEventHandlerAdaptor;
import org.hyperledger.besu.consensus.qbft.adaptor.BlockUtil; import org.hyperledger.besu.consensus.qbft.adaptor.BlockUtil;
import org.hyperledger.besu.consensus.qbft.adaptor.QbftBlockCodecAdaptor; import org.hyperledger.besu.consensus.qbft.adaptor.QbftBlockCodecAdaptor;
import org.hyperledger.besu.consensus.qbft.adaptor.QbftBlockCreatorFactoryAdaptor; import org.hyperledger.besu.consensus.qbft.adaptor.QbftBlockCreatorFactoryAdaptor;
import org.hyperledger.besu.consensus.qbft.adaptor.QbftBlockHashingAdaptor;
import org.hyperledger.besu.consensus.qbft.adaptor.QbftBlockInterfaceAdaptor; import org.hyperledger.besu.consensus.qbft.adaptor.QbftBlockInterfaceAdaptor;
import org.hyperledger.besu.consensus.qbft.adaptor.QbftBlockchainAdaptor;
import org.hyperledger.besu.consensus.qbft.adaptor.QbftExtraDataProviderAdaptor; import org.hyperledger.besu.consensus.qbft.adaptor.QbftExtraDataProviderAdaptor;
import org.hyperledger.besu.consensus.qbft.adaptor.QbftFinalStateImpl; import org.hyperledger.besu.consensus.qbft.adaptor.QbftFinalStateImpl;
import org.hyperledger.besu.consensus.qbft.adaptor.QbftProtocolScheduleAdaptor; import org.hyperledger.besu.consensus.qbft.adaptor.QbftProtocolScheduleAdaptor;
import org.hyperledger.besu.consensus.qbft.adaptor.QbftValidatorModeTransitionLoggerAdaptor;
import org.hyperledger.besu.consensus.qbft.adaptor.QbftValidatorProviderAdaptor;
import org.hyperledger.besu.consensus.qbft.blockcreation.QbftBlockCreatorFactory; import org.hyperledger.besu.consensus.qbft.blockcreation.QbftBlockCreatorFactory;
import org.hyperledger.besu.consensus.qbft.core.network.QbftGossip; 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.payload.MessageFactory;
@@ -62,16 +68,18 @@ 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.QbftBlockCodec;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockInterface; 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.QbftContext;
import org.hyperledger.besu.consensus.qbft.core.types.QbftEventHandler;
import org.hyperledger.besu.consensus.qbft.core.types.QbftFinalState; 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.QbftMinedBlockObserver;
import org.hyperledger.besu.consensus.qbft.core.types.QbftProtocolSchedule; import org.hyperledger.besu.consensus.qbft.core.types.QbftProtocolSchedule;
import org.hyperledger.besu.consensus.qbft.core.types.QbftValidatorProvider;
import org.hyperledger.besu.consensus.qbft.core.validation.MessageValidatorFactory; 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; import org.hyperledger.besu.consensus.qbft.jsonrpc.QbftJsonRpcMethods;
import org.hyperledger.besu.consensus.qbft.protocol.Istanbul100SubProtocol; import org.hyperledger.besu.consensus.qbft.protocol.Istanbul100SubProtocol;
import org.hyperledger.besu.consensus.qbft.validator.ForkingValidatorProvider; import org.hyperledger.besu.consensus.qbft.validator.ForkingValidatorProvider;
import org.hyperledger.besu.consensus.qbft.validator.TransactionValidatorProvider; import org.hyperledger.besu.consensus.qbft.validator.TransactionValidatorProvider;
import org.hyperledger.besu.consensus.qbft.validator.ValidatorContractController; import org.hyperledger.besu.consensus.qbft.validator.ValidatorContractController;
import org.hyperledger.besu.consensus.qbft.validator.ValidatorModeTransitionLogger;
import org.hyperledger.besu.datatypes.Address; import org.hyperledger.besu.datatypes.Address;
import org.hyperledger.besu.ethereum.ProtocolContext; import org.hyperledger.besu.ethereum.ProtocolContext;
import org.hyperledger.besu.ethereum.api.jsonrpc.methods.JsonRpcMethods; import org.hyperledger.besu.ethereum.api.jsonrpc.methods.JsonRpcMethods;
@@ -209,9 +217,11 @@ public class QbftBesuControllerBuilder extends BesuControllerBuilder {
final ValidatorProvider validatorProvider = final ValidatorProvider validatorProvider =
protocolContext.getConsensusContext(BftContext.class).getValidatorProvider(); protocolContext.getConsensusContext(BftContext.class).getValidatorProvider();
final QbftValidatorProvider qbftValidatorProvider =
new QbftValidatorProviderAdaptor(validatorProvider);
final QbftBlockInterface qbftBlockInterface = new QbftBlockInterfaceAdaptor(bftBlockInterface); final QbftBlockInterface qbftBlockInterface = new QbftBlockInterfaceAdaptor(bftBlockInterface);
final QbftContext qbftContext = new QbftContext(validatorProvider, qbftBlockInterface); final QbftContext qbftContext = new QbftContext(qbftValidatorProvider, qbftBlockInterface);
final ProtocolContext qbftProtocolContext = final ProtocolContext qbftProtocolContext =
new ProtocolContext( new ProtocolContext(
blockchain, blockchain,
@@ -267,27 +277,31 @@ public class QbftBesuControllerBuilder extends BesuControllerBuilder {
final MessageFactory messageFactory = new MessageFactory(nodeKey, blockEncoder); final MessageFactory messageFactory = new MessageFactory(nodeKey, blockEncoder);
QbftRoundFactory qbftRoundFactory =
new QbftRoundFactory(
finalState,
qbftProtocolContext,
qbftProtocolSchedule,
minedBlockObservers,
messageValidatorFactory,
messageFactory,
qbftExtraDataCodec,
new QbftExtraDataProviderAdaptor(qbftExtraDataCodec),
new QbftBlockHashingAdaptor(new BftBlockHashing(qbftExtraDataCodec)));
QbftBlockHeightManagerFactory qbftBlockHeightManagerFactory = QbftBlockHeightManagerFactory qbftBlockHeightManagerFactory =
new QbftBlockHeightManagerFactory( new QbftBlockHeightManagerFactory(
finalState, finalState,
new QbftRoundFactory( qbftRoundFactory,
finalState,
qbftProtocolContext,
qbftProtocolSchedule,
minedBlockObservers,
messageValidatorFactory,
messageFactory,
qbftExtraDataCodec,
new QbftExtraDataProviderAdaptor(qbftExtraDataCodec)),
messageValidatorFactory, messageValidatorFactory,
messageFactory, messageFactory,
new ValidatorModeTransitionLogger(qbftForksSchedule)); new QbftValidatorModeTransitionLoggerAdaptor(
new ValidatorModeTransitionLogger(qbftForksSchedule)));
qbftBlockHeightManagerFactory.isEarlyRoundChangeEnabled(isEarlyRoundChangeEnabled); qbftBlockHeightManagerFactory.isEarlyRoundChangeEnabled(isEarlyRoundChangeEnabled);
final BftEventHandler qbftController = final QbftEventHandler qbftController =
new QbftController( new QbftController(
blockchain, new QbftBlockchainAdaptor(blockchain),
finalState, finalState,
qbftBlockHeightManagerFactory, qbftBlockHeightManagerFactory,
gossiper, gossiper,
@@ -295,14 +309,15 @@ public class QbftBesuControllerBuilder extends BesuControllerBuilder {
futureMessageBuffer, futureMessageBuffer,
new EthSynchronizerUpdater(ethProtocolManager.ethContext().getEthPeers()), new EthSynchronizerUpdater(ethProtocolManager.ethContext().getEthPeers()),
blockEncoder); blockEncoder);
final BftEventHandler bftEventHandler = new BftEventHandlerAdaptor(qbftController);
final EventMultiplexer eventMultiplexer = new EventMultiplexer(qbftController); final EventMultiplexer eventMultiplexer = new EventMultiplexer(bftEventHandler);
final BftProcessor bftProcessor = new BftProcessor(bftEventQueue, eventMultiplexer); final BftProcessor bftProcessor = new BftProcessor(bftEventQueue, eventMultiplexer);
final MiningCoordinator miningCoordinator = final MiningCoordinator miningCoordinator =
new BftMiningCoordinator( new BftMiningCoordinator(
bftExecutors, bftExecutors,
qbftController, bftEventHandler,
bftProcessor, bftProcessor,
blockCreatorFactory, blockCreatorFactory,
blockchain, blockchain,

View File

@@ -17,12 +17,12 @@ package org.hyperledger.besu.consensus.common.bft;
import org.hyperledger.besu.config.BftConfigOptions; import org.hyperledger.besu.config.BftConfigOptions;
import org.hyperledger.besu.consensus.common.ForksSchedule; import org.hyperledger.besu.consensus.common.ForksSchedule;
import org.hyperledger.besu.consensus.common.bft.events.BlockTimerExpiry; import org.hyperledger.besu.consensus.common.bft.events.BlockTimerExpiry;
import org.hyperledger.besu.ethereum.core.BlockHeader;
import java.time.Clock; import java.time.Clock;
import java.util.Optional; import java.util.Optional;
import java.util.concurrent.ScheduledFuture; import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import java.util.function.Supplier;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
@@ -81,10 +81,10 @@ public class BlockTimer {
* Starts a timer for the supplied round cancelling any previously active block timer * Starts a timer for the supplied round cancelling any previously active block timer
* *
* @param round The round identifier which this timer is tracking * @param round The round identifier which this timer is tracking
* @param chainHeadHeader The header of the chain head * @param headerTimestamp The timestamp from the of the chain head header
*/ */
public synchronized void startTimer( public synchronized void startTimer(
final ConsensusRoundIdentifier round, final BlockHeader chainHeadHeader) { final ConsensusRoundIdentifier round, final Supplier<Long> headerTimestamp) {
cancelTimer(); cancelTimer();
final long expiryTime; final long expiryTime;
@@ -104,7 +104,7 @@ public class BlockTimer {
final int currentBlockPeriodSeconds = final int currentBlockPeriodSeconds =
forksSchedule.getFork(round.getSequenceNumber()).getValue().getBlockPeriodSeconds(); forksSchedule.getFork(round.getSequenceNumber()).getValue().getBlockPeriodSeconds();
final long minimumTimeBetweenBlocksMillis = currentBlockPeriodSeconds * 1000L; final long minimumTimeBetweenBlocksMillis = currentBlockPeriodSeconds * 1000L;
expiryTime = chainHeadHeader.getTimestamp() * 1_000 + minimumTimeBetweenBlocksMillis; expiryTime = headerTimestamp.get() * 1_000 + minimumTimeBetweenBlocksMillis;
} }
setBlockTimes(round); setBlockTimes(round);
@@ -115,14 +115,14 @@ public class BlockTimer {
/** /**
* Checks if the empty block timer is expired * Checks if the empty block timer is expired
* *
* @param chainHeadHeader The header of the chain head * @param headerTimestamp Function to get the chain head timestamp
* @param currentTimeInMillis The current time * @param currentTimeInMillis The current time
* @return a boolean value * @return a boolean value
*/ */
public synchronized boolean checkEmptyBlockExpired( public synchronized boolean checkEmptyBlockExpired(
final BlockHeader chainHeadHeader, final long currentTimeInMillis) { final Supplier<Long> headerTimestamp, final long currentTimeInMillis) {
final long emptyBlockPeriodExpiryTime = final long emptyBlockPeriodExpiryTime =
(chainHeadHeader.getTimestamp() + emptyBlockPeriodSeconds) * 1000; (headerTimestamp.get() + emptyBlockPeriodSeconds) * 1000;
if (currentTimeInMillis > emptyBlockPeriodExpiryTime) { if (currentTimeInMillis > emptyBlockPeriodExpiryTime) {
LOG.debug("Empty Block expired"); LOG.debug("Empty Block expired");
@@ -136,15 +136,15 @@ public class BlockTimer {
* Resets the empty block timer * Resets the empty block timer
* *
* @param roundIdentifier The current round identifier * @param roundIdentifier The current round identifier
* @param chainHeadHeader The header of the chain head * @param headerTimestamp Function to get timestamp from the header of the chain head
* @param currentTimeInMillis The current time * @param currentTimeInMillis The current time
*/ */
public void resetTimerForEmptyBlock( public void resetTimerForEmptyBlock(
final ConsensusRoundIdentifier roundIdentifier, final ConsensusRoundIdentifier roundIdentifier,
final BlockHeader chainHeadHeader, final Supplier<Long> headerTimestamp,
final long currentTimeInMillis) { final long currentTimeInMillis) {
final long emptyBlockPeriodExpiryTime = final long emptyBlockPeriodExpiryTime =
(chainHeadHeader.getTimestamp() + emptyBlockPeriodSeconds) * 1000; (headerTimestamp.get() + emptyBlockPeriodSeconds) * 1000;
final long nextBlockPeriodExpiryTime = currentTimeInMillis + blockPeriodSeconds * 1000; final long nextBlockPeriodExpiryTime = currentTimeInMillis + blockPeriodSeconds * 1000;
startTimer(roundIdentifier, Math.min(emptyBlockPeriodExpiryTime, nextBlockPeriodExpiryTime)); startTimer(roundIdentifier, Math.min(emptyBlockPeriodExpiryTime, nextBlockPeriodExpiryTime));

View File

@@ -102,7 +102,7 @@ public class BlockTimerTest {
bftExecutors.scheduleTask(any(Runnable.class), anyLong(), any())) bftExecutors.scheduleTask(any(Runnable.class), anyLong(), any()))
.thenReturn(mockedFuture); .thenReturn(mockedFuture);
timer.startTimer(round, header); timer.startTimer(round, header::getTimestamp);
verify(bftExecutors) verify(bftExecutors)
.scheduleTask(any(Runnable.class), eq(EXPECTED_DELAY), eq(TimeUnit.MILLISECONDS)); .scheduleTask(any(Runnable.class), eq(EXPECTED_DELAY), eq(TimeUnit.MILLISECONDS));
} }
@@ -136,7 +136,7 @@ public class BlockTimerTest {
final BftEventQueue eventQueue = new BftEventQueue(1000); final BftEventQueue eventQueue = new BftEventQueue(1000);
final BlockTimer timer = new BlockTimer(eventQueue, mockForksSchedule, bftExecutors, mockClock); final BlockTimer timer = new BlockTimer(eventQueue, mockForksSchedule, bftExecutors, mockClock);
timer.startTimer(round, header); timer.startTimer(round, header::getTimestamp);
// Verify that the event will not be added to the queue immediately // Verify that the event will not be added to the queue immediately
assertThat(eventQueue.isEmpty()).isTrue(); assertThat(eventQueue.isEmpty()).isTrue();
@@ -182,7 +182,7 @@ public class BlockTimerTest {
final ConsensusRoundIdentifier round = final ConsensusRoundIdentifier round =
new ConsensusRoundIdentifier(0xFEDBCA9876543210L, 0x12345678); new ConsensusRoundIdentifier(0xFEDBCA9876543210L, 0x12345678);
timer.startTimer(round, header); timer.startTimer(round, header::getTimestamp);
verify(bftExecutors, never()).scheduleTask(any(Runnable.class), anyLong(), any()); verify(bftExecutors, never()).scheduleTask(any(Runnable.class), anyLong(), any());
final ArgumentCaptor<BftEvent> bftEventCaptor = ArgumentCaptor.forClass(BftEvent.class); final ArgumentCaptor<BftEvent> bftEventCaptor = ArgumentCaptor.forClass(BftEvent.class);
@@ -218,7 +218,7 @@ public class BlockTimerTest {
final ConsensusRoundIdentifier round = final ConsensusRoundIdentifier round =
new ConsensusRoundIdentifier(0xFEDBCA9876543210L, 0x12345678); new ConsensusRoundIdentifier(0xFEDBCA9876543210L, 0x12345678);
timer.startTimer(round, header); timer.startTimer(round, header::getTimestamp);
verify(bftExecutors, never()).scheduleTask(any(Runnable.class), anyLong(), any()); verify(bftExecutors, never()).scheduleTask(any(Runnable.class), anyLong(), any());
final ArgumentCaptor<BftEvent> bftEventCaptor = ArgumentCaptor.forClass(BftEvent.class); final ArgumentCaptor<BftEvent> bftEventCaptor = ArgumentCaptor.forClass(BftEvent.class);
@@ -258,9 +258,9 @@ public class BlockTimerTest {
Mockito.<ScheduledFuture<?>>when( Mockito.<ScheduledFuture<?>>when(
bftExecutors.scheduleTask(any(Runnable.class), anyLong(), eq(TimeUnit.MILLISECONDS))) bftExecutors.scheduleTask(any(Runnable.class), anyLong(), eq(TimeUnit.MILLISECONDS)))
.thenReturn(mockedFuture); .thenReturn(mockedFuture);
timer.startTimer(round, header); timer.startTimer(round, header::getTimestamp);
verify(mockedFuture, times(0)).cancel(false); verify(mockedFuture, times(0)).cancel(false);
timer.startTimer(round, header); timer.startTimer(round, header::getTimestamp);
verify(mockedFuture, times(1)).cancel(false); verify(mockedFuture, times(1)).cancel(false);
} }
@@ -292,7 +292,7 @@ public class BlockTimerTest {
Mockito.<ScheduledFuture<?>>when( Mockito.<ScheduledFuture<?>>when(
bftExecutors.scheduleTask(any(Runnable.class), anyLong(), eq(TimeUnit.MILLISECONDS))) bftExecutors.scheduleTask(any(Runnable.class), anyLong(), eq(TimeUnit.MILLISECONDS)))
.thenReturn(mockedFuture); .thenReturn(mockedFuture);
timer.startTimer(round, header); timer.startTimer(round, header::getTimestamp);
when(mockedFuture.isDone()).thenReturn(false); when(mockedFuture.isDone()).thenReturn(false);
assertThat(timer.isRunning()).isTrue(); assertThat(timer.isRunning()).isTrue();
when(mockedFuture.isDone()).thenReturn(true); when(mockedFuture.isDone()).thenReturn(true);
@@ -322,7 +322,7 @@ public class BlockTimerTest {
MINIMAL_TIME_BETWEEN_EMPTY_BLOCKS_SECONDS))); MINIMAL_TIME_BETWEEN_EMPTY_BLOCKS_SECONDS)));
final BlockTimer timer = new BlockTimer(mockQueue, mockForksSchedule, bftExecutors, mockClock); final BlockTimer timer = new BlockTimer(mockQueue, mockForksSchedule, bftExecutors, mockClock);
timer.startTimer(round, header); timer.startTimer(round, header::getTimestamp);
assertThat(timer.getBlockPeriodSeconds()).isEqualTo(MINIMAL_TIME_BETWEEN_BLOCKS_SECONDS); assertThat(timer.getBlockPeriodSeconds()).isEqualTo(MINIMAL_TIME_BETWEEN_BLOCKS_SECONDS);
assertThat(timer.getEmptyBlockPeriodSeconds()) assertThat(timer.getEmptyBlockPeriodSeconds())

View File

@@ -117,7 +117,7 @@ public class IbftBlockHeightManager implements BaseIbftBlockHeightManager {
currentRound = roundFactory.createNewRound(parentHeader, 0); currentRound = roundFactory.createNewRound(parentHeader, 0);
if (finalState.isLocalNodeProposerForRound(currentRound.getRoundIdentifier())) { if (finalState.isLocalNodeProposerForRound(currentRound.getRoundIdentifier())) {
blockTimer.startTimer(currentRound.getRoundIdentifier(), parentHeader); blockTimer.startTimer(currentRound.getRoundIdentifier(), parentHeader::getTimestamp);
} }
} }

View File

@@ -19,13 +19,17 @@ import org.hyperledger.besu.consensus.common.bft.BftBlockInterface;
import org.hyperledger.besu.consensus.common.bft.ConsensusRoundIdentifier; import org.hyperledger.besu.consensus.common.bft.ConsensusRoundIdentifier;
import org.hyperledger.besu.consensus.common.bft.payload.SignedData; import org.hyperledger.besu.consensus.common.bft.payload.SignedData;
import org.hyperledger.besu.consensus.qbft.QbftExtraDataCodec; import org.hyperledger.besu.consensus.qbft.QbftExtraDataCodec;
import org.hyperledger.besu.consensus.qbft.adaptor.QbftBlockHashingAdaptor;
import org.hyperledger.besu.consensus.qbft.adaptor.QbftBlockInterfaceAdaptor; import org.hyperledger.besu.consensus.qbft.adaptor.QbftBlockInterfaceAdaptor;
import org.hyperledger.besu.consensus.qbft.adaptor.QbftExtraDataProviderAdaptor;
import org.hyperledger.besu.consensus.qbft.core.payload.CommitPayload; 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.payload.MessageFactory;
import org.hyperledger.besu.consensus.qbft.core.statemachine.PreparedCertificate; 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.QbftBlock;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockCodec; import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockCodec;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockHashing;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockInterface; import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockInterface;
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.QbftHashMode;
import org.hyperledger.besu.crypto.SECPSignature; import org.hyperledger.besu.crypto.SECPSignature;
import org.hyperledger.besu.cryptoservices.NodeKey; import org.hyperledger.besu.cryptoservices.NodeKey;
@@ -42,10 +46,14 @@ public class IntegrationTestHelpers {
final QbftBlock commitBlock = final QbftBlock commitBlock =
createCommitBlockFromProposalBlock(block, roundId.getRoundNumber()); createCommitBlockFromProposalBlock(block, roundId.getRoundNumber());
final QbftBlockHashing blockHashing =
new QbftBlockHashingAdaptor(new BftBlockHashing(qbftExtraDataEncoder));
final QbftExtraDataProvider extraDataProvider =
new QbftExtraDataProviderAdaptor(qbftExtraDataEncoder);
final SECPSignature commitSeal = final SECPSignature commitSeal =
nodeKey.sign( nodeKey.sign(
new BftBlockHashing(qbftExtraDataEncoder) blockHashing.calculateDataHashForCommittedSeal(
.calculateDataHashForCommittedSeal(commitBlock.getHeader())); commitBlock.getHeader(), extraDataProvider.getExtraData(commitBlock.getHeader())));
final MessageFactory messageFactory = new MessageFactory(nodeKey, blockEncoder); final MessageFactory messageFactory = new MessageFactory(nodeKey, blockEncoder);

View File

@@ -22,14 +22,20 @@ import org.hyperledger.besu.consensus.common.bft.ConsensusRoundIdentifier;
import org.hyperledger.besu.consensus.common.bft.EventMultiplexer; import org.hyperledger.besu.consensus.common.bft.EventMultiplexer;
import org.hyperledger.besu.consensus.common.bft.blockcreation.ProposerSelector; 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.inttest.NodeParams;
import org.hyperledger.besu.consensus.common.bft.statemachine.BftEventHandler;
import org.hyperledger.besu.consensus.common.validator.ValidatorProvider; import org.hyperledger.besu.consensus.common.validator.ValidatorProvider;
import org.hyperledger.besu.consensus.qbft.adaptor.BlockUtil; import org.hyperledger.besu.consensus.qbft.adaptor.BlockUtil;
import org.hyperledger.besu.consensus.qbft.adaptor.QbftBlockAdaptor; import org.hyperledger.besu.consensus.qbft.adaptor.QbftBlockAdaptor;
import org.hyperledger.besu.consensus.qbft.adaptor.QbftBlockHeaderAdaptor;
import org.hyperledger.besu.consensus.qbft.adaptor.QbftBlockchainAdaptor;
import org.hyperledger.besu.consensus.qbft.adaptor.QbftValidatorProviderAdaptor;
import org.hyperledger.besu.consensus.qbft.core.payload.MessageFactory; 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.QbftBlock;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockCodec; import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockCodec;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockHeader;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockchain;
import org.hyperledger.besu.consensus.qbft.core.types.QbftEventHandler;
import org.hyperledger.besu.consensus.qbft.core.types.QbftFinalState; import org.hyperledger.besu.consensus.qbft.core.types.QbftFinalState;
import org.hyperledger.besu.consensus.qbft.core.types.QbftValidatorProvider;
import org.hyperledger.besu.crypto.SECPSignature; import org.hyperledger.besu.crypto.SECPSignature;
import org.hyperledger.besu.datatypes.Address; import org.hyperledger.besu.datatypes.Address;
import org.hyperledger.besu.ethereum.chain.MutableBlockchain; import org.hyperledger.besu.ethereum.chain.MutableBlockchain;
@@ -57,7 +63,7 @@ public class TestContext {
private final Map<Address, ValidatorPeer> remotePeers; private final Map<Address, ValidatorPeer> remotePeers;
private final MutableBlockchain blockchain; private final MutableBlockchain blockchain;
private final BftExecutors bftExecutors; private final BftExecutors bftExecutors;
private final BftEventHandler controller; private final QbftEventHandler controller;
private final QbftFinalState finalState; private final QbftFinalState finalState;
private final EventMultiplexer eventMultiplexer; private final EventMultiplexer eventMultiplexer;
private final MessageFactory messageFactory; private final MessageFactory messageFactory;
@@ -70,7 +76,7 @@ public class TestContext {
final Map<Address, ValidatorPeer> remotePeers, final Map<Address, ValidatorPeer> remotePeers,
final MutableBlockchain blockchain, final MutableBlockchain blockchain,
final BftExecutors bftExecutors, final BftExecutors bftExecutors,
final BftEventHandler controller, final QbftEventHandler controller,
final QbftFinalState finalState, final QbftFinalState finalState,
final EventMultiplexer eventMultiplexer, final EventMultiplexer eventMultiplexer,
final MessageFactory messageFactory, final MessageFactory messageFactory,
@@ -96,11 +102,11 @@ public class TestContext {
controller.start(); controller.start();
} }
public MutableBlockchain getBlockchain() { public QbftBlockchain getBlockchain() {
return blockchain; return new QbftBlockchainAdaptor(blockchain);
} }
public BftEventHandler getController() { public QbftEventHandler getController() {
return controller; return controller;
} }
@@ -129,25 +135,30 @@ public class TestContext {
public QbftBlock createBlockForProposalFromChainHead( public QbftBlock createBlockForProposalFromChainHead(
final long timestamp, final Address proposer) { final long timestamp, final Address proposer) {
// this implies that EVERY block will have this node as the proposer :/ // this implies that EVERY block will have this node as the proposer :/
return createBlockForProposal(blockchain.getChainHeadHeader(), timestamp, proposer, 0); return createBlockForProposal(
new QbftBlockHeaderAdaptor(blockchain.getChainHeadHeader()), timestamp, proposer, 0);
} }
public QbftBlock createBlockForProposalFromChainHead( public QbftBlock createBlockForProposalFromChainHead(
final long timestamp, final Address proposer, final int roundNumber) { final long timestamp, final Address proposer, final int roundNumber) {
// this implies that EVERY block will have this node as the proposer :/ // this implies that EVERY block will have this node as the proposer :/
return createBlockForProposal( return createBlockForProposal(
blockchain.getChainHeadHeader(), timestamp, proposer, roundNumber); new QbftBlockHeaderAdaptor(blockchain.getChainHeadHeader()),
timestamp,
proposer,
roundNumber);
} }
public QbftBlock createBlockForProposal( public QbftBlock createBlockForProposal(
final BlockHeader parent, final QbftBlockHeader parent,
final long timestamp, final long timestamp,
final Address proposer, final Address proposer,
final int roundNumber) { final int roundNumber) {
final QbftBlock block = final QbftBlock block =
finalState.getBlockCreatorFactory().create(roundNumber).createBlock(timestamp, parent); finalState.getBlockCreatorFactory().create(roundNumber).createBlock(timestamp, parent);
final BlockHeaderBuilder headerBuilder = BlockHeaderBuilder.fromHeader(block.getHeader()); final BlockHeaderBuilder headerBuilder =
BlockHeaderBuilder.fromHeader(BlockUtil.toBesuBlockHeader(block.getHeader()));
headerBuilder headerBuilder
.coinbase(proposer) .coinbase(proposer)
.blockHeaderFunctions(BftBlockHeaderFunctions.forCommittedSeal(bftExtraDataCodec)); .blockHeaderFunctions(BftBlockHeaderFunctions.forCommittedSeal(bftExtraDataCodec));
@@ -158,7 +169,7 @@ public class TestContext {
} }
public QbftBlock createBlockForProposal( public QbftBlock createBlockForProposal(
final BlockHeader parent, final long timestamp, final Address proposer) { final QbftBlockHeader parent, final long timestamp, final Address proposer) {
return createBlockForProposal(parent, timestamp, proposer, 0); return createBlockForProposal(parent, timestamp, proposer, 0);
} }
@@ -192,12 +203,16 @@ public class TestContext {
return blockchain.getChainHeadBlockNumber(); return blockchain.getChainHeadBlockNumber();
} }
public ValidatorProvider getValidatorProvider() { public QbftValidatorProvider getValidatorProvider() {
return validatorProvider; return new QbftValidatorProviderAdaptor(validatorProvider);
} }
public void appendBlock(final QbftBlock signedCurrentHeightBlock) { public void appendBlock(final QbftBlock signedCurrentHeightBlock) {
blockchain.appendBlock( blockchain.appendBlock(
BlockUtil.toBesuBlock(signedCurrentHeightBlock), Collections.emptyList()); BlockUtil.toBesuBlock(signedCurrentHeightBlock), Collections.emptyList());
} }
public QbftBlockHeader getBlockHeader(final int blockNumber) {
return new QbftBlockHeaderAdaptor(blockchain.getBlockHeader(blockNumber).get());
}
} }

View File

@@ -32,6 +32,7 @@ import org.hyperledger.besu.config.StubGenesisConfigOptions;
import org.hyperledger.besu.consensus.common.BftValidatorOverrides; import org.hyperledger.besu.consensus.common.BftValidatorOverrides;
import org.hyperledger.besu.consensus.common.EpochManager; import org.hyperledger.besu.consensus.common.EpochManager;
import org.hyperledger.besu.consensus.common.ForksSchedule; import org.hyperledger.besu.consensus.common.ForksSchedule;
import org.hyperledger.besu.consensus.common.bft.BftBlockHashing;
import org.hyperledger.besu.consensus.common.bft.BftBlockHeaderFunctions; import org.hyperledger.besu.consensus.common.bft.BftBlockHeaderFunctions;
import org.hyperledger.besu.consensus.common.bft.BftBlockInterface; import org.hyperledger.besu.consensus.common.bft.BftBlockInterface;
import org.hyperledger.besu.consensus.common.bft.BftContext; import org.hyperledger.besu.consensus.common.bft.BftContext;
@@ -54,7 +55,6 @@ import org.hyperledger.besu.consensus.common.bft.inttest.NodeParams;
import org.hyperledger.besu.consensus.common.bft.inttest.StubValidatorMulticaster; import org.hyperledger.besu.consensus.common.bft.inttest.StubValidatorMulticaster;
import org.hyperledger.besu.consensus.common.bft.inttest.StubbedSynchronizerUpdater; 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.inttest.TestTransitions;
import org.hyperledger.besu.consensus.common.bft.statemachine.BftEventHandler;
import org.hyperledger.besu.consensus.common.bft.statemachine.FutureMessageBuffer; import org.hyperledger.besu.consensus.common.bft.statemachine.FutureMessageBuffer;
import org.hyperledger.besu.consensus.common.validator.ValidatorProvider; import org.hyperledger.besu.consensus.common.validator.ValidatorProvider;
import org.hyperledger.besu.consensus.common.validator.blockbased.BlockValidatorProvider; import org.hyperledger.besu.consensus.common.validator.blockbased.BlockValidatorProvider;
@@ -62,12 +62,17 @@ import org.hyperledger.besu.consensus.qbft.MutableQbftConfigOptions;
import org.hyperledger.besu.consensus.qbft.QbftExtraDataCodec; import org.hyperledger.besu.consensus.qbft.QbftExtraDataCodec;
import org.hyperledger.besu.consensus.qbft.QbftForksSchedulesFactory; import org.hyperledger.besu.consensus.qbft.QbftForksSchedulesFactory;
import org.hyperledger.besu.consensus.qbft.QbftProtocolScheduleBuilder; import org.hyperledger.besu.consensus.qbft.QbftProtocolScheduleBuilder;
import org.hyperledger.besu.consensus.qbft.adaptor.BftEventHandlerAdaptor;
import org.hyperledger.besu.consensus.qbft.adaptor.QbftBlockCodecAdaptor; import org.hyperledger.besu.consensus.qbft.adaptor.QbftBlockCodecAdaptor;
import org.hyperledger.besu.consensus.qbft.adaptor.QbftBlockCreatorFactoryAdaptor; import org.hyperledger.besu.consensus.qbft.adaptor.QbftBlockCreatorFactoryAdaptor;
import org.hyperledger.besu.consensus.qbft.adaptor.QbftBlockHashingAdaptor;
import org.hyperledger.besu.consensus.qbft.adaptor.QbftBlockInterfaceAdaptor; import org.hyperledger.besu.consensus.qbft.adaptor.QbftBlockInterfaceAdaptor;
import org.hyperledger.besu.consensus.qbft.adaptor.QbftBlockchainAdaptor;
import org.hyperledger.besu.consensus.qbft.adaptor.QbftExtraDataProviderAdaptor; import org.hyperledger.besu.consensus.qbft.adaptor.QbftExtraDataProviderAdaptor;
import org.hyperledger.besu.consensus.qbft.adaptor.QbftFinalStateImpl; import org.hyperledger.besu.consensus.qbft.adaptor.QbftFinalStateImpl;
import org.hyperledger.besu.consensus.qbft.adaptor.QbftProtocolScheduleAdaptor; import org.hyperledger.besu.consensus.qbft.adaptor.QbftProtocolScheduleAdaptor;
import org.hyperledger.besu.consensus.qbft.adaptor.QbftValidatorModeTransitionLoggerAdaptor;
import org.hyperledger.besu.consensus.qbft.adaptor.QbftValidatorProviderAdaptor;
import org.hyperledger.besu.consensus.qbft.blockcreation.QbftBlockCreatorFactory; import org.hyperledger.besu.consensus.qbft.blockcreation.QbftBlockCreatorFactory;
import org.hyperledger.besu.consensus.qbft.core.network.QbftGossip; 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.payload.MessageFactory;
@@ -75,16 +80,19 @@ import org.hyperledger.besu.consensus.qbft.core.statemachine.QbftBlockHeightMana
import org.hyperledger.besu.consensus.qbft.core.statemachine.QbftController; 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.statemachine.QbftRoundFactory;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockCodec; import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockCodec;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockHashing;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockInterface; 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.QbftContext;
import org.hyperledger.besu.consensus.qbft.core.types.QbftEventHandler;
import org.hyperledger.besu.consensus.qbft.core.types.QbftExtraDataProvider; 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.QbftFinalState;
import org.hyperledger.besu.consensus.qbft.core.types.QbftMinedBlockObserver; import org.hyperledger.besu.consensus.qbft.core.types.QbftMinedBlockObserver;
import org.hyperledger.besu.consensus.qbft.core.types.QbftValidatorProvider;
import org.hyperledger.besu.consensus.qbft.core.validation.MessageValidatorFactory; 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; import org.hyperledger.besu.consensus.qbft.validator.ForkingValidatorProvider;
import org.hyperledger.besu.consensus.qbft.validator.TransactionValidatorProvider; import org.hyperledger.besu.consensus.qbft.validator.TransactionValidatorProvider;
import org.hyperledger.besu.consensus.qbft.validator.ValidatorContractController; import org.hyperledger.besu.consensus.qbft.validator.ValidatorContractController;
import org.hyperledger.besu.consensus.qbft.validator.ValidatorModeTransitionLogger;
import org.hyperledger.besu.cryptoservices.NodeKey; import org.hyperledger.besu.cryptoservices.NodeKey;
import org.hyperledger.besu.datatypes.Address; import org.hyperledger.besu.datatypes.Address;
import org.hyperledger.besu.datatypes.Hash; import org.hyperledger.besu.datatypes.Hash;
@@ -148,7 +156,7 @@ public class TestContextBuilder {
"UnusedVariable") // false positive https://github.com/google/error-prone/issues/2713 "UnusedVariable") // false positive https://github.com/google/error-prone/issues/2713
private record ControllerAndState( private record ControllerAndState(
BftExecutors bftExecutors, BftExecutors bftExecutors,
BftEventHandler eventHandler, QbftEventHandler eventHandler,
QbftFinalState finalState, QbftFinalState finalState,
EventMultiplexer eventMultiplexer, EventMultiplexer eventMultiplexer,
MessageFactory messageFactory, MessageFactory messageFactory,
@@ -470,6 +478,8 @@ public class TestContextBuilder {
final ValidatorProvider validatorProvider = final ValidatorProvider validatorProvider =
new ForkingValidatorProvider( new ForkingValidatorProvider(
blockChain, forksSchedule, blockValidatorProvider, transactionValidatorProvider); blockChain, forksSchedule, blockValidatorProvider, transactionValidatorProvider);
final QbftValidatorProvider qbftValidatorProvider =
new QbftValidatorProviderAdaptor(validatorProvider);
final ProtocolContext bftProtocolContext = final ProtocolContext bftProtocolContext =
new ProtocolContext( new ProtocolContext(
@@ -481,7 +491,7 @@ public class TestContextBuilder {
new ProtocolContext( new ProtocolContext(
blockChain, blockChain,
worldStateArchive, worldStateArchive,
new QbftContext(validatorProvider, qbftBlockInterface), new QbftContext(qbftValidatorProvider, qbftBlockInterface),
new BadBlockManager()); new BadBlockManager());
final TransactionPoolConfiguration poolConf = final TransactionPoolConfiguration poolConf =
@@ -555,10 +565,15 @@ public class TestContextBuilder {
blockChain.getChainHeadBlockNumber()); blockChain.getChainHeadBlockNumber());
final QbftExtraDataProvider qbftExtraDataProvider = final QbftExtraDataProvider qbftExtraDataProvider =
new QbftExtraDataProviderAdaptor(BFT_EXTRA_DATA_ENCODER); new QbftExtraDataProviderAdaptor(BFT_EXTRA_DATA_ENCODER);
final QbftBlockHashing blockHashing =
new QbftBlockHashingAdaptor(new BftBlockHashing(BFT_EXTRA_DATA_ENCODER));
final QbftValidatorModeTransitionLoggerAdaptor validatorModeTransitionLogger =
new QbftValidatorModeTransitionLoggerAdaptor(
new ValidatorModeTransitionLogger(forksSchedule));
final QbftController qbftController = final QbftController qbftController =
new QbftController( new QbftController(
blockChain, new QbftBlockchainAdaptor(blockChain),
finalState, finalState,
new QbftBlockHeightManagerFactory( new QbftBlockHeightManagerFactory(
finalState, finalState,
@@ -570,17 +585,19 @@ public class TestContextBuilder {
messageValidatorFactory, messageValidatorFactory,
messageFactory, messageFactory,
BFT_EXTRA_DATA_ENCODER, BFT_EXTRA_DATA_ENCODER,
qbftExtraDataProvider), qbftExtraDataProvider,
blockHashing),
messageValidatorFactory, messageValidatorFactory,
messageFactory, messageFactory,
new ValidatorModeTransitionLogger(forksSchedule)), validatorModeTransitionLogger),
gossiper, gossiper,
duplicateMessageTracker, duplicateMessageTracker,
futureMessageBuffer, futureMessageBuffer,
synchronizerUpdater, synchronizerUpdater,
blockEncoder); blockEncoder);
final EventMultiplexer eventMultiplexer = new EventMultiplexer(qbftController); final EventMultiplexer eventMultiplexer =
new EventMultiplexer(new BftEventHandlerAdaptor(qbftController));
//////////////////////////// END QBFT BesuController //////////////////////////// //////////////////////////// END QBFT BesuController ////////////////////////////
return new ControllerAndState( return new ControllerAndState(

View File

@@ -19,7 +19,6 @@ import static org.hyperledger.besu.consensus.qbft.core.support.IntegrationTestHe
import org.hyperledger.besu.consensus.common.bft.BftExtraDataCodec; import org.hyperledger.besu.consensus.common.bft.BftExtraDataCodec;
import org.hyperledger.besu.consensus.common.bft.ConsensusRoundIdentifier; import org.hyperledger.besu.consensus.common.bft.ConsensusRoundIdentifier;
import org.hyperledger.besu.consensus.common.bft.events.NewChainHead;
import org.hyperledger.besu.consensus.qbft.QbftExtraDataCodec; import org.hyperledger.besu.consensus.qbft.QbftExtraDataCodec;
import org.hyperledger.besu.consensus.qbft.core.messagewrappers.Commit; 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.Prepare;
@@ -28,6 +27,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.TestContext;
import org.hyperledger.besu.consensus.qbft.core.support.TestContextBuilder; import org.hyperledger.besu.consensus.qbft.core.support.TestContextBuilder;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlock; import org.hyperledger.besu.consensus.qbft.core.types.QbftBlock;
import org.hyperledger.besu.consensus.qbft.core.types.QbftNewChainHead;
import java.time.Clock; import java.time.Clock;
import java.time.Instant; import java.time.Instant;
@@ -94,7 +94,7 @@ public class FutureHeightTest {
assertThat(context.getCurrentChainHeight()).isEqualTo(1); assertThat(context.getCurrentChainHeight()).isEqualTo(1);
context context
.getController() .getController()
.handleNewBlockEvent(new NewChainHead(signedCurrentHeightBlock.getHeader())); .handleNewBlockEvent(new QbftNewChainHead(signedCurrentHeightBlock.getHeader()));
final Prepare expectedPrepareMessage = final Prepare expectedPrepareMessage =
localNodeMessageFactory.createPrepare(futureHeightRoundId, futureHeightBlock.getHash()); localNodeMessageFactory.createPrepare(futureHeightRoundId, futureHeightBlock.getHash());
@@ -132,7 +132,7 @@ public class FutureHeightTest {
assertThat(context.getCurrentChainHeight()).isEqualTo(1); assertThat(context.getCurrentChainHeight()).isEqualTo(1);
context context
.getController() .getController()
.handleNewBlockEvent(new NewChainHead(signedCurrentHeightBlock.getHeader())); .handleNewBlockEvent(new QbftNewChainHead(signedCurrentHeightBlock.getHeader()));
// Inject prepares and commits from all peers for the 'previous' round (i.e. the height // Inject prepares and commits from all peers for the 'previous' round (i.e. the height
// from before the block arrived). // from before the block arrived).
@@ -156,7 +156,7 @@ public class FutureHeightTest {
context context
.getController() .getController()
.handleNewBlockEvent(new NewChainHead(context.getBlockchain().getChainHeadHeader())); .handleNewBlockEvent(new QbftNewChainHead(context.getBlockchain().getChainHeadHeader()));
// Should only require 1 more prepare to close it out // Should only require 1 more prepare to close it out
peers.getNonProposing(1).injectPrepare(roundId, currentHeightBlock.getHash()); peers.getNonProposing(1).injectPrepare(roundId, currentHeightBlock.getHash());
@@ -201,7 +201,7 @@ public class FutureHeightTest {
assertThat(context.getCurrentChainHeight()).isEqualTo(1); assertThat(context.getCurrentChainHeight()).isEqualTo(1);
context context
.getController() .getController()
.handleNewBlockEvent(new NewChainHead(signedCurrentHeightBlock.getHeader())); .handleNewBlockEvent(new QbftNewChainHead(signedCurrentHeightBlock.getHeader()));
peers.verifyNoMessagesReceived(); peers.verifyNoMessagesReceived();
peers.getProposer().injectProposal(nextHeightRoundId, nextHeightBlock); peers.getProposer().injectProposal(nextHeightRoundId, nextHeightBlock);
@@ -221,7 +221,7 @@ public class FutureHeightTest {
assertThat(context.getCurrentChainHeight()).isEqualTo(2); assertThat(context.getCurrentChainHeight()).isEqualTo(2);
context context
.getController() .getController()
.handleNewBlockEvent(new NewChainHead(signedNextHeightBlock.getHeader())); .handleNewBlockEvent(new QbftNewChainHead(signedNextHeightBlock.getHeader()));
final Prepare expectedFuturePrepareMessage = final Prepare expectedFuturePrepareMessage =
localNodeMessageFactory.createPrepare(futureHeightRoundId, futureHeightBlock.getHash()); localNodeMessageFactory.createPrepare(futureHeightRoundId, futureHeightBlock.getHash());

View File

@@ -17,7 +17,6 @@ package org.hyperledger.besu.consensus.qbft.core.test;
import static java.util.Collections.emptyList; import static java.util.Collections.emptyList;
import org.hyperledger.besu.consensus.common.bft.ConsensusRoundIdentifier; import org.hyperledger.besu.consensus.common.bft.ConsensusRoundIdentifier;
import org.hyperledger.besu.consensus.common.bft.events.NewChainHead;
import org.hyperledger.besu.consensus.qbft.QbftExtraDataCodec; import org.hyperledger.besu.consensus.qbft.QbftExtraDataCodec;
import org.hyperledger.besu.consensus.qbft.core.messagedata.ProposalMessageData; import org.hyperledger.besu.consensus.qbft.core.messagedata.ProposalMessageData;
import org.hyperledger.besu.consensus.qbft.core.messagewrappers.Commit; import org.hyperledger.besu.consensus.qbft.core.messagewrappers.Commit;
@@ -30,6 +29,7 @@ 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.TestContextBuilder;
import org.hyperledger.besu.consensus.qbft.core.support.ValidatorPeer; import org.hyperledger.besu.consensus.qbft.core.support.ValidatorPeer;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlock; import org.hyperledger.besu.consensus.qbft.core.types.QbftBlock;
import org.hyperledger.besu.consensus.qbft.core.types.QbftNewChainHead;
import org.hyperledger.besu.cryptoservices.NodeKeyUtils; import org.hyperledger.besu.cryptoservices.NodeKeyUtils;
import java.time.Clock; import java.time.Clock;
@@ -170,7 +170,7 @@ public class GossipTest {
context.appendBlock(signedCurrentHeightBlock); context.appendBlock(signedCurrentHeightBlock);
context context
.getController() .getController()
.handleNewBlockEvent(new NewChainHead(signedCurrentHeightBlock.getHeader())); .handleNewBlockEvent(new QbftNewChainHead(signedCurrentHeightBlock.getHeader()));
peers.verifyMessagesReceivedNonProposing(futurePrepare); peers.verifyMessagesReceivedNonProposing(futurePrepare);
} }

View File

@@ -19,7 +19,6 @@ import static org.hyperledger.besu.consensus.qbft.core.support.IntegrationTestHe
import org.hyperledger.besu.consensus.common.bft.ConsensusRoundIdentifier; import org.hyperledger.besu.consensus.common.bft.ConsensusRoundIdentifier;
import org.hyperledger.besu.consensus.common.bft.events.BlockTimerExpiry; 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.events.RoundExpiry;
import org.hyperledger.besu.consensus.qbft.core.messagewrappers.Commit; 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.Prepare;
@@ -29,6 +28,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.TestContext;
import org.hyperledger.besu.consensus.qbft.core.support.TestContextBuilder; import org.hyperledger.besu.consensus.qbft.core.support.TestContextBuilder;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlock; import org.hyperledger.besu.consensus.qbft.core.types.QbftBlock;
import org.hyperledger.besu.consensus.qbft.core.types.QbftNewChainHead;
import java.time.Clock; import java.time.Clock;
import java.time.Instant; import java.time.Instant;
@@ -138,7 +138,7 @@ public class LocalNodeIsProposerTest {
context context
.getController() .getController()
.handleNewBlockEvent(new NewChainHead(expectedProposedBlock.getHeader())); .handleNewBlockEvent(new QbftNewChainHead(expectedProposedBlock.getHeader()));
peers.verifyNoMessagesReceived(); peers.verifyNoMessagesReceived();
} }

View File

@@ -20,10 +20,10 @@ import org.hyperledger.besu.config.BftFork;
import org.hyperledger.besu.config.JsonUtil; import org.hyperledger.besu.config.JsonUtil;
import org.hyperledger.besu.config.QbftFork; import org.hyperledger.besu.config.QbftFork;
import org.hyperledger.besu.consensus.common.bft.BftEventQueue; import org.hyperledger.besu.consensus.common.bft.BftEventQueue;
import org.hyperledger.besu.consensus.common.bft.events.NewChainHead;
import org.hyperledger.besu.consensus.qbft.core.support.TestContext; 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.TestContextBuilder;
import org.hyperledger.besu.ethereum.core.BlockHeader; import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockHeader;
import org.hyperledger.besu.consensus.qbft.core.types.QbftNewChainHead;
import org.hyperledger.besu.testutil.TestClock; import org.hyperledger.besu.testutil.TestClock;
import java.time.Instant; import java.time.Instant;
@@ -63,13 +63,13 @@ public class TransitionsTest {
context context
.getController() .getController()
.handleNewBlockEvent(new NewChainHead(context.getBlockchain().getChainHeadHeader())); .handleNewBlockEvent(new QbftNewChainHead(context.getBlockchain().getChainHeadHeader()));
clock.stepMillis(20_000); clock.stepMillis(20_000);
context.getEventMultiplexer().handleBftEvent(bftEventQueue.poll(1, TimeUnit.SECONDS)); context.getEventMultiplexer().handleBftEvent(bftEventQueue.poll(1, TimeUnit.SECONDS));
final BlockHeader genesisBlock = context.getBlockchain().getBlockHeader(0).get(); final QbftBlockHeader genesisBlock = context.getBlockHeader(0);
final BlockHeader blockHeader1 = context.getBlockchain().getBlockHeader(1).get(); final QbftBlockHeader blockHeader1 = context.getBlockHeader(1);
final BlockHeader blockHeader2 = context.getBlockchain().getBlockHeader(2).get(); final QbftBlockHeader blockHeader2 = context.getBlockHeader(2);
assertThat(blockHeader1.getTimestamp()).isEqualTo(genesisBlock.getTimestamp() + 10); assertThat(blockHeader1.getTimestamp()).isEqualTo(genesisBlock.getTimestamp() + 10);
assertThat(blockHeader2.getTimestamp()).isEqualTo(blockHeader1.getTimestamp() + 20); assertThat(blockHeader2.getTimestamp()).isEqualTo(blockHeader1.getTimestamp() + 20);

View File

@@ -23,18 +23,20 @@ import org.hyperledger.besu.config.QbftFork;
import org.hyperledger.besu.config.QbftFork.VALIDATOR_SELECTION_MODE; import org.hyperledger.besu.config.QbftFork.VALIDATOR_SELECTION_MODE;
import org.hyperledger.besu.consensus.common.bft.ConsensusRoundIdentifier; import org.hyperledger.besu.consensus.common.bft.ConsensusRoundIdentifier;
import org.hyperledger.besu.consensus.common.bft.events.BlockTimerExpiry; 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.inttest.NodeParams; import org.hyperledger.besu.consensus.common.bft.inttest.NodeParams;
import org.hyperledger.besu.consensus.common.validator.ValidatorProvider;
import org.hyperledger.besu.consensus.qbft.QbftExtraDataCodec; import org.hyperledger.besu.consensus.qbft.QbftExtraDataCodec;
import org.hyperledger.besu.consensus.qbft.adaptor.QbftExtraDataProviderAdaptor;
import org.hyperledger.besu.consensus.qbft.core.support.RoundSpecificPeers; 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.TestContext;
import org.hyperledger.besu.consensus.qbft.core.support.TestContextBuilder; 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.support.ValidatorPeer;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlock; import org.hyperledger.besu.consensus.qbft.core.types.QbftBlock;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockHeader;
import org.hyperledger.besu.consensus.qbft.core.types.QbftExtraDataProvider;
import org.hyperledger.besu.consensus.qbft.core.types.QbftNewChainHead;
import org.hyperledger.besu.consensus.qbft.core.types.QbftValidatorProvider;
import org.hyperledger.besu.cryptoservices.NodeKeyUtils; import org.hyperledger.besu.cryptoservices.NodeKeyUtils;
import org.hyperledger.besu.datatypes.Address; import org.hyperledger.besu.datatypes.Address;
import org.hyperledger.besu.ethereum.core.BlockHeader;
import org.hyperledger.besu.testutil.TestClock; import org.hyperledger.besu.testutil.TestClock;
import java.time.Instant; import java.time.Instant;
@@ -66,7 +68,8 @@ public class ValidatorContractTest {
private TestClock clock; private TestClock clock;
private final QbftExtraDataCodec extraDataCodec = new QbftExtraDataCodec(); private final QbftExtraDataProvider extraDataProvider =
new QbftExtraDataProviderAdaptor(new QbftExtraDataCodec());
@BeforeEach @BeforeEach
public void setup() { public void setup() {
@@ -87,9 +90,9 @@ public class ValidatorContractTest {
createNewBlockAsProposer(context, 1); createNewBlockAsProposer(context, 1);
final ValidatorProvider validatorProvider = context.getValidatorProvider(); final QbftValidatorProvider validatorProvider = context.getValidatorProvider();
final BlockHeader genesisBlock = context.getBlockchain().getBlockHeader(0).get(); final QbftBlockHeader genesisBlock = context.getBlockHeader(0);
final BlockHeader block1 = context.getBlockchain().getBlockHeader(1).get(); final QbftBlockHeader block1 = context.getBlockHeader(1);
assertThat(validatorProvider.getValidatorsForBlock(genesisBlock)).containsExactly(NODE_ADDRESS); assertThat(validatorProvider.getValidatorsForBlock(genesisBlock)).containsExactly(NODE_ADDRESS);
assertThat(validatorProvider.getValidatorsForBlock(block1)).containsExactly(NODE_ADDRESS); assertThat(validatorProvider.getValidatorsForBlock(block1)).containsExactly(NODE_ADDRESS);
} }
@@ -109,9 +112,9 @@ public class ValidatorContractTest {
createNewBlockAsProposer(context, 1); createNewBlockAsProposer(context, 1);
final ValidatorProvider validatorProvider = context.getValidatorProvider(); final QbftValidatorProvider validatorProvider = context.getValidatorProvider();
final BlockHeader genesisBlock = context.getBlockchain().getBlockHeader(0).get(); final QbftBlockHeader genesisBlock = context.getBlockHeader(0);
final BlockHeader block1 = context.getBlockchain().getBlockHeader(1).get(); final QbftBlockHeader block1 = context.getBlockHeader(1);
assertThat(validatorProvider.getValidatorsForBlock(genesisBlock)).containsExactly(NODE_ADDRESS); assertThat(validatorProvider.getValidatorsForBlock(genesisBlock)).containsExactly(NODE_ADDRESS);
assertThat(validatorProvider.getValidatorsForBlock(block1)).containsExactly(NODE_ADDRESS); assertThat(validatorProvider.getValidatorsForBlock(block1)).containsExactly(NODE_ADDRESS);
} }
@@ -133,9 +136,9 @@ public class ValidatorContractTest {
createNewBlockAsProposer(context, 1); createNewBlockAsProposer(context, 1);
final ValidatorProvider validatorProvider = context.getValidatorProvider(); final QbftValidatorProvider validatorProvider = context.getValidatorProvider();
final BlockHeader genesisBlock = context.getBlockchain().getBlockHeader(0).get(); final QbftBlockHeader genesisBlock = context.getBlockHeader(0);
final BlockHeader block1 = context.getBlockchain().getBlockHeader(1).get(); final QbftBlockHeader block1 = context.getBlockHeader(1);
assertThat(validatorProvider.getValidatorsForBlock(genesisBlock)).containsExactly(NODE_ADDRESS); assertThat(validatorProvider.getValidatorsForBlock(genesisBlock)).containsExactly(NODE_ADDRESS);
assertThat(validatorProvider.getValidatorsForBlock(block1)).containsExactly(NODE_ADDRESS); assertThat(validatorProvider.getValidatorsForBlock(block1)).containsExactly(NODE_ADDRESS);
} }
@@ -160,9 +163,9 @@ public class ValidatorContractTest {
context, 1, context, 1,
266L); // 10s ahead of genesis timestamp in genesis_validator_contract_shanghai.json 266L); // 10s ahead of genesis timestamp in genesis_validator_contract_shanghai.json
final ValidatorProvider validatorProvider = context.getValidatorProvider(); final QbftValidatorProvider validatorProvider = context.getValidatorProvider();
final BlockHeader genesisBlock = context.getBlockchain().getBlockHeader(0).get(); final QbftBlockHeader genesisBlock = context.getBlockHeader(0);
final BlockHeader block1 = context.getBlockchain().getBlockHeader(1).get(); final QbftBlockHeader block1 = context.getBlockHeader(1);
assertThat(validatorProvider.getValidatorsForBlock(genesisBlock)).containsExactly(NODE_ADDRESS); assertThat(validatorProvider.getValidatorsForBlock(genesisBlock)).containsExactly(NODE_ADDRESS);
assertThat(validatorProvider.getValidatorsForBlock(block1)).containsExactly(NODE_ADDRESS); assertThat(validatorProvider.getValidatorsForBlock(block1)).containsExactly(NODE_ADDRESS);
} }
@@ -184,9 +187,9 @@ public class ValidatorContractTest {
createNewBlockAsProposer(context, 1); createNewBlockAsProposer(context, 1);
final ValidatorProvider validatorProvider = context.getValidatorProvider(); final QbftValidatorProvider validatorProvider = context.getValidatorProvider();
final BlockHeader genesisBlock = context.getBlockchain().getBlockHeader(0).get(); final QbftBlockHeader genesisBlock = context.getBlockHeader(0);
final BlockHeader block1 = context.getBlockchain().getBlockHeader(1).get(); final QbftBlockHeader block1 = context.getBlockHeader(1);
assertThat(validatorProvider.getValidatorsForBlock(genesisBlock)).containsExactly(NODE_ADDRESS); assertThat(validatorProvider.getValidatorsForBlock(genesisBlock)).containsExactly(NODE_ADDRESS);
assertThat(validatorProvider.getValidatorsForBlock(block1)).containsExactly(NODE_ADDRESS); assertThat(validatorProvider.getValidatorsForBlock(block1)).containsExactly(NODE_ADDRESS);
} }
@@ -211,9 +214,9 @@ public class ValidatorContractTest {
context, 1, context, 1,
266L); // 10s ahead of genesis timestamp in genesis_validator_contract_shanghai.json 266L); // 10s ahead of genesis timestamp in genesis_validator_contract_shanghai.json
final ValidatorProvider validatorProvider = context.getValidatorProvider(); final QbftValidatorProvider validatorProvider = context.getValidatorProvider();
final BlockHeader genesisBlock = context.getBlockchain().getBlockHeader(0).get(); final QbftBlockHeader genesisBlock = context.getBlockHeader(0);
final BlockHeader block1 = context.getBlockchain().getBlockHeader(1).get(); final QbftBlockHeader block1 = context.getBlockHeader(1);
assertThat(validatorProvider.getValidatorsForBlock(genesisBlock)).containsExactly(NODE_ADDRESS); assertThat(validatorProvider.getValidatorsForBlock(genesisBlock)).containsExactly(NODE_ADDRESS);
assertThat(validatorProvider.getValidatorsForBlock(block1)).containsExactly(NODE_ADDRESS); assertThat(validatorProvider.getValidatorsForBlock(block1)).containsExactly(NODE_ADDRESS);
} }
@@ -244,17 +247,18 @@ public class ValidatorContractTest {
remotePeerProposesNewBlock(context, 1L); remotePeerProposesNewBlock(context, 1L);
final ValidatorProvider validatorProvider = context.getValidatorProvider(); final QbftValidatorProvider validatorProvider = context.getValidatorProvider();
final BlockHeader genesisBlock = context.getBlockchain().getBlockHeader(0).get(); final QbftBlockHeader genesisBlock = context.getBlockHeader(0);
final BlockHeader block1 = context.getBlockchain().getBlockHeader(1).get(); final QbftBlockHeader block1 = context.getBlockHeader(1);
assertThat(validatorProvider.getValidatorsForBlock(genesisBlock)).isEqualTo(block0Addresses); assertThat(validatorProvider.getValidatorsForBlock(genesisBlock)).isEqualTo(block0Addresses);
assertThat(extraDataCodec.decode(genesisBlock).getValidators()).containsExactly(NODE_ADDRESS); assertThat(extraDataProvider.getExtraData(genesisBlock).getValidators())
.containsExactly(NODE_ADDRESS);
// contract block extra data cannot contain validators or vote // contract block extra data cannot contain validators or vote
assertThat(validatorProvider.getValidatorsForBlock(block1)).isEqualTo(block1Addresses); assertThat(validatorProvider.getValidatorsForBlock(block1)).isEqualTo(block1Addresses);
assertThat(extraDataCodec.decode(block1).getValidators()).isEmpty(); assertThat(extraDataProvider.getExtraData(block1).getValidators()).isEmpty();
assertThat(extraDataCodec.decode(block1).getVote()).isEmpty(); assertThat(extraDataProvider.getExtraData(block1).getVote()).isEmpty();
} }
@Test @Test
@@ -282,19 +286,19 @@ public class ValidatorContractTest {
remotePeerProposesNewBlock(context, 1L); remotePeerProposesNewBlock(context, 1L);
final ValidatorProvider validatorProvider = context.getValidatorProvider(); final QbftValidatorProvider validatorProvider = context.getValidatorProvider();
final BlockHeader genesisBlock = context.getBlockchain().getBlockHeader(0).get(); final QbftBlockHeader genesisBlock = context.getBlockHeader(0);
final BlockHeader block1 = context.getBlockchain().getBlockHeader(1).get(); final QbftBlockHeader block1 = context.getBlockHeader(1);
// contract block extra data cannot contain validators or vote // contract block extra data cannot contain validators or vote
assertThat(validatorProvider.getValidatorsForBlock(genesisBlock)).isEqualTo(block0Addresses); assertThat(validatorProvider.getValidatorsForBlock(genesisBlock)).isEqualTo(block0Addresses);
assertThat(extraDataCodec.decode(genesisBlock).getValidators()).isEmpty(); assertThat(extraDataProvider.getExtraData(genesisBlock).getValidators()).isEmpty();
assertThat(extraDataCodec.decode(genesisBlock).getVote()).isEmpty(); assertThat(extraDataProvider.getExtraData(genesisBlock).getVote()).isEmpty();
// contract block extra data cannot contain validators or vote // contract block extra data cannot contain validators or vote
assertThat(validatorProvider.getValidatorsForBlock(block1)).isEqualTo(block1Addresses); assertThat(validatorProvider.getValidatorsForBlock(block1)).isEqualTo(block1Addresses);
assertThat(extraDataCodec.decode(block1).getValidators()).isEmpty(); assertThat(extraDataProvider.getExtraData(block1).getValidators()).isEmpty();
assertThat(extraDataCodec.decode(block1).getVote()).isEmpty(); assertThat(extraDataProvider.getExtraData(block1).getVote()).isEmpty();
} }
@Test @Test
@@ -325,24 +329,24 @@ public class ValidatorContractTest {
clock.step(TestContextBuilder.BLOCK_TIMER_SEC, SECONDS); clock.step(TestContextBuilder.BLOCK_TIMER_SEC, SECONDS);
remotePeerProposesNewBlock(context, 2L); remotePeerProposesNewBlock(context, 2L);
final ValidatorProvider validatorProvider = context.getValidatorProvider(); final QbftValidatorProvider validatorProvider = context.getValidatorProvider();
final BlockHeader genesisBlock = context.getBlockchain().getBlockHeader(0).get(); final QbftBlockHeader genesisBlock = context.getBlockHeader(0);
final BlockHeader block1 = context.getBlockchain().getBlockHeader(1).get(); final QbftBlockHeader block1 = context.getBlockHeader(1);
final BlockHeader block2 = context.getBlockchain().getBlockHeader(2).get(); final QbftBlockHeader block2 = context.getBlockHeader(2);
// contract block extra data cannot contain validators or vote // contract block extra data cannot contain validators or vote
assertThat(validatorProvider.getValidatorsForBlock(genesisBlock)).isEqualTo(block0Addresses); assertThat(validatorProvider.getValidatorsForBlock(genesisBlock)).isEqualTo(block0Addresses);
assertThat(extraDataCodec.decode(genesisBlock).getValidators()).isEmpty(); assertThat(extraDataProvider.getExtraData(genesisBlock).getValidators()).isEmpty();
assertThat(extraDataCodec.decode(genesisBlock).getVote()).isEmpty(); assertThat(extraDataProvider.getExtraData(genesisBlock).getVote()).isEmpty();
// uses overridden validators // uses overridden validators
assertThat(validatorProvider.getValidatorsForBlock(block1)).isEqualTo(block1Addresses); assertThat(validatorProvider.getValidatorsForBlock(block1)).isEqualTo(block1Addresses);
assertThat(extraDataCodec.decode(block1).getValidators()) assertThat(extraDataProvider.getExtraData(block1).getValidators())
.containsExactly(NODE_2_ADDRESS, NODE_ADDRESS); .containsExactly(NODE_2_ADDRESS, NODE_ADDRESS);
// uses cached validators // uses cached validators
assertThat(validatorProvider.getValidatorsForBlock(block2)).isEqualTo(block1Addresses); assertThat(validatorProvider.getValidatorsForBlock(block2)).isEqualTo(block1Addresses);
assertThat(extraDataCodec.decode(block2).getValidators()) assertThat(extraDataProvider.getExtraData(block2).getValidators())
.containsExactly(NODE_2_ADDRESS, NODE_ADDRESS); .containsExactly(NODE_2_ADDRESS, NODE_ADDRESS);
} }
@@ -381,28 +385,29 @@ public class ValidatorContractTest {
clock.step(TestContextBuilder.BLOCK_TIMER_SEC, SECONDS); clock.step(TestContextBuilder.BLOCK_TIMER_SEC, SECONDS);
remotePeerProposesNewBlock(context, 3L); remotePeerProposesNewBlock(context, 3L);
final ValidatorProvider validatorProvider = context.getValidatorProvider(); final QbftValidatorProvider validatorProvider = context.getValidatorProvider();
final BlockHeader genesisBlock = context.getBlockchain().getBlockHeader(0).get(); final QbftBlockHeader genesisBlock = context.getBlockHeader(0);
final BlockHeader block1 = context.getBlockchain().getBlockHeader(1).get(); final QbftBlockHeader block1 = context.getBlockHeader(1);
final BlockHeader block2 = context.getBlockchain().getBlockHeader(2).get(); final QbftBlockHeader block2 = context.getBlockHeader(2);
final BlockHeader block3 = context.getBlockchain().getBlockHeader(3).get(); final QbftBlockHeader block3 = context.getBlockHeader(3);
assertThat(validatorProvider.getValidatorsForBlock(genesisBlock)).isEqualTo(block0Addresses); assertThat(validatorProvider.getValidatorsForBlock(genesisBlock)).isEqualTo(block0Addresses);
assertThat(extraDataCodec.decode(genesisBlock).getValidators()).containsExactly(NODE_ADDRESS); assertThat(extraDataProvider.getExtraData(genesisBlock).getValidators())
.containsExactly(NODE_ADDRESS);
// contract block extra data cannot contain validators or vote // contract block extra data cannot contain validators or vote
assertThat(validatorProvider.getValidatorsForBlock(block1)).isEqualTo(block1Addresses); assertThat(validatorProvider.getValidatorsForBlock(block1)).isEqualTo(block1Addresses);
assertThat(extraDataCodec.decode(block1).getValidators()).isEmpty(); assertThat(extraDataProvider.getExtraData(block1).getValidators()).isEmpty();
assertThat(extraDataCodec.decode(block1).getVote()).isEmpty(); assertThat(extraDataProvider.getExtraData(block1).getVote()).isEmpty();
// uses overridden validators // uses overridden validators
assertThat(validatorProvider.getValidatorsForBlock(block2)).isEqualTo(block1Addresses); assertThat(validatorProvider.getValidatorsForBlock(block2)).isEqualTo(block1Addresses);
assertThat(extraDataCodec.decode(block2).getValidators()) assertThat(extraDataProvider.getExtraData(block2).getValidators())
.containsExactly(NODE_2_ADDRESS, NODE_ADDRESS); .containsExactly(NODE_2_ADDRESS, NODE_ADDRESS);
// uses cached validators // uses cached validators
assertThat(validatorProvider.getValidatorsForBlock(block3)).isEqualTo(block1Addresses); assertThat(validatorProvider.getValidatorsForBlock(block3)).isEqualTo(block1Addresses);
assertThat(extraDataCodec.decode(block3).getValidators()) assertThat(extraDataProvider.getExtraData(block3).getValidators())
.containsExactly(NODE_2_ADDRESS, NODE_ADDRESS); .containsExactly(NODE_2_ADDRESS, NODE_ADDRESS);
} }
@@ -438,23 +443,24 @@ public class ValidatorContractTest {
clock.step(TestContextBuilder.BLOCK_TIMER_SEC, SECONDS); clock.step(TestContextBuilder.BLOCK_TIMER_SEC, SECONDS);
remotePeerProposesNewBlock(context, 2L); remotePeerProposesNewBlock(context, 2L);
final ValidatorProvider validatorProvider = context.getValidatorProvider(); final QbftValidatorProvider validatorProvider = context.getValidatorProvider();
final BlockHeader genesisBlock = context.getBlockchain().getBlockHeader(0).get(); final QbftBlockHeader genesisBlock = context.getBlockHeader(0);
final BlockHeader block1 = context.getBlockchain().getBlockHeader(1).get(); final QbftBlockHeader block1 = context.getBlockHeader(1);
final BlockHeader block2 = context.getBlockchain().getBlockHeader(2).get(); final QbftBlockHeader block2 = context.getBlockHeader(2);
// contract block extra data cannot contain validators or vote // contract block extra data cannot contain validators or vote
assertThat(validatorProvider.getValidatorsForBlock(genesisBlock)).isEqualTo(block0Addresses); assertThat(validatorProvider.getValidatorsForBlock(genesisBlock)).isEqualTo(block0Addresses);
assertThat(extraDataCodec.decode(genesisBlock).getValidators()).isEmpty(); assertThat(extraDataProvider.getExtraData(genesisBlock).getValidators()).isEmpty();
assertThat(extraDataCodec.decode(genesisBlock).getVote()).isEmpty(); assertThat(extraDataProvider.getExtraData(genesisBlock).getVote()).isEmpty();
assertThat(validatorProvider.getValidatorsForBlock(block1)).isEqualTo(block1Addresses); assertThat(validatorProvider.getValidatorsForBlock(block1)).isEqualTo(block1Addresses);
assertThat(extraDataCodec.decode(block1).getValidators()).containsExactly(NODE_2_ADDRESS); assertThat(extraDataProvider.getExtraData(block1).getValidators())
.containsExactly(NODE_2_ADDRESS);
// contract block extra data cannot contain validators or vote // contract block extra data cannot contain validators or vote
assertThat(validatorProvider.getValidatorsForBlock(block2)).isEqualTo(block2Addresses); assertThat(validatorProvider.getValidatorsForBlock(block2)).isEqualTo(block2Addresses);
assertThat(extraDataCodec.decode(block2).getValidators()).isEmpty(); assertThat(extraDataProvider.getExtraData(block2).getValidators()).isEmpty();
assertThat(extraDataCodec.decode(block2).getVote()).isEmpty(); assertThat(extraDataProvider.getExtraData(block2).getVote()).isEmpty();
} }
private void createNewBlockAsProposer(final TestContext context, final long blockNumber) { private void createNewBlockAsProposer(final TestContext context, final long blockNumber) {
@@ -472,7 +478,7 @@ public class ValidatorContractTest {
assertThat(context.getCurrentChainHeight()).isEqualTo(blockNumber); assertThat(context.getCurrentChainHeight()).isEqualTo(blockNumber);
context context
.getController() .getController()
.handleNewBlockEvent(new NewChainHead(context.getBlockchain().getChainHeadHeader())); .handleNewBlockEvent(new QbftNewChainHead(context.getBlockchain().getChainHeadHeader()));
} }
private void createNewBlockAsProposerFixedTime( private void createNewBlockAsProposerFixedTime(
@@ -490,7 +496,7 @@ public class ValidatorContractTest {
assertThat(context.getCurrentChainHeight()).isEqualTo(blockNumber); assertThat(context.getCurrentChainHeight()).isEqualTo(blockNumber);
context context
.getController() .getController()
.handleNewBlockEvent(new NewChainHead(context.getBlockchain().getChainHeadHeader())); .handleNewBlockEvent(new QbftNewChainHead(context.getBlockchain().getChainHeadHeader()));
} }
private void remotePeerProposesNewBlock(final TestContext context, final long blockNumber) { private void remotePeerProposesNewBlock(final TestContext context, final long blockNumber) {
@@ -507,7 +513,7 @@ public class ValidatorContractTest {
context context
.getController() .getController()
.handleNewBlockEvent(new NewChainHead(context.getBlockchain().getChainHeadHeader())); .handleNewBlockEvent(new QbftNewChainHead(context.getBlockchain().getChainHeadHeader()));
} }
private QbftFork createContractFork(final long block, final Address contractAddress) { private QbftFork createContractFork(final long block, final Address contractAddress) {

View File

@@ -30,6 +30,7 @@ import org.hyperledger.besu.consensus.common.bft.RoundTimer;
import org.hyperledger.besu.consensus.common.bft.inttest.StubValidatorMulticaster; import org.hyperledger.besu.consensus.common.bft.inttest.StubValidatorMulticaster;
import org.hyperledger.besu.consensus.qbft.QbftExtraDataCodec; import org.hyperledger.besu.consensus.qbft.QbftExtraDataCodec;
import org.hyperledger.besu.consensus.qbft.adaptor.QbftBlockAdaptor; import org.hyperledger.besu.consensus.qbft.adaptor.QbftBlockAdaptor;
import org.hyperledger.besu.consensus.qbft.adaptor.QbftBlockHeaderAdaptor;
import org.hyperledger.besu.consensus.qbft.adaptor.QbftBlockInterfaceAdaptor; import org.hyperledger.besu.consensus.qbft.adaptor.QbftBlockInterfaceAdaptor;
import org.hyperledger.besu.consensus.qbft.core.network.QbftMessageTransmitter; 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.MessageFactory;
@@ -38,6 +39,8 @@ 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.QbftBlock;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockCodec; 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.QbftBlockCreator;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockHashing;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockHeader;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockImporter; 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.QbftContext;
import org.hyperledger.besu.consensus.qbft.core.types.QbftExtraDataProvider; import org.hyperledger.besu.consensus.qbft.core.types.QbftExtraDataProvider;
@@ -97,9 +100,10 @@ public class QbftRoundIntegrationTest {
private MessageFactory throwingMessageFactory; private MessageFactory throwingMessageFactory;
private QbftMessageTransmitter transmitter; private QbftMessageTransmitter transmitter;
@Mock private StubValidatorMulticaster multicaster; @Mock private StubValidatorMulticaster multicaster;
@Mock private BlockHeader parentHeader; @Mock private QbftBlockHeader parentHeader;
@Mock private QbftBlockCodec blockEncoder; @Mock private QbftBlockCodec blockEncoder;
@Mock private QbftExtraDataProvider qbftExtraDataProvider; @Mock private QbftExtraDataProvider qbftExtraDataProvider;
@Mock private QbftBlockHashing blockHashing;
private QbftBlock proposedBlock; private QbftBlock proposedBlock;
@@ -129,8 +133,9 @@ public class QbftRoundIntegrationTest {
headerTestFixture.number(1); headerTestFixture.number(1);
final BlockHeader header = headerTestFixture.buildHeader(); final BlockHeader header = headerTestFixture.buildHeader();
final Block block = new Block(header, new BlockBody(emptyList(), emptyList())); final Block block = new Block(header, new BlockBody(emptyList(), emptyList()));
final QbftBlockHeader qbftBlockHeader = new QbftBlockHeaderAdaptor(header);
proposedBlock = new QbftBlockAdaptor(block); proposedBlock = new QbftBlockAdaptor(block);
when(qbftExtraDataProvider.getExtraData(header)).thenReturn(proposedExtraData); when(qbftExtraDataProvider.getExtraData(qbftBlockHeader)).thenReturn(proposedExtraData);
when(protocolSchedule.getByBlockHeader(any())).thenReturn(protocolSpec); when(protocolSchedule.getByBlockHeader(any())).thenReturn(protocolSpec);
when(protocolSpec.getBlockImporter()).thenReturn(blockImporter); when(protocolSpec.getBlockImporter()).thenReturn(blockImporter);
@@ -163,6 +168,7 @@ public class QbftRoundIntegrationTest {
roundTimer, roundTimer,
bftExtraDataCodec, bftExtraDataCodec,
qbftExtraDataProvider, qbftExtraDataProvider,
blockHashing,
parentHeader); parentHeader);
round.handleProposalMessage( round.handleProposalMessage(
@@ -203,6 +209,7 @@ public class QbftRoundIntegrationTest {
roundTimer, roundTimer,
bftExtraDataCodec, bftExtraDataCodec,
qbftExtraDataProvider, qbftExtraDataProvider,
blockHashing,
parentHeader); parentHeader);
// inject a block first, then a prepare on it. // inject a block first, then a prepare on it.

View File

@@ -14,14 +14,43 @@
*/ */
package org.hyperledger.besu.consensus.qbft.core.statemachine; package org.hyperledger.besu.consensus.qbft.core.statemachine;
import org.hyperledger.besu.consensus.common.bft.statemachine.BaseBlockHeightManager; import org.hyperledger.besu.consensus.common.bft.ConsensusRoundIdentifier;
import org.hyperledger.besu.consensus.common.bft.events.RoundExpiry;
import org.hyperledger.besu.consensus.qbft.core.messagewrappers.Commit; 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.Prepare;
import org.hyperledger.besu.consensus.qbft.core.messagewrappers.Proposal; 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.messagewrappers.RoundChange;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockHeader;
/** The interface Base qbft block height manager. */ /** The interface Base qbft block height manager. */
public interface BaseQbftBlockHeightManager extends BaseBlockHeightManager { public interface BaseQbftBlockHeightManager {
/**
* Handle block timer expiry.
*
* @param roundIdentifier the round identifier
*/
void handleBlockTimerExpiry(ConsensusRoundIdentifier roundIdentifier);
/**
* Round expired.
*
* @param expire the expiry
*/
void roundExpired(RoundExpiry expire);
/**
* Gets chain height.
*
* @return the chain height
*/
long getChainHeight();
/**
* Gets parent block header.
*
* @return the parent block header
*/
QbftBlockHeader getParentBlockHeader();
/** /**
* Handle proposal payload. * Handle proposal payload.

View File

@@ -20,19 +20,19 @@ 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.Prepare;
import org.hyperledger.besu.consensus.qbft.core.messagewrappers.Proposal; 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.messagewrappers.RoundChange;
import org.hyperledger.besu.ethereum.core.BlockHeader; import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockHeader;
/** The type NoOp block height manager. */ /** The type NoOp block height manager. */
public class NoOpBlockHeightManager implements BaseQbftBlockHeightManager { public class NoOpBlockHeightManager implements BaseQbftBlockHeightManager {
private final BlockHeader parentHeader; private final QbftBlockHeader parentHeader;
/** /**
* Instantiates a new NoOp block height manager. * Instantiates a new NoOp block height manager.
* *
* @param parentHeader the parent header * @param parentHeader the parent header
*/ */
public NoOpBlockHeightManager(final BlockHeader parentHeader) { public NoOpBlockHeightManager(final QbftBlockHeader parentHeader) {
this.parentHeader = parentHeader; this.parentHeader = parentHeader;
} }
@@ -60,7 +60,7 @@ public class NoOpBlockHeightManager implements BaseQbftBlockHeightManager {
} }
@Override @Override
public BlockHeader getParentBlockHeader() { public QbftBlockHeader getParentBlockHeader() {
return parentHeader; return parentHeader;
} }
} }

View File

@@ -25,11 +25,11 @@ 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.network.QbftMessageTransmitter;
import org.hyperledger.besu.consensus.qbft.core.payload.MessageFactory; 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.QbftBlock;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockHeader;
import org.hyperledger.besu.consensus.qbft.core.types.QbftFinalState; 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.FutureRoundProposalMessageValidator;
import org.hyperledger.besu.consensus.qbft.core.validation.MessageValidatorFactory; import org.hyperledger.besu.consensus.qbft.core.validation.MessageValidatorFactory;
import org.hyperledger.besu.datatypes.Address; import org.hyperledger.besu.datatypes.Address;
import org.hyperledger.besu.ethereum.core.BlockHeader;
import org.hyperledger.besu.plugin.services.securitymodule.SecurityModuleException; import org.hyperledger.besu.plugin.services.securitymodule.SecurityModuleException;
import java.time.Clock; import java.time.Clock;
@@ -57,7 +57,7 @@ public class QbftBlockHeightManager implements BaseQbftBlockHeightManager {
private final QbftRoundFactory roundFactory; private final QbftRoundFactory roundFactory;
private final RoundChangeManager roundChangeManager; private final RoundChangeManager roundChangeManager;
private final BlockHeader parentHeader; private final QbftBlockHeader parentHeader;
private final QbftMessageTransmitter transmitter; private final QbftMessageTransmitter transmitter;
private final MessageFactory messageFactory; private final MessageFactory messageFactory;
private final Map<Integer, RoundState> futureRoundStateBuffer = Maps.newHashMap(); private final Map<Integer, RoundState> futureRoundStateBuffer = Maps.newHashMap();
@@ -82,7 +82,7 @@ public class QbftBlockHeightManager implements BaseQbftBlockHeightManager {
* @param messageFactory the message factory * @param messageFactory the message factory
*/ */
public QbftBlockHeightManager( public QbftBlockHeightManager(
final BlockHeader parentHeader, final QbftBlockHeader parentHeader,
final QbftFinalState finalState, final QbftFinalState finalState,
final RoundChangeManager roundChangeManager, final RoundChangeManager roundChangeManager,
final QbftRoundFactory qbftRoundFactory, final QbftRoundFactory qbftRoundFactory,
@@ -113,7 +113,7 @@ public class QbftBlockHeightManager implements BaseQbftBlockHeightManager {
final ConsensusRoundIdentifier roundIdentifier = final ConsensusRoundIdentifier roundIdentifier =
new ConsensusRoundIdentifier(nextBlockHeight, 0); new ConsensusRoundIdentifier(nextBlockHeight, 0);
finalState.getBlockTimer().startTimer(roundIdentifier, parentHeader); finalState.getBlockTimer().startTimer(roundIdentifier, parentHeader::getTimestamp);
} }
/** /**
@@ -130,7 +130,7 @@ public class QbftBlockHeightManager implements BaseQbftBlockHeightManager {
* @param isEarlyRoundChangeEnabled enable round change when f+1 RC messages are received * @param isEarlyRoundChangeEnabled enable round change when f+1 RC messages are received
*/ */
public QbftBlockHeightManager( public QbftBlockHeightManager(
final BlockHeader parentHeader, final QbftBlockHeader parentHeader,
final QbftFinalState finalState, final QbftFinalState finalState,
final RoundChangeManager roundChangeManager, final RoundChangeManager roundChangeManager,
final QbftRoundFactory qbftRoundFactory, final QbftRoundFactory qbftRoundFactory,
@@ -199,7 +199,9 @@ public class QbftBlockHeightManager implements BaseQbftBlockHeightManager {
// handle the block times period // handle the block times period
final long currentTimeInMillis = finalState.getClock().millis(); final long currentTimeInMillis = finalState.getClock().millis();
boolean emptyBlockExpired = boolean emptyBlockExpired =
finalState.getBlockTimer().checkEmptyBlockExpired(parentHeader, currentTimeInMillis); finalState
.getBlockTimer()
.checkEmptyBlockExpired(parentHeader::getTimestamp, currentTimeInMillis);
if (emptyBlockExpired) { if (emptyBlockExpired) {
LOG.trace( LOG.trace(
"Block has no transactions and this node is a proposer so it will send a proposal: " "Block has no transactions and this node is a proposer so it will send a proposal: "
@@ -211,7 +213,8 @@ public class QbftBlockHeightManager implements BaseQbftBlockHeightManager {
+ roundIdentifier); + roundIdentifier);
finalState finalState
.getBlockTimer() .getBlockTimer()
.resetTimerForEmptyBlock(roundIdentifier, parentHeader, currentTimeInMillis); .resetTimerForEmptyBlock(
roundIdentifier, parentHeader::getTimestamp, currentTimeInMillis);
finalState.getRoundTimer().cancelTimer(); finalState.getRoundTimer().cancelTimer();
currentRound = Optional.empty(); currentRound = Optional.empty();
} }
@@ -452,7 +455,7 @@ public class QbftBlockHeightManager implements BaseQbftBlockHeightManager {
} }
@Override @Override
public BlockHeader getParentBlockHeader() { public QbftBlockHeader getParentBlockHeader() {
return parentHeader; return parentHeader;
} }

View File

@@ -16,10 +16,10 @@ package org.hyperledger.besu.consensus.qbft.core.statemachine;
import org.hyperledger.besu.consensus.common.bft.BftHelpers; import org.hyperledger.besu.consensus.common.bft.BftHelpers;
import org.hyperledger.besu.consensus.qbft.core.payload.MessageFactory; import org.hyperledger.besu.consensus.qbft.core.payload.MessageFactory;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockHeader;
import org.hyperledger.besu.consensus.qbft.core.types.QbftFinalState; import org.hyperledger.besu.consensus.qbft.core.types.QbftFinalState;
import org.hyperledger.besu.consensus.qbft.core.types.QbftValidatorModeTransitionLogger;
import org.hyperledger.besu.consensus.qbft.core.validation.MessageValidatorFactory; 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;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
@@ -33,7 +33,7 @@ public class QbftBlockHeightManagerFactory {
private final QbftFinalState finalState; private final QbftFinalState finalState;
private final MessageValidatorFactory messageValidatorFactory; private final MessageValidatorFactory messageValidatorFactory;
private final MessageFactory messageFactory; private final MessageFactory messageFactory;
private final ValidatorModeTransitionLogger validatorModeTransitionLogger; private final QbftValidatorModeTransitionLogger validatorModeTransitionLogger;
private boolean isEarlyRoundChangeEnabled = false; private boolean isEarlyRoundChangeEnabled = false;
/** /**
@@ -50,7 +50,7 @@ public class QbftBlockHeightManagerFactory {
final QbftRoundFactory roundFactory, final QbftRoundFactory roundFactory,
final MessageValidatorFactory messageValidatorFactory, final MessageValidatorFactory messageValidatorFactory,
final MessageFactory messageFactory, final MessageFactory messageFactory,
final ValidatorModeTransitionLogger validatorModeTransitionLogger) { final QbftValidatorModeTransitionLogger validatorModeTransitionLogger) {
this.roundFactory = roundFactory; this.roundFactory = roundFactory;
this.finalState = finalState; this.finalState = finalState;
this.messageValidatorFactory = messageValidatorFactory; this.messageValidatorFactory = messageValidatorFactory;
@@ -64,7 +64,7 @@ public class QbftBlockHeightManagerFactory {
* @param parentHeader the parent header * @param parentHeader the parent header
* @return the base qbft block height manager * @return the base qbft block height manager
*/ */
public BaseQbftBlockHeightManager create(final BlockHeader parentHeader) { public BaseQbftBlockHeightManager create(final QbftBlockHeader parentHeader) {
validatorModeTransitionLogger.logTransitionChange(parentHeader); validatorModeTransitionLogger.logTransitionChange(parentHeader);
if (finalState.isLocalNodeValidator()) { if (finalState.isLocalNodeValidator()) {
@@ -85,11 +85,13 @@ public class QbftBlockHeightManagerFactory {
this.isEarlyRoundChangeEnabled = isEarlyRoundChangeEnabled; this.isEarlyRoundChangeEnabled = isEarlyRoundChangeEnabled;
} }
private BaseQbftBlockHeightManager createNoOpBlockHeightManager(final BlockHeader parentHeader) { private BaseQbftBlockHeightManager createNoOpBlockHeightManager(
final QbftBlockHeader parentHeader) {
return new NoOpBlockHeightManager(parentHeader); return new NoOpBlockHeightManager(parentHeader);
} }
private BaseQbftBlockHeightManager createFullBlockHeightManager(final BlockHeader parentHeader) { private BaseQbftBlockHeightManager createFullBlockHeightManager(
final QbftBlockHeader parentHeader) {
QbftBlockHeightManager qbftBlockHeightManager; QbftBlockHeightManager qbftBlockHeightManager;
RoundChangeManager roundChangeManager; RoundChangeManager roundChangeManager;

View File

@@ -20,11 +20,9 @@ import org.hyperledger.besu.consensus.common.bft.MessageTracker;
import org.hyperledger.besu.consensus.common.bft.SynchronizerUpdater; import org.hyperledger.besu.consensus.common.bft.SynchronizerUpdater;
import org.hyperledger.besu.consensus.common.bft.events.BftReceivedMessageEvent; 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.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.events.RoundExpiry;
import org.hyperledger.besu.consensus.common.bft.messagewrappers.BftMessage; 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.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.common.bft.statemachine.FutureMessageBuffer;
import org.hyperledger.besu.consensus.qbft.core.messagedata.CommitMessageData; 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.PrepareMessageData;
@@ -32,9 +30,11 @@ 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.QbftV1;
import org.hyperledger.besu.consensus.qbft.core.messagedata.RoundChangeMessageData; 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.QbftBlockCodec;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockHeader;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockchain;
import org.hyperledger.besu.consensus.qbft.core.types.QbftEventHandler;
import org.hyperledger.besu.consensus.qbft.core.types.QbftFinalState; import org.hyperledger.besu.consensus.qbft.core.types.QbftFinalState;
import org.hyperledger.besu.ethereum.chain.Blockchain; import org.hyperledger.besu.consensus.qbft.core.types.QbftNewChainHead;
import org.hyperledger.besu.ethereum.core.BlockHeader;
import org.hyperledger.besu.ethereum.p2p.rlpx.wire.Message; import org.hyperledger.besu.ethereum.p2p.rlpx.wire.Message;
import org.hyperledger.besu.ethereum.p2p.rlpx.wire.MessageData; import org.hyperledger.besu.ethereum.p2p.rlpx.wire.MessageData;
@@ -45,10 +45,10 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
/** The Qbft controller. */ /** The Qbft controller. */
public class QbftController implements BftEventHandler { public class QbftController implements QbftEventHandler {
private static final Logger LOG = LoggerFactory.getLogger(QbftController.class); private static final Logger LOG = LoggerFactory.getLogger(QbftController.class);
private final Blockchain blockchain; private final QbftBlockchain blockchain;
private final QbftFinalState finalState; private final QbftFinalState finalState;
private final FutureMessageBuffer futureMessageBuffer; private final FutureMessageBuffer futureMessageBuffer;
private final Gossiper gossiper; private final Gossiper gossiper;
@@ -72,7 +72,7 @@ public class QbftController implements BftEventHandler {
* @param blockEncoder the block encoder * @param blockEncoder the block encoder
*/ */
public QbftController( public QbftController(
final Blockchain blockchain, final QbftBlockchain blockchain,
final QbftFinalState finalState, final QbftFinalState finalState,
final QbftBlockHeightManagerFactory qbftBlockHeightManagerFactory, final QbftBlockHeightManagerFactory qbftBlockHeightManagerFactory,
final Gossiper gossiper, final Gossiper gossiper,
@@ -131,7 +131,7 @@ public class QbftController implements BftEventHandler {
} }
} }
private void createNewHeightManager(final BlockHeader parentHeader) { private void createNewHeightManager(final QbftBlockHeader parentHeader) {
currentHeightManager = qbftBlockHeightManagerFactory.create(parentHeader); currentHeightManager = qbftBlockHeightManagerFactory.create(parentHeader);
} }
@@ -187,9 +187,9 @@ public class QbftController implements BftEventHandler {
} }
@Override @Override
public void handleNewBlockEvent(final NewChainHead newChainHead) { public void handleNewBlockEvent(final QbftNewChainHead newChainHead) {
final BlockHeader newBlockHeader = newChainHead.getNewChainHeadHeader(); final QbftBlockHeader newBlockHeader = newChainHead.newChainHeadHeader();
final BlockHeader currentMiningParent = getCurrentHeightManager().getParentBlockHeader(); final QbftBlockHeader currentMiningParent = getCurrentHeightManager().getParentBlockHeader();
LOG.debug( LOG.debug(
"New chain head detected (block number={})," + " currently mining on top of {}.", "New chain head detected (block number={})," + " currently mining on top of {}.",
newBlockHeader.getNumber(), newBlockHeader.getNumber(),
@@ -251,7 +251,7 @@ public class QbftController implements BftEventHandler {
} }
} }
private void startNewHeightManager(final BlockHeader parentHeader) { private void startNewHeightManager(final QbftBlockHeader parentHeader) {
createNewHeightManager(parentHeader); createNewHeightManager(parentHeader);
final long newChainHeight = getCurrentHeightManager().getChainHeight(); final long newChainHeight = getCurrentHeightManager().getChainHeight();
futureMessageBuffer.retrieveMessagesForHeight(newChainHeight).forEach(this::handleMessage); futureMessageBuffer.retrieveMessagesForHeight(newChainHeight).forEach(this::handleMessage);

View File

@@ -16,7 +16,6 @@ package org.hyperledger.besu.consensus.qbft.core.statemachine;
import static java.util.Collections.emptyList; import static java.util.Collections.emptyList;
import org.hyperledger.besu.consensus.common.bft.BftBlockHashing;
import org.hyperledger.besu.consensus.common.bft.BftExtraData; import org.hyperledger.besu.consensus.common.bft.BftExtraData;
import org.hyperledger.besu.consensus.common.bft.BftExtraDataCodec; import org.hyperledger.besu.consensus.common.bft.BftExtraDataCodec;
import org.hyperledger.besu.consensus.common.bft.ConsensusRoundIdentifier; import org.hyperledger.besu.consensus.common.bft.ConsensusRoundIdentifier;
@@ -31,6 +30,8 @@ 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.payload.RoundChangePayload;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlock; 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.QbftBlockCreator;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockHashing;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockHeader;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockImporter; 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.QbftBlockInterface;
import org.hyperledger.besu.consensus.qbft.core.types.QbftContext; import org.hyperledger.besu.consensus.qbft.core.types.QbftContext;
@@ -42,7 +43,6 @@ import org.hyperledger.besu.crypto.SECPSignature;
import org.hyperledger.besu.cryptoservices.NodeKey; import org.hyperledger.besu.cryptoservices.NodeKey;
import org.hyperledger.besu.datatypes.Hash; import org.hyperledger.besu.datatypes.Hash;
import org.hyperledger.besu.ethereum.ProtocolContext; import org.hyperledger.besu.ethereum.ProtocolContext;
import org.hyperledger.besu.ethereum.core.BlockHeader;
import org.hyperledger.besu.plugin.services.securitymodule.SecurityModuleException; import org.hyperledger.besu.plugin.services.securitymodule.SecurityModuleException;
import org.hyperledger.besu.util.Subscribers; import org.hyperledger.besu.util.Subscribers;
@@ -81,7 +81,8 @@ public class QbftRound {
/** The Bft extra data provider */ /** The Bft extra data provider */
protected final QbftExtraDataProvider qbftExtraDataProvider; protected final QbftExtraDataProvider qbftExtraDataProvider;
private final BlockHeader parentHeader; private final QbftBlockHashing blockHashing;
private final QbftBlockHeader parentHeader;
/** /**
* Instantiates a new Qbft round. * Instantiates a new Qbft round.
@@ -97,6 +98,7 @@ public class QbftRound {
* @param roundTimer the round timer * @param roundTimer the round timer
* @param bftExtraDataCodec the bft extra data codec * @param bftExtraDataCodec the bft extra data codec
* @param qbftExtraDataProvider the qbft extra data provider * @param qbftExtraDataProvider the qbft extra data provider
* @param blockHashing the block hashing
* @param parentHeader the parent header * @param parentHeader the parent header
*/ */
public QbftRound( public QbftRound(
@@ -111,7 +113,8 @@ public class QbftRound {
final RoundTimer roundTimer, final RoundTimer roundTimer,
final BftExtraDataCodec bftExtraDataCodec, final BftExtraDataCodec bftExtraDataCodec,
final QbftExtraDataProvider qbftExtraDataProvider, final QbftExtraDataProvider qbftExtraDataProvider,
final BlockHeader parentHeader) { final QbftBlockHashing blockHashing,
final QbftBlockHeader parentHeader) {
this.roundState = roundState; this.roundState = roundState;
this.blockCreator = blockCreator; this.blockCreator = blockCreator;
this.protocolContext = protocolContext; this.protocolContext = protocolContext;
@@ -122,6 +125,7 @@ public class QbftRound {
this.transmitter = transmitter; this.transmitter = transmitter;
this.bftExtraDataCodec = bftExtraDataCodec; this.bftExtraDataCodec = bftExtraDataCodec;
this.qbftExtraDataProvider = qbftExtraDataProvider; this.qbftExtraDataProvider = qbftExtraDataProvider;
this.blockHashing = blockHashing;
this.parentHeader = parentHeader; this.parentHeader = parentHeader;
roundTimer.startTimer(getRoundIdentifier()); roundTimer.startTimer(getRoundIdentifier());
} }
@@ -166,7 +170,7 @@ public class QbftRound {
"Sending proposal from PreparedCertificate. round={}", roundState.getRoundIdentifier()); "Sending proposal from PreparedCertificate. round={}", roundState.getRoundIdentifier());
QbftBlock preparedBlock = bestPreparedCertificate.get().getBlock(); QbftBlock preparedBlock = bestPreparedCertificate.get().getBlock();
final QbftBlockInterface bftBlockInterface = final QbftBlockInterface bftBlockInterface =
protocolContext.getConsensusContext(QbftContext.class).getBlockInterface(); protocolContext.getConsensusContext(QbftContext.class).blockInterface();
blockToPublish = blockToPublish =
bftBlockInterface.replaceRoundInBlock( bftBlockInterface.replaceRoundInBlock(
preparedBlock, preparedBlock,
@@ -391,17 +395,16 @@ public class QbftRound {
private SECPSignature createCommitSeal(final QbftBlock block) { private SECPSignature createCommitSeal(final QbftBlock block) {
final QbftBlock commitBlock = createCommitBlock(block); final QbftBlock commitBlock = createCommitBlock(block);
final BlockHeader proposedHeader = commitBlock.getHeader(); final QbftBlockHeader proposedHeader = commitBlock.getHeader();
final BftExtraData extraData = qbftExtraDataProvider.getExtraData(proposedHeader); final BftExtraData extraData = qbftExtraDataProvider.getExtraData(proposedHeader);
final Hash commitHash = final Hash commitHash =
new BftBlockHashing(bftExtraDataCodec) blockHashing.calculateDataHashForCommittedSeal(proposedHeader, extraData);
.calculateDataHashForCommittedSeal(proposedHeader, extraData);
return nodeKey.sign(commitHash); return nodeKey.sign(commitHash);
} }
private QbftBlock createCommitBlock(final QbftBlock block) { private QbftBlock createCommitBlock(final QbftBlock block) {
final QbftBlockInterface bftBlockInterface = final QbftBlockInterface bftBlockInterface =
protocolContext.getConsensusContext(QbftContext.class).getBlockInterface(); protocolContext.getConsensusContext(QbftContext.class).blockInterface();
return bftBlockInterface.replaceRoundInBlock( return bftBlockInterface.replaceRoundInBlock(
block, getRoundIdentifier().getRoundNumber(), QbftHashMode.COMMITTED_SEAL); block, getRoundIdentifier().getRoundNumber(), QbftHashMode.COMMITTED_SEAL);
} }

View File

@@ -20,13 +20,14 @@ 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.MessageFactory;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockCreator; 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.QbftBlockCreatorFactory;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockHashing;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockHeader;
import org.hyperledger.besu.consensus.qbft.core.types.QbftExtraDataProvider; 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.QbftFinalState;
import org.hyperledger.besu.consensus.qbft.core.types.QbftMinedBlockObserver; 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.QbftProtocolSchedule;
import org.hyperledger.besu.consensus.qbft.core.validation.MessageValidatorFactory; import org.hyperledger.besu.consensus.qbft.core.validation.MessageValidatorFactory;
import org.hyperledger.besu.ethereum.ProtocolContext; import org.hyperledger.besu.ethereum.ProtocolContext;
import org.hyperledger.besu.ethereum.core.BlockHeader;
import org.hyperledger.besu.util.Subscribers; import org.hyperledger.besu.util.Subscribers;
/** The Qbft round factory. */ /** The Qbft round factory. */
@@ -41,6 +42,7 @@ public class QbftRoundFactory {
private final MessageFactory messageFactory; private final MessageFactory messageFactory;
private final BftExtraDataCodec bftExtraDataCodec; private final BftExtraDataCodec bftExtraDataCodec;
private final QbftExtraDataProvider qbftExtraDataProvider; private final QbftExtraDataProvider qbftExtraDataProvider;
private final QbftBlockHashing blockHashing;
/** /**
* Instantiates a new Qbft round factory. * Instantiates a new Qbft round factory.
@@ -53,6 +55,7 @@ public class QbftRoundFactory {
* @param messageFactory the message factory * @param messageFactory the message factory
* @param bftExtraDataCodec the bft extra data codec * @param bftExtraDataCodec the bft extra data codec
* @param qbftExtraDataProvider the bft extra data codec * @param qbftExtraDataProvider the bft extra data codec
* @param blockHashing the block hashing
*/ */
public QbftRoundFactory( public QbftRoundFactory(
final QbftFinalState finalState, final QbftFinalState finalState,
@@ -62,7 +65,8 @@ public class QbftRoundFactory {
final MessageValidatorFactory messageValidatorFactory, final MessageValidatorFactory messageValidatorFactory,
final MessageFactory messageFactory, final MessageFactory messageFactory,
final BftExtraDataCodec bftExtraDataCodec, final BftExtraDataCodec bftExtraDataCodec,
final QbftExtraDataProvider qbftExtraDataProvider) { final QbftExtraDataProvider qbftExtraDataProvider,
final QbftBlockHashing blockHashing) {
this.finalState = finalState; this.finalState = finalState;
this.blockCreatorFactory = finalState.getBlockCreatorFactory(); this.blockCreatorFactory = finalState.getBlockCreatorFactory();
this.protocolContext = protocolContext; this.protocolContext = protocolContext;
@@ -72,6 +76,7 @@ public class QbftRoundFactory {
this.messageFactory = messageFactory; this.messageFactory = messageFactory;
this.bftExtraDataCodec = bftExtraDataCodec; this.bftExtraDataCodec = bftExtraDataCodec;
this.qbftExtraDataProvider = qbftExtraDataProvider; this.qbftExtraDataProvider = qbftExtraDataProvider;
this.blockHashing = blockHashing;
} }
/** /**
@@ -81,7 +86,7 @@ public class QbftRoundFactory {
* @param round the round * @param round the round
* @return the qbft round * @return the qbft round
*/ */
public QbftRound createNewRound(final BlockHeader parentHeader, final int round) { public QbftRound createNewRound(final QbftBlockHeader parentHeader, final int round) {
long nextBlockHeight = parentHeader.getNumber() + 1; long nextBlockHeight = parentHeader.getNumber() + 1;
final ConsensusRoundIdentifier roundIdentifier = final ConsensusRoundIdentifier roundIdentifier =
new ConsensusRoundIdentifier(nextBlockHeight, round); new ConsensusRoundIdentifier(nextBlockHeight, round);
@@ -103,7 +108,7 @@ public class QbftRoundFactory {
* @return the qbft round * @return the qbft round
*/ */
public QbftRound createNewRoundWithState( public QbftRound createNewRoundWithState(
final BlockHeader parentHeader, final RoundState roundState) { final QbftBlockHeader parentHeader, final RoundState roundState) {
final QbftBlockCreator blockCreator = final QbftBlockCreator blockCreator =
blockCreatorFactory.create(roundState.getRoundIdentifier().getRoundNumber()); blockCreatorFactory.create(roundState.getRoundIdentifier().getRoundNumber());
@@ -123,6 +128,7 @@ public class QbftRoundFactory {
finalState.getRoundTimer(), finalState.getRoundTimer(),
bftExtraDataCodec, bftExtraDataCodec,
qbftExtraDataProvider, qbftExtraDataProvider,
blockHashing,
parentHeader); parentHeader);
} }
} }

View File

@@ -15,7 +15,6 @@
package org.hyperledger.besu.consensus.qbft.core.types; package org.hyperledger.besu.consensus.qbft.core.types;
import org.hyperledger.besu.datatypes.Hash; import org.hyperledger.besu.datatypes.Hash;
import org.hyperledger.besu.ethereum.core.BlockHeader;
/** Represents a block in the context of the QBFT consensus mechanism. */ /** Represents a block in the context of the QBFT consensus mechanism. */
public interface QbftBlock { public interface QbftBlock {
@@ -25,7 +24,7 @@ public interface QbftBlock {
* *
* @return the block header. * @return the block header.
*/ */
BlockHeader getHeader(); QbftBlockHeader getHeader();
/** /**
* Whether the block is considered empty, generally this means that the block has no transactions. * Whether the block is considered empty, generally this means that the block has no transactions.

View File

@@ -15,7 +15,6 @@
package org.hyperledger.besu.consensus.qbft.core.types; package org.hyperledger.besu.consensus.qbft.core.types;
import org.hyperledger.besu.crypto.SECPSignature; import org.hyperledger.besu.crypto.SECPSignature;
import org.hyperledger.besu.ethereum.core.BlockHeader;
import java.util.Collection; import java.util.Collection;
@@ -29,7 +28,7 @@ public interface QbftBlockCreator {
* @param parentHeader the parent header * @param parentHeader the parent header
* @return the block * @return the block
*/ */
QbftBlock createBlock(long headerTimeStampSeconds, BlockHeader parentHeader); QbftBlock createBlock(long headerTimeStampSeconds, QbftBlockHeader parentHeader);
/** /**
* Create sealed block. * Create sealed block.

View File

@@ -0,0 +1,34 @@
/*
* 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.datatypes.Hash;
/** Provides hashing for QBFT block headers. */
public interface QbftBlockHashing {
/**
* Constructs a hash of the block header suitable for signing as a committed seal. The extra data
* in the hash uses an empty list for the committed seals.
*
* @param header The header for which a proposer seal is to be calculated (with or without extra
* data)
* @param extraData The extra data block which is to be inserted to the header once seal is
* calculated
* @return the hash of the header including the validator and proposer seal in the extra data
*/
Hash calculateDataHashForCommittedSeal(QbftBlockHeader header, BftExtraData extraData);
}

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.core.types;
import org.hyperledger.besu.datatypes.Address;
import org.hyperledger.besu.datatypes.Hash;
/** Represents a block header in the context of the QBFT consensus mechanism. */
public interface QbftBlockHeader {
/**
* Returns the block number of the block.
*
* @return the block number.
*/
long getNumber();
/**
* Returns the timestamp of the block.
*
* @return the timestamp.
*/
long getTimestamp();
/**
* Returns the coinbase of the block.
*
* @return the coinbase.
*/
Address getCoinbase();
/**
* Returns the hash of the block.
*
* @return the hash.
*/
Hash getHash();
}

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;
/** Minimal QBFT interface for reading data from the blockchain. */
public interface QbftBlockchain {
/**
* Returns the chain head header.
*
* @return the header.
*/
QbftBlockHeader getChainHeadHeader();
/**
* Returns the chain head block number.
*
* @return the block number.
*/
long getChainHeadBlockNumber();
}

View File

@@ -14,33 +14,25 @@
*/ */
package org.hyperledger.besu.consensus.qbft.core.types; package org.hyperledger.besu.consensus.qbft.core.types;
import org.hyperledger.besu.consensus.common.validator.ValidatorProvider;
import org.hyperledger.besu.ethereum.ConsensusContext; import org.hyperledger.besu.ethereum.ConsensusContext;
/** Holds the QBFT specific mutable state. */ /**
public class QbftContext implements ConsensusContext { * Holds the QBFT specific mutable state.
*
private final ValidatorProvider validatorProvider; * @param validatorProvider the validator provider
private final QbftBlockInterface blockInterface; * @param blockInterface the block interface
*/
/** public record QbftContext(
* Instantiates a new Bft context. QbftValidatorProvider validatorProvider, QbftBlockInterface blockInterface)
* implements ConsensusContext {
* @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. * Gets validator provider.
* *
* @return the validator provider * @return the validator provider
*/ */
public ValidatorProvider getValidatorProvider() { @Override
public QbftValidatorProvider validatorProvider() {
return validatorProvider; return validatorProvider;
} }
@@ -49,7 +41,8 @@ public class QbftContext implements ConsensusContext {
* *
* @return the block interface * @return the block interface
*/ */
public QbftBlockInterface getBlockInterface() { @Override
public QbftBlockInterface blockInterface() {
return blockInterface; return blockInterface;
} }

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.core.types;
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.RoundExpiry;
/** The interface QBFT event handler. */
public interface QbftEventHandler {
/** Start. */
void start();
/**
* Handle errorMessage event.
*
* @param msg the msg
*/
void handleMessageEvent(BftReceivedMessageEvent msg);
/**
* Handle new block event.
*
* @param newChainHead the new chain head
*/
void handleNewBlockEvent(QbftNewChainHead newChainHead);
/**
* Handle block timer expiry.
*
* @param blockTimerExpiry the block timer expiry
*/
void handleBlockTimerExpiry(BlockTimerExpiry blockTimerExpiry);
/**
* Handle round expiry.
*
* @param roundExpiry the round expiry
*/
void handleRoundExpiry(RoundExpiry roundExpiry);
}

View File

@@ -15,7 +15,6 @@
package org.hyperledger.besu.consensus.qbft.core.types; package org.hyperledger.besu.consensus.qbft.core.types;
import org.hyperledger.besu.consensus.common.bft.BftExtraData; import org.hyperledger.besu.consensus.common.bft.BftExtraData;
import org.hyperledger.besu.ethereum.core.BlockHeader;
/** Provides the extra data for a block. */ /** Provides the extra data for a block. */
public interface QbftExtraDataProvider { public interface QbftExtraDataProvider {
@@ -26,5 +25,5 @@ public interface QbftExtraDataProvider {
* @param header the block header to retrieve the extra data from * @param header the block header to retrieve the extra data from
* @return the extra data * @return the extra data
*/ */
BftExtraData getExtraData(BlockHeader header); BftExtraData getExtraData(QbftBlockHeader header);
} }

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.core.types;
import org.hyperledger.besu.consensus.common.bft.events.BftEvent;
import org.hyperledger.besu.consensus.common.bft.events.BftEvents;
/**
* Event indicating that new chain head has been received
*
* @param newChainHeadHeader the new chain head header
*/
public record QbftNewChainHead(QbftBlockHeader newChainHeadHeader) implements BftEvent {
@Override
public BftEvents.Type getType() {
return BftEvents.Type.NEW_CHAIN_HEAD;
}
/**
* Gets new chain head header.
*
* @return the new chain head header
*/
@Override
public QbftBlockHeader newChainHeadHeader() {
return newChainHeadHeader;
}
}

View File

@@ -14,8 +14,6 @@
*/ */
package org.hyperledger.besu.consensus.qbft.core.types; 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 * Provides the ability to select the appropriate QbftProtocolSpec containing the validation and
* import for the supplied block header. * import for the supplied block header.
@@ -28,5 +26,5 @@ public interface QbftProtocolSchedule {
* @param header The block header to select the appropriate QbftProtocolSpec for * @param header The block header to select the appropriate QbftProtocolSpec for
* @return The QbftProtocolSpec for the supplied block header * @return The QbftProtocolSpec for the supplied block header
*/ */
QbftProtocolSpec getByBlockHeader(BlockHeader header); QbftProtocolSpec getByBlockHeader(QbftBlockHeader header);
} }

View File

@@ -0,0 +1,26 @@
/*
* 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 logging for QBFT transition changes */
public interface QbftValidatorModeTransitionLogger {
/**
* Logs the transition change
*
* @param parentHeader the parent header
*/
void logTransitionChange(QbftBlockHeader parentHeader);
}

View File

@@ -0,0 +1,82 @@
/*
* 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.VoteProvider;
import org.hyperledger.besu.cryptoservices.NodeKey;
import org.hyperledger.besu.datatypes.Address;
import org.hyperledger.besu.ethereum.core.Util;
import java.util.Collection;
import java.util.Optional;
/** The interface Validator provider. */
public interface QbftValidatorProvider {
/**
* Gets validators at head.
*
* @return the validators at head
*/
Collection<Address> getValidatorsAtHead();
/**
* Gets validators after block.
*
* @param header the header
* @return the validators after block
*/
Collection<Address> getValidatorsAfterBlock(QbftBlockHeader header);
/**
* Gets validators for block.
*
* @param header the header
* @return the validators for block
*/
Collection<Address> getValidatorsForBlock(QbftBlockHeader header);
/**
* Gets vote provider at head.
*
* @return the vote provider at head
*/
Optional<VoteProvider> getVoteProviderAtHead();
/**
* Gets vote provider after block.
*
* @param header the header
* @return the vote provider after block
*/
/*
* ForkingValidatorProvider has a specific implementation but we don't want the client code to
* know it's using a ForkingValidatorProvider. ForkingValidatorProvider's voteProvider can be
* different per block. Other ValidatorProviders yield the same voteProvider at every block.
*/
default Optional<VoteProvider> getVoteProviderAfterBlock(final QbftBlockHeader header) {
return getVoteProviderAtHead();
}
/**
* Determines if this node is a validator
*
* @param nodekey our node key
* @return true if this node is a validator
*/
default boolean nodeIsValidator(final NodeKey nodekey) {
return this.getValidatorsAtHead().contains(Util.publicKeyToAddress(nodekey.getPublicKey()));
}
}

View File

@@ -16,14 +16,14 @@ package org.hyperledger.besu.consensus.qbft.core.validation;
import org.hyperledger.besu.consensus.common.bft.ConsensusRoundIdentifier; import org.hyperledger.besu.consensus.common.bft.ConsensusRoundIdentifier;
import org.hyperledger.besu.consensus.qbft.core.messagewrappers.Proposal; import org.hyperledger.besu.consensus.qbft.core.messagewrappers.Proposal;
import org.hyperledger.besu.ethereum.core.BlockHeader; import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockHeader;
/** The Future round proposal message validator. */ /** The Future round proposal message validator. */
public class FutureRoundProposalMessageValidator { public class FutureRoundProposalMessageValidator {
private final MessageValidatorFactory messageValidatorFactory; private final MessageValidatorFactory messageValidatorFactory;
private final long chainHeight; private final long chainHeight;
private final BlockHeader parentHeader; private final QbftBlockHeader parentHeader;
/** /**
* Instantiates a new Future round proposal message validator. * Instantiates a new Future round proposal message validator.
@@ -35,7 +35,7 @@ public class FutureRoundProposalMessageValidator {
public FutureRoundProposalMessageValidator( public FutureRoundProposalMessageValidator(
final MessageValidatorFactory messageValidatorFactory, final MessageValidatorFactory messageValidatorFactory,
final long chainHeight, final long chainHeight,
final BlockHeader parentHeader) { final QbftBlockHeader parentHeader) {
this.messageValidatorFactory = messageValidatorFactory; this.messageValidatorFactory = messageValidatorFactory;
this.chainHeight = chainHeight; this.chainHeight = chainHeight;
this.parentHeader = parentHeader; this.parentHeader = parentHeader;

View File

@@ -17,13 +17,13 @@ package org.hyperledger.besu.consensus.qbft.core.validation;
import org.hyperledger.besu.consensus.common.bft.BftHelpers; import org.hyperledger.besu.consensus.common.bft.BftHelpers;
import org.hyperledger.besu.consensus.common.bft.ConsensusRoundIdentifier; import org.hyperledger.besu.consensus.common.bft.ConsensusRoundIdentifier;
import org.hyperledger.besu.consensus.common.bft.blockcreation.ProposerSelector; import org.hyperledger.besu.consensus.common.bft.blockcreation.ProposerSelector;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockHeader;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockInterface; 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.QbftContext;
import org.hyperledger.besu.consensus.qbft.core.types.QbftProtocolSchedule; import org.hyperledger.besu.consensus.qbft.core.types.QbftProtocolSchedule;
import org.hyperledger.besu.consensus.qbft.core.validation.MessageValidator.SubsequentMessageValidator; import org.hyperledger.besu.consensus.qbft.core.validation.MessageValidator.SubsequentMessageValidator;
import org.hyperledger.besu.datatypes.Address; import org.hyperledger.besu.datatypes.Address;
import org.hyperledger.besu.ethereum.ProtocolContext; import org.hyperledger.besu.ethereum.ProtocolContext;
import org.hyperledger.besu.ethereum.core.BlockHeader;
import java.util.Collection; import java.util.Collection;
@@ -58,10 +58,10 @@ public class MessageValidatorFactory {
* @return the list of validators * @return the list of validators
*/ */
public static Collection<Address> getValidatorsAfterBlock( public static Collection<Address> getValidatorsAfterBlock(
final ProtocolContext protocolContext, final BlockHeader parentHeader) { final ProtocolContext protocolContext, final QbftBlockHeader parentHeader) {
return protocolContext return protocolContext
.getConsensusContext(QbftContext.class) .getConsensusContext(QbftContext.class)
.getValidatorProvider() .validatorProvider()
.getValidatorsAfterBlock(parentHeader); .getValidatorsAfterBlock(parentHeader);
} }
@@ -73,10 +73,10 @@ public class MessageValidatorFactory {
* @return the list of validators * @return the list of validators
*/ */
public static Collection<Address> getValidatorsForBlock( public static Collection<Address> getValidatorsForBlock(
final ProtocolContext protocolContext, final BlockHeader parentHeader) { final ProtocolContext protocolContext, final QbftBlockHeader parentHeader) {
return protocolContext return protocolContext
.getConsensusContext(QbftContext.class) .getConsensusContext(QbftContext.class)
.getValidatorProvider() .validatorProvider()
.getValidatorsForBlock(parentHeader); .getValidatorsForBlock(parentHeader);
} }
@@ -88,7 +88,7 @@ public class MessageValidatorFactory {
* @return the round change message validator * @return the round change message validator
*/ */
public RoundChangeMessageValidator createRoundChangeMessageValidator( public RoundChangeMessageValidator createRoundChangeMessageValidator(
final long chainHeight, final BlockHeader parentHeader) { final long chainHeight, final QbftBlockHeader parentHeader) {
final Collection<Address> validatorsForHeight = final Collection<Address> validatorsForHeight =
getValidatorsAfterBlock(protocolContext, parentHeader); getValidatorsAfterBlock(protocolContext, parentHeader);
@@ -112,7 +112,7 @@ public class MessageValidatorFactory {
* @return the message validator * @return the message validator
*/ */
public MessageValidator createMessageValidator( public MessageValidator createMessageValidator(
final ConsensusRoundIdentifier roundIdentifier, final BlockHeader parentHeader) { final ConsensusRoundIdentifier roundIdentifier, final QbftBlockHeader parentHeader) {
final Collection<Address> validatorsForHeight = final Collection<Address> validatorsForHeight =
getValidatorsAfterBlock(protocolContext, parentHeader); getValidatorsAfterBlock(protocolContext, parentHeader);
@@ -126,7 +126,7 @@ public class MessageValidatorFactory {
proposerSelector.selectProposerForRound(roundIdentifier)); proposerSelector.selectProposerForRound(roundIdentifier));
final QbftBlockInterface blockInterface = final QbftBlockInterface blockInterface =
protocolContext.getConsensusContext(QbftContext.class).getBlockInterface(); protocolContext.getConsensusContext(QbftContext.class).blockInterface();
return new MessageValidator( return new MessageValidator(
block -> block ->
new SubsequentMessageValidator( new SubsequentMessageValidator(
@@ -142,7 +142,7 @@ public class MessageValidatorFactory {
* @return the future round proposal message validator * @return the future round proposal message validator
*/ */
public FutureRoundProposalMessageValidator createFutureRoundProposalMessageValidator( public FutureRoundProposalMessageValidator createFutureRoundProposalMessageValidator(
final long chainHeight, final BlockHeader parentHeader) { final long chainHeight, final QbftBlockHeader parentHeader) {
return new FutureRoundProposalMessageValidator(this, chainHeight, parentHeader); return new FutureRoundProposalMessageValidator(this, chainHeight, parentHeader);
} }
} }

View File

@@ -144,7 +144,7 @@ public class ProposalValidator {
// Need to check that if we substitute the LatestPrepareCert round number into the supplied // Need to check that if we substitute the LatestPrepareCert round number into the supplied
// block that we get the SAME hash as PreparedCert. // block that we get the SAME hash as PreparedCert.
final QbftBlockInterface bftBlockInterface = final QbftBlockInterface bftBlockInterface =
protocolContext.getConsensusContext(QbftContext.class).getBlockInterface(); protocolContext.getConsensusContext(QbftContext.class).blockInterface();
final QbftBlock currentBlockWithOldRound = final QbftBlock currentBlockWithOldRound =
bftBlockInterface.replaceRoundInBlock( bftBlockInterface.replaceRoundInBlock(
proposal.getBlock(), metadata.getPreparedRound(), QbftHashMode.COMMITTED_SEAL); proposal.getBlock(), metadata.getPreparedRound(), QbftHashMode.COMMITTED_SEAL);

View File

@@ -15,15 +15,15 @@
package org.hyperledger.besu.consensus.qbft.core; package org.hyperledger.besu.consensus.qbft.core;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlock; import org.hyperledger.besu.consensus.qbft.core.types.QbftBlock;
import org.hyperledger.besu.ethereum.core.BlockDataGenerator; import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockHeader;
import org.hyperledger.besu.ethereum.core.BlockHeader; import org.hyperledger.besu.consensus.qbft.core.validation.QbftBlockHeaderTestFixture;
public class QbftBlockTestFixture { public class QbftBlockTestFixture {
private BlockHeader blockHeader = new BlockDataGenerator().header(); private QbftBlockHeader blockHeader = new QbftBlockHeaderTestFixture().buildHeader();
private boolean isEmpty = true; private boolean isEmpty = true;
public QbftBlockTestFixture blockHeader(final BlockHeader blockHeader) { public QbftBlockTestFixture blockHeader(final QbftBlockHeader blockHeader) {
this.blockHeader = blockHeader; this.blockHeader = blockHeader;
return this; return this;
} }
@@ -40,7 +40,7 @@ public class QbftBlockTestFixture {
class TestQbftBlock implements QbftBlock { class TestQbftBlock implements QbftBlock {
@Override @Override
public BlockHeader getHeader() { public QbftBlockHeader getHeader() {
return blockHeader; return blockHeader;
} }

View File

@@ -19,9 +19,9 @@ import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when; import static org.mockito.Mockito.when;
import static org.mockito.Mockito.withSettings; 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.QbftBlockInterface;
import org.hyperledger.besu.consensus.qbft.core.types.QbftContext; import org.hyperledger.besu.consensus.qbft.core.types.QbftContext;
import org.hyperledger.besu.consensus.qbft.core.types.QbftValidatorProvider;
import org.hyperledger.besu.datatypes.Address; import org.hyperledger.besu.datatypes.Address;
import java.util.Collection; import java.util.Collection;
@@ -35,11 +35,11 @@ public class QbftContextBuilder {
final Collection<Address> validators, final Collection<Address> validators,
final QbftBlockInterface bftBlockInterface) { final QbftBlockInterface bftBlockInterface) {
final T bftContext = mock(contextClazz, withSettings().strictness(Strictness.LENIENT)); final T bftContext = mock(contextClazz, withSettings().strictness(Strictness.LENIENT));
final ValidatorProvider mockValidatorProvider = final QbftValidatorProvider mockValidatorProvider =
mock(ValidatorProvider.class, withSettings().strictness(Strictness.LENIENT)); mock(QbftValidatorProvider.class, withSettings().strictness(Strictness.LENIENT));
when(bftContext.getValidatorProvider()).thenReturn(mockValidatorProvider); when(bftContext.validatorProvider()).thenReturn(mockValidatorProvider);
when(mockValidatorProvider.getValidatorsAfterBlock(any())).thenReturn(validators); when(mockValidatorProvider.getValidatorsAfterBlock(any())).thenReturn(validators);
when(bftContext.getBlockInterface()).thenReturn(bftBlockInterface); when(bftContext.blockInterface()).thenReturn(bftBlockInterface);
when(bftContext.as(any())).thenReturn(bftContext); when(bftContext.as(any())).thenReturn(bftContext);
return bftContext; return bftContext;

View File

@@ -37,7 +37,6 @@ import org.hyperledger.besu.consensus.common.bft.ConsensusRoundIdentifier;
import org.hyperledger.besu.consensus.common.bft.RoundTimer; import org.hyperledger.besu.consensus.common.bft.RoundTimer;
import org.hyperledger.besu.consensus.common.bft.events.RoundExpiry; 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.network.ValidatorMulticaster;
import org.hyperledger.besu.consensus.common.validator.ValidatorProvider;
import org.hyperledger.besu.consensus.qbft.core.QbftBlockTestFixture; import org.hyperledger.besu.consensus.qbft.core.QbftBlockTestFixture;
import org.hyperledger.besu.consensus.qbft.core.messagedata.RoundChangeMessageData; 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.Commit;
@@ -49,6 +48,8 @@ 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.QbftBlock;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockCodec; 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.QbftBlockCreator;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockHashing;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockHeader;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockImporter; 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.QbftBlockInterface;
import org.hyperledger.besu.consensus.qbft.core.types.QbftContext; import org.hyperledger.besu.consensus.qbft.core.types.QbftContext;
@@ -56,9 +57,11 @@ 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.QbftFinalState;
import org.hyperledger.besu.consensus.qbft.core.types.QbftProtocolSchedule; 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.types.QbftProtocolSpec;
import org.hyperledger.besu.consensus.qbft.core.types.QbftValidatorProvider;
import org.hyperledger.besu.consensus.qbft.core.validation.FutureRoundProposalMessageValidator; 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.MessageValidator;
import org.hyperledger.besu.consensus.qbft.core.validation.MessageValidatorFactory; import org.hyperledger.besu.consensus.qbft.core.validation.MessageValidatorFactory;
import org.hyperledger.besu.consensus.qbft.core.validation.QbftBlockHeaderTestFixture;
import org.hyperledger.besu.consensus.qbft.core.validation.RoundChangeMessageValidator; import org.hyperledger.besu.consensus.qbft.core.validation.RoundChangeMessageValidator;
import org.hyperledger.besu.crypto.SignatureAlgorithmFactory; import org.hyperledger.besu.crypto.SignatureAlgorithmFactory;
import org.hyperledger.besu.cryptoservices.NodeKey; import org.hyperledger.besu.cryptoservices.NodeKey;
@@ -66,8 +69,6 @@ import org.hyperledger.besu.cryptoservices.NodeKeyUtils;
import org.hyperledger.besu.datatypes.Address; import org.hyperledger.besu.datatypes.Address;
import org.hyperledger.besu.datatypes.Hash; import org.hyperledger.besu.datatypes.Hash;
import org.hyperledger.besu.ethereum.ProtocolContext; import org.hyperledger.besu.ethereum.ProtocolContext;
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.core.Util;
import org.hyperledger.besu.ethereum.p2p.rlpx.wire.MessageData; import org.hyperledger.besu.ethereum.p2p.rlpx.wire.MessageData;
import org.hyperledger.besu.util.Subscribers; import org.hyperledger.besu.util.Subscribers;
@@ -95,7 +96,7 @@ import org.mockito.quality.Strictness;
public class QbftBlockHeightManagerTest { public class QbftBlockHeightManagerTest {
private final NodeKey nodeKey = NodeKeyUtils.generate(); private final NodeKey nodeKey = NodeKeyUtils.generate();
private final BlockHeaderTestFixture headerTestFixture = new BlockHeaderTestFixture(); private final QbftBlockHeaderTestFixture headerTestFixture = new QbftBlockHeaderTestFixture();
private MessageFactory messageFactory; private MessageFactory messageFactory;
@Mock private QbftFinalState finalState; @Mock private QbftFinalState finalState;
@@ -111,14 +112,15 @@ public class QbftBlockHeightManagerTest {
@Mock private ValidatorMulticaster validatorMulticaster; @Mock private ValidatorMulticaster validatorMulticaster;
@Mock private ProtocolContext protocolContext; @Mock private ProtocolContext protocolContext;
@Mock private QbftProtocolSchedule protocolSchedule; @Mock private QbftProtocolSchedule protocolSchedule;
@Mock private BlockHeader parentHeader; @Mock private QbftBlockHeader parentHeader;
@Mock private BftExtraDataCodec bftExtraDataCodec; @Mock private BftExtraDataCodec bftExtraDataCodec;
@Mock private QbftBlockCodec blockEncoder; @Mock private QbftBlockCodec blockEncoder;
@Mock private QbftExtraDataProvider qbftExtraDataProvider; @Mock private QbftExtraDataProvider qbftExtraDataProvider;
@Mock private QbftBlockInterface blockInterface; @Mock private QbftBlockInterface blockInterface;
@Mock private ValidatorProvider validatorProvider; @Mock private QbftValidatorProvider validatorProvider;
@Mock private QbftProtocolSpec protocolSpec; @Mock private QbftProtocolSpec protocolSpec;
@Mock private QbftBlockImporter blockImporter; @Mock private QbftBlockImporter blockImporter;
@Mock private QbftBlockHashing blockHashing;
@Captor private ArgumentCaptor<MessageData> sentMessageArgCaptor; @Captor private ArgumentCaptor<MessageData> sentMessageArgCaptor;
@@ -176,6 +178,7 @@ public class QbftBlockHeightManagerTest {
roundTimer, roundTimer,
bftExtraDataCodec, bftExtraDataCodec,
qbftExtraDataProvider, qbftExtraDataProvider,
blockHashing,
parentHeader); parentHeader);
}); });
@@ -195,6 +198,7 @@ public class QbftBlockHeightManagerTest {
roundTimer, roundTimer,
bftExtraDataCodec, bftExtraDataCodec,
qbftExtraDataProvider, qbftExtraDataProvider,
blockHashing,
parentHeader); parentHeader);
}); });
@@ -406,6 +410,7 @@ public class QbftBlockHeightManagerTest {
when(finalState.getQuorum()).thenReturn(1); when(finalState.getQuorum()).thenReturn(1);
when(blockInterface.replaceRoundInBlock(eq(createdBlock), eq(2), any())) when(blockInterface.replaceRoundInBlock(eq(createdBlock), eq(2), any()))
.thenReturn(createdBlock); .thenReturn(createdBlock);
when(blockHashing.calculateDataHashForCommittedSeal(any(), any())).thenReturn(Hash.ZERO);
when(blockCreator.createSealedBlock(any(), any(), anyInt(), any())).thenReturn(createdBlock); when(blockCreator.createSealedBlock(any(), any(), anyInt(), any())).thenReturn(createdBlock);
when(protocolSchedule.getByBlockHeader(any())).thenReturn(protocolSpec); when(protocolSchedule.getByBlockHeader(any())).thenReturn(protocolSpec);
when(protocolSpec.getBlockImporter()).thenReturn(blockImporter); when(protocolSpec.getBlockImporter()).thenReturn(blockImporter);
@@ -456,6 +461,7 @@ public class QbftBlockHeightManagerTest {
when(blockTimer.checkEmptyBlockExpired(any(), eq(0L))).thenReturn(true); when(blockTimer.checkEmptyBlockExpired(any(), eq(0L))).thenReturn(true);
when(blockInterface.replaceRoundInBlock(eq(createdBlock), eq(0), any())) when(blockInterface.replaceRoundInBlock(eq(createdBlock), eq(0), any()))
.thenReturn(createdBlock); .thenReturn(createdBlock);
when(blockHashing.calculateDataHashForCommittedSeal(any(), any())).thenReturn(Hash.ZERO);
when(blockCreator.createSealedBlock(any(), any(), anyInt(), any())).thenReturn(createdBlock); when(blockCreator.createSealedBlock(any(), any(), anyInt(), any())).thenReturn(createdBlock);
when(protocolSchedule.getByBlockHeader(any())).thenReturn(protocolSpec); when(protocolSchedule.getByBlockHeader(any())).thenReturn(protocolSpec);
when(protocolSpec.getBlockImporter()).thenReturn(blockImporter); when(protocolSpec.getBlockImporter()).thenReturn(blockImporter);

View File

@@ -30,7 +30,6 @@ import org.hyperledger.besu.consensus.common.bft.EthSynchronizerUpdater;
import org.hyperledger.besu.consensus.common.bft.MessageTracker; import org.hyperledger.besu.consensus.common.bft.MessageTracker;
import org.hyperledger.besu.consensus.common.bft.events.BftReceivedMessageEvent; 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.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.events.RoundExpiry;
import org.hyperledger.besu.consensus.common.bft.statemachine.FutureMessageBuffer; 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.CommitMessageData;
@@ -44,11 +43,12 @@ 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.messagewrappers.RoundChange;
import org.hyperledger.besu.consensus.qbft.core.network.QbftGossip; 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.QbftBlockCodec;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockHeader;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockchain;
import org.hyperledger.besu.consensus.qbft.core.types.QbftFinalState; import org.hyperledger.besu.consensus.qbft.core.types.QbftFinalState;
import org.hyperledger.besu.consensus.qbft.core.types.QbftNewChainHead;
import org.hyperledger.besu.datatypes.Address; import org.hyperledger.besu.datatypes.Address;
import org.hyperledger.besu.datatypes.Hash; import org.hyperledger.besu.datatypes.Hash;
import org.hyperledger.besu.ethereum.chain.Blockchain;
import org.hyperledger.besu.ethereum.core.BlockHeader;
import org.hyperledger.besu.ethereum.p2p.rlpx.wire.DefaultMessage; import org.hyperledger.besu.ethereum.p2p.rlpx.wire.DefaultMessage;
import org.hyperledger.besu.ethereum.p2p.rlpx.wire.Message; import org.hyperledger.besu.ethereum.p2p.rlpx.wire.Message;
@@ -67,11 +67,11 @@ import org.mockito.quality.Strictness;
@ExtendWith(MockitoExtension.class) @ExtendWith(MockitoExtension.class)
@MockitoSettings(strictness = Strictness.LENIENT) @MockitoSettings(strictness = Strictness.LENIENT)
public class QbftControllerTest { public class QbftControllerTest {
@Mock private Blockchain blockChain; @Mock private QbftBlockchain blockChain;
@Mock private QbftFinalState qbftFinalState; @Mock private QbftFinalState qbftFinalState;
@Mock private QbftBlockHeightManagerFactory blockHeightManagerFactory; @Mock private QbftBlockHeightManagerFactory blockHeightManagerFactory;
@Mock private BlockHeader chainHeadBlockHeader; @Mock private QbftBlockHeader chainHeadBlockHeader;
@Mock private BlockHeader nextBlock; @Mock private QbftBlockHeader nextBlock;
@Mock private BaseQbftBlockHeightManager blockHeightManager; @Mock private BaseQbftBlockHeightManager blockHeightManager;
@Mock private Proposal proposal; @Mock private Proposal proposal;
@@ -187,7 +187,7 @@ public class QbftControllerTest {
constructQbftController(); constructQbftController();
qbftController.start(); qbftController.start();
final NewChainHead newChainHead = new NewChainHead(nextBlock); final QbftNewChainHead newChainHead = new QbftNewChainHead(nextBlock);
qbftController.handleNewBlockEvent(newChainHead); qbftController.handleNewBlockEvent(newChainHead);
verify(blockHeightManagerFactory).create(nextBlock); verify(blockHeightManagerFactory).create(nextBlock);
@@ -210,11 +210,11 @@ public class QbftControllerTest {
long chainHeadHeight = 4; long chainHeadHeight = 4;
when(nextBlock.getNumber()).thenReturn(chainHeadHeight); when(nextBlock.getNumber()).thenReturn(chainHeadHeight);
when(nextBlock.getHash()).thenReturn(Hash.ZERO); when(nextBlock.getHash()).thenReturn(Hash.ZERO);
final NewChainHead sameHeightBlock = new NewChainHead(nextBlock); final QbftNewChainHead sameHeightBlock = new QbftNewChainHead(nextBlock);
qbftController.handleNewBlockEvent(sameHeightBlock); qbftController.handleNewBlockEvent(sameHeightBlock);
when(nextBlock.getNumber()).thenReturn(chainHeadHeight - 1); when(nextBlock.getNumber()).thenReturn(chainHeadHeight - 1);
final NewChainHead priorBlock = new NewChainHead(nextBlock); final QbftNewChainHead priorBlock = new QbftNewChainHead(nextBlock);
qbftController.handleNewBlockEvent(priorBlock); qbftController.handleNewBlockEvent(priorBlock);
verify(blockHeightManagerFactory, times(2)).create(any()); // 2 blocks created verify(blockHeightManagerFactory, times(2)).create(any()); // 2 blocks created
} }

View File

@@ -42,6 +42,8 @@ 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.QbftBlock;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockCodec; 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.QbftBlockCreator;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockHashing;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockHeader;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockImporter; 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.QbftBlockInterface;
import org.hyperledger.besu.consensus.qbft.core.types.QbftContext; import org.hyperledger.besu.consensus.qbft.core.types.QbftContext;
@@ -51,15 +53,15 @@ 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.QbftProtocolSchedule;
import org.hyperledger.besu.consensus.qbft.core.types.QbftProtocolSpec; import org.hyperledger.besu.consensus.qbft.core.types.QbftProtocolSpec;
import org.hyperledger.besu.consensus.qbft.core.validation.MessageValidator; import org.hyperledger.besu.consensus.qbft.core.validation.MessageValidator;
import org.hyperledger.besu.consensus.qbft.core.validation.QbftBlockHeaderTestFixture;
import org.hyperledger.besu.crypto.SECPSignature; import org.hyperledger.besu.crypto.SECPSignature;
import org.hyperledger.besu.crypto.SignatureAlgorithmFactory; import org.hyperledger.besu.crypto.SignatureAlgorithmFactory;
import org.hyperledger.besu.cryptoservices.NodeKey; import org.hyperledger.besu.cryptoservices.NodeKey;
import org.hyperledger.besu.cryptoservices.NodeKeyUtils; import org.hyperledger.besu.cryptoservices.NodeKeyUtils;
import org.hyperledger.besu.datatypes.Hash;
import org.hyperledger.besu.ethereum.ProtocolContext; import org.hyperledger.besu.ethereum.ProtocolContext;
import org.hyperledger.besu.ethereum.chain.BadBlockManager; import org.hyperledger.besu.ethereum.chain.BadBlockManager;
import org.hyperledger.besu.ethereum.chain.MutableBlockchain; import org.hyperledger.besu.ethereum.chain.MutableBlockchain;
import org.hyperledger.besu.ethereum.core.BlockHeader;
import org.hyperledger.besu.ethereum.core.BlockHeaderTestFixture;
import org.hyperledger.besu.ethereum.worldstate.WorldStateArchive; import org.hyperledger.besu.ethereum.worldstate.WorldStateArchive;
import org.hyperledger.besu.plugin.services.securitymodule.SecurityModuleException; import org.hyperledger.besu.plugin.services.securitymodule.SecurityModuleException;
import org.hyperledger.besu.util.Subscribers; import org.hyperledger.besu.util.Subscribers;
@@ -102,11 +104,12 @@ public class QbftRoundTest {
@Mock private RoundTimer roundTimer; @Mock private RoundTimer roundTimer;
@Mock private QbftProtocolSpec protocolSpec; @Mock private QbftProtocolSpec protocolSpec;
@Mock private QbftBlockImporter blockImporter; @Mock private QbftBlockImporter blockImporter;
@Mock private org.hyperledger.besu.ethereum.core.BlockHeader parentHeader; @Mock private QbftBlockHeader parentHeader;
@Mock private BftExtraDataCodec bftExtraDataCodec; @Mock private BftExtraDataCodec bftExtraDataCodec;
@Mock private QbftBlockInterface blockInteface; @Mock private QbftBlockInterface blockInteface;
@Mock private QbftBlockCodec blockEncoder; @Mock private QbftBlockCodec blockEncoder;
@Mock private QbftExtraDataProvider qbftExtraDataProvider; @Mock private QbftExtraDataProvider qbftExtraDataProvider;
@Mock private QbftBlockHashing blockHashing;
@Captor private ArgumentCaptor<QbftBlock> blockCaptor; @Captor private ArgumentCaptor<QbftBlock> blockCaptor;
@@ -132,7 +135,7 @@ public class QbftRoundTest {
when(messageValidator.validatePrepare(any())).thenReturn(true); when(messageValidator.validatePrepare(any())).thenReturn(true);
when(messageValidator.validateCommit(any())).thenReturn(true); when(messageValidator.validateCommit(any())).thenReturn(true);
final BlockHeader header = new BlockHeaderTestFixture().number(1).buildHeader(); final QbftBlockHeader header = new QbftBlockHeaderTestFixture().number(1).buildHeader();
proposedBlock = new QbftBlockTestFixture().blockHeader(header).build(); proposedBlock = new QbftBlockTestFixture().blockHeader(header).build();
@@ -168,6 +171,7 @@ public class QbftRoundTest {
roundTimer, roundTimer,
bftExtraDataCodec, bftExtraDataCodec,
qbftExtraDataProvider, qbftExtraDataProvider,
blockHashing,
parentHeader); parentHeader);
verify(roundTimer, times(1)).startTimer(roundIdentifier); verify(roundTimer, times(1)).startTimer(roundIdentifier);
} }
@@ -188,6 +192,7 @@ public class QbftRoundTest {
roundTimer, roundTimer,
bftExtraDataCodec, bftExtraDataCodec,
qbftExtraDataProvider, qbftExtraDataProvider,
blockHashing,
parentHeader); parentHeader);
when(blockInteface.replaceRoundInBlock(eq(proposedBlock), eq(0), any())) when(blockInteface.replaceRoundInBlock(eq(proposedBlock), eq(0), any()))
@@ -202,7 +207,7 @@ public class QbftRoundTest {
@Test @Test
public void aProposalWithAnewBlockIsSentUponReceptionOfARoundChangeWithNoCertificate() { public void aProposalWithAnewBlockIsSentUponReceptionOfARoundChangeWithNoCertificate() {
final BlockHeader header = new BlockHeaderTestFixture().number(0).buildHeader(); final QbftBlockHeader header = new QbftBlockHeaderTestFixture().number(0).buildHeader();
final QbftBlock commitBlock = new QbftBlockTestFixture().blockHeader(header).build(); final QbftBlock commitBlock = new QbftBlockTestFixture().blockHeader(header).build();
when(blockInteface.replaceRoundInBlock(proposedBlock, 0, QbftHashMode.COMMITTED_SEAL)) when(blockInteface.replaceRoundInBlock(proposedBlock, 0, QbftHashMode.COMMITTED_SEAL))
@@ -222,6 +227,7 @@ public class QbftRoundTest {
roundTimer, roundTimer,
bftExtraDataCodec, bftExtraDataCodec,
qbftExtraDataProvider, qbftExtraDataProvider,
blockHashing,
parentHeader); parentHeader);
round.startRoundWith(new RoundChangeArtifacts(emptyList(), Optional.empty()), 15); round.startRoundWith(new RoundChangeArtifacts(emptyList(), Optional.empty()), 15);
@@ -234,11 +240,11 @@ public class QbftRoundTest {
public void aProposalMessageWithTheSameBlockIsSentUponReceptionOfARoundChangeWithCertificate() { public void aProposalMessageWithTheSameBlockIsSentUponReceptionOfARoundChangeWithCertificate() {
final QbftBlock publishBlock = final QbftBlock publishBlock =
new QbftBlockTestFixture() new QbftBlockTestFixture()
.blockHeader(new BlockHeaderTestFixture().number(0).buildHeader()) .blockHeader(new QbftBlockHeaderTestFixture().number(0).buildHeader())
.build(); .build();
final QbftBlock commitBlock = final QbftBlock commitBlock =
new QbftBlockTestFixture() new QbftBlockTestFixture()
.blockHeader(new BlockHeaderTestFixture().number(0).buildHeader()) .blockHeader(new QbftBlockHeaderTestFixture().number(0).buildHeader())
.build(); .build();
when(blockInteface.replaceRoundInBlock(proposedBlock, 0, QbftHashMode.COMMITTED_SEAL)) when(blockInteface.replaceRoundInBlock(proposedBlock, 0, QbftHashMode.COMMITTED_SEAL))
.thenReturn(publishBlock); .thenReturn(publishBlock);
@@ -260,6 +266,7 @@ public class QbftRoundTest {
roundTimer, roundTimer,
bftExtraDataCodec, bftExtraDataCodec,
qbftExtraDataProvider, qbftExtraDataProvider,
blockHashing,
parentHeader); parentHeader);
final SignedData<PreparePayload> preparedPayload = final SignedData<PreparePayload> preparedPayload =
@@ -294,7 +301,7 @@ public class QbftRoundTest {
public void creatingNewBlockFromEmptyPreparedCertificateUpdatesInternalState() { public void creatingNewBlockFromEmptyPreparedCertificateUpdatesInternalState() {
final QbftBlock commitBlock = final QbftBlock commitBlock =
new QbftBlockTestFixture() new QbftBlockTestFixture()
.blockHeader(new BlockHeaderTestFixture().number(0).buildHeader()) .blockHeader(new QbftBlockHeaderTestFixture().number(0).buildHeader())
.build(); .build();
when(blockInteface.replaceRoundInBlock(proposedBlock, 0, QbftHashMode.COMMITTED_SEAL)) when(blockInteface.replaceRoundInBlock(proposedBlock, 0, QbftHashMode.COMMITTED_SEAL))
.thenReturn(commitBlock); .thenReturn(commitBlock);
@@ -313,6 +320,7 @@ public class QbftRoundTest {
roundTimer, roundTimer,
bftExtraDataCodec, bftExtraDataCodec,
qbftExtraDataProvider, qbftExtraDataProvider,
blockHashing,
parentHeader); parentHeader);
final RoundChange roundChange = final RoundChange roundChange =
@@ -356,12 +364,14 @@ public class QbftRoundTest {
roundTimer, roundTimer,
bftExtraDataCodec, bftExtraDataCodec,
qbftExtraDataProvider, qbftExtraDataProvider,
blockHashing,
parentHeader); parentHeader);
when(blockInteface.replaceRoundInBlock(proposedBlock, 0, QbftHashMode.COMMITTED_SEAL)) when(blockInteface.replaceRoundInBlock(proposedBlock, 0, QbftHashMode.COMMITTED_SEAL))
.thenReturn(proposedBlock); .thenReturn(proposedBlock);
when(blockCreator.createSealedBlock(eq(qbftExtraDataProvider), eq(proposedBlock), eq(0), any())) when(blockCreator.createSealedBlock(eq(qbftExtraDataProvider), eq(proposedBlock), eq(0), any()))
.thenReturn(proposedBlock); .thenReturn(proposedBlock);
when(blockHashing.calculateDataHashForCommittedSeal(any(), any())).thenReturn(Hash.ZERO);
round.handleCommitMessage( round.handleCommitMessage(
messageFactory.createCommit(roundIdentifier, proposedBlock.getHash(), remoteCommitSeal)); messageFactory.createCommit(roundIdentifier, proposedBlock.getHash(), remoteCommitSeal));
@@ -390,12 +400,14 @@ public class QbftRoundTest {
roundTimer, roundTimer,
bftExtraDataCodec, bftExtraDataCodec,
qbftExtraDataProvider, qbftExtraDataProvider,
blockHashing,
parentHeader); parentHeader);
when(blockInteface.replaceRoundInBlock(eq(proposedBlock), eq(0), any())) when(blockInteface.replaceRoundInBlock(eq(proposedBlock), eq(0), any()))
.thenReturn(proposedBlock); .thenReturn(proposedBlock);
when(blockCreator.createSealedBlock(eq(qbftExtraDataProvider), eq(proposedBlock), eq(0), any())) when(blockCreator.createSealedBlock(eq(qbftExtraDataProvider), eq(proposedBlock), eq(0), any()))
.thenReturn(proposedBlock); .thenReturn(proposedBlock);
when(blockHashing.calculateDataHashForCommittedSeal(any(), any())).thenReturn(Hash.ZERO);
round.handleCommitMessage( round.handleCommitMessage(
messageFactory.createCommit(roundIdentifier, proposedBlock.getHash(), remoteCommitSeal)); messageFactory.createCommit(roundIdentifier, proposedBlock.getHash(), remoteCommitSeal));
@@ -428,6 +440,7 @@ public class QbftRoundTest {
roundTimer, roundTimer,
bftExtraDataCodec, bftExtraDataCodec,
qbftExtraDataProvider, qbftExtraDataProvider,
blockHashing,
parentHeader); parentHeader);
when(blockInteface.replaceRoundInBlock(eq(proposedBlock), eq(0), any())) when(blockInteface.replaceRoundInBlock(eq(proposedBlock), eq(0), any()))

View File

@@ -27,12 +27,11 @@ 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.MessageFactory;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlock; 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.QbftBlockCodec;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockHeader;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockValidator; import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockValidator;
import org.hyperledger.besu.cryptoservices.NodeKey; import org.hyperledger.besu.cryptoservices.NodeKey;
import org.hyperledger.besu.cryptoservices.NodeKeyUtils; import org.hyperledger.besu.cryptoservices.NodeKeyUtils;
import org.hyperledger.besu.datatypes.Address; import org.hyperledger.besu.datatypes.Address;
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.core.Util;
import java.util.Optional; import java.util.Optional;
@@ -68,8 +67,8 @@ public class ProposalPayloadValidatorTest {
public void validationPassesWhenProposerAndRoundMatchAndBlockIsValid() { public void validationPassesWhenProposerAndRoundMatchAndBlockIsValid() {
final ProposalPayloadValidator payloadValidator = final ProposalPayloadValidator payloadValidator =
new ProposalPayloadValidator(expectedProposer, roundIdentifier, blockValidator); new ProposalPayloadValidator(expectedProposer, roundIdentifier, blockValidator);
final BlockHeader header = final QbftBlockHeader header =
new BlockHeaderTestFixture().number(roundIdentifier.getSequenceNumber()).buildHeader(); new QbftBlockHeaderTestFixture().number(roundIdentifier.getSequenceNumber()).buildHeader();
final QbftBlock block = new QbftBlockTestFixture().blockHeader(header).build(); final QbftBlock block = new QbftBlockTestFixture().blockHeader(header).build();
final Proposal proposal = final Proposal proposal =
messageFactory.createProposal(roundIdentifier, block, emptyList(), emptyList()); messageFactory.createProposal(roundIdentifier, block, emptyList(), emptyList());
@@ -85,8 +84,8 @@ public class ProposalPayloadValidatorTest {
final ProposalPayloadValidator payloadValidator = final ProposalPayloadValidator payloadValidator =
new ProposalPayloadValidator(expectedProposer, roundIdentifier, blockValidator); new ProposalPayloadValidator(expectedProposer, roundIdentifier, blockValidator);
final BlockHeader header = final QbftBlockHeader header =
new BlockHeaderTestFixture().number(roundIdentifier.getSequenceNumber()).buildHeader(); new QbftBlockHeaderTestFixture().number(roundIdentifier.getSequenceNumber()).buildHeader();
final QbftBlock block = new QbftBlockTestFixture().blockHeader(header).build(); final QbftBlock block = new QbftBlockTestFixture().blockHeader(header).build();
final Proposal proposal = final Proposal proposal =
messageFactory.createProposal(roundIdentifier, block, emptyList(), emptyList()); messageFactory.createProposal(roundIdentifier, block, emptyList(), emptyList());
@@ -104,8 +103,8 @@ public class ProposalPayloadValidatorTest {
final ProposalPayloadValidator payloadValidator = final ProposalPayloadValidator payloadValidator =
new ProposalPayloadValidator(expectedProposer, roundIdentifier, blockValidator); new ProposalPayloadValidator(expectedProposer, roundIdentifier, blockValidator);
final BlockHeader header = final QbftBlockHeader header =
new BlockHeaderTestFixture().number(roundIdentifier.getSequenceNumber()).buildHeader(); new QbftBlockHeaderTestFixture().number(roundIdentifier.getSequenceNumber()).buildHeader();
final QbftBlock block = new QbftBlockTestFixture().blockHeader(header).build(); final QbftBlock block = new QbftBlockTestFixture().blockHeader(header).build();
final Proposal proposal = final Proposal proposal =
messageFactory.createProposal(roundIdentifier, block, emptyList(), emptyList()); messageFactory.createProposal(roundIdentifier, block, emptyList(), emptyList());
@@ -120,8 +119,8 @@ public class ProposalPayloadValidatorTest {
public void validationFailsWhenExpectedProposerDoesNotMatchPayloadsAuthor() { public void validationFailsWhenExpectedProposerDoesNotMatchPayloadsAuthor() {
final ProposalPayloadValidator payloadValidator = final ProposalPayloadValidator payloadValidator =
new ProposalPayloadValidator(Address.fromHexString("0x1"), roundIdentifier, blockValidator); new ProposalPayloadValidator(Address.fromHexString("0x1"), roundIdentifier, blockValidator);
final BlockHeader header = final QbftBlockHeader header =
new BlockHeaderTestFixture().number(roundIdentifier.getSequenceNumber()).buildHeader(); new QbftBlockHeaderTestFixture().number(roundIdentifier.getSequenceNumber()).buildHeader();
final QbftBlock block = new QbftBlockTestFixture().blockHeader(header).build(); final QbftBlock block = new QbftBlockTestFixture().blockHeader(header).build();
final Proposal proposal = final Proposal proposal =
messageFactory.createProposal(roundIdentifier, block, emptyList(), emptyList()); messageFactory.createProposal(roundIdentifier, block, emptyList(), emptyList());
@@ -135,8 +134,8 @@ public class ProposalPayloadValidatorTest {
final ProposalPayloadValidator payloadValidator = final ProposalPayloadValidator payloadValidator =
new ProposalPayloadValidator(expectedProposer, roundIdentifier, blockValidator); new ProposalPayloadValidator(expectedProposer, roundIdentifier, blockValidator);
final BlockHeader header = final QbftBlockHeader header =
new BlockHeaderTestFixture().number(roundIdentifier.getSequenceNumber()).buildHeader(); new QbftBlockHeaderTestFixture().number(roundIdentifier.getSequenceNumber()).buildHeader();
final QbftBlock block = new QbftBlockTestFixture().blockHeader(header).build(); final QbftBlock block = new QbftBlockTestFixture().blockHeader(header).build();
final Proposal proposal = final Proposal proposal =
messageFactory.createProposal( messageFactory.createProposal(
@@ -154,8 +153,8 @@ public class ProposalPayloadValidatorTest {
final ProposalPayloadValidator payloadValidator = final ProposalPayloadValidator payloadValidator =
new ProposalPayloadValidator(expectedProposer, roundIdentifier, blockValidator); new ProposalPayloadValidator(expectedProposer, roundIdentifier, blockValidator);
final BlockHeader header = final QbftBlockHeader header =
new BlockHeaderTestFixture().number(roundIdentifier.getSequenceNumber()).buildHeader(); new QbftBlockHeaderTestFixture().number(roundIdentifier.getSequenceNumber()).buildHeader();
final QbftBlock block = new QbftBlockTestFixture().blockHeader(header).build(); final QbftBlock block = new QbftBlockTestFixture().blockHeader(header).build();
final Proposal proposal = final Proposal proposal =
messageFactory.createProposal( messageFactory.createProposal(
@@ -172,8 +171,10 @@ public class ProposalPayloadValidatorTest {
public void validationFailsForBlockWithIncorrectHeight() { public void validationFailsForBlockWithIncorrectHeight() {
final ProposalPayloadValidator payloadValidator = final ProposalPayloadValidator payloadValidator =
new ProposalPayloadValidator(expectedProposer, roundIdentifier, blockValidator); new ProposalPayloadValidator(expectedProposer, roundIdentifier, blockValidator);
final BlockHeader header = final QbftBlockHeader header =
new BlockHeaderTestFixture().number(roundIdentifier.getSequenceNumber() + 1).buildHeader(); new QbftBlockHeaderTestFixture()
.number(roundIdentifier.getSequenceNumber() + 1)
.buildHeader();
final QbftBlock block = new QbftBlockTestFixture().blockHeader(header).build(); final QbftBlock block = new QbftBlockTestFixture().blockHeader(header).build();
final Proposal proposal = final Proposal proposal =
messageFactory.createProposal(roundIdentifier, block, emptyList(), emptyList()); messageFactory.createProposal(roundIdentifier, block, emptyList(), emptyList());

View File

@@ -27,7 +27,6 @@ import org.hyperledger.besu.consensus.common.bft.BftHelpers;
import org.hyperledger.besu.consensus.common.bft.ConsensusRoundHelpers; import org.hyperledger.besu.consensus.common.bft.ConsensusRoundHelpers;
import org.hyperledger.besu.consensus.common.bft.ConsensusRoundIdentifier; import org.hyperledger.besu.consensus.common.bft.ConsensusRoundIdentifier;
import org.hyperledger.besu.consensus.common.bft.payload.SignedData; 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.QbftBlockTestFixture;
import org.hyperledger.besu.consensus.qbft.core.messagewrappers.Prepare; 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.Proposal;
@@ -37,16 +36,16 @@ 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.payload.RoundChangePayload;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlock; 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.QbftBlockCodec;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockHeader;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockInterface; 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.QbftBlockValidator;
import org.hyperledger.besu.consensus.qbft.core.types.QbftContext; 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.QbftHashMode;
import org.hyperledger.besu.consensus.qbft.core.types.QbftProtocolSchedule; 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.types.QbftProtocolSpec;
import org.hyperledger.besu.consensus.qbft.core.types.QbftValidatorProvider;
import org.hyperledger.besu.datatypes.Hash; import org.hyperledger.besu.datatypes.Hash;
import org.hyperledger.besu.ethereum.ProtocolContext; import org.hyperledger.besu.ethereum.ProtocolContext;
import org.hyperledger.besu.ethereum.core.BlockHeader;
import org.hyperledger.besu.ethereum.core.BlockHeaderTestFixture;
import java.util.HashMap; import java.util.HashMap;
import java.util.List; import java.util.List;
@@ -89,7 +88,7 @@ public class ProposalValidatorTest {
@Mock private QbftProtocolSpec protocolSpec; @Mock private QbftProtocolSpec protocolSpec;
@Mock private QbftBlockCodec blockEncoder; @Mock private QbftBlockCodec blockEncoder;
@Mock private QbftBlockInterface blockInterface; @Mock private QbftBlockInterface blockInterface;
@Mock private ValidatorProvider validatorProvider; @Mock private QbftValidatorProvider validatorProvider;
@Mock private ProtocolContext protocolContext; @Mock private ProtocolContext protocolContext;
private QbftNodeList validators; private QbftNodeList validators;
@@ -115,8 +114,8 @@ public class ProposalValidatorTest {
private RoundSpecificItems createRoundSpecificItems(final int roundNumber) { private RoundSpecificItems createRoundSpecificItems(final int roundNumber) {
final ConsensusRoundIdentifier roundIdentifier = new ConsensusRoundIdentifier(1, roundNumber); final ConsensusRoundIdentifier roundIdentifier = new ConsensusRoundIdentifier(1, roundNumber);
final BlockHeader blockHeader = final QbftBlockHeader blockHeader =
new BlockHeaderTestFixture() new QbftBlockHeaderTestFixture()
.number(roundIdentifier.getSequenceNumber()) .number(roundIdentifier.getSequenceNumber())
.coinbase(validators.getNodeAddresses().getFirst()) .coinbase(validators.getNodeAddresses().getFirst())
.buildHeader(); .buildHeader();

View File

@@ -0,0 +1,71 @@
/*
* 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.validation;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockHeader;
import org.hyperledger.besu.datatypes.Address;
import org.hyperledger.besu.datatypes.Hash;
public class QbftBlockHeaderTestFixture {
private long number = 0;
private long timestamp = 0;
private Hash hash = Hash.EMPTY;
private Address coinbase = Address.ZERO;
public QbftBlockHeaderTestFixture number(final long number) {
this.number = number;
return this;
}
public QbftBlockHeaderTestFixture timestamp(final long timestamp) {
this.timestamp = timestamp;
return this;
}
public QbftBlockHeaderTestFixture hash(final Hash hash) {
this.hash = hash;
return this;
}
public QbftBlockHeaderTestFixture coinbase(final Address coinbase) {
this.coinbase = coinbase;
return this;
}
public QbftBlockHeader buildHeader() {
return new QbftBlockHeader() {
@Override
public long getNumber() {
return number;
}
@Override
public long getTimestamp() {
return timestamp;
}
@Override
public Hash getHash() {
return hash;
}
@Override
public Address getCoinbase() {
return coinbase;
}
};
}
}

View File

@@ -34,14 +34,13 @@ 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.statemachine.PreparedCertificate;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlock; 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.QbftBlockCodec;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockHeader;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockValidator; 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.QbftBlockValidator.ValidationResult;
import org.hyperledger.besu.consensus.qbft.core.types.QbftProtocolSchedule; 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.types.QbftProtocolSpec;
import org.hyperledger.besu.crypto.SECPSignature; import org.hyperledger.besu.crypto.SECPSignature;
import org.hyperledger.besu.datatypes.Hash; import org.hyperledger.besu.datatypes.Hash;
import org.hyperledger.besu.ethereum.core.BlockHeader;
import org.hyperledger.besu.ethereum.core.BlockHeaderTestFixture;
import java.util.Optional; import java.util.Optional;
import java.util.stream.Collectors; import java.util.stream.Collectors;
@@ -109,8 +108,8 @@ public class RoundChangeMessageValidatorTest {
validators.getNodeAddresses(), validators.getNodeAddresses(),
protocolSchedule); protocolSchedule);
final BlockHeader header = final QbftBlockHeader header =
new BlockHeaderTestFixture().number(roundIdentifier.getSequenceNumber()).buildHeader(); new QbftBlockHeaderTestFixture().number(roundIdentifier.getSequenceNumber()).buildHeader();
final QbftBlock block = new QbftBlockTestFixture().blockHeader(header).build(); final QbftBlock block = new QbftBlockTestFixture().blockHeader(header).build();
final PreparedCertificate prepCert = final PreparedCertificate prepCert =
createPreparedCertificate( createPreparedCertificate(
@@ -134,8 +133,8 @@ public class RoundChangeMessageValidatorTest {
validators.getNodeAddresses(), validators.getNodeAddresses(),
protocolSchedule); protocolSchedule);
final BlockHeader header = final QbftBlockHeader header =
new BlockHeaderTestFixture().number(roundIdentifier.getSequenceNumber()).buildHeader(); new QbftBlockHeaderTestFixture().number(roundIdentifier.getSequenceNumber()).buildHeader();
final QbftBlock block = new QbftBlockTestFixture().blockHeader(header).build(); final QbftBlock block = new QbftBlockTestFixture().blockHeader(header).build();
final PreparedCertificate prepCert = final PreparedCertificate prepCert =
createPreparedCertificate( createPreparedCertificate(
@@ -161,8 +160,8 @@ public class RoundChangeMessageValidatorTest {
validators.getNodeAddresses(), validators.getNodeAddresses(),
protocolSchedule); protocolSchedule);
final BlockHeader header = final QbftBlockHeader header =
new BlockHeaderTestFixture().number(roundIdentifier.getSequenceNumber()).buildHeader(); new QbftBlockHeaderTestFixture().number(roundIdentifier.getSequenceNumber()).buildHeader();
final QbftBlock block = new QbftBlockTestFixture().blockHeader(header).build(); final QbftBlock block = new QbftBlockTestFixture().blockHeader(header).build();
final PreparedCertificate prepCert = final PreparedCertificate prepCert =
createPreparedCertificate( createPreparedCertificate(
@@ -202,8 +201,8 @@ public class RoundChangeMessageValidatorTest {
validators.getNodeAddresses(), validators.getNodeAddresses(),
protocolSchedule); protocolSchedule);
final BlockHeader header = final QbftBlockHeader header =
new BlockHeaderTestFixture().number(roundIdentifier.getSequenceNumber()).buildHeader(); new QbftBlockHeaderTestFixture().number(roundIdentifier.getSequenceNumber()).buildHeader();
final QbftBlock block = new QbftBlockTestFixture().blockHeader(header).build(); final QbftBlock block = new QbftBlockTestFixture().blockHeader(header).build();
final PreparedCertificate prepCert = final PreparedCertificate prepCert =
createPreparedCertificate( createPreparedCertificate(
@@ -229,8 +228,8 @@ public class RoundChangeMessageValidatorTest {
final QbftNode nonValidator = QbftNode.create(blockEncoder); final QbftNode nonValidator = QbftNode.create(blockEncoder);
final BlockHeader header = final QbftBlockHeader header =
new BlockHeaderTestFixture().number(roundIdentifier.getSequenceNumber()).buildHeader(); new QbftBlockHeaderTestFixture().number(roundIdentifier.getSequenceNumber()).buildHeader();
final QbftBlock block = new QbftBlockTestFixture().blockHeader(header).build(); final QbftBlock block = new QbftBlockTestFixture().blockHeader(header).build();
final PreparedCertificate prepCert = final PreparedCertificate prepCert =
createPreparedCertificate( createPreparedCertificate(
@@ -254,8 +253,8 @@ public class RoundChangeMessageValidatorTest {
validators.getNodeAddresses(), validators.getNodeAddresses(),
protocolSchedule); protocolSchedule);
final BlockHeader header = final QbftBlockHeader header =
new BlockHeaderTestFixture().number(roundIdentifier.getSequenceNumber()).buildHeader(); new QbftBlockHeaderTestFixture().number(roundIdentifier.getSequenceNumber()).buildHeader();
final QbftBlock block = new QbftBlockTestFixture().blockHeader(header).build(); final QbftBlock block = new QbftBlockTestFixture().blockHeader(header).build();
final PreparedCertificate prepCert = final PreparedCertificate prepCert =
new PreparedCertificate( new PreparedCertificate(
@@ -287,8 +286,8 @@ public class RoundChangeMessageValidatorTest {
validators.getNodeAddresses(), validators.getNodeAddresses(),
protocolSchedule); protocolSchedule);
final BlockHeader header = final QbftBlockHeader header =
new BlockHeaderTestFixture().number(roundIdentifier.getSequenceNumber()).buildHeader(); new QbftBlockHeaderTestFixture().number(roundIdentifier.getSequenceNumber()).buildHeader();
final QbftBlock block = new QbftBlockTestFixture().blockHeader(header).build(); final QbftBlock block = new QbftBlockTestFixture().blockHeader(header).build();
final PreparedCertificate prepCert = final PreparedCertificate prepCert =
new PreparedCertificate( new PreparedCertificate(
@@ -322,8 +321,8 @@ public class RoundChangeMessageValidatorTest {
validators.getNodeAddresses(), validators.getNodeAddresses(),
protocolSchedule); protocolSchedule);
final BlockHeader header = final QbftBlockHeader header =
new BlockHeaderTestFixture().number(roundIdentifier.getSequenceNumber()).buildHeader(); new QbftBlockHeaderTestFixture().number(roundIdentifier.getSequenceNumber()).buildHeader();
final QbftBlock block = new QbftBlockTestFixture().blockHeader(header).build(); final QbftBlock block = new QbftBlockTestFixture().blockHeader(header).build();
final PreparedCertificate prepCert = final PreparedCertificate prepCert =
new PreparedCertificate( new PreparedCertificate(
@@ -357,8 +356,8 @@ public class RoundChangeMessageValidatorTest {
validators.getNodeAddresses(), validators.getNodeAddresses(),
protocolSchedule); protocolSchedule);
final BlockHeader header = final QbftBlockHeader header =
new BlockHeaderTestFixture().number(roundIdentifier.getSequenceNumber()).buildHeader(); new QbftBlockHeaderTestFixture().number(roundIdentifier.getSequenceNumber()).buildHeader();
final QbftBlock block = new QbftBlockTestFixture().blockHeader(header).build(); final QbftBlock block = new QbftBlockTestFixture().blockHeader(header).build();
final PreparedCertificate prepCert = final PreparedCertificate prepCert =
createPreparedCertificate( createPreparedCertificate(
@@ -387,8 +386,8 @@ public class RoundChangeMessageValidatorTest {
validators.getNodeAddresses(), validators.getNodeAddresses(),
protocolSchedule); protocolSchedule);
final BlockHeader header = final QbftBlockHeader header =
new BlockHeaderTestFixture().number(roundIdentifier.getSequenceNumber()).buildHeader(); new QbftBlockHeaderTestFixture().number(roundIdentifier.getSequenceNumber()).buildHeader();
final QbftBlock block = new QbftBlockTestFixture().blockHeader(header).build(); final QbftBlock block = new QbftBlockTestFixture().blockHeader(header).build();
final RoundChangePayload payload = new RoundChangePayload(targetRound, Optional.empty()); final RoundChangePayload payload = new RoundChangePayload(targetRound, Optional.empty());
@@ -415,8 +414,8 @@ public class RoundChangeMessageValidatorTest {
validators.getNodeAddresses(), validators.getNodeAddresses(),
protocolSchedule); protocolSchedule);
final BlockHeader header = final QbftBlockHeader header =
new BlockHeaderTestFixture().number(roundIdentifier.getSequenceNumber()).buildHeader(); new QbftBlockHeaderTestFixture().number(roundIdentifier.getSequenceNumber()).buildHeader();
final QbftBlock block = new QbftBlockTestFixture().blockHeader(header).build(); final QbftBlock block = new QbftBlockTestFixture().blockHeader(header).build();
final RoundChangePayload payload = final RoundChangePayload payload =

View File

@@ -0,0 +1,65 @@
/*
* 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.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.BftEventHandler;
import org.hyperledger.besu.consensus.qbft.core.types.QbftEventHandler;
import org.hyperledger.besu.consensus.qbft.core.types.QbftNewChainHead;
import org.hyperledger.besu.ethereum.core.BlockHeader;
/** Adaptor class to allow a {@link QbftEventHandler} to be used as a {@link BftEventHandler}. */
public class BftEventHandlerAdaptor implements BftEventHandler {
private final QbftEventHandler qbftEventHandler;
/**
* Create a new instance of the adaptor.
*
* @param qbftEventHandler The {@link QbftEventHandler} to adapt.
*/
public BftEventHandlerAdaptor(final QbftEventHandler qbftEventHandler) {
this.qbftEventHandler = qbftEventHandler;
}
@Override
public void start() {
qbftEventHandler.start();
}
@Override
public void handleMessageEvent(final BftReceivedMessageEvent msg) {
qbftEventHandler.handleMessageEvent(msg);
}
@Override
public void handleNewBlockEvent(final NewChainHead newChainHead) {
BlockHeader besuNewChainHeadHeader = newChainHead.getNewChainHeadHeader();
var qbftChainHead = new QbftNewChainHead(new QbftBlockHeaderAdaptor(besuNewChainHeadHeader));
qbftEventHandler.handleNewBlockEvent(qbftChainHead);
}
@Override
public void handleBlockTimerExpiry(final BlockTimerExpiry blockTimerExpiry) {
qbftEventHandler.handleBlockTimerExpiry(blockTimerExpiry);
}
@Override
public void handleRoundExpiry(final RoundExpiry roundExpiry) {
qbftEventHandler.handleRoundExpiry(roundExpiry);
}
}

View File

@@ -15,7 +15,9 @@
package org.hyperledger.besu.consensus.qbft.adaptor; package org.hyperledger.besu.consensus.qbft.adaptor;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlock; import org.hyperledger.besu.consensus.qbft.core.types.QbftBlock;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockHeader;
import org.hyperledger.besu.ethereum.core.Block; import org.hyperledger.besu.ethereum.core.Block;
import org.hyperledger.besu.ethereum.core.BlockHeader;
/** Utility class to convert between Besu and QBFT blocks. */ /** Utility class to convert between Besu and QBFT blocks. */
public class BlockUtil { public class BlockUtil {
@@ -36,4 +38,18 @@ public class BlockUtil {
throw new IllegalArgumentException("Unsupported block type"); throw new IllegalArgumentException("Unsupported block type");
} }
} }
/**
* Convert a QBFT block header to a Besu block header.
*
* @param header the QBFT block header
* @return the Besu block header
*/
public static BlockHeader toBesuBlockHeader(final QbftBlockHeader header) {
if (header instanceof QbftBlockHeaderAdaptor) {
return ((QbftBlockHeaderAdaptor) header).getBesuBlockHeader();
} else {
throw new IllegalArgumentException("Unsupported block header type");
}
}
} }

View File

@@ -15,35 +15,36 @@
package org.hyperledger.besu.consensus.qbft.adaptor; package org.hyperledger.besu.consensus.qbft.adaptor;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlock; import org.hyperledger.besu.consensus.qbft.core.types.QbftBlock;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockHeader;
import org.hyperledger.besu.datatypes.Hash;
import org.hyperledger.besu.ethereum.core.Block; import org.hyperledger.besu.ethereum.core.Block;
import org.hyperledger.besu.ethereum.core.BlockHeader;
import java.util.Objects; import java.util.Objects;
/** Adaptor class to allow a {@link Block} to be used as a {@link QbftBlock}. */ /** Adaptor class to allow a {@link Block} to be used as a {@link QbftBlock}. */
public class QbftBlockAdaptor implements QbftBlock { public class QbftBlockAdaptor implements QbftBlock {
private final BlockHeader header; private final Block besuBlock;
private final Block block; private final QbftBlockHeader qbftBlockHeader;
/** /**
* Constructs a QbftBlock from a Besu Block. * Constructs a QbftBlock from a Besu Block.
* *
* @param block the Besu Block * @param besuBlock the Besu Block
*/ */
public QbftBlockAdaptor(final Block block) { public QbftBlockAdaptor(final Block besuBlock) {
this.block = block; this.besuBlock = besuBlock;
this.header = block.getHeader(); this.qbftBlockHeader = new QbftBlockHeaderAdaptor(besuBlock.getHeader());
} }
@Override @Override
public BlockHeader getHeader() { public QbftBlockHeader getHeader() {
return header; return qbftBlockHeader;
} }
@Override @Override
public boolean isEmpty() { public boolean isEmpty() {
return block.getBody().getTransactions().isEmpty(); return besuBlock.getHeader().getTransactionsRoot().equals(Hash.EMPTY_TRIE_HASH);
} }
/** /**
@@ -53,17 +54,18 @@ public class QbftBlockAdaptor implements QbftBlock {
* @return the Besu Block * @return the Besu Block
*/ */
public Block getBesuBlock() { public Block getBesuBlock() {
return block; return besuBlock;
} }
@Override @Override
public boolean equals(final Object o) { public boolean equals(final Object o) {
if (!(o instanceof QbftBlockAdaptor qbftBlock)) return false; if (!(o instanceof QbftBlockAdaptor qbftBlock)) return false;
return Objects.equals(header, qbftBlock.header) && Objects.equals(block, qbftBlock.block); return Objects.equals(besuBlock, qbftBlock.besuBlock)
&& Objects.equals(qbftBlockHeader, qbftBlock.qbftBlockHeader);
} }
@Override @Override
public int hashCode() { public int hashCode() {
return Objects.hash(header, block); return Objects.hash(besuBlock, qbftBlockHeader);
} }
} }

View File

@@ -19,6 +19,7 @@ import org.hyperledger.besu.consensus.common.bft.BftExtraData;
import org.hyperledger.besu.consensus.common.bft.BftExtraDataCodec; 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.QbftBlock;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockCreator; import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockCreator;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockHeader;
import org.hyperledger.besu.consensus.qbft.core.types.QbftExtraDataProvider; import org.hyperledger.besu.consensus.qbft.core.types.QbftExtraDataProvider;
import org.hyperledger.besu.crypto.SECPSignature; import org.hyperledger.besu.crypto.SECPSignature;
import org.hyperledger.besu.ethereum.blockcreation.BlockCreator; import org.hyperledger.besu.ethereum.blockcreation.BlockCreator;
@@ -47,8 +48,11 @@ public class QbftBlockCreatorAdaptor implements QbftBlockCreator {
} }
@Override @Override
public QbftBlock createBlock(final long headerTimeStampSeconds, final BlockHeader parentHeader) { public QbftBlock createBlock(
var blockResult = besuBlockCreator.createBlock(headerTimeStampSeconds, parentHeader); final long headerTimeStampSeconds, final QbftBlockHeader parentHeader) {
var blockResult =
besuBlockCreator.createBlock(
headerTimeStampSeconds, BlockUtil.toBesuBlockHeader(parentHeader));
return new QbftBlockAdaptor(blockResult.getBlock()); return new QbftBlockAdaptor(blockResult.getBlock());
} }
@@ -59,8 +63,8 @@ public class QbftBlockCreatorAdaptor implements QbftBlockCreator {
final int roundNumber, final int roundNumber,
final Collection<SECPSignature> commitSeals) { final Collection<SECPSignature> commitSeals) {
final Block besuBlock = BlockUtil.toBesuBlock(block); final Block besuBlock = BlockUtil.toBesuBlock(block);
final BlockHeader initialBesuHeader = besuBlock.getHeader(); final QbftBlockHeader initialHeader = block.getHeader();
final BftExtraData initialExtraData = bftQbftExtraDataProvider.getExtraData(initialBesuHeader); final BftExtraData initialExtraData = bftQbftExtraDataProvider.getExtraData(initialHeader);
final BftExtraData sealedExtraData = final BftExtraData sealedExtraData =
new BftExtraData( new BftExtraData(
@@ -71,7 +75,7 @@ public class QbftBlockCreatorAdaptor implements QbftBlockCreator {
initialExtraData.getValidators()); initialExtraData.getValidators());
final BlockHeader sealedHeader = final BlockHeader sealedHeader =
BlockHeaderBuilder.fromHeader(initialBesuHeader) BlockHeaderBuilder.fromHeader(BlockUtil.toBesuBlockHeader(initialHeader))
.extraData(bftExtraDataCodec.encode(sealedExtraData)) .extraData(bftExtraDataCodec.encode(sealedExtraData))
.blockHeaderFunctions(BftBlockHeaderFunctions.forOnchainBlock(bftExtraDataCodec)) .blockHeaderFunctions(BftBlockHeaderFunctions.forOnchainBlock(bftExtraDataCodec))
.buildBlockHeader(); .buildBlockHeader();

View File

@@ -0,0 +1,42 @@
/*
* 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.BftBlockHashing;
import org.hyperledger.besu.consensus.common.bft.BftExtraData;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockHashing;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockHeader;
import org.hyperledger.besu.datatypes.Hash;
/** Adaptor to allow a {linkBftBlockHashing} to be used as a {@link QbftBlockHashing}. */
public class QbftBlockHashingAdaptor implements QbftBlockHashing {
private final BftBlockHashing bftBlockHashing;
/**
* Construct a new Qbft BlockHasher
*
* @param bftBlockHashing the BFT BlockHashing
*/
public QbftBlockHashingAdaptor(final BftBlockHashing bftBlockHashing) {
this.bftBlockHashing = bftBlockHashing;
}
@Override
public Hash calculateDataHashForCommittedSeal(
final QbftBlockHeader header, final BftExtraData extraData) {
return bftBlockHashing.calculateDataHashForCommittedSeal(
BlockUtil.toBesuBlockHeader(header), extraData);
}
}

View File

@@ -0,0 +1,77 @@
/*
* 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.QbftBlockHeader;
import org.hyperledger.besu.datatypes.Address;
import org.hyperledger.besu.datatypes.Hash;
import org.hyperledger.besu.ethereum.core.BlockHeader;
import java.util.Objects;
/** Adaptor class to allow a {@link BlockHeader} to be used as a {@link QbftBlockHeader}. */
public class QbftBlockHeaderAdaptor implements QbftBlockHeader {
private final BlockHeader blockHeader;
/**
* Construct a new QbftBlockHeader
*
* @param blockHeader the Besu block header
*/
public QbftBlockHeaderAdaptor(final BlockHeader blockHeader) {
this.blockHeader = blockHeader;
}
@Override
public long getNumber() {
return blockHeader.getNumber();
}
@Override
public long getTimestamp() {
return blockHeader.getTimestamp();
}
@Override
public Address getCoinbase() {
return blockHeader.getCoinbase();
}
@Override
public Hash getHash() {
return blockHeader.getHash();
}
/**
* Returns the Besu block header.
*
* @return the Besu block header.
*/
public BlockHeader getBesuBlockHeader() {
return blockHeader;
}
@Override
public boolean equals(final Object o) {
if (!(o instanceof QbftBlockHeaderAdaptor that)) return false;
return Objects.equals(blockHeader, that.blockHeader);
}
@Override
public int hashCode() {
return Objects.hashCode(blockHeader);
}
}

View File

@@ -0,0 +1,45 @@
/*
* 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.QbftBlockHeader;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockchain;
import org.hyperledger.besu.ethereum.chain.Blockchain;
import org.hyperledger.besu.ethereum.core.BlockHeader;
/** Adaptor class to allow a {@link Blockchain} to be used as a {@link QbftBlockchain}. */
public class QbftBlockchainAdaptor implements QbftBlockchain {
private final Blockchain blockchain;
/**
* Create a new instance of the adaptor.
*
* @param blockchain The {@link Blockchain} to adapt.
*/
public QbftBlockchainAdaptor(final Blockchain blockchain) {
this.blockchain = blockchain;
}
@Override
public QbftBlockHeader getChainHeadHeader() {
BlockHeader chainHeadHeader = blockchain.getChainHeadHeader();
return new QbftBlockHeaderAdaptor(chainHeadHeader);
}
@Override
public long getChainHeadBlockNumber() {
return blockchain.getChainHeadBlockNumber();
}
}

View File

@@ -16,8 +16,8 @@ package org.hyperledger.besu.consensus.qbft.adaptor;
import org.hyperledger.besu.consensus.common.bft.BftExtraData; import org.hyperledger.besu.consensus.common.bft.BftExtraData;
import org.hyperledger.besu.consensus.common.bft.BftExtraDataCodec; import org.hyperledger.besu.consensus.common.bft.BftExtraDataCodec;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockHeader;
import org.hyperledger.besu.consensus.qbft.core.types.QbftExtraDataProvider; 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}. * Adaptor class to allow a {@link BftExtraDataCodec} to be used as a {@link QbftExtraDataProvider}.
@@ -35,7 +35,7 @@ public class QbftExtraDataProviderAdaptor implements QbftExtraDataProvider {
} }
@Override @Override
public BftExtraData getExtraData(final BlockHeader header) { public BftExtraData getExtraData(final QbftBlockHeader header) {
return bftExtraDataCodec.decode(header); return bftExtraDataCodec.decode(BlockUtil.toBesuBlockHeader(header));
} }
} }

View File

@@ -14,10 +14,10 @@
*/ */
package org.hyperledger.besu.consensus.qbft.adaptor; package org.hyperledger.besu.consensus.qbft.adaptor;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockHeader;
import org.hyperledger.besu.consensus.qbft.core.types.QbftProtocolSchedule; 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.types.QbftProtocolSpec;
import org.hyperledger.besu.ethereum.ProtocolContext; 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.ProtocolSchedule;
import org.hyperledger.besu.ethereum.mainnet.ProtocolSpec; import org.hyperledger.besu.ethereum.mainnet.ProtocolSpec;
@@ -42,8 +42,9 @@ public class QbftProtocolScheduleAdaptor implements QbftProtocolSchedule {
} }
@Override @Override
public QbftProtocolSpec getByBlockHeader(final BlockHeader header) { public QbftProtocolSpec getByBlockHeader(final QbftBlockHeader header) {
final ProtocolSpec protocolSpec = besuProtocolSchedule.getByBlockHeader(header); final ProtocolSpec protocolSpec =
besuProtocolSchedule.getByBlockHeader(BlockUtil.toBesuBlockHeader(header));
return new QbftProtocolSpecAdaptor(protocolSpec, context); return new QbftProtocolSpecAdaptor(protocolSpec, context);
} }
} }

View File

@@ -0,0 +1,43 @@
/*
* 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.QbftBlockHeader;
import org.hyperledger.besu.consensus.qbft.core.types.QbftValidatorModeTransitionLogger;
import org.hyperledger.besu.consensus.qbft.validator.ValidatorModeTransitionLogger;
/**
* Adaptor class to allow the {@link ValidatorModeTransitionLogger} to be used as a {@link
* QbftValidatorModeTransitionLogger}.
*/
public class QbftValidatorModeTransitionLoggerAdaptor implements QbftValidatorModeTransitionLogger {
private final ValidatorModeTransitionLogger validatorModeTransitionLogger;
/**
* Create a new instance of the adaptor.
*
* @param validatorModeTransitionLogger the {@link ValidatorModeTransitionLogger} to adapt.
*/
public QbftValidatorModeTransitionLoggerAdaptor(
final ValidatorModeTransitionLogger validatorModeTransitionLogger) {
this.validatorModeTransitionLogger = validatorModeTransitionLogger;
}
@Override
public void logTransitionChange(final QbftBlockHeader parentHeader) {
validatorModeTransitionLogger.logTransitionChange(BlockUtil.toBesuBlockHeader(parentHeader));
}
}

View File

@@ -0,0 +1,62 @@
/*
* 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.validator.ValidatorProvider;
import org.hyperledger.besu.consensus.common.validator.VoteProvider;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockHeader;
import org.hyperledger.besu.consensus.qbft.core.types.QbftValidatorProvider;
import org.hyperledger.besu.datatypes.Address;
import java.util.Collection;
import java.util.Optional;
/**
* Adaptor class to allow the {@link ValidatorProvider} to be used as a {@link
* QbftValidatorProvider}.
*/
public class QbftValidatorProviderAdaptor implements QbftValidatorProvider {
private final ValidatorProvider validatorProvider;
/**
* Create a new instance of the adaptor.
*
* @param validatorProvider the {@link ValidatorProvider} to adapt.
*/
public QbftValidatorProviderAdaptor(final ValidatorProvider validatorProvider) {
this.validatorProvider = validatorProvider;
}
@Override
public Collection<Address> getValidatorsAtHead() {
return validatorProvider.getValidatorsAtHead();
}
@Override
public Collection<Address> getValidatorsAfterBlock(final QbftBlockHeader header) {
return validatorProvider.getValidatorsAfterBlock(BlockUtil.toBesuBlockHeader(header));
}
@Override
public Collection<Address> getValidatorsForBlock(final QbftBlockHeader header) {
return validatorProvider.getValidatorsForBlock(BlockUtil.toBesuBlockHeader(header));
}
@Override
public Optional<VoteProvider> getVoteProviderAtHead() {
return validatorProvider.getVoteProviderAtHead();
}
}

View File

@@ -12,7 +12,7 @@
* *
* SPDX-License-Identifier: Apache-2.0 * SPDX-License-Identifier: Apache-2.0
*/ */
package org.hyperledger.besu.consensus.qbft.core.validator; package org.hyperledger.besu.consensus.qbft.validator;
import org.hyperledger.besu.config.QbftConfigOptions; import org.hyperledger.besu.config.QbftConfigOptions;
import org.hyperledger.besu.consensus.common.ForkSpec; import org.hyperledger.besu.consensus.common.ForkSpec;

View File

@@ -0,0 +1,87 @@
/*
* 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.mockito.ArgumentMatchers.argThat;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;
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.qbft.core.types.QbftEventHandler;
import org.hyperledger.besu.ethereum.core.BlockHeader;
import org.hyperledger.besu.ethereum.core.BlockHeaderTestFixture;
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)
class BftEventHandlerAdaptorTest {
@Mock private QbftEventHandler qbftEventHandler;
@Mock private BftReceivedMessageEvent bftReceivedMessageEvent;
@Mock private BlockTimerExpiry blockTimerExpiry;
@Mock private RoundExpiry roundExpiry;
@Mock private NewChainHead newChainHead;
private BftEventHandlerAdaptor handler;
@BeforeEach
void start() {
handler = new BftEventHandlerAdaptor(qbftEventHandler);
}
@Test
void startDelegatesToQbftEventHandler() {
handler.start();
verify(qbftEventHandler).start();
}
@Test
void handleMessageEventDelegatesToQbftEventHandler() {
handler.handleMessageEvent(bftReceivedMessageEvent);
verify(qbftEventHandler).handleMessageEvent(bftReceivedMessageEvent);
}
@Test
void handleBlockTimerExpiryDelegatesToQbftEventHandler() {
handler.handleBlockTimerExpiry(blockTimerExpiry);
verify(qbftEventHandler).handleBlockTimerExpiry(blockTimerExpiry);
}
@Test
void handleRoundExpiryDelegatesToQbftEventHandler() {
handler.handleRoundExpiry(roundExpiry);
verify(qbftEventHandler).handleRoundExpiry(roundExpiry);
}
@Test
void handleNewBlockEventDelegatesToQbftEventHandler() {
BlockHeader header = new BlockHeaderTestFixture().buildHeader();
when(newChainHead.getNewChainHeadHeader()).thenReturn(header);
handler.handleNewBlockEvent(newChainHead);
verify(qbftEventHandler)
.handleNewBlockEvent(
argThat(
argument ->
((QbftBlockHeaderAdaptor) argument.newChainHeadHeader())
.getBesuBlockHeader()
.equals(header)));
}
}

View File

@@ -22,6 +22,7 @@ import org.hyperledger.besu.consensus.common.bft.BftExtraData;
import org.hyperledger.besu.consensus.common.bft.Vote; import org.hyperledger.besu.consensus.common.bft.Vote;
import org.hyperledger.besu.consensus.qbft.QbftExtraDataCodec; import org.hyperledger.besu.consensus.qbft.QbftExtraDataCodec;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlock; import org.hyperledger.besu.consensus.qbft.core.types.QbftBlock;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockHeader;
import org.hyperledger.besu.consensus.qbft.core.types.QbftExtraDataProvider; import org.hyperledger.besu.consensus.qbft.core.types.QbftExtraDataProvider;
import org.hyperledger.besu.crypto.SECPSignature; import org.hyperledger.besu.crypto.SECPSignature;
import org.hyperledger.besu.datatypes.Address; import org.hyperledger.besu.datatypes.Address;
@@ -44,14 +45,16 @@ import org.mockito.junit.jupiter.MockitoExtension;
@ExtendWith(MockitoExtension.class) @ExtendWith(MockitoExtension.class)
class QbftBlockCreatorAdaptorTest { class QbftBlockCreatorAdaptorTest {
@Mock private BlockCreator blockCreator; @Mock private BlockCreator blockCreator;
@Mock private BlockHeader parentHeader;
@Mock private Block besuBlock; @Mock private Block besuBlock;
@Mock private QbftExtraDataProvider qbftExtraDataProvider; @Mock private QbftExtraDataProvider qbftExtraDataProvider;
private final QbftExtraDataCodec qbftExtraDataCodec = new QbftExtraDataCodec(); private final QbftExtraDataCodec qbftExtraDataCodec = new QbftExtraDataCodec();
@Test @Test
void createsBlockUsingBesuBlockCreator() { void createsBlockUsingBesuBlockCreator() {
when(blockCreator.createBlock(10, parentHeader)) BlockHeader besuParentHeader = new BlockHeaderTestFixture().buildHeader();
QbftBlockHeader parentHeader = new QbftBlockHeaderAdaptor(besuParentHeader);
when(blockCreator.createBlock(10, besuParentHeader))
.thenReturn(new BlockCreator.BlockCreationResult(besuBlock, null, null)); .thenReturn(new BlockCreator.BlockCreationResult(besuBlock, null, null));
QbftBlockCreatorAdaptor qbftBlockCreator = QbftBlockCreatorAdaptor qbftBlockCreator =
@@ -73,13 +76,15 @@ class QbftBlockCreatorAdaptorTest {
Block besuBlock = new Block(header, BlockBody.empty()); Block besuBlock = new Block(header, BlockBody.empty());
QbftBlock block = new QbftBlockAdaptor(besuBlock); QbftBlock block = new QbftBlockAdaptor(besuBlock);
SECPSignature seal = new SECPSignature(BigInteger.ONE, BigInteger.ONE, (byte) 1); SECPSignature seal = new SECPSignature(BigInteger.ONE, BigInteger.ONE, (byte) 1);
when(qbftExtraDataProvider.getExtraData(header)).thenReturn(bftExtraData); when(qbftExtraDataProvider.getExtraData(new QbftBlockHeaderAdaptor(header)))
.thenReturn(bftExtraData);
QbftBlockCreatorAdaptor qbftBlockCreator = QbftBlockCreatorAdaptor qbftBlockCreator =
new QbftBlockCreatorAdaptor(blockCreator, qbftExtraDataCodec); new QbftBlockCreatorAdaptor(blockCreator, qbftExtraDataCodec);
QbftBlock sealedBlock = QbftBlock sealedBlock =
qbftBlockCreator.createSealedBlock(qbftExtraDataProvider, block, 1, List.of(seal)); qbftBlockCreator.createSealedBlock(qbftExtraDataProvider, block, 1, List.of(seal));
BftExtraData sealedExtraData = qbftExtraDataCodec.decode(sealedBlock.getHeader()); BftExtraData sealedExtraData =
qbftExtraDataCodec.decode(BlockUtil.toBesuBlockHeader(sealedBlock.getHeader()));
assertThat(sealedExtraData.getVanityData()).isEqualTo(Bytes.wrap(new byte[32])); assertThat(sealedExtraData.getVanityData()).isEqualTo(Bytes.wrap(new byte[32]));
assertThat(sealedExtraData.getVote()).contains(Vote.authVote(Address.ZERO)); assertThat(sealedExtraData.getVote()).contains(Vote.authVote(Address.ZERO));
assertThat(sealedExtraData.getValidators()).isEqualTo(List.of(Address.ZERO)); assertThat(sealedExtraData.getValidators()).isEqualTo(List.of(Address.ZERO));

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 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.BftBlockHashing;
import org.hyperledger.besu.consensus.common.bft.BftExtraData;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockHeader;
import org.hyperledger.besu.datatypes.Hash;
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.Mock;
import org.mockito.junit.jupiter.MockitoExtension;
@ExtendWith(MockitoExtension.class)
class QbftBlockHashingAdaptorTest {
@Mock private BftBlockHashing bftBlockHashing;
@Test
void calculatesHashForCommittedSeal() {
BlockHeader header = new BlockHeaderTestFixture().buildHeader();
QbftBlockHeader qbftBlockHeader = new QbftBlockHeaderAdaptor(header);
final BftExtraData bftExtraData =
new BftExtraData(Bytes.wrap(new byte[32]), emptyList(), Optional.empty(), 0, emptyList());
when(bftBlockHashing.calculateDataHashForCommittedSeal(header, bftExtraData))
.thenReturn(Hash.EMPTY);
QbftBlockHashingAdaptor qbftBlockHashingAdaptor = new QbftBlockHashingAdaptor(bftBlockHashing);
Hash hash =
qbftBlockHashingAdaptor.calculateDataHashForCommittedSeal(qbftBlockHeader, bftExtraData);
assertThat(hash).isEqualTo(Hash.EMPTY);
}
}

View File

@@ -0,0 +1,43 @@
/*
* 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.QbftBlockHeader;
import org.hyperledger.besu.datatypes.Address;
import org.hyperledger.besu.ethereum.core.BlockHeader;
import org.hyperledger.besu.ethereum.core.BlockHeaderTestFixture;
import org.junit.jupiter.api.Test;
class QbftBlockHeaderAdaptorTest {
@Test
void adaptsBesuBlockHeader() {
BlockHeader header =
new BlockHeaderTestFixture()
.number(1)
.timestamp(1000L)
.coinbase(Address.ZERO)
.buildHeader();
QbftBlockHeader qbftBlockHeader = new QbftBlockHeaderAdaptor(header);
assertThat(qbftBlockHeader.getNumber()).isEqualTo(1);
assertThat(qbftBlockHeader.getTimestamp()).isEqualTo(1000L);
assertThat(qbftBlockHeader.getHash()).isEqualTo(header.getHash());
assertThat(qbftBlockHeader.getCoinbase()).isEqualTo(Address.ZERO);
}
}

View File

@@ -53,7 +53,8 @@ class QbftBlockInterfaceAdaptorTest {
QbftBlockInterface qbftBlockInterface = new QbftBlockInterfaceAdaptor(bftBlockInterface); QbftBlockInterface qbftBlockInterface = new QbftBlockInterfaceAdaptor(bftBlockInterface);
QbftBlock updatedBlock = QbftBlock updatedBlock =
qbftBlockInterface.replaceRoundInBlock(block, 1, QbftHashMode.COMMITTED_SEAL); qbftBlockInterface.replaceRoundInBlock(block, 1, QbftHashMode.COMMITTED_SEAL);
BftExtraData extraData = qbftExtraDataCodec.decode(updatedBlock.getHeader()); BftExtraData extraData =
qbftExtraDataCodec.decode(BlockUtil.toBesuBlockHeader(updatedBlock.getHeader()));
assertThat(extraData.getRound()).isEqualTo(1); assertThat(extraData.getRound()).isEqualTo(1);
} }
} }

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 static org.assertj.core.api.Assertions.assertThat;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;
import org.hyperledger.besu.ethereum.chain.Blockchain;
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 QbftBlockchainAdaptorTest {
@Mock private Blockchain blockchain;
@Test
void returnsChainHeadBlockNumber() {
when(blockchain.getChainHeadBlockNumber()).thenReturn(1L);
long chainHeadBlockNumber = new QbftBlockchainAdaptor(blockchain).getChainHeadBlockNumber();
assertThat(chainHeadBlockNumber).isEqualTo(1L);
verify(blockchain).getChainHeadBlockNumber();
}
}

View File

@@ -19,6 +19,7 @@ import static org.assertj.core.api.Assertions.assertThat;
import org.hyperledger.besu.consensus.common.bft.BftExtraData; import org.hyperledger.besu.consensus.common.bft.BftExtraData;
import org.hyperledger.besu.consensus.qbft.QbftExtraDataCodec; import org.hyperledger.besu.consensus.qbft.QbftExtraDataCodec;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockHeader;
import org.hyperledger.besu.ethereum.core.BlockHeader; import org.hyperledger.besu.ethereum.core.BlockHeader;
import org.hyperledger.besu.ethereum.core.BlockHeaderTestFixture; import org.hyperledger.besu.ethereum.core.BlockHeaderTestFixture;
@@ -35,12 +36,13 @@ class QbftExtraDataProviderAdaptorTest {
final BftExtraData bftExtraData = final BftExtraData bftExtraData =
new BftExtraData(Bytes.wrap(new byte[32]), emptyList(), Optional.empty(), 0, emptyList()); new BftExtraData(Bytes.wrap(new byte[32]), emptyList(), Optional.empty(), 0, emptyList());
final Bytes encoded = qbftExtraDataCodec.encode(bftExtraData); final Bytes encoded = qbftExtraDataCodec.encode(bftExtraData);
final BlockHeader header = final BlockHeader besuHeader =
new BlockHeaderTestFixture().number(1).extraData(encoded).buildHeader(); new BlockHeaderTestFixture().number(1).extraData(encoded).buildHeader();
final QbftBlockHeader qbftHeader = new QbftBlockHeaderAdaptor(besuHeader);
final QbftExtraDataProviderAdaptor qbftExtraDataProvider = final QbftExtraDataProviderAdaptor qbftExtraDataProvider =
new QbftExtraDataProviderAdaptor(new QbftExtraDataCodec()); new QbftExtraDataProviderAdaptor(new QbftExtraDataCodec());
final BftExtraData retrievedExtraData = qbftExtraDataProvider.getExtraData(header); final BftExtraData retrievedExtraData = qbftExtraDataProvider.getExtraData(qbftHeader);
assertThat(retrievedExtraData).isEqualToComparingFieldByField(bftExtraData); assertThat(retrievedExtraData).isEqualToComparingFieldByField(bftExtraData);
} }
} }

View File

@@ -17,10 +17,12 @@ package org.hyperledger.besu.consensus.qbft.adaptor;
import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThat;
import static org.mockito.Mockito.when; import static org.mockito.Mockito.when;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockHeader;
import org.hyperledger.besu.consensus.qbft.core.types.QbftProtocolSchedule; 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.types.QbftProtocolSpec;
import org.hyperledger.besu.ethereum.ProtocolContext; import org.hyperledger.besu.ethereum.ProtocolContext;
import org.hyperledger.besu.ethereum.core.BlockHeader; import org.hyperledger.besu.ethereum.core.BlockHeader;
import org.hyperledger.besu.ethereum.core.BlockHeaderTestFixture;
import org.hyperledger.besu.ethereum.mainnet.ProtocolSchedule; import org.hyperledger.besu.ethereum.mainnet.ProtocolSchedule;
import org.hyperledger.besu.ethereum.mainnet.ProtocolSpec; import org.hyperledger.besu.ethereum.mainnet.ProtocolSpec;
@@ -34,15 +36,17 @@ class QbftProtocolScheduleAdaptorTest {
@Mock private ProtocolSchedule besuProtocolSchedule; @Mock private ProtocolSchedule besuProtocolSchedule;
@Mock private ProtocolSpec besuProtocolSpec; @Mock private ProtocolSpec besuProtocolSpec;
@Mock private ProtocolContext besuProtocolContext; @Mock private ProtocolContext besuProtocolContext;
@Mock private BlockHeader blockHeader;
@Test @Test
void createsAProtocolSpecUsingBesuProtocolSpec() { void createsAProtocolSpecUsingBesuProtocolSpec() {
when(besuProtocolSchedule.getByBlockHeader(blockHeader)).thenReturn(besuProtocolSpec); final BlockHeader besuHeader = new BlockHeaderTestFixture().number(1).buildHeader();
final QbftBlockHeader qbftHeader = new QbftBlockHeaderAdaptor(besuHeader);
when(besuProtocolSchedule.getByBlockHeader(besuHeader)).thenReturn(besuProtocolSpec);
final QbftProtocolSchedule qbftProtocolSchedule = final QbftProtocolSchedule qbftProtocolSchedule =
new QbftProtocolScheduleAdaptor(besuProtocolSchedule, besuProtocolContext); new QbftProtocolScheduleAdaptor(besuProtocolSchedule, besuProtocolContext);
final QbftProtocolSpec protocolSpec = qbftProtocolSchedule.getByBlockHeader(blockHeader); final QbftProtocolSpec protocolSpec = qbftProtocolSchedule.getByBlockHeader(qbftHeader);
assertThat(protocolSpec).hasFieldOrPropertyWithValue("besuProtocolSpec", besuProtocolSpec); assertThat(protocolSpec).hasFieldOrPropertyWithValue("besuProtocolSpec", besuProtocolSpec);
} }
} }

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 static org.mockito.Mockito.verify;
import org.hyperledger.besu.consensus.qbft.validator.ValidatorModeTransitionLogger;
import org.hyperledger.besu.ethereum.core.BlockHeader;
import org.hyperledger.besu.ethereum.core.BlockHeaderTestFixture;
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 QbftValidatorModeTransitionLoggerAdaptorTest {
@Mock private ValidatorModeTransitionLogger logger;
@Test
void logTransitionChangeDelegatesToLogger() {
BlockHeader header = new BlockHeaderTestFixture().buildHeader();
QbftBlockHeaderAdaptor qbftHeader = new QbftBlockHeaderAdaptor(header);
var adaptor = new QbftValidatorModeTransitionLoggerAdaptor(logger);
adaptor.logTransitionChange(qbftHeader);
verify(logger).logTransitionChange(header);
}
}

View File

@@ -0,0 +1,87 @@
/*
* 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.common.validator.ValidatorProvider;
import org.hyperledger.besu.consensus.common.validator.VoteProvider;
import org.hyperledger.besu.consensus.qbft.core.types.QbftBlockHeader;
import org.hyperledger.besu.datatypes.Address;
import org.hyperledger.besu.ethereum.core.BlockHeader;
import org.hyperledger.besu.ethereum.core.BlockHeaderTestFixture;
import java.util.List;
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 QbftValidatorProviderAdaptorTest {
@Mock private ValidatorProvider validatorProvider;
@Mock private VoteProvider voteProvider;
@Test
void returnsValidatorsAtHead() {
List<Address> validatorsAtHead =
List.of(Address.fromHexString("0x095e7baea6a6c7c4c2dfeb977efac326af552d87"));
when(validatorProvider.getValidatorsAtHead()).thenReturn(validatorsAtHead);
QbftValidatorProviderAdaptor qbftValidatorProviderAdaptor =
new QbftValidatorProviderAdaptor(validatorProvider);
assertThat(qbftValidatorProviderAdaptor.getValidatorsAtHead()).isEqualTo(validatorsAtHead);
}
@Test
void returnsValidatorsAfterBlock() {
BlockHeader header = new BlockHeaderTestFixture().buildHeader();
QbftBlockHeader qbftBlockHeader = new QbftBlockHeaderAdaptor(header);
List<Address> validatorsAfterBlock =
List.of(Address.fromHexString("0x095e7baea6a6c7c4c2dfeb977efac326af552d87"));
when(validatorProvider.getValidatorsAfterBlock(header)).thenReturn(validatorsAfterBlock);
QbftValidatorProviderAdaptor qbftValidatorProviderAdaptor =
new QbftValidatorProviderAdaptor(validatorProvider);
assertThat(qbftValidatorProviderAdaptor.getValidatorsAfterBlock(qbftBlockHeader))
.isEqualTo(validatorsAfterBlock);
}
@Test
void returnsValidatorsForBlock() {
BlockHeader header = new BlockHeaderTestFixture().buildHeader();
QbftBlockHeader qbftBlockHeader = new QbftBlockHeaderAdaptor(header);
List<Address> validatorsForBlock =
List.of(Address.fromHexString("0x095e7baea6a6c7c4c2dfeb977efac326af552d87"));
when(validatorProvider.getValidatorsForBlock(header)).thenReturn(validatorsForBlock);
QbftValidatorProviderAdaptor qbftValidatorProviderAdaptor =
new QbftValidatorProviderAdaptor(validatorProvider);
assertThat(qbftValidatorProviderAdaptor.getValidatorsForBlock(qbftBlockHeader))
.isEqualTo(validatorsForBlock);
}
@Test
void returnsVoteProviderAtHead() {
when(validatorProvider.getVoteProviderAtHead()).thenReturn(Optional.of(voteProvider));
QbftValidatorProviderAdaptor qbftValidatorProviderAdaptor =
new QbftValidatorProviderAdaptor(validatorProvider);
assertThat(qbftValidatorProviderAdaptor.getVoteProviderAtHead()).contains(voteProvider);
}
}

View File

@@ -25,7 +25,6 @@ import org.hyperledger.besu.config.QbftConfigOptions;
import org.hyperledger.besu.consensus.common.ForkSpec; import org.hyperledger.besu.consensus.common.ForkSpec;
import org.hyperledger.besu.consensus.common.ForksSchedule; import org.hyperledger.besu.consensus.common.ForksSchedule;
import org.hyperledger.besu.consensus.qbft.MutableQbftConfigOptions; import org.hyperledger.besu.consensus.qbft.MutableQbftConfigOptions;
import org.hyperledger.besu.consensus.qbft.core.validator.ValidatorModeTransitionLogger;
import org.hyperledger.besu.ethereum.core.BlockHeader; import org.hyperledger.besu.ethereum.core.BlockHeader;
import org.hyperledger.besu.ethereum.core.BlockHeaderTestFixture; import org.hyperledger.besu.ethereum.core.BlockHeaderTestFixture;

View File

@@ -48,6 +48,7 @@ dependencies {
api project(':consensus:ibft') api project(':consensus:ibft')
api project(':consensus:merge') api project(':consensus:merge')
api project(':consensus:qbft') api project(':consensus:qbft')
api project(':consensus:qbft-core')
api project(':crypto:algorithms') api project(':crypto:algorithms')
api project(':crypto:services') api project(':crypto:services')
api project(':datatypes') api project(':datatypes')