Enable Quorum/IBFT1 to Besu migration (#8262)

* Enable Quorum/IBFT1 to Besu migration

Signed-off-by: Bhanu Pulluri <bhanu.pulluri@kaleido.io>

* Fix BftMining acceptance test

Signed-off-by: Bhanu Pulluri <bhanu.pulluri@kaleido.io>

* Introduce delay after London fork update in BFT mining test to prevent timing issues

Signed-off-by: Bhanu Pulluri <bhanu.pulluri@kaleido.io>

* Update besu/src/main/java/org/hyperledger/besu/controller/IbftLegacyBesuControllerBuilder.java

Co-authored-by: Matt Whitehead <matthew1001@hotmail.com>
Signed-off-by: Bhanu Pulluri <59369753+pullurib@users.noreply.github.com>

* Review changes

Signed-off-by: Bhanu Pulluri <bhanu.pulluri@kaleido.io>

* update creating additional JSON RPC methods for all controllerbuidlers in consensus schedule

Signed-off-by: Bhanu Pulluri <bhanu.pulluri@kaleido.io>

* Create ethprotocol manager and plugin factory for both consensus controllers in migration

Signed-off-by: Bhanu Pulluri <bhanu.pulluri@kaleido.io>

* Refactor resource files

Signed-off-by: Bhanu Pulluri <bhanu.pulluri@kaleido.io>

* fix verification metadata

Signed-off-by: Bhanu Pulluri <bhanu.pulluri@kaleido.io>

* fix regression

Signed-off-by: Bhanu Pulluri <bhanu.pulluri@kaleido.io>

* update changelog

Signed-off-by: Bhanu Pulluri <bhanu.pulluri@kaleido.io>

* Fix controller selection at the transition block

Signed-off-by: Bhanu Pulluri <bhanu.pulluri@kaleido.io>

* Review changes

Signed-off-by: Bhanu Pulluri <bhanu.pulluri@kaleido.io>

* Revert BftExtraData changes

Signed-off-by: Bhanu Pulluri <bhanu.pulluri@kaleido.io>

---------

Signed-off-by: Bhanu Pulluri <bhanu.pulluri@kaleido.io>
Signed-off-by: Bhanu Pulluri <59369753+pullurib@users.noreply.github.com>
Co-authored-by: Bhanu Pulluri <bhanu.pulluri@kaleido.io>
Co-authored-by: Matt Whitehead <matthew1001@hotmail.com>
Co-authored-by: Matt Whitehead <matthew.whitehead@kaleido.io>
Co-authored-by: Sally MacFarlane <macfarla.github@gmail.com>
This commit is contained in:
Bhanu Pulluri
2025-03-12 10:08:51 -04:00
committed by GitHub
parent 083b1d3986
commit 2db46e964c
55 changed files with 1708 additions and 79 deletions

View File

@@ -18,6 +18,7 @@ package org.hyperledger.besu.consensus.common;
public class EpochManager {
private final long epochLengthInBlocks;
private final long startBlock;
/**
* Instantiates a new Epoch manager.
@@ -25,7 +26,18 @@ public class EpochManager {
* @param epochLengthInBlocks the epoch length in blocks
*/
public EpochManager(final long epochLengthInBlocks) {
this(epochLengthInBlocks, 0);
}
/**
* Instantiates a new Epoch manager.
*
* @param epochLengthInBlocks the epoch length in blocks
* @param startBlock the block number where the epoch counting starts
*/
public EpochManager(final long epochLengthInBlocks, final long startBlock) {
this.epochLengthInBlocks = epochLengthInBlocks;
this.startBlock = startBlock;
}
/**
@@ -35,7 +47,13 @@ public class EpochManager {
* @return the boolean
*/
public boolean isEpochBlock(final long blockNumber) {
return (blockNumber % epochLengthInBlocks) == 0;
if (blockNumber < 0) {
throw new IllegalArgumentException("Block number must be 0 or greater.");
}
if (blockNumber < startBlock - 1) {
return false;
}
return (blockNumber - (startBlock == 0 ? 0 : startBlock - 1)) % epochLengthInBlocks == 0;
}
/**
@@ -45,6 +63,9 @@ public class EpochManager {
* @return the last epoch block
*/
public long getLastEpochBlock(final long blockNumber) {
return blockNumber - (blockNumber % epochLengthInBlocks);
if (blockNumber < startBlock) {
throw new IllegalArgumentException("Block number is before start block.");
}
return startBlock + ((blockNumber - startBlock) / epochLengthInBlocks) * epochLengthInBlocks;
}
}

View File

@@ -54,7 +54,7 @@ public class MigratingMiningCoordinator implements MiningCoordinator, BlockAdded
this.miningCoordinatorSchedule = miningCoordinatorSchedule;
this.blockchain = blockchain;
this.activeMiningCoordinator =
this.miningCoordinatorSchedule.getFork(blockchain.getChainHeadBlockNumber()).getValue();
this.miningCoordinatorSchedule.getFork(blockchain.getChainHeadBlockNumber() + 1).getValue();
}
@Override

View File

