Added IBFT Proposed block creation classes (#169)

IBFT requires various aspects of the mining infrastructure in order
to create a proposed block.

This includes specifically the BlockCreator and MiningCoordinating,
the mining executor is not required at this stage, nor is the miner.
tmohay 6 years ago committed by GitHub
parent 481e17d5fa
commit 7b2621060e
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
  1. 3
      consensus/clique/src/main/java/tech/pegasys/pantheon/consensus/clique/CliqueVotingBlockInterface.java
  2. 8
      consensus/clique/src/main/java/tech/pegasys/pantheon/consensus/clique/blockcreation/CliqueBlockCreator.java
  3. 11
      consensus/clique/src/main/java/tech/pegasys/pantheon/consensus/clique/blockcreation/CliqueMinerExecutor.java
  4. 6
      consensus/clique/src/test/java/tech/pegasys/pantheon/consensus/clique/CliqueVotingBlockInterfaceTest.java
  5. 25
      consensus/common/src/main/java/tech/pegasys/pantheon/consensus/common/ConsensusHelpers.java
  6. 4
      consensus/common/src/main/java/tech/pegasys/pantheon/consensus/common/VoteBlockInterface.java
  7. 8
      consensus/ibft/src/main/java/tech/pegasys/pantheon/consensus/ibft/IbftStateMachine.java
  8. 24
      consensus/ibft/src/main/java/tech/pegasys/pantheon/consensus/ibft/IbftVotingBlockInterface.java
  9. 27
      consensus/ibft/src/main/java/tech/pegasys/pantheon/consensus/ibft/blockcreation/IbftBlockCreator.java
  10. 110
      consensus/ibft/src/main/java/tech/pegasys/pantheon/consensus/ibft/blockcreation/IbftBlockCreatorFactory.java
  11. 50
      consensus/ibft/src/main/java/tech/pegasys/pantheon/consensus/ibft/blockcreation/IbftMiningCoordinator.java
  12. 3
      consensus/ibftlegacy/src/main/java/tech/pegasys/pantheon/consensus/ibftlegacy/IbftLegacyVotingBlockInterface.java
  13. 9
      consensus/ibftlegacy/src/test/java/tech/pegasys/pantheon/consensus/ibftlegacy/IbftLegacyVotingBlockInterfaceTest.java
  14. 33
      ethereum/blockcreation/src/main/java/tech/pegasys/pantheon/ethereum/blockcreation/AbstractMiningCoordinator.java
  15. 33
      ethereum/blockcreation/src/main/java/tech/pegasys/pantheon/ethereum/blockcreation/MiningCoordinator.java
  16. 27
      pantheon/src/main/java/tech/pegasys/pantheon/controller/IbftPantheonController.java
  17. 1
      pantheon/src/main/java/tech/pegasys/pantheon/controller/PantheonController.java

@ -52,8 +52,7 @@ public class CliqueVotingBlockInterface implements VoteBlockInterface {
return Optional.empty();
}
@Override
public BlockHeaderBuilder insertVoteToHeaderBuilder(
public static BlockHeaderBuilder insertVoteToHeaderBuilder(
final BlockHeaderBuilder builder, final Optional<ValidatorVote> vote) {
if (vote.isPresent()) {
final ValidatorVote voteToCast = vote.get();

@ -40,8 +40,6 @@ import java.util.function.Function;
public class CliqueBlockCreator extends AbstractBlockCreator<CliqueContext> {
private final KeyPair nodeKeys;
private static final CliqueVotingBlockInterface votingInterface =
new CliqueVotingBlockInterface();
public CliqueBlockCreator(
final Address coinbase,
@ -93,9 +91,11 @@ public class CliqueBlockCreator extends AbstractBlockCreator<CliqueContext> {
cliqueContext
.getVoteProposer()
.getVote(Util.publicKeyToAddress(nodeKeys.getPublicKey()), voteTally);
votingInterface.insertVoteToHeaderBuilder(builder, vote);
final BlockHeaderBuilder builderIncludingProposedVotes =
CliqueVotingBlockInterface.insertVoteToHeaderBuilder(builder, vote);
final CliqueExtraData sealedExtraData = constructSignedExtraData(builder.buildBlockHeader());
final CliqueExtraData sealedExtraData =
constructSignedExtraData(builderIncludingProposedVotes.buildBlockHeader());
// Replace the extraData in the BlockHeaderBuilder, and return header.
return builder.extraData(sealedExtraData.encode()).buildBlockHeader();

@ -14,6 +14,7 @@ package tech.pegasys.pantheon.consensus.clique.blockcreation;
import tech.pegasys.pantheon.consensus.clique.CliqueContext;
import tech.pegasys.pantheon.consensus.clique.CliqueExtraData;
import tech.pegasys.pantheon.consensus.common.ConsensusHelpers;
import tech.pegasys.pantheon.consensus.common.EpochManager;
import tech.pegasys.pantheon.consensus.common.VoteTally;
import tech.pegasys.pantheon.crypto.SECP256K1.KeyPair;
@ -29,7 +30,6 @@ import tech.pegasys.pantheon.ethereum.core.Util;
import tech.pegasys.pantheon.ethereum.mainnet.ProtocolSchedule;
import tech.pegasys.pantheon.util.Subscribers;
import tech.pegasys.pantheon.util.bytes.BytesValue;
import tech.pegasys.pantheon.util.bytes.BytesValues;
import java.util.List;
import java.util.concurrent.ExecutorService;
@ -96,7 +96,8 @@ public class CliqueMinerExecutor extends AbstractMinerExecutor<CliqueContext, Cl
public BytesValue calculateExtraData(final BlockHeader parentHeader) {
final List<Address> validators = Lists.newArrayList();
final BytesValue vanityDataToInsert = createCorrectlySizedVanityData();
final BytesValue vanityDataToInsert =
ConsensusHelpers.zeroLeftPad(vanityData, CliqueExtraData.EXTRA_VANITY_LENGTH);
// Building ON TOP of canonical head, if the next block is epoch, include validators.
if (epochManager.isEpochBlock(parentHeader.getNumber() + 1)) {
final VoteTally voteTally =
@ -108,10 +109,4 @@ public class CliqueMinerExecutor extends AbstractMinerExecutor<CliqueContext, Cl
return extraData.encode();
}
private BytesValue createCorrectlySizedVanityData() {
final int vanityPadding = Math.max(0, CliqueExtraData.EXTRA_VANITY_LENGTH - vanityData.size());
return BytesValues.concatenate(BytesValue.wrap(new byte[vanityPadding]), vanityData)
.slice(0, CliqueExtraData.EXTRA_VANITY_LENGTH);
}
}

@ -86,7 +86,7 @@ public class CliqueVotingBlockInterfaceTest {
final ValidatorVote vote =
new ValidatorVote(ADD, AddressHelpers.ofValue(1), AddressHelpers.ofValue(2));
final BlockHeaderBuilder builderWithVote =
blockInterface.insertVoteToHeaderBuilder(builder, Optional.of(vote));
CliqueVotingBlockInterface.insertVoteToHeaderBuilder(builder, Optional.of(vote));
final BlockHeader header = builderWithVote.buildBlockHeader();
@ -100,7 +100,7 @@ public class CliqueVotingBlockInterfaceTest {
final ValidatorVote vote =
new ValidatorVote(DROP, AddressHelpers.ofValue(1), AddressHelpers.ofValue(2));
final BlockHeaderBuilder builderWithVote =
blockInterface.insertVoteToHeaderBuilder(builder, Optional.of(vote));
CliqueVotingBlockInterface.insertVoteToHeaderBuilder(builder, Optional.of(vote));
final BlockHeader header = builderWithVote.buildBlockHeader();
@ -111,7 +111,7 @@ public class CliqueVotingBlockInterfaceTest {
@Test
public void nonVoteBlendedIntoHeaderResultsInACoinbaseOfZero() {
final BlockHeaderBuilder builderWithVote =
blockInterface.insertVoteToHeaderBuilder(builder, Optional.empty());
CliqueVotingBlockInterface.insertVoteToHeaderBuilder(builder, Optional.empty());
final BlockHeader header = builderWithVote.buildBlockHeader();

@ -0,0 +1,25 @@
/*
* 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.util.bytes.BytesValue;
import tech.pegasys.pantheon.util.bytes.BytesValues;
public class ConsensusHelpers {
public static BytesValue zeroLeftPad(final BytesValue input, final int requiredLength) {
final int paddingByteCount = Math.max(0, requiredLength - input.size());
return BytesValues.concatenate(BytesValue.wrap(new byte[paddingByteCount]), input)
.slice(0, requiredLength);
}
}

@ -14,7 +14,6 @@ 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;
@ -23,8 +22,5 @@ public interface VoteBlockInterface {
Optional<ValidatorVote> extractVoteFromHeader(final BlockHeader header);
BlockHeaderBuilder insertVoteToHeaderBuilder(
final BlockHeaderBuilder builder, final Optional<ValidatorVote> vote);
List<Address> validatorsInBlock(final BlockHeader header);
}

@ -12,9 +12,17 @@
*/
package tech.pegasys.pantheon.consensus.ibft;
import tech.pegasys.pantheon.consensus.ibft.blockcreation.IbftBlockCreatorFactory;
/** Stateful evaluator for ibft events */
public class IbftStateMachine {
private final IbftBlockCreatorFactory blockCreatorFactory;
public IbftStateMachine(final IbftBlockCreatorFactory blockCreatorFactory) {
this.blockCreatorFactory = blockCreatorFactory;
}
/**
* Attempt to consume the event and update the maintained state
*

@ -17,7 +17,6 @@ import tech.pegasys.pantheon.consensus.common.VoteBlockInterface;
import tech.pegasys.pantheon.consensus.common.VoteType;
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;
@ -40,29 +39,6 @@ public class IbftVotingBlockInterface implements VoteBlockInterface {
return Optional.empty();
}
@Override
public BlockHeaderBuilder insertVoteToHeaderBuilder(
final BlockHeaderBuilder builder, final Optional<ValidatorVote> vote) {
final BlockHeader header = builder.buildBlockHeader();
final IbftExtraData extraData = IbftExtraData.decode(header.getExtraData());
Optional<Vote> headerVote = Optional.empty();
if (vote.isPresent()) {
final ValidatorVote voteToCast = vote.get();
headerVote = Optional.of(new Vote(voteToCast.getRecipient(), voteToCast.getVotePolarity()));
}
final IbftExtraData includingVote =
new IbftExtraData(
extraData.getVanityData(),
extraData.getSeals(),
headerVote,
extraData.getRound(),
extraData.getValidators());
return BlockHeaderBuilder.fromHeader(header).extraData(includingVote.encode());
}
@Override
public List<Address> validatorsInBlock(final BlockHeader header) {
final IbftExtraData ibftExtraData = IbftExtraData.decode(header.getExtraData());

@ -16,40 +16,55 @@ import tech.pegasys.pantheon.consensus.ibft.IbftContext;
import tech.pegasys.pantheon.ethereum.ProtocolContext;
import tech.pegasys.pantheon.ethereum.blockcreation.AbstractBlockCreator;
import tech.pegasys.pantheon.ethereum.core.Address;
import tech.pegasys.pantheon.ethereum.core.BlockHashFunction;
import tech.pegasys.pantheon.ethereum.core.BlockHeader;
import tech.pegasys.pantheon.ethereum.core.BlockHeaderBuilder;
import tech.pegasys.pantheon.ethereum.core.Hash;
import tech.pegasys.pantheon.ethereum.core.PendingTransactions;
import tech.pegasys.pantheon.ethereum.core.SealableBlockHeader;
import tech.pegasys.pantheon.ethereum.core.Wei;
import tech.pegasys.pantheon.ethereum.mainnet.ProtocolSchedule;
import tech.pegasys.pantheon.ethereum.mainnet.ScheduleBasedBlockHashFunction;
import java.util.function.Function;
// TODO: Just a placeholder. Implementation is required.
// This class is responsible for creating a block without committer seals (basically it was just
// too hard to coordinate with the state machine).
public class IbftBlockCreator extends AbstractBlockCreator<IbftContext> {
public IbftBlockCreator(
final Address coinbase,
final Address localAddress,
final ExtraDataCalculator extraDataCalculator,
final PendingTransactions pendingTransactions,
final ProtocolContext<IbftContext> protocolContext,
final ProtocolSchedule<IbftContext> protocolSchedule,
final Function<Long, Long> gasLimitCalculator,
final Wei minTransactionGasPrice,
final Address miningBeneficiary,
final BlockHeader parentHeader) {
super(
coinbase,
localAddress,
extraDataCalculator,
pendingTransactions,
protocolContext,
protocolSchedule,
gasLimitCalculator,
minTransactionGasPrice,
miningBeneficiary,
localAddress,
parentHeader);
}
@Override
protected BlockHeader createFinalBlockHeader(final SealableBlockHeader sealableBlockHeader) {
return null;
final BlockHashFunction blockHashFunction =
ScheduleBasedBlockHashFunction.create(protocolSchedule);
final BlockHeaderBuilder builder =
BlockHeaderBuilder.create()
.populateFrom(sealableBlockHeader)
.mixHash(Hash.ZERO)
.nonce(0L)
.blockHashFunction(blockHashFunction);
return builder.buildBlockHeader();
}
}

@ -0,0 +1,110 @@
/*
* 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.blockcreation;
import tech.pegasys.pantheon.consensus.common.ConsensusHelpers;
import tech.pegasys.pantheon.consensus.common.ValidatorVote;
import tech.pegasys.pantheon.consensus.common.VoteTally;
import tech.pegasys.pantheon.consensus.ibft.IbftContext;
import tech.pegasys.pantheon.consensus.ibft.IbftExtraData;
import tech.pegasys.pantheon.consensus.ibft.Vote;
import tech.pegasys.pantheon.ethereum.ProtocolContext;
import tech.pegasys.pantheon.ethereum.core.Address;
import tech.pegasys.pantheon.ethereum.core.BlockHeader;
import tech.pegasys.pantheon.ethereum.core.MiningParameters;
import tech.pegasys.pantheon.ethereum.core.PendingTransactions;
import tech.pegasys.pantheon.ethereum.core.Wei;
import tech.pegasys.pantheon.ethereum.mainnet.ProtocolSchedule;
import tech.pegasys.pantheon.util.bytes.BytesValue;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Optional;
import java.util.function.Function;
public class IbftBlockCreatorFactory {
private final Function<Long, Long> gasLimitCalculator;
private final PendingTransactions pendingTransactions;
protected final ProtocolContext<IbftContext> protocolContext;
protected final ProtocolSchedule<IbftContext> protocolSchedule;
private final Address localAddress;
private volatile BytesValue vanityData;
private volatile Wei minTransactionGasPrice;
public IbftBlockCreatorFactory(
final Function<Long, Long> gasLimitCalculator,
final PendingTransactions pendingTransactions,
final ProtocolContext<IbftContext> protocolContext,
final ProtocolSchedule<IbftContext> protocolSchedule,
final MiningParameters miningParams,
final Address localAddress) {
this.gasLimitCalculator = gasLimitCalculator;
this.pendingTransactions = pendingTransactions;
this.protocolContext = protocolContext;
this.protocolSchedule = protocolSchedule;
this.localAddress = localAddress;
this.minTransactionGasPrice = miningParams.getMinTransactionGasPrice();
this.vanityData = miningParams.getExtraData();
}
public IbftBlockCreator create(final BlockHeader parentHeader, final int round) {
return new IbftBlockCreator(
localAddress,
ph -> createExtraData(round),
pendingTransactions,
protocolContext,
protocolSchedule,
gasLimitCalculator,
minTransactionGasPrice,
parentHeader);
}
public void setExtraData(final BytesValue extraData) {
this.vanityData = extraData.copy();
}
public void setMinTransactionGasPrice(final Wei minTransactionGasPrice) {
this.minTransactionGasPrice = minTransactionGasPrice.copy();
}
public Wei getMinTransactionGasPrice() {
return minTransactionGasPrice;
}
public BytesValue createExtraData(final int round) {
final VoteTally voteTally = protocolContext.getConsensusState().getVoteTally();
final Optional<ValidatorVote> proposal =
protocolContext.getConsensusState().getVoteProposer().getVote(localAddress, voteTally);
final List<Address> validators = new ArrayList<>(voteTally.getCurrentValidators());
final IbftExtraData extraData =
new IbftExtraData(
ConsensusHelpers.zeroLeftPad(vanityData, IbftExtraData.EXTRA_VANITY_LENGTH),
Collections.emptyList(),
toVote(proposal),
round,
validators);
return extraData.encode();
}
private static Optional<Vote> toVote(final Optional<ValidatorVote> input) {
return input
.map(v -> Optional.of(new Vote(v.getRecipient(), v.getVotePolarity())))
.orElse(Optional.empty());
}
}

@ -0,0 +1,50 @@
/*
* 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.blockcreation;
import tech.pegasys.pantheon.ethereum.blockcreation.MiningCoordinator;
import tech.pegasys.pantheon.ethereum.core.Wei;
import tech.pegasys.pantheon.util.bytes.BytesValue;
public class IbftMiningCoordinator implements MiningCoordinator {
private final IbftBlockCreatorFactory blockCreatorFactory;
public IbftMiningCoordinator(final IbftBlockCreatorFactory blockCreatorFactory) {
this.blockCreatorFactory = blockCreatorFactory;
}
@Override
public void enable() {}
@Override
public void disable() {}
@Override
public boolean isRunning() {
return true;
}
@Override
public void setMinTransactionGasPrice(final Wei minGasPrice) {}
@Override
public Wei getMinTransactionGasPrice() {
return null;
}
@Override
public void setExtraData(final BytesValue extraData) {
blockCreatorFactory.setExtraData(extraData);
}
}

@ -52,8 +52,7 @@ public class IbftLegacyVotingBlockInterface implements VoteBlockInterface {
return Optional.empty();
}
@Override
public BlockHeaderBuilder insertVoteToHeaderBuilder(
public static BlockHeaderBuilder insertVoteToHeaderBuilder(
final BlockHeaderBuilder builder, final Optional<ValidatorVote> vote) {
if (vote.isPresent()) {
final ValidatorVote voteToCast = vote.get();

@ -86,7 +86,7 @@ public class IbftLegacyVotingBlockInterfaceTest {
final ValidatorVote vote =
new ValidatorVote(ADD, AddressHelpers.ofValue(1), AddressHelpers.ofValue(2));
final BlockHeaderBuilder builderWithVote =
blockInterface.insertVoteToHeaderBuilder(builder, Optional.of(vote));
IbftLegacyVotingBlockInterface.insertVoteToHeaderBuilder(builder, Optional.of(vote));
final BlockHeader header = builderWithVote.buildBlockHeader();
@ -99,7 +99,7 @@ public class IbftLegacyVotingBlockInterfaceTest {
final ValidatorVote vote =
new ValidatorVote(DROP, AddressHelpers.ofValue(1), AddressHelpers.ofValue(2));
final BlockHeaderBuilder builderWithVote =
blockInterface.insertVoteToHeaderBuilder(builder, Optional.of(vote));
IbftLegacyVotingBlockInterface.insertVoteToHeaderBuilder(builder, Optional.of(vote));
final BlockHeader header = builderWithVote.buildBlockHeader();
@ -110,7 +110,7 @@ public class IbftLegacyVotingBlockInterfaceTest {
@Test
public void nonVoteBlendedIntoHeaderResultsInACoinbaseOfZero() {
final BlockHeaderBuilder builderWithVote =
blockInterface.insertVoteToHeaderBuilder(builder, Optional.empty());
IbftLegacyVotingBlockInterface.insertVoteToHeaderBuilder(builder, Optional.empty());
final BlockHeader header = builderWithVote.buildBlockHeader();
@ -123,8 +123,7 @@ public class IbftLegacyVotingBlockInterfaceTest {
final BlockHeader header =
TestHelpers.createIbftSignedBlockHeader(headerBuilder, proposerKeys, validatorList);
final IbftLegacyVotingBlockInterface serDeser = new IbftLegacyVotingBlockInterface();
final List<Address> extractedValidators = serDeser.validatorsInBlock(header);
final List<Address> extractedValidators = blockInterface.validatorsInBlock(header);
assertThat(extractedValidators).isEqualTo(validatorList);
}

@ -18,12 +18,9 @@ import tech.pegasys.pantheon.ethereum.chain.BlockAddedEvent;
import tech.pegasys.pantheon.ethereum.chain.BlockAddedObserver;
import tech.pegasys.pantheon.ethereum.chain.Blockchain;
import tech.pegasys.pantheon.ethereum.chain.MinedBlockObserver;
import tech.pegasys.pantheon.ethereum.core.Address;
import tech.pegasys.pantheon.ethereum.core.BlockHeader;
import tech.pegasys.pantheon.ethereum.core.Wei;
import tech.pegasys.pantheon.ethereum.eth.sync.state.SyncState;
import tech.pegasys.pantheon.ethereum.mainnet.EthHashSolution;
import tech.pegasys.pantheon.ethereum.mainnet.EthHashSolverInputs;
import tech.pegasys.pantheon.util.Subscribers;
import tech.pegasys.pantheon.util.bytes.BytesValue;
@ -138,34 +135,4 @@ public abstract class AbstractMiningCoordinator<
public void setExtraData(final BytesValue extraData) {
executor.setExtraData(extraData);
}
@Override
public void setCoinbase(final Address coinbase) {
throw new UnsupportedOperationException(
"Current consensus mechanism prevents setting coinbase.");
}
@Override
public Optional<Address> getCoinbase() {
throw new UnsupportedOperationException(
"Current consensus mechanism prevents querying of coinbase.");
}
@Override
public Optional<Long> hashesPerSecond() {
throw new UnsupportedOperationException(
"Current consensus mechanism prevents querying of hashrate.");
}
@Override
public Optional<EthHashSolverInputs> getWorkDefinition() {
throw new UnsupportedOperationException(
"Current consensus mechanism prevents querying work definition.");
}
@Override
public boolean submitWork(final EthHashSolution solution) {
throw new UnsupportedOperationException(
"Current consensus mechanism prevents submission of work solutions.");
}
}

@ -35,13 +35,28 @@ public interface MiningCoordinator {
void setExtraData(BytesValue extraData);
void setCoinbase(Address coinbase);
Optional<Address> getCoinbase();
Optional<Long> hashesPerSecond();
Optional<EthHashSolverInputs> getWorkDefinition();
boolean submitWork(EthHashSolution solution);
default void setCoinbase(final Address coinbase) {
throw new UnsupportedOperationException(
"Current consensus mechanism prevents setting coinbase.");
}
default Optional<Address> getCoinbase() {
throw new UnsupportedOperationException(
"Current consensus mechanism prevents querying of coinbase.");
}
default Optional<Long> hashesPerSecond() {
throw new UnsupportedOperationException(
"Current consensus mechanism prevents querying of hashrate.");
}
default Optional<EthHashSolverInputs> getWorkDefinition() {
throw new UnsupportedOperationException(
"Current consensus mechanism prevents querying work definition.");
}
default boolean submitWork(final EthHashSolution solution) {
throw new UnsupportedOperationException(
"Current consensus mechanism prevents submission of work solutions.");
}
}

@ -25,6 +25,8 @@ import tech.pegasys.pantheon.consensus.ibft.IbftContext;
import tech.pegasys.pantheon.consensus.ibft.IbftEventQueue;
import tech.pegasys.pantheon.consensus.ibft.IbftProcessor;
import tech.pegasys.pantheon.consensus.ibft.IbftStateMachine;
import tech.pegasys.pantheon.consensus.ibft.blockcreation.IbftBlockCreatorFactory;
import tech.pegasys.pantheon.consensus.ibft.blockcreation.IbftMiningCoordinator;
import tech.pegasys.pantheon.consensus.ibft.network.IbftNetworkPeers;
import tech.pegasys.pantheon.consensus.ibft.protocol.IbftProtocolManager;
import tech.pegasys.pantheon.consensus.ibft.protocol.IbftSubProtocol;
@ -38,8 +40,10 @@ import tech.pegasys.pantheon.ethereum.blockcreation.MiningCoordinator;
import tech.pegasys.pantheon.ethereum.chain.GenesisState;
import tech.pegasys.pantheon.ethereum.chain.MutableBlockchain;
import tech.pegasys.pantheon.ethereum.core.Hash;
import tech.pegasys.pantheon.ethereum.core.MiningParameters;
import tech.pegasys.pantheon.ethereum.core.Synchronizer;
import tech.pegasys.pantheon.ethereum.core.TransactionPool;
import tech.pegasys.pantheon.ethereum.core.Util;
import tech.pegasys.pantheon.ethereum.db.BlockchainStorage;
import tech.pegasys.pantheon.ethereum.db.DefaultMutableBlockchain;
import tech.pegasys.pantheon.ethereum.db.WorldStateArchive;
@ -106,6 +110,7 @@ public class IbftPantheonController implements PantheonController<IbftContext> {
final StorageProvider storageProvider,
final GenesisConfigFile genesisConfig,
final SynchronizerConfiguration taintedSyncConfig,
final MiningParameters miningParams,
final boolean ottomanTestnetOperation,
final int networkId,
final KeyPair nodeKeys) {
@ -167,19 +172,35 @@ public class IbftPantheonController implements PantheonController<IbftContext> {
ethProtocolManager.ethContext(),
syncState);
final TransactionPool transactionPool =
TransactionPoolFactory.createTransactionPool(
protocolSchedule, protocolContext, ethProtocolManager.ethContext());
final IbftEventQueue ibftEventQueue = new IbftEventQueue();
blockchain.observeBlockAdded(new IbftChainObserver(ibftEventQueue));
final IbftStateMachine ibftStateMachine = new IbftStateMachine();
final IbftBlockCreatorFactory blockCreatorFactory =
new IbftBlockCreatorFactory(
(gasLimit) -> gasLimit,
transactionPool.getPendingTransactions(),
protocolContext,
protocolSchedule,
miningParams,
Util.publicKeyToAddress(nodeKeys.getPublicKey()));
final IbftStateMachine ibftStateMachine = new IbftStateMachine(blockCreatorFactory);
final IbftProcessor ibftProcessor =
new IbftProcessor(ibftEventQueue, ibftConfig.getRequestTimeoutMillis(), ibftStateMachine);
final ExecutorService processorExecutor = Executors.newSingleThreadExecutor();
processorExecutor.submit(ibftProcessor);
final MiningCoordinator ibftMiningCoordinator = new IbftMiningCoordinator(blockCreatorFactory);
final Runnable closer =
() -> {
ibftProcessor.stop();
ibftMiningCoordinator.disable();
processorExecutor.shutdownNow();
try {
processorExecutor.awaitTermination(5, TimeUnit.SECONDS);
@ -193,10 +214,6 @@ public class IbftPantheonController implements PantheonController<IbftContext> {
}
};
final TransactionPool transactionPool =
TransactionPoolFactory.createTransactionPool(
protocolSchedule, protocolContext, ethProtocolManager.ethContext());
final IbftNetworkPeers peers =
new IbftNetworkPeers(protocolContext.getConsensusState().getVoteTally());

@ -56,6 +56,7 @@ public interface PantheonController<C> extends Closeable {
storageProvider,
genesisConfigFile,
syncConfig,
miningParameters,
ottomanTestnetOperation,
networkId,
nodeKeys);

Loading…
Cancel
Save