Bft block period transition (#2902)

Allow block period to be configured using transitions for IBFT2 and QBFT

Signed-off-by: Jason Frame <jasonwframe@gmail.com>
pull/2918/head
Jason Frame 3 years ago committed by GitHub
parent 73ddc6aec2
commit 0e0d67dbe5
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
  1. 2
      besu/src/main/java/org/hyperledger/besu/controller/IbftBesuControllerBuilder.java
  2. 2
      besu/src/main/java/org/hyperledger/besu/controller/QbftBesuControllerBuilder.java
  3. 16
      consensus/common/src/main/java/org/hyperledger/besu/consensus/common/bft/BlockTimer.java
  4. 27
      consensus/common/src/test-support/java/org/hyperledger/besu/consensus/common/bft/inttest/TestTransitions.java
  5. 61
      consensus/common/src/test/java/org/hyperledger/besu/consensus/common/bft/BlockTimerTest.java
  6. 19
      consensus/ibft/src/integration-test/java/org/hyperledger/besu/consensus/ibft/support/TestContextBuilder.java
  7. 76
      consensus/ibft/src/integration-test/java/org/hyperledger/besu/consensus/ibft/tests/TransitionsTest.java
  8. 103
      consensus/ibft/src/test/java/org/hyperledger/besu/consensus/ibft/IbftBlockHeaderUtils.java
  9. 124
      consensus/ibft/src/test/java/org/hyperledger/besu/consensus/ibft/IbftProtocolScheduleTest.java
  10. 6
      consensus/qbft/src/integration-test/java/org/hyperledger/besu/consensus/qbft/support/TestContextBuilder.java
  11. 77
      consensus/qbft/src/integration-test/java/org/hyperledger/besu/consensus/qbft/test/TransitionsTest.java
  12. 16
      consensus/qbft/src/integration-test/resources/log4j2.xml
  13. 4
      consensus/qbft/src/test/java/org/hyperledger/besu/consensus/qbft/QbftBlockHeaderUtils.java
  14. 6
      testutil/src/main/java/org/hyperledger/besu/testutil/TestClock.java

@ -163,7 +163,7 @@ public class IbftBesuControllerBuilder extends BftBesuControllerBuilder {
proposerSelector,
uniqueMessageMulticaster,
new RoundTimer(bftEventQueue, bftConfig.getRequestTimeoutSeconds(), bftExecutors),
new BlockTimer(bftEventQueue, bftConfig.getBlockPeriodSeconds(), bftExecutors, clock),
new BlockTimer(bftEventQueue, bftForksSchedule, bftExecutors, clock),
blockCreatorFactory,
clock);

@ -203,7 +203,7 @@ public class QbftBesuControllerBuilder extends BftBesuControllerBuilder {
proposerSelector,
uniqueMessageMulticaster,
new RoundTimer(bftEventQueue, qbftConfig.getRequestTimeoutSeconds(), bftExecutors),
new BlockTimer(bftEventQueue, qbftConfig.getBlockPeriodSeconds(), bftExecutors, clock),
new BlockTimer(bftEventQueue, qbftForksSchedule, bftExecutors, clock),
blockCreatorFactory,
clock);

@ -14,6 +14,7 @@
*/
package org.hyperledger.besu.consensus.common.bft;
import org.hyperledger.besu.config.BftConfigOptions;
import org.hyperledger.besu.consensus.common.bft.events.BlockTimerExpiry;
import org.hyperledger.besu.ethereum.core.BlockHeader;
@ -24,29 +25,30 @@ import java.util.concurrent.TimeUnit;
/** Class for starting and keeping organised block timers */
public class BlockTimer {
private final BftForksSchedule<? extends BftConfigOptions> bftForksSchedule;
private final BftExecutors bftExecutors;
private Optional<ScheduledFuture<?>> currentTimerTask;
private final BftEventQueue queue;
private final long minimumTimeBetweenBlocksMillis;
private final Clock clock;
/**
* Construct a BlockTimer with primed executor service ready to start timers
*
* @param queue The queue in which to put block expiry events
* @param minimumTimeBetweenBlocksSeconds Minimum timestamp difference between blocks
* @param bftForksSchedule Bft fork schedule that contains block period seconds
* @param bftExecutors Executor services that timers can be scheduled with
* @param clock System clock
*/
public BlockTimer(
final BftEventQueue queue,
final long minimumTimeBetweenBlocksSeconds,
final BftForksSchedule<? extends BftConfigOptions> bftForksSchedule,
final BftExecutors bftExecutors,
final Clock clock) {
this.queue = queue;
this.bftForksSchedule = bftForksSchedule;
this.bftExecutors = bftExecutors;
this.currentTimerTask = Optional.empty();
this.minimumTimeBetweenBlocksMillis = minimumTimeBetweenBlocksSeconds * 1000;
this.clock = clock;
}
@ -78,6 +80,12 @@ public class BlockTimer {
final long now = clock.millis();
// absolute time when the timer is supposed to expire
final int blockPeriodSeconds =
bftForksSchedule
.getFork(round.getSequenceNumber())
.getConfigOptions()
.getBlockPeriodSeconds();
final long minimumTimeBetweenBlocksMillis = blockPeriodSeconds * 1000L;
final long expiryTime = chainHeadHeader.getTimestamp() * 1_000 + minimumTimeBetweenBlocksMillis;
if (expiryTime > now) {

@ -12,25 +12,42 @@
*
* SPDX-License-Identifier: Apache-2.0
*/
package org.hyperledger.besu.consensus.qbft.support;
package org.hyperledger.besu.consensus.common.bft.inttest;
import org.hyperledger.besu.config.BftFork;
import org.hyperledger.besu.config.JsonUtil;
import org.hyperledger.besu.config.QbftFork;
import org.hyperledger.besu.config.TransitionsConfigOptions;
import java.util.Collections;
import java.util.List;
public class TestTransitions extends TransitionsConfigOptions {
private final List<QbftFork> forks;
private final List<QbftFork> qbftForks;
private final List<BftFork> ibftForks;
public TestTransitions(final List<QbftFork> forks) {
public static TestTransitions createQbftTestTransitions(final List<QbftFork> qbftForks) {
return new TestTransitions(Collections.emptyList(), qbftForks);
}
public static TestTransitions createIbftTestTransitions(final List<BftFork> bftForks) {
return new TestTransitions(bftForks, Collections.emptyList());
}
public TestTransitions(final List<BftFork> ibftForks, final List<QbftFork> qbftForks) {
super(JsonUtil.createEmptyObjectNode());
this.forks = forks;
this.ibftForks = ibftForks;
this.qbftForks = qbftForks;
}
@Override
public List<QbftFork> getQbftForks() {
return forks;
return qbftForks;
}
@Override
public List<BftFork> getIbftForks() {
return ibftForks;
}
}

@ -24,6 +24,8 @@ import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;
import org.hyperledger.besu.config.BftConfigOptions;
import org.hyperledger.besu.config.JsonBftConfigOptions;
import org.hyperledger.besu.consensus.common.bft.events.BftEvent;
import org.hyperledger.besu.consensus.common.bft.events.BlockTimerExpiry;
import org.hyperledger.besu.ethereum.core.BlockHeader;
@ -46,17 +48,20 @@ public class BlockTimerTest {
private BftExecutors bftExecutors;
private BftEventQueue mockQueue;
private Clock mockClock;
private BftForksSchedule<BftConfigOptions> mockForksSchedule;
@Before
@SuppressWarnings("unchecked")
public void initialise() {
bftExecutors = mock(BftExecutors.class);
mockQueue = mock(BftEventQueue.class);
mockClock = mock(Clock.class);
mockForksSchedule = mock(BftForksSchedule.class);
}
@Test
public void cancelTimerCancelsWhenNoTimer() {
final BlockTimer timer = new BlockTimer(mockQueue, 15, bftExecutors, mockClock);
final BlockTimer timer = new BlockTimer(mockQueue, mockForksSchedule, bftExecutors, mockClock);
// Starts with nothing running
assertThat(timer.isRunning()).isFalse();
// cancel shouldn't die if there's nothing running
@ -67,13 +72,15 @@ public class BlockTimerTest {
@Test
public void startTimerSchedulesCorrectlyWhenExpiryIsInTheFuture() {
final long MINIMAL_TIME_BETWEEN_BLOCKS_SECONDS = 15;
final int MINIMAL_TIME_BETWEEN_BLOCKS_SECONDS = 15;
final long NOW_MILLIS = 505_000L;
final long BLOCK_TIME_STAMP = 500L;
final long EXPECTED_DELAY = 10_000L;
final BlockTimer timer =
new BlockTimer(mockQueue, MINIMAL_TIME_BETWEEN_BLOCKS_SECONDS, bftExecutors, mockClock);
when(mockForksSchedule.getFork(anyLong()))
.thenReturn(new BftForkSpec<>(0, createBftFork(MINIMAL_TIME_BETWEEN_BLOCKS_SECONDS)));
final BlockTimer timer = new BlockTimer(mockQueue, mockForksSchedule, bftExecutors, mockClock);
when(mockClock.millis()).thenReturn(NOW_MILLIS);
@ -94,11 +101,13 @@ public class BlockTimerTest {
@Test
public void aBlockTimerExpiryEventIsAddedToTheQueueOnExpiry() throws InterruptedException {
final long MINIMAL_TIME_BETWEEN_BLOCKS_SECONDS = 1;
final int MINIMAL_TIME_BETWEEN_BLOCKS_SECONDS = 1;
final long NOW_MILLIS = 300_500L;
final long BLOCK_TIME_STAMP = 300;
final long EXPECTED_DELAY = 500;
when(mockForksSchedule.getFork(anyLong()))
.thenReturn(new BftForkSpec<>(0, createBftFork(MINIMAL_TIME_BETWEEN_BLOCKS_SECONDS)));
when(mockClock.millis()).thenReturn(NOW_MILLIS);
final BlockHeader header =
@ -112,8 +121,7 @@ public class BlockTimerTest {
.thenReturn(mockedFuture);
final BftEventQueue eventQueue = new BftEventQueue(1000);
final BlockTimer timer =
new BlockTimer(eventQueue, MINIMAL_TIME_BETWEEN_BLOCKS_SECONDS, bftExecutors, mockClock);
final BlockTimer timer = new BlockTimer(eventQueue, mockForksSchedule, bftExecutors, mockClock);
timer.startTimer(round, header);
// Verify that the event will not be added to the queue immediately
@ -137,12 +145,14 @@ public class BlockTimerTest {
@Test
public void eventIsImmediatelyAddedToTheQueueIfAbsoluteExpiryIsEqualToNow() {
final long MINIMAL_TIME_BETWEEN_BLOCKS_SECONDS = 15;
final int MINIMAL_TIME_BETWEEN_BLOCKS_SECONDS = 15;
final long NOW_MILLIS = 515_000L;
final long BLOCK_TIME_STAMP = 500;
final BlockTimer timer =
new BlockTimer(mockQueue, MINIMAL_TIME_BETWEEN_BLOCKS_SECONDS, bftExecutors, mockClock);
when(mockForksSchedule.getFork(anyLong()))
.thenReturn(new BftForkSpec<>(0, createBftFork(MINIMAL_TIME_BETWEEN_BLOCKS_SECONDS)));
final BlockTimer timer = new BlockTimer(mockQueue, mockForksSchedule, bftExecutors, mockClock);
when(mockClock.millis()).thenReturn(NOW_MILLIS);
@ -164,12 +174,14 @@ public class BlockTimerTest {
@Test
public void eventIsImmediatelyAddedToTheQueueIfAbsoluteExpiryIsInThePast() {
final long MINIMAL_TIME_BETWEEN_BLOCKS_SECONDS = 15;
final int MINIMAL_TIME_BETWEEN_BLOCKS_SECONDS = 15;
final long NOW_MILLIS = 520_000L;
final long BLOCK_TIME_STAMP = 500L;
final BlockTimer timer =
new BlockTimer(mockQueue, MINIMAL_TIME_BETWEEN_BLOCKS_SECONDS, bftExecutors, mockClock);
when(mockForksSchedule.getFork(anyLong()))
.thenReturn(new BftForkSpec<>(0, createBftFork(MINIMAL_TIME_BETWEEN_BLOCKS_SECONDS)));
final BlockTimer timer = new BlockTimer(mockQueue, mockForksSchedule, bftExecutors, mockClock);
when(mockClock.millis()).thenReturn(NOW_MILLIS);
@ -191,12 +203,14 @@ public class BlockTimerTest {
@Test
public void startTimerCancelsExistingTimer() {
final long MINIMAL_TIME_BETWEEN_BLOCKS_SECONDS = 15;
final int MINIMAL_TIME_BETWEEN_BLOCKS_SECONDS = 15;
final long NOW_MILLIS = 500_000L;
final long BLOCK_TIME_STAMP = 500L;
final BlockTimer timer =
new BlockTimer(mockQueue, MINIMAL_TIME_BETWEEN_BLOCKS_SECONDS, bftExecutors, mockClock);
when(mockForksSchedule.getFork(anyLong()))
.thenReturn(new BftForkSpec<>(0, createBftFork(MINIMAL_TIME_BETWEEN_BLOCKS_SECONDS)));
final BlockTimer timer = new BlockTimer(mockQueue, mockForksSchedule, bftExecutors, mockClock);
when(mockClock.millis()).thenReturn(NOW_MILLIS);
@ -217,12 +231,14 @@ public class BlockTimerTest {
@Test
public void runningFollowsTheStateOfTheTimer() {
final long MINIMAL_TIME_BETWEEN_BLOCKS_SECONDS = 15;
final int MINIMAL_TIME_BETWEEN_BLOCKS_SECONDS = 15;
final long NOW_MILLIS = 500_000L;
final long BLOCK_TIME_STAMP = 500L;
final BlockTimer timer =
new BlockTimer(mockQueue, MINIMAL_TIME_BETWEEN_BLOCKS_SECONDS, bftExecutors, mockClock);
when(mockForksSchedule.getFork(anyLong()))
.thenReturn(new BftForkSpec<>(0, createBftFork(MINIMAL_TIME_BETWEEN_BLOCKS_SECONDS)));
final BlockTimer timer = new BlockTimer(mockQueue, mockForksSchedule, bftExecutors, mockClock);
when(mockClock.millis()).thenReturn(NOW_MILLIS);
@ -241,4 +257,11 @@ public class BlockTimerTest {
when(mockedFuture.isDone()).thenReturn(true);
assertThat(timer.isRunning()).isFalse();
}
private BftConfigOptions createBftFork(final int blockPeriodSeconds) {
final MutableBftConfigOptions bftConfigOptions =
new MutableBftConfigOptions(JsonBftConfigOptions.DEFAULT);
bftConfigOptions.setBlockPeriodSeconds(blockPeriodSeconds);
return bftConfigOptions;
}
}

@ -20,6 +20,7 @@ import static org.hyperledger.besu.ethereum.core.InMemoryKeyValueStorageProvider
import static org.mockito.Mockito.mock;
import org.hyperledger.besu.config.BftConfigOptions;
import org.hyperledger.besu.config.BftFork;
import org.hyperledger.besu.config.StubGenesisConfigOptions;
import org.hyperledger.besu.consensus.common.EpochManager;
import org.hyperledger.besu.consensus.common.bft.BftBlockHeaderFunctions;
@ -44,6 +45,7 @@ import org.hyperledger.besu.consensus.common.bft.inttest.NetworkLayout;
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.StubbedSynchronizerUpdater;
import org.hyperledger.besu.consensus.common.bft.inttest.TestTransitions;
import org.hyperledger.besu.consensus.common.bft.statemachine.BftEventHandler;
import org.hyperledger.besu.consensus.common.bft.statemachine.BftFinalState;
import org.hyperledger.besu.consensus.common.bft.statemachine.FutureMessageBuffer;
@ -158,6 +160,7 @@ public class TestContextBuilder {
private int validatorCount = 4;
private int indexOfFirstLocallyProposedBlock = 0; // Meaning first block is from remote peer.
private boolean useGossip = false;
private List<BftFork> bftForks = Collections.emptyList();
private static final IbftExtraDataCodec IBFT_EXTRA_DATA_ENCODER = new IbftExtraDataCodec();
public TestContextBuilder clock(final Clock clock) {
@ -165,7 +168,7 @@ public class TestContextBuilder {
return this;
}
public TestContextBuilder ibftEventQueue(final BftEventQueue bftEventQueue) {
public TestContextBuilder eventQueue(final BftEventQueue bftEventQueue) {
this.bftEventQueue = bftEventQueue;
return this;
}
@ -186,6 +189,11 @@ public class TestContextBuilder {
return this;
}
public TestContextBuilder bftForks(final List<BftFork> bftForks) {
this.bftForks = bftForks;
return this;
}
public TestContext build() {
final NetworkLayout networkNodes =
NetworkLayout.createNetworkLayout(validatorCount, indexOfFirstLocallyProposedBlock);
@ -213,7 +221,8 @@ public class TestContextBuilder {
clock,
bftEventQueue,
gossiper,
synchronizerUpdater);
synchronizerUpdater,
bftForks);
// Add each networkNode to the Multicaster (such that each can receive msgs from local node).
// NOTE: the remotePeers needs to be ordered based on Address (as this is used to determine
@ -281,7 +290,8 @@ public class TestContextBuilder {
final Clock clock,
final BftEventQueue bftEventQueue,
final Gossiper gossiper,
final SynchronizerUpdater synchronizerUpdater) {
final SynchronizerUpdater synchronizerUpdater,
final List<BftFork> bftForks) {
final WorldStateArchive worldStateArchive = createInMemoryWorldStateArchive();
@ -295,6 +305,7 @@ public class TestContextBuilder {
final StubGenesisConfigOptions genesisConfigOptions = new StubGenesisConfigOptions();
genesisConfigOptions.byzantiumBlock(0);
genesisConfigOptions.transitions(TestTransitions.createIbftTestTransitions(bftForks));
final BftForksSchedule<BftConfigOptions> forksSchedule =
IbftForksSchedulesFactory.create(genesisConfigOptions);
@ -352,7 +363,7 @@ public class TestContextBuilder {
proposerSelector,
multicaster,
new RoundTimer(bftEventQueue, ROUND_TIMER_SEC, bftExecutors),
new BlockTimer(bftEventQueue, BLOCK_TIMER_SEC, bftExecutors, TestClock.fixed()),
new BlockTimer(bftEventQueue, forksSchedule, bftExecutors, TestClock.fixed()),
blockCreatorFactory,
clock);

@ -0,0 +1,76 @@
/*
* Copyright Hyperledger Besu Contributors.
*
* Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
* an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
* specific language governing permissions and limitations under the License.
*
* SPDX-License-Identifier: Apache-2.0
*/
package org.hyperledger.besu.consensus.ibft.tests;
import static org.assertj.core.api.Assertions.assertThat;
import org.hyperledger.besu.config.BftFork;
import org.hyperledger.besu.config.JsonUtil;
import org.hyperledger.besu.consensus.common.bft.BftEventQueue;
import org.hyperledger.besu.consensus.common.bft.events.NewChainHead;
import org.hyperledger.besu.consensus.ibft.support.TestContext;
import org.hyperledger.besu.consensus.ibft.support.TestContextBuilder;
import org.hyperledger.besu.ethereum.core.BlockHeader;
import org.hyperledger.besu.testutil.TestClock;
import java.time.Instant;
import java.util.List;
import java.util.Map;
import java.util.concurrent.TimeUnit;
import org.junit.Test;
public class TransitionsTest {
@Test
public void transitionsBlockPeriod() throws InterruptedException {
final TestClock clock = new TestClock(Instant.EPOCH);
final List<BftFork> bftForks =
List.of(
new BftFork(
JsonUtil.objectNodeFromMap(
Map.of(BftFork.FORK_BLOCK_KEY, 1, BftFork.BLOCK_PERIOD_SECONDS_KEY, 10))),
new BftFork(
JsonUtil.objectNodeFromMap(
Map.of(BftFork.FORK_BLOCK_KEY, 2, BftFork.BLOCK_PERIOD_SECONDS_KEY, 20))));
final BftEventQueue bftEventQueue = new BftEventQueue(TestContextBuilder.MESSAGE_QUEUE_LIMIT);
final TestContext context =
new TestContextBuilder()
.indexOfFirstLocallyProposedBlock(0)
.validatorCount(1)
.clock(clock)
.bftForks(bftForks)
.eventQueue(bftEventQueue)
.buildAndStart();
clock.stepMillis(10_000);
context.getEventMultiplexer().handleBftEvent(bftEventQueue.poll(1, TimeUnit.SECONDS));
context
.getController()
.handleNewBlockEvent(new NewChainHead(context.getBlockchain().getChainHeadHeader()));
clock.stepMillis(20_000);
context.getEventMultiplexer().handleBftEvent(bftEventQueue.poll(1, TimeUnit.SECONDS));
final BlockHeader genesisBlock = context.getBlockchain().getBlockHeader(0).get();
final BlockHeader blockHeader1 = context.getBlockchain().getBlockHeader(1).get();
final BlockHeader blockHeader2 = context.getBlockchain().getBlockHeader(2).get();
assertThat(blockHeader1.getTimestamp()).isEqualTo(genesisBlock.getTimestamp() + 10);
assertThat(blockHeader2.getTimestamp()).isEqualTo(blockHeader1.getTimestamp() + 20);
}
}

@ -0,0 +1,103 @@
/*
* Copyright Hyperledger Besu Contributors.
*
* Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
* an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
* specific language governing permissions and limitations under the License.
*
* SPDX-License-Identifier: Apache-2.0
*/
package org.hyperledger.besu.consensus.ibft;
import static java.util.Collections.singletonList;
import org.hyperledger.besu.consensus.common.bft.BftBlockHeaderFunctions;
import org.hyperledger.besu.consensus.common.bft.BftExtraData;
import org.hyperledger.besu.consensus.common.bft.BftExtraDataCodec;
import org.hyperledger.besu.consensus.common.bft.BftExtraDataFixture;
import org.hyperledger.besu.consensus.common.bft.Vote;
import org.hyperledger.besu.crypto.NodeKey;
import org.hyperledger.besu.datatypes.Address;
import org.hyperledger.besu.datatypes.Hash;
import org.hyperledger.besu.ethereum.core.BlockHeader;
import org.hyperledger.besu.ethereum.core.BlockHeaderTestFixture;
import org.hyperledger.besu.ethereum.core.Difficulty;
import org.hyperledger.besu.ethereum.core.Util;
import java.util.List;
import java.util.Optional;
import org.apache.tuweni.bytes.Bytes;
public class IbftBlockHeaderUtils {
private static final int ROUND_NUMBER = 0x2A;
@FunctionalInterface
public interface HeaderModifier {
void update(BlockHeaderTestFixture blockHeaderTestFixture);
}
public static BlockHeaderTestFixture createPresetHeaderBuilder(
final long number,
final NodeKey proposerNodeKey,
final List<Address> validators,
final BlockHeader parent) {
return createPresetHeaderBuilder(number, proposerNodeKey, validators, parent, null);
}
public static BlockHeaderTestFixture createPresetHeaderBuilder(
final long number,
final NodeKey proposerNodeKey,
final List<Address> validators,
final BlockHeader parent,
final HeaderModifier modifier) {
final BlockHeaderTestFixture builder = new BlockHeaderTestFixture();
final IbftExtraDataCodec ibftExtraDataEncoder = new IbftExtraDataCodec();
populateDefaultBlockHeader(
number, proposerNodeKey, parent, modifier, builder, ibftExtraDataEncoder);
final BftExtraData bftExtraData =
BftExtraDataFixture.createExtraData(
builder.buildHeader(),
Bytes.wrap(new byte[BftExtraDataCodec.EXTRA_VANITY_LENGTH]),
Optional.of(Vote.authVote(Address.fromHexString("1"))),
validators,
singletonList(proposerNodeKey),
ROUND_NUMBER,
ibftExtraDataEncoder);
builder.extraData(ibftExtraDataEncoder.encode(bftExtraData));
return builder;
}
private static void populateDefaultBlockHeader(
final long number,
final NodeKey proposerNodeKey,
final BlockHeader parent,
final HeaderModifier modifier,
final BlockHeaderTestFixture builder,
final IbftExtraDataCodec ibftExtraDataEncoder) {
if (parent != null) {
builder.parentHash(parent.getHash());
}
builder.number(number);
builder.gasLimit(5000);
builder.timestamp(6 * number);
builder.mixHash(
Hash.fromHexString("0x63746963616c2062797a616e74696e65206661756c7420746f6c6572616e6365"));
builder.difficulty(Difficulty.ONE);
builder.coinbase(Util.publicKeyToAddress(proposerNodeKey.getPublicKey()));
builder.blockHeaderFunctions(BftBlockHeaderFunctions.forCommittedSeal(ibftExtraDataEncoder));
if (modifier != null) {
modifier.update(builder);
}
}
}

@ -0,0 +1,124 @@
/*
* Copyright Hyperledger Besu Contributors.
*
* Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
* an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
* specific language governing permissions and limitations under the License.
*
* SPDX-License-Identifier: Apache-2.0
*/
package org.hyperledger.besu.consensus.ibft;
import static java.util.Collections.singletonList;
import static org.assertj.core.api.AssertionsForClassTypes.assertThat;
import static org.hyperledger.besu.consensus.common.bft.BftContextBuilder.setupContextWithBftExtraDataEncoder;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
import org.hyperledger.besu.config.BftConfigOptions;
import org.hyperledger.besu.config.GenesisConfigOptions;
import org.hyperledger.besu.config.JsonGenesisConfigOptions;
import org.hyperledger.besu.config.JsonQbftConfigOptions;
import org.hyperledger.besu.config.JsonUtil;
import org.hyperledger.besu.consensus.common.bft.BftContext;
import org.hyperledger.besu.consensus.common.bft.BftExtraData;
import org.hyperledger.besu.consensus.common.bft.BftExtraDataCodec;
import org.hyperledger.besu.consensus.common.bft.BftForkSpec;
import org.hyperledger.besu.consensus.common.bft.BftForksSchedule;
import org.hyperledger.besu.consensus.common.bft.MutableBftConfigOptions;
import org.hyperledger.besu.crypto.NodeKey;
import org.hyperledger.besu.crypto.NodeKeyUtils;
import org.hyperledger.besu.datatypes.Address;
import org.hyperledger.besu.ethereum.ProtocolContext;
import org.hyperledger.besu.ethereum.core.BlockHeader;
import org.hyperledger.besu.ethereum.core.PrivacyParameters;
import org.hyperledger.besu.ethereum.core.Util;
import org.hyperledger.besu.ethereum.mainnet.HeaderValidationMode;
import org.hyperledger.besu.ethereum.mainnet.ProtocolSchedule;
import org.hyperledger.besu.evm.internal.EvmConfiguration;
import java.math.BigInteger;
import java.util.Collection;
import java.util.List;
import org.junit.Before;
import org.junit.Test;
public class IbftProtocolScheduleTest {
private final BftExtraDataCodec bftExtraDataCodec = mock(BftExtraDataCodec.class);
private final BftExtraData bftExtraData = mock(BftExtraData.class);
private final NodeKey proposerNodeKey = NodeKeyUtils.generate();
private final Address proposerAddress = Util.publicKeyToAddress(proposerNodeKey.getPublicKey());
private final List<Address> validators = singletonList(proposerAddress);
@Before
public void setup() {
when(bftExtraDataCodec.decode(any())).thenReturn(bftExtraData);
when(bftExtraData.getValidators()).thenReturn(validators);
}
@Test
public void blockModeTransitionsCreatesBlockModeHeaderValidators() {
final MutableBftConfigOptions arbitraryTransition =
new MutableBftConfigOptions(JsonQbftConfigOptions.DEFAULT);
arbitraryTransition.setBlockRewardWei(BigInteger.ONE);
final BlockHeader parentHeader =
IbftBlockHeaderUtils.createPresetHeaderBuilder(1, proposerNodeKey, validators, null)
.buildHeader();
final BlockHeader blockHeader =
IbftBlockHeaderUtils.createPresetHeaderBuilder(2, proposerNodeKey, validators, parentHeader)
.buildHeader();
final ProtocolSchedule schedule =
createProtocolSchedule(
JsonGenesisConfigOptions.fromJsonObject(JsonUtil.createEmptyObjectNode()),
new BftForkSpec<>(0, JsonQbftConfigOptions.DEFAULT),
List.of(
new BftForkSpec<>(1, arbitraryTransition),
new BftForkSpec<>(2, JsonQbftConfigOptions.DEFAULT)));
assertThat(schedule.streamMilestoneBlocks().count()).isEqualTo(3);
assertThat(validateHeader(schedule, validators, parentHeader, blockHeader, 0)).isTrue();
assertThat(validateHeader(schedule, validators, parentHeader, blockHeader, 1)).isTrue();
assertThat(validateHeader(schedule, validators, parentHeader, blockHeader, 2)).isTrue();
}
private ProtocolSchedule createProtocolSchedule(
final GenesisConfigOptions genesisConfig,
final BftForkSpec<BftConfigOptions> genesisFork,
final List<BftForkSpec<BftConfigOptions>> forks) {
return IbftProtocolSchedule.create(
genesisConfig,
new BftForksSchedule<>(genesisFork, forks),
PrivacyParameters.DEFAULT,
false,
bftExtraDataCodec,
EvmConfiguration.DEFAULT);
}
private boolean validateHeader(
final ProtocolSchedule schedule,
final List<Address> validators,
final BlockHeader parentHeader,
final BlockHeader blockHeader,
final int block) {
return schedule
.getByBlockNumber(block)
.getBlockHeaderValidator()
.validateHeader(
blockHeader, parentHeader, protocolContext(validators), HeaderValidationMode.LIGHT);
}
private ProtocolContext protocolContext(final Collection<Address> validators) {
return new ProtocolContext(
null,
null,
setupContextWithBftExtraDataEncoder(BftContext.class, validators, bftExtraDataCodec));
}
}

@ -49,6 +49,7 @@ import org.hyperledger.besu.consensus.common.bft.inttest.NetworkLayout;
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.StubbedSynchronizerUpdater;
import org.hyperledger.besu.consensus.common.bft.inttest.TestTransitions;
import org.hyperledger.besu.consensus.common.bft.statemachine.BftEventHandler;
import org.hyperledger.besu.consensus.common.bft.statemachine.BftFinalState;
import org.hyperledger.besu.consensus.common.bft.statemachine.FutureMessageBuffer;
@ -398,7 +399,7 @@ public class TestContextBuilder {
genesisConfigOptions.byzantiumBlock(0);
genesisConfigOptions.qbftConfigOptions(
new JsonQbftConfigOptions(JsonUtil.objectNodeFromMap(qbftConfigValues)));
genesisConfigOptions.transitions(new TestTransitions(qbftForks));
genesisConfigOptions.transitions(TestTransitions.createQbftTestTransitions(qbftForks));
genesisConfigOptions.qbftConfigOptions(qbftConfigOptions);
final EpochManager epochManager = new EpochManager(EPOCH_LENGTH);
@ -467,7 +468,7 @@ public class TestContextBuilder {
proposerSelector,
multicaster,
new RoundTimer(bftEventQueue, ROUND_TIMER_SEC, bftExecutors),
new BlockTimer(bftEventQueue, BLOCK_TIMER_SEC, bftExecutors, TestClock.fixed()),
new BlockTimer(bftEventQueue, forksSchedule, bftExecutors, TestClock.fixed()),
blockCreatorFactory,
clock);
@ -524,6 +525,7 @@ public class TestContextBuilder {
private static QbftConfigOptions createGenesisConfig(final boolean useValidatorContract) {
final MutableQbftConfigOptions qbftConfigOptions =
new MutableQbftConfigOptions(JsonQbftConfigOptions.DEFAULT);
qbftConfigOptions.setBlockPeriodSeconds(BLOCK_TIMER_SEC);
if (useValidatorContract) {
qbftConfigOptions.setValidatorContractAddress(
Optional.of(VALIDATOR_CONTRACT_ADDRESS.toHexString()));

@ -0,0 +1,77 @@
/*
* Copyright Hyperledger Besu Contributors.
*
* Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
* an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
* specific language governing permissions and limitations under the License.
*
* SPDX-License-Identifier: Apache-2.0
*/
package org.hyperledger.besu.consensus.qbft.test;
import static org.assertj.core.api.Assertions.assertThat;
import org.hyperledger.besu.config.BftFork;
import org.hyperledger.besu.config.JsonUtil;
import org.hyperledger.besu.config.QbftFork;
import org.hyperledger.besu.consensus.common.bft.BftEventQueue;
import org.hyperledger.besu.consensus.common.bft.events.NewChainHead;
import org.hyperledger.besu.consensus.qbft.support.TestContext;
import org.hyperledger.besu.consensus.qbft.support.TestContextBuilder;
import org.hyperledger.besu.ethereum.core.BlockHeader;
import org.hyperledger.besu.testutil.TestClock;
import java.time.Instant;
import java.util.List;
import java.util.Map;
import java.util.concurrent.TimeUnit;
import org.junit.Test;
public class TransitionsTest {
@Test
public void transitionsBlockPeriod() throws InterruptedException {
final TestClock clock = new TestClock(Instant.EPOCH);
final List<QbftFork> qbftForks =
List.of(
new QbftFork(
JsonUtil.objectNodeFromMap(
Map.of(BftFork.FORK_BLOCK_KEY, 1, BftFork.BLOCK_PERIOD_SECONDS_KEY, 10))),
new QbftFork(
JsonUtil.objectNodeFromMap(
Map.of(BftFork.FORK_BLOCK_KEY, 2, BftFork.BLOCK_PERIOD_SECONDS_KEY, 20))));
final BftEventQueue bftEventQueue = new BftEventQueue(TestContextBuilder.MESSAGE_QUEUE_LIMIT);
final TestContext context =
new TestContextBuilder()
.indexOfFirstLocallyProposedBlock(0)
.validatorCount(1)
.clock(clock)
.qbftForks(qbftForks)
.eventQueue(bftEventQueue)
.buildAndStart();
clock.stepMillis(10_000);
context.getEventMultiplexer().handleBftEvent(bftEventQueue.poll(1, TimeUnit.SECONDS));
context
.getController()
.handleNewBlockEvent(new NewChainHead(context.getBlockchain().getChainHeadHeader()));
clock.stepMillis(20_000);
context.getEventMultiplexer().handleBftEvent(bftEventQueue.poll(1, TimeUnit.SECONDS));
final BlockHeader genesisBlock = context.getBlockchain().getBlockHeader(0).get();
final BlockHeader blockHeader1 = context.getBlockchain().getBlockHeader(1).get();
final BlockHeader blockHeader2 = context.getBlockchain().getBlockHeader(2).get();
assertThat(blockHeader1.getTimestamp()).isEqualTo(genesisBlock.getTimestamp() + 10);
assertThat(blockHeader2.getTimestamp()).isEqualTo(blockHeader1.getTimestamp() + 20);
}
}

@ -0,0 +1,16 @@
<?xml version="1.0" encoding="UTF-8"?>
<Configuration status="WARN">
<Properties>
<Property name="root.log.level">DEBUG</Property>
</Properties>
<Appenders>
<Console name="Console" target="SYSTEM_OUT">
<PatternLayout pattern="%d{yyyy-MM-dd HH:mm:ss.SSSZZZ} | %t | %-5level | %c{1} | %msg%n" /> </Console>
</Appenders>
<Loggers>
<Root level="${sys:root.log.level}">
<AppenderRef ref="Console" />
</Root>
</Loggers>
</Configuration>

@ -95,7 +95,7 @@ public class QbftBlockHeaderUtils {
Optional.of(Vote.authVote(Address.fromHexString("1"))),
validators,
singletonList(proposerNodeKey),
0x2A,
ROUND_NUMBER,
qbftExtraDataEncoder);
builder.extraData(qbftExtraDataEncoder.encode(bftExtraData));
@ -114,7 +114,7 @@ public class QbftBlockHeaderUtils {
}
builder.number(number);
builder.gasLimit(5000);
builder.timestamp(6000 * number);
builder.timestamp(6 * number);
builder.mixHash(
Hash.fromHexString("0x63746963616c2062797a616e74696e65206661756c7420746f6c6572616e6365"));
builder.difficulty(Difficulty.ONE);

@ -27,6 +27,12 @@ public class TestClock extends Clock {
private Instant now = Instant.ofEpochSecond(24982948294L);
public TestClock() {}
public TestClock(final Instant now) {
this.now = now;
}
@Override
public ZoneId getZone() {
return ZoneOffset.UTC;

Loading…
Cancel
Save