@@ -45,6 +45,12 @@ public class MigratingProtocolContext extends ProtocolContext {
@Override
public <C extends ConsensusContext> C getConsensusContext(final Class<C> klass) {
final long chainHeadBlockNumber = getBlockchain().getChainHeadBlockNumber();
return consensusContextSchedule.getFork(chainHeadBlockNumber).getValue().as(klass);
return consensusContextSchedule.getFork(chainHeadBlockNumber + 1).getValue().as(klass);
}
@Override
public <C extends ConsensusContext> C getConsensusContext(
final Class<C> klass, final long blockNumber) {
return consensusContextSchedule.getFork(blockNumber).getValue().as(klass);
}
}

View File

@@ -28,6 +28,8 @@ import org.hyperledger.besu.ethereum.chain.Blockchain;
import org.hyperledger.besu.ethereum.core.Block;
import org.hyperledger.besu.ethereum.core.BlockHeader;
import org.hyperledger.besu.ethereum.core.Transaction;
import org.hyperledger.besu.ethereum.eth.sync.state.SyncState;
import org.hyperledger.besu.plugin.services.BesuEvents;
import java.util.List;
import java.util.Optional;
@@ -65,6 +67,8 @@ public class BftMiningCoordinator implements MiningCoordinator, BlockAddedObserv
private long blockAddedObserverId;
private final AtomicReference<State> state = new AtomicReference<>(State.PAUSED);
private SyncState syncState;
/**
* Instantiates a new Bft mining coordinator.
*
@@ -91,6 +95,35 @@ public class BftMiningCoordinator implements MiningCoordinator, BlockAddedObserv
this.blockchain = blockchain;
}
/**
* Instantiates a new Bft mining coordinator.
*
* @param bftExecutors the bft executors
* @param eventHandler the event handler
* @param bftProcessor the bft processor
* @param blockCreatorFactory the block creator factory
* @param blockchain the blockchain
* @param eventQueue the event queue
* @param syncState the sync state
*/
public BftMiningCoordinator(
final BftExecutors bftExecutors,
final BftEventHandler eventHandler,
final BftProcessor bftProcessor,
final BftBlockCreatorFactory<?> blockCreatorFactory,
final Blockchain blockchain,
final BftEventQueue eventQueue,
final SyncState syncState) {
this.bftExecutors = bftExecutors;
this.eventHandler = eventHandler;
this.bftProcessor = bftProcessor;
this.blockCreatorFactory = blockCreatorFactory;
this.eventQueue = eventQueue;
this.blockchain = blockchain;
this.syncState = syncState;
}
@Override
public void start() {
if (state.compareAndSet(State.IDLE, State.RUNNING)
@@ -120,6 +153,41 @@ public class BftMiningCoordinator implements MiningCoordinator, BlockAddedObserv
}
}
@Override
public void subscribe() {
if (syncState == null) {
return;
}
syncState.subscribeSyncStatus(
syncStatus -> {
if (syncState.syncTarget().isPresent()) {
// We're syncing so stop doing other stuff
LOG.info("Stopping BFT mining coordinator while we are syncing");
stop();
} else {
LOG.info("Starting BFT mining coordinator following sync");
enable();
start();
}
});
syncState.subscribeCompletionReached(
new BesuEvents.InitialSyncCompletionListener() {
@Override
public void onInitialSyncCompleted() {
LOG.info("Starting BFT mining coordinator following initial sync");
enable();
start();
}
@Override
public void onInitialSyncRestart() {
// Nothing to do. The mining coordinator won't be started until
// sync has completed.
}
});
}
@Override
public void awaitStop() throws InterruptedException {
bftExecutors.awaitStop();

View File

@@ -0,0 +1,55 @@
apply plugin: 'java-library'
jar {
archiveBaseName = 'besu-ibftlegacy'
manifest {
attributes(
'Specification-Title': archiveBaseName,
'Specification-Version': project.version,
'Implementation-Title': archiveBaseName,
'Implementation-Version': calculateVersion(),
'Commit-Hash': getGitCommitDetails(40).hash
)
}
}
dependencies {
implementation project(':config')
implementation project(':consensus:common')
implementation project(':consensus:ibft')
implementation project(':crypto:algorithms')
implementation project(':datatypes')
implementation project(':ethereum:api')
implementation project(':ethereum:blockcreation')
implementation project(':ethereum:core')
implementation project(':ethereum:eth')
implementation project(':ethereum:p2p')
implementation project(':ethereum:rlp')
implementation project(':evm')
implementation project(':metrics:core')
implementation project(':services:kvstore')
implementation 'com.google.guava:guava'
implementation 'io.vertx:vertx-core'
implementation 'com.fasterxml.jackson.core:jackson-databind'
implementation 'io.tmio:tuweni-bytes'
implementation 'io.tmio:tuweni-units'
testImplementation project(path: ':consensus:common', configuration: 'testSupportArtifacts')
testImplementation project(path: ':consensus:ibft', configuration: 'testSupportArtifacts')
testImplementation project(path: ':ethereum:core', configuration: 'testSupportArtifacts')
testImplementation project(path: ':ethereum:eth', configuration: 'testSupportArtifacts')
testImplementation project(':metrics:core')
testImplementation project(':testutil')
integrationTestImplementation 'org.assertj:assertj-core'
integrationTestImplementation 'org.junit.jupiter:junit-jupiter-api'
integrationTestImplementation 'org.mockito:mockito-core'
integrationTestImplementation 'org.mockito:mockito-junit-jupiter'
testImplementation 'org.assertj:assertj-core'
testImplementation 'org.junit.jupiter:junit-jupiter'
testImplementation 'org.awaitility:awaitility'
testImplementation 'org.mockito:mockito-core'
testImplementation 'org.mockito:mockito-junit-jupiter'
}

View File

@@ -0,0 +1,169 @@
/*
* 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.ibftlegacy;
import org.hyperledger.besu.consensus.common.bft.BftBlockHeaderFunctions;
import org.hyperledger.besu.consensus.common.bft.BftExtraData;
import org.hyperledger.besu.crypto.SECPSignature;
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.BlockHeaderBuilder;
import org.hyperledger.besu.ethereum.core.Util;
import org.hyperledger.besu.ethereum.rlp.BytesValueRLPOutput;
import java.util.List;
import java.util.function.Supplier;
import java.util.stream.Collectors;
import org.apache.tuweni.bytes.Bytes;
/** The Ibft block hashing. */
public class IbftBlockHashing {
/** Default constructor */
public IbftBlockHashing() {}
private static final Bytes COMMIT_MSG_CODE = Bytes.wrap(new byte[] {2});
private static final IbftExtraDataCodec ibftExtraDataCodec = new IbftExtraDataCodec();
/**
* Constructs a hash of the block header, suitable for use when creating the proposer seal. The
* extra data is modified to have a null proposer seal and empty list of committed seals.
*
* @param header The header for which a proposer seal is to be calculated
* @param ibftExtraData The extra data block which is to be inserted to the header once seal is
* calculated
* @return the hash of the header suitable for signing as the proposer seal
*/
public static Hash calculateDataHashForProposerSeal(
final BlockHeader header, final BftExtraData ibftExtraData) {
final Bytes headerRlp =
serializeHeader(header, () -> encodeExtraDataWithoutCommittedSeals(ibftExtraData, null));
// Proposer hash is the hash of the hash
return Hash.hash(Hash.hash(headerRlp));
}
/**
* 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 (without extra data)
* @param ibftExtraData 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
*/
public static Hash calculateDataHashForCommittedSeal(
final BlockHeader header, final IbftLegacyExtraData ibftExtraData) {
// The data signed by a committer is an array of [Hash, COMMIT_MSG_CODE]
final Hash dataHash = Hash.hash(serializeHeaderWithoutCommittedSeals(header, ibftExtraData));
final Bytes seal = Bytes.wrap(dataHash, COMMIT_MSG_CODE);
return Hash.hash(seal);
}
/**
* Constructs a hash of the block header, but omits the committerSeals (as this changes on each of
* the potentially circulated blocks at the current chain height).
*
* @param header The header for which a block hash is to be calculated
* @return the hash of the header including the validator and proposer seal in the extra data
*/
public static Hash calculateHashOfIbftBlockOnchain(final BlockHeader header) {
final IbftLegacyExtraData ibftExtraData = ibftExtraDataCodec.decode(header);
Hash hash = Hash.hash(serializeHeaderWithoutCommittedSeals(header, ibftExtraData));
return hash;
}
private static Bytes serializeHeaderWithoutCommittedSeals(
final BlockHeader header, final IbftLegacyExtraData ibftExtraData) {
return serializeHeader(
header,
() -> encodeExtraDataWithoutCommittedSeals(ibftExtraData, ibftExtraData.getProposerSeal()));
}
/**
* Recovers the proposer's {@link Address} from the proposer seal.
*
* @param header the block header that was signed by the proposer seal
* @param ibftExtraData the parsed IBftExtraData from the header
* @return the proposer address
*/
public static Address recoverProposerAddress(
final BlockHeader header, final IbftLegacyExtraData ibftExtraData) {
final Hash proposerHash = calculateDataHashForProposerSeal(header, ibftExtraData);
Address addr = Util.signatureToAddress(ibftExtraData.getProposerSeal(), proposerHash);
return addr;
}
/**
* Recovers the {@link Address} for each validator that contributed a committed seal to the block.
*
* @param header the block header that was signed by the committed seals
* @param ibftExtraData the parsed IBftExtraData from the header
* @return the addresses of validators that provided a committed seal
*/
public static List<Address> recoverCommitterAddresses(
final BlockHeader header, final IbftLegacyExtraData ibftExtraData) {
final Hash committerHash =
IbftBlockHashing.calculateDataHashForCommittedSeal(header, ibftExtraData);
return ibftExtraData.getSeals().stream()
.map(p -> Util.signatureToAddress(p, committerHash))
.collect(Collectors.toList());
}
private static Bytes encodeExtraDataWithoutCommittedSeals(
final BftExtraData ibftExtraData, final SECPSignature proposerSeal) {
final BytesValueRLPOutput extraDataEncoding = new BytesValueRLPOutput();
extraDataEncoding.startList();
extraDataEncoding.writeList(
ibftExtraData.getValidators(), (validator, rlp) -> rlp.writeBytes(validator));
if (proposerSeal != null) {
extraDataEncoding.writeBytes(proposerSeal.encodedBytes());
} else {
extraDataEncoding.writeNull();
}
// Represents an empty committer list (i.e this is not included in the hashing of the block)
extraDataEncoding.writeEmptyList();
extraDataEncoding.endList();
Bytes vanityBytes = ibftExtraData.getVanityData();
return Bytes.wrap(vanityBytes, extraDataEncoding.encoded());
}
private static Bytes serializeHeader(
final BlockHeader header, final Supplier<Bytes> extraDataSerializer) {
// create a block header which is a copy of the header supplied as parameter except of the
// extraData field
final BlockHeaderBuilder builder = BlockHeaderBuilder.fromHeader(header);
builder.blockHeaderFunctions(BftBlockHeaderFunctions.forOnchainBlock(ibftExtraDataCodec));
// set the extraData field using the supplied extraDataSerializer if the block height is not 0
if (header.getNumber() == BlockHeader.GENESIS_BLOCK_NUMBER) {
builder.extraData(header.getExtraData());
} else {
builder.extraData(extraDataSerializer.get());
}
final BytesValueRLPOutput out = new BytesValueRLPOutput();
builder.buildBlockHeader().writeTo(out);
return out.encoded();
}
}

View File

@@ -0,0 +1,93 @@
/*
* 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.ibftlegacy;
import static org.hyperledger.besu.ethereum.mainnet.AbstractGasLimitSpecification.DEFAULT_MAX_GAS_LIMIT;
import static org.hyperledger.besu.ethereum.mainnet.AbstractGasLimitSpecification.DEFAULT_MIN_GAS_LIMIT;
import org.hyperledger.besu.consensus.ibftlegacy.headervalidationrules.IbftExtraDataValidationRule;
import org.hyperledger.besu.consensus.ibftlegacy.headervalidationrules.VoteValidationRule;
import org.hyperledger.besu.datatypes.Hash;
import org.hyperledger.besu.ethereum.core.BlockHeader;
import org.hyperledger.besu.ethereum.mainnet.BlockHeaderValidator;
import org.hyperledger.besu.ethereum.mainnet.headervalidationrules.AncestryValidationRule;
import org.hyperledger.besu.ethereum.mainnet.headervalidationrules.ConstantFieldValidationRule;
import org.hyperledger.besu.ethereum.mainnet.headervalidationrules.GasLimitRangeAndDeltaValidationRule;
import org.hyperledger.besu.ethereum.mainnet.headervalidationrules.GasUsageValidationRule;
import org.hyperledger.besu.ethereum.mainnet.headervalidationrules.TimestampBoundedByFutureParameter;
import org.hyperledger.besu.ethereum.mainnet.headervalidationrules.TimestampMoreRecentThanParent;
import org.apache.tuweni.units.bigints.UInt256;
/** The Ibft block header validation ruleset factory. */
public class IbftBlockHeaderValidationRulesetFactory {
/** Default constructor */
public IbftBlockHeaderValidationRulesetFactory() {}
/**
* Produces a BlockHeaderValidator configured for assessing ibft block headers which are to form
* part of the BlockChain (i.e. not proposed blocks, which do not contain commit seals)
*
* @param secondsBetweenBlocks the minimum number of seconds which must elapse between blocks.
* @param ceil2nBy3Block the block after which 2/3n commit seals must exist, rather than 2F+1
* @return BlockHeaderValidator configured for assessing ibft block headers
*/
public static BlockHeaderValidator.Builder ibftBlockHeaderValidatorBuilder(
final long secondsBetweenBlocks, final long ceil2nBy3Block) {
return createBlockHeaderValidatorBuilder(secondsBetweenBlocks, true, ceil2nBy3Block);
}
/**
* Creates a builder for the IBFT block header validator.
*
* @param secondsBetweenBlocks the minimum number of seconds which must elapse between blocks.
* @return a builder for the IBFT block header validator
*/
public static BlockHeaderValidator.Builder ibftBlockHeaderValidatorBuilder(
final long secondsBetweenBlocks) {
return createBlockHeaderValidatorBuilder(secondsBetweenBlocks);
}
private static BlockHeaderValidator.Builder createBlockHeaderValidatorBuilder(
final long secondsBetweenBlocks,
final boolean validateCommitSeals,
final long ceil2nBy3Block) {
BlockHeaderValidator.Builder builder = createBlockHeaderValidatorBuilder(secondsBetweenBlocks);
builder.addRule(new IbftExtraDataValidationRule(validateCommitSeals, ceil2nBy3Block));
return builder;
}
private static BlockHeaderValidator.Builder createBlockHeaderValidatorBuilder(
final long secondsBetweenBlocks) {
return new BlockHeaderValidator.Builder()
.addRule(new AncestryValidationRule())
.addRule(new GasUsageValidationRule())
.addRule(
new GasLimitRangeAndDeltaValidationRule(DEFAULT_MIN_GAS_LIMIT, DEFAULT_MAX_GAS_LIMIT))
.addRule(new TimestampBoundedByFutureParameter(1))
.addRule(new TimestampMoreRecentThanParent(secondsBetweenBlocks))
.addRule(
new ConstantFieldValidationRule<>(
"MixHash", BlockHeader::getMixHash, IbftHelpers.EXPECTED_MIX_HASH))
.addRule(
new ConstantFieldValidationRule<>(
"OmmersHash", BlockHeader::getOmmersHash, Hash.EMPTY_LIST_HASH))
.addRule(
new ConstantFieldValidationRule<>(
"Difficulty", BlockHeader::getDifficulty, UInt256.ONE))
.addRule(new VoteValidationRule());
}
}

View File

@@ -0,0 +1,113 @@
/*
* 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.ibftlegacy;
import static com.google.common.base.Preconditions.checkArgument;
import org.hyperledger.besu.consensus.common.bft.BftExtraData;
import org.hyperledger.besu.consensus.common.bft.BftExtraDataCodec;
import org.hyperledger.besu.crypto.SECPSignature;
import org.hyperledger.besu.crypto.SignatureAlgorithm;
import org.hyperledger.besu.crypto.SignatureAlgorithmFactory;
import org.hyperledger.besu.datatypes.Address;
import org.hyperledger.besu.ethereum.core.BlockHeader;
import org.hyperledger.besu.ethereum.rlp.BytesValueRLPInput;
import org.hyperledger.besu.ethereum.rlp.RLPInput;
import java.util.Collection;
import com.google.common.base.Supplier;
import com.google.common.base.Suppliers;
import org.apache.tuweni.bytes.Bytes;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Represents encoder/decoder of the serialized data structure stored in the extraData field of the
* BlockHeader used when operating under an IBFT consensus mechanism.
*/
public class IbftExtraDataCodec extends BftExtraDataCodec {
/** The constant EXTRA_VANITY_LENGTH. */
public static final int EXTRA_VANITY_LENGTH = 32;
private static final Supplier<SignatureAlgorithm> SIGNATURE_ALGORITHM =
Suppliers.memoize(SignatureAlgorithmFactory::getInstance);
private static final Logger LOG = LoggerFactory.getLogger(IbftExtraDataCodec.class);
/** Default constructor */
public IbftExtraDataCodec() {}
/**
* Decode.
*
* @param blockHeader the block header
* @return the bft extra data
*/
@Override
public IbftLegacyExtraData decode(final BlockHeader blockHeader) {
final Object inputExtraData = blockHeader.getParsedExtraData();
if (inputExtraData instanceof IbftLegacyExtraData) {
return (IbftLegacyExtraData) inputExtraData;
}
LOG.warn(
"Expected a BftExtraData instance but got {}. Reparsing required.",
inputExtraData != null ? inputExtraData.getClass().getName() : "null");
return decodeRaw(blockHeader.getExtraData());
}
/**
* Decode raw input and return ibft extra data.
*
* @param input the input
* @return the ibft extra data
*/
@Override
public IbftLegacyExtraData decodeRaw(final Bytes input) {
checkArgument(
input.size() > EXTRA_VANITY_LENGTH,
"Invalid Bytes supplied - too short to produce a valid IBFT Extra Data object.");
final Bytes vanityData = input.slice(0, EXTRA_VANITY_LENGTH);
final Bytes rlpData = input.slice(EXTRA_VANITY_LENGTH);
final RLPInput rlpInput = new BytesValueRLPInput(rlpData, false);
rlpInput.enterList(); // This accounts for the "root node" which contains IBFT data items.
final Collection<Address> validators = rlpInput.readList(Address::readFrom);
final SECPSignature proposerSeal = parseProposerSeal(rlpInput);
final Collection<SECPSignature> seals =
rlpInput.readList(rlp -> SIGNATURE_ALGORITHM.get().decodeSignature(rlp.readBytes()));
rlpInput.leaveList();
return new IbftLegacyExtraData(vanityData, seals, proposerSeal, validators);
}
private static SECPSignature parseProposerSeal(final RLPInput rlpInput) {
final Bytes data = rlpInput.readBytes();
return data.isZero() ? null : SIGNATURE_ALGORITHM.get().decodeSignature(data);
}
/**
* Encode extra data to bytes.
*
* @return the bytes
*/
@Override
public Bytes encode(final BftExtraData bftExtraData, final EncodingType encodingType) {
throw new UnsupportedOperationException("The encode method is not supported.");
}
}

View File

@@ -0,0 +1,38 @@
/*
* 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.ibftlegacy;
import org.hyperledger.besu.datatypes.Hash;
/** The Ibft helpers utility class. */
public class IbftHelpers {
/** Default constructor */
public IbftHelpers() {}
/** The constant EXPECTED_MIX_HASH. */
public static final Hash EXPECTED_MIX_HASH =
Hash.fromHexString("0x63746963616c2062797a616e74696e65206661756c7420746f6c6572616e6365");
/**
* Calculate required validator quorum.
*
* @param validatorCount the validator count
* @return the int
*/
public static int calculateRequiredValidatorQuorum(final int validatorCount) {
final int F = (validatorCount - 1) / 3;
return (2 * F) + 1;
}
}

View File

@@ -0,0 +1,103 @@
/*
* 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.ibftlegacy;
import org.hyperledger.besu.consensus.common.bft.BftBlockHashing;
import org.hyperledger.besu.consensus.common.bft.BftBlockHeaderFunctions;
import org.hyperledger.besu.consensus.common.bft.BftBlockInterface;
import org.hyperledger.besu.consensus.common.bft.BftExtraDataCodec;
import org.hyperledger.besu.consensus.common.validator.ValidatorVote;
import org.hyperledger.besu.consensus.common.validator.VoteType;
import org.hyperledger.besu.datatypes.Address;
import org.hyperledger.besu.ethereum.core.BlockHeader;
import java.util.Collection;
import java.util.Optional;
import com.google.common.collect.ImmutableBiMap;
import org.apache.tuweni.bytes.Bytes;
/** The Ibft legacy block interface. */
public class IbftLegacyBlockInterface extends BftBlockInterface {
/** The constant NO_VOTE_SUBJECT. */
public static final Address NO_VOTE_SUBJECT = Address.wrap(Bytes.wrap(new byte[Address.SIZE]));
/** The constant ADD_NONCE. */
public static final long ADD_NONCE = 0xFFFFFFFFFFFFFFFFL;
/** The constant DROP_NONCE. */
public static final long DROP_NONCE = 0x0L;
private static final ImmutableBiMap<VoteType, Long> voteToValue =
ImmutableBiMap.of(
VoteType.ADD, ADD_NONCE,
VoteType.DROP, DROP_NONCE);
private static final IbftExtraDataCodec ibftExtraDataCodec = new IbftExtraDataCodec();
/**
* Constructor for IbftLegacyBlockInterface.
*
* @param bftExtraDataCodec the codec for BFT extra data
*/
public IbftLegacyBlockInterface(final BftExtraDataCodec bftExtraDataCodec) {
super(bftExtraDataCodec);
}
@Override
public Address getProposerOfBlock(final BlockHeader header) {
final IbftLegacyExtraData ibftExtraData = ibftExtraDataCodec.decode(header);
return IbftBlockHashing.recoverProposerAddress(header, ibftExtraData);
}
@Override
public Address getProposerOfBlock(final org.hyperledger.besu.plugin.data.BlockHeader header) {
return getProposerOfBlock(
BlockHeader.convertPluginBlockHeader(
header,
new BftBlockHeaderFunctions(
h -> new BftBlockHashing(ibftExtraDataCodec).calculateDataHashForCommittedSeal(h),
ibftExtraDataCodec)));
}
@Override
public Optional<ValidatorVote> extractVoteFromHeader(final BlockHeader header) {
final Address candidate = header.getCoinbase();
if (!candidate.equals(NO_VOTE_SUBJECT)) {
final Address proposer = getProposerOfBlock(header);
final VoteType votePolarity = voteToValue.inverse().get(header.getNonce());
final Address recipient = header.getCoinbase();
return Optional.of(new ValidatorVote(votePolarity, proposer, recipient));
}
return Optional.empty();
}
@Override
public Collection<Address> validatorsInBlock(final BlockHeader header) {
return ibftExtraDataCodec.decode(header).getValidators();
}
/**
* Is valid vote value.
*
* @param value the value
* @return the boolean
*/
public static boolean isValidVoteValue(final long value) {
return voteToValue.values().contains(value);
}
}

View File

@@ -0,0 +1,61 @@
/*
* 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.ibftlegacy;
import org.hyperledger.besu.consensus.common.bft.BftExtraData;
import org.hyperledger.besu.crypto.SECPSignature;
import org.hyperledger.besu.datatypes.Address;
import java.util.Collection;
import java.util.Optional;
import org.apache.tuweni.bytes.Bytes;
/** The Ibft Legacy extra data. */
public class IbftLegacyExtraData extends BftExtraData {
private final SECPSignature proposerSeal;
/**
* Instantiates a new Bft extra data.
*
* @param vanityData the vanity data
* @param seals the seals
* @param validators the validators
* @param proposerSeal the proposer seal
*/
public IbftLegacyExtraData(
final Bytes vanityData,
final Collection<SECPSignature> seals,
final SECPSignature proposerSeal,
final Collection<Address> validators) {
super(vanityData, seals, Optional.empty(), 0, validators);
this.proposerSeal = proposerSeal;
}
/**
* Gets proposer seal.
*
* @return the proposer seal
*/
public SECPSignature getProposerSeal() {
return proposerSeal;
}
@Override
public String toString() {
return "IbftLegacyExtraData{" + super.toString() + ", proposerSeal=" + proposerSeal + '}';
}
}

View File

@@ -0,0 +1,110 @@
/*
* 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.ibftlegacy;
import static org.hyperledger.besu.consensus.ibftlegacy.IbftBlockHeaderValidationRulesetFactory.ibftBlockHeaderValidatorBuilder;
import org.hyperledger.besu.config.GenesisConfigOptions;
import org.hyperledger.besu.config.IbftLegacyConfigOptions;
import org.hyperledger.besu.consensus.common.bft.BftBlockHeaderFunctions;
import org.hyperledger.besu.datatypes.Wei;
import org.hyperledger.besu.ethereum.chain.BadBlockManager;
import org.hyperledger.besu.ethereum.core.PrivacyParameters;
import org.hyperledger.besu.ethereum.mainnet.MainnetBlockBodyValidator;
import org.hyperledger.besu.ethereum.mainnet.MainnetBlockImporter;
import org.hyperledger.besu.ethereum.mainnet.MainnetProtocolSpecs;
import org.hyperledger.besu.ethereum.mainnet.ProtocolSchedule;
import org.hyperledger.besu.ethereum.mainnet.ProtocolScheduleBuilder;
import org.hyperledger.besu.ethereum.mainnet.ProtocolSpecAdapters;
import org.hyperledger.besu.ethereum.mainnet.ProtocolSpecBuilder;
import org.hyperledger.besu.evm.internal.EvmConfiguration;
import java.math.BigInteger;
import java.util.Optional;
/** Defines the protocol behaviours for a blockchain using IBFT. */
public class IbftProtocolSchedule {
// Default constructor
/** Default constructor */
public IbftProtocolSchedule() {}
private static final BigInteger DEFAULT_CHAIN_ID = BigInteger.ONE;
private static final IbftExtraDataCodec ibftExtraDataCodec = new IbftExtraDataCodec();
/**
* Create protocol schedule.
*
* @param config the config
* @param privacyParameters the privacy parameters
* @param isRevertReasonEnabled the is revert reason enabled
* @param evmConfiguration the evm configuration
* @return the protocol schedule
*/
public static ProtocolSchedule create(
final GenesisConfigOptions config,
final PrivacyParameters privacyParameters,
final boolean isRevertReasonEnabled,
final EvmConfiguration evmConfiguration) {
final IbftLegacyConfigOptions ibftConfig = config.getIbftLegacyConfigOptions();
final long blockPeriod = ibftConfig.getBlockPeriodSeconds();
return new ProtocolScheduleBuilder(
config,
Optional.of(DEFAULT_CHAIN_ID),
ProtocolSpecAdapters.create(0, builder -> applyIbftChanges(blockPeriod, builder)),
privacyParameters,
isRevertReasonEnabled,
evmConfiguration,
null,
new BadBlockManager(),
false,
null)
.createProtocolSchedule();
}
/**
* Create protocol schedule.
*
* @param config the config
* @param isRevertReasonEnabled the is revert reason enabled
* @param evmConfiguration the evm configuration
* @return the protocol schedule
*/
public static ProtocolSchedule create(
final GenesisConfigOptions config,
final boolean isRevertReasonEnabled,
final EvmConfiguration evmConfiguration) {
return create(config, PrivacyParameters.DEFAULT, isRevertReasonEnabled, evmConfiguration);
}
private static ProtocolSpecBuilder applyIbftChanges(
final long secondsBetweenBlocks, final ProtocolSpecBuilder builder) {
return builder
.blockHeaderValidatorBuilder(
feeMarket -> ibftBlockHeaderValidatorBuilder(secondsBetweenBlocks))
.ommerHeaderValidatorBuilder(
feeMarket -> ibftBlockHeaderValidatorBuilder(secondsBetweenBlocks))
.blockBodyValidatorBuilder(MainnetBlockBodyValidator::new)
.blockValidatorBuilder(MainnetProtocolSpecs.blockValidatorBuilder())
.blockImporterBuilder(MainnetBlockImporter::new)
.difficultyCalculator((time, parent) -> BigInteger.ONE)
.blockReward(Wei.ZERO)
.skipZeroBlockRewards(true)
.blockHeaderFunctions(
new BftBlockHeaderFunctions(
IbftBlockHashing::calculateHashOfIbftBlockOnchain, ibftExtraDataCodec));
}
}

View File

@@ -0,0 +1,149 @@
/*
* 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.ibftlegacy.headervalidationrules;
import org.hyperledger.besu.consensus.common.bft.BftContext;
import org.hyperledger.besu.consensus.common.bft.BftHelpers;
import org.hyperledger.besu.consensus.ibftlegacy.IbftBlockHashing;
import org.hyperledger.besu.consensus.ibftlegacy.IbftExtraDataCodec;
import org.hyperledger.besu.consensus.ibftlegacy.IbftHelpers;
import org.hyperledger.besu.consensus.ibftlegacy.IbftLegacyExtraData;
import org.hyperledger.besu.datatypes.Address;
import org.hyperledger.besu.ethereum.ProtocolContext;
import org.hyperledger.besu.ethereum.core.BlockHeader;
import org.hyperledger.besu.ethereum.mainnet.AttachedBlockHeaderValidationRule;
import org.hyperledger.besu.ethereum.rlp.RLPException;
import java.util.Collection;
import java.util.List;
import java.util.NavigableSet;
import java.util.TreeSet;
import com.google.common.collect.Iterables;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Ensures the byte content of the extraData field can be deserialised into an appropriate
* structure, and that the structure created contains data matching expectations from preceding
* blocks.
*/
public class IbftExtraDataValidationRule implements AttachedBlockHeaderValidationRule {
private static final Logger LOG = LoggerFactory.getLogger(IbftExtraDataValidationRule.class);
private static final IbftExtraDataCodec ibftExtraDataCodec = new IbftExtraDataCodec();
private final boolean validateCommitSeals;
private final long ceil2nBy3Block;
/**
* Instantiates a new Ibft extra data validation rule.
*
* @param validateCommitSeals the validate commit seals
* @param ceil2nBy3Block the ceil 2 n by 3 block
*/
public IbftExtraDataValidationRule(final boolean validateCommitSeals, final long ceil2nBy3Block) {
this.validateCommitSeals = validateCommitSeals;
this.ceil2nBy3Block = ceil2nBy3Block;
}
@Override
public boolean validate(
final BlockHeader header, final BlockHeader parent, final ProtocolContext context) {
try {
final Collection<Address> storedValidators =
context
.getConsensusContext(BftContext.class)
.getValidatorProvider()
.getValidatorsAfterBlock(parent);
final IbftLegacyExtraData ibftExtraData = ibftExtraDataCodec.decode(header);
final Address proposer = IbftBlockHashing.recoverProposerAddress(header, ibftExtraData);
if (!storedValidators.contains(proposer)) {
LOG.info("Invalid block header: Proposer sealing block is not a member of the validators.");
return false;
}
if (validateCommitSeals) {
final List<Address> committers =
IbftBlockHashing.recoverCommitterAddresses(header, ibftExtraData);
final int minimumSealsRequired =
header.getNumber() < ceil2nBy3Block
? IbftHelpers.calculateRequiredValidatorQuorum(storedValidators.size())
: BftHelpers.calculateRequiredValidatorQuorum(storedValidators.size());
if (!validateCommitters(committers, storedValidators, minimumSealsRequired)) {
return false;
}
}
final NavigableSet<Address> sortedReportedValidators =
new TreeSet<>(ibftExtraData.getValidators());
if (!Iterables.elementsEqual(ibftExtraData.getValidators(), sortedReportedValidators)) {
LOG.info(
"Invalid block header: Validators are not sorted in ascending order. Expected {} but got {}.",
sortedReportedValidators,
ibftExtraData.getValidators());
return false;
}
if (!Iterables.elementsEqual(ibftExtraData.getValidators(), storedValidators)) {
LOG.info(
"Invalid block header: Incorrect validators. Expected {} but got {}.",
storedValidators,
ibftExtraData.getValidators());
return false;
}
} catch (final RLPException ex) {
LOG.info(
"Invalid block header: ExtraData field was unable to be deserialised into an IBFT Struct.",
ex);
return false;
} catch (final IllegalArgumentException ex) {
LOG.info("Invalid block header: Failed to verify extra data", ex);
return false;
} catch (final RuntimeException ex) {
LOG.info("Invalid block header: Failed to find validators at parent");
return false;
}
return true;
}
private boolean validateCommitters(
final Collection<Address> committers,
final Collection<Address> storedValidators,
final int minimumSealsRequired) {
if (committers.size() < minimumSealsRequired) {
LOG.info(
"Invalid block header: Insufficient committers to seal block. (Required {}, received {})",
minimumSealsRequired,
committers.size());
return false;
}
if (!storedValidators.containsAll(committers)) {
LOG.info(
"Invalid block header: Not all committers are in the locally maintained validator list.");
return false;
}
return true;
}
}

View File

@@ -0,0 +1,48 @@
/*
* 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.ibftlegacy.headervalidationrules;
import org.hyperledger.besu.consensus.ibftlegacy.IbftLegacyBlockInterface;
import org.hyperledger.besu.ethereum.core.BlockHeader;
import org.hyperledger.besu.ethereum.mainnet.DetachedBlockHeaderValidationRule;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/** The Vote validation rule. */
public class VoteValidationRule implements DetachedBlockHeaderValidationRule {
private static final Logger LOG = LoggerFactory.getLogger(VoteValidationRule.class);
/** Default constructor */
public VoteValidationRule() {}
/**
* Responsible for ensuring the nonce is either auth or drop.
*
* @param header the block header to validate
* @param parent the block header corresponding to the parent of the header being validated.
* @return true if the nonce in the header is a valid validator vote value.
*/
@Override
public boolean validate(final BlockHeader header, final BlockHeader parent) {
final long nonce = header.getNonce();
if (!IbftLegacyBlockInterface.isValidVoteValue(nonce)) {
LOG.info("Invalid block header: Nonce value ({}) is neither auth or drop.", nonce);
return false;
}
return true;
}
}

View File

@@ -236,7 +236,7 @@ public class QbftBlockHeightManager implements BaseQbftBlockHeightManager {
if (!(validatorsForHeight.containsAll(previousValidators))
|| !(previousValidators.containsAll(validatorsForHeight))) {
LOG.info(
"Validator list change. Previous chain height {}: {}. Current chain height {}: {}.",
"QBFT Validator list change. Previous chain height {}: {}. Current chain height {}: {}.",
parentHeader.getNumber(),
previousValidators,
parentHeader.getNumber() + 1,

View File

@@ -32,6 +32,7 @@ dependencies {
implementation project(':config')
implementation project(':consensus:common')
implementation project(':consensus:qbft-core')
implementation project(':consensus:ibftlegacy')
implementation project(':crypto:services')
implementation project(':datatypes')
implementation project(':ethereum:api')