Repair VoteTallyCache incorrectly applying vote to parent (#360)

A problem was identified whereby the vote in a child block was applied to the parent, this was resolved by
duplicating the parent VoteTally before modifying it, and injecting to the child block.

Some refactoring was also conducted to make VoteTally logic simpler.
tmohay 6 years ago committed by GitHub
parent d864247733
commit aac4fd40c5
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
  1. 6
      consensus/clique/src/main/java/tech/pegasys/pantheon/consensus/clique/CliqueHelpers.java
  2. 36
      consensus/clique/src/main/java/tech/pegasys/pantheon/consensus/clique/VoteTallyCache.java
  3. 3
      consensus/clique/src/main/java/tech/pegasys/pantheon/consensus/clique/blockcreation/CliqueBlockCreator.java
  4. 4
      consensus/clique/src/main/java/tech/pegasys/pantheon/consensus/clique/blockcreation/CliqueBlockScheduler.java
  5. 7
      consensus/clique/src/main/java/tech/pegasys/pantheon/consensus/clique/blockcreation/CliqueMinerExecutor.java
  6. 4
      consensus/clique/src/main/java/tech/pegasys/pantheon/consensus/clique/blockcreation/CliqueProposerSelector.java
  7. 4
      consensus/clique/src/main/java/tech/pegasys/pantheon/consensus/clique/headervalidationrules/CliqueExtraDataValidationRule.java
  8. 2
      consensus/clique/src/main/java/tech/pegasys/pantheon/consensus/clique/jsonrpc/methods/CliqueGetSigners.java
  9. 2
      consensus/clique/src/main/java/tech/pegasys/pantheon/consensus/clique/jsonrpc/methods/CliqueGetSignersAtHash.java
  10. 2
      consensus/clique/src/test/java/tech/pegasys/pantheon/consensus/clique/CliqueDifficultyCalculatorTest.java
  11. 14
      consensus/clique/src/test/java/tech/pegasys/pantheon/consensus/clique/NodeCanProduceNextBlockTest.java
  12. 54
      consensus/clique/src/test/java/tech/pegasys/pantheon/consensus/clique/VoteTallyCacheTest.java
  13. 2
      consensus/clique/src/test/java/tech/pegasys/pantheon/consensus/clique/blockcreation/CliqueBlockCreatorTest.java
  14. 2
      consensus/clique/src/test/java/tech/pegasys/pantheon/consensus/clique/blockcreation/CliqueBlockSchedulerTest.java
  15. 2
      consensus/clique/src/test/java/tech/pegasys/pantheon/consensus/clique/blockcreation/CliqueMinerExecutorTest.java
  16. 2
      consensus/clique/src/test/java/tech/pegasys/pantheon/consensus/clique/blockcreation/CliqueProposerSelectorTest.java
  17. 2
      consensus/clique/src/test/java/tech/pegasys/pantheon/consensus/clique/headervalidationrules/CliqueDifficultyValidationRuleTest.java
  18. 2
      consensus/clique/src/test/java/tech/pegasys/pantheon/consensus/clique/headervalidationrules/CliqueExtraDataValidationRuleTest.java
  19. 4
      consensus/clique/src/test/java/tech/pegasys/pantheon/consensus/clique/jsonrpc/methods/CliqueGetSignersAtHashTest.java
  20. 8
      consensus/clique/src/test/java/tech/pegasys/pantheon/consensus/clique/jsonrpc/methods/CliqueGetSignersTest.java
  21. 2
      consensus/common/src/main/java/tech/pegasys/pantheon/consensus/common/ValidatorProvider.java
  22. 2
      consensus/common/src/main/java/tech/pegasys/pantheon/consensus/common/VoteProposer.java
  23. 6
      consensus/common/src/main/java/tech/pegasys/pantheon/consensus/common/VoteTally.java
  24. 60
      consensus/common/src/test/java/tech/pegasys/pantheon/consensus/common/VoteTallyTest.java
  25. 6
      consensus/common/src/test/java/tech/pegasys/pantheon/consensus/common/VoteTallyUpdaterTest.java
  26. 2
      consensus/ibft/src/main/java/tech/pegasys/pantheon/consensus/ibft/blockcreation/IbftBlockCreatorFactory.java
  27. 6
      consensus/ibft/src/main/java/tech/pegasys/pantheon/consensus/ibft/blockcreation/ProposerSelector.java
  28. 2
      consensus/ibft/src/main/java/tech/pegasys/pantheon/consensus/ibft/headervalidationrules/IbftCoinbaseValidationRule.java
  29. 2
      consensus/ibft/src/main/java/tech/pegasys/pantheon/consensus/ibft/headervalidationrules/IbftExtraDataValidationRule.java
  30. 2
      consensus/ibft/src/main/java/tech/pegasys/pantheon/consensus/ibft/network/IbftNetworkPeers.java
  31. 4
      consensus/ibft/src/test/java/tech/pegasys/pantheon/consensus/ibft/network/IbftNetworkPeersTest.java
  32. 2
      consensus/ibftlegacy/src/main/java/tech/pegasys/pantheon/consensus/ibftlegacy/blockcreation/IbftExtraDataCalculator.java
  33. 2
      consensus/ibftlegacy/src/main/java/tech/pegasys/pantheon/consensus/ibftlegacy/headervalidationrules/IbftExtraDataValidationRule.java

@ -38,9 +38,9 @@ public class CliqueHelpers {
final ProtocolContext<CliqueContext> protocolContext,
final BlockHeader parent) {
final VoteTally validatorProvider =
protocolContext.getConsensusState().getVoteTallyCache().getVoteTallyAtBlock(parent);
protocolContext.getConsensusState().getVoteTallyCache().getVoteTallyAfterBlock(parent);
if (!validatorProvider.getCurrentValidators().contains(candidate)) {
if (!validatorProvider.getValidators().contains(candidate)) {
return false;
}
@ -72,7 +72,7 @@ public class CliqueHelpers {
}
private static int minimumBlocksSincePreviousSigning(final ValidatorProvider validatorProvider) {
final int validatorCount = validatorProvider.getCurrentValidators().size();
final int validatorCount = validatorProvider.getValidators().size();
// The number of contiguous blocks in which a signer may only sign 1 (as taken from clique spec)
final int signerLimit = (validatorCount / 2) + 1;
return signerLimit - 1;

@ -50,7 +50,18 @@ public class VoteTallyCache {
this.epochManager = epochManager;
}
public VoteTally getVoteTallyAtBlock(final BlockHeader header) {
/**
* Determines the VoteTally for a given block header, by back-tracing the blockchain to a
* previously cached value or epoch block. Then appyling votes in each intermediate header such
* that representative state can be provided. This function assumes the vote cast in {@code
* header} is applied, thus the voteTally returned contains the group of validators who are
* permitted to partake in the next block's creation.
*
* @param header the header of the block after which the VoteTally is to be returned
* @return The Vote Tally (and therefore validators) following the application of all votes upto
* and including the requested header.
*/
public VoteTally getVoteTallyAfterBlock(final BlockHeader header) {
try {
return voteTallyCache.get(header.getHash(), () -> populateCacheUptoAndIncluding(header));
} catch (final ExecutionException ex) {
@ -64,7 +75,8 @@ public class VoteTallyCache {
VoteTally voteTally = null;
while (true) { // Will run into an epoch block (and thus a VoteTally) to break loop.
voteTally = findMostRecentAvailableVoteTally(header, intermediateBlocks);
intermediateBlocks.push(header);
voteTally = getValidatorsAfter(header);
if (voteTally != null) {
break;
}
@ -80,25 +92,23 @@ public class VoteTallyCache {
return constructMissingCacheEntries(intermediateBlocks, voteTally);
}
private VoteTally findMostRecentAvailableVoteTally(
final BlockHeader header, final Deque<BlockHeader> intermediateBlockHeaders) {
intermediateBlockHeaders.push(header);
VoteTally voteTally = voteTallyCache.getIfPresent(header.getParentHash());
if ((voteTally == null) && (epochManager.isEpochBlock(header.getNumber()))) {
final CliqueExtraData extraData = CliqueExtraData.decode(header.getExtraData());
voteTally = new VoteTally(extraData.getValidators());
private VoteTally getValidatorsAfter(final BlockHeader header) {
if (epochManager.isEpochBlock(header.getNumber())) {
final CliqueBlockInterface blockInterface = new CliqueBlockInterface();
return new VoteTally(blockInterface.validatorsInBlock(header));
}
return voteTally;
return voteTallyCache.getIfPresent(header.getParentHash());
}
private VoteTally constructMissingCacheEntries(
final Deque<BlockHeader> headers, final VoteTally tally) {
final VoteTally mutableVoteTally = tally.copy();
while (!headers.isEmpty()) {
final BlockHeader h = headers.pop();
voteTallyUpdater.updateForBlock(h, tally);
voteTallyCache.put(h.getHash(), tally.copy());
voteTallyUpdater.updateForBlock(h, mutableVoteTally);
voteTallyCache.put(h.getHash(), mutableVoteTally.copy());
}
return tally;
return mutableVoteTally;
}
}

@ -85,7 +85,8 @@ public class CliqueBlockCreator extends AbstractBlockCreator<CliqueContext> {
.blockHashFunction(blockHashFunction);
final CliqueContext cliqueContext = protocolContext.getConsensusState();
final VoteTally voteTally = cliqueContext.getVoteTallyCache().getVoteTallyAtBlock(parentHeader);
final VoteTally voteTally =
cliqueContext.getVoteTallyCache().getVoteTallyAfterBlock(parentHeader);
final Optional<ValidatorVote> vote =
cliqueContext

@ -60,11 +60,11 @@ public class CliqueBlockScheduler extends DefaultBlockScheduler {
if (nextProposer.equals(localNodeAddress)) {
return 0;
}
return calculatorOutOfTurnDelay(voteTallyCache.getVoteTallyAtBlock(parentHeader));
return calculatorOutOfTurnDelay(voteTallyCache.getVoteTallyAfterBlock(parentHeader));
}
private int calculatorOutOfTurnDelay(final ValidatorProvider validators) {
final int countSigners = validators.getCurrentValidators().size();
final int countSigners = validators.getValidators().size();
final double multiplier = (countSigners / 2d) + 1;
final int maxDelay = (int) (multiplier * OUT_OF_TURN_DELAY_MULTIPLIER_MILLIS);
return r.nextInt(maxDelay) + 1;

@ -101,8 +101,11 @@ public class CliqueMinerExecutor extends AbstractMinerExecutor<CliqueContext, Cl
// Building ON TOP of canonical head, if the next block is epoch, include validators.
if (epochManager.isEpochBlock(parentHeader.getNumber() + 1)) {
final VoteTally voteTally =
protocolContext.getConsensusState().getVoteTallyCache().getVoteTallyAtBlock(parentHeader);
validators.addAll(voteTally.getCurrentValidators());
protocolContext
.getConsensusState()
.getVoteTallyCache()
.getVoteTallyAfterBlock(parentHeader);
validators.addAll(voteTally.getValidators());
}
final CliqueExtraData extraData = new CliqueExtraData(vanityDataToInsert, null, validators);

@ -45,8 +45,8 @@ public class CliqueProposerSelector {
*/
public Address selectProposerForNextBlock(final BlockHeader parentHeader) {
final VoteTally parentVoteTally = voteTallyCache.getVoteTallyAtBlock(parentHeader);
final List<Address> validatorSet = new ArrayList<>(parentVoteTally.getCurrentValidators());
final VoteTally parentVoteTally = voteTallyCache.getVoteTallyAfterBlock(parentHeader);
final List<Address> validatorSet = new ArrayList<>(parentVoteTally.getValidators());
final long nextBlockNumber = parentHeader.getNumber() + 1L;
final int indexIntoValidators = (int) (nextBlockNumber % validatorSet.size());

@ -59,9 +59,9 @@ public class CliqueExtraDataValidationRule
final ProtocolContext<CliqueContext> protocolContext) {
try {
final VoteTally validatorProvider =
protocolContext.getConsensusState().getVoteTallyCache().getVoteTallyAtBlock(parent);
protocolContext.getConsensusState().getVoteTallyCache().getVoteTallyAfterBlock(parent);
final Collection<Address> storedValidators = validatorProvider.getCurrentValidators();
final Collection<Address> storedValidators = validatorProvider.getValidators();
return extraDataIsValid(storedValidators, header);
} catch (final RLPException ex) {

@ -53,7 +53,7 @@ public class CliqueGetSigners implements JsonRpcMethod {
public JsonRpcResponse response(final JsonRpcRequest request) {
final Optional<BlockHeader> blockHeader = determineBlockHeader(request);
return blockHeader
.map(bh -> voteTallyCache.getVoteTallyAtBlock(bh).getCurrentValidators())
.map(bh -> voteTallyCache.getVoteTallyAfterBlock(bh).getValidators())
.map(addresses -> addresses.stream().map(Objects::toString).collect(Collectors.toList()))
.<JsonRpcResponse>map(addresses -> new JsonRpcSuccessResponse(request.getId(), addresses))
.orElse(new JsonRpcErrorResponse(request.getId(), JsonRpcError.INTERNAL_ERROR));

@ -53,7 +53,7 @@ public class CliqueGetSignersAtHash implements JsonRpcMethod {
public JsonRpcResponse response(final JsonRpcRequest request) {
final Optional<BlockHeader> blockHeader = determineBlockHeader(request);
return blockHeader
.map(bh -> voteTallyCache.getVoteTallyAtBlock(bh).getCurrentValidators())
.map(bh -> voteTallyCache.getVoteTallyAfterBlock(bh).getValidators())
.map(addresses -> addresses.stream().map(Objects::toString).collect(Collectors.toList()))
.<JsonRpcResponse>map(addresses -> new JsonRpcSuccessResponse(request.getId(), addresses))
.orElse(new JsonRpcErrorResponse(request.getId(), JsonRpcError.INTERNAL_ERROR));

@ -51,7 +51,7 @@ public class CliqueDifficultyCalculatorTest {
validatorList.add(AddressHelpers.calculateAddressWithRespectTo(localAddr, 1));
final VoteTallyCache voteTallyCache = mock(VoteTallyCache.class);
when(voteTallyCache.getVoteTallyAtBlock(any())).thenReturn(new VoteTally(validatorList));
when(voteTallyCache.getVoteTallyAfterBlock(any())).thenReturn(new VoteTally(validatorList));
final VoteProposer voteProposer = new VoteProposer();
final CliqueContext cliqueContext = new CliqueContext(voteTallyCache, voteProposer, null);

@ -73,7 +73,7 @@ public class NodeCanProduceNextBlockTest {
blockChain = createInMemoryBlockchain(genesisBlock);
final VoteTallyCache voteTallyCache = mock(VoteTallyCache.class);
when(voteTallyCache.getVoteTallyAtBlock(any())).thenReturn(new VoteTally(validatorList));
when(voteTallyCache.getVoteTallyAfterBlock(any())).thenReturn(new VoteTally(validatorList));
final VoteProposer voteProposer = new VoteProposer();
final CliqueContext cliqueContext = new CliqueContext(voteTallyCache, voteProposer, null);
cliqueProtocolContext = new ProtocolContext<>(blockChain, null, cliqueContext);
@ -98,7 +98,7 @@ public class NodeCanProduceNextBlockTest {
blockChain = createInMemoryBlockchain(genesisBlock);
final VoteTallyCache voteTallyCache = mock(VoteTallyCache.class);
when(voteTallyCache.getVoteTallyAtBlock(any())).thenReturn(new VoteTally(validatorList));
when(voteTallyCache.getVoteTallyAfterBlock(any())).thenReturn(new VoteTally(validatorList));
final VoteProposer voteProposer = new VoteProposer();
final CliqueContext cliqueContext = new CliqueContext(voteTallyCache, voteProposer, null);
cliqueProtocolContext = new ProtocolContext<>(blockChain, null, cliqueContext);
@ -132,7 +132,7 @@ public class NodeCanProduceNextBlockTest {
blockChain = createInMemoryBlockchain(genesisBlock);
final VoteTallyCache voteTallyCache = mock(VoteTallyCache.class);
when(voteTallyCache.getVoteTallyAtBlock(any())).thenReturn(new VoteTally(validatorList));
when(voteTallyCache.getVoteTallyAfterBlock(any())).thenReturn(new VoteTally(validatorList));
final VoteProposer voteProposer = new VoteProposer();
final CliqueContext cliqueContext = new CliqueContext(voteTallyCache, voteProposer, null);
cliqueProtocolContext = new ProtocolContext<>(blockChain, null, cliqueContext);
@ -162,7 +162,7 @@ public class NodeCanProduceNextBlockTest {
blockChain = createInMemoryBlockchain(genesisBlock);
final VoteTallyCache voteTallyCache = mock(VoteTallyCache.class);
when(voteTallyCache.getVoteTallyAtBlock(any())).thenReturn(new VoteTally(validatorList));
when(voteTallyCache.getVoteTallyAfterBlock(any())).thenReturn(new VoteTally(validatorList));
final VoteProposer voteProposer = new VoteProposer();
final CliqueContext cliqueContext = new CliqueContext(voteTallyCache, voteProposer, null);
cliqueProtocolContext = new ProtocolContext<>(blockChain, null, cliqueContext);
@ -207,7 +207,7 @@ public class NodeCanProduceNextBlockTest {
blockChain = createInMemoryBlockchain(genesisBlock);
final VoteTallyCache voteTallyCache = mock(VoteTallyCache.class);
when(voteTallyCache.getVoteTallyAtBlock(any())).thenReturn(new VoteTally(validatorList));
when(voteTallyCache.getVoteTallyAfterBlock(any())).thenReturn(new VoteTally(validatorList));
final VoteProposer voteProposer = new VoteProposer();
final CliqueContext cliqueContext = new CliqueContext(voteTallyCache, voteProposer, null);
cliqueProtocolContext = new ProtocolContext<>(blockChain, null, cliqueContext);
@ -236,7 +236,7 @@ public class NodeCanProduceNextBlockTest {
blockChain = createInMemoryBlockchain(genesisBlock);
final VoteTallyCache voteTallyCache = mock(VoteTallyCache.class);
when(voteTallyCache.getVoteTallyAtBlock(any())).thenReturn(new VoteTally(validatorList));
when(voteTallyCache.getVoteTallyAfterBlock(any())).thenReturn(new VoteTally(validatorList));
final VoteProposer voteProposer = new VoteProposer();
final CliqueContext cliqueContext = new CliqueContext(voteTallyCache, voteProposer, null);
cliqueProtocolContext = new ProtocolContext<>(blockChain, null, cliqueContext);
@ -260,7 +260,7 @@ public class NodeCanProduceNextBlockTest {
blockChain = createInMemoryBlockchain(genesisBlock);
final VoteTallyCache voteTallyCache = mock(VoteTallyCache.class);
when(voteTallyCache.getVoteTallyAtBlock(any())).thenReturn(new VoteTally(validatorList));
when(voteTallyCache.getVoteTallyAfterBlock(any())).thenReturn(new VoteTally(validatorList));
final VoteProposer voteProposer = new VoteProposer();
final CliqueContext cliqueContext = new CliqueContext(voteTallyCache, voteProposer, null);
cliqueProtocolContext = new ProtocolContext<>(blockChain, null, cliqueContext);

@ -20,12 +20,17 @@ import static org.mockito.Mockito.reset;
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.verifyZeroInteractions;
import static org.mockito.Mockito.when;
import static tech.pegasys.pantheon.consensus.common.VoteType.DROP;
import static tech.pegasys.pantheon.ethereum.core.InMemoryStorageProvider.createInMemoryBlockchain;
import tech.pegasys.pantheon.consensus.common.EpochManager;
import tech.pegasys.pantheon.consensus.common.ValidatorVote;
import tech.pegasys.pantheon.consensus.common.VoteTally;
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.Address;
import tech.pegasys.pantheon.ethereum.core.AddressHelpers;
import tech.pegasys.pantheon.ethereum.core.Block;
import tech.pegasys.pantheon.ethereum.core.BlockBody;
@ -36,6 +41,8 @@ import tech.pegasys.pantheon.util.bytes.BytesValue;
import java.math.BigInteger;
import java.util.Arrays;
import java.util.List;
import java.util.Optional;
import com.google.common.util.concurrent.UncheckedExecutionException;
import org.assertj.core.util.Lists;
@ -58,13 +65,18 @@ public class VoteTallyCacheTest {
private Block block_1;
private Block block_2;
private final List<Address> validators = Lists.newArrayList();
@Before
public void constructThreeBlockChain() {
for (int i = 0; i < 3; i++) {
validators.add(AddressHelpers.ofValue(i));
}
headerBuilder.extraData(
new CliqueExtraData(
BytesValue.wrap(new byte[32]),
Signature.create(BigInteger.TEN, BigInteger.TEN, (byte) 1),
Lists.emptyList())
validators)
.encode());
genesisBlock = createEmptyBlock(0, Hash.ZERO);
@ -87,7 +99,7 @@ public class VoteTallyCacheTest {
// The votetallyUpdater should be invoked for the requested block, and all parents including
// the epoch (genesis) block.
final ArgumentCaptor<BlockHeader> varArgs = ArgumentCaptor.forClass(BlockHeader.class);
cache.getVoteTallyAtBlock(block_2.getHeader());
cache.getVoteTallyAfterBlock(block_2.getHeader());
verify(tallyUpdater, times(3)).updateForBlock(varArgs.capture(), any());
assertThat(varArgs.getAllValues())
.isEqualTo(
@ -97,10 +109,10 @@ public class VoteTallyCacheTest {
// Requesting the vote tally to the parent block should not invoke the voteTallyUpdater as the
// vote tally was cached from previous operation.
cache.getVoteTallyAtBlock(block_1.getHeader());
cache.getVoteTallyAfterBlock(block_1.getHeader());
verifyZeroInteractions(tallyUpdater);
cache.getVoteTallyAtBlock(block_2.getHeader());
cache.getVoteTallyAfterBlock(block_2.getHeader());
verifyZeroInteractions(tallyUpdater);
}
@ -113,7 +125,7 @@ public class VoteTallyCacheTest {
final Block orphanBlock = createEmptyBlock(4, Hash.ZERO);
assertThatExceptionOfType(UncheckedExecutionException.class)
.isThrownBy(() -> cache.getVoteTallyAtBlock(orphanBlock.getHeader()))
.isThrownBy(() -> cache.getVoteTallyAfterBlock(orphanBlock.getHeader()))
.withMessageContaining(
"Supplied block was on a orphaned chain, unable to generate " + "VoteTally.");
}
@ -125,14 +137,14 @@ public class VoteTallyCacheTest {
new VoteTallyCache(blockChain, tallyUpdater, new EpochManager(30_000));
// Load the Cache up to block_2
cache.getVoteTallyAtBlock(block_2.getHeader());
cache.getVoteTallyAfterBlock(block_2.getHeader());
reset(tallyUpdater);
// Append new blocks to the chain, and ensure the walkback only goes as far as block_2.
final Block block_3 = createEmptyBlock(4, block_2.getHeader().getHash());
// Load the Cache up to block_2
cache.getVoteTallyAtBlock(block_3.getHeader());
cache.getVoteTallyAfterBlock(block_3.getHeader());
// The votetallyUpdater should be invoked for the requested block, and all parents including
// the epoch (genesis) block.
@ -140,4 +152,32 @@ public class VoteTallyCacheTest {
verify(tallyUpdater, times(1)).updateForBlock(varArgs.capture(), any());
assertThat(varArgs.getAllValues()).isEqualTo(Arrays.asList(block_3.getHeader()));
}
// A bug was identified in VoteTallyCache whereby a vote cast in the next block *could* be applied
// to the parent block (depending on cache creation ordering). This test ensure the problem is
// resolved.
@Test
public void integrationTestingVotesBeingApplied() {
final EpochManager epochManager = new EpochManager(30_000);
final CliqueBlockInterface blockInterface = mock(CliqueBlockInterface.class);
final VoteTallyUpdater tallyUpdater = new VoteTallyUpdater(epochManager, blockInterface);
when(blockInterface.extractVoteFromHeader(block_1.getHeader()))
.thenReturn(Optional.of(new ValidatorVote(DROP, validators.get(0), validators.get(2))));
when(blockInterface.extractVoteFromHeader(block_2.getHeader()))
.thenReturn(Optional.of(new ValidatorVote(DROP, validators.get(1), validators.get(2))));
final VoteTallyCache cache = new VoteTallyCache(blockChain, tallyUpdater, epochManager);
VoteTally voteTally = cache.getVoteTallyAfterBlock(block_1.getHeader());
assertThat(voteTally.getValidators()).containsAll(validators);
voteTally = cache.getVoteTallyAfterBlock(block_2.getHeader());
assertThat(voteTally.getValidators()).containsExactly(validators.get(0), validators.get(1));
voteTally = cache.getVoteTallyAfterBlock(block_1.getHeader());
assertThat(voteTally.getValidators()).containsAll(validators);
}
}

@ -75,7 +75,7 @@ public class CliqueBlockCreatorTest {
validatorList.add(otherAddress);
final VoteTallyCache voteTallyCache = mock(VoteTallyCache.class);
when(voteTallyCache.getVoteTallyAtBlock(any())).thenReturn(new VoteTally(validatorList));
when(voteTallyCache.getVoteTallyAfterBlock(any())).thenReturn(new VoteTally(validatorList));
voteProposer = new VoteProposer();
final CliqueContext cliqueContext = new CliqueContext(voteTallyCache, voteProposer, null);

@ -51,7 +51,7 @@ public class CliqueBlockSchedulerTest {
validatorList.add(AddressHelpers.calculateAddressWithRespectTo(localAddr, 1));
voteTallyCache = mock(VoteTallyCache.class);
when(voteTallyCache.getVoteTallyAtBlock(any())).thenReturn(new VoteTally(validatorList));
when(voteTallyCache.getVoteTallyAfterBlock(any())).thenReturn(new VoteTally(validatorList));
blockHeaderBuilder = new BlockHeaderTestFixture();
}

@ -66,7 +66,7 @@ public class CliqueMinerExecutorTest {
validatorList.add(AddressHelpers.calculateAddressWithRespectTo(localAddress, 3));
final VoteTallyCache voteTallyCache = mock(VoteTallyCache.class);
when(voteTallyCache.getVoteTallyAtBlock(any())).thenReturn(new VoteTally(validatorList));
when(voteTallyCache.getVoteTallyAfterBlock(any())).thenReturn(new VoteTally(validatorList));
final VoteProposer voteProposer = new VoteProposer();
final CliqueContext cliqueContext = new CliqueContext(voteTallyCache, voteProposer, null);

@ -43,7 +43,7 @@ public class CliqueProposerSelectorTest {
@Before
public void setup() {
voteTallyCache = mock(VoteTallyCache.class);
when(voteTallyCache.getVoteTallyAtBlock(any())).thenReturn(voteTally);
when(voteTallyCache.getVoteTallyAfterBlock(any())).thenReturn(voteTally);
}
@Test

@ -56,7 +56,7 @@ public class CliqueDifficultyValidationRuleTest {
validatorList.add(AddressHelpers.calculateAddressWithRespectTo(localAddress, 1));
final VoteTallyCache voteTallyCache = mock(VoteTallyCache.class);
when(voteTallyCache.getVoteTallyAtBlock(any())).thenReturn(new VoteTally(validatorList));
when(voteTallyCache.getVoteTallyAfterBlock(any())).thenReturn(new VoteTally(validatorList));
final VoteProposer voteProposer = new VoteProposer();
final CliqueContext cliqueContext = new CliqueContext(voteTallyCache, voteProposer, null);

@ -54,7 +54,7 @@ public class CliqueExtraDataValidationRuleTest {
validatorList.add(AddressHelpers.calculateAddressWithRespectTo(localAddr, 1));
final VoteTallyCache voteTallyCache = mock(VoteTallyCache.class);
when(voteTallyCache.getVoteTallyAtBlock(any())).thenReturn(new VoteTally(validatorList));
when(voteTallyCache.getVoteTallyAfterBlock(any())).thenReturn(new VoteTally(validatorList));
final CliqueContext cliqueContext = new CliqueContext(voteTallyCache, null, null);
cliqueProtocolContext = new ProtocolContext<>(null, null, cliqueContext);

@ -109,8 +109,8 @@ public class CliqueGetSignersAtHashTest {
when(blockchainQueries.blockByHash(Hash.fromHexString(BLOCK_HASH)))
.thenReturn(Optional.of(blockWithMetadata));
when(blockWithMetadata.getHeader()).thenReturn(blockHeader);
when(voteTallyCache.getVoteTallyAtBlock(blockHeader)).thenReturn(voteTally);
when(voteTally.getCurrentValidators()).thenReturn(validators);
when(voteTallyCache.getVoteTallyAfterBlock(blockHeader)).thenReturn(voteTally);
when(voteTally.getValidators()).thenReturn(validators);
final JsonRpcSuccessResponse response = (JsonRpcSuccessResponse) method.response(request);
assertEquals(validatorsAsStrings, response.getResult());

@ -89,8 +89,8 @@ public class CliqueGetSignersTest {
when(blockchainQueries.headBlockNumber()).thenReturn(3065995L);
when(blockchainQueries.blockByNumber(3065995L)).thenReturn(Optional.of(blockWithMetadata));
when(blockWithMetadata.getHeader()).thenReturn(blockHeader);
when(voteTallyCache.getVoteTallyAtBlock(blockHeader)).thenReturn(voteTally);
when(voteTally.getCurrentValidators()).thenReturn(validators);
when(voteTallyCache.getVoteTallyAfterBlock(blockHeader)).thenReturn(voteTally);
when(voteTally.getValidators()).thenReturn(validators);
final JsonRpcSuccessResponse response = (JsonRpcSuccessResponse) method.response(request);
assertEquals(validatorAsStrings, response.getResult());
@ -104,8 +104,8 @@ public class CliqueGetSignersTest {
when(blockchainQueries.blockByNumber(3065995L)).thenReturn(Optional.of(blockWithMetadata));
when(blockWithMetadata.getHeader()).thenReturn(blockHeader);
when(voteTallyCache.getVoteTallyAtBlock(blockHeader)).thenReturn(voteTally);
when(voteTally.getCurrentValidators()).thenReturn(validators);
when(voteTallyCache.getVoteTallyAfterBlock(blockHeader)).thenReturn(voteTally);
when(voteTally.getValidators()).thenReturn(validators);
final JsonRpcSuccessResponse response = (JsonRpcSuccessResponse) method.response(request);
assertEquals(validatorAsStrings, response.getResult());

@ -19,5 +19,5 @@ import java.util.Collection;
public interface ValidatorProvider {
// Returns the current list of validators
Collection<Address> getCurrentValidators();
Collection<Address> getValidators();
}

@ -107,7 +107,7 @@ public class VoteProposer {
* votes
*/
public Optional<ValidatorVote> getVote(final Address localAddress, final VoteTally tally) {
final Collection<Address> validators = tally.getCurrentValidators();
final Collection<Address> validators = tally.getValidators();
final List<Map.Entry<Address, VoteType>> validVotes = new ArrayList<>();
proposals

@ -107,8 +107,12 @@ public class VoteTally implements ValidatorProvider {
addVotesBySubject.clear();
}
/**
* @return The collection of validators after the voting at the most recent block has been
* finalised.
*/
@Override
public Collection<Address> getCurrentValidators() {
public Collection<Address> getValidators() {
return currentValidators;
}

@ -41,7 +41,7 @@ public class VoteTallyTest {
voteTally.addVote(new ValidatorVote(ADD, validator1, validator5));
voteTally.addVote(new ValidatorVote(ADD, validator2, validator5));
assertThat(voteTally.getCurrentValidators())
assertThat(voteTally.getValidators())
.containsExactly(validator1, validator2, validator3, validator4);
}
@ -52,7 +52,7 @@ public class VoteTallyTest {
voteTally.addVote(new ValidatorVote(ADD, validator2, validator5));
voteTally.addVote(new ValidatorVote(ADD, validator3, validator5));
assertThat(voteTally.getCurrentValidators())
assertThat(voteTally.getValidators())
.containsExactly(validator1, validator2, validator3, validator4, validator5);
}
@ -64,7 +64,7 @@ public class VoteTallyTest {
voteTally.addVote(new ValidatorVote(ADD, validator2, validator4));
voteTally.addVote(new ValidatorVote(ADD, validator3, validator4));
assertThat(voteTally.getCurrentValidators())
assertThat(voteTally.getValidators())
.containsExactly(validator1, validator2, validator3, validator4, validator5);
}
@ -75,7 +75,7 @@ public class VoteTallyTest {
voteTally.addVote(new ValidatorVote(ADD, validator2, validator5));
voteTally.addVote(new ValidatorVote(ADD, validator2, validator5));
assertThat(voteTally.getCurrentValidators())
assertThat(voteTally.getValidators())
.containsExactly(validator1, validator2, validator3, validator4);
}
@ -87,7 +87,7 @@ public class VoteTallyTest {
voteTally.addVote(new ValidatorVote(ADD, validator2, validator5));
voteTally.addVote(new ValidatorVote(ADD, validator3, validator5));
assertThat(voteTally.getCurrentValidators())
assertThat(voteTally.getValidators())
.containsExactly(validator1, validator2, validator3, validator4);
}
@ -99,7 +99,7 @@ public class VoteTallyTest {
voteTally.addVote(new ValidatorVote(ADD, validator3, validator5));
voteTally.addVote(new ValidatorVote(DROP, validator1, validator5));
assertThat(voteTally.getCurrentValidators())
assertThat(voteTally.getValidators())
.containsExactly(validator1, validator2, validator3, validator4, validator5);
}
@ -111,20 +111,20 @@ public class VoteTallyTest {
voteTally.addVote(new ValidatorVote(ADD, validator2, validator5));
voteTally.addVote(new ValidatorVote(ADD, validator3, validator5));
assertThat(voteTally.getCurrentValidators())
assertThat(voteTally.getValidators())
.containsExactly(validator1, validator2, validator3, validator4, validator5);
// Then vote it back out
voteTally.addVote(new ValidatorVote(DROP, validator2, validator5));
voteTally.addVote(new ValidatorVote(DROP, validator3, validator5));
voteTally.addVote(new ValidatorVote(DROP, validator4, validator5));
assertThat(voteTally.getCurrentValidators())
assertThat(voteTally.getValidators())
.containsExactly(validator1, validator2, validator3, validator4);
// And then start voting to add it back in, but validator1's vote should have been discarded
voteTally.addVote(new ValidatorVote(ADD, validator2, validator5));
voteTally.addVote(new ValidatorVote(ADD, validator3, validator5));
assertThat(voteTally.getCurrentValidators())
assertThat(voteTally.getValidators())
.containsExactly(validator1, validator2, validator3, validator4);
}
@ -133,7 +133,7 @@ public class VoteTallyTest {
final VoteTally voteTally = new VoteTally(singletonList(validator1));
voteTally.addVote(new ValidatorVote(ADD, validator1, validator2));
assertThat(voteTally.getCurrentValidators()).containsExactly(validator1, validator2);
assertThat(voteTally.getValidators()).containsExactly(validator1, validator2);
}
@Test
@ -141,11 +141,10 @@ public class VoteTallyTest {
final VoteTally voteTally = new VoteTally(asList(validator1, validator2));
voteTally.addVote(new ValidatorVote(ADD, validator1, validator3));
assertThat(voteTally.getCurrentValidators()).containsExactly(validator1, validator2);
assertThat(voteTally.getValidators()).containsExactly(validator1, validator2);
voteTally.addVote(new ValidatorVote(ADD, validator2, validator3));
assertThat(voteTally.getCurrentValidators())
.containsExactly(validator1, validator2, validator3);
assertThat(voteTally.getValidators()).containsExactly(validator1, validator2, validator3);
}
@Test
@ -156,7 +155,7 @@ public class VoteTallyTest {
voteTally.discardOutstandingVotes();
voteTally.addVote(new ValidatorVote(ADD, validator3, validator5));
assertThat(voteTally.getCurrentValidators())
assertThat(voteTally.getValidators())
.containsExactly(validator1, validator2, validator3, validator4);
}
@ -166,7 +165,7 @@ public class VoteTallyTest {
voteTally.addVote(new ValidatorVote(DROP, validator1, validator4));
voteTally.addVote(new ValidatorVote(DROP, validator2, validator4));
assertThat(voteTally.getCurrentValidators())
assertThat(voteTally.getValidators())
.containsExactly(validator1, validator2, validator3, validator4);
}
@ -177,8 +176,7 @@ public class VoteTallyTest {
voteTally.addVote(new ValidatorVote(DROP, validator2, validator4));
voteTally.addVote(new ValidatorVote(DROP, validator3, validator4));
assertThat(voteTally.getCurrentValidators())
.containsExactly(validator1, validator2, validator3);
assertThat(voteTally.getValidators()).containsExactly(validator1, validator2, validator3);
}
@Test
@ -188,8 +186,7 @@ public class VoteTallyTest {
voteTally.addVote(new ValidatorVote(DROP, validator2, validator3));
voteTally.addVote(new ValidatorVote(DROP, validator4, validator3));
assertThat(voteTally.getCurrentValidators())
.containsExactly(validator1, validator2, validator4);
assertThat(voteTally.getValidators()).containsExactly(validator1, validator2, validator4);
}
@Test
@ -199,7 +196,7 @@ public class VoteTallyTest {
voteTally.addVote(new ValidatorVote(DROP, validator2, validator4));
voteTally.addVote(new ValidatorVote(DROP, validator2, validator4));
assertThat(voteTally.getCurrentValidators())
assertThat(voteTally.getValidators())
.containsExactly(validator1, validator2, validator3, validator4);
}
@ -211,7 +208,7 @@ public class VoteTallyTest {
voteTally.addVote(new ValidatorVote(DROP, validator2, validator4));
voteTally.addVote(new ValidatorVote(DROP, validator3, validator4));
assertThat(voteTally.getCurrentValidators())
assertThat(voteTally.getValidators())
.containsExactly(validator1, validator2, validator3, validator4);
}
@ -223,8 +220,7 @@ public class VoteTallyTest {
voteTally.addVote(new ValidatorVote(DROP, validator3, validator4));
voteTally.addVote(new ValidatorVote(ADD, validator1, validator4));
assertThat(voteTally.getCurrentValidators())
.containsExactly(validator1, validator2, validator3);
assertThat(voteTally.getValidators()).containsExactly(validator1, validator2, validator3);
}
@Test
@ -236,16 +232,14 @@ public class VoteTallyTest {
voteTally.addVote(new ValidatorVote(DROP, validator1, validator4));
voteTally.addVote(new ValidatorVote(DROP, validator2, validator4));
voteTally.addVote(new ValidatorVote(DROP, validator3, validator4));
assertThat(voteTally.getCurrentValidators())
.containsExactly(validator1, validator2, validator3);
assertThat(voteTally.getValidators()).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(new ValidatorVote(ADD, validator1, validator5));
voteTally.addVote(new ValidatorVote(DROP, validator1, validator3));
assertThat(voteTally.getCurrentValidators())
.containsExactly(validator1, validator2, validator3);
assertThat(voteTally.getValidators()).containsExactly(validator1, validator2, validator3);
}
@Test
@ -257,20 +251,20 @@ public class VoteTallyTest {
voteTally.addVote(new ValidatorVote(DROP, validator2, validator5));
voteTally.addVote(new ValidatorVote(DROP, validator3, validator5));
assertThat(voteTally.getCurrentValidators())
assertThat(voteTally.getValidators())
.containsExactly(validator1, validator2, validator3, validator4);
// Then vote it back in
voteTally.addVote(new ValidatorVote(ADD, validator2, validator5));
voteTally.addVote(new ValidatorVote(ADD, validator3, validator5));
voteTally.addVote(new ValidatorVote(ADD, validator4, validator5));
assertThat(voteTally.getCurrentValidators())
assertThat(voteTally.getValidators())
.containsExactly(validator1, validator2, validator3, validator4, validator5);
// And then start voting to drop it again, but validator1's vote should have been discarded
voteTally.addVote(new ValidatorVote(DROP, validator2, validator5));
voteTally.addVote(new ValidatorVote(DROP, validator3, validator5));
assertThat(voteTally.getCurrentValidators())
assertThat(voteTally.getValidators())
.containsExactly(validator1, validator2, validator3, validator4, validator5);
}
@ -284,21 +278,21 @@ public class VoteTallyTest {
voteTally.addVote(new ValidatorVote(DROP, validator2, validator1));
// Neither vote has enough votes to complete.
assertThat(voteTally.getCurrentValidators())
assertThat(voteTally.getValidators())
.containsExactly(validator1, validator2, validator3, validator4);
voteTally.addVote(new ValidatorVote(ADD, validator3, validator5));
voteTally.addVote(new ValidatorVote(DROP, validator3, validator1));
// Validator 5 now has 3 votes and is added
assertThat(voteTally.getCurrentValidators())
assertThat(voteTally.getValidators())
.containsExactly(validator1, validator2, validator3, validator4, validator5);
voteTally.addVote(new ValidatorVote(ADD, validator4, validator5));
voteTally.addVote(new ValidatorVote(DROP, validator4, validator1));
// Validator 1 now gets dropped.
assertThat(voteTally.getCurrentValidators())
assertThat(voteTally.getValidators())
.containsExactly(validator2, validator3, validator4, validator5);
}

@ -102,7 +102,7 @@ public class VoteTallyUpdaterTest {
when(blockchain.getBlockHeader(EPOCH_LENGTH)).thenReturn(Optional.of(header));
final VoteTally voteTally = updater.buildVoteTallyFromBlockchain(blockchain);
assertThat(voteTally.getCurrentValidators()).containsExactly(subject, validator1);
assertThat(voteTally.getValidators()).containsExactly(subject, validator1);
}
@Test
@ -115,7 +115,7 @@ public class VoteTallyUpdaterTest {
when(blockchain.getBlockHeader(EPOCH_LENGTH)).thenReturn(Optional.of(header));
final VoteTally voteTally = updater.buildVoteTallyFromBlockchain(blockchain);
assertThat(voteTally.getCurrentValidators()).containsExactly(subject, validator1);
assertThat(voteTally.getValidators()).containsExactly(subject, validator1);
}
@Test
@ -136,6 +136,6 @@ public class VoteTallyUpdaterTest {
when(blockchain.getBlockHeader(EPOCH_LENGTH + 1)).thenReturn(Optional.of(voteBlockHeader));
final VoteTally voteTally = updater.buildVoteTallyFromBlockchain(blockchain);
assertThat(voteTally.getCurrentValidators()).containsExactly(subject, validator1);
assertThat(voteTally.getValidators()).containsExactly(subject, validator1);
}
}

@ -89,7 +89,7 @@ public class IbftBlockCreatorFactory {
final Optional<ValidatorVote> proposal =
protocolContext.getConsensusState().getVoteProposer().getVote(localAddress, voteTally);
final List<Address> validators = new ArrayList<>(voteTally.getCurrentValidators());
final List<Address> validators = new ArrayList<>(voteTally.getValidators());
final IbftExtraData extraData =
new IbftExtraData(

@ -82,7 +82,7 @@ public class ProposerSelector {
final long prevBlockNumber = roundIdentifier.getSequenceNumber() - 1;
final Address prevBlockProposer = getProposerOfBlock(prevBlockNumber);
if (!validators.getCurrentValidators().contains(prevBlockProposer)) {
if (!validators.getValidators().contains(prevBlockProposer)) {
return handleMissingProposer(prevBlockProposer, roundIdentifier);
} else {
return handleWithExistingProposer(prevBlockProposer, roundIdentifier);
@ -97,7 +97,7 @@ public class ProposerSelector {
*/
private Address handleMissingProposer(
final Address prevBlockProposer, final ConsensusRoundIdentifier roundIdentifier) {
final NavigableSet<Address> validatorSet = new TreeSet<>(validators.getCurrentValidators());
final NavigableSet<Address> validatorSet = new TreeSet<>(validators.getValidators());
final SortedSet<Address> latterValidators = validatorSet.tailSet(prevBlockProposer, false);
final Address nextProposer;
if (latterValidators.isEmpty()) {
@ -138,7 +138,7 @@ public class ProposerSelector {
*/
private Address calculateRoundSpecificValidator(
final Address baseProposer, final int indexOffset) {
final List<Address> currentValidatorList = new ArrayList<>(validators.getCurrentValidators());
final List<Address> currentValidatorList = new ArrayList<>(validators.getValidators());
final int prevProposerIndex = currentValidatorList.indexOf(baseProposer);
final int roundValidatorIndex = (prevProposerIndex + indexOffset) % currentValidatorList.size();
return currentValidatorList.get(roundValidatorIndex);

@ -41,7 +41,7 @@ public class IbftCoinbaseValidationRule implements AttachedBlockHeaderValidation
final ValidatorProvider validatorProvider = context.getConsensusState().getVoteTally();
Address proposer = header.getCoinbase();
final Collection<Address> storedValidators = validatorProvider.getCurrentValidators();
final Collection<Address> storedValidators = validatorProvider.getValidators();
if (!storedValidators.contains(proposer)) {
LOGGER.trace("Block proposer is not a member of the validators.");

@ -73,7 +73,7 @@ public class IbftExtraDataValidationRule implements AttachedBlockHeaderValidatio
final ValidatorProvider validatorProvider = context.getConsensusState().getVoteTally();
final IbftExtraData ibftExtraData = IbftExtraData.decode(header.getExtraData());
final Collection<Address> storedValidators = validatorProvider.getCurrentValidators();
final Collection<Address> storedValidators = validatorProvider.getValidators();
if (validateCommitSeals) {
final List<Address> committers =

@ -53,7 +53,7 @@ public class IbftNetworkPeers {
}
public void multicastToValidators(final MessageData message) {
final Collection<Address> validators = validatorProvider.getCurrentValidators();
final Collection<Address> validators = validatorProvider.getValidators();
sendMessageToSpecificAddresses(validators, message);
}

@ -67,7 +67,7 @@ public class IbftNetworkPeersTest {
// Only add the first Peer's address to the validators.
validators.add(Util.publicKeyToAddress(publicKeys.get(0)));
final ValidatorProvider validatorProvider = mock(ValidatorProvider.class);
when(validatorProvider.getCurrentValidators()).thenReturn(validators);
when(validatorProvider.getValidators()).thenReturn(validators);
final IbftNetworkPeers peers = new IbftNetworkPeers(validatorProvider);
for (final PeerConnection peer : peerConnections) {
@ -88,7 +88,7 @@ public class IbftNetworkPeersTest {
validators.add(Util.publicKeyToAddress(publicKeys.get(0)));
final ValidatorProvider validatorProvider = mock(ValidatorProvider.class);
when(validatorProvider.getCurrentValidators()).thenReturn(validators);
when(validatorProvider.getValidators()).thenReturn(validators);
final IbftNetworkPeers peers = new IbftNetworkPeers(validatorProvider);

@ -36,7 +36,7 @@ public class IbftExtraDataCalculator implements ExtraDataCalculator {
vanityData,
Lists.newArrayList(),
null,
Lists.newArrayList(validatorProvider.getCurrentValidators()));
Lists.newArrayList(validatorProvider.getValidators()));
return baseExtraData.encode();
}
}

@ -74,7 +74,7 @@ public class IbftExtraDataValidationRule implements AttachedBlockHeaderValidatio
final Address proposer = IbftBlockHashing.recoverProposerAddress(header, ibftExtraData);
final Collection<Address> storedValidators = validatorProvider.getCurrentValidators();
final Collection<Address> storedValidators = validatorProvider.getValidators();
if (!storedValidators.contains(proposer)) {
LOG.trace("Proposer sealing block is not a member of the validators.");

Loading…
Cancel
Save