ibft controller and future msgs handling (#431)

Jason Frame 6 years ago committed by GitHub
parent 5425a92c0d
commit e5b859eaf5
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
  1. 2
      consensus/ibft/src/main/java/tech/pegasys/pantheon/consensus/ibft/IbftEventQueue.java
  2. 2
      consensus/ibft/src/main/java/tech/pegasys/pantheon/consensus/ibft/IbftProcessor.java
  3. 1
      consensus/ibft/src/main/java/tech/pegasys/pantheon/consensus/ibft/IbftStateMachine.java
  4. 3
      consensus/ibft/src/main/java/tech/pegasys/pantheon/consensus/ibft/ibftevent/BlockTimerExpiry.java
  5. 4
      consensus/ibft/src/main/java/tech/pegasys/pantheon/consensus/ibft/ibftevent/IbftEvent.java
  6. 7
      consensus/ibft/src/main/java/tech/pegasys/pantheon/consensus/ibft/ibftevent/IbftEvents.java
  7. 34
      consensus/ibft/src/main/java/tech/pegasys/pantheon/consensus/ibft/ibftevent/IbftReceivedMessageEvent.java
  8. 3
      consensus/ibft/src/main/java/tech/pegasys/pantheon/consensus/ibft/ibftevent/NewChainHead.java
  9. 3
      consensus/ibft/src/main/java/tech/pegasys/pantheon/consensus/ibft/ibftevent/RoundExpiry.java
  10. 4
      consensus/ibft/src/main/java/tech/pegasys/pantheon/consensus/ibft/protocol/IbftProtocolManager.java
  11. 46
      consensus/ibft/src/main/java/tech/pegasys/pantheon/consensus/ibft/statemachine/IbftBlockHeightManager.java
  12. 23
      consensus/ibft/src/main/java/tech/pegasys/pantheon/consensus/ibft/statemachine/IbftBlockHeightManagerFactory.java
  13. 190
      consensus/ibft/src/main/java/tech/pegasys/pantheon/consensus/ibft/statemachine/IbftController.java
  14. 1
      consensus/ibft/src/test/java/tech/pegasys/pantheon/consensus/ibft/BlockTimerTest.java
  15. 1
      consensus/ibft/src/test/java/tech/pegasys/pantheon/consensus/ibft/IbftChainObserverTest.java
  16. 3
      consensus/ibft/src/test/java/tech/pegasys/pantheon/consensus/ibft/IbftEventQueueTest.java
  17. 443
      consensus/ibft/src/test/java/tech/pegasys/pantheon/consensus/ibft/statemachine/IbftControllerTest.java

@ -12,6 +12,8 @@
*/
package tech.pegasys.pantheon.consensus.ibft;
import tech.pegasys.pantheon.consensus.ibft.ibftevent.IbftEvent;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.TimeUnit;

@ -12,6 +12,8 @@
*/
package tech.pegasys.pantheon.consensus.ibft;
import tech.pegasys.pantheon.consensus.ibft.ibftevent.IbftEvent;
import java.util.Optional;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;

@ -13,6 +13,7 @@
package tech.pegasys.pantheon.consensus.ibft;
import tech.pegasys.pantheon.consensus.ibft.blockcreation.IbftBlockCreatorFactory;
import tech.pegasys.pantheon.consensus.ibft.ibftevent.IbftEvent;
/** Stateful evaluator for ibft events */
public class IbftStateMachine {

@ -13,8 +13,7 @@
package tech.pegasys.pantheon.consensus.ibft.ibftevent;
import tech.pegasys.pantheon.consensus.ibft.ConsensusRoundIdentifier;
import tech.pegasys.pantheon.consensus.ibft.IbftEvent;
import tech.pegasys.pantheon.consensus.ibft.IbftEvents.Type;
import tech.pegasys.pantheon.consensus.ibft.ibftevent.IbftEvents.Type;
import java.util.Objects;

@ -10,9 +10,9 @@
* an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
* specific language governing permissions and limitations under the License.
*/
package tech.pegasys.pantheon.consensus.ibft;
package tech.pegasys.pantheon.consensus.ibft.ibftevent;
import tech.pegasys.pantheon.consensus.ibft.IbftEvents.Type;
import tech.pegasys.pantheon.consensus.ibft.ibftevent.IbftEvents.Type;
/** Category of events that will effect and are interpretable by the Ibft processing mechanism */
public interface IbftEvent {

@ -10,19 +10,20 @@
* an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
* specific language governing permissions and limitations under the License.
*/
package tech.pegasys.pantheon.consensus.ibft;
package tech.pegasys.pantheon.consensus.ibft.ibftevent;
import tech.pegasys.pantheon.ethereum.p2p.api.Message;
/** Static helper functions for producing and working with IbftEvent objects */
public class IbftEvents {
public static IbftEvent fromMessage(final Message message) {
throw new IllegalStateException("No IbftEvents are implemented yet");
return new IbftReceivedMessageEvent(message.getData());
}
public enum Type {
ROUND_EXPIRY,
NEW_CHAIN_HEAD,
BLOCK_TIMER_EXPIRY
BLOCK_TIMER_EXPIRY,
MESSAGE
}
}

@ -0,0 +1,34 @@
/*
* 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.ibftevent;
import tech.pegasys.pantheon.consensus.ibft.ibftevent.IbftEvents.Type;
import tech.pegasys.pantheon.ethereum.p2p.api.MessageData;
public class IbftReceivedMessageEvent implements IbftEvent {
private final MessageData messageData;
public IbftReceivedMessageEvent(final MessageData messageData) {
this.messageData = messageData;
}
public MessageData getMessageData() {
return messageData;
}
@Override
public Type getType() {
return Type.MESSAGE;
}
}

@ -12,8 +12,7 @@
*/
package tech.pegasys.pantheon.consensus.ibft.ibftevent;
import tech.pegasys.pantheon.consensus.ibft.IbftEvent;
import tech.pegasys.pantheon.consensus.ibft.IbftEvents.Type;
import tech.pegasys.pantheon.consensus.ibft.ibftevent.IbftEvents.Type;
import tech.pegasys.pantheon.ethereum.core.BlockHeader;
import java.util.Objects;

@ -13,8 +13,7 @@
package tech.pegasys.pantheon.consensus.ibft.ibftevent;
import tech.pegasys.pantheon.consensus.ibft.ConsensusRoundIdentifier;
import tech.pegasys.pantheon.consensus.ibft.IbftEvent;
import tech.pegasys.pantheon.consensus.ibft.IbftEvents.Type;
import tech.pegasys.pantheon.consensus.ibft.ibftevent.IbftEvents.Type;
import java.util.Objects;

@ -12,9 +12,9 @@
*/
package tech.pegasys.pantheon.consensus.ibft.protocol;
import tech.pegasys.pantheon.consensus.ibft.IbftEvent;
import tech.pegasys.pantheon.consensus.ibft.IbftEventQueue;
import tech.pegasys.pantheon.consensus.ibft.IbftEvents;
import tech.pegasys.pantheon.consensus.ibft.ibftevent.IbftEvent;
import tech.pegasys.pantheon.consensus.ibft.ibftevent.IbftEvents;
import tech.pegasys.pantheon.consensus.ibft.network.IbftNetworkPeers;
import tech.pegasys.pantheon.ethereum.p2p.api.Message;
import tech.pegasys.pantheon.ethereum.p2p.api.PeerConnection;

@ -0,0 +1,46 @@
/*
* 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.statemachine;
import tech.pegasys.pantheon.consensus.ibft.ConsensusRoundIdentifier;
import tech.pegasys.pantheon.consensus.ibft.ibftevent.RoundExpiry;
import tech.pegasys.pantheon.consensus.ibft.ibftmessagedata.CommitPayload;
import tech.pegasys.pantheon.consensus.ibft.ibftmessagedata.NewRoundPayload;
import tech.pegasys.pantheon.consensus.ibft.ibftmessagedata.PreparePayload;
import tech.pegasys.pantheon.consensus.ibft.ibftmessagedata.ProposalPayload;
import tech.pegasys.pantheon.consensus.ibft.ibftmessagedata.RoundChangePayload;
import tech.pegasys.pantheon.consensus.ibft.ibftmessagedata.SignedData;
/** This no-op version will be replaced with an implementation in another PR */
public class IbftBlockHeightManager {
public void handleProposalMessage(final SignedData<ProposalPayload> proposalMsg) {}
public void handlePrepareMessage(final SignedData<PreparePayload> prepareMsg) {}
public void handleCommitMessage(final SignedData<CommitPayload> commitMsg) {}
public void handleBlockTimerExpiry(final ConsensusRoundIdentifier roundIndentifier) {}
public void handleRoundChangeMessage(final SignedData<RoundChangePayload> roundChangeMsg) {}
public void handleNewRoundMessage(final SignedData<NewRoundPayload> newRoundMsg) {}
public void start() {}
public long getChainHeight() {
return 0;
}
public void roundExpired(final RoundExpiry expired) {}
}

@ -0,0 +1,23 @@
/*
* 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.statemachine;
import tech.pegasys.pantheon.ethereum.core.BlockHeader;
/** This no-op version will be replaced with an implementation in another PR */
public class IbftBlockHeightManagerFactory {
public IbftBlockHeightManager create(final BlockHeader parentHeader) {
return new IbftBlockHeightManager();
}
}

@ -0,0 +1,190 @@
/*
* 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.statemachine;
import static java.util.Collections.emptyList;
import tech.pegasys.pantheon.consensus.ibft.ConsensusRoundIdentifier;
import tech.pegasys.pantheon.consensus.ibft.ibftevent.BlockTimerExpiry;
import tech.pegasys.pantheon.consensus.ibft.ibftevent.IbftReceivedMessageEvent;
import tech.pegasys.pantheon.consensus.ibft.ibftevent.NewChainHead;
import tech.pegasys.pantheon.consensus.ibft.ibftevent.RoundExpiry;
import tech.pegasys.pantheon.consensus.ibft.ibftmessage.CommitMessage;
import tech.pegasys.pantheon.consensus.ibft.ibftmessage.IbftV2;
import tech.pegasys.pantheon.consensus.ibft.ibftmessage.NewRoundMessage;
import tech.pegasys.pantheon.consensus.ibft.ibftmessage.PrepareMessage;
import tech.pegasys.pantheon.consensus.ibft.ibftmessage.ProposalMessage;
import tech.pegasys.pantheon.consensus.ibft.ibftmessage.RoundChangeMessage;
import tech.pegasys.pantheon.consensus.ibft.ibftmessagedata.CommitPayload;
import tech.pegasys.pantheon.consensus.ibft.ibftmessagedata.NewRoundPayload;
import tech.pegasys.pantheon.consensus.ibft.ibftmessagedata.Payload;
import tech.pegasys.pantheon.consensus.ibft.ibftmessagedata.PreparePayload;
import tech.pegasys.pantheon.consensus.ibft.ibftmessagedata.ProposalPayload;
import tech.pegasys.pantheon.consensus.ibft.ibftmessagedata.RoundChangePayload;
import tech.pegasys.pantheon.consensus.ibft.ibftmessagedata.SignedData;
import tech.pegasys.pantheon.ethereum.chain.Blockchain;
import tech.pegasys.pantheon.ethereum.core.BlockHeader;
import tech.pegasys.pantheon.ethereum.p2p.api.MessageData;
import java.util.List;
import java.util.Map;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
public class IbftController {
private static final Logger LOG = LogManager.getLogger();
private final Blockchain blockchain;
private final IbftFinalState ibftFinalState;
private final IbftBlockHeightManagerFactory ibftBlockHeightManagerFactory;
private final Map<Long, List<MessageData>> futureMessages;
private IbftBlockHeightManager currentHeightManager;
public IbftController(
final Blockchain blockchain,
final IbftFinalState ibftFinalState,
final IbftBlockHeightManagerFactory ibftBlockHeightManagerFactory) {
this(blockchain, ibftFinalState, ibftBlockHeightManagerFactory, Maps.newHashMap());
}
@VisibleForTesting
IbftController(
final Blockchain blockchain,
final IbftFinalState ibftFinalState,
final IbftBlockHeightManagerFactory ibftBlockHeightManagerFactory,
final Map<Long, List<MessageData>> futureMessages) {
this.blockchain = blockchain;
this.ibftFinalState = ibftFinalState;
this.ibftBlockHeightManagerFactory = ibftBlockHeightManagerFactory;
this.futureMessages = futureMessages;
}
public void start() {
startNewHeightManager(blockchain.getChainHeadHeader());
}
public void handleMessageEvent(final IbftReceivedMessageEvent msg) {
handleMessage(msg.getMessageData());
}
private void handleMessage(final MessageData messageData) {
switch (messageData.getCode()) {
case IbftV2.PROPOSAL:
final SignedData<ProposalPayload> proposalMsg =
ProposalMessage.fromMessage(messageData).decode();
if (processMessage(proposalMsg, messageData)) {
currentHeightManager.handleProposalMessage(proposalMsg);
}
break;
case IbftV2.PREPARE:
final SignedData<PreparePayload> prepareMsg =
PrepareMessage.fromMessage(messageData).decode();
if (processMessage(prepareMsg, messageData)) {
currentHeightManager.handlePrepareMessage(prepareMsg);
}
break;
case IbftV2.COMMIT:
final SignedData<CommitPayload> commitMsg = CommitMessage.fromMessage(messageData).decode();
if (processMessage(commitMsg, messageData)) {
currentHeightManager.handleCommitMessage(commitMsg);
}
break;
case IbftV2.ROUND_CHANGE:
final SignedData<RoundChangePayload> roundChangeMsg =
RoundChangeMessage.fromMessage(messageData).decode();
if (processMessage(roundChangeMsg, messageData)) {
currentHeightManager.handleRoundChangeMessage(roundChangeMsg);
}
break;
case IbftV2.NEW_ROUND:
final SignedData<NewRoundPayload> newRoundMsg =
NewRoundMessage.fromMessage(messageData).decode();
if (processMessage(newRoundMsg, messageData)) {
currentHeightManager.handleNewRoundMessage(newRoundMsg);
}
break;
default:
throw new IllegalArgumentException(
"Received message does not conform to any recognised IBFT message structure.");
}
}
public void handleNewBlockEvent(final NewChainHead newChainHead) {
startNewHeightManager(newChainHead.getNewChainHeadHeader());
}
public void handleBlockTimerExpiry(final BlockTimerExpiry blockTimerExpiry) {
if (isMsgForCurrentHeight(blockTimerExpiry.getRoundIndentifier())) {
currentHeightManager.handleBlockTimerExpiry(blockTimerExpiry.getRoundIndentifier());
} else {
LOG.info("Block timer event discarded as it is not for current block height");
}
}
public void handleRoundExpiry(final RoundExpiry roundExpiry) {
if (isMsgForCurrentHeight(roundExpiry.getView())) {
currentHeightManager.roundExpired(roundExpiry);
} else {
LOG.info("Round expiry event discarded as it is not for current block height");
}
}
private void startNewHeightManager(final BlockHeader parentHeader) {
currentHeightManager = ibftBlockHeightManagerFactory.create(parentHeader);
currentHeightManager.start();
final long newChainHeight = currentHeightManager.getChainHeight();
List<MessageData> orDefault = futureMessages.getOrDefault(newChainHeight, emptyList());
orDefault.forEach(this::handleMessage);
futureMessages.remove(newChainHeight);
}
private boolean processMessage(
final SignedData<? extends Payload> msg, final MessageData rawMsg) {
final ConsensusRoundIdentifier msgRoundIdentifier = msg.getPayload().getRoundIdentifier();
if (isMsgForCurrentHeight(msgRoundIdentifier)) {
return isMsgFromKnownValidator(msg);
} else if (isMsgForFutureChainHeight(msgRoundIdentifier)) {
addMessageToFutureMessageBuffer(msgRoundIdentifier.getSequenceNumber(), rawMsg);
} else {
LOG.info("IBFT message discarded as it is not for the current block height");
}
return false;
}
private boolean isMsgFromKnownValidator(final SignedData<? extends Payload> msg) {
return ibftFinalState.getValidators().contains(msg.getSender());
}
private boolean isMsgForCurrentHeight(final ConsensusRoundIdentifier roundIdentifier) {
return roundIdentifier.getSequenceNumber() == currentHeightManager.getChainHeight();
}
private boolean isMsgForFutureChainHeight(final ConsensusRoundIdentifier roundIdentifier) {
return roundIdentifier.getSequenceNumber() > currentHeightManager.getChainHeight();
}
private void addMessageToFutureMessageBuffer(final long chainHeight, final MessageData rawMsg) {
if (!futureMessages.containsKey(chainHeight)) {
futureMessages.put(chainHeight, Lists.newArrayList());
}
futureMessages.get(chainHeight).add(rawMsg);
}
}

@ -25,6 +25,7 @@ import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;
import tech.pegasys.pantheon.consensus.ibft.ibftevent.BlockTimerExpiry;
import tech.pegasys.pantheon.consensus.ibft.ibftevent.IbftEvent;
import tech.pegasys.pantheon.ethereum.core.BlockHeader;
import tech.pegasys.pantheon.ethereum.core.BlockHeaderTestFixture;

@ -18,6 +18,7 @@ import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;
import tech.pegasys.pantheon.consensus.ibft.ibftevent.IbftEvent;
import tech.pegasys.pantheon.consensus.ibft.ibftevent.NewChainHead;
import tech.pegasys.pantheon.ethereum.chain.BlockAddedEvent;
import tech.pegasys.pantheon.ethereum.chain.Blockchain;

@ -15,7 +15,8 @@ package tech.pegasys.pantheon.consensus.ibft;
import static org.assertj.core.api.Assertions.assertThat;
import static org.assertj.core.api.Assertions.assertThatCode;
import tech.pegasys.pantheon.consensus.ibft.IbftEvents.Type;
import tech.pegasys.pantheon.consensus.ibft.ibftevent.IbftEvent;
import tech.pegasys.pantheon.consensus.ibft.ibftevent.IbftEvents.Type;
import java.util.ArrayList;
import java.util.List;

@ -0,0 +1,443 @@
/*
* 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.statemachine;
import static org.assertj.core.api.Java6Assertions.assertThat;
import static org.assertj.core.util.Lists.newArrayList;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.Mockito.atLeastOnce;
import static org.mockito.Mockito.never;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.verifyNoMoreInteractions;
import static org.mockito.Mockito.when;
import tech.pegasys.pantheon.consensus.ibft.ConsensusRoundIdentifier;
import tech.pegasys.pantheon.consensus.ibft.ibftevent.BlockTimerExpiry;
import tech.pegasys.pantheon.consensus.ibft.ibftevent.IbftReceivedMessageEvent;
import tech.pegasys.pantheon.consensus.ibft.ibftevent.NewChainHead;
import tech.pegasys.pantheon.consensus.ibft.ibftevent.RoundExpiry;
import tech.pegasys.pantheon.consensus.ibft.ibftmessage.CommitMessage;
import tech.pegasys.pantheon.consensus.ibft.ibftmessage.IbftV2;
import tech.pegasys.pantheon.consensus.ibft.ibftmessage.NewRoundMessage;
import tech.pegasys.pantheon.consensus.ibft.ibftmessage.PrepareMessage;
import tech.pegasys.pantheon.consensus.ibft.ibftmessage.ProposalMessage;
import tech.pegasys.pantheon.consensus.ibft.ibftmessage.RoundChangeMessage;
import tech.pegasys.pantheon.consensus.ibft.ibftmessagedata.CommitPayload;
import tech.pegasys.pantheon.consensus.ibft.ibftmessagedata.NewRoundPayload;
import tech.pegasys.pantheon.consensus.ibft.ibftmessagedata.PreparePayload;
import tech.pegasys.pantheon.consensus.ibft.ibftmessagedata.ProposalPayload;
import tech.pegasys.pantheon.consensus.ibft.ibftmessagedata.RoundChangePayload;
import tech.pegasys.pantheon.consensus.ibft.ibftmessagedata.SignedData;
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.ethereum.p2p.api.MessageData;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import org.junit.Before;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.mockito.Mock;
import org.mockito.junit.MockitoJUnitRunner;
@RunWith(MockitoJUnitRunner.class)
public class IbftControllerTest {
@Mock private Blockchain blockChain;
@Mock private IbftFinalState ibftFinalState;
@Mock private IbftBlockHeightManagerFactory blockHeightManagerFactory;
@Mock private BlockHeader blockHeader;
@Mock private IbftBlockHeightManager blockHeightManager;
@Mock private SignedData<ProposalPayload> signedProposal;
@Mock private ProposalMessage proposalMessage;
@Mock private ProposalPayload proposalPayload;
@Mock private SignedData<PreparePayload> signedPrepare;
@Mock private PrepareMessage prepareMessage;
@Mock private PreparePayload preparePayload;
@Mock private SignedData<CommitPayload> signedCommit;
@Mock private CommitMessage commitMessage;
@Mock private CommitPayload commitPayload;
@Mock private SignedData<NewRoundPayload> signedNewRound;
@Mock private NewRoundMessage newRoundMessage;
@Mock private NewRoundPayload newRoundPayload;
@Mock private SignedData<RoundChangePayload> signedRoundChange;
@Mock private RoundChangeMessage roundChangeMessage;
@Mock private RoundChangePayload roundChangePayload;
private final Map<Long, List<MessageData>> futureMessages = new HashMap<>();
private final Address validator = Address.fromHexString("0x0");
private final Address unknownValidator = Address.fromHexString("0x2");
private final ConsensusRoundIdentifier futureRoundIdentifier = new ConsensusRoundIdentifier(2, 0);
private ConsensusRoundIdentifier roundIdentifier = new ConsensusRoundIdentifier(0, 0);
private IbftController ibftController;
@Before
public void setup() {
ibftController =
new IbftController(blockChain, ibftFinalState, blockHeightManagerFactory, futureMessages);
when(blockChain.getChainHeadHeader()).thenReturn(blockHeader);
when(blockHeightManagerFactory.create(blockHeader)).thenReturn(blockHeightManager);
when(ibftFinalState.getValidators()).thenReturn(ImmutableList.of(validator));
}
@Test
public void createsNewBlockHeightManagerWhenStarted() {
ibftController.start();
assertThat(futureMessages).isEmpty();
verify(blockHeightManagerFactory).create(blockHeader);
}
@Test
public void startsNewBlockHeightManagerAndReplaysFutureMessages() {
final ConsensusRoundIdentifier roundIdentifierHeight3 = new ConsensusRoundIdentifier(3, 0);
setupPrepare(futureRoundIdentifier, validator);
setupProposal(roundIdentifierHeight3, validator);
setupCommit(futureRoundIdentifier, validator);
setupRoundChange(futureRoundIdentifier, validator);
setupNewRound(roundIdentifierHeight3, validator);
final List<MessageData> height2Msgs =
newArrayList(prepareMessage, commitMessage, roundChangeMessage);
final List<MessageData> height3Msgs = newArrayList(proposalMessage, newRoundMessage);
futureMessages.put(2L, height2Msgs);
futureMessages.put(3L, height3Msgs);
when(blockHeightManager.getChainHeight()).thenReturn(2L);
ibftController.start();
assertThat(futureMessages.keySet()).hasSize(1);
assertThat(futureMessages.get(3L)).isEqualTo(height3Msgs);
verify(blockHeightManagerFactory).create(blockHeader);
verify(blockHeightManager, atLeastOnce()).getChainHeight();
verify(blockHeightManager).start();
verify(blockHeightManager, never()).handleProposalMessage(signedProposal);
verify(blockHeightManager).handlePrepareMessage(signedPrepare);
verify(blockHeightManager).handleCommitMessage(signedCommit);
verify(blockHeightManager).handleRoundChangeMessage(signedRoundChange);
verify(blockHeightManager, never()).handleNewRoundMessage(signedNewRound);
}
@Test
public void createsNewBlockHeightManagerAndReplaysFutureMessagesOnNewChainHeadEvent() {
setupPrepare(futureRoundIdentifier, validator);
setupProposal(futureRoundIdentifier, validator);
setupCommit(futureRoundIdentifier, validator);
setupRoundChange(futureRoundIdentifier, validator);
setupNewRound(futureRoundIdentifier, validator);
futureMessages.put(
2L,
ImmutableList.of(
prepareMessage, proposalMessage, commitMessage, roundChangeMessage, newRoundMessage));
when(blockHeightManager.getChainHeight()).thenReturn(2L);
final NewChainHead newChainHead = new NewChainHead(blockHeader);
ibftController.handleNewBlockEvent(newChainHead);
verify(blockHeightManagerFactory).create(blockHeader);
verify(blockHeightManager, atLeastOnce()).getChainHeight();
verify(blockHeightManager).start();
verify(blockHeightManager).handleProposalMessage(signedProposal);
verify(blockHeightManager).handlePrepareMessage(signedPrepare);
verify(blockHeightManager).handleCommitMessage(signedCommit);
verify(blockHeightManager).handleRoundChangeMessage(signedRoundChange);
verify(blockHeightManager).handleNewRoundMessage(signedNewRound);
}
@Test
public void handlesRoundExpiry() {
final RoundExpiry roundExpiry = new RoundExpiry(roundIdentifier);
ibftController.start();
ibftController.handleRoundExpiry(roundExpiry);
verify(blockHeightManager).roundExpired(roundExpiry);
}
@Test
public void handlesBlockTimerExpiry() {
final BlockTimerExpiry blockTimerExpiry = new BlockTimerExpiry(roundIdentifier);
ibftController.start();
ibftController.handleBlockTimerExpiry(blockTimerExpiry);
verify(blockHeightManager).handleBlockTimerExpiry(roundIdentifier);
}
@Test
public void proposalForCurrentHeightIsPassedToBlockHeightManager() {
setupProposal(roundIdentifier, validator);
ibftController.start();
ibftController.handleMessageEvent(new IbftReceivedMessageEvent(proposalMessage));
assertThat(futureMessages).isEmpty();
verify(blockHeightManager).handleProposalMessage(signedProposal);
verify(blockHeightManager, atLeastOnce()).getChainHeight();
verify(blockHeightManager).start();
verifyNoMoreInteractions(blockHeightManager);
}
@Test
public void prepareForCurrentHeightIsPassedToBlockHeightManager() {
setupPrepare(roundIdentifier, validator);
ibftController.start();
ibftController.handleMessageEvent(new IbftReceivedMessageEvent(prepareMessage));
assertThat(futureMessages).isEmpty();
verify(blockHeightManager).handlePrepareMessage(signedPrepare);
verify(blockHeightManager, atLeastOnce()).getChainHeight();
verify(blockHeightManager).start();
verifyNoMoreInteractions(blockHeightManager);
}
@Test
public void commitForCurrentHeightIsPassedToBlockHeightManager() {
setupCommit(roundIdentifier, validator);
ibftController.start();
ibftController.handleMessageEvent(new IbftReceivedMessageEvent(commitMessage));
assertThat(futureMessages).isEmpty();
verify(blockHeightManager).handleCommitMessage(signedCommit);
verify(blockHeightManager, atLeastOnce()).getChainHeight();
verify(blockHeightManager).start();
verifyNoMoreInteractions(blockHeightManager);
}
@Test
public void newRoundForCurrentHeightIsPassedToBlockHeightManager() {
roundIdentifier = new ConsensusRoundIdentifier(0, 1);
setupNewRound(roundIdentifier, validator);
ibftController.start();
ibftController.handleMessageEvent(new IbftReceivedMessageEvent(newRoundMessage));
assertThat(futureMessages).isEmpty();
verify(blockHeightManager).handleNewRoundMessage(signedNewRound);
verify(blockHeightManager, atLeastOnce()).getChainHeight();
verify(blockHeightManager).start();
verifyNoMoreInteractions(blockHeightManager);
}
@Test
public void roundChangeForCurrentHeightIsPassedToBlockHeightManager() {
roundIdentifier = new ConsensusRoundIdentifier(0, 1);
setupRoundChange(roundIdentifier, validator);
ibftController.start();
ibftController.handleMessageEvent(new IbftReceivedMessageEvent(roundChangeMessage));
assertThat(futureMessages).isEmpty();
verify(blockHeightManager).handleRoundChangeMessage(signedRoundChange);
verify(blockHeightManager, atLeastOnce()).getChainHeight();
verify(blockHeightManager).start();
verifyNoMoreInteractions(blockHeightManager);
}
@Test
public void proposalForPastHeightIsDiscarded() {
setupProposal(roundIdentifier, validator);
when(blockHeightManager.getChainHeight()).thenReturn(1L);
verifyNotHandledAndNoFutureMsgs(new IbftReceivedMessageEvent(proposalMessage));
}
@Test
public void prepareForPastHeightIsDiscarded() {
setupPrepare(roundIdentifier, validator);
when(blockHeightManager.getChainHeight()).thenReturn(1L);
verifyNotHandledAndNoFutureMsgs(new IbftReceivedMessageEvent(prepareMessage));
}
@Test
public void commitForPastHeightIsDiscarded() {
setupCommit(roundIdentifier, validator);
when(blockHeightManager.getChainHeight()).thenReturn(1L);
verifyNotHandledAndNoFutureMsgs(new IbftReceivedMessageEvent(commitMessage));
}
@Test
public void newRoundForPastHeightIsDiscarded() {
setupNewRound(roundIdentifier, validator);
when(blockHeightManager.getChainHeight()).thenReturn(1L);
verifyNotHandledAndNoFutureMsgs(new IbftReceivedMessageEvent(newRoundMessage));
}
@Test
public void roundChangeForPastHeightIsDiscarded() {
setupRoundChange(roundIdentifier, validator);
when(blockHeightManager.getChainHeight()).thenReturn(1L);
verifyNotHandledAndNoFutureMsgs(new IbftReceivedMessageEvent(roundChangeMessage));
}
@Test
public void roundExpiryForPastHeightIsDiscarded() {
final RoundExpiry roundExpiry = new RoundExpiry(roundIdentifier);
when(blockHeightManager.getChainHeight()).thenReturn(1L);
ibftController.start();
ibftController.handleRoundExpiry(roundExpiry);
assertThat(futureMessages).isEmpty();
verify(blockHeightManager, never()).roundExpired(any());
}
@Test
public void blockTimerForPastHeightIsDiscarded() {
final BlockTimerExpiry blockTimerExpiry = new BlockTimerExpiry(roundIdentifier);
when(blockHeightManager.getChainHeight()).thenReturn(1L);
ibftController.start();
ibftController.handleBlockTimerExpiry(blockTimerExpiry);
assertThat(futureMessages).isEmpty();
verify(blockHeightManager, never()).handleBlockTimerExpiry(any());
}
@Test
public void proposalForUnknownValidatorIsDiscarded() {
setupProposal(roundIdentifier, unknownValidator);
verifyNotHandledAndNoFutureMsgs(new IbftReceivedMessageEvent(proposalMessage));
}
@Test
public void prepareForUnknownValidatorIsDiscarded() {
setupPrepare(roundIdentifier, unknownValidator);
verifyNotHandledAndNoFutureMsgs(new IbftReceivedMessageEvent(prepareMessage));
}
@Test
public void commitForUnknownValidatorIsDiscarded() {
setupCommit(roundIdentifier, unknownValidator);
verifyNotHandledAndNoFutureMsgs(new IbftReceivedMessageEvent(commitMessage));
}
@Test
public void newRoundForUnknownValidatorIsDiscarded() {
setupNewRound(roundIdentifier, unknownValidator);
verifyNotHandledAndNoFutureMsgs(new IbftReceivedMessageEvent(newRoundMessage));
}
@Test
public void roundChangeForUnknownValidatorIsDiscarded() {
setupRoundChange(roundIdentifier, unknownValidator);
verifyNotHandledAndNoFutureMsgs(new IbftReceivedMessageEvent(roundChangeMessage));
}
@Test
public void proposalForFutureHeightIsBuffered() {
setupProposal(futureRoundIdentifier, validator);
final Map<Long, List<MessageData>> expectedFutureMsgs =
ImmutableMap.of(2L, ImmutableList.of(proposalMessage));
verifyHasFutureMessages(new IbftReceivedMessageEvent(proposalMessage), expectedFutureMsgs);
}
@Test
public void prepareForFutureHeightIsBuffered() {
setupPrepare(futureRoundIdentifier, validator);
final Map<Long, List<MessageData>> expectedFutureMsgs =
ImmutableMap.of(2L, ImmutableList.of(prepareMessage));
verifyHasFutureMessages(new IbftReceivedMessageEvent(prepareMessage), expectedFutureMsgs);
}
@Test
public void commitForFutureHeightIsBuffered() {
setupCommit(futureRoundIdentifier, validator);
final Map<Long, List<MessageData>> expectedFutureMsgs =
ImmutableMap.of(2L, ImmutableList.of(commitMessage));
verifyHasFutureMessages(new IbftReceivedMessageEvent(commitMessage), expectedFutureMsgs);
}
@Test
public void newRoundForFutureHeightIsBuffered() {
setupNewRound(futureRoundIdentifier, validator);
final Map<Long, List<MessageData>> expectedFutureMsgs =
ImmutableMap.of(2L, ImmutableList.of(newRoundMessage));
verifyHasFutureMessages(new IbftReceivedMessageEvent(newRoundMessage), expectedFutureMsgs);
}
@Test
public void roundChangeForFutureHeightIsBuffered() {
setupRoundChange(futureRoundIdentifier, validator);
final Map<Long, List<MessageData>> expectedFutureMsgs =
ImmutableMap.of(2L, ImmutableList.of(roundChangeMessage));
verifyHasFutureMessages(new IbftReceivedMessageEvent(roundChangeMessage), expectedFutureMsgs);
}
private void verifyNotHandledAndNoFutureMsgs(final IbftReceivedMessageEvent msg) {
ibftController.start();
ibftController.handleMessageEvent(msg);
assertThat(futureMessages).isEmpty();
verify(blockHeightManager, atLeastOnce()).getChainHeight();
verify(blockHeightManager).start();
verifyNoMoreInteractions(blockHeightManager);
}
private void verifyHasFutureMessages(
final IbftReceivedMessageEvent msg, final Map<Long, List<MessageData>> expectedFutureMsgs) {
ibftController.start();
ibftController.handleMessageEvent(msg);
assertThat(futureMessages).hasSize(expectedFutureMsgs.size());
assertThat(futureMessages).isEqualTo(expectedFutureMsgs);
verify(blockHeightManager, atLeastOnce()).getChainHeight();
verify(blockHeightManager).start();
verifyNoMoreInteractions(blockHeightManager);
}
private void setupProposal(
final ConsensusRoundIdentifier roundIdentifier, final Address validator) {
when(signedProposal.getPayload()).thenReturn(proposalPayload);
when(signedProposal.getSender()).thenReturn(validator);
when(proposalPayload.getRoundIdentifier()).thenReturn(roundIdentifier);
when(proposalMessage.getCode()).thenReturn(IbftV2.PROPOSAL);
when(proposalMessage.decode()).thenReturn(signedProposal);
}
private void setupPrepare(
final ConsensusRoundIdentifier roundIdentifier, final Address validator) {
when(signedPrepare.getPayload()).thenReturn(preparePayload);
when(signedPrepare.getSender()).thenReturn(validator);
when(preparePayload.getRoundIdentifier()).thenReturn(roundIdentifier);
when(prepareMessage.getCode()).thenReturn(IbftV2.PREPARE);
when(prepareMessage.decode()).thenReturn(signedPrepare);
}
private void setupCommit(
final ConsensusRoundIdentifier roundIdentifier, final Address validator) {
when(signedCommit.getPayload()).thenReturn(commitPayload);
when(signedCommit.getSender()).thenReturn(validator);
when(commitPayload.getRoundIdentifier()).thenReturn(roundIdentifier);
when(commitMessage.getCode()).thenReturn(IbftV2.COMMIT);
when(commitMessage.decode()).thenReturn(signedCommit);
}
private void setupNewRound(
final ConsensusRoundIdentifier roundIdentifier, final Address validator) {
when(signedNewRound.getPayload()).thenReturn(newRoundPayload);
when(signedNewRound.getSender()).thenReturn(validator);
when(newRoundPayload.getRoundIdentifier()).thenReturn(roundIdentifier);
when(newRoundMessage.getCode()).thenReturn(IbftV2.NEW_ROUND);
when(newRoundMessage.decode()).thenReturn(signedNewRound);
}
private void setupRoundChange(
final ConsensusRoundIdentifier roundIdentifier, final Address validator) {
when(signedRoundChange.getPayload()).thenReturn(roundChangePayload);
when(signedRoundChange.getSender()).thenReturn(validator);
when(roundChangePayload.getRoundIdentifier()).thenReturn(roundIdentifier);
when(roundChangeMessage.getCode()).thenReturn(IbftV2.ROUND_CHANGE);
when(roundChangeMessage.decode()).thenReturn(signedRoundChange);
}
}
Loading…
Cancel
Save