Separating Validator voting from block serialisation values (#263)

The consensus-mechanism specific block values have been separated
from the VoteTallyUpdater such that there is a single updater rather
than one per consensus mechanism.

This has necessitated the creation of a custom serialiser/
deserialiser for each mechanism instead.

This change will ultimatley bleed through to the proposed votes and
their insertion to mined blocks.
tmohay 6 years ago committed by GitHub
parent cc4f4df6a5
commit c2e98152e0
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
  1. 76
      consensus/clique/src/main/java/tech/pegasys/pantheon/consensus/clique/CliqueVoteTallyUpdater.java
  2. 70
      consensus/clique/src/main/java/tech/pegasys/pantheon/consensus/clique/CliqueVotingBlockInterface.java
  3. 5
      consensus/clique/src/main/java/tech/pegasys/pantheon/consensus/clique/VoteTallyCache.java
  4. 4
      consensus/clique/src/main/java/tech/pegasys/pantheon/consensus/clique/headervalidationrules/CoinbaseHeaderValidationRule.java
  5. 8
      consensus/clique/src/main/java/tech/pegasys/pantheon/consensus/clique/jsonrpc/CliqueJsonRpcMethodsFactory.java
  6. 126
      consensus/clique/src/test/java/tech/pegasys/pantheon/consensus/clique/CliqueVotingBlockInterfaceTest.java
  7. 7
      consensus/clique/src/test/java/tech/pegasys/pantheon/consensus/clique/VoteTallyCacheTest.java
  8. 1
      consensus/common/build.gradle
  9. 67
      consensus/common/src/main/java/tech/pegasys/pantheon/consensus/common/CastVote.java
  10. 32
      consensus/common/src/main/java/tech/pegasys/pantheon/consensus/common/ValidatorVotePolarity.java
  11. 30
      consensus/common/src/main/java/tech/pegasys/pantheon/consensus/common/VoteBlockInterface.java
  12. 33
      consensus/common/src/main/java/tech/pegasys/pantheon/consensus/common/VoteTally.java
  13. 30
      consensus/common/src/main/java/tech/pegasys/pantheon/consensus/common/VoteTallyUpdater.java
  14. 12
      consensus/common/src/test/java/tech/pegasys/pantheon/consensus/common/VoteProposerTest.java
  15. 152
      consensus/common/src/test/java/tech/pegasys/pantheon/consensus/common/VoteTallyTest.java
  16. 92
      consensus/common/src/test/java/tech/pegasys/pantheon/consensus/common/VoteTallyUpdaterTest.java
  17. 1
      consensus/ibft/src/main/java/tech/pegasys/pantheon/consensus/ibft/IbftBlockImporter.java
  18. 35
      consensus/ibft/src/main/java/tech/pegasys/pantheon/consensus/ibft/VoteTallyUpdater.java
  19. 1
      consensus/ibft/src/test/java/tech/pegasys/pantheon/consensus/ibft/IbftBlockImporterTest.java
  20. 70
      consensus/ibftlegacy/src/main/java/tech/pegasys/pantheon/consensus/ibftlegacy/IbftLegacyVotingBlockInterface.java
  21. 3
      consensus/ibftlegacy/src/main/java/tech/pegasys/pantheon/consensus/ibftlegacy/IbftProtocolSchedule.java
  22. 127
      consensus/ibftlegacy/src/test/java/tech/pegasys/pantheon/consensus/ibftlegacy/IbftLegacyVotingBlockInterfaceTest.java
  23. 198
      consensus/ibftlegacy/src/test/java/tech/pegasys/pantheon/consensus/ibftlegacy/IbftVoteTallyUpdaterTest.java
  24. 55
      consensus/ibftlegacy/src/test/java/tech/pegasys/pantheon/consensus/ibftlegacy/TestHelpers.java
  25. 7
      pantheon/src/main/java/tech/pegasys/pantheon/controller/CliquePantheonController.java
  26. 6
      pantheon/src/main/java/tech/pegasys/pantheon/controller/IbftPantheonController.java

@ -1,76 +0,0 @@
/*
* 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.clique;
import static org.apache.logging.log4j.LogManager.getLogger;
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.Logger;
public class CliqueVoteTallyUpdater {
private static final Logger LOG = getLogger();
public static final Address NO_VOTE_SUBJECT = Address.wrap(BytesValue.wrap(new byte[20]));
private final EpochManager epochManager;
public CliqueVoteTallyUpdater(final EpochManager epochManager) {
this.epochManager = epochManager;
}
public VoteTally buildVoteTallyFromBlockchain(final Blockchain blockchain) {
final long chainHeadBlockNumber = blockchain.getChainHeadBlockNumber();
final long epochBlockNumber = epochManager.getLastEpochBlock(chainHeadBlockNumber);
LOG.debug("Loading validator voting state starting from block {}", epochBlockNumber);
final BlockHeader epochBlock = blockchain.getBlockHeader(epochBlockNumber).get();
final List<Address> initialValidators =
CliqueExtraData.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
*/
public void updateForBlock(final BlockHeader header, final VoteTally voteTally) {
final Address candidate = header.getCoinbase();
if (epochManager.isEpochBlock(header.getNumber())) {
// epoch blocks are not allowed to include a vote
voteTally.discardOutstandingVotes();
return;
}
if (!candidate.equals(NO_VOTE_SUBJECT)) {
final CliqueExtraData extraData = CliqueExtraData.decode(header.getExtraData());
final Address proposer = CliqueBlockHashing.recoverProposerAddress(header, extraData);
voteTally.addVote(proposer, candidate, VoteType.fromNonce(header.getNonce()).get());
}
}
}

@ -0,0 +1,70 @@
/*
* 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.clique;
import tech.pegasys.pantheon.consensus.common.CastVote;
import tech.pegasys.pantheon.consensus.common.ValidatorVotePolarity;
import tech.pegasys.pantheon.consensus.common.VoteBlockInterface;
import tech.pegasys.pantheon.ethereum.core.Address;
import tech.pegasys.pantheon.ethereum.core.BlockHeader;
import tech.pegasys.pantheon.ethereum.core.BlockHeaderBuilder;
import tech.pegasys.pantheon.util.bytes.BytesValue;
import java.util.List;
import java.util.Optional;
import com.google.common.collect.ImmutableBiMap;
public class CliqueVotingBlockInterface implements VoteBlockInterface {
public static final Address NO_VOTE_SUBJECT =
Address.wrap(BytesValue.wrap(new byte[Address.SIZE]));
private static final ImmutableBiMap<ValidatorVotePolarity, Long> voteToValue =
ImmutableBiMap.of(
ValidatorVotePolarity.ADD, 0xFFFFFFFFFFFFFFFFL,
ValidatorVotePolarity.DROP, 0x0L);
@Override
public Optional<CastVote> extractVoteFromHeader(final BlockHeader header) {
final Address candidate = header.getCoinbase();
if (!candidate.equals(NO_VOTE_SUBJECT)) {
final CliqueExtraData cliqueExtraData = CliqueExtraData.decode(header.getExtraData());
final Address proposer = CliqueBlockHashing.recoverProposerAddress(header, cliqueExtraData);
final ValidatorVotePolarity votePolarity = voteToValue.inverse().get(header.getNonce());
final Address recipient = header.getCoinbase();
return Optional.of(new CastVote(votePolarity, proposer, recipient));
}
return Optional.empty();
}
@Override
public BlockHeaderBuilder insertVoteToHeaderBuilder(
final BlockHeaderBuilder builder, final Optional<CastVote> vote) {
if (vote.isPresent()) {
final CastVote voteToCast = vote.get();
builder.nonce(voteToValue.get(voteToCast.getVotePolarity()));
builder.coinbase(voteToCast.getRecipient());
} else {
builder.nonce(voteToValue.get(ValidatorVotePolarity.DROP));
builder.coinbase(NO_VOTE_SUBJECT);
}
return builder;
}
@Override
public List<Address> validatorsInBlock(final BlockHeader header) {
return CliqueExtraData.decode(header.getExtraData()).getValidators();
}
}

@ -16,6 +16,7 @@ import static com.google.common.base.Preconditions.checkNotNull;
import tech.pegasys.pantheon.consensus.common.EpochManager;
import tech.pegasys.pantheon.consensus.common.VoteTally;
import tech.pegasys.pantheon.consensus.common.VoteTallyUpdater;
import tech.pegasys.pantheon.ethereum.chain.Blockchain;
import tech.pegasys.pantheon.ethereum.core.BlockHeader;
import tech.pegasys.pantheon.ethereum.core.Hash;
@ -32,14 +33,14 @@ public class VoteTallyCache {
private final Blockchain blockchain;
private final EpochManager epochManager;
private final CliqueVoteTallyUpdater voteTallyUpdater;
private final VoteTallyUpdater voteTallyUpdater;
private final Cache<Hash, VoteTally> voteTallyCache =
CacheBuilder.newBuilder().maximumSize(100).build();
public VoteTallyCache(
final Blockchain blockchain,
final CliqueVoteTallyUpdater voteTallyUpdater,
final VoteTallyUpdater voteTallyUpdater,
final EpochManager epochManager) {
checkNotNull(blockchain);
checkNotNull(voteTallyUpdater);

@ -12,7 +12,7 @@
*/
package tech.pegasys.pantheon.consensus.clique.headervalidationrules;
import tech.pegasys.pantheon.consensus.clique.CliqueVoteTallyUpdater;
import tech.pegasys.pantheon.consensus.clique.CliqueVotingBlockInterface;
import tech.pegasys.pantheon.consensus.common.EpochManager;
import tech.pegasys.pantheon.ethereum.core.BlockHeader;
import tech.pegasys.pantheon.ethereum.mainnet.DetachedBlockHeaderValidationRule;
@ -30,7 +30,7 @@ public class CoinbaseHeaderValidationRule implements DetachedBlockHeaderValidati
// are allowed to be cast on epoch blocks
public boolean validate(final BlockHeader header, final BlockHeader parent) {
if (epochManager.isEpochBlock(header.getNumber())) {
return header.getCoinbase().equals(CliqueVoteTallyUpdater.NO_VOTE_SUBJECT);
return header.getCoinbase().equals(CliqueVotingBlockInterface.NO_VOTE_SUBJECT);
}
return true;
}

