[NC-1582] Moved all the legacy iBFT classes for block importing into the newly created ibftlegacy module (#96)

Signed-off-by: Adrian Sutton <adrian.sutton@consensys.net>
pull/2/head
Roberto Saltini 6 years ago committed by GitHub
parent ceec103c29
commit bb7b5423f6
  1. 5
      consensus/ibft/build.gradle
  2. 10
      consensus/ibft/src/main/java/tech/pegasys/pantheon/consensus/ibft/IbftVoteType.java
  3. 16
      consensus/ibft/src/main/java/tech/pegasys/pantheon/consensus/ibft/Vote.java
  4. 55
      consensus/ibft/src/main/java/tech/pegasys/pantheon/consensus/ibft/VoteTallyUpdater.java
  5. 72
      consensus/ibft/src/main/java/tech/pegasys/pantheon/consensus/ibft/blockcreation/IbftBlockCreator.java
  6. 0
      consensus/ibft/src/test-support/java/tech/pegasys/pantheon/consensus/ibft/IbftProtocolContextFixture.java
  7. 20
      consensus/ibft/src/test/java/tech/pegasys/pantheon/consensus/ibft/IbftBlockImporterTest.java
  8. 33
      consensus/ibftlegacy/build.gradle
  9. 2
      consensus/ibftlegacy/src/main/java/tech/pegasys/pantheon/consensus/ibftlegacy/IbftBlockHashing.java
  10. 5
      consensus/ibftlegacy/src/main/java/tech/pegasys/pantheon/consensus/ibftlegacy/IbftBlockHeaderValidationRulesetFactory.java
  11. 2
      consensus/ibftlegacy/src/main/java/tech/pegasys/pantheon/consensus/ibftlegacy/IbftExtraData.java
  12. 2
      consensus/ibftlegacy/src/main/java/tech/pegasys/pantheon/consensus/ibftlegacy/IbftHelpers.java
  13. 3
      consensus/ibftlegacy/src/main/java/tech/pegasys/pantheon/consensus/ibftlegacy/IbftProtocolSchedule.java
  14. 8
      consensus/ibftlegacy/src/main/java/tech/pegasys/pantheon/consensus/ibftlegacy/IbftProtocolSpecs.java
  15. 87
      consensus/ibftlegacy/src/main/java/tech/pegasys/pantheon/consensus/ibftlegacy/IbftVoteTallyUpdater.java
  16. 119
      consensus/ibftlegacy/src/main/java/tech/pegasys/pantheon/consensus/ibftlegacy/blockcreation/IbftBlockCreator.java
  17. 4
      consensus/ibftlegacy/src/main/java/tech/pegasys/pantheon/consensus/ibftlegacy/blockcreation/IbftExtraDataCalculator.java
  18. 6
      consensus/ibftlegacy/src/main/java/tech/pegasys/pantheon/consensus/ibftlegacy/blockcreation/ProposerSelector.java
  19. 12
      consensus/ibftlegacy/src/main/java/tech/pegasys/pantheon/consensus/ibftlegacy/headervalidationrules/IbftExtraDataValidationRule.java
  20. 2
      consensus/ibftlegacy/src/main/java/tech/pegasys/pantheon/consensus/ibftlegacy/protocol/Istanbul64Protocol.java
  21. 2
      consensus/ibftlegacy/src/main/java/tech/pegasys/pantheon/consensus/ibftlegacy/protocol/Istanbul64ProtocolManager.java
  22. 5
      consensus/ibftlegacy/src/test/java/tech/pegasys/pantheon/consensus/ibftlegacy/IbftBlockHashingTest.java
  23. 15
      consensus/ibftlegacy/src/test/java/tech/pegasys/pantheon/consensus/ibftlegacy/IbftBlockHeaderValidationRulesetFactoryTest.java
  24. 15
      consensus/ibftlegacy/src/test/java/tech/pegasys/pantheon/consensus/ibftlegacy/IbftExtraDataTest.java
  25. 7
      consensus/ibftlegacy/src/test/java/tech/pegasys/pantheon/consensus/ibftlegacy/IbftVoteTallyUpdaterTest.java
  26. 8
      consensus/ibftlegacy/src/test/java/tech/pegasys/pantheon/consensus/ibftlegacy/blockcreation/IbftBlockCreatorTest.java
  27. 6
      consensus/ibftlegacy/src/test/java/tech/pegasys/pantheon/consensus/ibftlegacy/blockcreation/ProposerSelectorTest.java
  28. 9
      consensus/ibftlegacy/src/test/java/tech/pegasys/pantheon/consensus/ibftlegacy/headervalidationrules/IbftExtraDataValidationRuleTest.java
  29. 1
      pantheon/build.gradle
  30. 10
      pantheon/src/main/java/tech/pegasys/pantheon/controller/IbftPantheonController.java
  31. 2
      pantheon/src/main/java/tech/pegasys/pantheon/controller/PantheonController.java
  32. 1
      settings.gradle

@ -36,11 +36,12 @@ dependencies {
testImplementation project( path: ':ethereum:core', configuration: 'testSupportArtifacts')
testImplementation group: 'org.powermock', name: 'powermock-api-mockito2', version: '2.0.0-beta.5'
testImplementation group: 'org.powermock', name: 'powermock-module-junit4', version: '2.0.0-beta.5'
testImplementation group: 'junit', name: 'junit', version: '4.12'
testImplementation "org.awaitility:awaitility:3.1.2"
testImplementation "org.assertj:assertj-core:3.10.0"
testImplementation 'org.mockito:mockito-core'
}
artifacts { testSupportArtifacts testSupportJar }

@ -16,13 +16,13 @@ import tech.pegasys.pantheon.ethereum.rlp.RLPException;
import tech.pegasys.pantheon.ethereum.rlp.RLPInput;
import tech.pegasys.pantheon.ethereum.rlp.RLPOutput;
public enum Ibft2VoteType {
public enum IbftVoteType {
ADD((byte) 0xFF),
DROP((byte) 0x00);
private final byte voteValue;
Ibft2VoteType(final byte voteValue) {
IbftVoteType(final byte voteValue) {
this.voteValue = voteValue;
}
@ -30,15 +30,15 @@ public enum Ibft2VoteType {
return voteValue;
}
public static Ibft2VoteType readFrom(final RLPInput rlpInput) {
public static IbftVoteType readFrom(final RLPInput rlpInput) {
byte encodedByteValue = rlpInput.readByte();
for (final Ibft2VoteType voteType : values()) {
for (final IbftVoteType voteType : values()) {
if (voteType.voteValue == encodedByteValue) {
return voteType;
}
}
throw new RLPException("Invalid Ibft2VoteType RLP encoding");
throw new RLPException("Invalid IbftVoteType RLP encoding");
}
public void writeTo(final RLPOutput rlpOutput) {

@ -20,19 +20,19 @@ import com.google.common.base.Objects;
public class Vote {
private final Address recipient;
private final Ibft2VoteType voteType;
private final IbftVoteType voteType;
private Vote(final Address recipient, final Ibft2VoteType voteType) {
private Vote(final Address recipient, final IbftVoteType voteType) {
this.recipient = recipient;
this.voteType = voteType;
}
public static Vote authVote(final Address address) {
return new Vote(address, Ibft2VoteType.ADD);
return new Vote(address, IbftVoteType.ADD);
}
public static Vote dropVote(final Address address) {
return new Vote(address, Ibft2VoteType.DROP);
return new Vote(address, IbftVoteType.DROP);
}
public Address getRecipient() {
@ -40,11 +40,11 @@ public class Vote {
}
public boolean isAuth() {
return voteType.equals(Ibft2VoteType.ADD);
return voteType.equals(IbftVoteType.ADD);
}
public boolean isDrop() {
return voteType.equals(Ibft2VoteType.DROP);
return voteType.equals(IbftVoteType.DROP);
}
@Override
@ -64,7 +64,7 @@ public class Vote {
return Objects.hashCode(recipient, voteType);
}
public Ibft2VoteType getVoteType() {
public IbftVoteType getVoteType() {
return voteType;
}
@ -79,7 +79,7 @@ public class Vote {
rlpInput.enterList();
final Address recipient = Address.readFrom(rlpInput);
final Ibft2VoteType vote = Ibft2VoteType.readFrom(rlpInput);
final IbftVoteType vote = IbftVoteType.readFrom(rlpInput);
rlpInput.leaveList();
return new Vote(recipient, vote);

@ -12,55 +12,18 @@
*/
package tech.pegasys.pantheon.consensus.ibft;
import tech.pegasys.pantheon.consensus.common.EpochManager;
import tech.pegasys.pantheon.consensus.common.VoteTally;
import tech.pegasys.pantheon.consensus.common.VoteType;
import tech.pegasys.pantheon.ethereum.chain.Blockchain;
import tech.pegasys.pantheon.ethereum.core.Address;
import tech.pegasys.pantheon.ethereum.core.BlockHeader;
import tech.pegasys.pantheon.util.bytes.BytesValue;
import java.util.List;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
/**
* Provides the logic to extract vote tally state from the blockchain and update it as blocks are
* added.
*/
public class VoteTallyUpdater {
private static final Logger LOG = LogManager.getLogger();
private static final Address NO_VOTE_SUBJECT = Address.wrap(BytesValue.wrap(new byte[20]));
private final EpochManager epochManager;
public VoteTallyUpdater(final EpochManager epochManager) {
this.epochManager = epochManager;
}
public interface VoteTallyUpdater {
/**
* Create a new VoteTally based on the current blockchain state.
*
* @param blockchain the blockchain to load the current state from
* @return a VoteTally reflecting the state of the blockchain head
*/
public VoteTally buildVoteTallyFromBlockchain(final Blockchain blockchain) {
final long chainHeadBlockNumber = blockchain.getChainHeadBlockNumber();
final long epochBlockNumber = epochManager.getLastEpochBlock(chainHeadBlockNumber);
LOG.info("Loading validator voting state starting from block {}", epochBlockNumber);
final BlockHeader epochBlock = blockchain.getBlockHeader(epochBlockNumber).get();
final List<Address> initialValidators =
IbftExtraData.decode(epochBlock.getExtraData()).getValidators();
final VoteTally voteTally = new VoteTally(initialValidators);
for (long blockNumber = epochBlockNumber + 1;
blockNumber <= chainHeadBlockNumber;
blockNumber++) {
updateForBlock(blockchain.getBlockHeader(blockNumber).get(), voteTally);
}
return voteTally;
}
VoteTally buildVoteTallyFromBlockchain(final Blockchain blockchain);
/**
* Update the vote tally to reflect changes caused by appending a new block to the chain.
@ -68,17 +31,5 @@ public class VoteTallyUpdater {
* @param header the header of the block being added
* @param voteTally the vote tally to update
*/
public void updateForBlock(final BlockHeader header, final VoteTally voteTally) {
final Address candidate = header.getCoinbase();
if (epochManager.isEpochBlock(header.getNumber())) {
voteTally.discardOutstandingVotes();
return;
}
if (!candidate.equals(NO_VOTE_SUBJECT)) {
final IbftExtraData ibftExtraData = IbftExtraData.decode(header.getExtraData());
final Address proposer = IbftBlockHashing.recoverProposerAddress(header, ibftExtraData);
voteTally.addVote(proposer, candidate, VoteType.fromNonce(header.getNonce()).get());
}
}
void updateForBlock(final BlockHeader header, final VoteTally voteTally);
}

@ -12,42 +12,20 @@
*/
package tech.pegasys.pantheon.consensus.ibft.blockcreation;
import tech.pegasys.pantheon.consensus.ibft.IbftBlockHashing;
import tech.pegasys.pantheon.consensus.ibft.IbftContext;
import tech.pegasys.pantheon.consensus.ibft.IbftExtraData;
import tech.pegasys.pantheon.consensus.ibft.IbftHelpers;
import tech.pegasys.pantheon.crypto.SECP256K1;
import tech.pegasys.pantheon.crypto.SECP256K1.KeyPair;
import tech.pegasys.pantheon.ethereum.ProtocolContext;
import tech.pegasys.pantheon.ethereum.blockcreation.AbstractBlockCreator;
import tech.pegasys.pantheon.ethereum.core.Address;
import tech.pegasys.pantheon.ethereum.core.BlockHashFunction;
import tech.pegasys.pantheon.ethereum.core.BlockHeader;
import tech.pegasys.pantheon.ethereum.core.BlockHeaderBuilder;
import tech.pegasys.pantheon.ethereum.core.Hash;
import tech.pegasys.pantheon.ethereum.core.PendingTransactions;
import tech.pegasys.pantheon.ethereum.core.SealableBlockHeader;
import tech.pegasys.pantheon.ethereum.core.Util;
import tech.pegasys.pantheon.ethereum.core.Wei;
import tech.pegasys.pantheon.ethereum.mainnet.ProtocolSchedule;
import tech.pegasys.pantheon.ethereum.mainnet.ScheduleBasedBlockHashFunction;
import java.util.function.Function;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
/**
* Responsible for producing a Block which conforms to IBFT validation rules (other than missing
* commit seals). Transactions and associated Hashes (stateroot, receipts etc.) are loaded into the
* Block in the base class as part of the transaction selection process.
*/
// TODO: Just a placeholder. Implementation is required.
public class IbftBlockCreator extends AbstractBlockCreator<IbftContext> {
private static final Logger LOG = LogManager.getLogger();
private final KeyPair nodeKeys;
public IbftBlockCreator(
final Address coinbase,
final ExtraDataCalculator extraDataCalculator,
@ -55,8 +33,8 @@ public class IbftBlockCreator extends AbstractBlockCreator<IbftContext> {
final ProtocolContext<IbftContext> protocolContext,
final ProtocolSchedule<IbftContext> protocolSchedule,
final Function<Long, Long> gasLimitCalculator,
final KeyPair nodeKeys,
final Wei minTransactionGasPrice,
final Address miningBeneficiary,
final BlockHeader parentHeader) {
super(
coinbase,
@ -66,54 +44,12 @@ public class IbftBlockCreator extends AbstractBlockCreator<IbftContext> {
protocolSchedule,
gasLimitCalculator,
minTransactionGasPrice,
Util.publicKeyToAddress(nodeKeys.getPublicKey()),
miningBeneficiary,
parentHeader);
this.nodeKeys = nodeKeys;
}
/**
* Responsible for signing (hash of) the block (including MixHash and Nonce), and then injecting
* the seal into the extraData. This is called after a suitable set of transactions have been
* identified, and all resulting hashes have been inserted into the passed-in SealableBlockHeader.
*
* @param sealableBlockHeader A block header containing StateRoots, TransactionHashes etc.
* @return The blockhead which is to be added to the block being proposed.
*/
@Override
protected BlockHeader createFinalBlockHeader(final SealableBlockHeader sealableBlockHeader) {
final BlockHashFunction blockHashFunction =
ScheduleBasedBlockHashFunction.create(protocolSchedule);
final BlockHeaderBuilder builder =
BlockHeaderBuilder.create()
.populateFrom(sealableBlockHeader)
.mixHash(IbftHelpers.EXPECTED_MIX_HASH)
.nonce(0)
.blockHashFunction(blockHashFunction);
final IbftExtraData sealedExtraData = constructSignedExtraData(builder.buildBlockHeader());
// Replace the extraData in the BlockHeaderBuilder, and return header.
return builder.extraData(sealedExtraData.encode()).buildBlockHeader();
}
/**
* Produces an IbftExtraData object with a populated proposerSeal. The signature in the block is
* generated from the Hash of the header (minus proposer and committer seals) and the nodeKeys.
*
* @param headerToSign An almost fully populated header (proposer and committer seals are empty)
* @return Extra data containing the same vanity data and validators as extraData, however
* proposerSeal will also be populated.
*/
private IbftExtraData constructSignedExtraData(final BlockHeader headerToSign) {
final IbftExtraData extraData = IbftExtraData.decode(headerToSign.getExtraData());
final Hash hashToSign =
IbftBlockHashing.calculateDataHashForProposerSeal(headerToSign, extraData);
return new IbftExtraData(
extraData.getVanityData(),
extraData.getSeals(),
SECP256K1.sign(hashToSign, nodeKeys),
extraData.getValidators());
return null;
}
}

@ -29,8 +29,6 @@ import tech.pegasys.pantheon.ethereum.core.BlockImporter;
import tech.pegasys.pantheon.ethereum.db.WorldStateArchive;
import tech.pegasys.pantheon.ethereum.mainnet.HeaderValidationMode;
import java.util.Collections;
import org.junit.Test;
public class IbftBlockImporterTest {
@ -54,9 +52,7 @@ public class IbftBlockImporterTest {
public void voteTallyNotUpdatedWhenBlockImportFails() {
final BlockHeaderTestFixture headerBuilder = new BlockHeaderTestFixture();
final Block block =
new Block(
headerBuilder.buildHeader(),
new BlockBody(Collections.emptyList(), Collections.emptyList()));
new Block(headerBuilder.buildHeader(), new BlockBody(emptyList(), emptyList()));
when(delegate.importBlock(context, block, HeaderValidationMode.FULL, HeaderValidationMode.FULL))
.thenReturn(false);
@ -70,14 +66,12 @@ public class IbftBlockImporterTest {
public void voteTallyNotUpdatedWhenFastBlockImportFails() {
final BlockHeaderTestFixture headerBuilder = new BlockHeaderTestFixture();
final Block block =
new Block(
headerBuilder.buildHeader(),
new BlockBody(Collections.emptyList(), Collections.emptyList()));
new Block(headerBuilder.buildHeader(), new BlockBody(emptyList(), emptyList()));
when(delegate.fastImportBlock(context, block, emptyList(), HeaderValidationMode.LIGHT))
.thenReturn(false);
importer.fastImportBlock(context, block, Collections.emptyList(), HeaderValidationMode.LIGHT);
importer.fastImportBlock(context, block, emptyList(), HeaderValidationMode.LIGHT);
verifyZeroInteractions(voteTallyUpdater);
}
@ -86,8 +80,7 @@ public class IbftBlockImporterTest {
public void voteTallyUpdatedWhenBlockImportSucceeds() {
final Block block =
new Block(
new BlockHeaderTestFixture().buildHeader(),
new BlockBody(Collections.emptyList(), Collections.emptyList()));
new BlockHeaderTestFixture().buildHeader(), new BlockBody(emptyList(), emptyList()));
when(delegate.importBlock(context, block, HeaderValidationMode.FULL, HeaderValidationMode.FULL))
.thenReturn(true);
@ -101,13 +94,12 @@ public class IbftBlockImporterTest {
public void voteTallyUpdatedWhenFastBlockImportSucceeds() {
final Block block =
new Block(
new BlockHeaderTestFixture().buildHeader(),
new BlockBody(Collections.emptyList(), Collections.emptyList()));
new BlockHeaderTestFixture().buildHeader(), new BlockBody(emptyList(), emptyList()));
when(delegate.fastImportBlock(context, block, emptyList(), HeaderValidationMode.LIGHT))
.thenReturn(true);
importer.fastImportBlock(context, block, Collections.emptyList(), HeaderValidationMode.LIGHT);
importer.fastImportBlock(context, block, emptyList(), HeaderValidationMode.LIGHT);
verify(voteTallyUpdater).updateForBlock(block.getHeader(), voteTally);
}

@ -0,0 +1,33 @@
apply plugin: 'java-library'
jar {
baseName 'pantheon-ibftlegacy'
manifest {
attributes('Implementation-Title': baseName,
'Implementation-Version': project.version)
}
}
dependencies {
implementation project(':consensus:common')
implementation project(':consensus:ibft')
implementation project(':crypto')
implementation project(':ethereum:core')
implementation project(':ethereum:eth')
implementation project(':ethereum:jsonrpc')
implementation project(':ethereum:rlp')
implementation project(':ethereum:p2p')
implementation project(':services:kvstore')
implementation 'com.google.guava:guava'
implementation 'io.vertx:vertx-core'
testImplementation project( path: ':ethereum:core', configuration: 'testSupportArtifacts')
testImplementation project( path: ':consensus:ibft', configuration: 'testSupportArtifacts')
testImplementation group: 'junit', name: 'junit', version: '4.12'
testImplementation "org.awaitility:awaitility:3.1.2"
testImplementation "org.assertj:assertj-core:3.10.0"
testImplementation 'org.mockito:mockito-core'
}

@ -10,7 +10,7 @@
* 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.
*/
package tech.pegasys.pantheon.consensus.ibft;
package tech.pegasys.pantheon.consensus.ibftlegacy;
import tech.pegasys.pantheon.crypto.SECP256K1.Signature;
import tech.pegasys.pantheon.ethereum.core.Address;

@ -10,10 +10,11 @@
* 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.
*/
package tech.pegasys.pantheon.consensus.ibft;
package tech.pegasys.pantheon.consensus.ibftlegacy;
import tech.pegasys.pantheon.consensus.common.headervalidationrules.VoteValidationRule;
import tech.pegasys.pantheon.consensus.ibft.headervalidationrules.IbftExtraDataValidationRule;
import tech.pegasys.pantheon.consensus.ibft.IbftContext;
import tech.pegasys.pantheon.consensus.ibftlegacy.headervalidationrules.IbftExtraDataValidationRule;
import tech.pegasys.pantheon.ethereum.core.BlockHeader;
import tech.pegasys.pantheon.ethereum.core.Hash;
import tech.pegasys.pantheon.ethereum.mainnet.BlockHeaderValidator;

@ -10,7 +10,7 @@
* 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.
*/
package tech.pegasys.pantheon.consensus.ibft;
package tech.pegasys.pantheon.consensus.ibftlegacy;
import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Preconditions.checkNotNull;

@ -10,7 +10,7 @@
* 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.
*/
package tech.pegasys.pantheon.consensus.ibft;
package tech.pegasys.pantheon.consensus.ibftlegacy;
import tech.pegasys.pantheon.ethereum.core.Hash;

@ -10,8 +10,9 @@
* 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.
*/
package tech.pegasys.pantheon.consensus.ibft;
package tech.pegasys.pantheon.consensus.ibftlegacy;
import tech.pegasys.pantheon.consensus.ibft.IbftContext;
import tech.pegasys.pantheon.ethereum.mainnet.MutableProtocolSchedule;
import tech.pegasys.pantheon.ethereum.mainnet.ProtocolSchedule;

@ -10,11 +10,13 @@
* 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.
*/
package tech.pegasys.pantheon.consensus.ibft;
package tech.pegasys.pantheon.consensus.ibftlegacy;
import static tech.pegasys.pantheon.consensus.ibft.IbftBlockHeaderValidationRulesetFactory.ibftBlockHeaderValidator;
import static tech.pegasys.pantheon.consensus.ibftlegacy.IbftBlockHeaderValidationRulesetFactory.ibftBlockHeaderValidator;
import tech.pegasys.pantheon.consensus.common.EpochManager;
import tech.pegasys.pantheon.consensus.ibft.IbftBlockImporter;
import tech.pegasys.pantheon.consensus.ibft.IbftContext;
import tech.pegasys.pantheon.ethereum.core.Wei;
import tech.pegasys.pantheon.ethereum.mainnet.MainnetBlockBodyValidator;
import tech.pegasys.pantheon.ethereum.mainnet.MainnetBlockImporter;
@ -51,7 +53,7 @@ public class IbftProtocolSpecs {
new IbftBlockImporter(
new MainnetBlockImporter<>(
blockHeaderValidator, blockBodyValidator, blockProcessor),
new VoteTallyUpdater(epochManager)),
new IbftVoteTallyUpdater(epochManager)),
(time, parent, protocolContext) -> BigInteger.ONE)
.blockReward(Wei.ZERO)
.blockHashFunction(IbftBlockHashing::calculateHashOfIbftBlockOnChain)

@ -0,0 +1,87 @@
/*
* Copyright 2018 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.
*/
package tech.pegasys.pantheon.consensus.ibftlegacy;
import tech.pegasys.pantheon.consensus.common.EpochManager;
import tech.pegasys.pantheon.consensus.common.VoteTally;
import tech.pegasys.pantheon.consensus.common.VoteType;
import tech.pegasys.pantheon.consensus.ibft.VoteTallyUpdater;
import tech.pegasys.pantheon.ethereum.chain.Blockchain;
import tech.pegasys.pantheon.ethereum.core.Address;
import tech.pegasys.pantheon.ethereum.core.BlockHeader;
import tech.pegasys.pantheon.util.bytes.BytesValue;
import java.util.List;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
/**
* Provides the logic to extract vote tally state from the blockchain and update it as blocks are
* added.
*/
public class IbftVoteTallyUpdater implements VoteTallyUpdater {
private static final Logger LOG = LogManager.getLogger();
private static final Address NO_VOTE_SUBJECT = Address.wrap(BytesValue.wrap(new byte[20]));
private final EpochManager epochManager;
public IbftVoteTallyUpdater(final EpochManager epochManager) {
this.epochManager = epochManager;
}
/**
* Create a new VoteTally based on the current blockchain state.
*
* @param blockchain the blockchain to load the current state from
* @return a VoteTally reflecting the state of the blockchain head
*/
@Override
public VoteTally buildVoteTallyFromBlockchain(final Blockchain blockchain) {
final long chainHeadBlockNumber = blockchain.getChainHeadBlockNumber();
final long epochBlockNumber = epochManager.getLastEpochBlock(chainHeadBlockNumber);
LOG.info("Loading validator voting state starting from block {}", epochBlockNumber);
final BlockHeader epochBlock = blockchain.getBlockHeader(epochBlockNumber).get();
final List<Address> initialValidators =
IbftExtraData.decode(epochBlock.getExtraData()).getValidators();
final VoteTally voteTally = new VoteTally(initialValidators);
for (long blockNumber = epochBlockNumber + 1;
blockNumber <= chainHeadBlockNumber;
blockNumber++) {
updateForBlock(blockchain.getBlockHeader(blockNumber).get(), voteTally);
}
return voteTally;
}
/**
* Update the vote tally to reflect changes caused by appending a new block to the chain.
*
* @param header the header of the block being added
* @param voteTally the vote tally to update
*/
@Override
public void updateForBlock(final BlockHeader header, final VoteTally voteTally) {
final Address candidate = header.getCoinbase();
if (epochManager.isEpochBlock(header.getNumber())) {
voteTally.discardOutstandingVotes();
return;
}
if (!candidate.equals(NO_VOTE_SUBJECT)) {
final IbftExtraData ibftExtraData = IbftExtraData.decode(header.getExtraData());
final Address proposer = IbftBlockHashing.recoverProposerAddress(header, ibftExtraData);
voteTally.addVote(proposer, candidate, VoteType.fromNonce(header.getNonce()).get());
}
}
}

@ -0,0 +1,119 @@
/*
* Copyright 2018 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.
*/
package tech.pegasys.pantheon.consensus.ibftlegacy.blockcreation;
import tech.pegasys.pantheon.consensus.ibft.IbftContext;
import tech.pegasys.pantheon.consensus.ibftlegacy.IbftBlockHashing;
import tech.pegasys.pantheon.consensus.ibftlegacy.IbftExtraData;
import tech.pegasys.pantheon.consensus.ibftlegacy.IbftHelpers;
import tech.pegasys.pantheon.crypto.SECP256K1;
import tech.pegasys.pantheon.crypto.SECP256K1.KeyPair;
import tech.pegasys.pantheon.ethereum.ProtocolContext;
import tech.pegasys.pantheon.ethereum.blockcreation.AbstractBlockCreator;
import tech.pegasys.pantheon.ethereum.core.Address;
import tech.pegasys.pantheon.ethereum.core.BlockHashFunction;
import tech.pegasys.pantheon.ethereum.core.BlockHeader;
import tech.pegasys.pantheon.ethereum.core.BlockHeaderBuilder;
import tech.pegasys.pantheon.ethereum.core.Hash;
import tech.pegasys.pantheon.ethereum.core.PendingTransactions;
import tech.pegasys.pantheon.ethereum.core.SealableBlockHeader;
import tech.pegasys.pantheon.ethereum.core.Util;
import tech.pegasys.pantheon.ethereum.core.Wei;
import tech.pegasys.pantheon.ethereum.mainnet.ProtocolSchedule;
import tech.pegasys.pantheon.ethereum.mainnet.ScheduleBasedBlockHashFunction;
import java.util.function.Function;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
/**
* Responsible for producing a Block which conforms to IBFT validation rules (other than missing
* commit seals). Transactions and associated Hashes (stateroot, receipts etc.) are loaded into the
* Block in the base class as part of the transaction selection process.
*/
public class IbftBlockCreator extends AbstractBlockCreator<IbftContext> {
private static final Logger LOG = LogManager.getLogger();
private final KeyPair nodeKeys;
public IbftBlockCreator(
final Address coinbase,
final ExtraDataCalculator extraDataCalculator,
final PendingTransactions pendingTransactions,
final ProtocolContext<IbftContext> protocolContext,
final ProtocolSchedule<IbftContext> protocolSchedule,
final Function<Long, Long> gasLimitCalculator,
final KeyPair nodeKeys,
final Wei minTransactionGasPrice,
final BlockHeader parentHeader) {
super(
coinbase,
extraDataCalculator,
pendingTransactions,
protocolContext,
protocolSchedule,
gasLimitCalculator,
minTransactionGasPrice,
Util.publicKeyToAddress(nodeKeys.getPublicKey()),
parentHeader);
this.nodeKeys = nodeKeys;
}
/**
* Responsible for signing (hash of) the block (including MixHash and Nonce), and then injecting
* the seal into the extraData. This is called after a suitable set of transactions have been
* identified, and all resulting hashes have been inserted into the passed-in SealableBlockHeader.
*
* @param sealableBlockHeader A block header containing StateRoots, TransactionHashes etc.
* @return The blockhead which is to be added to the block being proposed.
*/
@Override
protected BlockHeader createFinalBlockHeader(final SealableBlockHeader sealableBlockHeader) {
final BlockHashFunction blockHashFunction =
ScheduleBasedBlockHashFunction.create(protocolSchedule);
final BlockHeaderBuilder builder =
BlockHeaderBuilder.create()
.populateFrom(sealableBlockHeader)
.mixHash(IbftHelpers.EXPECTED_MIX_HASH)
.nonce(0)
.blockHashFunction(blockHashFunction);
final IbftExtraData sealedExtraData = constructSignedExtraData(builder.buildBlockHeader());
// Replace the extraData in the BlockHeaderBuilder, and return header.
return builder.extraData(sealedExtraData.encode()).buildBlockHeader();
}
/**
* Produces an IbftExtraData object with a populated proposerSeal. The signature in the block is
* generated from the Hash of the header (minus proposer and committer seals) and the nodeKeys.
*
* @param headerToSign An almost fully populated header (proposer and committer seals are empty)
* @return Extra data containing the same vanity data and validators as extraData, however
* proposerSeal will also be populated.
*/
private IbftExtraData constructSignedExtraData(final BlockHeader headerToSign) {
final IbftExtraData extraData = IbftExtraData.decode(headerToSign.getExtraData());
final Hash hashToSign =
IbftBlockHashing.calculateDataHashForProposerSeal(headerToSign, extraData);
return new IbftExtraData(
extraData.getVanityData(),
extraData.getSeals(),
SECP256K1.sign(hashToSign, nodeKeys),
extraData.getValidators());
}
}

@ -10,10 +10,10 @@
* 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.
*/
package tech.pegasys.pantheon.consensus.ibft.blockcreation;
package tech.pegasys.pantheon.consensus.ibftlegacy.blockcreation;
import tech.pegasys.pantheon.consensus.common.ValidatorProvider;
import tech.pegasys.pantheon.consensus.ibft.IbftExtraData;
import tech.pegasys.pantheon.consensus.ibftlegacy.IbftExtraData;
import tech.pegasys.pantheon.ethereum.blockcreation.AbstractBlockCreator.ExtraDataCalculator;
import tech.pegasys.pantheon.ethereum.core.BlockHeader;
import tech.pegasys.pantheon.util.bytes.BytesValue;

@ -10,14 +10,14 @@
* 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.
*/
package tech.pegasys.pantheon.consensus.ibft.blockcreation;
package tech.pegasys.pantheon.consensus.ibftlegacy.blockcreation;
import static com.google.common.base.Preconditions.checkArgument;
import tech.pegasys.pantheon.consensus.common.ValidatorProvider;
import tech.pegasys.pantheon.consensus.ibft.ConsensusRoundIdentifier;
import tech.pegasys.pantheon.consensus.ibft.IbftBlockHashing;
import tech.pegasys.pantheon.consensus.ibft.IbftExtraData;
import tech.pegasys.pantheon.consensus.ibftlegacy.IbftBlockHashing;
import tech.pegasys.pantheon.consensus.ibftlegacy.IbftExtraData;
import tech.pegasys.pantheon.ethereum.chain.Blockchain;
import tech.pegasys.pantheon.ethereum.core.Address;
import tech.pegasys.pantheon.ethereum.core.BlockHeader;

@ -10,14 +10,13 @@
* 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.
*/
package tech.pegasys.pantheon.consensus.ibft.headervalidationrules;
import static tech.pegasys.pantheon.consensus.ibft.IbftHelpers.calculateRequiredValidatorQuorum;
package tech.pegasys.pantheon.consensus.ibftlegacy.headervalidationrules;
import tech.pegasys.pantheon.consensus.common.ValidatorProvider;
import tech.pegasys.pantheon.consensus.ibft.IbftBlockHashing;
import tech.pegasys.pantheon.consensus.ibft.IbftContext;
import tech.pegasys.pantheon.consensus.ibft.IbftExtraData;
import tech.pegasys.pantheon.consensus.ibftlegacy.IbftBlockHashing;
import tech.pegasys.pantheon.consensus.ibftlegacy.IbftExtraData;
import tech.pegasys.pantheon.consensus.ibftlegacy.IbftHelpers;
import tech.pegasys.pantheon.ethereum.ProtocolContext;
import tech.pegasys.pantheon.ethereum.core.Address;
import tech.pegasys.pantheon.ethereum.core.BlockHeader;
@ -112,7 +111,8 @@ public class IbftExtraDataValidationRule implements AttachedBlockHeaderValidatio
private boolean validateCommitters(
final Collection<Address> committers, final Collection<Address> storedValidators) {
final int minimumSealsRequired = calculateRequiredValidatorQuorum(storedValidators.size());
final int minimumSealsRequired =
IbftHelpers.calculateRequiredValidatorQuorum(storedValidators.size());
if (committers.size() < minimumSealsRequired) {
LOG.trace(
"Insufficient committers to seal block. (Required {}, received {})",

@ -10,7 +10,7 @@
* 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.
*/
package tech.pegasys.pantheon.consensus.ibft.protocol;
package tech.pegasys.pantheon.consensus.ibftlegacy.protocol;
import tech.pegasys.pantheon.ethereum.eth.messages.EthPV62;
import tech.pegasys.pantheon.ethereum.eth.messages.EthPV63;

@ -10,7 +10,7 @@
* 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.
*/
package tech.pegasys.pantheon.consensus.ibft.protocol;
package tech.pegasys.pantheon.consensus.ibftlegacy.protocol;
import tech.pegasys.pantheon.ethereum.chain.Blockchain;
import tech.pegasys.pantheon.ethereum.eth.EthProtocol;

@ -10,7 +10,7 @@
* 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.
*/
package tech.pegasys.pantheon.consensus.ibft;
package tech.pegasys.pantheon.consensus.ibftlegacy;
import static org.assertj.core.api.Assertions.assertThat;
@ -25,6 +25,7 @@ import tech.pegasys.pantheon.util.uint.UInt256;
import java.util.Arrays;
import java.util.List;
import org.assertj.core.api.Assertions;
import org.junit.Test;
public class IbftBlockHashingTest {
@ -58,7 +59,7 @@ public class IbftBlockHashingTest {
@Test
public void readValidatorListFromExtraData() {
final IbftExtraData ibftExtraData = IbftExtraData.decode(header.getExtraData());
assertThat(ibftExtraData.getValidators()).isEqualTo(VALIDATORS_IN_HEADER);
Assertions.assertThat(ibftExtraData.getValidators()).isEqualTo(VALIDATORS_IN_HEADER);
}
@Test

@ -10,14 +10,15 @@
* 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.
*/
package tech.pegasys.pantheon.consensus.ibft;
package tech.pegasys.pantheon.consensus.ibftlegacy;
import static java.util.Collections.emptyList;
import static java.util.Collections.singletonList;
import static org.assertj.core.api.Assertions.assertThat;
import static tech.pegasys.pantheon.consensus.ibft.IbftProtocolContextFixture.protocolContext;
import tech.pegasys.pantheon.consensus.common.VoteType;
import tech.pegasys.pantheon.consensus.ibft.IbftContext;
import tech.pegasys.pantheon.consensus.ibft.IbftProtocolContextFixture;
import tech.pegasys.pantheon.crypto.SECP256K1;
import tech.pegasys.pantheon.crypto.SECP256K1.KeyPair;
import tech.pegasys.pantheon.crypto.SECP256K1.Signature;
@ -53,7 +54,10 @@ public class IbftBlockHeaderValidationRulesetFactoryTest {
assertThat(
validator.validateHeader(
blockHeader, parentHeader, protocolContext(validators), HeaderValidationMode.FULL))
blockHeader,
parentHeader,
IbftProtocolContextFixture.protocolContext(validators),
HeaderValidationMode.FULL))
.isTrue();
}
@ -73,7 +77,10 @@ public class IbftBlockHeaderValidationRulesetFactoryTest {
assertThat(
validator.validateHeader(
blockHeader, parentHeader, protocolContext(validators), HeaderValidationMode.FULL))
blockHeader,
parentHeader,
IbftProtocolContextFixture.protocolContext(validators),
HeaderValidationMode.FULL))
.isFalse();
}

@ -10,7 +10,7 @@
* 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.
*/
package tech.pegasys.pantheon.consensus.ibft;
package tech.pegasys.pantheon.consensus.ibftlegacy;
import static org.assertj.core.api.Assertions.assertThat;
@ -26,6 +26,7 @@ import java.util.List;
import java.util.Random;
import com.google.common.collect.Lists;
import org.assertj.core.api.Assertions;
import org.bouncycastle.util.encoders.Hex;
import org.junit.Test;
@ -52,10 +53,10 @@ public class IbftExtraDataTest {
final IbftExtraData extraData = IbftExtraData.decode(bufferToInject);
assertThat(extraData.getVanityData()).isEqualTo(vanity_data);
Assertions.assertThat(extraData.getVanityData()).isEqualTo(vanity_data);
assertThat(extraData.getProposerSeal()).isEqualTo(proposerSeal);
assertThat(extraData.getSeals()).isEqualTo(committerSeals);
assertThat(extraData.getValidators()).isEqualTo(validators);
Assertions.assertThat(extraData.getSeals()).isEqualTo(committerSeals);
Assertions.assertThat(extraData.getValidators()).isEqualTo(validators);
}
@Test
@ -83,10 +84,10 @@ public class IbftExtraDataTest {
final IbftExtraData extraData = IbftExtraData.decode(bufferToInject);
assertThat(extraData.getVanityData()).isEqualTo(vanity_data);
Assertions.assertThat(extraData.getVanityData()).isEqualTo(vanity_data);
assertThat(extraData.getProposerSeal()).isEqualTo(proposerSeal);
assertThat(extraData.getSeals()).isEqualTo(committerSeals);
assertThat(extraData.getValidators()).isEqualTo(validators);
Assertions.assertThat(extraData.getSeals()).isEqualTo(committerSeals);
Assertions.assertThat(extraData.getValidators()).isEqualTo(validators);
}
@Test(expected = RLPException.class)

@ -10,7 +10,7 @@
* 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.
*/
package tech.pegasys.pantheon.consensus.ibft;
package tech.pegasys.pantheon.consensus.ibftlegacy;
import static java.util.Arrays.asList;
import static java.util.Collections.emptyList;
@ -41,7 +41,7 @@ import java.util.Optional;
import org.junit.Test;
public class VoteTallyUpdaterTest {
public class IbftVoteTallyUpdaterTest {
private static final long EPOCH_LENGTH = 30_000;
public static final Signature INVALID_SEAL =
@ -55,7 +55,8 @@ public class VoteTallyUpdaterTest {
private final Address validator1 =
Address.fromHexString("00dae27b350bae20c5652124af5d8b5cba001ec1");
private final VoteTallyUpdater updater = new VoteTallyUpdater(new EpochManager(EPOCH_LENGTH));
private final IbftVoteTallyUpdater updater =
new IbftVoteTallyUpdater(new EpochManager(EPOCH_LENGTH));
@Test
public void voteTallyUpdatedWithVoteFromBlock() {

@ -10,7 +10,7 @@
* 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.
*/
package tech.pegasys.pantheon.consensus.ibft.blockcreation;
package tech.pegasys.pantheon.consensus.ibftlegacy.blockcreation;
import static org.assertj.core.api.Assertions.assertThat;
import static org.mockito.ArgumentMatchers.any;
@ -20,10 +20,10 @@ import static tech.pegasys.pantheon.ethereum.core.InMemoryWorldState.createInMem
import tech.pegasys.pantheon.consensus.common.VoteProposer;
import tech.pegasys.pantheon.consensus.common.VoteTally;
import tech.pegasys.pantheon.consensus.ibft.IbftBlockHeaderValidationRulesetFactory;
import tech.pegasys.pantheon.consensus.ibft.IbftContext;
import tech.pegasys.pantheon.consensus.ibft.IbftExtraData;
import tech.pegasys.pantheon.consensus.ibft.IbftProtocolSchedule;
import tech.pegasys.pantheon.consensus.ibftlegacy.IbftBlockHeaderValidationRulesetFactory;
import tech.pegasys.pantheon.consensus.ibftlegacy.IbftExtraData;
import tech.pegasys.pantheon.consensus.ibftlegacy.IbftProtocolSchedule;
import tech.pegasys.pantheon.crypto.SECP256K1.KeyPair;
import tech.pegasys.pantheon.ethereum.ProtocolContext;
import tech.pegasys.pantheon.ethereum.chain.MutableBlockchain;

@ -10,7 +10,7 @@
* 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.
*/
package tech.pegasys.pantheon.consensus.ibft.blockcreation;
package tech.pegasys.pantheon.consensus.ibftlegacy.blockcreation;
import static org.assertj.core.api.Assertions.assertThat;
import static org.mockito.ArgumentMatchers.anyLong;
@ -19,8 +19,8 @@ import static org.mockito.Mockito.when;
import tech.pegasys.pantheon.consensus.common.VoteTally;
import tech.pegasys.pantheon.consensus.ibft.ConsensusRoundIdentifier;
import tech.pegasys.pantheon.consensus.ibft.IbftBlockHashing;
import tech.pegasys.pantheon.consensus.ibft.IbftExtraData;
import tech.pegasys.pantheon.consensus.ibftlegacy.IbftBlockHashing;
import tech.pegasys.pantheon.consensus.ibftlegacy.IbftExtraData;
import tech.pegasys.pantheon.crypto.SECP256K1;
import tech.pegasys.pantheon.crypto.SECP256K1.KeyPair;
import tech.pegasys.pantheon.crypto.SECP256K1.Signature;

@ -10,16 +10,16 @@
* 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.
*/
package tech.pegasys.pantheon.consensus.ibft.headervalidationrules;
package tech.pegasys.pantheon.consensus.ibftlegacy.headervalidationrules;
import static java.util.Collections.emptyList;
import static java.util.Collections.singletonList;
import static org.assertj.core.api.Assertions.assertThat;
import tech.pegasys.pantheon.consensus.common.VoteTally;
import tech.pegasys.pantheon.consensus.ibft.IbftBlockHashing;
import tech.pegasys.pantheon.consensus.ibft.IbftContext;
import tech.pegasys.pantheon.consensus.ibft.IbftExtraData;
import tech.pegasys.pantheon.consensus.ibftlegacy.IbftBlockHashing;
import tech.pegasys.pantheon.consensus.ibftlegacy.IbftExtraData;
import tech.pegasys.pantheon.crypto.SECP256K1;
import tech.pegasys.pantheon.crypto.SECP256K1.KeyPair;
import tech.pegasys.pantheon.crypto.SECP256K1.Signature;
@ -37,6 +37,7 @@ import java.util.List;
import java.util.stream.Collectors;
import com.google.common.collect.Lists;
import org.assertj.core.api.Assertions;
import org.junit.Test;
public class IbftExtraDataValidationRuleTest {
@ -141,7 +142,7 @@ public class IbftExtraDataValidationRuleTest {
// Note that no committer seals are in the header's IBFT extra data.
final IbftExtraData headerExtraData = IbftExtraData.decode(header.getExtraData());
assertThat(headerExtraData.getSeals().size()).isEqualTo(0);
Assertions.assertThat(headerExtraData.getSeals().size()).isEqualTo(0);
assertThat(extraDataValidationRule.validate(header, null, context)).isFalse();
}

@ -26,6 +26,7 @@ dependencies {
implementation project(':consensus:common')
implementation project(':consensus:clique')
implementation project(':consensus:ibft')
implementation project(':consensus:ibftlegacy')
implementation project(':consensus:clique')
implementation project(':ethereum:eth')
implementation project(':ethereum:core')

@ -21,14 +21,14 @@ import tech.pegasys.pantheon.consensus.ibft.IbftChainObserver;
import tech.pegasys.pantheon.consensus.ibft.IbftContext;
import tech.pegasys.pantheon.consensus.ibft.IbftEventQueue;
import tech.pegasys.pantheon.consensus.ibft.IbftProcessor;
import tech.pegasys.pantheon.consensus.ibft.IbftProtocolSchedule;
import tech.pegasys.pantheon.consensus.ibft.IbftStateMachine;
import tech.pegasys.pantheon.consensus.ibft.VoteTallyUpdater;
import tech.pegasys.pantheon.consensus.ibft.blockcreation.IbftBlockMiner;
import tech.pegasys.pantheon.consensus.ibft.protocol.IbftProtocolManager;
import tech.pegasys.pantheon.consensus.ibft.protocol.IbftSubProtocol;
import tech.pegasys.pantheon.consensus.ibft.protocol.Istanbul64Protocol;
import tech.pegasys.pantheon.consensus.ibft.protocol.Istanbul64ProtocolManager;
import tech.pegasys.pantheon.consensus.ibftlegacy.IbftProtocolSchedule;
import tech.pegasys.pantheon.consensus.ibftlegacy.IbftVoteTallyUpdater;
import tech.pegasys.pantheon.consensus.ibftlegacy.protocol.Istanbul64Protocol;
import tech.pegasys.pantheon.consensus.ibftlegacy.protocol.Istanbul64ProtocolManager;
import tech.pegasys.pantheon.crypto.SECP256K1.KeyPair;
import tech.pegasys.pantheon.ethereum.ProtocolContext;
import tech.pegasys.pantheon.ethereum.blockcreation.AbstractMiningCoordinator;
@ -129,7 +129,7 @@ public class IbftPantheonController implements PantheonController<IbftContext, I
new EpochManager(IbftProtocolSchedule.getEpochLength(Optional.of(ibftConfig)));
final VoteTally voteTally =
new VoteTallyUpdater(epochManager).buildVoteTallyFromBlockchain(blockchain);
new IbftVoteTallyUpdater(epochManager).buildVoteTallyFromBlockchain(blockchain);
final VoteProposer voteProposer = new VoteProposer();

@ -13,7 +13,7 @@
package tech.pegasys.pantheon.controller;
import tech.pegasys.pantheon.consensus.clique.CliqueProtocolSchedule;
import tech.pegasys.pantheon.consensus.ibft.IbftProtocolSchedule;
import tech.pegasys.pantheon.consensus.ibftlegacy.IbftProtocolSchedule;
import tech.pegasys.pantheon.crypto.SECP256K1.KeyPair;
import tech.pegasys.pantheon.ethereum.ProtocolContext;
import tech.pegasys.pantheon.ethereum.blockcreation.AbstractBlockCreator;

@ -17,6 +17,7 @@ include 'consensus'
include 'consensus:clique'
include 'consensus:common'
include 'consensus:ibft'
include 'consensus:ibftlegacy'
include 'crypto'
include 'ethereum:p2p'
include 'ethereum:mock-p2p'

Loading…
Cancel
Save