mirror of
https://github.com/vacp2p/linea-besu.git
synced 2026-01-09 21:17:54 -05:00
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>
This commit is contained in:
@@ -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);
|
||||
}
|
||||
}
|
||||
16
consensus/qbft/src/integration-test/resources/log4j2.xml
Normal file
16
consensus/qbft/src/integration-test/resources/log4j2.xml
Normal file
@@ -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;
|
||||
|
||||
Reference in New Issue
Block a user