From 4c001614b4cf00973f8d7a8e6478627d119d9126 Mon Sep 17 00:00:00 2001 From: Leonor Boga Date: Fri, 23 Jul 2021 14:47:25 +0200 Subject: [PATCH 01/13] ETCM-1058 Add new Consensus, by just moving the code in BlockImport --- .../sync/regular/BlockImportResult.scala | 25 + .../sync/regular/BlockImporter.scala | 20 +- .../iohk/ethereum/consensus/Consensus.scala | 68 ++ .../ethereum/consensus/ConsensusImpl.scala | 357 +++++++++ .../io/iohk/ethereum/ledger/BlockImport.scala | 708 +++++++++--------- 5 files changed, 802 insertions(+), 376 deletions(-) create mode 100644 src/main/scala/io/iohk/ethereum/blockchain/sync/regular/BlockImportResult.scala create mode 100644 src/main/scala/io/iohk/ethereum/consensus/Consensus.scala create mode 100644 src/main/scala/io/iohk/ethereum/consensus/ConsensusImpl.scala diff --git a/src/main/scala/io/iohk/ethereum/blockchain/sync/regular/BlockImportResult.scala b/src/main/scala/io/iohk/ethereum/blockchain/sync/regular/BlockImportResult.scala new file mode 100644 index 0000000000..2ab921d5cb --- /dev/null +++ b/src/main/scala/io/iohk/ethereum/blockchain/sync/regular/BlockImportResult.scala @@ -0,0 +1,25 @@ +package io.iohk.ethereum.blockchain.sync.regular + +import io.iohk.ethereum.domain.{Block, ChainWeight} +import io.iohk.ethereum.ledger.BlockData +import io.iohk.ethereum.mpt.MerklePatriciaTrie.MissingNodeException + +sealed trait BlockImportResult + +case class BlockImportedToTop(blockImportData: List[BlockData]) extends BlockImportResult + +case object BlockEnqueued extends BlockImportResult + +case object DuplicateBlock extends BlockImportResult + +case class ChainReorganised( + oldBranch: List[Block], + newBranch: List[Block], + weights: List[ChainWeight] +) extends BlockImportResult + +case class BlockImportFailed(error: String) extends BlockImportResult + +case class BlockImportFailedDueToMissingNode(reason: MissingNodeException) extends BlockImportResult + +case object UnknownParent extends BlockImportResult diff --git a/src/main/scala/io/iohk/ethereum/blockchain/sync/regular/BlockImporter.scala b/src/main/scala/io/iohk/ethereum/blockchain/sync/regular/BlockImporter.scala index 97de4c3842..b951b8d66f 100644 --- a/src/main/scala/io/iohk/ethereum/blockchain/sync/regular/BlockImporter.scala +++ b/src/main/scala/io/iohk/ethereum/blockchain/sync/regular/BlockImporter.scala @@ -7,19 +7,17 @@ import akka.actor.ActorRef import akka.actor.NotInfluenceReceiveTimeout import akka.actor.Props import akka.actor.ReceiveTimeout - import cats.data.NonEmptyList import cats.implicits._ - import monix.eval.Task import monix.execution.Scheduler import scala.concurrent.duration._ - import io.iohk.ethereum.blockchain.sync.Blacklist.BlacklistReason import io.iohk.ethereum.blockchain.sync.regular.BlockBroadcast.BlockToBroadcast import io.iohk.ethereum.blockchain.sync.regular.BlockBroadcasterActor.BroadcastBlocks import io.iohk.ethereum.blockchain.sync.regular.RegularSync.ProgressProtocol +import io.iohk.ethereum.consensus.Consensus import io.iohk.ethereum.crypto.kec256 import io.iohk.ethereum.db.storage.StateStorage import io.iohk.ethereum.domain._ @@ -37,8 +35,7 @@ import io.iohk.ethereum.utils.FunctorOps._ class BlockImporter( fetcher: ActorRef, - blockImport: BlockImport, - blockchain: Blockchain, + consensus: Consensus, blockchainReader: BlockchainReader, stateStorage: StateStorage, branchResolution: BranchResolution, @@ -50,6 +47,7 @@ class BlockImporter( configBuilder: BlockchainConfigBuilder ) extends Actor with ActorLogging { + import BlockImporter._ import configBuilder._ @@ -199,8 +197,8 @@ class BlockImporter( Task.now((importedBlocks, None)) } else { val restOfBlocks = blocks.tail - blockImport - .importBlock(blocks.head) + consensus + .evaluateBranchBlock(blocks.head) .flatMap { case BlockImportedToTop(_) => tryImportBlocks(restOfBlocks, blocks.head :: importedBlocks) @@ -238,7 +236,7 @@ class BlockImporter( def doLog(entry: ImportMessages.LogEntry): Unit = log.log(entry._1, entry._2) importWith( Task(doLog(importMessages.preImport())) - .flatMap(_ => blockImport.importBlock(block)) + .flatMap(_ => consensus.evaluateBranchBlock(block)) .tap((importMessages.messageForImportResult _).andThen(doLog)) .tap { case BlockImportedToTop(importedBlocksData) => @@ -330,8 +328,7 @@ object BlockImporter { // scalastyle:off parameter.number def props( fetcher: ActorRef, - blockImport: BlockImport, - blockchain: Blockchain, + consensus: Consensus, blockchainReader: BlockchainReader, stateStorage: StateStorage, branchResolution: BranchResolution, @@ -345,8 +342,7 @@ object BlockImporter { Props( new BlockImporter( fetcher, - blockImport, - blockchain, + consensus, blockchainReader, stateStorage, branchResolution, diff --git a/src/main/scala/io/iohk/ethereum/consensus/Consensus.scala b/src/main/scala/io/iohk/ethereum/consensus/Consensus.scala new file mode 100644 index 0000000000..9165d6653c --- /dev/null +++ b/src/main/scala/io/iohk/ethereum/consensus/Consensus.scala @@ -0,0 +1,68 @@ +package io.iohk.ethereum.consensus + +import io.iohk.ethereum.blockchain.sync.regular.BlockImportResult +import io.iohk.ethereum.domain.Block +import io.iohk.ethereum.utils.BlockchainConfig +import monix.eval.Task +import monix.execution.Scheduler + +/** This file documents the original interface that was designed at ETCM-1018 + * but implements a different one to be used as a stepping stone to the new architecture + * still in progress + */ +trait Consensus { + def evaluateBranchBlock( + block: Block + )(implicit blockExecutionScheduler: Scheduler, blockchainConfig: BlockchainConfig): Task[BlockImportResult] + + /** Original interface from ETCM-1018, for temporary documentation purposes + */ + /** Answer which branch is best + * @return branch.Branch + */ +// def getBestBranch(): branch.Branch = blockchainReader.getBestBranch() + + /** @param branch + * This methods received a Branch that was updated by ChainManagement. + * When a Branch is updated we need to compare the weight of the current best branch with the + * updated one. + * If the current best branch is still the best then nothing needs to be done. + * If the updated branch is heavier than an attempt to set the updated branch as best branch is done by + * executing the blocks in the updated branch to see if it is a valid branch. + * If it is not a valid branch then ExecutingSync has to be informed, otherwise update state with new best branch. + */ +// def evaluateBranch(branch: UpdatedBranch): Either[BlockExecutionError, Boolean] = +// if (extendsBestBranch()) { +// // just validate the latest block +// Right(true) +// } else { +// if (isHeavierThanBestBranch(branch)) { +// // create a queue of (branchTip, CancelableFuture) +// // if any branch is being executed at the moment while a better one comes is then call the cancellation hook +// attemptToSetNewBestBranch(branch) match { +// case Right(result) => // save pointer to new best branch +// Right(true) +// case Left(error) => Left(error) +// } +// } else { +// // nothing +// Right(true) +// } +// } + +// private def extendsBestBranch(): Boolean = ??? + + /** Compares the weight of the updatedBranch with the weight of the current best branch + * @param updatedBranch + * @return true if updatedBranch is heavier than current best branch, false otherwise + */ +// private def isHeavierThanBestBranch(updatedBranch: UpdatedBranch): Boolean = ??? + + /** Tries to set a new best branch by executing all blocks in the branch, from the HCB to the branch tip. + * We assume the pre validation of the blocks of the branch was done already + * @param branch + * @return Either[BlockExecutionError, Boolean] + */ +// private def attemptToSetNewBestBranch(branch: UpdatedBranch): Either[BlockExecutionError, Boolean] = ??? + +} diff --git a/src/main/scala/io/iohk/ethereum/consensus/ConsensusImpl.scala b/src/main/scala/io/iohk/ethereum/consensus/ConsensusImpl.scala new file mode 100644 index 0000000000..7c36333a03 --- /dev/null +++ b/src/main/scala/io/iohk/ethereum/consensus/ConsensusImpl.scala @@ -0,0 +1,357 @@ +package io.iohk.ethereum.consensus + +import akka.util.ByteString +import io.iohk.ethereum.blockchain.sync.regular.{ + BlockEnqueued, + BlockImportFailed, + BlockImportFailedDueToMissingNode, + BlockImportResult, + BlockImportedToTop, + ChainReorganised, + DuplicateBlock, + UnknownParent +} +import io.iohk.ethereum.consensus.validators.BlockHeaderError.HeaderParentNotFoundError +import io.iohk.ethereum.domain.{Block, BlockHeader, BlockchainImpl, BlockchainReader, BlockchainWriter, ChainWeight} +import io.iohk.ethereum.ledger.BlockExecutionError.{MPTError, ValidationBeforeExecError} +import io.iohk.ethereum.ledger.BlockQueue.Leaf +import io.iohk.ethereum.ledger.{ + BlockData, + BlockExecution, + BlockExecutionError, + BlockMetrics, + BlockQueue, + BlockValidation +} +import io.iohk.ethereum.mpt.MerklePatriciaTrie.MissingNodeException +import io.iohk.ethereum.utils.{BlockchainConfig, ByteStringUtils, Logger} +import monix.eval.Task +import monix.execution.Scheduler +import org.bouncycastle.util.encoders.Hex + +import scala.annotation.tailrec + +class ConsensusImpl( + blockchain: BlockchainImpl, + blockchainReader: BlockchainReader, + blockchainWriter: BlockchainWriter, + blockQueue: BlockQueue, + blockValidation: BlockValidation, + blockExecution: BlockExecution, + validationScheduler: Scheduler // Can't be implicit because of importToTop method and ambiguous of Scheduler +) extends Consensus + with Logger { + + /** Tries to import the block as the new best block in the chain or enqueue it for later processing. + * + * @param block block to be imported + * @param blockExecutionContext threadPool on which the execution should be run + * @return One of: + * - [[BlockImportedToTop]] - if the block was added as the new best block + * - [[BlockEnqueued]] - block is stored in the [[io.iohk.ethereum.ledger.BlockQueue]] + * - [[ChainReorganised]] - a better new branch was found causing chain reorganisation + * - [[DuplicateBlock]] - block already exists either in the main chain or in the queue + * - [[BlockImportFailed]] - block failed to execute (when importing to top or reorganising the chain) + */ + override def evaluateBranchBlock( + block: Block + )(implicit blockExecutionScheduler: Scheduler, blockchainConfig: BlockchainConfig): Task[BlockImportResult] = + blockchainReader.getBestBlock() match { + case Some(bestBlock) => + if (isBlockADuplicate(block.header, bestBlock.header.number)) { + Task(log.debug(s"Ignoring duplicate block: (${block.idTag})")) + .map(_ => DuplicateBlock) + } else { + val hash = bestBlock.header.hash + blockchain.getChainWeightByHash(hash) match { + case Some(weight) => + val importResult = if (isPossibleNewBestBlock(block.header, bestBlock.header)) { + importToTop(block, bestBlock, weight) + } else { + reorganise(block, bestBlock, weight) + } + importResult.foreach(measureBlockMetrics) + importResult + case None => + log.error( + "Getting total difficulty for current best block with hash: {} failed", + bestBlock.header.hashAsHexString + ) + Task.now( + BlockImportFailed( + s"Couldn't get total difficulty for current best block with hash: ${bestBlock.header.hashAsHexString}" + ) + ) + } + } + case None => + log.error("Getting current best block failed") + Task.now(BlockImportFailed("Couldn't find the current best block")) + } + + private def isBlockADuplicate(block: BlockHeader, currentBestBlockNumber: BigInt): Boolean = { + val hash = block.hash + blockchainReader.getBlockByHash(hash).isDefined && block.number <= currentBestBlockNumber || blockQueue.isQueued( + hash + ) + } + + private def isPossibleNewBestBlock(newBlock: BlockHeader, currentBestBlock: BlockHeader): Boolean = + newBlock.parentHash == currentBestBlock.hash && newBlock.number == currentBestBlock.number + 1 + + private def measureBlockMetrics(importResult: BlockImportResult): Unit = + importResult match { + case BlockImportedToTop(blockImportData) => + blockImportData.foreach(blockData => BlockMetrics.measure(blockData.block, blockchainReader.getBlockByHash)) + case ChainReorganised(_, newBranch, _) => + newBranch.foreach(block => BlockMetrics.measure(block, blockchainReader.getBlockByHash)) + case _ => () + } + + private def importToTop( + block: Block, + currentBestBlock: Block, + currentWeight: ChainWeight + )(implicit blockExecutionScheduler: Scheduler, blockchainConfig: BlockchainConfig): Task[BlockImportResult] = { + val validationResult = + Task.evalOnce(blockValidation.validateBlockBeforeExecution(block)).executeOn(validationScheduler) + val importResult = + Task + .evalOnce(importBlockToTop(block, currentBestBlock.header.number, currentWeight)) + .executeOn(blockExecutionScheduler) + + Task.parMap2(validationResult, importResult) { case (validationResult, importResult) => + validationResult.fold( + error => { + log.error("Error while validation block before execution: {}", error.reason) + handleImportTopValidationError(error, block, importResult) + }, + _ => importResult + ) + } + } + + private def importBlockToTop(block: Block, bestBlockNumber: BigInt, currentWeight: ChainWeight)(implicit + blockchainConfig: BlockchainConfig + ): BlockImportResult = { + val executionResult = for { + topBlock <- blockQueue.enqueueBlock(block, bestBlockNumber) + topBlocks = blockQueue.getBranch(topBlock.hash, dequeue = true) + (executed, errors) = blockExecution.executeAndValidateBlocks(topBlocks, currentWeight) + } yield (executed, errors, topBlocks) + + executionResult match { + case Some((importedBlocks, maybeError, topBlocks)) => + val result = maybeError match { + case None => + BlockImportedToTop(importedBlocks) + + case Some(MPTError(reason)) if reason.isInstanceOf[MissingNodeException] => + BlockImportFailedDueToMissingNode(reason.asInstanceOf[MissingNodeException]) + + case Some(error) if importedBlocks.isEmpty => + blockQueue.removeSubtree(block.header.hash) + BlockImportFailed(error.toString) + + case Some(_) => + topBlocks.drop(importedBlocks.length).headOption.foreach { failedBlock => + blockQueue.removeSubtree(failedBlock.header.hash) + } + BlockImportedToTop(importedBlocks) + } + log.debug( + "{}", { + val result = importedBlocks.map { blockData => + val header = blockData.block.header + s"Imported new block (${header.number}: ${Hex.toHexString(header.hash.toArray)}) to the top of chain \n" + } + result.toString + } + ) + + result + + case None => + BlockImportFailed(s"Newly enqueued block with hash: ${block.header.hash} is not part of a known branch") + } + } + + private def handleImportTopValidationError( + error: ValidationBeforeExecError, + block: Block, + blockImportResult: BlockImportResult + ): BlockImportResult = { + blockImportResult match { + case BlockImportedToTop(blockImportData) => + blockImportData.foreach { blockData => + val hash = blockData.block.header.hash + blockQueue.removeSubtree(hash) + blockchain.removeBlock(hash, withState = true) + } + case _ => () + } + handleBlockValidationError(error, block) + } + + private def handleBlockValidationError(error: ValidationBeforeExecError, block: Block): BlockImportResult = + error match { + case ValidationBeforeExecError(HeaderParentNotFoundError) => + log.debug(s"Block(${block.idTag}) has unknown parent") + UnknownParent + + case ValidationBeforeExecError(reason) => + log.debug(s"Block(${block.idTag}) failed pre-import validation") + BlockImportFailed(reason.toString) + } + + private def reorganise( + block: Block, + currentBestBlock: Block, + currentWeight: ChainWeight + )(implicit blockchainConfig: BlockchainConfig): Task[BlockImportResult] = + Task.evalOnce { + blockValidation + .validateBlockBeforeExecution(block) + .fold( + error => handleBlockValidationError(error, block), + _ => + blockQueue.enqueueBlock(block, currentBestBlock.header.number) match { + case Some(Leaf(leafHash, leafWeight)) if leafWeight > currentWeight => + log.debug("Found a better chain, about to reorganise") + reorganiseChainFromQueue(leafHash) + + case _ => + BlockEnqueued + } + ) + } + + /** Once a better branch was found this attempts to reorganise the chain + * + * @param queuedLeaf a block hash that determines a new branch stored in the queue (newest block from the branch) + * + * @return [[BlockExecutionError]] if one of the blocks in the new branch failed to execute, otherwise: + * (oldBranch, newBranch) as lists of blocks + */ + private def reorganiseChainFromQueue( + queuedLeaf: ByteString + )(implicit blockchainConfig: BlockchainConfig): BlockImportResult = { + log.debug("Reorganising chain from leaf {}", ByteStringUtils.hash2string(queuedLeaf)) + val newBranch = blockQueue.getBranch(queuedLeaf, dequeue = true) + val bestNumber = blockchainReader.getBestBlockNumber() + + val reorgResult = for { + parent <- newBranch.headOption + parentHash = parent.header.parentHash + parentWeight <- blockchain.getChainWeightByHash(parentHash) + } yield { + log.debug( + "Removing blocks starting from number {} and parent {}", + bestNumber, + ByteStringUtils.hash2string(parentHash) + ) + val oldBlocksData = removeBlocksUntil(parentHash, bestNumber) + oldBlocksData.foreach(block => blockQueue.enqueueBlock(block.block)) + handleBlockExecResult(newBranch, parentWeight, oldBlocksData) + } + + reorgResult match { + case Some(execResult) => + execResult.fold( + { + case MPTError(reason: MissingNodeException) => BlockImportFailedDueToMissingNode(reason) + case err => BlockImportFailed(s"Error while trying to reorganise chain: $err") + }, + ChainReorganised.tupled + ) + + case None => + BlockImportFailed("Error while trying to reorganise chain with parent of new branch") + } + } + + private def handleBlockExecResult( + newBranch: List[Block], + parentWeight: ChainWeight, + oldBlocksData: List[BlockData] + )(implicit + blockchainConfig: BlockchainConfig + ): Either[BlockExecutionError, (List[Block], List[Block], List[ChainWeight])] = { + val (executedBlocks, maybeError) = blockExecution.executeAndValidateBlocks(newBranch, parentWeight) + maybeError match { + case None => + Right((oldBlocksData.map(_.block), executedBlocks.map(_.block), executedBlocks.map(_.weight))) + + case Some(error) => + revertChainReorganisation(newBranch, oldBlocksData, executedBlocks) + Left(error) + } + } + + /** Reverts chain reorganisation in the event that one of the blocks from new branch fails to execute + * + * @param newBranch new blocks + * @param oldBranch old blocks along with corresponding receipts and totalDifficulties + * @param executedBlocks sub-sequence of new branch that was executed correctly + */ + private def revertChainReorganisation( + newBranch: List[Block], + oldBranch: List[BlockData], + executedBlocks: List[BlockData] + ): Unit = { + if (executedBlocks.nonEmpty) { + removeBlocksUntil(executedBlocks.head.block.header.parentHash, executedBlocks.last.block.header.number) + } + + oldBranch.foreach { case BlockData(block, receipts, weight) => + blockchainWriter.save(block, receipts, weight, saveAsBestBlock = false) + } + + import cats.implicits._ + val checkpointNumber = oldBranch.collect { + case BlockData(block, _, _) if block.hasCheckpoint => block.number + }.maximumOption + + val bestNumber = oldBranch.last.block.header.number + blockchain.saveBestKnownBlocks(bestNumber, checkpointNumber) + executedBlocks.foreach(data => blockQueue.enqueueBlock(data.block, bestNumber)) + + newBranch.diff(executedBlocks.map(_.block)).headOption.foreach { block => + blockQueue.removeSubtree(block.header.hash) + } + } + + /** Removes blocks from the [[Blockchain]] along with receipts and total difficulties. + * + * @param parent remove blocks until this hash (exclusive) + * @param fromNumber start removing from this number (downwards) + * + * @return the list of removed blocks along with receipts and total difficulties + */ + private def removeBlocksUntil(parent: ByteString, fromNumber: BigInt): List[BlockData] = { + @tailrec + def removeBlocksUntil(parent: ByteString, fromNumber: BigInt, acc: List[BlockData]): List[BlockData] = + blockchainReader.getBestBranch().getBlockByNumber(fromNumber) match { + case Some(block) if block.header.hash == parent || fromNumber == 0 => + acc + + case Some(block) => + val hash = block.header.hash + + val blockDataOpt = for { + receipts <- blockchainReader.getReceiptsByHash(hash) + weight <- blockchain.getChainWeightByHash(hash) + } yield BlockData(block, receipts, weight) + + blockchain.removeBlock(hash, withState = true) + + removeBlocksUntil(parent, fromNumber - 1, blockDataOpt.map(_ :: acc).getOrElse(acc)) + + case None => + log.error(s"Unexpected missing block number: $fromNumber") + acc + } + + removeBlocksUntil(parent, fromNumber, Nil) + } +} diff --git a/src/main/scala/io/iohk/ethereum/ledger/BlockImport.scala b/src/main/scala/io/iohk/ethereum/ledger/BlockImport.scala index 788634fe9a..f365aaa88b 100644 --- a/src/main/scala/io/iohk/ethereum/ledger/BlockImport.scala +++ b/src/main/scala/io/iohk/ethereum/ledger/BlockImport.scala @@ -1,364 +1,344 @@ -package io.iohk.ethereum.ledger - -import akka.util.ByteString - -import monix.eval.Task -import monix.execution.Scheduler - -import scala.annotation.tailrec - -import org.bouncycastle.util.encoders.Hex - -import io.iohk.ethereum.consensus.validators.BlockHeaderError.HeaderParentNotFoundError -import io.iohk.ethereum.domain._ -import io.iohk.ethereum.ledger.BlockExecutionError.MPTError -import io.iohk.ethereum.ledger.BlockExecutionError.ValidationBeforeExecError -import io.iohk.ethereum.ledger.BlockQueue.Leaf -import io.iohk.ethereum.mpt.MerklePatriciaTrie.MissingNodeException -import io.iohk.ethereum.utils.BlockchainConfig -import io.iohk.ethereum.utils.ByteStringUtils -import io.iohk.ethereum.utils.Logger - -class BlockImport( - blockchain: BlockchainImpl, - blockchainReader: BlockchainReader, - blockchainWriter: BlockchainWriter, - blockQueue: BlockQueue, - blockValidation: BlockValidation, - private[ledger] val blockExecution: BlockExecution, - validationScheduler: Scheduler // Can't be implicit because of importToTop method and ambiguous of Scheduler -) extends Logger { - - /** Tries to import the block as the new best block in the chain or enqueue it for later processing. - * - * @param block block to be imported - * @param blockExecutionContext threadPool on which the execution should be run - * @return One of: - * - [[io.iohk.ethereum.ledger.BlockImportedToTop]] - if the block was added as the new best block - * - [[io.iohk.ethereum.ledger.BlockEnqueued]] - block is stored in the [[io.iohk.ethereum.ledger.BlockQueue]] - * - [[io.iohk.ethereum.ledger.ChainReorganised]] - a better new branch was found causing chain reorganisation - * - [[io.iohk.ethereum.ledger.DuplicateBlock]] - block already exists either in the main chain or in the queue - * - [[io.iohk.ethereum.ledger.BlockImportFailed]] - block failed to execute (when importing to top or reorganising the chain) - */ - def importBlock( - block: Block - )(implicit blockExecutionScheduler: Scheduler, blockchainConfig: BlockchainConfig): Task[BlockImportResult] = - blockchainReader.getBestBlock() match { - case Some(bestBlock) => - if (isBlockADuplicate(block.header, bestBlock.header.number)) { - Task(log.debug(s"Ignoring duplicate block: (${block.idTag})")) - .map(_ => DuplicateBlock) - } else { - val hash = bestBlock.header.hash - blockchain.getChainWeightByHash(hash) match { - case Some(weight) => - val importResult = if (isPossibleNewBestBlock(block.header, bestBlock.header)) { - importToTop(block, bestBlock, weight) - } else { - reorganise(block, bestBlock, weight) - } - importResult.foreach(measureBlockMetrics) - importResult - case None => - log.error( - "Getting total difficulty for current best block with hash: {} failed", - bestBlock.header.hashAsHexString - ) - Task.now( - BlockImportFailed( - s"Couldn't get total difficulty for current best block with hash: ${bestBlock.header.hashAsHexString}" - ) - ) - } - } - case None => - log.error("Getting current best block failed") - Task.now(BlockImportFailed("Couldn't find the current best block")) - } - - private def isBlockADuplicate(block: BlockHeader, currentBestBlockNumber: BigInt): Boolean = { - val hash = block.hash - blockchainReader.getBlockByHash(hash).isDefined && block.number <= currentBestBlockNumber || blockQueue.isQueued( - hash - ) - } - - private def isPossibleNewBestBlock(newBlock: BlockHeader, currentBestBlock: BlockHeader): Boolean = - newBlock.parentHash == currentBestBlock.hash && newBlock.number == currentBestBlock.number + 1 - - private def measureBlockMetrics(importResult: BlockImportResult): Unit = - importResult match { - case BlockImportedToTop(blockImportData) => - blockImportData.foreach(blockData => BlockMetrics.measure(blockData.block, blockchainReader.getBlockByHash)) - case ChainReorganised(_, newBranch, _) => - newBranch.foreach(block => BlockMetrics.measure(block, blockchainReader.getBlockByHash)) - case _ => () - } - - private def importToTop( - block: Block, - currentBestBlock: Block, - currentWeight: ChainWeight - )(implicit blockExecutionScheduler: Scheduler, blockchainConfig: BlockchainConfig): Task[BlockImportResult] = { - val validationResult = - Task.evalOnce(blockValidation.validateBlockBeforeExecution(block)).executeOn(validationScheduler) - val importResult = - Task - .evalOnce(importBlockToTop(block, currentBestBlock.header.number, currentWeight)) - .executeOn(blockExecutionScheduler) - - Task.parMap2(validationResult, importResult) { case (validationResult, importResult) => - validationResult.fold( - error => { - log.error("Error while validation block before execution: {}", error.reason) - handleImportTopValidationError(error, block, importResult) - }, - _ => importResult - ) - } - } - - private def importBlockToTop(block: Block, bestBlockNumber: BigInt, currentWeight: ChainWeight)(implicit - blockchainConfig: BlockchainConfig - ): BlockImportResult = { - val executionResult = for { - topBlock <- blockQueue.enqueueBlock(block, bestBlockNumber) - topBlocks = blockQueue.getBranch(topBlock.hash, dequeue = true) - (executed, errors) = blockExecution.executeAndValidateBlocks(topBlocks, currentWeight) - } yield (executed, errors, topBlocks) - - executionResult match { - case Some((importedBlocks, maybeError, topBlocks)) => - val result = maybeError match { - case None => - BlockImportedToTop(importedBlocks) - - case Some(MPTError(reason)) if reason.isInstanceOf[MissingNodeException] => - BlockImportFailedDueToMissingNode(reason.asInstanceOf[MissingNodeException]) - - case Some(error) if importedBlocks.isEmpty => - blockQueue.removeSubtree(block.header.hash) - BlockImportFailed(error.toString) - - case Some(_) => - topBlocks.drop(importedBlocks.length).headOption.foreach { failedBlock => - blockQueue.removeSubtree(failedBlock.header.hash) - } - BlockImportedToTop(importedBlocks) - } - log.debug( - "{}", { - val result = importedBlocks.map { blockData => - val header = blockData.block.header - s"Imported new block (${header.number}: ${Hex.toHexString(header.hash.toArray)}) to the top of chain \n" - } - result.toString - } - ) - - result - - case None => - BlockImportFailed(s"Newly enqueued block with hash: ${block.header.hash} is not part of a known branch") - } - } - - private def handleImportTopValidationError( - error: ValidationBeforeExecError, - block: Block, - blockImportResult: BlockImportResult - ): BlockImportResult = { - blockImportResult match { - case BlockImportedToTop(blockImportData) => - blockImportData.foreach { blockData => - val hash = blockData.block.header.hash - blockQueue.removeSubtree(hash) - blockchain.removeBlock(hash, withState = true) - } - case _ => () - } - handleBlockValidationError(error, block) - } - - private def handleBlockValidationError(error: ValidationBeforeExecError, block: Block): BlockImportResult = - error match { - case ValidationBeforeExecError(HeaderParentNotFoundError) => - log.debug(s"Block(${block.idTag}) has unknown parent") - UnknownParent - - case ValidationBeforeExecError(reason) => - log.debug(s"Block(${block.idTag}) failed pre-import validation") - BlockImportFailed(reason.toString) - } - - private def reorganise( - block: Block, - currentBestBlock: Block, - currentWeight: ChainWeight - )(implicit blockchainConfig: BlockchainConfig): Task[BlockImportResult] = - Task.evalOnce { - blockValidation - .validateBlockBeforeExecution(block) - .fold( - error => handleBlockValidationError(error, block), - _ => - blockQueue.enqueueBlock(block, currentBestBlock.header.number) match { - case Some(Leaf(leafHash, leafWeight)) if leafWeight > currentWeight => - log.debug("Found a better chain, about to reorganise") - reorganiseChainFromQueue(leafHash) - - case _ => - BlockEnqueued - } - ) - } - - /** Once a better branch was found this attempts to reorganise the chain - * - * @param queuedLeaf a block hash that determines a new branch stored in the queue (newest block from the branch) - * - * @return [[BlockExecutionError]] if one of the blocks in the new branch failed to execute, otherwise: - * (oldBranch, newBranch) as lists of blocks - */ - private def reorganiseChainFromQueue( - queuedLeaf: ByteString - )(implicit blockchainConfig: BlockchainConfig): BlockImportResult = { - log.debug("Reorganising chain from leaf {}", ByteStringUtils.hash2string(queuedLeaf)) - val newBranch = blockQueue.getBranch(queuedLeaf, dequeue = true) - val bestNumber = blockchainReader.getBestBlockNumber() - - val reorgResult = for { - parent <- newBranch.headOption - parentHash = parent.header.parentHash - parentWeight <- blockchain.getChainWeightByHash(parentHash) - } yield { - log.debug( - "Removing blocks starting from number {} and parent {}", - bestNumber, - ByteStringUtils.hash2string(parentHash) - ) - val oldBlocksData = removeBlocksUntil(parentHash, bestNumber) - oldBlocksData.foreach(block => blockQueue.enqueueBlock(block.block)) - handleBlockExecResult(newBranch, parentWeight, oldBlocksData) - } - - reorgResult match { - case Some(execResult) => - execResult.fold( - { - case MPTError(reason: MissingNodeException) => BlockImportFailedDueToMissingNode(reason) - case err => BlockImportFailed(s"Error while trying to reorganise chain: $err") - }, - ChainReorganised.tupled - ) - - case None => - BlockImportFailed("Error while trying to reorganise chain with parent of new branch") - } - } - - private def handleBlockExecResult( - newBranch: List[Block], - parentWeight: ChainWeight, - oldBlocksData: List[BlockData] - )(implicit - blockchainConfig: BlockchainConfig - ): Either[BlockExecutionError, (List[Block], List[Block], List[ChainWeight])] = { - val (executedBlocks, maybeError) = blockExecution.executeAndValidateBlocks(newBranch, parentWeight) - maybeError match { - case None => - Right((oldBlocksData.map(_.block), executedBlocks.map(_.block), executedBlocks.map(_.weight))) - - case Some(error) => - revertChainReorganisation(newBranch, oldBlocksData, executedBlocks) - Left(error) - } - } - - /** Reverts chain reorganisation in the event that one of the blocks from new branch fails to execute - * - * @param newBranch new blocks - * @param oldBranch old blocks along with corresponding receipts and totalDifficulties - * @param executedBlocks sub-sequence of new branch that was executed correctly - */ - private def revertChainReorganisation( - newBranch: List[Block], - oldBranch: List[BlockData], - executedBlocks: List[BlockData] - ): Unit = { - if (executedBlocks.nonEmpty) { - removeBlocksUntil(executedBlocks.head.block.header.parentHash, executedBlocks.last.block.header.number) - } - - oldBranch.foreach { case BlockData(block, receipts, weight) => - blockchainWriter.save(block, receipts, weight, saveAsBestBlock = false) - } - - import cats.implicits._ - val checkpointNumber = oldBranch.collect { - case BlockData(block, _, _) if block.hasCheckpoint => block.number - }.maximumOption - - val bestNumber = oldBranch.last.block.header.number - blockchain.saveBestKnownBlocks(bestNumber, checkpointNumber) - executedBlocks.foreach(data => blockQueue.enqueueBlock(data.block, bestNumber)) - - newBranch.diff(executedBlocks.map(_.block)).headOption.foreach { block => - blockQueue.removeSubtree(block.header.hash) - } - } - - /** Removes blocks from the [[Blockchain]] along with receipts and total difficulties. - * - * @param parent remove blocks until this hash (exclusive) - * @param fromNumber start removing from this number (downwards) - * - * @return the list of removed blocks along with receipts and total difficulties - */ - private def removeBlocksUntil(parent: ByteString, fromNumber: BigInt): List[BlockData] = { - @tailrec - def removeBlocksUntil(parent: ByteString, fromNumber: BigInt, acc: List[BlockData]): List[BlockData] = - blockchainReader.getBlockByNumber(blockchainReader.getBestBranch(), fromNumber) match { - case Some(block) if block.header.hash == parent || fromNumber == 0 => - acc - - case Some(block) => - val hash = block.header.hash - - val blockDataOpt = for { - receipts <- blockchainReader.getReceiptsByHash(hash) - weight <- blockchain.getChainWeightByHash(hash) - } yield BlockData(block, receipts, weight) - - blockchain.removeBlock(hash, withState = true) - - removeBlocksUntil(parent, fromNumber - 1, blockDataOpt.map(_ :: acc).getOrElse(acc)) - - case None => - log.error(s"Unexpected missing block number: $fromNumber") - acc - } - - removeBlocksUntil(parent, fromNumber, Nil) - } -} - -sealed trait BlockImportResult - -case class BlockImportedToTop(blockImportData: List[BlockData]) extends BlockImportResult - -case object BlockEnqueued extends BlockImportResult - -case object DuplicateBlock extends BlockImportResult - -case class ChainReorganised( - oldBranch: List[Block], - newBranch: List[Block], - weights: List[ChainWeight] -) extends BlockImportResult - -case class BlockImportFailed(error: String) extends BlockImportResult - -case class BlockImportFailedDueToMissingNode(reason: MissingNodeException) extends BlockImportResult - -case object UnknownParent extends BlockImportResult +//package io.iohk.ethereum.ledger +// +//import akka.util.ByteString +// +//import monix.eval.Task +//import monix.execution.Scheduler +// +//import scala.annotation.tailrec +// +//import org.bouncycastle.util.encoders.Hex +// +//import io.iohk.ethereum.consensus.validators.BlockHeaderError.HeaderParentNotFoundError +//import io.iohk.ethereum.domain._ +//import io.iohk.ethereum.ledger.BlockExecutionError.MPTError +//import io.iohk.ethereum.ledger.BlockExecutionError.ValidationBeforeExecError +//import io.iohk.ethereum.ledger.BlockQueue.Leaf +//import io.iohk.ethereum.mpt.MerklePatriciaTrie.MissingNodeException +//import io.iohk.ethereum.utils.BlockchainConfig +//import io.iohk.ethereum.utils.ByteStringUtils +//import io.iohk.ethereum.utils.Logger +// +//class BlockImport( +// blockchain: BlockchainImpl, +// blockchainReader: BlockchainReader, +// blockchainWriter: BlockchainWriter, +// blockQueue: BlockQueue, +// blockValidation: BlockValidation, +// private[ledger] val blockExecution: BlockExecution, +// validationScheduler: Scheduler // Can't be implicit because of importToTop method and ambiguous of Scheduler +//) extends Logger { +// +// /** Tries to import the block as the new best block in the chain or enqueue it for later processing. +// * +// * @param block block to be imported +// * @param blockExecutionContext threadPool on which the execution should be run +// * @return One of: +// * - [[io.iohk.ethereum.ledger.BlockImportedToTop]] - if the block was added as the new best block +// * - [[io.iohk.ethereum.ledger.BlockEnqueued]] - block is stored in the [[io.iohk.ethereum.ledger.BlockQueue]] +// * - [[io.iohk.ethereum.ledger.ChainReorganised]] - a better new branch was found causing chain reorganisation +// * - [[io.iohk.ethereum.ledger.DuplicateBlock]] - block already exists either in the main chain or in the queue +// * - [[io.iohk.ethereum.ledger.BlockImportFailed]] - block failed to execute (when importing to top or reorganising the chain) +// */ +// def importBlock( +// block: Block +// )(implicit blockExecutionScheduler: Scheduler, blockchainConfig: BlockchainConfig): Task[BlockImportResult] = +// blockchainReader.getBestBlock() match { +// case Some(bestBlock) => +// if (isBlockADuplicate(block.header, bestBlock.header.number)) { +// Task(log.debug(s"Ignoring duplicate block: (${block.idTag})")) +// .map(_ => DuplicateBlock) +// } else { +// val hash = bestBlock.header.hash +// blockchain.getChainWeightByHash(hash) match { +// case Some(weight) => +// val importResult = if (isPossibleNewBestBlock(block.header, bestBlock.header)) { +// importToTop(block, bestBlock, weight) +// } else { +// reorganise(block, bestBlock, weight) +// } +// importResult.foreach(measureBlockMetrics) +// importResult +// case None => +// log.error( +// "Getting total difficulty for current best block with hash: {} failed", +// bestBlock.header.hashAsHexString +// ) +// Task.now( +// BlockImportFailed( +// s"Couldn't get total difficulty for current best block with hash: ${bestBlock.header.hashAsHexString}" +// ) +// ) +// } +// } +// case None => +// log.error("Getting current best block failed") +// Task.now(BlockImportFailed("Couldn't find the current best block")) +// } +// +// private def isBlockADuplicate(block: BlockHeader, currentBestBlockNumber: BigInt): Boolean = { +// val hash = block.hash +// blockchainReader.getBlockByHash(hash).isDefined && block.number <= currentBestBlockNumber || blockQueue.isQueued( +// hash +// ) +// } +// +// private def isPossibleNewBestBlock(newBlock: BlockHeader, currentBestBlock: BlockHeader): Boolean = +// newBlock.parentHash == currentBestBlock.hash && newBlock.number == currentBestBlock.number + 1 +// +// private def measureBlockMetrics(importResult: BlockImportResult)(implicit blockchainConfig: BlockchainConfig): Unit = +// importResult match { +// case BlockImportedToTop(blockImportData) => +// blockImportData.foreach(blockData => BlockMetrics.measure(blockData.block, blockchainReader.getBlockByHash)) +// case ChainReorganised(_, newBranch, _) => +// newBranch.foreach(block => BlockMetrics.measure(block, blockchainReader.getBlockByHash)) +// case _ => () +// } +// +// private def importToTop( +// block: Block, +// currentBestBlock: Block, +// currentWeight: ChainWeight +// )(implicit blockExecutionScheduler: Scheduler, blockchainConfig: BlockchainConfig): Task[BlockImportResult] = { +// val validationResult = +// Task.evalOnce(blockValidation.validateBlockBeforeExecution(block)).executeOn(validationScheduler) +// val importResult = +// Task +// .evalOnce(importBlockToTop(block, currentBestBlock.header.number, currentWeight)) +// .executeOn(blockExecutionScheduler) +// +// Task.parMap2(validationResult, importResult) { case (validationResult, importResult) => +// validationResult.fold( +// error => { +// log.error("Error while validation block before execution: {}", error.reason) +// handleImportTopValidationError(error, block, importResult) +// }, +// _ => importResult +// ) +// } +// } +// +// private def importBlockToTop(block: Block, bestBlockNumber: BigInt, currentWeight: ChainWeight)(implicit +// blockchainConfig: BlockchainConfig +// ): BlockImportResult = { +// val executionResult = for { +// topBlock <- blockQueue.enqueueBlock(block, bestBlockNumber) +// topBlocks = blockQueue.getBranch(topBlock.hash, dequeue = true) +// (executed, errors) = blockExecution.executeAndValidateBlocks(topBlocks, currentWeight) +// } yield (executed, errors, topBlocks) +// +// executionResult match { +// case Some((importedBlocks, maybeError, topBlocks)) => +// val result = maybeError match { +// case None => +// BlockImportedToTop(importedBlocks) +// +// case Some(MPTError(reason)) if reason.isInstanceOf[MissingNodeException] => +// BlockImportFailedDueToMissingNode(reason.asInstanceOf[MissingNodeException]) +// +// case Some(error) if importedBlocks.isEmpty => +// blockQueue.removeSubtree(block.header.hash) +// BlockImportFailed(error.toString) +// +// case Some(_) => +// topBlocks.drop(importedBlocks.length).headOption.foreach { failedBlock => +// blockQueue.removeSubtree(failedBlock.header.hash) +// } +// BlockImportedToTop(importedBlocks) +// } +// log.debug( +// "{}", { +// val result = importedBlocks.map { blockData => +// val header = blockData.block.header +// s"Imported new block (${header.number}: ${Hex.toHexString(header.hash.toArray)}) to the top of chain \n" +// } +// result.toString +// } +// ) +// +// result +// +// case None => +// BlockImportFailed(s"Newly enqueued block with hash: ${block.header.hash} is not part of a known branch") +// } +// } +// +// private def handleImportTopValidationError( +// error: ValidationBeforeExecError, +// block: Block, +// blockImportResult: BlockImportResult +// ): BlockImportResult = { +// blockImportResult match { +// case BlockImportedToTop(blockImportData) => +// blockImportData.foreach { blockData => +// val hash = blockData.block.header.hash +// blockQueue.removeSubtree(hash) +// blockchain.removeBlock(hash, withState = true) +// } +// case _ => () +// } +// handleBlockValidationError(error, block) +// } +// +// private def handleBlockValidationError(error: ValidationBeforeExecError, block: Block): BlockImportResult = +// error match { +// case ValidationBeforeExecError(HeaderParentNotFoundError) => +// log.debug(s"Block(${block.idTag}) has unknown parent") +// UnknownParent +// +// case ValidationBeforeExecError(reason) => +// log.debug(s"Block(${block.idTag}) failed pre-import validation") +// BlockImportFailed(reason.toString) +// } +// +// private def reorganise( +// block: Block, +// currentBestBlock: Block, +// currentWeight: ChainWeight +// )(implicit blockchainConfig: BlockchainConfig): Task[BlockImportResult] = +// Task.evalOnce { +// blockValidation +// .validateBlockBeforeExecution(block) +// .fold( +// error => handleBlockValidationError(error, block), +// _ => +// blockQueue.enqueueBlock(block, currentBestBlock.header.number) match { +// case Some(Leaf(leafHash, leafWeight)) if leafWeight > currentWeight => +// log.debug("Found a better chain, about to reorganise") +// reorganiseChainFromQueue(leafHash) +// +// case _ => +// BlockEnqueued +// } +// ) +// } +// +// /** Once a better branch was found this attempts to reorganise the chain +// * +// * @param queuedLeaf a block hash that determines a new branch stored in the queue (newest block from the branch) +// * +// * @return [[BlockExecutionError]] if one of the blocks in the new branch failed to execute, otherwise: +// * (oldBranch, newBranch) as lists of blocks +// */ +// private def reorganiseChainFromQueue( +// queuedLeaf: ByteString +// )(implicit blockchainConfig: BlockchainConfig): BlockImportResult = { +// log.debug("Reorganising chain from leaf {}", ByteStringUtils.hash2string(queuedLeaf)) +// val newBranch = blockQueue.getBranch(queuedLeaf, dequeue = true) +// val bestNumber = blockchainReader.getBestBlockNumber() +// +// val reorgResult = for { +// parent <- newBranch.headOption +// parentHash = parent.header.parentHash +// parentWeight <- blockchain.getChainWeightByHash(parentHash) +// } yield { +// log.debug( +// "Removing blocks starting from number {} and parent {}", +// bestNumber, +// ByteStringUtils.hash2string(parentHash) +// ) +// val oldBlocksData = removeBlocksUntil(parentHash, bestNumber) +// oldBlocksData.foreach(block => blockQueue.enqueueBlock(block.block)) +// handleBlockExecResult(newBranch, parentWeight, oldBlocksData) +// } +// +// reorgResult match { +// case Some(execResult) => +// execResult.fold( +// { +// case MPTError(reason: MissingNodeException) => BlockImportFailedDueToMissingNode(reason) +// case err => BlockImportFailed(s"Error while trying to reorganise chain: $err") +// }, +// ChainReorganised.tupled +// ) +// +// case None => +// BlockImportFailed("Error while trying to reorganise chain with parent of new branch") +// } +// } +// +// private def handleBlockExecResult( +// newBranch: List[Block], +// parentWeight: ChainWeight, +// oldBlocksData: List[BlockData] +// )(implicit +// blockchainConfig: BlockchainConfig +// ): Either[BlockExecutionError, (List[Block], List[Block], List[ChainWeight])] = { +// val (executedBlocks, maybeError) = blockExecution.executeAndValidateBlocks(newBranch, parentWeight) +// maybeError match { +// case None => +// Right((oldBlocksData.map(_.block), executedBlocks.map(_.block), executedBlocks.map(_.weight))) +// +// case Some(error) => +// revertChainReorganisation(newBranch, oldBlocksData, executedBlocks) +// Left(error) +// } +// } +// +// /** Reverts chain reorganisation in the event that one of the blocks from new branch fails to execute +// * +// * @param newBranch new blocks +// * @param oldBranch old blocks along with corresponding receipts and totalDifficulties +// * @param executedBlocks sub-sequence of new branch that was executed correctly +// */ +// private def revertChainReorganisation( +// newBranch: List[Block], +// oldBranch: List[BlockData], +// executedBlocks: List[BlockData] +// ): Unit = { +// if (executedBlocks.nonEmpty) { +// removeBlocksUntil(executedBlocks.head.block.header.parentHash, executedBlocks.last.block.header.number) +// } +// +// oldBranch.foreach { case BlockData(block, receipts, weight) => +// blockchainWriter.save(block, receipts, weight, saveAsBestBlock = false) +// } +// +// import cats.implicits._ +// val checkpointNumber = oldBranch.collect { +// case BlockData(block, _, _) if block.hasCheckpoint => block.number +// }.maximumOption +// +// val bestNumber = oldBranch.last.block.header.number +// blockchain.saveBestKnownBlocks(bestNumber, checkpointNumber) +// executedBlocks.foreach(data => blockQueue.enqueueBlock(data.block, bestNumber)) +// +// newBranch.diff(executedBlocks.map(_.block)).headOption.foreach { block => +// blockQueue.removeSubtree(block.header.hash) +// } +// } +// +// /** Removes blocks from the [[Blockchain]] along with receipts and total difficulties. +// * +// * @param parent remove blocks until this hash (exclusive) +// * @param fromNumber start removing from this number (downwards) +// * +// * @return the list of removed blocks along with receipts and total difficulties +// */ +// private def removeBlocksUntil(parent: ByteString, fromNumber: BigInt): List[BlockData] = { +// @tailrec +// def removeBlocksUntil(parent: ByteString, fromNumber: BigInt, acc: List[BlockData]): List[BlockData] = +// blockchainReader.getBestBranch().getBlockByNumber(fromNumber) match { +// case Some(block) if block.header.hash == parent || fromNumber == 0 => +// acc +// +// case Some(block) => +// val hash = block.header.hash +// +// val blockDataOpt = for { +// receipts <- blockchainReader.getReceiptsByHash(hash) +// weight <- blockchain.getChainWeightByHash(hash) +// } yield BlockData(block, receipts, weight) +// +// blockchain.removeBlock(hash, withState = true) +// +// removeBlocksUntil(parent, fromNumber - 1, blockDataOpt.map(_ :: acc).getOrElse(acc)) +// +// case None => +// log.error(s"Unexpected missing block number: $fromNumber") +// acc +// } +// +// removeBlocksUntil(parent, fromNumber, Nil) +// } +//} From d0f828fac8f3ae9cd90b8e8223cf6a2f5685617d Mon Sep 17 00:00:00 2001 From: Leonor Boga Date: Fri, 23 Jul 2021 14:49:23 +0200 Subject: [PATCH 02/13] ETCM-1018 Replace references to BlockImport by Consensus/ConsensusImpl --- .../ethereum/ledger/BlockImporterItSpec.scala | 16 +++---- .../sync/util/RegularSyncItSpecUtils.scala | 11 ++--- .../blockchain/sync/SyncController.scala | 18 ++++---- .../blockchain/sync/regular/RegularSync.scala | 16 +++---- .../consensus/mining/TestMiningBuilder.scala | 2 +- .../iohk/ethereum/jsonrpc/TestService.scala | 13 +++--- .../ethereum/nodebuilder/NodeBuilder.scala | 42 +++++++++---------- .../testmode/TestModeComponentsProvider.scala | 12 ++---- .../blockchain/sync/ScenarioSetup.scala | 10 ++--- .../blockchain/sync/SyncControllerSpec.scala | 4 +- .../sync/regular/RegularSyncFixtures.scala | 27 +++++------- .../sync/regular/RegularSyncSpec.scala | 25 +++++------ .../ethereum/ledger/LedgerTestSetup.scala | 24 ++++------- 13 files changed, 87 insertions(+), 133 deletions(-) diff --git a/src/it/scala/io/iohk/ethereum/ledger/BlockImporterItSpec.scala b/src/it/scala/io/iohk/ethereum/ledger/BlockImporterItSpec.scala index 33114c8078..2ce4add6b2 100644 --- a/src/it/scala/io/iohk/ethereum/ledger/BlockImporterItSpec.scala +++ b/src/it/scala/io/iohk/ethereum/ledger/BlockImporterItSpec.scala @@ -3,20 +3,16 @@ package io.iohk.ethereum.ledger import akka.actor.ActorRef import akka.testkit.TestProbe import akka.util.ByteString - import cats.data.NonEmptyList - import monix.execution.Scheduler import monix.execution.schedulers.SchedulerService import scala.concurrent.duration._ - import org.scalamock.scalatest.MockFactory import org.scalatest.BeforeAndAfterAll import org.scalatest.concurrent.Eventually import org.scalatest.flatspec.AnyFlatSpecLike import org.scalatest.matchers.should.Matchers - import io.iohk.ethereum.Fixtures import io.iohk.ethereum.Mocks import io.iohk.ethereum.NormalPatience @@ -26,6 +22,7 @@ import io.iohk.ethereum.blockchain.sync.regular.BlockFetcher import io.iohk.ethereum.blockchain.sync.regular.BlockImporter import io.iohk.ethereum.blockchain.sync.regular.BlockImporter.NewCheckpoint import io.iohk.ethereum.checkpointing.CheckpointingTestHelpers +import io.iohk.ethereum.consensus.Consensus import io.iohk.ethereum.consensus.blocks.CheckpointBlockGenerator import io.iohk.ethereum.consensus.pow.validators.OmmersValidator import io.iohk.ethereum.consensus.pow.validators.StdOmmersValidator @@ -57,8 +54,7 @@ class BlockImporterItSpec override val blockImporter = system.actorOf( BlockImporter.props( fetcherProbe.ref, - mkBlockImport(validators = successValidators), - blockchain, + mkConsensus(validators = successValidators), blockchainReader, storagesInstance.storages.stateStorage, new BranchResolution(blockchain, blockchainReader), @@ -171,8 +167,7 @@ class BlockImporterItSpec override val blockImporter = system.actorOf( BlockImporter.props( fetcherProbe.ref, - mkBlockImport(validators = successValidators), - blockchain, + mkConsensus(validators = successValidators), blockchainReader, storagesInstance.storages.stateStorage, new BranchResolution(blockchain, blockchainReader), @@ -236,7 +231,7 @@ class TestFixture extends TestSetupWithVmAndValidators { override val ommersValidator: OmmersValidator = new StdOmmersValidator(blockHeaderValidator) } - override lazy val blockImport: BlockImport = mkBlockImport( + override lazy val consensus: Consensus = mkConsensus( validators = successValidators, blockExecutionOpt = Some( new BlockExecution( @@ -259,8 +254,7 @@ class TestFixture extends TestSetupWithVmAndValidators { val blockImporter: ActorRef = system.actorOf( BlockImporter.props( fetcherProbe.ref, - blockImport, - blockchain, + consensus, blockchainReader, storagesInstance.storages.stateStorage, new BranchResolution(blockchain, blockchainReader), diff --git a/src/it/scala/io/iohk/ethereum/sync/util/RegularSyncItSpecUtils.scala b/src/it/scala/io/iohk/ethereum/sync/util/RegularSyncItSpecUtils.scala index 422c13df99..2f6f22bfd9 100644 --- a/src/it/scala/io/iohk/ethereum/sync/util/RegularSyncItSpecUtils.scala +++ b/src/it/scala/io/iohk/ethereum/sync/util/RegularSyncItSpecUtils.scala @@ -4,14 +4,11 @@ import akka.actor.ActorRef import akka.actor.typed import akka.actor.typed.scaladsl.adapter._ import akka.util.ByteString - import cats.effect.Resource - import monix.eval.Task import monix.execution.Scheduler import scala.concurrent.duration._ - import io.iohk.ethereum.Mocks.MockValidatorsAlwaysSucceed import io.iohk.ethereum.blockchain.sync.PeersClient import io.iohk.ethereum.blockchain.sync.SyncProtocol @@ -30,7 +27,7 @@ import io.iohk.ethereum.consensus.blocks.CheckpointBlockGenerator import io.iohk.ethereum.consensus.mining.FullMiningConfig import io.iohk.ethereum.consensus.mining.MiningConfig import io.iohk.ethereum.consensus.mining.Protocol.NoAdditionalPoWData -import io.iohk.ethereum.consensus.pow +import io.iohk.ethereum.consensus.{Consensus, ConsensusImpl, pow} import io.iohk.ethereum.consensus.pow.EthashConfig import io.iohk.ethereum.consensus.pow.PoWMining import io.iohk.ethereum.consensus.pow.validators.ValidatorsExecutor @@ -103,8 +100,8 @@ object RegularSyncItSpecUtils { mining.blockPreparator, blockValidation ) - lazy val blockImport: BlockImport = - new BlockImport( + lazy val blockImport: Consensus = + new ConsensusImpl( bl, blockchainReader, blockchainWriter, @@ -146,7 +143,6 @@ object RegularSyncItSpecUtils { BlockImporter.props( fetcher.toClassic, blockImport, - bl, blockchainReader, storagesInstance.storages.stateStorage, new BranchResolution(bl, blockchainReader), @@ -165,7 +161,6 @@ object RegularSyncItSpecUtils { etcPeerManager, peerEventBus, blockImport, - bl, blockchainReader, storagesInstance.storages.stateStorage, new BranchResolution(bl, blockchainReader), diff --git a/src/main/scala/io/iohk/ethereum/blockchain/sync/SyncController.scala b/src/main/scala/io/iohk/ethereum/blockchain/sync/SyncController.scala index 15c9f2abba..8ff922902f 100644 --- a/src/main/scala/io/iohk/ethereum/blockchain/sync/SyncController.scala +++ b/src/main/scala/io/iohk/ethereum/blockchain/sync/SyncController.scala @@ -6,9 +6,9 @@ import akka.actor.ActorRef import akka.actor.PoisonPill import akka.actor.Props import akka.actor.Scheduler - import io.iohk.ethereum.blockchain.sync.fast.FastSync import io.iohk.ethereum.blockchain.sync.regular.RegularSync +import io.iohk.ethereum.consensus.Consensus import io.iohk.ethereum.consensus.validators.Validators import io.iohk.ethereum.db.storage.AppStateStorage import io.iohk.ethereum.db.storage.EvmCodeStorage @@ -18,21 +18,20 @@ import io.iohk.ethereum.db.storage.StateStorage import io.iohk.ethereum.domain.Blockchain import io.iohk.ethereum.domain.BlockchainReader import io.iohk.ethereum.domain.BlockchainWriter -import io.iohk.ethereum.ledger.BlockImport import io.iohk.ethereum.ledger.BranchResolution import io.iohk.ethereum.nodebuilder.BlockchainConfigBuilder import io.iohk.ethereum.utils.Config.SyncConfig class SyncController( - appStateStorage: AppStateStorage, blockchain: Blockchain, blockchainReader: BlockchainReader, blockchainWriter: BlockchainWriter, + appStateStorage: AppStateStorage, evmCodeStorage: EvmCodeStorage, stateStorage: StateStorage, nodeStorage: NodeStorage, fastSyncStateStorage: FastSyncStateStorage, - blockImport: BlockImport, + consensus: Consensus, validators: Validators, peerEventBus: ActorRef, pendingTransactionsManager: ActorRef, @@ -124,8 +123,7 @@ class SyncController( peersClient, etcPeerManager, peerEventBus, - blockImport, - blockchain, + consensus, blockchainReader, stateStorage, new BranchResolution(blockchain, blockchainReader), @@ -148,15 +146,15 @@ class SyncController( object SyncController { // scalastyle:off parameter.number def props( - appStateStorage: AppStateStorage, blockchain: Blockchain, blockchainReader: BlockchainReader, blockchainWriter: BlockchainWriter, + appStateStorage: AppStateStorage, evmCodeStorage: EvmCodeStorage, stateStorage: StateStorage, nodeStorage: NodeStorage, syncStateStorage: FastSyncStateStorage, - blockImport: BlockImport, + consensus: Consensus, validators: Validators, peerEventBus: ActorRef, pendingTransactionsManager: ActorRef, @@ -168,15 +166,15 @@ object SyncController { ): Props = Props( new SyncController( - appStateStorage, blockchain, blockchainReader, blockchainWriter, + appStateStorage, evmCodeStorage, stateStorage, nodeStorage, syncStateStorage, - blockImport, + consensus, validators, peerEventBus, pendingTransactionsManager, diff --git a/src/main/scala/io/iohk/ethereum/blockchain/sync/regular/RegularSync.scala b/src/main/scala/io/iohk/ethereum/blockchain/sync/regular/RegularSync.scala index 66e337a6f6..bd9e86f2e0 100644 --- a/src/main/scala/io/iohk/ethereum/blockchain/sync/regular/RegularSync.scala +++ b/src/main/scala/io/iohk/ethereum/blockchain/sync/regular/RegularSync.scala @@ -10,7 +10,6 @@ import akka.actor.Scheduler import akka.actor.SupervisorStrategy import akka.actor.typed.scaladsl.adapter._ import akka.actor.typed.{ActorRef => TypedActorRef} - import io.iohk.ethereum.blockchain.sync.Blacklist import io.iohk.ethereum.blockchain.sync.SyncProtocol import io.iohk.ethereum.blockchain.sync.SyncProtocol.Status @@ -19,12 +18,11 @@ import io.iohk.ethereum.blockchain.sync.regular.BlockFetcher.InternalLastBlockIm import io.iohk.ethereum.blockchain.sync.regular.RegularSync.NewCheckpoint import io.iohk.ethereum.blockchain.sync.regular.RegularSync.ProgressProtocol import io.iohk.ethereum.blockchain.sync.regular.RegularSync.ProgressState +import io.iohk.ethereum.consensus.Consensus import io.iohk.ethereum.consensus.validators.BlockValidator import io.iohk.ethereum.db.storage.StateStorage import io.iohk.ethereum.domain.Block -import io.iohk.ethereum.domain.Blockchain import io.iohk.ethereum.domain.BlockchainReader -import io.iohk.ethereum.ledger.BlockImport import io.iohk.ethereum.ledger.BranchResolution import io.iohk.ethereum.nodebuilder.BlockchainConfigBuilder import io.iohk.ethereum.utils.ByteStringUtils @@ -34,8 +32,7 @@ class RegularSync( peersClient: ActorRef, etcPeerManager: ActorRef, peerEventBus: ActorRef, - blockImport: BlockImport, - blockchain: Blockchain, + consensus: Consensus, blockchainReader: BlockchainReader, stateStorage: StateStorage, branchResolution: BranchResolution, @@ -66,8 +63,7 @@ class RegularSync( context.actorOf( BlockImporter.props( fetcher.toClassic, - blockImport, - blockchain, + consensus, blockchainReader, stateStorage, branchResolution, @@ -140,8 +136,7 @@ object RegularSync { peersClient: ActorRef, etcPeerManager: ActorRef, peerEventBus: ActorRef, - blockImport: BlockImport, - blockchain: Blockchain, + consensus: Consensus, blockchainReader: BlockchainReader, stateStorage: StateStorage, branchResolution: BranchResolution, @@ -158,8 +153,7 @@ object RegularSync { peersClient, etcPeerManager, peerEventBus, - blockImport, - blockchain, + consensus, blockchainReader, stateStorage, branchResolution, diff --git a/src/main/scala/io/iohk/ethereum/consensus/mining/TestMiningBuilder.scala b/src/main/scala/io/iohk/ethereum/consensus/mining/TestMiningBuilder.scala index 0df8be2504..907ece8cee 100644 --- a/src/main/scala/io/iohk/ethereum/consensus/mining/TestMiningBuilder.scala +++ b/src/main/scala/io/iohk/ethereum/consensus/mining/TestMiningBuilder.scala @@ -20,7 +20,7 @@ trait StdTestMiningBuilder with ActorSystemBuilder with BlockchainBuilder with BlockQueueBuilder - with BlockImportBuilder + with ConsensusBuilder with StorageBuilder with BlockchainConfigBuilder with NodeKeyBuilder diff --git a/src/main/scala/io/iohk/ethereum/jsonrpc/TestService.scala b/src/main/scala/io/iohk/ethereum/jsonrpc/TestService.scala index 60e43d3e8a..d4254bce8e 100644 --- a/src/main/scala/io/iohk/ethereum/jsonrpc/TestService.scala +++ b/src/main/scala/io/iohk/ethereum/jsonrpc/TestService.scala @@ -3,7 +3,6 @@ package io.iohk.ethereum.jsonrpc import akka.actor.ActorRef import akka.util.ByteString import akka.util.Timeout - import monix.eval.Task import monix.execution.Scheduler @@ -11,12 +10,11 @@ import scala.concurrent.duration._ import scala.util.Failure import scala.util.Success import scala.util.Try - import org.bouncycastle.util.encoders.Hex - import io.iohk.ethereum.blockchain.data.GenesisAccount import io.iohk.ethereum.blockchain.data.GenesisData import io.iohk.ethereum.blockchain.data.GenesisDataLoader +import io.iohk.ethereum.blockchain.sync.regular.{BlockEnqueued, BlockImportResult, BlockImportedToTop, ChainReorganised} import io.iohk.ethereum.consensus.blocks._ import io.iohk.ethereum.consensus.mining.MiningConfig import io.iohk.ethereum.crypto @@ -34,7 +32,6 @@ import io.iohk.ethereum.domain.BlockchainReader import io.iohk.ethereum.domain.BlockchainWriter import io.iohk.ethereum.domain.UInt256 import io.iohk.ethereum.jsonrpc.JsonMethodsImplicits._ -import io.iohk.ethereum.ledger._ import io.iohk.ethereum.nodebuilder.TestNode import io.iohk.ethereum.rlp import io.iohk.ethereum.rlp.RLPList @@ -261,8 +258,8 @@ class TestService( getBlockForMining(blockchainReader.getBestBlock().get) .flatMap(blockForMining => testModeComponentsProvider - .blockImport(preimageCache) - .importBlock(blockForMining.block) + .evaluateBranchBlock(preimageCache) + .evaluateBranchBlock(blockForMining.block) ) .map { res => log.info("Block mining result: " + res) @@ -301,8 +298,8 @@ class TestService( Task.now(Left(JsonRpcError(-1, "block validation failed!", None))) case Success(value) => testModeComponentsProvider - .blockImport(preimageCache) - .importBlock(value) + .evaluateBranchBlock(preimageCache) + .evaluateBranchBlock(value) .flatMap(handleResult(value)) } diff --git a/src/main/scala/io/iohk/ethereum/nodebuilder/NodeBuilder.scala b/src/main/scala/io/iohk/ethereum/nodebuilder/NodeBuilder.scala index 685773be9c..7974a8c27a 100644 --- a/src/main/scala/io/iohk/ethereum/nodebuilder/NodeBuilder.scala +++ b/src/main/scala/io/iohk/ethereum/nodebuilder/NodeBuilder.scala @@ -2,13 +2,10 @@ package io.iohk.ethereum.nodebuilder import java.time.Clock import java.util.concurrent.atomic.AtomicReference - import akka.actor.ActorRef import akka.actor.ActorSystem import akka.util.ByteString - import cats.implicits._ - import monix.eval.Task import monix.execution.Scheduler @@ -17,14 +14,13 @@ import scala.concurrent.duration._ import scala.util.Failure import scala.util.Success import scala.util.Try - import org.bouncycastle.crypto.AsymmetricCipherKeyPair - import io.iohk.ethereum.blockchain.data.GenesisDataLoader import io.iohk.ethereum.blockchain.sync.Blacklist import io.iohk.ethereum.blockchain.sync.BlockchainHostActor import io.iohk.ethereum.blockchain.sync.CacheBasedBlacklist import io.iohk.ethereum.blockchain.sync.SyncController +import io.iohk.ethereum.consensus.{Consensus, ConsensusImpl} import io.iohk.ethereum.consensus.blocks.CheckpointBlockGenerator import io.iohk.ethereum.consensus.mining.MiningBuilder import io.iohk.ethereum.consensus.mining.MiningConfigBuilder @@ -190,28 +186,29 @@ trait BlockQueueBuilder { lazy val blockQueue: BlockQueue = BlockQueue(blockchain, blockchainReader, syncConfig) } -trait BlockImportBuilder { +trait ConsensusBuilder { self: BlockchainBuilder with BlockQueueBuilder with MiningBuilder with ActorSystemBuilder with StorageBuilder => - lazy val blockImport: BlockImport = { - val blockValidation = new BlockValidation(mining, blockchainReader, blockQueue) - new BlockImport( + lazy val blockValidation = new BlockValidation(mining, blockchainReader, blockQueue) + lazy val blockExecution = new BlockExecution( + blockchain, + blockchainReader, + blockchainWriter, + storagesInstance.storages.evmCodeStorage, + mining.blockPreparator, + blockValidation + ) + + lazy val consensus: Consensus = + new ConsensusImpl( blockchain, blockchainReader, blockchainWriter, blockQueue, blockValidation, - new BlockExecution( - blockchain, - blockchainReader, - blockchainWriter, - storagesInstance.storages.evmCodeStorage, - mining.blockPreparator, - blockValidation - ), + blockExecution, Scheduler(system.dispatchers.lookup("validation-context")) ) - } } trait ForkResolverBuilder { @@ -409,7 +406,6 @@ trait FilterManagerBuilder { lazy val filterManager: ActorRef = system.actorOf( FilterManager.props( - blockchain, blockchainReader, mining.blockGenerator, keyStore, @@ -735,7 +731,7 @@ trait SyncControllerBuilder { with ServerActorBuilder with BlockchainBuilder with BlockchainConfigBuilder - with BlockImportBuilder + with ConsensusBuilder with NodeStatusBuilder with StorageBuilder with StxLedgerBuilder @@ -750,15 +746,15 @@ trait SyncControllerBuilder { lazy val syncController: ActorRef = system.actorOf( SyncController.props( - storagesInstance.storages.appStateStorage, blockchain, blockchainReader, blockchainWriter, + storagesInstance.storages.appStateStorage, storagesInstance.storages.evmCodeStorage, storagesInstance.storages.stateStorage, storagesInstance.storages.nodeStorage, storagesInstance.storages.fastSyncStateStorage, - blockImport, + consensus, mining.validators, peerEventBus, pendingTransactionsManager, @@ -850,7 +846,7 @@ trait Node with StorageBuilder with BlockchainBuilder with BlockQueueBuilder - with BlockImportBuilder + with ConsensusBuilder with NodeStatusBuilder with ForkResolverBuilder with HandshakerBuilder diff --git a/src/main/scala/io/iohk/ethereum/testmode/TestModeComponentsProvider.scala b/src/main/scala/io/iohk/ethereum/testmode/TestModeComponentsProvider.scala index 46aee13901..bfa773156e 100644 --- a/src/main/scala/io/iohk/ethereum/testmode/TestModeComponentsProvider.scala +++ b/src/main/scala/io/iohk/ethereum/testmode/TestModeComponentsProvider.scala @@ -1,9 +1,8 @@ package io.iohk.ethereum.testmode import akka.util.ByteString - +import io.iohk.ethereum.consensus.{Consensus, ConsensusImpl} import monix.execution.Scheduler - import io.iohk.ethereum.consensus.mining.MiningConfig import io.iohk.ethereum.crypto import io.iohk.ethereum.db.storage.EvmCodeStorage @@ -11,11 +10,9 @@ import io.iohk.ethereum.domain.BlockchainImpl import io.iohk.ethereum.domain.BlockchainReader import io.iohk.ethereum.domain.BlockchainWriter import io.iohk.ethereum.domain.UInt256 -import io.iohk.ethereum.ledger.BlockImport import io.iohk.ethereum.ledger.BlockValidation import io.iohk.ethereum.ledger.VMImpl import io.iohk.ethereum.nodebuilder.TestNode -import io.iohk.ethereum.utils.Config.SyncConfig /** Provides a ledger or consensus instances with modifiable blockchain config (used in test mode). */ class TestModeComponentsProvider( @@ -23,16 +20,15 @@ class TestModeComponentsProvider( blockchainReader: BlockchainReader, blockchainWriter: BlockchainWriter, evmCodeStorage: EvmCodeStorage, - syncConfig: SyncConfig, validationExecutionContext: Scheduler, miningConfig: MiningConfig, vm: VMImpl, node: TestNode ) { - def blockImport( + def evaluateBranchBlock( preimageCache: collection.concurrent.Map[ByteString, UInt256] - ): BlockImport = { + ): Consensus = { val consensuz = consensus() val blockValidation = new BlockValidation(consensuz, blockchainReader, node.blockQueue) val blockExecution = @@ -46,7 +42,7 @@ class TestModeComponentsProvider( (key: UInt256) => preimageCache.put(crypto.kec256(key.bytes), key) ) - new BlockImport( + new ConsensusImpl( blockchain, blockchainReader, blockchainWriter, diff --git a/src/test/scala/io/iohk/ethereum/blockchain/sync/ScenarioSetup.scala b/src/test/scala/io/iohk/ethereum/blockchain/sync/ScenarioSetup.scala index c845e38694..bb32314a21 100644 --- a/src/test/scala/io/iohk/ethereum/blockchain/sync/ScenarioSetup.scala +++ b/src/test/scala/io/iohk/ethereum/blockchain/sync/ScenarioSetup.scala @@ -1,14 +1,13 @@ package io.iohk.ethereum.blockchain.sync import java.util.concurrent.Executors - import monix.execution.Scheduler import scala.concurrent.ExecutionContext import scala.concurrent.ExecutionContextExecutor - import io.iohk.ethereum.Mocks import io.iohk.ethereum.Mocks.MockVM +import io.iohk.ethereum.consensus.{Consensus, ConsensusImpl} import io.iohk.ethereum.consensus.mining.Mining import io.iohk.ethereum.consensus.mining.Protocol import io.iohk.ethereum.consensus.mining.StdTestMiningBuilder @@ -16,7 +15,6 @@ import io.iohk.ethereum.consensus.mining.TestMining import io.iohk.ethereum.consensus.pow.validators.ValidatorsExecutor import io.iohk.ethereum.consensus.validators.Validators import io.iohk.ethereum.ledger.BlockExecution -import io.iohk.ethereum.ledger.BlockImport import io.iohk.ethereum.ledger.BlockValidation import io.iohk.ethereum.ledger.VMImpl import io.iohk.ethereum.nodebuilder._ @@ -71,13 +69,13 @@ trait ScenarioSetup extends StdTestMiningBuilder with StxLedgerBuilder { protected def newTestMining(validators: Validators = mining.validators, vm: VMImpl = mining.vm): Mining = mining.withValidators(validators).withVM(vm) - protected def mkBlockImport( + protected def mkConsensus( validators: Validators = validators, blockExecutionOpt: Option[BlockExecution] = None - ): BlockImport = { + ): Consensus = { val consensuz = mining.withValidators(validators).withVM(new Mocks.MockVM()) val blockValidation = new BlockValidation(consensuz, blockchainReader, blockQueue) - new BlockImport( + new ConsensusImpl( blockchain, blockchainReader, blockchainWriter, diff --git a/src/test/scala/io/iohk/ethereum/blockchain/sync/SyncControllerSpec.scala b/src/test/scala/io/iohk/ethereum/blockchain/sync/SyncControllerSpec.scala index 3cee3d43bd..7694ea7162 100644 --- a/src/test/scala/io/iohk/ethereum/blockchain/sync/SyncControllerSpec.scala +++ b/src/test/scala/io/iohk/ethereum/blockchain/sync/SyncControllerSpec.scala @@ -548,15 +548,15 @@ class SyncControllerSpec lazy val syncController: TestActorRef[Nothing] = TestActorRef( Props( new SyncController( - storagesInstance.storages.appStateStorage, blockchain, blockchainReader, blockchainWriter, + storagesInstance.storages.appStateStorage, storagesInstance.storages.evmCodeStorage, storagesInstance.storages.stateStorage, storagesInstance.storages.nodeStorage, storagesInstance.storages.fastSyncStateStorage, - blockImport, + consensus, validators, peerMessageBus.ref, pendingTransactionsManager.ref, diff --git a/src/test/scala/io/iohk/ethereum/blockchain/sync/regular/RegularSyncFixtures.scala b/src/test/scala/io/iohk/ethereum/blockchain/sync/regular/RegularSyncFixtures.scala index 763082e1c4..2d9b3bf6fa 100644 --- a/src/test/scala/io/iohk/ethereum/blockchain/sync/regular/RegularSyncFixtures.scala +++ b/src/test/scala/io/iohk/ethereum/blockchain/sync/regular/RegularSyncFixtures.scala @@ -1,7 +1,6 @@ package io.iohk.ethereum.blockchain.sync.regular import java.net.InetSocketAddress - import akka.actor.ActorRef import akka.actor.ActorSystem import akka.actor.PoisonPill @@ -11,11 +10,9 @@ import akka.testkit.TestKitBase import akka.testkit.TestProbe import akka.util.ByteString import akka.util.Timeout - import cats.Eq import cats.data.NonEmptyList import cats.implicits._ - import monix.eval.Task import monix.execution.Scheduler import monix.reactive.Observable @@ -26,12 +23,11 @@ import scala.concurrent.duration.DurationInt import scala.concurrent.duration.FiniteDuration import scala.math.BigInt import scala.reflect.ClassTag - import org.scalamock.scalatest.AsyncMockFactory import org.scalatest.matchers.should.Matchers - import io.iohk.ethereum.BlockHelpers import io.iohk.ethereum.blockchain.sync._ +import io.iohk.ethereum.consensus.{Consensus, ConsensusImpl} import io.iohk.ethereum.consensus.blocks.CheckpointBlockGenerator import io.iohk.ethereum.db.storage.StateStorage import io.iohk.ethereum.domain.BlockHeaderImplicits._ @@ -85,8 +81,7 @@ trait RegularSyncFixtures { self: Matchers with AsyncMockFactory => peersClient.ref, etcPeerManager.ref, peerEventBus.ref, - blockImport, - blockchain, + consensus, blockchainReader, stateStorage, branchResolution, @@ -106,7 +101,7 @@ trait RegularSyncFixtures { self: Matchers with AsyncMockFactory => val testBlocks: List[Block] = BlockHelpers.generateChain(20, BlockHelpers.genesis) val testBlocksChunked: List[List[Block]] = testBlocks.grouped(syncConfig.blockHeadersPerRequest).toList - override lazy val blockImport: BlockImport = new TestBlockImport() + override lazy val consensus: Consensus = new TestConsensus() blockchainWriter.save( block = BlockHelpers.genesis, @@ -199,8 +194,8 @@ trait RegularSyncFixtures { self: Matchers with AsyncMockFactory => def setImportResult(block: Block, result: Task[BlockImportResult]): Unit = results(block.header.hash) = result - class TestBlockImport - extends BlockImport( + class TestConsensus + extends ConsensusImpl( stub[BlockchainImpl], stub[BlockchainReader], stub[BlockchainWriter], @@ -209,7 +204,7 @@ trait RegularSyncFixtures { self: Matchers with AsyncMockFactory => stub[BlockExecution], stub[Scheduler] ) { - override def importBlock( + override def evaluateBranchBlock( block: Block )(implicit blockExecutionScheduler: Scheduler, blockchainConfig: BlockchainConfig): Task[BlockImportResult] = { importedBlocksSet.add(block) @@ -315,8 +310,8 @@ trait RegularSyncFixtures { self: Matchers with AsyncMockFactory => implicit def eqInstanceForPeersClientRequest[T <: Message]: Eq[PeersClient.Request[T]] = (x, y) => x.message == y.message && x.peerSelector == y.peerSelector - class FakeImportBlock extends TestBlockImport { - override def importBlock( + class FakeConsensus extends TestConsensus { + override def evaluateBranchBlock( block: Block )(implicit blockExecutionScheduler: Scheduler, blockchainConfig: BlockchainConfig): Task[BlockImportResult] = { val result: BlockImportResult = if (didTryToImportBlock(block)) { @@ -359,7 +354,7 @@ trait RegularSyncFixtures { self: Matchers with AsyncMockFactory => val newBlock: Block = BlockHelpers.generateBlock(testBlocks.last) - override lazy val blockImport: BlockImport = stub[BlockImport] + override lazy val consensus: Consensus = stub[ConsensusImpl] var blockFetcher: ActorRef = _ @@ -369,8 +364,8 @@ trait RegularSyncFixtures { self: Matchers with AsyncMockFactory => override lazy val branchResolution: BranchResolution = stub[BranchResolution] (branchResolution.resolveBranch _).when(*).returns(NewBetterBranch(Nil)) - (blockImport - .importBlock(_: Block)(_: Scheduler, _: BlockchainConfig)) + (consensus + .evaluateBranchBlock(_: Block)(_: Scheduler, _: BlockchainConfig)) .when(*, *, *) .onCall { (block, _, _) => if (block == newBlock) { diff --git a/src/test/scala/io/iohk/ethereum/blockchain/sync/regular/RegularSyncSpec.scala b/src/test/scala/io/iohk/ethereum/blockchain/sync/regular/RegularSyncSpec.scala index a4c4fbe91e..c91fd8b69d 100644 --- a/src/test/scala/io/iohk/ethereum/blockchain/sync/regular/RegularSyncSpec.scala +++ b/src/test/scala/io/iohk/ethereum/blockchain/sync/regular/RegularSyncSpec.scala @@ -8,10 +8,8 @@ import akka.testkit.TestActor.AutoPilot import akka.testkit.TestKit import akka.testkit.TestProbe import akka.util.ByteString - import cats.effect.Resource import cats.syntax.traverse._ - import monix.eval.Task import monix.execution.Scheduler @@ -20,13 +18,11 @@ import scala.concurrent.Future import scala.concurrent.Promise import scala.concurrent.duration._ import scala.math.BigInt - import org.scalamock.scalatest.AsyncMockFactory import org.scalatest.Assertion import org.scalatest.BeforeAndAfterEach import org.scalatest.diagrams.Diagrams import org.scalatest.matchers.should.Matchers - import io.iohk.ethereum.BlockHelpers import io.iohk.ethereum.ObjectGenerators import io.iohk.ethereum.ResourceFixtures @@ -38,6 +34,7 @@ import io.iohk.ethereum.blockchain.sync.SyncProtocol.Status import io.iohk.ethereum.blockchain.sync.SyncProtocol.Status.Progress import io.iohk.ethereum.blockchain.sync.regular.BlockFetcher.Start import io.iohk.ethereum.blockchain.sync.regular.RegularSync.NewCheckpoint +import io.iohk.ethereum.consensus.{Consensus, ConsensusImpl} import io.iohk.ethereum.crypto.kec256 import io.iohk.ethereum.domain.BlockHeaderImplicits._ import io.iohk.ethereum.domain._ @@ -293,7 +290,7 @@ class RegularSyncSpec override lazy val blockchain: BlockchainImpl = stub[BlockchainImpl] override lazy val blockchainReader: BlockchainReader = stub[BlockchainReader] (blockchainReader.getBestBlockNumber _).when().onCall(() => bestBlock.number) - override lazy val blockImport: BlockImport = new FakeImportBlock() + override lazy val consensus: Consensus = new FakeConsensus() override lazy val branchResolution: BranchResolution = new FakeBranchResolution() override lazy val syncConfig = defaultSyncConfig.copy( blockHeadersPerRequest = 5, @@ -328,7 +325,7 @@ class RegularSyncSpec peersClient.setAutoPilot(new BranchResolutionAutoPilot(didResponseWithNewBranch = false, testBlocks)) - Await.result(blockImport.importBlock(BlockHelpers.genesis).runToFuture, remainingOrDefault) + Await.result(consensus.evaluateBranchBlock(BlockHelpers.genesis).runToFuture, remainingOrDefault) regularSync ! SyncProtocol.Start @@ -349,7 +346,7 @@ class RegularSyncSpec override lazy val blockchainReader: BlockchainReader = stub[BlockchainReader] override lazy val blockchain: BlockchainImpl = stub[BlockchainImpl] (blockchainReader.getBestBlockNumber _).when().onCall(() => bestBlock.number) - override lazy val blockImport: BlockImport = new FakeImportBlock() + override lazy val consensus: Consensus = new FakeConsensus() override lazy val branchResolution: BranchResolution = new FakeBranchResolution() override lazy val syncConfig = defaultSyncConfig.copy( syncRetryInterval = 1.second, @@ -379,7 +376,7 @@ class RegularSyncSpec peersClient.setAutoPilot(new ForkingAutoPilot(originalBranch, Some(betterBranch))) - Await.result(blockImport.importBlock(BlockHelpers.genesis).runToFuture, remainingOrDefault) + Await.result(consensus.evaluateBranchBlock(BlockHelpers.genesis).runToFuture, remainingOrDefault) regularSync ! SyncProtocol.Start @@ -470,7 +467,7 @@ class RegularSyncSpec "save fetched node" in sync(new Fixture(testSystem) { override lazy val blockchain: BlockchainImpl = stub[BlockchainImpl] - override lazy val blockImport: BlockImport = stub[BlockImport] + override lazy val consensus: Consensus = stub[ConsensusImpl] override lazy val blockchainReader: BlockchainReader = stub[BlockchainReader] val failingBlock: Block = testBlocksChunked.head.head @@ -478,8 +475,8 @@ class RegularSyncSpec override lazy val branchResolution: BranchResolution = stub[BranchResolution] (blockchainReader.getBestBlockNumber _).when().returns(0) (branchResolution.resolveBranch _).when(*).returns(NewBetterBranch(Nil)).atLeastOnce() - (blockImport - .importBlock(_: Block)(_: Scheduler, _: BlockchainConfig)) + (consensus + .evaluateBranchBlock(_: Block)(_: Scheduler, _: BlockchainConfig)) .when(*, *, *) .returns(Task.now(BlockImportFailedDueToMissingNode(new MissingNodeException(failingBlock.hash)))) @@ -507,7 +504,7 @@ class RegularSyncSpec "catching the top" should { "ignore new blocks if they are too new" in sync(new Fixture(testSystem) { - override lazy val blockImport: BlockImport = stub[BlockImport] + override lazy val consensus: Consensus = stub[ConsensusImpl] val newBlock: Block = testBlocks.last @@ -518,7 +515,7 @@ class RegularSyncSpec Thread.sleep(remainingOrDefault.toMillis) - (blockImport.importBlock(_: Block)(_: Scheduler, _: BlockchainConfig)).verify(*, *, *).never() + (consensus.evaluateBranchBlock(_: Block)(_: Scheduler, _: BlockchainConfig)).verify(*, *, *).never() }) "retry fetch of block that failed to import" in sync(new Fixture(testSystem) { @@ -674,7 +671,7 @@ class RegularSyncSpec val parentBlock = testBlocks.last setImportResult(parentBlock, Task.eval(BlockImportedToTop(Nil))) - blockImport.importBlock(parentBlock)(Scheduler.global, implicitly[BlockchainConfig]) + consensus.evaluateBranchBlock(parentBlock)(Scheduler.global, implicitly[BlockchainConfig]) val checkpointBlock = checkpointBlockGenerator.generate(parentBlock, checkpoint) val newCheckpointMsg = NewCheckpoint(checkpointBlock) diff --git a/src/test/scala/io/iohk/ethereum/ledger/LedgerTestSetup.scala b/src/test/scala/io/iohk/ethereum/ledger/LedgerTestSetup.scala index 2002637fb7..c3bc3b21bc 100644 --- a/src/test/scala/io/iohk/ethereum/ledger/LedgerTestSetup.scala +++ b/src/test/scala/io/iohk/ethereum/ledger/LedgerTestSetup.scala @@ -2,11 +2,8 @@ package io.iohk.ethereum.ledger import akka.util.ByteString import akka.util.ByteString.{empty => bEmpty} - import cats.data.NonEmptyList - import monix.execution.Scheduler - import org.bouncycastle.crypto.AsymmetricCipherKeyPair import org.bouncycastle.crypto.params.ECPublicKeyParameters import org.bouncycastle.util.encoders.Hex @@ -15,11 +12,11 @@ import org.scalamock.handlers.CallHandler1 import org.scalamock.handlers.CallHandler2 import org.scalamock.handlers.CallHandler4 import org.scalamock.scalatest.MockFactory - import io.iohk.ethereum.Fixtures import io.iohk.ethereum.Mocks import io.iohk.ethereum.ObjectGenerators import io.iohk.ethereum.blockchain.sync.EphemBlockchainTestSetup +import io.iohk.ethereum.consensus.{Consensus, ConsensusImpl} import io.iohk.ethereum.consensus.blocks.CheckpointBlockGenerator import io.iohk.ethereum.consensus.mining.GetBlockHeaderByHash import io.iohk.ethereum.consensus.mining.TestMining @@ -35,9 +32,6 @@ import io.iohk.ethereum.domain._ import io.iohk.ethereum.domain.branch.Branch import io.iohk.ethereum.domain.branch.EmptyBranch import io.iohk.ethereum.ledger.BlockExecutionError.ValidationAfterExecError -import io.iohk.ethereum.ledger.PC -import io.iohk.ethereum.ledger.PR -import io.iohk.ethereum.ledger.VMImpl import io.iohk.ethereum.mpt.MerklePatriciaTrie import io.iohk.ethereum.security.SecureRandomBuilder import io.iohk.ethereum.utils.BlockchainConfig @@ -289,7 +283,7 @@ trait TestSetupWithVmAndValidators extends EphemBlockchainTestSetup { implicit val schedulerContext: Scheduler = Scheduler.fixedPool("ledger-test-pool", 4) - override lazy val blockImport: BlockImport = mkBlockImport() + override lazy val consensus: Consensus = mkConsensus() def randomHash(): ByteString = ObjectGenerators.byteStringOfLengthNGen(32).sample.get @@ -375,12 +369,12 @@ trait TestSetupWithVmAndValidators extends EphemBlockchainTestSetup { ) } - lazy val failBlockImport: BlockImport = mkBlockImport(validators = FailHeaderValidation) + lazy val failConsensus: Consensus = mkConsensus(validators = FailHeaderValidation) - lazy val blockImportNotFailingAfterExecValidation: BlockImport = { + lazy val blockImportNotFailingAfterExecValidation: Consensus = { val consensuz = mining.withValidators(NotFailAfterExecValidation).withVM(new Mocks.MockVM()) val blockValidation = new BlockValidation(consensuz, blockchainReader, blockQueue) - new BlockImport( + new ConsensusImpl( blockchain, blockchainReader, blockchainWriter, @@ -473,8 +467,8 @@ trait MockBlockchain extends MockFactory { self: TestSetupWithVmAndValidators => trait EphemBlockchain extends TestSetupWithVmAndValidators with MockFactory { override lazy val blockQueue: BlockQueue = BlockQueue(blockchain, blockchainReader, SyncConfig(Config.config)) - lazy val blockImportWithMockedBlockExecution: BlockImport = - mkBlockImport(blockExecutionOpt = Some(mock[BlockExecution])) + lazy val blockImportWithMockedBlockExecution: Consensus = + mkConsensus(blockExecutionOpt = Some(mock[BlockExecution])) } trait CheckpointHelpers { @@ -490,6 +484,6 @@ trait OmmersTestSetup extends EphemBlockchain { new StdOmmersValidator(blockHeaderValidator) } - override lazy val blockImportWithMockedBlockExecution: BlockImport = - mkBlockImport(validators = OmmerValidation, blockExecutionOpt = Some(mock[BlockExecution])) + override lazy val blockImportWithMockedBlockExecution: Consensus = + mkConsensus(validators = OmmerValidation, blockExecutionOpt = Some(mock[BlockExecution])) } From ef65e30221e042c17e13afd419d2a45a36627c3c Mon Sep 17 00:00:00 2001 From: Leonor Boga Date: Fri, 23 Jul 2021 14:51:46 +0200 Subject: [PATCH 03/13] ETCM-1018 Remove unused references to blockchain --- .../io/iohk/ethereum/blockchain/sync/fast/FastSync.scala | 1 - .../ethereum/blockchain/sync/fast/SyncStateScheduler.scala | 2 -- .../ethereum/blockchain/sync/regular/ImportMessages.scala | 1 - .../io/iohk/ethereum/consensus/pow/miners/MockedMiner.scala | 5 ----- src/main/scala/io/iohk/ethereum/jsonrpc/FilterManager.scala | 3 --- src/main/scala/io/iohk/ethereum/nodebuilder/TestNode.scala | 1 - .../io/iohk/ethereum/blockchain/sync/StateSyncSpec.scala | 1 - .../ethereum/blockchain/sync/SyncStateSchedulerSpec.scala | 1 - .../iohk/ethereum/consensus/pow/miners/MockedMinerSpec.scala | 1 - .../scala/io/iohk/ethereum/jsonrpc/FilterManagerSpec.scala | 1 - 10 files changed, 17 deletions(-) diff --git a/src/main/scala/io/iohk/ethereum/blockchain/sync/fast/FastSync.scala b/src/main/scala/io/iohk/ethereum/blockchain/sync/fast/FastSync.scala index 86e355794d..34e0b38797 100644 --- a/src/main/scala/io/iohk/ethereum/blockchain/sync/fast/FastSync.scala +++ b/src/main/scala/io/iohk/ethereum/blockchain/sync/fast/FastSync.scala @@ -178,7 +178,6 @@ class FastSync( SyncStateSchedulerActor .props( SyncStateScheduler( - blockchain, blockchainReader, evmCodeStorage, stateStorage, diff --git a/src/main/scala/io/iohk/ethereum/blockchain/sync/fast/SyncStateScheduler.scala b/src/main/scala/io/iohk/ethereum/blockchain/sync/fast/SyncStateScheduler.scala index f5b2bd4143..25b95dcc0f 100644 --- a/src/main/scala/io/iohk/ethereum/blockchain/sync/fast/SyncStateScheduler.scala +++ b/src/main/scala/io/iohk/ethereum/blockchain/sync/fast/SyncStateScheduler.scala @@ -23,7 +23,6 @@ import io.iohk.ethereum.db.storage.EvmCodeStorage import io.iohk.ethereum.db.storage.NodeStorage import io.iohk.ethereum.db.storage.StateStorage import io.iohk.ethereum.domain.Account -import io.iohk.ethereum.domain.Blockchain import io.iohk.ethereum.domain.BlockchainReader import io.iohk.ethereum.mpt.BranchNode import io.iohk.ethereum.mpt.ExtensionNode @@ -292,7 +291,6 @@ object SyncStateScheduler { BloomFilter.create[ByteString](ByteStringFunnel, expectedFilterSize) def apply( - blockchain: Blockchain, blockchainReader: BlockchainReader, evmCodeStorage: EvmCodeStorage, stateStorage: StateStorage, diff --git a/src/main/scala/io/iohk/ethereum/blockchain/sync/regular/ImportMessages.scala b/src/main/scala/io/iohk/ethereum/blockchain/sync/regular/ImportMessages.scala index 4d2b2354b4..d2eae37d23 100644 --- a/src/main/scala/io/iohk/ethereum/blockchain/sync/regular/ImportMessages.scala +++ b/src/main/scala/io/iohk/ethereum/blockchain/sync/regular/ImportMessages.scala @@ -4,7 +4,6 @@ import akka.event.Logging._ import akka.util.ByteString import io.iohk.ethereum.domain.Block -import io.iohk.ethereum.ledger._ import io.iohk.ethereum.mpt.MerklePatriciaTrie.MissingNodeException import io.iohk.ethereum.network.PeerId import io.iohk.ethereum.utils.ByteStringUtils._ diff --git a/src/main/scala/io/iohk/ethereum/consensus/pow/miners/MockedMiner.scala b/src/main/scala/io/iohk/ethereum/consensus/pow/miners/MockedMiner.scala index 6dcf74d745..f16ae096a1 100644 --- a/src/main/scala/io/iohk/ethereum/consensus/pow/miners/MockedMiner.scala +++ b/src/main/scala/io/iohk/ethereum/consensus/pow/miners/MockedMiner.scala @@ -25,7 +25,6 @@ import io.iohk.ethereum.consensus.pow.miners.MockedMiner.MockedMinerResponses.Mi import io.iohk.ethereum.consensus.pow.miners.MockedMiner.MockedMinerResponses.MiningError import io.iohk.ethereum.consensus.pow.miners.MockedMiner.MockedMinerResponses.MiningOrdered import io.iohk.ethereum.domain.Block -import io.iohk.ethereum.domain.Blockchain import io.iohk.ethereum.domain.BlockchainReader import io.iohk.ethereum.ledger.InMemoryWorldStateProxy import io.iohk.ethereum.nodebuilder.BlockchainConfigBuilder @@ -34,7 +33,6 @@ import io.iohk.ethereum.utils.ByteStringUtils import io.iohk.ethereum.utils.ByteStringUtils.ByteStringOps class MockedMiner( - blockchain: Blockchain, blockchainReader: BlockchainReader, blockCreator: PoWBlockCreator, syncEventListener: ActorRef, @@ -122,7 +120,6 @@ object MockedMiner { case object MineBlock private[pow] def props( - blockchain: Blockchain, blockchainReader: BlockchainReader, blockCreator: PoWBlockCreator, syncEventListener: ActorRef, @@ -130,7 +127,6 @@ object MockedMiner { ): Props = Props( new MockedMiner( - blockchain, blockchainReader, blockCreator, syncEventListener, @@ -148,7 +144,6 @@ object MockedMiner { ommersPool = node.ommersPool ) val minerProps = props( - blockchain = node.blockchain, blockchainReader = node.blockchainReader, blockCreator = blockCreator, syncEventListener = node.syncController, diff --git a/src/main/scala/io/iohk/ethereum/jsonrpc/FilterManager.scala b/src/main/scala/io/iohk/ethereum/jsonrpc/FilterManager.scala index 5920cfecd4..e354bc9463 100644 --- a/src/main/scala/io/iohk/ethereum/jsonrpc/FilterManager.scala +++ b/src/main/scala/io/iohk/ethereum/jsonrpc/FilterManager.scala @@ -25,7 +25,6 @@ import io.iohk.ethereum.utils.FilterConfig import io.iohk.ethereum.utils.TxPoolConfig class FilterManager( - blockchain: Blockchain, blockchainReader: BlockchainReader, blockGenerator: BlockGenerator, keyStore: KeyStore, @@ -278,7 +277,6 @@ class FilterManager( object FilterManager { def props( - blockchain: Blockchain, blockchainReader: BlockchainReader, blockGenerator: BlockGenerator, keyStore: KeyStore, @@ -288,7 +286,6 @@ object FilterManager { ): Props = Props( new FilterManager( - blockchain, blockchainReader, blockGenerator, keyStore, diff --git a/src/main/scala/io/iohk/ethereum/nodebuilder/TestNode.scala b/src/main/scala/io/iohk/ethereum/nodebuilder/TestNode.scala index 7bbece8eca..e2d7ffc314 100644 --- a/src/main/scala/io/iohk/ethereum/nodebuilder/TestNode.scala +++ b/src/main/scala/io/iohk/ethereum/nodebuilder/TestNode.scala @@ -21,7 +21,6 @@ class TestNode extends BaseNode { blockchainReader, blockchainWriter, storagesInstance.storages.evmCodeStorage, - syncConfig, scheduler, miningConfig, vm, diff --git a/src/test/scala/io/iohk/ethereum/blockchain/sync/StateSyncSpec.scala b/src/test/scala/io/iohk/ethereum/blockchain/sync/StateSyncSpec.scala index 3a2bda5687..e6e935e7aa 100644 --- a/src/test/scala/io/iohk/ethereum/blockchain/sync/StateSyncSpec.scala +++ b/src/test/scala/io/iohk/ethereum/blockchain/sync/StateSyncSpec.scala @@ -254,7 +254,6 @@ class StateSyncSpec system.actorOf( SyncStateSchedulerActor.props( SyncStateScheduler( - blockchain, blockchainReader, getNewStorages.storages.evmCodeStorage, getNewStorages.storages.stateStorage, diff --git a/src/test/scala/io/iohk/ethereum/blockchain/sync/SyncStateSchedulerSpec.scala b/src/test/scala/io/iohk/ethereum/blockchain/sync/SyncStateSchedulerSpec.scala index b11ea2c570..c260831f7d 100644 --- a/src/test/scala/io/iohk/ethereum/blockchain/sync/SyncStateSchedulerSpec.scala +++ b/src/test/scala/io/iohk/ethereum/blockchain/sync/SyncStateSchedulerSpec.scala @@ -316,7 +316,6 @@ class SyncStateSchedulerSpec val freshBlockchainWriter = BlockchainWriter(freshStorage.storages, freshBlockchainMetadata) ( SyncStateScheduler( - freshBlockchain, freshBlockchainReader, freshStorage.storages.evmCodeStorage, freshStorage.storages.stateStorage, diff --git a/src/test/scala/io/iohk/ethereum/consensus/pow/miners/MockedMinerSpec.scala b/src/test/scala/io/iohk/ethereum/consensus/pow/miners/MockedMinerSpec.scala index ec7d1437a9..30cc6d71a1 100644 --- a/src/test/scala/io/iohk/ethereum/consensus/pow/miners/MockedMinerSpec.scala +++ b/src/test/scala/io/iohk/ethereum/consensus/pow/miners/MockedMinerSpec.scala @@ -222,7 +222,6 @@ class MockedMinerSpec val miner: TestActorRef[Nothing] = TestActorRef( MockedMiner.props( - blockchain, blockchainReader, blockCreator, sync.ref, diff --git a/src/test/scala/io/iohk/ethereum/jsonrpc/FilterManagerSpec.scala b/src/test/scala/io/iohk/ethereum/jsonrpc/FilterManagerSpec.scala index 7544c18d06..db0fb31d0e 100644 --- a/src/test/scala/io/iohk/ethereum/jsonrpc/FilterManagerSpec.scala +++ b/src/test/scala/io/iohk/ethereum/jsonrpc/FilterManagerSpec.scala @@ -523,7 +523,6 @@ class FilterManagerSpec val filterManager: TestActorRef[FilterManager] = TestActorRef[FilterManager]( Props( new FilterManager( - blockchain, blockchainReader, blockGenerator, keyStore, From 7c2948d527b5a1af17fefc702fa1fb51c615869b Mon Sep 17 00:00:00 2001 From: Leonor Boga Date: Fri, 23 Jul 2021 15:32:36 +0200 Subject: [PATCH 04/13] ETCM-1058 Fix unit tests --- .../blockchain/sync/ScenarioSetup.scala | 28 ++++--- .../consensus/blocks/BlockGeneratorSpec.scala | 6 +- .../ethereum/ledger/BlockExecutionSpec.scala | 42 +++++------ .../ethereum/ledger/BlockImportSpec.scala | 75 +++++++++++++------ .../ethereum/ledger/LedgerTestSetup.scala | 8 +- 5 files changed, 95 insertions(+), 64 deletions(-) diff --git a/src/test/scala/io/iohk/ethereum/blockchain/sync/ScenarioSetup.scala b/src/test/scala/io/iohk/ethereum/blockchain/sync/ScenarioSetup.scala index bb32314a21..5412cfe0cb 100644 --- a/src/test/scala/io/iohk/ethereum/blockchain/sync/ScenarioSetup.scala +++ b/src/test/scala/io/iohk/ethereum/blockchain/sync/ScenarioSetup.scala @@ -69,28 +69,32 @@ trait ScenarioSetup extends StdTestMiningBuilder with StxLedgerBuilder { protected def newTestMining(validators: Validators = mining.validators, vm: VMImpl = mining.vm): Mining = mining.withValidators(validators).withVM(vm) + protected def mkBlockExecution(validators: Validators = validators) = { + val consensuz = mining.withValidators(validators).withVM(new Mocks.MockVM()) + val blockValidation = new BlockValidation(consensuz, blockchainReader, blockQueue) + new BlockExecution( + blockchain, + blockchainReader, + blockchainWriter, + storagesInstance.storages.evmCodeStorage, + consensuz.blockPreparator, + blockValidation + ) + } + protected def mkConsensus( validators: Validators = validators, blockExecutionOpt: Option[BlockExecution] = None ): Consensus = { - val consensuz = mining.withValidators(validators).withVM(new Mocks.MockVM()) - val blockValidation = new BlockValidation(consensuz, blockchainReader, blockQueue) + val testMining = mining.withValidators(validators).withVM(new Mocks.MockVM()) + val blockValidation = new BlockValidation(testMining, blockchainReader, blockQueue) new ConsensusImpl( blockchain, blockchainReader, blockchainWriter, blockQueue, blockValidation, - blockExecutionOpt.getOrElse( - new BlockExecution( - blockchain, - blockchainReader, - blockchainWriter, - storagesInstance.storages.evmCodeStorage, - consensuz.blockPreparator, - blockValidation - ) - ), + blockExecutionOpt.getOrElse(mkBlockExecution(validators)), Scheduler(system.dispatchers.lookup("validation-context")) ) } diff --git a/src/test/scala/io/iohk/ethereum/consensus/blocks/BlockGeneratorSpec.scala b/src/test/scala/io/iohk/ethereum/consensus/blocks/BlockGeneratorSpec.scala index 516d947456..3bba5f2032 100644 --- a/src/test/scala/io/iohk/ethereum/consensus/blocks/BlockGeneratorSpec.scala +++ b/src/test/scala/io/iohk/ethereum/consensus/blocks/BlockGeneratorSpec.scala @@ -109,7 +109,7 @@ class BlockGeneratorSpec extends AnyFlatSpec with Matchers with ScalaCheckProper ) // Import Block, to create some existing state - blockImport.importBlock(fullBlock).runSyncUnsafe() + consensus.evaluateBranchBlock(fullBlock).runSyncUnsafe() // Create new pending block, with updated stateRootHash val pendBlockAndState = blockGenerator.generateBlock( @@ -690,9 +690,9 @@ class BlockGeneratorSpec extends AnyFlatSpec with Matchers with ScalaCheckProper lazy val blockGenerator: TestBlockGenerator = mining.blockGenerator.withBlockTimestampProvider(blockTimestampProvider) - lazy val blockValidation = + override lazy val blockValidation = new BlockValidation(mining, blockchainReader, BlockQueue(blockchain, blockchainReader, syncConfig)) - lazy val blockExecution = + override lazy val blockExecution = new BlockExecution( blockchain, blockchainReader, diff --git a/src/test/scala/io/iohk/ethereum/ledger/BlockExecutionSpec.scala b/src/test/scala/io/iohk/ethereum/ledger/BlockExecutionSpec.scala index ddb4383a5f..495084f814 100644 --- a/src/test/scala/io/iohk/ethereum/ledger/BlockExecutionSpec.scala +++ b/src/test/scala/io/iohk/ethereum/ledger/BlockExecutionSpec.scala @@ -61,9 +61,9 @@ class BlockExecutionSpec extends AnyWordSpec with Matchers with ScalaCheckProper val mockValidators = new MockValidatorsFailOnSpecificBlockNumber(block1.header.number) val newMining: TestMining = mining.withVM(vm).withValidators(mockValidators) - val blockValidation = + override lazy val blockValidation = new BlockValidation(newMining, blockchainReader, BlockQueue(blockchain, blockchainReader, syncConfig)) - val blockExecution = + override lazy val blockExecution = new BlockExecution( blockchain, blockchainReader, @@ -103,9 +103,9 @@ class BlockExecutionSpec extends AnyWordSpec with Matchers with ScalaCheckProper ) val mockValidators = new MockValidatorsFailOnSpecificBlockNumber(block2.header.number) val newMining: TestMining = mining.withVM(mockVm).withValidators(mockValidators) - val blockValidation = + override lazy val blockValidation = new BlockValidation(newMining, blockchainReader, BlockQueue(blockchain, blockchainReader, syncConfig)) - val blockExecution = + override lazy val blockExecution = new BlockExecution( blockchain, blockchainReader, @@ -138,9 +138,9 @@ class BlockExecutionSpec extends AnyWordSpec with Matchers with ScalaCheckProper ) val mockValidators = new MockValidatorsFailOnSpecificBlockNumber(chain.last.number) val newMining: TestMining = mining.withVM(mockVm).withValidators(mockValidators) - val blockValidation = + override lazy val blockValidation = new BlockValidation(newMining, blockchainReader, BlockQueue(blockchain, blockchainReader, syncConfig)) - val blockExecution = + override lazy val blockExecution = new BlockExecution( blockchain, blockchainReader, @@ -169,9 +169,9 @@ class BlockExecutionSpec extends AnyWordSpec with Matchers with ScalaCheckProper val mockValidators = MockValidatorsAlwaysSucceed val newMining: TestMining = mining.withVM(vm).withValidators(mockValidators) - val blockValidation = + override lazy val blockValidation = new BlockValidation(newMining, blockchainReader, BlockQueue(blockchain, blockchainReader, syncConfig)) - val blockExecution = + override lazy val blockExecution = new BlockExecution( blockchain, blockchainReader, @@ -212,7 +212,7 @@ class BlockExecutionSpec extends AnyWordSpec with Matchers with ScalaCheckProper txsExecResult.isRight shouldBe true - val BlockResult(resultingWorldState, resultingGasUsed, resultingReceipts) = txsExecResult.toOption.get + val BlockResult(_, resultingGasUsed, resultingReceipts) = txsExecResult.toOption.get resultingGasUsed shouldBe 0 resultingReceipts shouldBe Nil } @@ -236,9 +236,9 @@ class BlockExecutionSpec extends AnyWordSpec with Matchers with ScalaCheckProper val newMining: TestMining = mining.withVM(mockVm) - val blockValidation = + override lazy val blockValidation = new BlockValidation(newMining, blockchainReader, BlockQueue(blockchain, blockchainReader, syncConfig)) - val blockExecution = + override lazy val blockExecution = new BlockExecution( blockchain, blockchainReader, @@ -434,7 +434,7 @@ class BlockExecutionSpec extends AnyWordSpec with Matchers with ScalaCheckProper ) val block = Block(blockHeader, blockBodyWithOmmers) - val blockExecResult = blockImport.blockExecution.executeAndValidateBlock(block) + val blockExecResult = blockExecution.executeAndValidateBlock(block) assert(blockExecResult.isRight) } } @@ -481,7 +481,7 @@ class BlockExecutionSpec extends AnyWordSpec with Matchers with ScalaCheckProper val block = Block(blockHeader, validBlockBodyWithNoTxs) assert(seqFailingValidators.forall { _ => - val blockExecResult = blockImport.blockExecution.executeAndValidateBlock(block) + val blockExecResult = blockExecution.executeAndValidateBlock(block) blockExecResult.left.forall { case _: BlockExecutionError.ValidationBeforeExecError => true @@ -530,12 +530,12 @@ class BlockExecutionSpec extends AnyWordSpec with Matchers with ScalaCheckProper ) forAll(table) { (stateRootHash, cumulativeGasUsedBlock, validators) => - val blockImport = mkBlockImport(validators = validators) + val blockExecution = mkBlockExecution(validators = validators) val blockHeader: BlockHeader = validBlockHeader.copy(gasUsed = cumulativeGasUsedBlock, stateRoot = stateRootHash) val block = Block(blockHeader, validBlockBodyWithNoTxs) - val blockExecResult = blockImport.blockExecution.executeAndValidateBlock(block) + val blockExecResult = blockExecution.executeAndValidateBlock(block) assert(blockExecResult match { case Left(_: BlockExecutionError.ValidationAfterExecError) => true @@ -585,7 +585,7 @@ class BlockExecutionSpec extends AnyWordSpec with Matchers with ScalaCheckProper val validBlockBodyWithTxs: BlockBody = validBlockBodyWithNoTxs.copy(transactionList = Seq(stx1.tx, stx2.tx)) val block = Block(validBlockHeader, validBlockBodyWithTxs) - val txsExecResult = blockImport.blockExecution.executeBlockTransactions(block, initialWorld) + val txsExecResult = blockExecution.executeBlockTransactions(block, initialWorld) assert(txsExecResult.isRight) val BlockResult(resultingWorldState, resultingGasUsed, resultingReceipts) = txsExecResult.toOption.get @@ -641,7 +641,7 @@ class BlockExecutionSpec extends AnyWordSpec with Matchers with ScalaCheckProper val blockWithCorrectStateAndGasUsed = block.copy( header = block.header.copy(stateRoot = blockExpectedStateRoot, gasUsed = gasUsedReceipt2) ) - assert(blockImport.blockExecution.executeAndValidateBlock(blockWithCorrectStateAndGasUsed).isRight) + assert(blockExecution.executeAndValidateBlock(blockWithCorrectStateAndGasUsed).isRight) } } @@ -662,7 +662,7 @@ class BlockExecutionSpec extends AnyWordSpec with Matchers with ScalaCheckProper } // We don't care about block txs in this test - blockImport.blockExecution.executeBlockTransactions( + blockExecution.executeBlockTransactions( proDaoBlock.copy(body = proDaoBlock.body.copy(transactionList = Seq.empty)), initialWorld ) @@ -675,7 +675,7 @@ class BlockExecutionSpec extends AnyWordSpec with Matchers with ScalaCheckProper } // We don't care about block txs in this test - blockImport.blockExecution.executeBlockTransactions( + blockExecution.executeBlockTransactions( proDaoBlock.copy(body = proDaoBlock.body.copy(transactionList = Seq.empty)), initialWorld ) @@ -684,9 +684,9 @@ class BlockExecutionSpec extends AnyWordSpec with Matchers with ScalaCheckProper trait BlockExecutionTestSetup extends BlockchainSetup { - val blockValidation = + override lazy val blockValidation = new BlockValidation(mining, blockchainReader, BlockQueue(blockchain, blockchainReader, syncConfig)) - val blockExecution = + override lazy val blockExecution = new BlockExecution( blockchain, blockchainReader, diff --git a/src/test/scala/io/iohk/ethereum/ledger/BlockImportSpec.scala b/src/test/scala/io/iohk/ethereum/ledger/BlockImportSpec.scala index 671d1b7654..118ab7fc55 100644 --- a/src/test/scala/io/iohk/ethereum/ledger/BlockImportSpec.scala +++ b/src/test/scala/io/iohk/ethereum/ledger/BlockImportSpec.scala @@ -4,13 +4,19 @@ import akka.util.ByteString import scala.concurrent.duration._ import scala.language.postfixOps - import org.scalatest.concurrent.ScalaFutures import org.scalatest.flatspec.AnyFlatSpec import org.scalatest.matchers.should.Matchers - import io.iohk.ethereum.Mocks import io.iohk.ethereum.Mocks.MockValidatorsAlwaysSucceed +import io.iohk.ethereum.blockchain.sync.regular.{ + BlockEnqueued, + BlockImportFailed, + BlockImportedToTop, + ChainReorganised, + DuplicateBlock, + UnknownParent +} import io.iohk.ethereum.consensus.mining._ import io.iohk.ethereum.consensus.validators.BlockHeaderError.HeaderDifficultyError import io.iohk.ethereum.consensus.validators.BlockHeaderError.HeaderParentNotFoundError @@ -34,12 +40,12 @@ class BlockImportSpec extends AnyFlatSpec with Matchers with ScalaFutures { setBlockExists(block1, inChain = true, inQueue = false) setBestBlock(bestBlock) - whenReady(blockImport.importBlock(block1).runToFuture)(_ shouldEqual DuplicateBlock) + whenReady(consensus.evaluateBranchBlock(block1).runToFuture)(_ shouldEqual DuplicateBlock) setBlockExists(block2, inChain = false, inQueue = true) setBestBlock(bestBlock) - whenReady(blockImport.importBlock(block2).runToFuture)(_ shouldEqual DuplicateBlock) + whenReady(consensus.evaluateBranchBlock(block2).runToFuture)(_ shouldEqual DuplicateBlock) } it should "import a block to top of the main chain" in new ImportBlockTestSetup { @@ -66,7 +72,7 @@ class BlockImportSpec extends AnyFlatSpec with Matchers with ScalaFutures { .returning(storagesInstance.storages.stateStorage.getBackingStorage(6)) expectBlockSaved(block, Seq.empty[Receipt], newWeight, saveAsBestBlock = true) - whenReady(blockImportNotFailingAfterExecValidation.importBlock(block).runToFuture) { + whenReady(blockImportNotFailingAfterExecValidation.evaluateBranchBlock(block).runToFuture) { _ shouldEqual BlockImportedToTop(List(blockData)) } } @@ -99,7 +105,7 @@ class BlockImportSpec extends AnyFlatSpec with Matchers with ScalaFutures { (blockQueue.removeSubtree _).expects(*) - whenReady(blockImport.importBlock(block).runToFuture)(_ shouldBe a[BlockImportFailed]) + whenReady(consensus.evaluateBranchBlock(block).runToFuture)(_ shouldBe a[BlockImportFailed]) } // scalastyle:off magic.number @@ -128,13 +134,17 @@ class BlockImportSpec extends AnyFlatSpec with Matchers with ScalaFutures { val blockData2 = BlockData(newBlock2, Seq.empty[Receipt], newWeight2) val blockData3 = BlockData(newBlock3, Seq.empty[Receipt], newWeight3) - (blockImportWithMockedBlockExecution.blockExecution + val mockExecution = mock[BlockExecution] + (mockExecution .executeAndValidateBlocks(_: List[Block], _: ChainWeight)(_: BlockchainConfig)) .expects(newBranch, *, *) .returning((List(blockData2, blockData3), None)) - whenReady(blockImportWithMockedBlockExecution.importBlock(newBlock3).runToFuture)(_ shouldEqual BlockEnqueued) - whenReady(blockImportWithMockedBlockExecution.importBlock(newBlock2).runToFuture) { result => + val withMockedBlockExecution = blockImportWithMockedBlockExecution(mockExecution) + whenReady(withMockedBlockExecution.evaluateBranchBlock(newBlock3).runToFuture)( + _ shouldEqual BlockEnqueued + ) + whenReady(withMockedBlockExecution.evaluateBranchBlock(newBlock2).runToFuture) { result => result shouldEqual ChainReorganised(oldBranch, newBranch, List(newWeight2, newWeight3)) } @@ -178,13 +188,17 @@ class BlockImportSpec extends AnyFlatSpec with Matchers with ScalaFutures { val blockData2 = BlockData(newBlock2, Seq.empty[Receipt], newWeight2) val blockData3 = BlockData(newBlock3, Seq.empty[Receipt], newWeight3) - (blockImportWithMockedBlockExecution.blockExecution + val mockExecution = mock[BlockExecution] + (mockExecution .executeAndValidateBlocks(_: List[Block], _: ChainWeight)(_: BlockchainConfig)) .expects(newBranch, *, *) .returning((List(blockData2, blockData3), None)) - whenReady(blockImportWithMockedBlockExecution.importBlock(newBlock3).runToFuture)(_ shouldEqual BlockEnqueued) - whenReady(blockImportWithMockedBlockExecution.importBlock(newBlock2).runToFuture) { result => + val withMockedBlockExecution = blockImportWithMockedBlockExecution(mockExecution) + whenReady(withMockedBlockExecution.evaluateBranchBlock(newBlock3).runToFuture)( + _ shouldEqual BlockEnqueued + ) + whenReady(withMockedBlockExecution.evaluateBranchBlock(newBlock2).runToFuture) { result => result shouldEqual ChainReorganised(oldBranch, newBranch, List(newWeight2, newWeight3)) } @@ -222,13 +236,17 @@ class BlockImportSpec extends AnyFlatSpec with Matchers with ScalaFutures { val newBranch = List(newBlock2, newBlock3) val blockData2 = BlockData(newBlock2, Seq.empty[Receipt], newWeight2) - (blockImportWithMockedBlockExecution.blockExecution + val mockExecution = mock[BlockExecution] + (mockExecution .executeAndValidateBlocks(_: List[Block], _: ChainWeight)(_: BlockchainConfig)) .expects(newBranch, *, *) .returning((List(blockData2), Some(execError))) - whenReady(blockImportWithMockedBlockExecution.importBlock(newBlock3).runToFuture)(_ shouldEqual BlockEnqueued) - whenReady(blockImportWithMockedBlockExecution.importBlock(newBlock2).runToFuture) { + val withMockedBlockExecution = blockImportWithMockedBlockExecution(mockExecution) + whenReady(withMockedBlockExecution.evaluateBranchBlock(newBlock3).runToFuture)( + _ shouldEqual BlockEnqueued + ) + whenReady(withMockedBlockExecution.evaluateBranchBlock(newBlock2).runToFuture) { _ shouldBe a[BlockImportFailed] } @@ -254,7 +272,7 @@ class BlockImportSpec extends AnyFlatSpec with Matchers with ScalaFutures { .expects(newBlock.header, *, *) .returning(Left(HeaderParentNotFoundError)) - whenReady(blockImport.importBlock(newBlock).runToFuture)(_ shouldEqual UnknownParent) + whenReady(consensus.evaluateBranchBlock(newBlock).runToFuture)(_ shouldEqual UnknownParent) } it should "validate blocks prior to import" in new ImportBlockTestSetup { @@ -272,7 +290,7 @@ class BlockImportSpec extends AnyFlatSpec with Matchers with ScalaFutures { .expects(newBlock.header, *, *) .returning(Left(HeaderDifficultyError)) - whenReady(blockImport.importBlock(newBlock).runToFuture) { + whenReady(consensus.evaluateBranchBlock(newBlock).runToFuture) { _ shouldEqual BlockImportFailed(HeaderDifficultyError.toString) } } @@ -283,7 +301,7 @@ class BlockImportSpec extends AnyFlatSpec with Matchers with ScalaFutures { setBestBlock(genesisBlock) setBlockExists(genesisBlock, inChain = true, inQueue = true) - whenReady(failBlockImport.importBlock(genesisBlock).runToFuture)(_ shouldEqual DuplicateBlock) + whenReady(failConsensus.evaluateBranchBlock(genesisBlock).runToFuture)(_ shouldEqual DuplicateBlock) } it should "correctly import block with ommers and ancestor in block queue " in new OmmersTestSetup { @@ -320,13 +338,17 @@ class BlockImportSpec extends AnyFlatSpec with Matchers with ScalaFutures { val blockData2 = BlockData(newBlock2, Seq.empty[Receipt], newWeight2) val blockData3 = BlockData(newBlock3WithOmmer, Seq.empty[Receipt], newWeight3) - (blockImportWithMockedBlockExecution.blockExecution + val mockExecution = mock[BlockExecution] + (mockExecution .executeAndValidateBlocks(_: List[Block], _: ChainWeight)(_: BlockchainConfig)) .expects(newBranch, *, *) .returning((List(blockData2, blockData3), None)) - whenReady(blockImportWithMockedBlockExecution.importBlock(newBlock2).runToFuture)(_ shouldEqual BlockEnqueued) - whenReady(blockImportWithMockedBlockExecution.importBlock(newBlock3WithOmmer).runToFuture) { result => + val withMockedBlockExecution = blockImportWithMockedBlockExecution(mockExecution) + whenReady(withMockedBlockExecution.evaluateBranchBlock(newBlock2).runToFuture)( + _ shouldEqual BlockEnqueued + ) + whenReady(withMockedBlockExecution.evaluateBranchBlock(newBlock3WithOmmer).runToFuture) { result => result shouldEqual ChainReorganised(oldBranch, newBranch, List(newWeight2, newWeight3)) } @@ -349,12 +371,14 @@ class BlockImportSpec extends AnyFlatSpec with Matchers with ScalaFutures { blockchainWriter.save(parentBlock, Nil, weightParent, saveAsBestBlock = true) blockchainWriter.save(regularBlock, Nil, weightRegular, saveAsBestBlock = true) - (blockImportWithMockedBlockExecution.blockExecution + val mockExecution = mock[BlockExecution] + (mockExecution .executeAndValidateBlocks(_: List[Block], _: ChainWeight)(_: BlockchainConfig)) .expects(List(checkpointBlock), *, *) .returning((List(BlockData(checkpointBlock, Nil, weightCheckpoint)), None)) - whenReady(blockImportWithMockedBlockExecution.importBlock(checkpointBlock).runToFuture) { result => + val withMockedBlockExecution = blockImportWithMockedBlockExecution(mockExecution) + whenReady(withMockedBlockExecution.evaluateBranchBlock(checkpointBlock).runToFuture) { result => result shouldEqual ChainReorganised( List(regularBlock), List(checkpointBlock), @@ -382,7 +406,10 @@ class BlockImportSpec extends AnyFlatSpec with Matchers with ScalaFutures { blockchainWriter.save(parentBlock, Nil, weightParent, saveAsBestBlock = true) blockchainWriter.save(checkpointBlock, Nil, weightCheckpoint, saveAsBestBlock = true) - whenReady(blockImportWithMockedBlockExecution.importBlock(regularBlock).runToFuture)(_ shouldEqual BlockEnqueued) + val withMockedBlockExecution = blockImportWithMockedBlockExecution(mock[BlockExecution]) + whenReady(withMockedBlockExecution.evaluateBranchBlock(regularBlock).runToFuture)( + _ shouldEqual BlockEnqueued + ) blockchainReader.getBestBlock().get shouldEqual checkpointBlock } diff --git a/src/test/scala/io/iohk/ethereum/ledger/LedgerTestSetup.scala b/src/test/scala/io/iohk/ethereum/ledger/LedgerTestSetup.scala index c3bc3b21bc..fa7413aa9c 100644 --- a/src/test/scala/io/iohk/ethereum/ledger/LedgerTestSetup.scala +++ b/src/test/scala/io/iohk/ethereum/ledger/LedgerTestSetup.scala @@ -467,8 +467,8 @@ trait MockBlockchain extends MockFactory { self: TestSetupWithVmAndValidators => trait EphemBlockchain extends TestSetupWithVmAndValidators with MockFactory { override lazy val blockQueue: BlockQueue = BlockQueue(blockchain, blockchainReader, SyncConfig(Config.config)) - lazy val blockImportWithMockedBlockExecution: Consensus = - mkConsensus(blockExecutionOpt = Some(mock[BlockExecution])) + def blockImportWithMockedBlockExecution(blockExecutionMock: BlockExecution): Consensus = + mkConsensus(blockExecutionOpt = Some(blockExecutionMock)) } trait CheckpointHelpers { @@ -484,6 +484,6 @@ trait OmmersTestSetup extends EphemBlockchain { new StdOmmersValidator(blockHeaderValidator) } - override lazy val blockImportWithMockedBlockExecution: Consensus = - mkConsensus(validators = OmmerValidation, blockExecutionOpt = Some(mock[BlockExecution])) + override def blockImportWithMockedBlockExecution(blockExecutionMock: BlockExecution): Consensus = + mkConsensus(validators = OmmerValidation, blockExecutionOpt = Some(blockExecutionMock)) } From 8a9f21749363c3c767497acc14aeac6834c3cdc8 Mon Sep 17 00:00:00 2001 From: Leonor Boga Date: Fri, 23 Jul 2021 15:36:18 +0200 Subject: [PATCH 05/13] ETCM-1018 Fix integration tests --- src/it/scala/io/iohk/ethereum/txExecTest/ContractTest.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/it/scala/io/iohk/ethereum/txExecTest/ContractTest.scala b/src/it/scala/io/iohk/ethereum/txExecTest/ContractTest.scala index bf9a11c8a9..c58e146c11 100644 --- a/src/it/scala/io/iohk/ethereum/txExecTest/ContractTest.scala +++ b/src/it/scala/io/iohk/ethereum/txExecTest/ContractTest.scala @@ -21,9 +21,9 @@ class ContractTest extends AnyFlatSpec with Matchers { lazy val testBlockchainStorages = FixtureProvider.prepareStorages(2, fixtures) //block only with ether transfers - val blockValidation = + override lazy val blockValidation = new BlockValidation(mining, blockchainReader, BlockQueue(blockchain, blockchainReader, syncConfig)) - val blockExecution = + override lazy val blockExecution = new BlockExecution( blockchain, blockchainReader, From 09df254d979448ec00524708da5311c9bd79727f Mon Sep 17 00:00:00 2001 From: Leonor Boga Date: Fri, 23 Jul 2021 16:20:39 +0200 Subject: [PATCH 06/13] ETCM-1018 Rename BlockImportSpec to ConsensusSpec --- .../ConsensusSpec.scala} | 31 ++++++++++++------- 1 file changed, 19 insertions(+), 12 deletions(-) rename src/test/scala/io/iohk/ethereum/{ledger/BlockImportSpec.scala => consensus/ConsensusSpec.scala} (97%) diff --git a/src/test/scala/io/iohk/ethereum/ledger/BlockImportSpec.scala b/src/test/scala/io/iohk/ethereum/consensus/ConsensusSpec.scala similarity index 97% rename from src/test/scala/io/iohk/ethereum/ledger/BlockImportSpec.scala rename to src/test/scala/io/iohk/ethereum/consensus/ConsensusSpec.scala index 118ab7fc55..ff9f092cd7 100644 --- a/src/test/scala/io/iohk/ethereum/ledger/BlockImportSpec.scala +++ b/src/test/scala/io/iohk/ethereum/consensus/ConsensusSpec.scala @@ -1,12 +1,6 @@ -package io.iohk.ethereum.ledger +package io.iohk.ethereum.consensus import akka.util.ByteString - -import scala.concurrent.duration._ -import scala.language.postfixOps -import org.scalatest.concurrent.ScalaFutures -import org.scalatest.flatspec.AnyFlatSpec -import org.scalatest.matchers.should.Matchers import io.iohk.ethereum.Mocks import io.iohk.ethereum.Mocks.MockValidatorsAlwaysSucceed import io.iohk.ethereum.blockchain.sync.regular.{ @@ -18,17 +12,30 @@ import io.iohk.ethereum.blockchain.sync.regular.{ UnknownParent } import io.iohk.ethereum.consensus.mining._ -import io.iohk.ethereum.consensus.validators.BlockHeaderError.HeaderDifficultyError -import io.iohk.ethereum.consensus.validators.BlockHeaderError.HeaderParentNotFoundError +import io.iohk.ethereum.consensus.validators.BlockHeaderError.{HeaderDifficultyError, HeaderParentNotFoundError} import io.iohk.ethereum.consensus.validators._ import io.iohk.ethereum.db.storage.MptStorage import io.iohk.ethereum.domain._ import io.iohk.ethereum.ledger.BlockQueue.Leaf -import io.iohk.ethereum.mpt.LeafNode -import io.iohk.ethereum.mpt.MerklePatriciaTrie +import io.iohk.ethereum.ledger.{ + BlockData, + BlockExecution, + CheckpointHelpers, + EphemBlockchain, + MockBlockchain, + OmmersTestSetup, + TestSetupWithVmAndValidators +} +import io.iohk.ethereum.mpt.{LeafNode, MerklePatriciaTrie} import io.iohk.ethereum.utils.BlockchainConfig +import org.scalatest.concurrent.ScalaFutures +import org.scalatest.flatspec.AnyFlatSpec +import org.scalatest.matchers.should.Matchers + +import scala.concurrent.duration._ +import scala.language.postfixOps -class BlockImportSpec extends AnyFlatSpec with Matchers with ScalaFutures { +class ConsensusSpec extends AnyFlatSpec with Matchers with ScalaFutures { implicit override val patienceConfig: PatienceConfig = PatienceConfig(timeout = scaled(2 seconds), interval = scaled(1 second)) From 8169f0cf02b78f23cc673f12587d19cc295bf243 Mon Sep 17 00:00:00 2001 From: Leonor Boga Date: Mon, 26 Jul 2021 14:12:24 +0200 Subject: [PATCH 07/13] ETCM-1058 Extend ConsensusSpec error handling tests --- .../ethereum/consensus/ConsensusSpec.scala | 40 +++++++++++++++++-- 1 file changed, 36 insertions(+), 4 deletions(-) diff --git a/src/test/scala/io/iohk/ethereum/consensus/ConsensusSpec.scala b/src/test/scala/io/iohk/ethereum/consensus/ConsensusSpec.scala index ff9f092cd7..f479293d3d 100644 --- a/src/test/scala/io/iohk/ethereum/consensus/ConsensusSpec.scala +++ b/src/test/scala/io/iohk/ethereum/consensus/ConsensusSpec.scala @@ -26,7 +26,7 @@ import io.iohk.ethereum.ledger.{ OmmersTestSetup, TestSetupWithVmAndValidators } -import io.iohk.ethereum.mpt.{LeafNode, MerklePatriciaTrie} +import io.iohk.ethereum.mpt.{LeafNode, MerklePatriciaTrie, NullNode} import io.iohk.ethereum.utils.BlockchainConfig import org.scalatest.concurrent.ScalaFutures import org.scalatest.flatspec.AnyFlatSpec @@ -40,7 +40,7 @@ class ConsensusSpec extends AnyFlatSpec with Matchers with ScalaFutures { implicit override val patienceConfig: PatienceConfig = PatienceConfig(timeout = scaled(2 seconds), interval = scaled(1 second)) - "Importing blocks" should "ignore existing block" in new ImportBlockTestSetup { + "Consensus" should "ignore duplicated block" in new ImportBlockTestSetup { val block1: Block = getBlock() val block2: Block = getBlock() @@ -55,7 +55,7 @@ class ConsensusSpec extends AnyFlatSpec with Matchers with ScalaFutures { whenReady(consensus.evaluateBranchBlock(block2).runToFuture)(_ shouldEqual DuplicateBlock) } - it should "import a block to top of the main chain" in new ImportBlockTestSetup { + it should "import a block to the top of the main chain" in new ImportBlockTestSetup { val block: Block = getBlock(6, parent = bestBlock.header.hash) val difficulty: BigInt = block.header.difficulty val hash: ByteString = block.header.hash @@ -112,7 +112,39 @@ class ConsensusSpec extends AnyFlatSpec with Matchers with ScalaFutures { (blockQueue.removeSubtree _).expects(*) - whenReady(consensus.evaluateBranchBlock(block).runToFuture)(_ shouldBe a[BlockImportFailed]) + whenReady(consensus.evaluateBranchBlock(block).runToFuture)( + _ shouldBe BlockImportFailed("MPTError(io.iohk.ethereum.mpt.MerklePatriciaTrie$MPTException: Invalid Node)") + ) + } + + it should "handle no best block available error when importing to top" in new ImportBlockTestSetup { + val block: Block = getBlock(6, parent = bestBlock.header.hash) + + setBlockExists(block, inChain = false, inQueue = false) + (blockchainReader.getBestBlock _).expects().returning(None) + setChainWeightForBlock(bestBlock, currentWeight) + + whenReady(consensus.evaluateBranchBlock(block).runToFuture)( + _ shouldBe BlockImportFailed("Couldn't find the current best block") + ) + } + + it should "handle total difficulty error when importing to top" in new ImportBlockTestSetup { + val block: Block = getBlock(6, parent = bestBlock.header.hash) + + setBlockExists(block, inChain = false, inQueue = false) + setBestBlock(bestBlock) + (blockchain.getChainWeightByHash _).expects(*).returning(None) + + (blockchainReader.getBlockHeaderByHash _).expects(*).returning(Some(block.header)) + + whenReady(consensus.evaluateBranchBlock(block).runToFuture) { result => + result shouldBe a[BlockImportFailed] + result + .asInstanceOf[BlockImportFailed] + .error + .should(startWith("Couldn't get total difficulty for current best block")) + } } // scalastyle:off magic.number From dd7ba8f317297ce4e3d01674fe897270fb5d0ae1 Mon Sep 17 00:00:00 2001 From: Leonor Boga Date: Mon, 26 Jul 2021 14:13:01 +0200 Subject: [PATCH 08/13] ETCM-1058 Refactor ConsensusImpl --- .../ethereum/consensus/ConsensusImpl.scala | 83 +++++++++++-------- 1 file changed, 47 insertions(+), 36 deletions(-) diff --git a/src/main/scala/io/iohk/ethereum/consensus/ConsensusImpl.scala b/src/main/scala/io/iohk/ethereum/consensus/ConsensusImpl.scala index 7c36333a03..1bde189e20 100644 --- a/src/main/scala/io/iohk/ethereum/consensus/ConsensusImpl.scala +++ b/src/main/scala/io/iohk/ethereum/consensus/ConsensusImpl.scala @@ -44,8 +44,9 @@ class ConsensusImpl( /** Tries to import the block as the new best block in the chain or enqueue it for later processing. * - * @param block block to be imported - * @param blockExecutionContext threadPool on which the execution should be run + * @param block block to be imported + * @param blockExecutionScheduler threadPool on which the execution should be run + * @param blockchainConfig blockchain configuration * @return One of: * - [[BlockImportedToTop]] - if the block was added as the new best block * - [[BlockEnqueued]] - block is stored in the [[io.iohk.ethereum.ledger.BlockQueue]] @@ -59,8 +60,8 @@ class ConsensusImpl( blockchainReader.getBestBlock() match { case Some(bestBlock) => if (isBlockADuplicate(block.header, bestBlock.header.number)) { - Task(log.debug(s"Ignoring duplicate block: (${block.idTag})")) - .map(_ => DuplicateBlock) + log.debug("Ignoring duplicated block: {}", block.idTag) + Task.now(DuplicateBlock) } else { val hash = bestBlock.header.hash blockchain.getChainWeightByHash(hash) match { @@ -68,27 +69,33 @@ class ConsensusImpl( val importResult = if (isPossibleNewBestBlock(block.header, bestBlock.header)) { importToTop(block, bestBlock, weight) } else { - reorganise(block, bestBlock, weight) + reorganiseOrEnqueue(block, bestBlock, weight) } importResult.foreach(measureBlockMetrics) importResult - case None => - log.error( - "Getting total difficulty for current best block with hash: {} failed", - bestBlock.header.hashAsHexString - ) - Task.now( - BlockImportFailed( - s"Couldn't get total difficulty for current best block with hash: ${bestBlock.header.hashAsHexString}" - ) - ) + case None => returnNoTotalDifficulty(bestBlock) } } - case None => - log.error("Getting current best block failed") - Task.now(BlockImportFailed("Couldn't find the current best block")) + case None => returnNoBestBlock() } + private def returnNoTotalDifficulty(bestBlock: Block): Task[BlockImportFailed] = { + log.error( + "Getting total difficulty for current best block with hash: {} failed", + bestBlock.header.hashAsHexString + ) + Task.now( + BlockImportFailed( + s"Couldn't get total difficulty for current best block with hash: ${bestBlock.header.hashAsHexString}" + ) + ) + } + + private def returnNoBestBlock(): Task[BlockImportFailed] = { + log.error("Getting current best block failed") + Task.now(BlockImportFailed("Couldn't find the current best block")) + } + private def isBlockADuplicate(block: BlockHeader, currentBestBlockNumber: BigInt): Boolean = { val hash = block.hash blockchainReader.getBlockByHash(hash).isDefined && block.number <= currentBestBlockNumber || blockQueue.isQueued( @@ -97,7 +104,7 @@ class ConsensusImpl( } private def isPossibleNewBestBlock(newBlock: BlockHeader, currentBestBlock: BlockHeader): Boolean = - newBlock.parentHash == currentBestBlock.hash && newBlock.number == currentBestBlock.number + 1 + newBlock.number == currentBestBlock.number + 1 && newBlock.parentHash == currentBestBlock.hash private def measureBlockMetrics(importResult: BlockImportResult): Unit = importResult match { @@ -120,10 +127,10 @@ class ConsensusImpl( .evalOnce(importBlockToTop(block, currentBestBlock.header.number, currentWeight)) .executeOn(blockExecutionScheduler) - Task.parMap2(validationResult, importResult) { case (validationResult, importResult) => + Task.map2(validationResult, importResult) { case (validationResult, importResult) => validationResult.fold( error => { - log.error("Error while validation block before execution: {}", error.reason) + log.error("Error while validating block before execution: {}", error.reason) handleImportTopValidationError(error, block, importResult) }, _ => importResult @@ -131,6 +138,12 @@ class ConsensusImpl( } } + /** * + * Open for discussion: this is code that was in BlockImport. Even thought is tested (before) that only one block + * is being added to the blockchain, this code assumes that the import may be of more than one block (a branch) + * and because it is assuming that it may be dealing with a branch the code to handle errors assumes several scenarios. + * Is there is reason for this over-complication? + */ private def importBlockToTop(block: Block, bestBlockNumber: BigInt, currentWeight: ChainWeight)(implicit blockchainConfig: BlockchainConfig ): BlockImportResult = { @@ -143,8 +156,7 @@ class ConsensusImpl( executionResult match { case Some((importedBlocks, maybeError, topBlocks)) => val result = maybeError match { - case None => - BlockImportedToTop(importedBlocks) + case None => BlockImportedToTop(importedBlocks) case Some(MPTError(reason)) if reason.isInstanceOf[MissingNodeException] => BlockImportFailedDueToMissingNode(reason.asInstanceOf[MissingNodeException]) @@ -159,15 +171,15 @@ class ConsensusImpl( } BlockImportedToTop(importedBlocks) } - log.debug( - "{}", { - val result = importedBlocks.map { blockData => - val header = blockData.block.header - s"Imported new block (${header.number}: ${Hex.toHexString(header.hash.toArray)}) to the top of chain \n" - } - result.toString - } - ) + + importedBlocks.foreach { blockData => + val header = blockData.block.header + log.debug( + "Imported new block ({}: {}) to the top of chain \n", + header.number, + Hex.toHexString(header.hash.toArray) + ) + } result @@ -204,11 +216,11 @@ class ConsensusImpl( BlockImportFailed(reason.toString) } - private def reorganise( + private def reorganiseOrEnqueue( block: Block, currentBestBlock: Block, currentWeight: ChainWeight - )(implicit blockchainConfig: BlockchainConfig): Task[BlockImportResult] = + )(implicit blockExecutionScheduler: Scheduler, blockchainConfig: BlockchainConfig): Task[BlockImportResult] = Task.evalOnce { blockValidation .validateBlockBeforeExecution(block) @@ -217,7 +229,6 @@ class ConsensusImpl( _ => blockQueue.enqueueBlock(block, currentBestBlock.header.number) match { case Some(Leaf(leafHash, leafWeight)) if leafWeight > currentWeight => - log.debug("Found a better chain, about to reorganise") reorganiseChainFromQueue(leafHash) case _ => @@ -236,7 +247,7 @@ class ConsensusImpl( private def reorganiseChainFromQueue( queuedLeaf: ByteString )(implicit blockchainConfig: BlockchainConfig): BlockImportResult = { - log.debug("Reorganising chain from leaf {}", ByteStringUtils.hash2string(queuedLeaf)) + log.info("Reorganising chain from leaf {}", ByteStringUtils.hash2string(queuedLeaf)) val newBranch = blockQueue.getBranch(queuedLeaf, dequeue = true) val bestNumber = blockchainReader.getBestBlockNumber() From fbef76049429e96c28b35c4e62ce45a625a9e98c Mon Sep 17 00:00:00 2001 From: Leonor Boga Date: Mon, 26 Jul 2021 16:20:52 +0200 Subject: [PATCH 09/13] ETCM-1058 Extract duplicated code for block pre validation --- .../ethereum/consensus/ConsensusImpl.scala | 119 +++++++----------- .../ethereum/consensus/ConsensusSpec.scala | 12 +- 2 files changed, 52 insertions(+), 79 deletions(-) diff --git a/src/main/scala/io/iohk/ethereum/consensus/ConsensusImpl.scala b/src/main/scala/io/iohk/ethereum/consensus/ConsensusImpl.scala index 1bde189e20..70f1fa69ff 100644 --- a/src/main/scala/io/iohk/ethereum/consensus/ConsensusImpl.scala +++ b/src/main/scala/io/iohk/ethereum/consensus/ConsensusImpl.scala @@ -1,6 +1,7 @@ package io.iohk.ethereum.consensus import akka.util.ByteString +import cats.implicits._ import io.iohk.ethereum.blockchain.sync.regular.{ BlockEnqueued, BlockImportFailed, @@ -8,10 +9,8 @@ import io.iohk.ethereum.blockchain.sync.regular.{ BlockImportResult, BlockImportedToTop, ChainReorganised, - DuplicateBlock, - UnknownParent + DuplicateBlock } -import io.iohk.ethereum.consensus.validators.BlockHeaderError.HeaderParentNotFoundError import io.iohk.ethereum.domain.{Block, BlockHeader, BlockchainImpl, BlockchainReader, BlockchainWriter, ChainWeight} import io.iohk.ethereum.ledger.BlockExecutionError.{MPTError, ValidationBeforeExecError} import io.iohk.ethereum.ledger.BlockQueue.Leaf @@ -19,6 +18,7 @@ import io.iohk.ethereum.ledger.{ BlockData, BlockExecution, BlockExecutionError, + BlockExecutionSuccess, BlockMetrics, BlockQueue, BlockValidation @@ -63,22 +63,44 @@ class ConsensusImpl( log.debug("Ignoring duplicated block: {}", block.idTag) Task.now(DuplicateBlock) } else { - val hash = bestBlock.header.hash - blockchain.getChainWeightByHash(hash) match { + blockchain.getChainWeightByHash(bestBlock.header.hash) match { case Some(weight) => - val importResult = if (isPossibleNewBestBlock(block.header, bestBlock.header)) { - importToTop(block, bestBlock, weight) - } else { - reorganiseOrEnqueue(block, bestBlock, weight) + doBlockPreValidation(block).flatMap { + case Left(error) => Task.now(BlockImportFailed(error.reason.toString)) + case Right(_) => handleBlockImport(block, bestBlock, weight) } - importResult.foreach(measureBlockMetrics) - importResult case None => returnNoTotalDifficulty(bestBlock) } } case None => returnNoBestBlock() } + private def handleBlockImport(block: Block, bestBlock: Block, weight: ChainWeight)(implicit + blockExecutionScheduler: Scheduler, + blockchainConfig: BlockchainConfig + ): Task[BlockImportResult] = { + val importResult = if (isPossibleNewBestBlock(block.header, bestBlock.header)) { + importToTop(block, bestBlock, weight) + } else { + reorganiseOrEnqueue(block, bestBlock, weight) + } + importResult.foreach(measureBlockMetrics) + importResult + } + + private def doBlockPreValidation(block: Block)(implicit + blockchainConfig: BlockchainConfig + ): Task[Either[ValidationBeforeExecError, BlockExecutionSuccess]] = + Task + .evalOnce { + val validationResult = blockValidation.validateBlockBeforeExecution(block) + validationResult.left.foreach { error => + log.error("Error while validating block with hash {} before execution: {}", block.hash, error.reason) + } + validationResult + } + .executeOn(validationScheduler) + private def returnNoTotalDifficulty(bestBlock: Block): Task[BlockImportFailed] = { log.error( "Getting total difficulty for current best block with hash: {} failed", @@ -119,24 +141,10 @@ class ConsensusImpl( block: Block, currentBestBlock: Block, currentWeight: ChainWeight - )(implicit blockExecutionScheduler: Scheduler, blockchainConfig: BlockchainConfig): Task[BlockImportResult] = { - val validationResult = - Task.evalOnce(blockValidation.validateBlockBeforeExecution(block)).executeOn(validationScheduler) - val importResult = - Task - .evalOnce(importBlockToTop(block, currentBestBlock.header.number, currentWeight)) - .executeOn(blockExecutionScheduler) - - Task.map2(validationResult, importResult) { case (validationResult, importResult) => - validationResult.fold( - error => { - log.error("Error while validating block before execution: {}", error.reason) - handleImportTopValidationError(error, block, importResult) - }, - _ => importResult - ) - } - } + )(implicit blockExecutionScheduler: Scheduler, blockchainConfig: BlockchainConfig): Task[BlockImportResult] = + Task + .evalOnce(importBlockToTop(block, currentBestBlock.header.number, currentWeight)) + .executeOn(blockExecutionScheduler) /** * * Open for discussion: this is code that was in BlockImport. Even thought is tested (before) that only one block @@ -188,54 +196,18 @@ class ConsensusImpl( } } - private def handleImportTopValidationError( - error: ValidationBeforeExecError, - block: Block, - blockImportResult: BlockImportResult - ): BlockImportResult = { - blockImportResult match { - case BlockImportedToTop(blockImportData) => - blockImportData.foreach { blockData => - val hash = blockData.block.header.hash - blockQueue.removeSubtree(hash) - blockchain.removeBlock(hash, withState = true) - } - case _ => () - } - handleBlockValidationError(error, block) - } - - private def handleBlockValidationError(error: ValidationBeforeExecError, block: Block): BlockImportResult = - error match { - case ValidationBeforeExecError(HeaderParentNotFoundError) => - log.debug(s"Block(${block.idTag}) has unknown parent") - UnknownParent - - case ValidationBeforeExecError(reason) => - log.debug(s"Block(${block.idTag}) failed pre-import validation") - BlockImportFailed(reason.toString) - } - private def reorganiseOrEnqueue( block: Block, currentBestBlock: Block, currentWeight: ChainWeight - )(implicit blockExecutionScheduler: Scheduler, blockchainConfig: BlockchainConfig): Task[BlockImportResult] = - Task.evalOnce { - blockValidation - .validateBlockBeforeExecution(block) - .fold( - error => handleBlockValidationError(error, block), - _ => - blockQueue.enqueueBlock(block, currentBestBlock.header.number) match { - case Some(Leaf(leafHash, leafWeight)) if leafWeight > currentWeight => - reorganiseChainFromQueue(leafHash) - - case _ => - BlockEnqueued - } - ) - } + )(implicit blockchainConfig: BlockchainConfig): Task[BlockImportResult] = + Task.evalOnce(blockQueue.enqueueBlock(block, currentBestBlock.header.number) match { + case Some(Leaf(leafHash, leafWeight)) if leafWeight > currentWeight => + reorganiseChainFromQueue(leafHash) + + case _ => + BlockEnqueued + }) /** Once a better branch was found this attempts to reorganise the chain * @@ -318,7 +290,6 @@ class ConsensusImpl( blockchainWriter.save(block, receipts, weight, saveAsBestBlock = false) } - import cats.implicits._ val checkpointNumber = oldBranch.collect { case BlockData(block, _, _) if block.hasCheckpoint => block.number }.maximumOption diff --git a/src/test/scala/io/iohk/ethereum/consensus/ConsensusSpec.scala b/src/test/scala/io/iohk/ethereum/consensus/ConsensusSpec.scala index f479293d3d..543846df80 100644 --- a/src/test/scala/io/iohk/ethereum/consensus/ConsensusSpec.scala +++ b/src/test/scala/io/iohk/ethereum/consensus/ConsensusSpec.scala @@ -8,8 +8,7 @@ import io.iohk.ethereum.blockchain.sync.regular.{ BlockImportFailed, BlockImportedToTop, ChainReorganised, - DuplicateBlock, - UnknownParent + DuplicateBlock } import io.iohk.ethereum.consensus.mining._ import io.iohk.ethereum.consensus.validators.BlockHeaderError.{HeaderDifficultyError, HeaderParentNotFoundError} @@ -26,7 +25,7 @@ import io.iohk.ethereum.ledger.{ OmmersTestSetup, TestSetupWithVmAndValidators } -import io.iohk.ethereum.mpt.{LeafNode, MerklePatriciaTrie, NullNode} +import io.iohk.ethereum.mpt.{LeafNode, MerklePatriciaTrie} import io.iohk.ethereum.utils.BlockchainConfig import org.scalatest.concurrent.ScalaFutures import org.scalatest.flatspec.AnyFlatSpec @@ -245,7 +244,8 @@ class ConsensusSpec extends AnyFlatSpec with Matchers with ScalaFutures { blockchainWriter.save(blockData2.block, blockData2.receipts, blockData2.weight, saveAsBestBlock = true) blockchainWriter.save(blockData3.block, blockData3.receipts, blockData3.weight, saveAsBestBlock = true) - //saving to cache the value of the best block from the initial chain. This recreates the bug ETCM-626, where (possibly) because of the thread of execution + // saving to cache the value of the best block from the initial chain. This recreates the bug ETCM-626, + // where (possibly) because of the thread of execution // dying before updating the storage but after updating the cache, inconsistency is created blockchain.saveBestKnownBlocks(oldBlock4.number) @@ -311,7 +311,9 @@ class ConsensusSpec extends AnyFlatSpec with Matchers with ScalaFutures { .expects(newBlock.header, *, *) .returning(Left(HeaderParentNotFoundError)) - whenReady(consensus.evaluateBranchBlock(newBlock).runToFuture)(_ shouldEqual UnknownParent) + whenReady(consensus.evaluateBranchBlock(newBlock).runToFuture)( + _ shouldEqual BlockImportFailed("HeaderParentNotFoundError") + ) } it should "validate blocks prior to import" in new ImportBlockTestSetup { From 8532b9798bb9aea3561930ec3f36d77eee93531f Mon Sep 17 00:00:00 2001 From: Leonor Boga Date: Mon, 26 Jul 2021 16:56:08 +0200 Subject: [PATCH 10/13] ETCM-1058 Run scalafixAll --- .../ethereum/ledger/BlockImporterItSpec.scala | 4 ++ .../sync/util/RegularSyncItSpecUtils.scala | 7 ++- .../blockchain/sync/SyncController.scala | 1 + .../sync/regular/BlockImportResult.scala | 3 +- .../sync/regular/BlockImporter.scala | 3 + .../blockchain/sync/regular/RegularSync.scala | 1 + .../iohk/ethereum/consensus/Consensus.scala | 5 +- .../ethereum/consensus/ConsensusImpl.scala | 56 +++++++++++-------- .../iohk/ethereum/jsonrpc/TestService.scala | 8 ++- .../ethereum/nodebuilder/NodeBuilder.scala | 8 ++- .../testmode/TestModeComponentsProvider.scala | 5 +- .../blockchain/sync/ScenarioSetup.scala | 7 ++- .../sync/regular/RegularSyncFixtures.scala | 8 ++- .../sync/regular/RegularSyncSpec.scala | 7 ++- .../ethereum/consensus/ConsensusSpec.scala | 48 ++++++++-------- .../ethereum/ledger/LedgerTestSetup.scala | 7 ++- 16 files changed, 118 insertions(+), 60 deletions(-) diff --git a/src/it/scala/io/iohk/ethereum/ledger/BlockImporterItSpec.scala b/src/it/scala/io/iohk/ethereum/ledger/BlockImporterItSpec.scala index 2ce4add6b2..33908ea701 100644 --- a/src/it/scala/io/iohk/ethereum/ledger/BlockImporterItSpec.scala +++ b/src/it/scala/io/iohk/ethereum/ledger/BlockImporterItSpec.scala @@ -3,16 +3,20 @@ package io.iohk.ethereum.ledger import akka.actor.ActorRef import akka.testkit.TestProbe import akka.util.ByteString + import cats.data.NonEmptyList + import monix.execution.Scheduler import monix.execution.schedulers.SchedulerService import scala.concurrent.duration._ + import org.scalamock.scalatest.MockFactory import org.scalatest.BeforeAndAfterAll import org.scalatest.concurrent.Eventually import org.scalatest.flatspec.AnyFlatSpecLike import org.scalatest.matchers.should.Matchers + import io.iohk.ethereum.Fixtures import io.iohk.ethereum.Mocks import io.iohk.ethereum.NormalPatience diff --git a/src/it/scala/io/iohk/ethereum/sync/util/RegularSyncItSpecUtils.scala b/src/it/scala/io/iohk/ethereum/sync/util/RegularSyncItSpecUtils.scala index 2f6f22bfd9..23403a8485 100644 --- a/src/it/scala/io/iohk/ethereum/sync/util/RegularSyncItSpecUtils.scala +++ b/src/it/scala/io/iohk/ethereum/sync/util/RegularSyncItSpecUtils.scala @@ -4,11 +4,14 @@ import akka.actor.ActorRef import akka.actor.typed import akka.actor.typed.scaladsl.adapter._ import akka.util.ByteString + import cats.effect.Resource + import monix.eval.Task import monix.execution.Scheduler import scala.concurrent.duration._ + import io.iohk.ethereum.Mocks.MockValidatorsAlwaysSucceed import io.iohk.ethereum.blockchain.sync.PeersClient import io.iohk.ethereum.blockchain.sync.SyncProtocol @@ -23,11 +26,13 @@ import io.iohk.ethereum.blockchain.sync.regular.BlockImporter.Start import io.iohk.ethereum.blockchain.sync.regular.RegularSync import io.iohk.ethereum.blockchain.sync.regular.RegularSync.NewCheckpoint import io.iohk.ethereum.checkpointing.CheckpointingTestHelpers +import io.iohk.ethereum.consensus.Consensus +import io.iohk.ethereum.consensus.ConsensusImpl import io.iohk.ethereum.consensus.blocks.CheckpointBlockGenerator import io.iohk.ethereum.consensus.mining.FullMiningConfig import io.iohk.ethereum.consensus.mining.MiningConfig import io.iohk.ethereum.consensus.mining.Protocol.NoAdditionalPoWData -import io.iohk.ethereum.consensus.{Consensus, ConsensusImpl, pow} +import io.iohk.ethereum.consensus.pow import io.iohk.ethereum.consensus.pow.EthashConfig import io.iohk.ethereum.consensus.pow.PoWMining import io.iohk.ethereum.consensus.pow.validators.ValidatorsExecutor diff --git a/src/main/scala/io/iohk/ethereum/blockchain/sync/SyncController.scala b/src/main/scala/io/iohk/ethereum/blockchain/sync/SyncController.scala index 8ff922902f..9c9635457f 100644 --- a/src/main/scala/io/iohk/ethereum/blockchain/sync/SyncController.scala +++ b/src/main/scala/io/iohk/ethereum/blockchain/sync/SyncController.scala @@ -6,6 +6,7 @@ import akka.actor.ActorRef import akka.actor.PoisonPill import akka.actor.Props import akka.actor.Scheduler + import io.iohk.ethereum.blockchain.sync.fast.FastSync import io.iohk.ethereum.blockchain.sync.regular.RegularSync import io.iohk.ethereum.consensus.Consensus diff --git a/src/main/scala/io/iohk/ethereum/blockchain/sync/regular/BlockImportResult.scala b/src/main/scala/io/iohk/ethereum/blockchain/sync/regular/BlockImportResult.scala index 2ab921d5cb..57c07d21a0 100644 --- a/src/main/scala/io/iohk/ethereum/blockchain/sync/regular/BlockImportResult.scala +++ b/src/main/scala/io/iohk/ethereum/blockchain/sync/regular/BlockImportResult.scala @@ -1,6 +1,7 @@ package io.iohk.ethereum.blockchain.sync.regular -import io.iohk.ethereum.domain.{Block, ChainWeight} +import io.iohk.ethereum.domain.Block +import io.iohk.ethereum.domain.ChainWeight import io.iohk.ethereum.ledger.BlockData import io.iohk.ethereum.mpt.MerklePatriciaTrie.MissingNodeException diff --git a/src/main/scala/io/iohk/ethereum/blockchain/sync/regular/BlockImporter.scala b/src/main/scala/io/iohk/ethereum/blockchain/sync/regular/BlockImporter.scala index b951b8d66f..ded43c9ff5 100644 --- a/src/main/scala/io/iohk/ethereum/blockchain/sync/regular/BlockImporter.scala +++ b/src/main/scala/io/iohk/ethereum/blockchain/sync/regular/BlockImporter.scala @@ -7,12 +7,15 @@ import akka.actor.ActorRef import akka.actor.NotInfluenceReceiveTimeout import akka.actor.Props import akka.actor.ReceiveTimeout + import cats.data.NonEmptyList import cats.implicits._ + import monix.eval.Task import monix.execution.Scheduler import scala.concurrent.duration._ + import io.iohk.ethereum.blockchain.sync.Blacklist.BlacklistReason import io.iohk.ethereum.blockchain.sync.regular.BlockBroadcast.BlockToBroadcast import io.iohk.ethereum.blockchain.sync.regular.BlockBroadcasterActor.BroadcastBlocks diff --git a/src/main/scala/io/iohk/ethereum/blockchain/sync/regular/RegularSync.scala b/src/main/scala/io/iohk/ethereum/blockchain/sync/regular/RegularSync.scala index bd9e86f2e0..4a930ebbe0 100644 --- a/src/main/scala/io/iohk/ethereum/blockchain/sync/regular/RegularSync.scala +++ b/src/main/scala/io/iohk/ethereum/blockchain/sync/regular/RegularSync.scala @@ -10,6 +10,7 @@ import akka.actor.Scheduler import akka.actor.SupervisorStrategy import akka.actor.typed.scaladsl.adapter._ import akka.actor.typed.{ActorRef => TypedActorRef} + import io.iohk.ethereum.blockchain.sync.Blacklist import io.iohk.ethereum.blockchain.sync.SyncProtocol import io.iohk.ethereum.blockchain.sync.SyncProtocol.Status diff --git a/src/main/scala/io/iohk/ethereum/consensus/Consensus.scala b/src/main/scala/io/iohk/ethereum/consensus/Consensus.scala index 9165d6653c..da6e3a13b7 100644 --- a/src/main/scala/io/iohk/ethereum/consensus/Consensus.scala +++ b/src/main/scala/io/iohk/ethereum/consensus/Consensus.scala @@ -1,10 +1,11 @@ package io.iohk.ethereum.consensus +import monix.eval.Task +import monix.execution.Scheduler + import io.iohk.ethereum.blockchain.sync.regular.BlockImportResult import io.iohk.ethereum.domain.Block import io.iohk.ethereum.utils.BlockchainConfig -import monix.eval.Task -import monix.execution.Scheduler /** This file documents the original interface that was designed at ETCM-1018 * but implements a different one to be used as a stepping stone to the new architecture diff --git a/src/main/scala/io/iohk/ethereum/consensus/ConsensusImpl.scala b/src/main/scala/io/iohk/ethereum/consensus/ConsensusImpl.scala index 70f1fa69ff..b5e193afdd 100644 --- a/src/main/scala/io/iohk/ethereum/consensus/ConsensusImpl.scala +++ b/src/main/scala/io/iohk/ethereum/consensus/ConsensusImpl.scala @@ -1,36 +1,44 @@ package io.iohk.ethereum.consensus import akka.util.ByteString + import cats.implicits._ -import io.iohk.ethereum.blockchain.sync.regular.{ - BlockEnqueued, - BlockImportFailed, - BlockImportFailedDueToMissingNode, - BlockImportResult, - BlockImportedToTop, - ChainReorganised, - DuplicateBlock -} -import io.iohk.ethereum.domain.{Block, BlockHeader, BlockchainImpl, BlockchainReader, BlockchainWriter, ChainWeight} -import io.iohk.ethereum.ledger.BlockExecutionError.{MPTError, ValidationBeforeExecError} -import io.iohk.ethereum.ledger.BlockQueue.Leaf -import io.iohk.ethereum.ledger.{ - BlockData, - BlockExecution, - BlockExecutionError, - BlockExecutionSuccess, - BlockMetrics, - BlockQueue, - BlockValidation -} -import io.iohk.ethereum.mpt.MerklePatriciaTrie.MissingNodeException -import io.iohk.ethereum.utils.{BlockchainConfig, ByteStringUtils, Logger} + import monix.eval.Task import monix.execution.Scheduler -import org.bouncycastle.util.encoders.Hex import scala.annotation.tailrec +import org.bouncycastle.util.encoders.Hex + +import io.iohk.ethereum.blockchain.sync.regular.BlockEnqueued +import io.iohk.ethereum.blockchain.sync.regular.BlockImportFailed +import io.iohk.ethereum.blockchain.sync.regular.BlockImportFailedDueToMissingNode +import io.iohk.ethereum.blockchain.sync.regular.BlockImportResult +import io.iohk.ethereum.blockchain.sync.regular.BlockImportedToTop +import io.iohk.ethereum.blockchain.sync.regular.ChainReorganised +import io.iohk.ethereum.blockchain.sync.regular.DuplicateBlock +import io.iohk.ethereum.domain.Block +import io.iohk.ethereum.domain.BlockHeader +import io.iohk.ethereum.domain.BlockchainImpl +import io.iohk.ethereum.domain.BlockchainReader +import io.iohk.ethereum.domain.BlockchainWriter +import io.iohk.ethereum.domain.ChainWeight +import io.iohk.ethereum.ledger.BlockData +import io.iohk.ethereum.ledger.BlockExecution +import io.iohk.ethereum.ledger.BlockExecutionError +import io.iohk.ethereum.ledger.BlockExecutionError.MPTError +import io.iohk.ethereum.ledger.BlockExecutionError.ValidationBeforeExecError +import io.iohk.ethereum.ledger.BlockExecutionSuccess +import io.iohk.ethereum.ledger.BlockMetrics +import io.iohk.ethereum.ledger.BlockQueue +import io.iohk.ethereum.ledger.BlockQueue.Leaf +import io.iohk.ethereum.ledger.BlockValidation +import io.iohk.ethereum.mpt.MerklePatriciaTrie.MissingNodeException +import io.iohk.ethereum.utils.BlockchainConfig +import io.iohk.ethereum.utils.ByteStringUtils +import io.iohk.ethereum.utils.Logger + class ConsensusImpl( blockchain: BlockchainImpl, blockchainReader: BlockchainReader, diff --git a/src/main/scala/io/iohk/ethereum/jsonrpc/TestService.scala b/src/main/scala/io/iohk/ethereum/jsonrpc/TestService.scala index d4254bce8e..67b6f8ee29 100644 --- a/src/main/scala/io/iohk/ethereum/jsonrpc/TestService.scala +++ b/src/main/scala/io/iohk/ethereum/jsonrpc/TestService.scala @@ -3,6 +3,7 @@ package io.iohk.ethereum.jsonrpc import akka.actor.ActorRef import akka.util.ByteString import akka.util.Timeout + import monix.eval.Task import monix.execution.Scheduler @@ -10,11 +11,16 @@ import scala.concurrent.duration._ import scala.util.Failure import scala.util.Success import scala.util.Try + import org.bouncycastle.util.encoders.Hex + import io.iohk.ethereum.blockchain.data.GenesisAccount import io.iohk.ethereum.blockchain.data.GenesisData import io.iohk.ethereum.blockchain.data.GenesisDataLoader -import io.iohk.ethereum.blockchain.sync.regular.{BlockEnqueued, BlockImportResult, BlockImportedToTop, ChainReorganised} +import io.iohk.ethereum.blockchain.sync.regular.BlockEnqueued +import io.iohk.ethereum.blockchain.sync.regular.BlockImportResult +import io.iohk.ethereum.blockchain.sync.regular.BlockImportedToTop +import io.iohk.ethereum.blockchain.sync.regular.ChainReorganised import io.iohk.ethereum.consensus.blocks._ import io.iohk.ethereum.consensus.mining.MiningConfig import io.iohk.ethereum.crypto diff --git a/src/main/scala/io/iohk/ethereum/nodebuilder/NodeBuilder.scala b/src/main/scala/io/iohk/ethereum/nodebuilder/NodeBuilder.scala index 7974a8c27a..4234ea0ae3 100644 --- a/src/main/scala/io/iohk/ethereum/nodebuilder/NodeBuilder.scala +++ b/src/main/scala/io/iohk/ethereum/nodebuilder/NodeBuilder.scala @@ -2,10 +2,13 @@ package io.iohk.ethereum.nodebuilder import java.time.Clock import java.util.concurrent.atomic.AtomicReference + import akka.actor.ActorRef import akka.actor.ActorSystem import akka.util.ByteString + import cats.implicits._ + import monix.eval.Task import monix.execution.Scheduler @@ -14,13 +17,16 @@ import scala.concurrent.duration._ import scala.util.Failure import scala.util.Success import scala.util.Try + import org.bouncycastle.crypto.AsymmetricCipherKeyPair + import io.iohk.ethereum.blockchain.data.GenesisDataLoader import io.iohk.ethereum.blockchain.sync.Blacklist import io.iohk.ethereum.blockchain.sync.BlockchainHostActor import io.iohk.ethereum.blockchain.sync.CacheBasedBlacklist import io.iohk.ethereum.blockchain.sync.SyncController -import io.iohk.ethereum.consensus.{Consensus, ConsensusImpl} +import io.iohk.ethereum.consensus.Consensus +import io.iohk.ethereum.consensus.ConsensusImpl import io.iohk.ethereum.consensus.blocks.CheckpointBlockGenerator import io.iohk.ethereum.consensus.mining.MiningBuilder import io.iohk.ethereum.consensus.mining.MiningConfigBuilder diff --git a/src/main/scala/io/iohk/ethereum/testmode/TestModeComponentsProvider.scala b/src/main/scala/io/iohk/ethereum/testmode/TestModeComponentsProvider.scala index bfa773156e..45927acad4 100644 --- a/src/main/scala/io/iohk/ethereum/testmode/TestModeComponentsProvider.scala +++ b/src/main/scala/io/iohk/ethereum/testmode/TestModeComponentsProvider.scala @@ -1,8 +1,11 @@ package io.iohk.ethereum.testmode import akka.util.ByteString -import io.iohk.ethereum.consensus.{Consensus, ConsensusImpl} + import monix.execution.Scheduler + +import io.iohk.ethereum.consensus.Consensus +import io.iohk.ethereum.consensus.ConsensusImpl import io.iohk.ethereum.consensus.mining.MiningConfig import io.iohk.ethereum.crypto import io.iohk.ethereum.db.storage.EvmCodeStorage diff --git a/src/test/scala/io/iohk/ethereum/blockchain/sync/ScenarioSetup.scala b/src/test/scala/io/iohk/ethereum/blockchain/sync/ScenarioSetup.scala index 5412cfe0cb..f872aab56e 100644 --- a/src/test/scala/io/iohk/ethereum/blockchain/sync/ScenarioSetup.scala +++ b/src/test/scala/io/iohk/ethereum/blockchain/sync/ScenarioSetup.scala @@ -1,13 +1,16 @@ package io.iohk.ethereum.blockchain.sync import java.util.concurrent.Executors + import monix.execution.Scheduler import scala.concurrent.ExecutionContext import scala.concurrent.ExecutionContextExecutor + import io.iohk.ethereum.Mocks import io.iohk.ethereum.Mocks.MockVM -import io.iohk.ethereum.consensus.{Consensus, ConsensusImpl} +import io.iohk.ethereum.consensus.Consensus +import io.iohk.ethereum.consensus.ConsensusImpl import io.iohk.ethereum.consensus.mining.Mining import io.iohk.ethereum.consensus.mining.Protocol import io.iohk.ethereum.consensus.mining.StdTestMiningBuilder @@ -69,7 +72,7 @@ trait ScenarioSetup extends StdTestMiningBuilder with StxLedgerBuilder { protected def newTestMining(validators: Validators = mining.validators, vm: VMImpl = mining.vm): Mining = mining.withValidators(validators).withVM(vm) - protected def mkBlockExecution(validators: Validators = validators) = { + protected def mkBlockExecution(validators: Validators = validators): BlockExecution = { val consensuz = mining.withValidators(validators).withVM(new Mocks.MockVM()) val blockValidation = new BlockValidation(consensuz, blockchainReader, blockQueue) new BlockExecution( diff --git a/src/test/scala/io/iohk/ethereum/blockchain/sync/regular/RegularSyncFixtures.scala b/src/test/scala/io/iohk/ethereum/blockchain/sync/regular/RegularSyncFixtures.scala index 2d9b3bf6fa..02b3981208 100644 --- a/src/test/scala/io/iohk/ethereum/blockchain/sync/regular/RegularSyncFixtures.scala +++ b/src/test/scala/io/iohk/ethereum/blockchain/sync/regular/RegularSyncFixtures.scala @@ -1,6 +1,7 @@ package io.iohk.ethereum.blockchain.sync.regular import java.net.InetSocketAddress + import akka.actor.ActorRef import akka.actor.ActorSystem import akka.actor.PoisonPill @@ -10,9 +11,11 @@ import akka.testkit.TestKitBase import akka.testkit.TestProbe import akka.util.ByteString import akka.util.Timeout + import cats.Eq import cats.data.NonEmptyList import cats.implicits._ + import monix.eval.Task import monix.execution.Scheduler import monix.reactive.Observable @@ -23,11 +26,14 @@ import scala.concurrent.duration.DurationInt import scala.concurrent.duration.FiniteDuration import scala.math.BigInt import scala.reflect.ClassTag + import org.scalamock.scalatest.AsyncMockFactory import org.scalatest.matchers.should.Matchers + import io.iohk.ethereum.BlockHelpers import io.iohk.ethereum.blockchain.sync._ -import io.iohk.ethereum.consensus.{Consensus, ConsensusImpl} +import io.iohk.ethereum.consensus.Consensus +import io.iohk.ethereum.consensus.ConsensusImpl import io.iohk.ethereum.consensus.blocks.CheckpointBlockGenerator import io.iohk.ethereum.db.storage.StateStorage import io.iohk.ethereum.domain.BlockHeaderImplicits._ diff --git a/src/test/scala/io/iohk/ethereum/blockchain/sync/regular/RegularSyncSpec.scala b/src/test/scala/io/iohk/ethereum/blockchain/sync/regular/RegularSyncSpec.scala index c91fd8b69d..0436bd93f2 100644 --- a/src/test/scala/io/iohk/ethereum/blockchain/sync/regular/RegularSyncSpec.scala +++ b/src/test/scala/io/iohk/ethereum/blockchain/sync/regular/RegularSyncSpec.scala @@ -8,8 +8,10 @@ import akka.testkit.TestActor.AutoPilot import akka.testkit.TestKit import akka.testkit.TestProbe import akka.util.ByteString + import cats.effect.Resource import cats.syntax.traverse._ + import monix.eval.Task import monix.execution.Scheduler @@ -18,11 +20,13 @@ import scala.concurrent.Future import scala.concurrent.Promise import scala.concurrent.duration._ import scala.math.BigInt + import org.scalamock.scalatest.AsyncMockFactory import org.scalatest.Assertion import org.scalatest.BeforeAndAfterEach import org.scalatest.diagrams.Diagrams import org.scalatest.matchers.should.Matchers + import io.iohk.ethereum.BlockHelpers import io.iohk.ethereum.ObjectGenerators import io.iohk.ethereum.ResourceFixtures @@ -34,7 +38,8 @@ import io.iohk.ethereum.blockchain.sync.SyncProtocol.Status import io.iohk.ethereum.blockchain.sync.SyncProtocol.Status.Progress import io.iohk.ethereum.blockchain.sync.regular.BlockFetcher.Start import io.iohk.ethereum.blockchain.sync.regular.RegularSync.NewCheckpoint -import io.iohk.ethereum.consensus.{Consensus, ConsensusImpl} +import io.iohk.ethereum.consensus.Consensus +import io.iohk.ethereum.consensus.ConsensusImpl import io.iohk.ethereum.crypto.kec256 import io.iohk.ethereum.domain.BlockHeaderImplicits._ import io.iohk.ethereum.domain._ diff --git a/src/test/scala/io/iohk/ethereum/consensus/ConsensusSpec.scala b/src/test/scala/io/iohk/ethereum/consensus/ConsensusSpec.scala index 543846df80..ea575b5ed9 100644 --- a/src/test/scala/io/iohk/ethereum/consensus/ConsensusSpec.scala +++ b/src/test/scala/io/iohk/ethereum/consensus/ConsensusSpec.scala @@ -1,38 +1,38 @@ package io.iohk.ethereum.consensus import akka.util.ByteString + +import scala.concurrent.duration._ +import scala.language.postfixOps + +import org.scalatest.concurrent.ScalaFutures +import org.scalatest.flatspec.AnyFlatSpec +import org.scalatest.matchers.should.Matchers + import io.iohk.ethereum.Mocks import io.iohk.ethereum.Mocks.MockValidatorsAlwaysSucceed -import io.iohk.ethereum.blockchain.sync.regular.{ - BlockEnqueued, - BlockImportFailed, - BlockImportedToTop, - ChainReorganised, - DuplicateBlock -} +import io.iohk.ethereum.blockchain.sync.regular.BlockEnqueued +import io.iohk.ethereum.blockchain.sync.regular.BlockImportFailed +import io.iohk.ethereum.blockchain.sync.regular.BlockImportedToTop +import io.iohk.ethereum.blockchain.sync.regular.ChainReorganised +import io.iohk.ethereum.blockchain.sync.regular.DuplicateBlock import io.iohk.ethereum.consensus.mining._ -import io.iohk.ethereum.consensus.validators.BlockHeaderError.{HeaderDifficultyError, HeaderParentNotFoundError} +import io.iohk.ethereum.consensus.validators.BlockHeaderError.HeaderDifficultyError +import io.iohk.ethereum.consensus.validators.BlockHeaderError.HeaderParentNotFoundError import io.iohk.ethereum.consensus.validators._ import io.iohk.ethereum.db.storage.MptStorage import io.iohk.ethereum.domain._ +import io.iohk.ethereum.ledger.BlockData +import io.iohk.ethereum.ledger.BlockExecution import io.iohk.ethereum.ledger.BlockQueue.Leaf -import io.iohk.ethereum.ledger.{ - BlockData, - BlockExecution, - CheckpointHelpers, - EphemBlockchain, - MockBlockchain, - OmmersTestSetup, - TestSetupWithVmAndValidators -} -import io.iohk.ethereum.mpt.{LeafNode, MerklePatriciaTrie} +import io.iohk.ethereum.ledger.CheckpointHelpers +import io.iohk.ethereum.ledger.EphemBlockchain +import io.iohk.ethereum.ledger.MockBlockchain +import io.iohk.ethereum.ledger.OmmersTestSetup +import io.iohk.ethereum.ledger.TestSetupWithVmAndValidators +import io.iohk.ethereum.mpt.LeafNode +import io.iohk.ethereum.mpt.MerklePatriciaTrie import io.iohk.ethereum.utils.BlockchainConfig -import org.scalatest.concurrent.ScalaFutures -import org.scalatest.flatspec.AnyFlatSpec -import org.scalatest.matchers.should.Matchers - -import scala.concurrent.duration._ -import scala.language.postfixOps class ConsensusSpec extends AnyFlatSpec with Matchers with ScalaFutures { diff --git a/src/test/scala/io/iohk/ethereum/ledger/LedgerTestSetup.scala b/src/test/scala/io/iohk/ethereum/ledger/LedgerTestSetup.scala index fa7413aa9c..ee61e521ba 100644 --- a/src/test/scala/io/iohk/ethereum/ledger/LedgerTestSetup.scala +++ b/src/test/scala/io/iohk/ethereum/ledger/LedgerTestSetup.scala @@ -2,8 +2,11 @@ package io.iohk.ethereum.ledger import akka.util.ByteString import akka.util.ByteString.{empty => bEmpty} + import cats.data.NonEmptyList + import monix.execution.Scheduler + import org.bouncycastle.crypto.AsymmetricCipherKeyPair import org.bouncycastle.crypto.params.ECPublicKeyParameters import org.bouncycastle.util.encoders.Hex @@ -12,11 +15,13 @@ import org.scalamock.handlers.CallHandler1 import org.scalamock.handlers.CallHandler2 import org.scalamock.handlers.CallHandler4 import org.scalamock.scalatest.MockFactory + import io.iohk.ethereum.Fixtures import io.iohk.ethereum.Mocks import io.iohk.ethereum.ObjectGenerators import io.iohk.ethereum.blockchain.sync.EphemBlockchainTestSetup -import io.iohk.ethereum.consensus.{Consensus, ConsensusImpl} +import io.iohk.ethereum.consensus.Consensus +import io.iohk.ethereum.consensus.ConsensusImpl import io.iohk.ethereum.consensus.blocks.CheckpointBlockGenerator import io.iohk.ethereum.consensus.mining.GetBlockHeaderByHash import io.iohk.ethereum.consensus.mining.TestMining From 27208b161b8f928a6296b66220476f9ae35936e2 Mon Sep 17 00:00:00 2001 From: Leonor Boga Date: Wed, 28 Jul 2021 10:02:31 +0200 Subject: [PATCH 11/13] ETCM-1058 Remove file BlockImport --- .../io/iohk/ethereum/ledger/BlockImport.scala | 344 ------------------ 1 file changed, 344 deletions(-) delete mode 100644 src/main/scala/io/iohk/ethereum/ledger/BlockImport.scala diff --git a/src/main/scala/io/iohk/ethereum/ledger/BlockImport.scala b/src/main/scala/io/iohk/ethereum/ledger/BlockImport.scala deleted file mode 100644 index f365aaa88b..0000000000 --- a/src/main/scala/io/iohk/ethereum/ledger/BlockImport.scala +++ /dev/null @@ -1,344 +0,0 @@ -//package io.iohk.ethereum.ledger -// -//import akka.util.ByteString -// -//import monix.eval.Task -//import monix.execution.Scheduler -// -//import scala.annotation.tailrec -// -//import org.bouncycastle.util.encoders.Hex -// -//import io.iohk.ethereum.consensus.validators.BlockHeaderError.HeaderParentNotFoundError -//import io.iohk.ethereum.domain._ -//import io.iohk.ethereum.ledger.BlockExecutionError.MPTError -//import io.iohk.ethereum.ledger.BlockExecutionError.ValidationBeforeExecError -//import io.iohk.ethereum.ledger.BlockQueue.Leaf -//import io.iohk.ethereum.mpt.MerklePatriciaTrie.MissingNodeException -//import io.iohk.ethereum.utils.BlockchainConfig -//import io.iohk.ethereum.utils.ByteStringUtils -//import io.iohk.ethereum.utils.Logger -// -//class BlockImport( -// blockchain: BlockchainImpl, -// blockchainReader: BlockchainReader, -// blockchainWriter: BlockchainWriter, -// blockQueue: BlockQueue, -// blockValidation: BlockValidation, -// private[ledger] val blockExecution: BlockExecution, -// validationScheduler: Scheduler // Can't be implicit because of importToTop method and ambiguous of Scheduler -//) extends Logger { -// -// /** Tries to import the block as the new best block in the chain or enqueue it for later processing. -// * -// * @param block block to be imported -// * @param blockExecutionContext threadPool on which the execution should be run -// * @return One of: -// * - [[io.iohk.ethereum.ledger.BlockImportedToTop]] - if the block was added as the new best block -// * - [[io.iohk.ethereum.ledger.BlockEnqueued]] - block is stored in the [[io.iohk.ethereum.ledger.BlockQueue]] -// * - [[io.iohk.ethereum.ledger.ChainReorganised]] - a better new branch was found causing chain reorganisation -// * - [[io.iohk.ethereum.ledger.DuplicateBlock]] - block already exists either in the main chain or in the queue -// * - [[io.iohk.ethereum.ledger.BlockImportFailed]] - block failed to execute (when importing to top or reorganising the chain) -// */ -// def importBlock( -// block: Block -// )(implicit blockExecutionScheduler: Scheduler, blockchainConfig: BlockchainConfig): Task[BlockImportResult] = -// blockchainReader.getBestBlock() match { -// case Some(bestBlock) => -// if (isBlockADuplicate(block.header, bestBlock.header.number)) { -// Task(log.debug(s"Ignoring duplicate block: (${block.idTag})")) -// .map(_ => DuplicateBlock) -// } else { -// val hash = bestBlock.header.hash -// blockchain.getChainWeightByHash(hash) match { -// case Some(weight) => -// val importResult = if (isPossibleNewBestBlock(block.header, bestBlock.header)) { -// importToTop(block, bestBlock, weight) -// } else { -// reorganise(block, bestBlock, weight) -// } -// importResult.foreach(measureBlockMetrics) -// importResult -// case None => -// log.error( -// "Getting total difficulty for current best block with hash: {} failed", -// bestBlock.header.hashAsHexString -// ) -// Task.now( -// BlockImportFailed( -// s"Couldn't get total difficulty for current best block with hash: ${bestBlock.header.hashAsHexString}" -// ) -// ) -// } -// } -// case None => -// log.error("Getting current best block failed") -// Task.now(BlockImportFailed("Couldn't find the current best block")) -// } -// -// private def isBlockADuplicate(block: BlockHeader, currentBestBlockNumber: BigInt): Boolean = { -// val hash = block.hash -// blockchainReader.getBlockByHash(hash).isDefined && block.number <= currentBestBlockNumber || blockQueue.isQueued( -// hash -// ) -// } -// -// private def isPossibleNewBestBlock(newBlock: BlockHeader, currentBestBlock: BlockHeader): Boolean = -// newBlock.parentHash == currentBestBlock.hash && newBlock.number == currentBestBlock.number + 1 -// -// private def measureBlockMetrics(importResult: BlockImportResult)(implicit blockchainConfig: BlockchainConfig): Unit = -// importResult match { -// case BlockImportedToTop(blockImportData) => -// blockImportData.foreach(blockData => BlockMetrics.measure(blockData.block, blockchainReader.getBlockByHash)) -// case ChainReorganised(_, newBranch, _) => -// newBranch.foreach(block => BlockMetrics.measure(block, blockchainReader.getBlockByHash)) -// case _ => () -// } -// -// private def importToTop( -// block: Block, -// currentBestBlock: Block, -// currentWeight: ChainWeight -// )(implicit blockExecutionScheduler: Scheduler, blockchainConfig: BlockchainConfig): Task[BlockImportResult] = { -// val validationResult = -// Task.evalOnce(blockValidation.validateBlockBeforeExecution(block)).executeOn(validationScheduler) -// val importResult = -// Task -// .evalOnce(importBlockToTop(block, currentBestBlock.header.number, currentWeight)) -// .executeOn(blockExecutionScheduler) -// -// Task.parMap2(validationResult, importResult) { case (validationResult, importResult) => -// validationResult.fold( -// error => { -// log.error("Error while validation block before execution: {}", error.reason) -// handleImportTopValidationError(error, block, importResult) -// }, -// _ => importResult -// ) -// } -// } -// -// private def importBlockToTop(block: Block, bestBlockNumber: BigInt, currentWeight: ChainWeight)(implicit -// blockchainConfig: BlockchainConfig -// ): BlockImportResult = { -// val executionResult = for { -// topBlock <- blockQueue.enqueueBlock(block, bestBlockNumber) -// topBlocks = blockQueue.getBranch(topBlock.hash, dequeue = true) -// (executed, errors) = blockExecution.executeAndValidateBlocks(topBlocks, currentWeight) -// } yield (executed, errors, topBlocks) -// -// executionResult match { -// case Some((importedBlocks, maybeError, topBlocks)) => -// val result = maybeError match { -// case None => -// BlockImportedToTop(importedBlocks) -// -// case Some(MPTError(reason)) if reason.isInstanceOf[MissingNodeException] => -// BlockImportFailedDueToMissingNode(reason.asInstanceOf[MissingNodeException]) -// -// case Some(error) if importedBlocks.isEmpty => -// blockQueue.removeSubtree(block.header.hash) -// BlockImportFailed(error.toString) -// -// case Some(_) => -// topBlocks.drop(importedBlocks.length).headOption.foreach { failedBlock => -// blockQueue.removeSubtree(failedBlock.header.hash) -// } -// BlockImportedToTop(importedBlocks) -// } -// log.debug( -// "{}", { -// val result = importedBlocks.map { blockData => -// val header = blockData.block.header -// s"Imported new block (${header.number}: ${Hex.toHexString(header.hash.toArray)}) to the top of chain \n" -// } -// result.toString -// } -// ) -// -// result -// -// case None => -// BlockImportFailed(s"Newly enqueued block with hash: ${block.header.hash} is not part of a known branch") -// } -// } -// -// private def handleImportTopValidationError( -// error: ValidationBeforeExecError, -// block: Block, -// blockImportResult: BlockImportResult -// ): BlockImportResult = { -// blockImportResult match { -// case BlockImportedToTop(blockImportData) => -// blockImportData.foreach { blockData => -// val hash = blockData.block.header.hash -// blockQueue.removeSubtree(hash) -// blockchain.removeBlock(hash, withState = true) -// } -// case _ => () -// } -// handleBlockValidationError(error, block) -// } -// -// private def handleBlockValidationError(error: ValidationBeforeExecError, block: Block): BlockImportResult = -// error match { -// case ValidationBeforeExecError(HeaderParentNotFoundError) => -// log.debug(s"Block(${block.idTag}) has unknown parent") -// UnknownParent -// -// case ValidationBeforeExecError(reason) => -// log.debug(s"Block(${block.idTag}) failed pre-import validation") -// BlockImportFailed(reason.toString) -// } -// -// private def reorganise( -// block: Block, -// currentBestBlock: Block, -// currentWeight: ChainWeight -// )(implicit blockchainConfig: BlockchainConfig): Task[BlockImportResult] = -// Task.evalOnce { -// blockValidation -// .validateBlockBeforeExecution(block) -// .fold( -// error => handleBlockValidationError(error, block), -// _ => -// blockQueue.enqueueBlock(block, currentBestBlock.header.number) match { -// case Some(Leaf(leafHash, leafWeight)) if leafWeight > currentWeight => -// log.debug("Found a better chain, about to reorganise") -// reorganiseChainFromQueue(leafHash) -// -// case _ => -// BlockEnqueued -// } -// ) -// } -// -// /** Once a better branch was found this attempts to reorganise the chain -// * -// * @param queuedLeaf a block hash that determines a new branch stored in the queue (newest block from the branch) -// * -// * @return [[BlockExecutionError]] if one of the blocks in the new branch failed to execute, otherwise: -// * (oldBranch, newBranch) as lists of blocks -// */ -// private def reorganiseChainFromQueue( -// queuedLeaf: ByteString -// )(implicit blockchainConfig: BlockchainConfig): BlockImportResult = { -// log.debug("Reorganising chain from leaf {}", ByteStringUtils.hash2string(queuedLeaf)) -// val newBranch = blockQueue.getBranch(queuedLeaf, dequeue = true) -// val bestNumber = blockchainReader.getBestBlockNumber() -// -// val reorgResult = for { -// parent <- newBranch.headOption -// parentHash = parent.header.parentHash -// parentWeight <- blockchain.getChainWeightByHash(parentHash) -// } yield { -// log.debug( -// "Removing blocks starting from number {} and parent {}", -// bestNumber, -// ByteStringUtils.hash2string(parentHash) -// ) -// val oldBlocksData = removeBlocksUntil(parentHash, bestNumber) -// oldBlocksData.foreach(block => blockQueue.enqueueBlock(block.block)) -// handleBlockExecResult(newBranch, parentWeight, oldBlocksData) -// } -// -// reorgResult match { -// case Some(execResult) => -// execResult.fold( -// { -// case MPTError(reason: MissingNodeException) => BlockImportFailedDueToMissingNode(reason) -// case err => BlockImportFailed(s"Error while trying to reorganise chain: $err") -// }, -// ChainReorganised.tupled -// ) -// -// case None => -// BlockImportFailed("Error while trying to reorganise chain with parent of new branch") -// } -// } -// -// private def handleBlockExecResult( -// newBranch: List[Block], -// parentWeight: ChainWeight, -// oldBlocksData: List[BlockData] -// )(implicit -// blockchainConfig: BlockchainConfig -// ): Either[BlockExecutionError, (List[Block], List[Block], List[ChainWeight])] = { -// val (executedBlocks, maybeError) = blockExecution.executeAndValidateBlocks(newBranch, parentWeight) -// maybeError match { -// case None => -// Right((oldBlocksData.map(_.block), executedBlocks.map(_.block), executedBlocks.map(_.weight))) -// -// case Some(error) => -// revertChainReorganisation(newBranch, oldBlocksData, executedBlocks) -// Left(error) -// } -// } -// -// /** Reverts chain reorganisation in the event that one of the blocks from new branch fails to execute -// * -// * @param newBranch new blocks -// * @param oldBranch old blocks along with corresponding receipts and totalDifficulties -// * @param executedBlocks sub-sequence of new branch that was executed correctly -// */ -// private def revertChainReorganisation( -// newBranch: List[Block], -// oldBranch: List[BlockData], -// executedBlocks: List[BlockData] -// ): Unit = { -// if (executedBlocks.nonEmpty) { -// removeBlocksUntil(executedBlocks.head.block.header.parentHash, executedBlocks.last.block.header.number) -// } -// -// oldBranch.foreach { case BlockData(block, receipts, weight) => -// blockchainWriter.save(block, receipts, weight, saveAsBestBlock = false) -// } -// -// import cats.implicits._ -// val checkpointNumber = oldBranch.collect { -// case BlockData(block, _, _) if block.hasCheckpoint => block.number -// }.maximumOption -// -// val bestNumber = oldBranch.last.block.header.number -// blockchain.saveBestKnownBlocks(bestNumber, checkpointNumber) -// executedBlocks.foreach(data => blockQueue.enqueueBlock(data.block, bestNumber)) -// -// newBranch.diff(executedBlocks.map(_.block)).headOption.foreach { block => -// blockQueue.removeSubtree(block.header.hash) -// } -// } -// -// /** Removes blocks from the [[Blockchain]] along with receipts and total difficulties. -// * -// * @param parent remove blocks until this hash (exclusive) -// * @param fromNumber start removing from this number (downwards) -// * -// * @return the list of removed blocks along with receipts and total difficulties -// */ -// private def removeBlocksUntil(parent: ByteString, fromNumber: BigInt): List[BlockData] = { -// @tailrec -// def removeBlocksUntil(parent: ByteString, fromNumber: BigInt, acc: List[BlockData]): List[BlockData] = -// blockchainReader.getBestBranch().getBlockByNumber(fromNumber) match { -// case Some(block) if block.header.hash == parent || fromNumber == 0 => -// acc -// -// case Some(block) => -// val hash = block.header.hash -// -// val blockDataOpt = for { -// receipts <- blockchainReader.getReceiptsByHash(hash) -// weight <- blockchain.getChainWeightByHash(hash) -// } yield BlockData(block, receipts, weight) -// -// blockchain.removeBlock(hash, withState = true) -// -// removeBlocksUntil(parent, fromNumber - 1, blockDataOpt.map(_ :: acc).getOrElse(acc)) -// -// case None => -// log.error(s"Unexpected missing block number: $fromNumber") -// acc -// } -// -// removeBlocksUntil(parent, fromNumber, Nil) -// } -//} From 7cf67e7375aaa9559b741277acd9ab3615743f5f Mon Sep 17 00:00:00 2001 From: Leonor Boga Date: Mon, 2 Aug 2021 09:34:54 +0200 Subject: [PATCH 12/13] ETCM-1058 Fix code after rebase --- src/main/scala/io/iohk/ethereum/consensus/ConsensusImpl.scala | 2 +- src/main/scala/io/iohk/ethereum/jsonrpc/TestService.scala | 4 ++-- .../iohk/ethereum/testmode/TestModeComponentsProvider.scala | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/main/scala/io/iohk/ethereum/consensus/ConsensusImpl.scala b/src/main/scala/io/iohk/ethereum/consensus/ConsensusImpl.scala index b5e193afdd..a7fcef3d27 100644 --- a/src/main/scala/io/iohk/ethereum/consensus/ConsensusImpl.scala +++ b/src/main/scala/io/iohk/ethereum/consensus/ConsensusImpl.scala @@ -321,7 +321,7 @@ class ConsensusImpl( private def removeBlocksUntil(parent: ByteString, fromNumber: BigInt): List[BlockData] = { @tailrec def removeBlocksUntil(parent: ByteString, fromNumber: BigInt, acc: List[BlockData]): List[BlockData] = - blockchainReader.getBestBranch().getBlockByNumber(fromNumber) match { + blockchainReader.getBlockByNumber(blockchainReader.getBestBranch(), fromNumber) match { case Some(block) if block.header.hash == parent || fromNumber == 0 => acc diff --git a/src/main/scala/io/iohk/ethereum/jsonrpc/TestService.scala b/src/main/scala/io/iohk/ethereum/jsonrpc/TestService.scala index 67b6f8ee29..9652419baa 100644 --- a/src/main/scala/io/iohk/ethereum/jsonrpc/TestService.scala +++ b/src/main/scala/io/iohk/ethereum/jsonrpc/TestService.scala @@ -264,7 +264,7 @@ class TestService( getBlockForMining(blockchainReader.getBestBlock().get) .flatMap(blockForMining => testModeComponentsProvider - .evaluateBranchBlock(preimageCache) + .getConsensus(preimageCache) .evaluateBranchBlock(blockForMining.block) ) .map { res => @@ -304,7 +304,7 @@ class TestService( Task.now(Left(JsonRpcError(-1, "block validation failed!", None))) case Success(value) => testModeComponentsProvider - .evaluateBranchBlock(preimageCache) + .getConsensus(preimageCache) .evaluateBranchBlock(value) .flatMap(handleResult(value)) } diff --git a/src/main/scala/io/iohk/ethereum/testmode/TestModeComponentsProvider.scala b/src/main/scala/io/iohk/ethereum/testmode/TestModeComponentsProvider.scala index 45927acad4..83fdecbe16 100644 --- a/src/main/scala/io/iohk/ethereum/testmode/TestModeComponentsProvider.scala +++ b/src/main/scala/io/iohk/ethereum/testmode/TestModeComponentsProvider.scala @@ -29,7 +29,7 @@ class TestModeComponentsProvider( node: TestNode ) { - def evaluateBranchBlock( + def getConsensus( preimageCache: collection.concurrent.Map[ByteString, UInt256] ): Consensus = { val consensuz = consensus() From b8a48820a8389e96e6ea661b84c6b494b2256df4 Mon Sep 17 00:00:00 2001 From: Leonor Boga Date: Mon, 2 Aug 2021 13:22:32 +0200 Subject: [PATCH 13/13] ETCM-1058 Simplify code using tap method --- .../io/iohk/ethereum/consensus/ConsensusImpl.scala | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/main/scala/io/iohk/ethereum/consensus/ConsensusImpl.scala b/src/main/scala/io/iohk/ethereum/consensus/ConsensusImpl.scala index a7fcef3d27..adbeef8d0a 100644 --- a/src/main/scala/io/iohk/ethereum/consensus/ConsensusImpl.scala +++ b/src/main/scala/io/iohk/ethereum/consensus/ConsensusImpl.scala @@ -37,6 +37,7 @@ import io.iohk.ethereum.ledger.BlockValidation import io.iohk.ethereum.mpt.MerklePatriciaTrie.MissingNodeException import io.iohk.ethereum.utils.BlockchainConfig import io.iohk.ethereum.utils.ByteStringUtils +import io.iohk.ethereum.utils.FunctorOps._ import io.iohk.ethereum.utils.Logger class ConsensusImpl( @@ -100,12 +101,11 @@ class ConsensusImpl( blockchainConfig: BlockchainConfig ): Task[Either[ValidationBeforeExecError, BlockExecutionSuccess]] = Task - .evalOnce { - val validationResult = blockValidation.validateBlockBeforeExecution(block) - validationResult.left.foreach { error => + .evalOnce(blockValidation.validateBlockBeforeExecution(block)) + .tap { + case Left(error) => log.error("Error while validating block with hash {} before execution: {}", block.hash, error.reason) - } - validationResult + case Right(_) => log.debug("Block with hash {} validated successfully", block.hash) } .executeOn(validationScheduler)