@ -13,7 +13,7 @@
package tech.pegasys.pantheon.consensus.clique.jsonrpc;
import tech.pegasys.pantheon.consensus.clique.CliqueContext;
import tech.pegasys.pantheon.consensus.clique.CliqueVoteTallyUpdater;
import tech.pegasys.pantheon.consensus.clique.CliqueVotingBlockInterface;
import tech.pegasys.pantheon.consensus.clique.VoteTallyCache;
import tech.pegasys.pantheon.consensus.clique.jsonrpc.methods.CliqueGetSigners;
import tech.pegasys.pantheon.consensus.clique.jsonrpc.methods.CliqueGetSignersAtHash;
@ -22,6 +22,7 @@ import tech.pegasys.pantheon.consensus.clique.jsonrpc.methods.Discard;
import tech.pegasys.pantheon.consensus.clique.jsonrpc.methods.Propose;
import tech.pegasys.pantheon.consensus.common.EpochManager;
import tech.pegasys.pantheon.consensus.common.VoteProposer;
import tech.pegasys.pantheon.consensus.common.VoteTallyUpdater;
import tech.pegasys.pantheon.ethereum.ProtocolContext;
import tech.pegasys.pantheon.ethereum.chain.MutableBlockchain;
import tech.pegasys.pantheon.ethereum.db.WorldStateArchive;
@ -70,7 +71,8 @@ public class CliqueJsonRpcMethodsFactory {
private VoteTallyCache createVoteTallyCache(
final ProtocolContext<CliqueContext> context, final MutableBlockchain blockchain) {
final EpochManager epochManager = context.getConsensusState().getEpochManager();
final CliqueVoteTallyUpdater cliqueVoteTallyUpdater = new CliqueVoteTallyUpdater(epochManager);
return new VoteTallyCache(blockchain, cliqueVoteTallyUpdater, epochManager);
final VoteTallyUpdater voteTallyUpdater =
new VoteTallyUpdater(epochManager, new CliqueVotingBlockInterface());
return new VoteTallyCache(blockchain, voteTallyUpdater, epochManager);
}
}

@ -0,0 +1,126 @@
/*
* 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.clique;
import static java.util.Collections.singletonList;
import static org.assertj.core.api.Assertions.assertThat;
import static tech.pegasys.pantheon.consensus.common.ValidatorVotePolarity.ADD;
import static tech.pegasys.pantheon.consensus.common.ValidatorVotePolarity.DROP;
import tech.pegasys.pantheon.consensus.common.CastVote;
import tech.pegasys.pantheon.crypto.SECP256K1.KeyPair;
import tech.pegasys.pantheon.ethereum.core.Address;
import tech.pegasys.pantheon.ethereum.core.AddressHelpers;
import tech.pegasys.pantheon.ethereum.core.BlockHeader;
import tech.pegasys.pantheon.ethereum.core.BlockHeaderBuilder;
import tech.pegasys.pantheon.ethereum.core.BlockHeaderTestFixture;
import tech.pegasys.pantheon.ethereum.core.Util;
import tech.pegasys.pantheon.ethereum.mainnet.MainnetBlockHashFunction;
import java.util.List;
import java.util.Optional;
import org.junit.Test;
public class CliqueVotingBlockInterfaceTest {
private static final KeyPair proposerKeys = KeyPair.generate();
private static final Address proposerAddress =
Util.publicKeyToAddress(proposerKeys.getPublicKey());
private static final List<Address> validatorList = singletonList(proposerAddress);
private final CliqueVotingBlockInterface blockInterface = new CliqueVotingBlockInterface();
private final BlockHeaderTestFixture headerBuilder = new BlockHeaderTestFixture();
private final BlockHeader header =
TestHelpers.createCliqueSignedBlockHeader(headerBuilder, proposerKeys, validatorList);
private final BlockHeaderBuilder builder =
BlockHeaderBuilder.fromHeader(headerBuilder.buildHeader())
.blockHashFunction(MainnetBlockHashFunction::createHash);
@Test
public void headerWithZeroCoinbaseReturnsAnEmptyVote() {
final BlockHeaderTestFixture headerBuilder = new BlockHeaderTestFixture();
headerBuilder.coinbase(AddressHelpers.ofValue(0));
assertThat(blockInterface.extractVoteFromHeader(headerBuilder.buildHeader())).isEmpty();
}
@Test
public void headerWithNonceOfZeroReportsDropVote() {
headerBuilder.coinbase(AddressHelpers.ofValue(1)).nonce(0L);
final Optional<CastVote> extractedVote = blockInterface.extractVoteFromHeader(header);
assertThat(extractedVote).contains(new CastVote(DROP, proposerAddress, header.getCoinbase()));
}
@Test
public void headerWithNonceOfMaxLongReportsAddVote() {
headerBuilder.coinbase(AddressHelpers.ofValue(2)).nonce(0xFFFFFFFFFFFFFFFFL);
final BlockHeader header =
TestHelpers.createCliqueSignedBlockHeader(headerBuilder, proposerKeys, validatorList);
final Optional<CastVote> extractedVote = blockInterface.extractVoteFromHeader(header);
assertThat(extractedVote).contains(new CastVote(ADD, proposerAddress, header.getCoinbase()));
}
@Test
public void blendingAddVoteToHeaderResultsInHeaderWithNonceOfMaxLong() {
final CastVote vote = new CastVote(ADD, AddressHelpers.ofValue(1), AddressHelpers.ofValue(2));
final BlockHeaderBuilder builderWithVote =
blockInterface.insertVoteToHeaderBuilder(builder, Optional.of(vote));
final BlockHeader header = builderWithVote.buildBlockHeader();
assertThat(header.getCoinbase()).isEqualTo(vote.getRecipient());
assertThat(header.getNonce()).isEqualTo(0xFFFFFFFFFFFFFFFFL);
}
@Test
public void blendingDropVoteToHeaderResultsInHeaderWithNonceOfZero() {
final CastVote vote = new CastVote(DROP, AddressHelpers.ofValue(1), AddressHelpers.ofValue(2));
final BlockHeaderBuilder builderWithVote =
blockInterface.insertVoteToHeaderBuilder(builder, Optional.of(vote));
final BlockHeader header = builderWithVote.buildBlockHeader();
assertThat(header.getCoinbase()).isEqualTo(vote.getRecipient());
assertThat(header.getNonce()).isEqualTo(0x0L);
}
@Test
public void nonVoteBlendedIntoHeaderResultsInACoinbaseOfZero() {
final BlockHeaderBuilder builderWithVote =
blockInterface.insertVoteToHeaderBuilder(builder, Optional.empty());
final BlockHeader header = builderWithVote.buildBlockHeader();
assertThat(header.getCoinbase()).isEqualTo(AddressHelpers.ofValue(0));
assertThat(header.getNonce()).isEqualTo(0x0L);
}
@Test
public void extractsValidatorsFromHeader() {
final BlockHeader header =
TestHelpers.createCliqueSignedBlockHeader(headerBuilder, proposerKeys, validatorList);
final List<Address> extractedValidators = blockInterface.validatorsInBlock(header);
assertThat(extractedValidators).isEqualTo(validatorList);
}
}

@ -22,6 +22,7 @@ import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.verifyZeroInteractions;
import tech.pegasys.pantheon.consensus.common.EpochManager;
import tech.pegasys.pantheon.consensus.common.VoteTallyUpdater;
import tech.pegasys.pantheon.crypto.SECP256K1.Signature;
import tech.pegasys.pantheon.ethereum.chain.MutableBlockchain;
import tech.pegasys.pantheon.ethereum.core.AddressHelpers;
@ -79,7 +80,7 @@ public class VoteTallyCacheTest {
@Test
public void parentBlockVoteTallysAreCachedWhenChildVoteTallyRequested() {
final CliqueVoteTallyUpdater tallyUpdater = mock(CliqueVoteTallyUpdater.class);
final VoteTallyUpdater tallyUpdater = mock(VoteTallyUpdater.class);
final VoteTallyCache cache =
new VoteTallyCache(blockChain, tallyUpdater, new EpochManager(30_000));
@ -105,7 +106,7 @@ public class VoteTallyCacheTest {
@Test
public void exceptionThrownIfNoParentBlockExists() {
final CliqueVoteTallyUpdater tallyUpdater = mock(CliqueVoteTallyUpdater.class);
final VoteTallyUpdater tallyUpdater = mock(VoteTallyUpdater.class);
final VoteTallyCache cache =
new VoteTallyCache(blockChain, tallyUpdater, new EpochManager(30_000));
@ -119,7 +120,7 @@ public class VoteTallyCacheTest {
@Test
public void walkBackStopsWhenACachedVoteTallyIsFound() {
final CliqueVoteTallyUpdater tallyUpdater = mock(CliqueVoteTallyUpdater.class);
final VoteTallyUpdater tallyUpdater = mock(VoteTallyUpdater.class);
final VoteTallyCache cache =
new VoteTallyCache(blockChain, tallyUpdater, new EpochManager(30_000));

@ -30,6 +30,7 @@ dependencies {
implementation project(':util')
implementation 'com.google.guava:guava'
testImplementation project(':crypto')
testImplementation project( path: ':ethereum:core', configuration: 'testSupportArtifacts')
testImplementation 'junit:junit'
testImplementation "org.assertj:assertj-core"

@ -0,0 +1,67 @@
/*
* 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.common;
import tech.pegasys.pantheon.ethereum.core.Address;
import java.util.Objects;
import com.google.common.base.Preconditions;
public class CastVote {
private final ValidatorVotePolarity votePolarity;
private final Address proposer;
private final Address recipient;
public CastVote(
final ValidatorVotePolarity votePolarity, final Address proposer, final Address recipient) {
Preconditions.checkNotNull(votePolarity);
Preconditions.checkNotNull(proposer);
Preconditions.checkNotNull(recipient);
this.votePolarity = votePolarity;
this.proposer = proposer;
this.recipient = recipient;
}
public ValidatorVotePolarity getVotePolarity() {
return votePolarity;
}
public Address getProposer() {
return proposer;
}
public Address getRecipient() {
return recipient;
}
@Override
public boolean equals(final Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
CastVote castVote = (CastVote) o;
return votePolarity == castVote.votePolarity
&& Objects.equals(proposer, castVote.proposer)
&& Objects.equals(recipient, castVote.recipient);
}
@Override
public int hashCode() {
return Objects.hash(votePolarity, proposer, recipient);
}
}

@ -0,0 +1,32 @@
/*
* 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.common;
/**
* Determines if a validator vote is indicating that they should be added, or removed. This does not
* attempt to determine how said vote should be serialised/deserialised.
*/
public enum ValidatorVotePolarity implements ValidatorVote {
ADD,
DROP;
@Override
public boolean isAddVote() {
return this.equals(ADD);
}
@Override
public boolean isDropVote() {
return this.equals(DROP);
}
}

@ -0,0 +1,30 @@
/*
* 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.common;
import tech.pegasys.pantheon.ethereum.core.Address;
import tech.pegasys.pantheon.ethereum.core.BlockHeader;
import tech.pegasys.pantheon.ethereum.core.BlockHeaderBuilder;
import java.util.List;
import java.util.Optional;
public interface VoteBlockInterface {
Optional<CastVote> extractVoteFromHeader(final BlockHeader header);
BlockHeaderBuilder insertVoteToHeaderBuilder(
final BlockHeaderBuilder builder, final Optional<CastVote> vote);
List<Address> validatorsInBlock(final BlockHeader header);
}

@ -52,35 +52,32 @@ public class VoteTally implements ValidatorProvider {
* Add a vote to the current tally. The current validator list will be updated if this vote takes
* the tally past the required votes to approve the change.
*
* @param proposer the address of the validator casting the vote via block proposal
* @param subject the validator the vote is about
* @param validatorVote the type of vote, either add or drop
* @param castVote The vote which was cast in a block header.
*/
public void addVote(
final Address proposer, final Address subject, final ValidatorVote validatorVote) {
public void addVote(final CastVote castVote) {
final Set<Address> addVotesForSubject =
addVotesBySubject.computeIfAbsent(subject, target -> new HashSet<>());
addVotesBySubject.computeIfAbsent(castVote.getRecipient(), target -> new HashSet<>());
final Set<Address> removeVotesForSubject =
removeVotesBySubject.computeIfAbsent(subject, target -> new HashSet<>());
removeVotesBySubject.computeIfAbsent(castVote.getRecipient(), target -> new HashSet<>());
if (validatorVote.isAddVote()) {
addVotesForSubject.add(proposer);
removeVotesForSubject.remove(proposer);
if (castVote.getVotePolarity().isAddVote()) {
addVotesForSubject.add(castVote.getProposer());
removeVotesForSubject.remove(castVote.getProposer());
} else {
removeVotesForSubject.add(proposer);
addVotesForSubject.remove(proposer);
removeVotesForSubject.add(castVote.getProposer());
addVotesForSubject.remove(castVote.getProposer());
}
final int validatorLimit = validatorLimit();
if (addVotesForSubject.size() >= validatorLimit) {
currentValidators.add(subject);
discardOutstandingVotesFor(subject);
currentValidators.add(castVote.getRecipient());
discardOutstandingVotesFor(castVote.getRecipient());
}
if (removeVotesForSubject.size() >= validatorLimit) {
currentValidators.remove(subject);
discardOutstandingVotesFor(subject);
addVotesBySubject.values().forEach(votes -> votes.remove(subject));
removeVotesBySubject.values().forEach(votes -> votes.remove(subject));
currentValidators.remove(castVote.getRecipient());
discardOutstandingVotesFor(castVote.getRecipient());
addVotesBySubject.values().forEach(votes -> votes.remove(castVote.getRecipient()));
removeVotesBySubject.values().forEach(votes -> votes.remove(castVote.getRecipient()));
}
}

@ -10,18 +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.ibftlegacy;
package tech.pegasys.pantheon.consensus.common;
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 java.util.Optional;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
@ -30,15 +26,17 @@ 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 {
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;
private final VoteBlockInterface blockInterface;
public IbftVoteTallyUpdater(final EpochManager epochManager) {
public VoteTallyUpdater(
final EpochManager epochManager, final VoteBlockInterface blockInterface) {
this.epochManager = epochManager;
this.blockInterface = blockInterface;
}
/**
@ -47,14 +45,12 @@ public class IbftVoteTallyUpdater implements VoteTallyUpdater {
* @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 List<Address> initialValidators = blockInterface.validatorsInBlock(epochBlock);
final VoteTally voteTally = new VoteTally(initialValidators);
for (long blockNumber = epochBlockNumber + 1;
blockNumber <= chainHeadBlockNumber;
@ -70,18 +66,12 @@ public class IbftVoteTallyUpdater implements VoteTallyUpdater {
* @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());
}
final Optional<CastVote> vote = blockInterface.extractVoteFromHeader(header);
vote.ifPresent(voteTally::addVote);
}
}

@ -13,8 +13,8 @@
package tech.pegasys.pantheon.consensus.common;
import static org.assertj.core.api.Assertions.assertThat;
import static tech.pegasys.pantheon.consensus.common.VoteType.ADD;
import static tech.pegasys.pantheon.consensus.common.VoteType.DROP;
import static tech.pegasys.pantheon.consensus.common.ValidatorVotePolarity.ADD;
import static tech.pegasys.pantheon.consensus.common.ValidatorVotePolarity.DROP;
import tech.pegasys.pantheon.ethereum.core.Address;
@ -158,7 +158,7 @@ public class VoteProposerTest {
proposer.drop(a1);
final VoteTally tally = new VoteTally(Arrays.asList(a2, a3));
tally.addVote(localAddress, a1, ADD);
tally.addVote(new CastVote(ADD, localAddress, a1));
assertThat(proposer.getVote(localAddress, tally))
.isEqualTo(Optional.of(new AbstractMap.SimpleEntry<>(a1, VoteType.DROP)));
@ -174,7 +174,7 @@ public class VoteProposerTest {
proposer.auth(a1);
final VoteTally tally = new VoteTally(Arrays.asList(a2, a3));
tally.addVote(localAddress, a1, DROP);
tally.addVote(new CastVote(DROP, localAddress, a1));
assertThat(proposer.getVote(localAddress, tally))
.isEqualTo(Optional.of(new AbstractMap.SimpleEntry<>(a1, VoteType.ADD)));
@ -188,7 +188,7 @@ public class VoteProposerTest {
final Address a3 = Address.fromHexString("3");
final VoteTally tally = new VoteTally(Arrays.asList(a1, a2, a3));
tally.addVote(localAddress, a1, ADD);
tally.addVote(new CastVote(ADD, localAddress, a1));
proposer.drop(a1);
@ -204,7 +204,7 @@ public class VoteProposerTest {
final Address a3 = Address.fromHexString("3");
final VoteTally tally = new VoteTally(Arrays.asList(a1, a2, a3));
tally.addVote(localAddress, a1, DROP);
tally.addVote(new CastVote(DROP, localAddress, a1));
proposer.auth(a1);

@ -15,6 +15,8 @@ package tech.pegasys.pantheon.consensus.common;
import static java.util.Arrays.asList;
import static java.util.Collections.singletonList;
import static org.assertj.core.api.Assertions.assertThat;
import static tech.pegasys.pantheon.consensus.common.ValidatorVotePolarity.ADD;
import static tech.pegasys.pantheon.consensus.common.ValidatorVotePolarity.DROP;
import tech.pegasys.pantheon.ethereum.core.Address;
@ -36,8 +38,8 @@ public class VoteTallyTest {
@Test
public void validatorsAreNotAddedBeforeRequiredVoteCountReached() {
final VoteTally voteTally = fourValidators();
voteTally.addVote(validator1, validator5, VoteType.ADD);
voteTally.addVote(validator2, validator5, VoteType.ADD);
voteTally.addVote(new CastVote(ADD, validator1, validator5));
voteTally.addVote(new CastVote(ADD, validator2, validator5));
assertThat(voteTally.getCurrentValidators())
.containsExactly(validator1, validator2, validator3, validator4);
@ -46,9 +48,9 @@ public class VoteTallyTest {
@Test
public void validatorAddedToListWhenMoreThanHalfOfProposersVoteToAdd() {
final VoteTally voteTally = fourValidators();
voteTally.addVote(validator1, validator5, VoteType.ADD);
voteTally.addVote(validator2, validator5, VoteType.ADD);
voteTally.addVote(validator3, validator5, VoteType.ADD);
voteTally.addVote(new CastVote(ADD, validator1, validator5));
voteTally.addVote(new CastVote(ADD, validator2, validator5));
voteTally.addVote(new CastVote(ADD, validator3, validator5));
assertThat(voteTally.getCurrentValidators())
.containsExactly(validator1, validator2, validator3, validator4, validator5);
@ -58,9 +60,9 @@ public class VoteTallyTest {
public void validatorsAreAddedInCorrectOrder() {
final VoteTally voteTally =
new VoteTally(asList(validator1, validator2, validator3, validator5));
voteTally.addVote(validator1, validator4, VoteType.ADD);
voteTally.addVote(validator2, validator4, VoteType.ADD);
voteTally.addVote(validator3, validator4, VoteType.ADD);
voteTally.addVote(new CastVote(ADD, validator1, validator4));
voteTally.addVote(new CastVote(ADD, validator2, validator4));
voteTally.addVote(new CastVote(ADD, validator3, validator4));
assertThat(voteTally.getCurrentValidators())
.containsExactly(validator1, validator2, validator3, validator4, validator5);
@ -69,9 +71,9 @@ public class VoteTallyTest {
@Test
public void duplicateVotesFromSameProposerAreIgnored() {
final VoteTally voteTally = fourValidators();
voteTally.addVote(validator1, validator5, VoteType.ADD);
voteTally.addVote(validator2, validator5, VoteType.ADD);
voteTally.addVote(validator2, validator5, VoteType.ADD);
voteTally.addVote(new CastVote(ADD, validator1, validator5));
voteTally.addVote(new CastVote(ADD, validator2, validator5));
voteTally.addVote(new CastVote(ADD, validator2, validator5));
assertThat(voteTally.getCurrentValidators())
.containsExactly(validator1, validator2, validator3, validator4);
@ -80,10 +82,10 @@ public class VoteTallyTest {
@Test
public void proposerChangingAddVoteToDropBeforeLimitReachedDiscardsAddVote() {
final VoteTally voteTally = fourValidators();
voteTally.addVote(validator1, validator5, VoteType.ADD);
voteTally.addVote(validator1, validator5, VoteType.DROP);
voteTally.addVote(validator2, validator5, VoteType.ADD);
voteTally.addVote(validator3, validator5, VoteType.ADD);
voteTally.addVote(new CastVote(ADD, validator1, validator5));
voteTally.addVote(new CastVote(DROP, validator1, validator5));
voteTally.addVote(new CastVote(ADD, validator2, validator5));
voteTally.addVote(new CastVote(ADD, validator3, validator5));
assertThat(voteTally.getCurrentValidators())
.containsExactly(validator1, validator2, validator3, validator4);
@ -92,10 +94,10 @@ public class VoteTallyTest {
@Test
public void proposerChangingAddVoteToDropAfterLimitReachedPreservesAddVote() {
final VoteTally voteTally = fourValidators();
voteTally.addVote(validator1, validator5, VoteType.ADD);
voteTally.addVote(validator2, validator5, VoteType.ADD);
voteTally.addVote(validator3, validator5, VoteType.ADD);
voteTally.addVote(validator1, validator5, VoteType.DROP);
voteTally.addVote(new CastVote(ADD, validator1, validator5));
voteTally.addVote(new CastVote(ADD, validator2, validator5));
voteTally.addVote(new CastVote(ADD, validator3, validator5));
voteTally.addVote(new CastVote(DROP, validator1, validator5));
assertThat(voteTally.getCurrentValidators())
.containsExactly(validator1, validator2, validator3, validator4, validator5);
@ -105,23 +107,23 @@ public class VoteTallyTest {
public void clearVotesAboutAValidatorWhenItIsAdded() {
final VoteTally voteTally = fourValidators();
// Vote to add validator5
voteTally.addVote(validator1, validator5, VoteType.ADD);
voteTally.addVote(validator2, validator5, VoteType.ADD);
voteTally.addVote(validator3, validator5, VoteType.ADD);
voteTally.addVote(new CastVote(ADD, validator1, validator5));
voteTally.addVote(new CastVote(ADD, validator2, validator5));
voteTally.addVote(new CastVote(ADD, validator3, validator5));
assertThat(voteTally.getCurrentValidators())
.containsExactly(validator1, validator2, validator3, validator4, validator5);
// Then vote it back out
voteTally.addVote(validator2, validator5, VoteType.DROP);
voteTally.addVote(validator3, validator5, VoteType.DROP);
voteTally.addVote(validator4, validator5, VoteType.DROP);
voteTally.addVote(new CastVote(DROP, validator2, validator5));
voteTally.addVote(new CastVote(DROP, validator3, validator5));
voteTally.addVote(new CastVote(DROP, validator4, validator5));
assertThat(voteTally.getCurrentValidators())
.containsExactly(validator1, validator2, validator3, validator4);
// And then start voting to add it back in, but validator1's vote should have been discarded
voteTally.addVote(validator2, validator5, VoteType.ADD);
voteTally.addVote(validator3, validator5, VoteType.ADD);
voteTally.addVote(new CastVote(ADD, validator2, validator5));
voteTally.addVote(new CastVote(ADD, validator3, validator5));
assertThat(voteTally.getCurrentValidators())
.containsExactly(validator1, validator2, validator3, validator4);
}
@ -129,7 +131,7 @@ public class VoteTallyTest {
@Test
public void requiresASingleVoteWhenThereIsOnlyOneValidator() {
final VoteTally voteTally = new VoteTally(singletonList(validator1));
voteTally.addVote(validator1, validator2, VoteType.ADD);
voteTally.addVote(new CastVote(ADD, validator1, validator2));
assertThat(voteTally.getCurrentValidators()).containsExactly(validator1, validator2);
}
@ -137,11 +139,11 @@ public class VoteTallyTest {
@Test
public void requiresTwoVotesWhenThereAreTwoValidators() {
final VoteTally voteTally = new VoteTally(asList(validator1, validator2));
voteTally.addVote(validator1, validator3, VoteType.ADD);
voteTally.addVote(new CastVote(ADD, validator1, validator3));
assertThat(voteTally.getCurrentValidators()).containsExactly(validator1, validator2);
voteTally.addVote(validator2, validator3, VoteType.ADD);
voteTally.addVote(new CastVote(ADD, validator2, validator3));
assertThat(voteTally.getCurrentValidators())
.containsExactly(validator1, validator2, validator3);
}
@ -149,10 +151,10 @@ public class VoteTallyTest {
@Test
public void resetVotes() {
final VoteTally voteTally = fourValidators();
voteTally.addVote(validator1, validator5, VoteType.ADD);
voteTally.addVote(validator2, validator5, VoteType.ADD);
voteTally.addVote(new CastVote(ADD, validator1, validator5));
voteTally.addVote(new CastVote(ADD, validator2, validator5));
voteTally.discardOutstandingVotes();
voteTally.addVote(validator3, validator5, VoteType.ADD);
voteTally.addVote(new CastVote(ADD, validator3, validator5));
assertThat(voteTally.getCurrentValidators())
.containsExactly(validator1, validator2, validator3, validator4);
@ -161,8 +163,8 @@ public class VoteTallyTest {
@Test
public void validatorsAreNotRemovedBeforeRequiredVoteCountReached() {
final VoteTally voteTally = fourValidators();
voteTally.addVote(validator1, validator4, VoteType.DROP);
voteTally.addVote(validator2, validator4, VoteType.DROP);
voteTally.addVote(new CastVote(DROP, validator1, validator4));
voteTally.addVote(new CastVote(DROP, validator2, validator4));
assertThat(voteTally.getCurrentValidators())
.containsExactly(validator1, validator2, validator3, validator4);
@ -171,9 +173,9 @@ public class VoteTallyTest {
@Test
public void validatorRemovedFromListWhenMoreThanHalfOfProposersVoteToDrop() {
final VoteTally voteTally = fourValidators();
voteTally.addVote(validator1, validator4, VoteType.DROP);
voteTally.addVote(validator2, validator4, VoteType.DROP);
voteTally.addVote(validator3, validator4, VoteType.DROP);
voteTally.addVote(new CastVote(DROP, validator1, validator4));
voteTally.addVote(new CastVote(DROP, validator2, validator4));
voteTally.addVote(new CastVote(DROP, validator3, validator4));
assertThat(voteTally.getCurrentValidators())
.containsExactly(validator1, validator2, validator3);
@ -182,9 +184,9 @@ public class VoteTallyTest {
@Test
public void validatorsAreInCorrectOrderAfterRemoval() {
final VoteTally voteTally = new VoteTally(asList(validator1, validator2, validator4));
voteTally.addVote(validator1, validator3, VoteType.DROP);
voteTally.addVote(validator2, validator3, VoteType.DROP);
voteTally.addVote(validator4, validator3, VoteType.DROP);
voteTally.addVote(new CastVote(DROP, validator1, validator3));
voteTally.addVote(new CastVote(DROP, validator2, validator3));
voteTally.addVote(new CastVote(DROP, validator4, validator3));
assertThat(voteTally.getCurrentValidators())
.containsExactly(validator1, validator2, validator4);
@ -193,9 +195,9 @@ public class VoteTallyTest {
@Test
public void duplicateDropVotesFromSameProposerAreIgnored() {
final VoteTally voteTally = fourValidators();
voteTally.addVote(validator1, validator4, VoteType.DROP);
voteTally.addVote(validator2, validator4, VoteType.DROP);
voteTally.addVote(validator2, validator4, VoteType.DROP);
voteTally.addVote(new CastVote(DROP, validator1, validator4));
voteTally.addVote(new CastVote(DROP, validator2, validator4));
voteTally.addVote(new CastVote(DROP, validator2, validator4));
assertThat(voteTally.getCurrentValidators())
.containsExactly(validator1, validator2, validator3, validator4);
@ -204,10 +206,10 @@ public class VoteTallyTest {
@Test
public void proposerChangingDropVoteToAddBeforeLimitReachedDiscardsDropVote() {
final VoteTally voteTally = fourValidators();
voteTally.addVote(validator1, validator4, VoteType.DROP);
voteTally.addVote(validator1, validator4, VoteType.ADD);
voteTally.addVote(validator2, validator4, VoteType.DROP);
voteTally.addVote(validator3, validator4, VoteType.DROP);
voteTally.addVote(new CastVote(DROP, validator1, validator4));
voteTally.addVote(new CastVote(ADD, validator1, validator4));
voteTally.addVote(new CastVote(DROP, validator2, validator4));
voteTally.addVote(new CastVote(DROP, validator3, validator4));
assertThat(voteTally.getCurrentValidators())
.containsExactly(validator1, validator2, validator3, validator4);
@ -216,10 +218,10 @@ public class VoteTallyTest {
@Test
public void proposerChangingDropVoteToAddAfterLimitReachedPreservesDropVote() {
final VoteTally voteTally = fourValidators();
voteTally.addVote(validator1, validator4, VoteType.DROP);
voteTally.addVote(validator2, validator4, VoteType.DROP);
voteTally.addVote(validator3, validator4, VoteType.DROP);
voteTally.addVote(validator1, validator4, VoteType.ADD);
voteTally.addVote(new CastVote(DROP, validator1, validator4));
voteTally.addVote(new CastVote(DROP, validator2, validator4));
voteTally.addVote(new CastVote(DROP, validator3, validator4));
voteTally.addVote(new CastVote(ADD, validator1, validator4));
assertThat(voteTally.getCurrentValidators())
.containsExactly(validator1, validator2, validator3);
@ -228,19 +230,19 @@ public class VoteTallyTest {
@Test
public void removedValidatorsVotesAreDiscarded() {
final VoteTally voteTally = fourValidators();
voteTally.addVote(validator4, validator5, VoteType.ADD);
voteTally.addVote(validator4, validator3, VoteType.DROP);
voteTally.addVote(new CastVote(ADD, validator4, validator5));
voteTally.addVote(new CastVote(DROP, validator4, validator3));
voteTally.addVote(validator1, validator4, VoteType.DROP);
voteTally.addVote(validator2, validator4, VoteType.DROP);
voteTally.addVote(validator3, validator4, VoteType.DROP);
voteTally.addVote(new CastVote(DROP, validator1, validator4));
voteTally.addVote(new CastVote(DROP, validator2, validator4));
voteTally.addVote(new CastVote(DROP, validator3, validator4));
assertThat(voteTally.getCurrentValidators())
.containsExactly(validator1, validator2, validator3);
// Now adding only requires 2 votes (>50% of the 3 remaining validators)
// but validator4's vote no longer counts
voteTally.addVote(validator1, validator5, VoteType.ADD);
voteTally.addVote(validator1, validator3, VoteType.DROP);
voteTally.addVote(new CastVote(ADD, validator1, validator5));
voteTally.addVote(new CastVote(DROP, validator1, validator3));
assertThat(voteTally.getCurrentValidators())
.containsExactly(validator1, validator2, validator3);
@ -251,23 +253,23 @@ public class VoteTallyTest {
final VoteTally voteTally =
new VoteTally(asList(validator1, validator2, validator3, validator4, validator5));
// Vote to remove validator5
voteTally.addVote(validator1, validator5, VoteType.DROP);
voteTally.addVote(validator2, validator5, VoteType.DROP);
voteTally.addVote(validator3, validator5, VoteType.DROP);
voteTally.addVote(new CastVote(DROP, validator1, validator5));
voteTally.addVote(new CastVote(DROP, validator2, validator5));
voteTally.addVote(new CastVote(DROP, validator3, validator5));
assertThat(voteTally.getCurrentValidators())
.containsExactly(validator1, validator2, validator3, validator4);
// Then vote it back in
voteTally.addVote(validator2, validator5, VoteType.ADD);
voteTally.addVote(validator3, validator5, VoteType.ADD);
voteTally.addVote(validator4, validator5, VoteType.ADD);
voteTally.addVote(new CastVote(ADD, validator2, validator5));
voteTally.addVote(new CastVote(ADD, validator3, validator5));
voteTally.addVote(new CastVote(ADD, validator4, validator5));
assertThat(voteTally.getCurrentValidators())
.containsExactly(validator1, validator2, validator3, validator4, validator5);
// And then start voting to drop it again, but validator1's vote should have been discarded
voteTally.addVote(validator2, validator5, VoteType.DROP);
voteTally.addVote(validator3, validator5, VoteType.DROP);
voteTally.addVote(new CastVote(DROP, validator2, validator5));
voteTally.addVote(new CastVote(DROP, validator3, validator5));
assertThat(voteTally.getCurrentValidators())
.containsExactly(validator1, validator2, validator3, validator4, validator5);
}
@ -275,25 +277,25 @@ public class VoteTallyTest {
@Test
public void trackMultipleOngoingVotesIndependently() {
final VoteTally voteTally = fourValidators();
voteTally.addVote(validator1, validator5, VoteType.ADD);
voteTally.addVote(validator1, validator3, VoteType.DROP);
voteTally.addVote(new CastVote(ADD, validator1, validator5));
voteTally.addVote(new CastVote(DROP, validator1, validator3));
voteTally.addVote(validator2, validator5, VoteType.ADD);
voteTally.addVote(validator2, validator1, VoteType.DROP);
voteTally.addVote(new CastVote(ADD, validator2, validator5));
voteTally.addVote(new CastVote(DROP, validator2, validator1));
// Neither vote has enough votes to complete.
assertThat(voteTally.getCurrentValidators())
.containsExactly(validator1, validator2, validator3, validator4);
voteTally.addVote(validator3, validator5, VoteType.ADD);
voteTally.addVote(validator3, validator1, VoteType.DROP);
voteTally.addVote(new CastVote(ADD, validator3, validator5));
voteTally.addVote(new CastVote(DROP, validator3, validator1));
// Validator 5 now has 3 votes and is added
assertThat(voteTally.getCurrentValidators())
.containsExactly(validator1, validator2, validator3, validator4, validator5);
voteTally.addVote(validator4, validator5, VoteType.ADD);
voteTally.addVote(validator4, validator1, VoteType.DROP);
voteTally.addVote(new CastVote(ADD, validator4, validator5));
voteTally.addVote(new CastVote(DROP, validator4, validator1));
// Validator 1 now gets dropped.
assertThat(voteTally.getCurrentValidators())

@ -10,41 +10,32 @@
* 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.clique;
package tech.pegasys.pantheon.consensus.common;
import static java.util.Arrays.asList;
import static java.util.Collections.singletonList;
import static org.assertj.core.api.Assertions.assertThat;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.verifyNoMoreInteractions;
import static org.mockito.Mockito.verifyZeroInteractions;
import static org.mockito.Mockito.when;
import static tech.pegasys.pantheon.consensus.common.ValidatorVotePolarity.ADD;
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.crypto.SECP256K1;
import tech.pegasys.pantheon.crypto.SECP256K1.KeyPair;
import tech.pegasys.pantheon.crypto.SECP256K1.Signature;
import tech.pegasys.pantheon.ethereum.chain.MutableBlockchain;
import tech.pegasys.pantheon.ethereum.core.Address;
import tech.pegasys.pantheon.ethereum.core.BlockHeader;
import tech.pegasys.pantheon.ethereum.core.BlockHeaderTestFixture;
import tech.pegasys.pantheon.ethereum.core.Hash;
import tech.pegasys.pantheon.util.bytes.BytesValue;
import java.math.BigInteger;
import java.util.List;
import java.util.Optional;
import org.junit.Test;
public class CliqueVoteTallyUpdaterTest {
public class VoteTallyUpdaterTest {
private static final long EPOCH_LENGTH = 30_000;
public static final Signature INVALID_SEAL =
Signature.create(BigInteger.ONE, BigInteger.ONE, (byte) 0);
private final VoteTally voteTally = mock(VoteTally.class);
private final MutableBlockchain blockchain = mock(MutableBlockchain.class);
private final KeyPair proposerKeyPair = KeyPair.generate();
@ -54,30 +45,31 @@ public class CliqueVoteTallyUpdaterTest {
private final Address validator1 =
Address.fromHexString("00dae27b350bae20c5652124af5d8b5cba001ec1");
private final CliqueVoteTallyUpdater updater =
new CliqueVoteTallyUpdater(new EpochManager(EPOCH_LENGTH));
private final VoteBlockInterface serialiser = mock(VoteBlockInterface.class);
private final VoteTallyUpdater updater =
new VoteTallyUpdater(new EpochManager(EPOCH_LENGTH), serialiser);
@Test
public void voteTallyUpdatedWithVoteFromBlock() {
public void voteTallyUpdatedWithAddVote() {
when(serialiser.extractVoteFromHeader(any()))
.thenReturn(Optional.of(new CastVote(ADD, proposerAddress, subject)));
final BlockHeaderTestFixture headerBuilder = new BlockHeaderTestFixture();
headerBuilder.number(1);
headerBuilder.nonce(VoteType.ADD.getNonceValue());
headerBuilder.coinbase(subject);
addProposer(headerBuilder);
headerBuilder.number(EPOCH_LENGTH - 1);
final BlockHeader header = headerBuilder.buildHeader();
updater.updateForBlock(header, voteTally);
verify(voteTally).addVote(proposerAddress, subject, VoteType.ADD);
verify(voteTally).addVote(new CastVote(ADD, proposerAddress, subject));
}
@Test
public void voteTallyNotUpdatedWhenBlockHasNoVoteSubject() {
when(serialiser.extractVoteFromHeader(any())).thenReturn(Optional.empty());
final BlockHeaderTestFixture headerBuilder = new BlockHeaderTestFixture();
headerBuilder.number(1);
headerBuilder.nonce(VoteType.ADD.getNonceValue());
headerBuilder.coinbase(Address.fromHexString("0000000000000000000000000000000000000000"));
addProposer(headerBuilder);
headerBuilder.number(EPOCH_LENGTH - 1);
final BlockHeader header = headerBuilder.buildHeader();
updater.updateForBlock(header, voteTally);
@ -87,11 +79,10 @@ public class CliqueVoteTallyUpdaterTest {
@Test
public void outstandingVotesDiscardedWhenEpochReached() {
when(serialiser.extractVoteFromHeader(any())).thenReturn(Optional.empty());
final BlockHeaderTestFixture headerBuilder = new BlockHeaderTestFixture();
headerBuilder.number(EPOCH_LENGTH);
headerBuilder.nonce(VoteType.ADD.getNonceValue());
headerBuilder.coinbase(Address.fromHexString("0000000000000000000000000000000000000000"));
addProposer(headerBuilder);
final BlockHeader header = headerBuilder.buildHeader();
updater.updateForBlock(header, voteTally);
@ -102,11 +93,9 @@ public class CliqueVoteTallyUpdaterTest {
@Test
public void buildVoteTallyByExtractingValidatorsFromGenesisBlock() {
when(serialiser.validatorsInBlock(any())).thenReturn(asList(subject, validator1));
final BlockHeaderTestFixture headerBuilder = new BlockHeaderTestFixture();
headerBuilder.number(0);
headerBuilder.nonce(VoteType.ADD.getNonceValue());
headerBuilder.coinbase(Address.fromHexString("0000000000000000000000000000000000000000"));
addProposer(headerBuilder, asList(subject, validator1));
final BlockHeader header = headerBuilder.buildHeader();
when(blockchain.getChainHeadBlockNumber()).thenReturn(EPOCH_LENGTH);
@ -118,11 +107,8 @@ public class CliqueVoteTallyUpdaterTest {
@Test
public void buildVoteTallyByExtractingValidatorsFromEpochBlock() {
when(serialiser.validatorsInBlock(any())).thenReturn(asList(subject, validator1));
final BlockHeaderTestFixture headerBuilder = new BlockHeaderTestFixture();
headerBuilder.number(EPOCH_LENGTH);
headerBuilder.nonce(VoteType.ADD.getNonceValue());
headerBuilder.coinbase(Address.fromHexString("0000000000000000000000000000000000000000"));
addProposer(headerBuilder, asList(subject, validator1));
final BlockHeader header = headerBuilder.buildHeader();
when(blockchain.getChainHeadBlockNumber()).thenReturn(EPOCH_LENGTH);
@ -134,15 +120,15 @@ public class CliqueVoteTallyUpdaterTest {
@Test
public void addVotesFromBlocksAfterMostRecentEpoch() {
when(serialiser.validatorsInBlock(any())).thenReturn(asList(validator1));
when(serialiser.extractVoteFromHeader(any()))
.thenReturn(Optional.of(new CastVote(ADD, proposerAddress, subject)));
final BlockHeaderTestFixture headerBuilder = new BlockHeaderTestFixture();
headerBuilder.number(EPOCH_LENGTH);
headerBuilder.nonce(VoteType.ADD.getNonceValue());
headerBuilder.coinbase(Address.fromHexString("0000000000000000000000000000000000000000"));
addProposer(headerBuilder, singletonList(validator1));
final BlockHeader epochHeader = headerBuilder.buildHeader();
headerBuilder.number(EPOCH_LENGTH + 1);
headerBuilder.coinbase(subject);
final BlockHeader voteBlockHeader = headerBuilder.buildHeader();
when(blockchain.getChainHeadBlockNumber()).thenReturn(EPOCH_LENGTH + 1);
@ -152,32 +138,4 @@ public class CliqueVoteTallyUpdaterTest {
final VoteTally voteTally = updater.buildVoteTallyFromBlockchain(blockchain);
assertThat(voteTally.getCurrentValidators()).containsExactly(subject, validator1);
}
private void addProposer(final BlockHeaderTestFixture builder) {
addProposer(builder, singletonList(proposerAddress));
}
private void addProposer(final BlockHeaderTestFixture builder, final List<Address> validators) {
final CliqueExtraData initialIbftExtraData =
new CliqueExtraData(
BytesValue.wrap(new byte[CliqueExtraData.EXTRA_VANITY_LENGTH]),
INVALID_SEAL,
validators);
builder.extraData(initialIbftExtraData.encode());
final BlockHeader header = builder.buildHeader();
final Hash proposerSealHash =
CliqueBlockHashing.calculateDataHashForProposerSeal(header, initialIbftExtraData);
final Signature proposerSignature = SECP256K1.sign(proposerSealHash, proposerKeyPair);
final CliqueExtraData sealedData =
new CliqueExtraData(
BytesValue.wrap(new byte[CliqueExtraData.EXTRA_VANITY_LENGTH]),
proposerSignature,
validators);
builder.extraData(sealedData.encode());
}
}

@ -12,6 +12,7 @@
*/
package tech.pegasys.pantheon.consensus.ibft;
import tech.pegasys.pantheon.consensus.common.VoteTallyUpdater;
import tech.pegasys.pantheon.ethereum.ProtocolContext;
import tech.pegasys.pantheon.ethereum.core.Block;
import tech.pegasys.pantheon.ethereum.core.BlockHeader;

@ -1,35 +0,0 @@
/*
* 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.ibft;
import tech.pegasys.pantheon.consensus.common.VoteTally;
import tech.pegasys.pantheon.ethereum.chain.Blockchain;
import tech.pegasys.pantheon.ethereum.core.BlockHeader;
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
*/
VoteTally buildVoteTallyFromBlockchain(final Blockchain blockchain);
/**
* 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
*/
void updateForBlock(final BlockHeader header, final VoteTally voteTally);
}

@ -20,6 +20,7 @@ import static org.mockito.Mockito.when;
import tech.pegasys.pantheon.consensus.common.VoteProposer;
import tech.pegasys.pantheon.consensus.common.VoteTally;
import tech.pegasys.pantheon.consensus.common.VoteTallyUpdater;
import tech.pegasys.pantheon.ethereum.ProtocolContext;
import tech.pegasys.pantheon.ethereum.chain.MutableBlockchain;
import tech.pegasys.pantheon.ethereum.core.Block;

@ -0,0 +1,70 @@
/*
* 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.CastVote;
import tech.pegasys.pantheon.consensus.common.ValidatorVotePolarity;
import tech.pegasys.pantheon.consensus.common.VoteBlockInterface;
import tech.pegasys.pantheon.ethereum.core.Address;
import tech.pegasys.pantheon.ethereum.core.BlockHeader;
import tech.pegasys.pantheon.ethereum.core.BlockHeaderBuilder;
import tech.pegasys.pantheon.util.bytes.BytesValue;
import java.util.List;
import java.util.Optional;
import com.google.common.collect.ImmutableBiMap;
public class IbftLegacyVotingBlockInterface implements VoteBlockInterface {
private static final Address NO_VOTE_SUBJECT =
Address.wrap(BytesValue.wrap(new byte[Address.SIZE]));
private static final ImmutableBiMap<ValidatorVotePolarity, Long> voteToValue =
ImmutableBiMap.of(
ValidatorVotePolarity.ADD, 0xFFFFFFFFFFFFFFFFL,
ValidatorVotePolarity.DROP, 0x0L);
@Override
public Optional<CastVote> extractVoteFromHeader(final BlockHeader header) {
final Address candidate = header.getCoinbase();
if (!candidate.equals(NO_VOTE_SUBJECT)) {
final IbftExtraData ibftExtraData = IbftExtraData.decode(header.getExtraData());
final Address proposer = IbftBlockHashing.recoverProposerAddress(header, ibftExtraData);
final ValidatorVotePolarity votePolarity = voteToValue.inverse().get(header.getNonce());
final Address recipient = header.getCoinbase();
return Optional.of(new CastVote(votePolarity, proposer, recipient));
}
return Optional.empty();
}
@Override
public BlockHeaderBuilder insertVoteToHeaderBuilder(
final BlockHeaderBuilder builder, final Optional<CastVote> vote) {
if (vote.isPresent()) {
final CastVote voteToCast = vote.get();
builder.nonce(voteToValue.get(voteToCast.getVotePolarity()));
builder.coinbase(voteToCast.getRecipient());
} else {
builder.nonce(voteToValue.get(ValidatorVotePolarity.DROP));
builder.coinbase(NO_VOTE_SUBJECT);
}
return builder;
}
@Override
public List<Address> validatorsInBlock(final BlockHeader header) {
return IbftExtraData.decode(header.getExtraData()).getValidators();
}
}

@ -17,6 +17,7 @@ import static tech.pegasys.pantheon.consensus.ibftlegacy.IbftBlockHeaderValidati
import tech.pegasys.pantheon.config.GenesisConfigOptions;
import tech.pegasys.pantheon.config.IbftConfigOptions;
import tech.pegasys.pantheon.consensus.common.EpochManager;
import tech.pegasys.pantheon.consensus.common.VoteTallyUpdater;
import tech.pegasys.pantheon.consensus.ibft.IbftBlockImporter;
import tech.pegasys.pantheon.consensus.ibft.IbftContext;
import tech.pegasys.pantheon.ethereum.core.Wei;
@ -59,7 +60,7 @@ public class IbftProtocolSchedule {
new IbftBlockImporter(
new MainnetBlockImporter<>(
blockHeaderValidator, blockBodyValidator, blockProcessor),
new IbftVoteTallyUpdater(epochManager)),
new VoteTallyUpdater(epochManager, new IbftLegacyVotingBlockInterface())),
(time, parent, protocolContext) -> BigInteger.ONE)
.blockReward(Wei.ZERO)
.blockHashFunction(IbftBlockHashing::calculateHashOfIbftBlockOnChain);

@ -0,0 +1,127 @@
/*
* 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 static java.util.Collections.singletonList;
import static org.assertj.core.api.Assertions.assertThat;
import static tech.pegasys.pantheon.consensus.common.ValidatorVotePolarity.ADD;
import static tech.pegasys.pantheon.consensus.common.ValidatorVotePolarity.DROP;
import tech.pegasys.pantheon.consensus.common.CastVote;
import tech.pegasys.pantheon.crypto.SECP256K1.KeyPair;
import tech.pegasys.pantheon.ethereum.core.Address;
import tech.pegasys.pantheon.ethereum.core.AddressHelpers;
import tech.pegasys.pantheon.ethereum.core.BlockHeader;
import tech.pegasys.pantheon.ethereum.core.BlockHeaderBuilder;
import tech.pegasys.pantheon.ethereum.core.BlockHeaderTestFixture;
import tech.pegasys.pantheon.ethereum.core.Util;
import tech.pegasys.pantheon.ethereum.mainnet.MainnetBlockHashFunction;
import java.util.List;
import java.util.Optional;
import org.junit.Before;
import org.junit.Test;
public class IbftLegacyVotingBlockInterfaceTest {
private static final KeyPair proposerKeys = KeyPair.generate();
private static final Address proposerAddress =
Util.publicKeyToAddress(proposerKeys.getPublicKey());
private static final List<Address> validatorList = singletonList(proposerAddress);
private final BlockHeaderTestFixture headerBuilder = new BlockHeaderTestFixture();
private final IbftLegacyVotingBlockInterface blockInterface =
new IbftLegacyVotingBlockInterface();
private final BlockHeaderBuilder builder =
BlockHeaderBuilder.fromHeader(headerBuilder.buildHeader())
.blockHashFunction(MainnetBlockHashFunction::createHash);
@Before
public void setup() {
// must set "number" to ensure extradata is correctly deserialised during hashing.
headerBuilder.coinbase(AddressHelpers.ofValue(0)).number(1);
}
@Test
public void headerWithZeroCoinbaseReturnsAnEmptyVote() {
assertThat(blockInterface.extractVoteFromHeader(headerBuilder.buildHeader())).isEmpty();
}
@Test
public void headerWithNonceOfZeroReportsDropVote() {
headerBuilder.nonce(0x0L).coinbase(AddressHelpers.ofValue(2));
final BlockHeader header =
TestHelpers.createIbftSignedBlockHeader(headerBuilder, proposerKeys, validatorList);
final Optional<CastVote> extractedVote = blockInterface.extractVoteFromHeader(header);
assertThat(extractedVote).contains(new CastVote(DROP, proposerAddress, header.getCoinbase()));
}
@Test
public void headerWithNonceOfMaxLongReportsAddVote() {
headerBuilder.nonce(0xFFFFFFFFFFFFFFFFL).coinbase(AddressHelpers.ofValue(2));
final BlockHeader header =
TestHelpers.createIbftSignedBlockHeader(headerBuilder, proposerKeys, validatorList);
final Optional<CastVote> extractedVote = blockInterface.extractVoteFromHeader(header);
assertThat(extractedVote).contains(new CastVote(ADD, proposerAddress, header.getCoinbase()));
}
@Test
public void blendingAddVoteToHeaderResultsInHeaderWithNonceOfMaxLong() {
final CastVote vote = new CastVote(ADD, AddressHelpers.ofValue(1), AddressHelpers.ofValue(2));
final BlockHeaderBuilder builderWithVote =
blockInterface.insertVoteToHeaderBuilder(builder, Optional.of(vote));
final BlockHeader header = builderWithVote.buildBlockHeader();
assertThat(header.getCoinbase()).isEqualTo(vote.getRecipient());
assertThat(header.getNonce()).isEqualTo(0xFFFFFFFFFFFFFFFFL);
}
@Test
public void blendingDropVoteToHeaderResultsInHeaderWithNonceOfZero() {
final CastVote vote = new CastVote(DROP, AddressHelpers.ofValue(1), AddressHelpers.ofValue(2));
final BlockHeaderBuilder builderWithVote =
blockInterface.insertVoteToHeaderBuilder(builder, Optional.of(vote));
final BlockHeader header = builderWithVote.buildBlockHeader();
assertThat(header.getCoinbase()).isEqualTo(vote.getRecipient());
assertThat(header.getNonce()).isEqualTo(0x0L);
}
@Test
public void nonVoteBlendedIntoHeaderResultsInACoinbaseOfZero() {
final BlockHeaderBuilder builderWithVote =
blockInterface.insertVoteToHeaderBuilder(builder, Optional.empty());
final BlockHeader header = builderWithVote.buildBlockHeader();
assertThat(header.getCoinbase()).isEqualTo(AddressHelpers.ofValue(0));
assertThat(header.getNonce()).isEqualTo(0x0L);
}
@Test
public void extractsValidatorsFromHeader() {
final BlockHeader header =
TestHelpers.createIbftSignedBlockHeader(headerBuilder, proposerKeys, validatorList);
final IbftLegacyVotingBlockInterface serDeser = new IbftLegacyVotingBlockInterface();
final List<Address> extractedValidators = serDeser.validatorsInBlock(header);
assertThat(extractedValidators).isEqualTo(validatorList);
}
}

@ -1,198 +0,0 @@
/*
* 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 static java.util.Arrays.asList;
import static java.util.Collections.emptyList;
import static java.util.Collections.singletonList;
import static org.assertj.core.api.Assertions.assertThat;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.verifyNoMoreInteractions;
import static org.mockito.Mockito.verifyZeroInteractions;
import static org.mockito.Mockito.when;
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.crypto.SECP256K1;
import tech.pegasys.pantheon.crypto.SECP256K1.KeyPair;
import tech.pegasys.pantheon.crypto.SECP256K1.Signature;
import tech.pegasys.pantheon.ethereum.chain.MutableBlockchain;
import tech.pegasys.pantheon.ethereum.core.Address;
import tech.pegasys.pantheon.ethereum.core.BlockHeader;
import tech.pegasys.pantheon.ethereum.core.BlockHeaderTestFixture;
import tech.pegasys.pantheon.ethereum.core.Hash;
import tech.pegasys.pantheon.util.bytes.BytesValue;
import java.math.BigInteger;
import java.util.List;
import java.util.Optional;
import org.junit.Test;
public class IbftVoteTallyUpdaterTest {
private static final long EPOCH_LENGTH = 30_000;
public static final Signature INVALID_SEAL =
Signature.create(BigInteger.ONE, BigInteger.ONE, (byte) 0);
private final VoteTally voteTally = mock(VoteTally.class);
private final MutableBlockchain blockchain = mock(MutableBlockchain.class);
private final KeyPair proposerKeyPair = KeyPair.generate();
private final Address proposerAddress =
Address.extract(Hash.hash(proposerKeyPair.getPublicKey().getEncodedBytes()));
private final Address subject = Address.fromHexString("007f4a23ca00cd043d25c2888c1aa5688f81a344");
private final Address validator1 =
Address.fromHexString("00dae27b350bae20c5652124af5d8b5cba001ec1");
private final IbftVoteTallyUpdater updater =
new IbftVoteTallyUpdater(new EpochManager(EPOCH_LENGTH));
@Test
public void voteTallyUpdatedWithVoteFromBlock() {
final BlockHeaderTestFixture headerBuilder = new BlockHeaderTestFixture();
headerBuilder.number(1);
headerBuilder.nonce(VoteType.ADD.getNonceValue());
headerBuilder.coinbase(subject);
addProposer(headerBuilder);
final BlockHeader header = headerBuilder.buildHeader();
updater.updateForBlock(header, voteTally);
verify(voteTally).addVote(proposerAddress, subject, VoteType.ADD);
}
@Test
public void voteTallyNotUpdatedWhenBlockHasNoVoteSubject() {
final BlockHeaderTestFixture headerBuilder = new BlockHeaderTestFixture();
headerBuilder.number(1);
headerBuilder.nonce(VoteType.ADD.getNonceValue());
headerBuilder.coinbase(Address.fromHexString("0000000000000000000000000000000000000000"));
addProposer(headerBuilder);
final BlockHeader header = headerBuilder.buildHeader();
updater.updateForBlock(header, voteTally);
verifyZeroInteractions(voteTally);
}
@Test
public void outstandingVotesDiscardedWhenEpochReached() {
final BlockHeaderTestFixture headerBuilder = new BlockHeaderTestFixture();
headerBuilder.number(EPOCH_LENGTH);
headerBuilder.nonce(VoteType.ADD.getNonceValue());
headerBuilder.coinbase(Address.fromHexString("0000000000000000000000000000000000000000"));
addProposer(headerBuilder);
final BlockHeader header = headerBuilder.buildHeader();
updater.updateForBlock(header, voteTally);
verify(voteTally).discardOutstandingVotes();
verifyNoMoreInteractions(voteTally);
}
@Test
public void buildVoteTallyByExtractingValidatorsFromGenesisBlock() {
final BlockHeaderTestFixture headerBuilder = new BlockHeaderTestFixture();
headerBuilder.number(0);
headerBuilder.nonce(VoteType.ADD.getNonceValue());
headerBuilder.coinbase(Address.fromHexString("0000000000000000000000000000000000000000"));
addProposer(headerBuilder, asList(subject, validator1));
final BlockHeader header = headerBuilder.buildHeader();
when(blockchain.getChainHeadBlockNumber()).thenReturn(EPOCH_LENGTH);
when(blockchain.getBlockHeader(EPOCH_LENGTH)).thenReturn(Optional.of(header));
final VoteTally voteTally = updater.buildVoteTallyFromBlockchain(blockchain);
assertThat(voteTally.getCurrentValidators()).containsExactly(subject, validator1);
}
@Test
public void buildVoteTallyByExtractingValidatorsFromEpochBlock() {
final BlockHeaderTestFixture headerBuilder = new BlockHeaderTestFixture();
headerBuilder.number(EPOCH_LENGTH);
headerBuilder.nonce(VoteType.ADD.getNonceValue());
headerBuilder.coinbase(Address.fromHexString("0000000000000000000000000000000000000000"));
addProposer(headerBuilder, asList(subject, validator1));
final BlockHeader header = headerBuilder.buildHeader();
when(blockchain.getChainHeadBlockNumber()).thenReturn(EPOCH_LENGTH);
when(blockchain.getBlockHeader(EPOCH_LENGTH)).thenReturn(Optional.of(header));
final VoteTally voteTally = updater.buildVoteTallyFromBlockchain(blockchain);
assertThat(voteTally.getCurrentValidators()).containsExactly(subject, validator1);
}
@Test
public void addVotesFromBlocksAfterMostRecentEpoch() {
final BlockHeaderTestFixture headerBuilder = new BlockHeaderTestFixture();
headerBuilder.number(EPOCH_LENGTH);
headerBuilder.nonce(VoteType.ADD.getNonceValue());
headerBuilder.coinbase(Address.fromHexString("0000000000000000000000000000000000000000"));
addProposer(headerBuilder, singletonList(validator1));
final BlockHeader epochHeader = headerBuilder.buildHeader();
headerBuilder.number(EPOCH_LENGTH + 1);
headerBuilder.coinbase(subject);
final BlockHeader voteBlockHeader = headerBuilder.buildHeader();
when(blockchain.getChainHeadBlockNumber()).thenReturn(EPOCH_LENGTH + 1);
when(blockchain.getBlockHeader(EPOCH_LENGTH)).thenReturn(Optional.of(epochHeader));
when(blockchain.getBlockHeader(EPOCH_LENGTH + 1)).thenReturn(Optional.of(voteBlockHeader));
final VoteTally voteTally = updater.buildVoteTallyFromBlockchain(blockchain);
assertThat(voteTally.getCurrentValidators()).containsExactly(subject, validator1);
}
private void addProposer(final BlockHeaderTestFixture builder) {
addProposer(builder, singletonList(proposerAddress));
}
private void addProposer(final BlockHeaderTestFixture builder, final List<Address> validators) {
final IbftExtraData initialIbftExtraData =
new IbftExtraData(
BytesValue.wrap(new byte[IbftExtraData.EXTRA_VANITY_LENGTH]),
emptyList(),
INVALID_SEAL,
validators);
builder.extraData(initialIbftExtraData.encode());
final BlockHeader header = builder.buildHeader();
final Hash proposerSealHash =
IbftBlockHashing.calculateDataHashForProposerSeal(header, initialIbftExtraData);
final Signature proposerSignature = SECP256K1.sign(proposerSealHash, proposerKeyPair);
final IbftExtraData proposedData =
new IbftExtraData(
BytesValue.wrap(new byte[IbftExtraData.EXTRA_VANITY_LENGTH]),
singletonList(proposerSignature),
proposerSignature,
validators);
final Hash headerHashForCommitters =
IbftBlockHashing.calculateDataHashForCommittedSeal(header, proposedData);
final Signature proposerAsCommitterSignature =
SECP256K1.sign(headerHashForCommitters, proposerKeyPair);
final IbftExtraData sealedData =
new IbftExtraData(
BytesValue.wrap(new byte[IbftExtraData.EXTRA_VANITY_LENGTH]),
singletonList(proposerAsCommitterSignature),
proposerSignature,
validators);
builder.extraData(sealedData.encode());
}
}

@ -0,0 +1,55 @@
/*
* 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.crypto.SECP256K1;
import tech.pegasys.pantheon.crypto.SECP256K1.KeyPair;
import tech.pegasys.pantheon.crypto.SECP256K1.Signature;
import tech.pegasys.pantheon.ethereum.core.Address;
import tech.pegasys.pantheon.ethereum.core.BlockHeader;
import tech.pegasys.pantheon.ethereum.core.BlockHeaderTestFixture;
import tech.pegasys.pantheon.ethereum.core.Hash;
import tech.pegasys.pantheon.util.bytes.BytesValue;
import java.util.Collections;
import java.util.List;
public class TestHelpers {
public static BlockHeader createIbftSignedBlockHeader(
final BlockHeaderTestFixture blockHeaderBuilder,
final KeyPair signer,
final List<Address> validators) {
final IbftExtraData unsignedExtraData =
new IbftExtraData(BytesValue.wrap(new byte[32]), Collections.emptyList(), null, validators);
blockHeaderBuilder.extraData(unsignedExtraData.encode());
final Hash signingHash =
IbftBlockHashing.calculateDataHashForProposerSeal(
blockHeaderBuilder.buildHeader(), unsignedExtraData);
final Signature proposerSignature = SECP256K1.sign(signingHash, signer);
final IbftExtraData signedExtraData =
new IbftExtraData(
unsignedExtraData.getVanityData(),
unsignedExtraData.getSeals(),
proposerSignature,
unsignedExtraData.getValidators());
blockHeaderBuilder.extraData(signedExtraData.encode());
return blockHeaderBuilder.buildHeader();
}
}

@ -19,13 +19,14 @@ import tech.pegasys.pantheon.config.GenesisConfigFile;
import tech.pegasys.pantheon.config.GenesisConfigOptions;
import tech.pegasys.pantheon.consensus.clique.CliqueContext;
import tech.pegasys.pantheon.consensus.clique.CliqueProtocolSchedule;
import tech.pegasys.pantheon.consensus.clique.CliqueVoteTallyUpdater;
import tech.pegasys.pantheon.consensus.clique.CliqueVotingBlockInterface;
import tech.pegasys.pantheon.consensus.clique.VoteTallyCache;
import tech.pegasys.pantheon.consensus.clique.blockcreation.CliqueBlockScheduler;
import tech.pegasys.pantheon.consensus.clique.blockcreation.CliqueMinerExecutor;
import tech.pegasys.pantheon.consensus.clique.blockcreation.CliqueMiningCoordinator;
import tech.pegasys.pantheon.consensus.common.EpochManager;
import tech.pegasys.pantheon.consensus.common.VoteProposer;
import tech.pegasys.pantheon.consensus.common.VoteTallyUpdater;
import tech.pegasys.pantheon.crypto.SECP256K1.KeyPair;
import tech.pegasys.pantheon.ethereum.ProtocolContext;
import tech.pegasys.pantheon.ethereum.blockcreation.MiningCoordinator;
@ -138,7 +139,9 @@ public class CliquePantheonController implements PantheonController<CliqueContex
worldStateArchive,
new CliqueContext(
new VoteTallyCache(
blockchain, new CliqueVoteTallyUpdater(epochManger), epochManger),
blockchain,
new VoteTallyUpdater(epochManger, new CliqueVotingBlockInterface()),
epochManger),
new VoteProposer(),
epochManger));

@ -20,6 +20,7 @@ import tech.pegasys.pantheon.config.IbftConfigOptions;
import tech.pegasys.pantheon.consensus.common.EpochManager;
import tech.pegasys.pantheon.consensus.common.VoteProposer;
import tech.pegasys.pantheon.consensus.common.VoteTally;
import tech.pegasys.pantheon.consensus.common.VoteTallyUpdater;
import tech.pegasys.pantheon.consensus.ibft.IbftChainObserver;
import tech.pegasys.pantheon.consensus.ibft.IbftContext;
import tech.pegasys.pantheon.consensus.ibft.IbftEventQueue;
@ -28,8 +29,8 @@ import tech.pegasys.pantheon.consensus.ibft.IbftStateMachine;
import tech.pegasys.pantheon.consensus.ibft.network.IbftNetworkPeers;
import tech.pegasys.pantheon.consensus.ibft.protocol.IbftProtocolManager;
import tech.pegasys.pantheon.consensus.ibft.protocol.IbftSubProtocol;
import tech.pegasys.pantheon.consensus.ibftlegacy.IbftLegacyVotingBlockInterface;
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;
@ -139,7 +140,8 @@ public class IbftPantheonController implements PantheonController<IbftContext> {
final EpochManager epochManager = new EpochManager(ibftConfig.getEpochLength());
final VoteTally voteTally =
new IbftVoteTallyUpdater(epochManager).buildVoteTallyFromBlockchain(blockchain);
new VoteTallyUpdater(epochManager, new IbftLegacyVotingBlockInterface())
.buildVoteTallyFromBlockchain(blockchain);
final VoteProposer voteProposer = new VoteProposer();

Loading…
Cancel
Save