From 32768dc5249aa0b756bb0b34b3cfe362e17178b4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Pawe=C5=82=20=C5=9Al=C4=85ski?= Date: Thu, 29 Oct 2020 16:20:17 +0100 Subject: [PATCH] ETCM-284 block with checkpoint will not generate coinbase reward --- .../ethereum/txExecTest/ContractTest.scala | 8 +- .../ethereum/txExecTest/ECIP1017Test.scala | 2 +- .../iohk/ethereum/txExecTest/ForksTest.scala | 2 +- .../iohk/ethereum/jsonrpc/BlockResponse.scala | 15 +++- .../iohk/ethereum/ledger/BlockExecution.scala | 54 +++++++---- .../io/iohk/ethereum/ledger/BlockImport.scala | 4 +- .../io/iohk/ethereum/snappy/SnappyTest.scala | 2 +- .../CheckpointingTestHelpers.scala | 6 -- .../consensus/BlockGeneratorSpec.scala | 20 ++--- .../iohk/ethereum/domain/BlockchainSpec.scala | 2 +- .../jsonrpc/JsonRpcControllerEthSpec.scala | 90 +++++++++++++++++-- .../jsonrpc/JsonRpcControllerFixture.scala | 11 ++- .../ethereum/ledger/BlockExecutionSpec.scala | 36 ++++++-- .../ethereum/ledger/BlockImportSpec.scala | 8 +- .../io/iohk/ethereum/ledger/LedgerSpec.scala | 8 +- 15 files changed, 206 insertions(+), 62 deletions(-) diff --git a/src/it/scala/io/iohk/ethereum/txExecTest/ContractTest.scala b/src/it/scala/io/iohk/ethereum/txExecTest/ContractTest.scala index 1b81b0d4e3..0c4e6e66e8 100644 --- a/src/it/scala/io/iohk/ethereum/txExecTest/ContractTest.scala +++ b/src/it/scala/io/iohk/ethereum/txExecTest/ContractTest.scala @@ -26,7 +26,7 @@ class ContractTest extends AnyFlatSpec with Matchers { //block only with ether transfers val blockValidation = new BlockValidation(consensus, blockchain, BlockQueue(blockchain, syncConfig)) val blockExecution = new BlockExecution(blockchain, blockchainConfig, consensus.blockPreparator, blockValidation) - blockExecution.executeBlock(fixtures.blockByNumber(1)) shouldBe noErrors + blockExecution.executeAndValidateBlock(fixtures.blockByNumber(1)) shouldBe noErrors } it should "deploy contract" in new ScenarioSetup { @@ -37,7 +37,7 @@ class ContractTest extends AnyFlatSpec with Matchers { //contract creation val blockValidation = new BlockValidation(consensus, blockchain, BlockQueue(blockchain, syncConfig)) val blockExecution = new BlockExecution(blockchain, blockchainConfig, consensus.blockPreparator, blockValidation) - blockExecution.executeBlock(fixtures.blockByNumber(2)) shouldBe noErrors + blockExecution.executeAndValidateBlock(fixtures.blockByNumber(2)) shouldBe noErrors } it should "execute contract call" in new ScenarioSetup { @@ -48,7 +48,7 @@ class ContractTest extends AnyFlatSpec with Matchers { //block with ether transfers and contract call val blockValidation = new BlockValidation(consensus, blockchain, BlockQueue(blockchain, syncConfig)) val blockExecution = new BlockExecution(blockchain, blockchainConfig, consensus.blockPreparator, blockValidation) - blockExecution.executeBlock(fixtures.blockByNumber(3)) shouldBe noErrors + blockExecution.executeAndValidateBlock(fixtures.blockByNumber(3)) shouldBe noErrors } it should "execute contract that pays 2 accounts" in new ScenarioSetup { @@ -59,6 +59,6 @@ class ContractTest extends AnyFlatSpec with Matchers { //block contains contract paying 2 accounts val blockValidation = new BlockValidation(consensus, blockchain, BlockQueue(blockchain, syncConfig)) val blockExecution = new BlockExecution(blockchain, blockchainConfig, consensus.blockPreparator, blockValidation) - blockExecution.executeBlock(fixtures.blockByNumber(3)) shouldBe noErrors + blockExecution.executeAndValidateBlock(fixtures.blockByNumber(3)) shouldBe noErrors } } diff --git a/src/it/scala/io/iohk/ethereum/txExecTest/ECIP1017Test.scala b/src/it/scala/io/iohk/ethereum/txExecTest/ECIP1017Test.scala index 5b9740d3cf..7dad3e801c 100644 --- a/src/it/scala/io/iohk/ethereum/txExecTest/ECIP1017Test.scala +++ b/src/it/scala/io/iohk/ethereum/txExecTest/ECIP1017Test.scala @@ -76,7 +76,7 @@ class ECIP1017Test extends AnyFlatSpec with Matchers { val blockchain = BlockchainImpl(storages) val blockValidation = new BlockValidation(consensus, blockchain, BlockQueue(blockchain, syncConfig)) val blockExecution = new BlockExecution(blockchain, blockchainConfig, consensus.blockPreparator, blockValidation) - blockExecution.executeBlock(fixtures.blockByNumber(blockToExecute)) shouldBe noErrors + blockExecution.executeAndValidateBlock(fixtures.blockByNumber(blockToExecute)) shouldBe noErrors } } diff --git a/src/it/scala/io/iohk/ethereum/txExecTest/ForksTest.scala b/src/it/scala/io/iohk/ethereum/txExecTest/ForksTest.scala index ea1373961f..cbd8249358 100644 --- a/src/it/scala/io/iohk/ethereum/txExecTest/ForksTest.scala +++ b/src/it/scala/io/iohk/ethereum/txExecTest/ForksTest.scala @@ -67,7 +67,7 @@ class ForksTest extends AnyFlatSpec with Matchers { val blockchain = BlockchainImpl(storages) val blockValidation = new BlockValidation(consensus, blockchain, BlockQueue(blockchain, syncConfig)) val blockExecution = new BlockExecution(blockchain, blockchainConfig, consensus.blockPreparator, blockValidation) - blockExecution.executeBlock(fixtures.blockByNumber(blockToExecute)) shouldBe noErrors + blockExecution.executeAndValidateBlock(fixtures.blockByNumber(blockToExecute)) shouldBe noErrors } } diff --git a/src/main/scala/io/iohk/ethereum/jsonrpc/BlockResponse.scala b/src/main/scala/io/iohk/ethereum/jsonrpc/BlockResponse.scala index e29e8b8d2b..fed9a8803e 100644 --- a/src/main/scala/io/iohk/ethereum/jsonrpc/BlockResponse.scala +++ b/src/main/scala/io/iohk/ethereum/jsonrpc/BlockResponse.scala @@ -1,7 +1,10 @@ package io.iohk.ethereum.jsonrpc import akka.util.ByteString -import io.iohk.ethereum.domain.{Block, BlockHeader, BlockBody} +import io.iohk.ethereum.crypto.ECDSASignature +import io.iohk.ethereum.domain.{Block, BlockBody, BlockHeader} + +case class CheckpointResponse(signatures: Seq[ECDSASignature], signers: Seq[ByteString]) case class BlockResponse( number: BigInt, @@ -21,6 +24,8 @@ case class BlockResponse( gasLimit: BigInt, gasUsed: BigInt, timestamp: BigInt, + checkpoint: Option[CheckpointResponse], + treasuryOptOut: Option[Boolean], transactions: Either[Seq[ByteString], Seq[TransactionResponse]], uncles: Seq[ByteString] ) @@ -41,6 +46,12 @@ object BlockResponse { else Left(block.body.transactionList.map(_.hash)) + val checkpoint = block.header.checkpoint.map { checkpoint => + val signers = checkpoint.signatures.flatMap(_.publicKey(block.header.parentHash)) + + CheckpointResponse(checkpoint.signatures, signers) + } + BlockResponse( number = block.header.number, hash = if (pendingBlock) None else Some(block.header.hash), @@ -59,6 +70,8 @@ object BlockResponse { gasLimit = block.header.gasLimit, gasUsed = block.header.gasUsed, timestamp = block.header.unixTimestamp, + checkpoint = checkpoint, + treasuryOptOut = block.header.treasuryOptOut, transactions = transactions, uncles = block.body.uncleNodesList.map(_.hash) ) diff --git a/src/main/scala/io/iohk/ethereum/ledger/BlockExecution.scala b/src/main/scala/io/iohk/ethereum/ledger/BlockExecution.scala index 1362385553..fd529761a3 100644 --- a/src/main/scala/io/iohk/ethereum/ledger/BlockExecution.scala +++ b/src/main/scala/io/iohk/ethereum/ledger/BlockExecution.scala @@ -14,25 +14,36 @@ class BlockExecution( blockValidation: BlockValidation ) extends Logger { - /** Executes a block + /** Executes and validate a block * * @param alreadyValidated should we skip pre-execution validation (if the block has already been validated, * eg. in the importBlock method) */ - def executeBlock(block: Block, alreadyValidated: Boolean = false): Either[BlockExecutionError, Seq[Receipt]] = { + def executeAndValidateBlock( + block: Block, + alreadyValidated: Boolean = false + ): Either[BlockExecutionError, Seq[Receipt]] = { val preExecValidationResult = if (alreadyValidated) Right(block) else blockValidation.validateBlockBeforeExecution(block) - val blockExecResult = for { - _ <- preExecValidationResult - execResult <- executeBlockTransactions(block) - BlockResult(resultingWorldStateProxy, gasUsed, receipts) = execResult - worldToPersist = blockPreparator.payBlockReward(block, resultingWorldStateProxy) - // State root hash needs to be up-to-date for validateBlockAfterExecution - worldPersisted = InMemoryWorldStateProxy.persistState(worldToPersist) - _ <- blockValidation.validateBlockAfterExecution(block, worldPersisted.stateRootHash, receipts, gasUsed) - - } yield receipts + val blockExecResult = { + if (block.hasCheckpoint) { + // block with checkpoint is not executed normally - it's not need to do after execution validation + preExecValidationResult + .map(_ => Seq.empty[Receipt]) + } else { + for { + _ <- preExecValidationResult + result <- executeBlock(block) + _ <- blockValidation.validateBlockAfterExecution( + block, + result.worldState.stateRootHash, + result.receipts, + result.gasUsed + ) + } yield result.receipts + } + } if (blockExecResult.isRight) { log.debug(s"Block ${block.header.number} (with hash: ${block.header.hashAsHexString}) executed correctly") @@ -41,6 +52,16 @@ class BlockExecution( blockExecResult } + /** Executes a block (executes transactions and pays rewards) */ + private def executeBlock(block: Block): Either[BlockExecutionError, BlockResult] = { + for { + execResult <- executeBlockTransactions(block) + worldToPersist = blockPreparator.payBlockReward(block, execResult.worldState) + // State root hash needs to be up-to-date for validateBlockAfterExecution + worldPersisted = InMemoryWorldStateProxy.persistState(worldToPersist) + } yield execResult.copy(worldState = worldPersisted) + } + /** This function runs transactions * * @param block the block with transactions to run @@ -97,14 +118,17 @@ class BlockExecution( } } - /** Executes a list of blocks, storing the results in the blockchain. + /** Executes and validates a list of blocks, storing the results in the blockchain. * * @param blocks blocks to be executed * @param parentTd transaction difficulty of the parent * * @return a list of blocks that were correctly executed and an optional [[BlockExecutionError]] */ - def executeBlocks(blocks: List[Block], parentTd: BigInt): (List[BlockData], Option[BlockExecutionError]) = { + def executeAndValidateBlocks( + blocks: List[Block], + parentTd: BigInt + ): (List[BlockData], Option[BlockExecutionError]) = { @tailrec def go( executedBlocks: List[BlockData], @@ -118,7 +142,7 @@ class BlockExecution( (executedBlocks, error) } else { val blockToExecute = remainingBlocks.head - executeBlock(blockToExecute, alreadyValidated = true) match { + executeAndValidateBlock(blockToExecute, alreadyValidated = true) match { case Right(receipts) => val td = parentTd + blockToExecute.header.difficulty val newBlockData = BlockData(blockToExecute, receipts, td) diff --git a/src/main/scala/io/iohk/ethereum/ledger/BlockImport.scala b/src/main/scala/io/iohk/ethereum/ledger/BlockImport.scala index 61d4486f9a..ecad82ebb6 100644 --- a/src/main/scala/io/iohk/ethereum/ledger/BlockImport.scala +++ b/src/main/scala/io/iohk/ethereum/ledger/BlockImport.scala @@ -42,7 +42,7 @@ class BlockImport( val executionResult = for { topBlock <- blockQueue.enqueueBlock(block, bestBlockNumber) topBlocks = blockQueue.getBranch(topBlock.hash, dequeue = true) - (executed, errors) = blockExecution.executeBlocks(topBlocks, currentTd) + (executed, errors) = blockExecution.executeAndValidateBlocks(topBlocks, currentTd) } yield (executed, errors, topBlocks) executionResult match { @@ -190,7 +190,7 @@ class BlockImport( parentTd: BigInt, oldBlocksData: List[BlockData] ): Either[BlockExecutionError, (List[Block], List[Block])] = { - val (executedBlocks, maybeError) = blockExecution.executeBlocks(newBranch, parentTd) + val (executedBlocks, maybeError) = blockExecution.executeAndValidateBlocks(newBranch, parentTd) maybeError match { case None => Right(oldBlocksData.map(_.block), executedBlocks.map(_.block)) diff --git a/src/snappy/scala/io/iohk/ethereum/snappy/SnappyTest.scala b/src/snappy/scala/io/iohk/ethereum/snappy/SnappyTest.scala index 6b1f720ad4..9c0069f904 100644 --- a/src/snappy/scala/io/iohk/ethereum/snappy/SnappyTest.scala +++ b/src/snappy/scala/io/iohk/ethereum/snappy/SnappyTest.scala @@ -64,7 +64,7 @@ class SnappyTest extends AnyFreeSpec with Matchers with Logger { val blockValidation = new BlockValidation(consensus, blockchain, BlockQueue(blockchain, syncConfig)) val blockExecution = new BlockExecution(blockchain, blockchainConfig, consensus.blockPreparator, blockValidation) - blockExecution.executeBlock(block) + blockExecution.executeAndValidateBlock(block) case None => // this seems to discard failures, for better errors messages we might want to implement a different method (simulateBlock?) diff --git a/src/test/scala/io/iohk/ethereum/checkpointing/CheckpointingTestHelpers.scala b/src/test/scala/io/iohk/ethereum/checkpointing/CheckpointingTestHelpers.scala index f598344acf..8f3e8ee5ec 100644 --- a/src/test/scala/io/iohk/ethereum/checkpointing/CheckpointingTestHelpers.scala +++ b/src/test/scala/io/iohk/ethereum/checkpointing/CheckpointingTestHelpers.scala @@ -9,12 +9,6 @@ import org.bouncycastle.crypto.AsymmetricCipherKeyPair import io.iohk.ethereum.crypto.ECDSASignatureImplicits.ECDSASignatureOrdering object CheckpointingTestHelpers { - def createBlockWithCheckpoint( - parentHeader: BlockHeader, - checkpoint: Checkpoint - ): Block = { - Block(createBlockHeaderWithCheckpoint(parentHeader, checkpoint), BlockBody(Nil, Nil)) - } def createBlockHeaderWithCheckpoint( parentHeader: BlockHeader, diff --git a/src/test/scala/io/iohk/ethereum/consensus/BlockGeneratorSpec.scala b/src/test/scala/io/iohk/ethereum/consensus/BlockGeneratorSpec.scala index 08055726b5..6012dee4e8 100644 --- a/src/test/scala/io/iohk/ethereum/consensus/BlockGeneratorSpec.scala +++ b/src/test/scala/io/iohk/ethereum/consensus/BlockGeneratorSpec.scala @@ -47,7 +47,7 @@ class BlockGeneratorSpec extends AnyFlatSpec with Matchers with ScalaCheckProper fullBlock.header, blockchain.getBlockHeaderByHash ) shouldBe Right(BlockHeaderValid) - blockExecution.executeBlock(fullBlock) shouldBe a[Right[_, Seq[Receipt]]] + blockExecution.executeAndValidateBlock(fullBlock) shouldBe a[Right[_, Seq[Receipt]]] fullBlock.header.extraData shouldBe headerExtraData } @@ -67,7 +67,7 @@ class BlockGeneratorSpec extends AnyFlatSpec with Matchers with ScalaCheckProper fullBlock.header, blockchain.getBlockHeaderByHash ) shouldBe Right(BlockHeaderValid) - blockExecution.executeBlock(fullBlock) shouldBe a[Right[_, Seq[Receipt]]] + blockExecution.executeAndValidateBlock(fullBlock) shouldBe a[Right[_, Seq[Receipt]]] fullBlock.header.extraData shouldBe headerExtraData } @@ -135,7 +135,7 @@ class BlockGeneratorSpec extends AnyFlatSpec with Matchers with ScalaCheckProper blockchain.getBlockHeaderByHash ) shouldBe Right(BlockHeaderValid) - blockExecution.executeBlock(fullBlock) shouldBe a[Right[_, Seq[Receipt]]] + blockExecution.executeAndValidateBlock(fullBlock) shouldBe a[Right[_, Seq[Receipt]]] fullBlock.body.transactionList shouldBe Seq(signedTransaction.tx) fullBlock.header.extraData shouldBe headerExtraData } @@ -166,7 +166,7 @@ class BlockGeneratorSpec extends AnyFlatSpec with Matchers with ScalaCheckProper fullBlock.header, blockchain.getBlockHeaderByHash ) shouldBe Right(BlockHeaderValid) - blockExecution.executeBlock(fullBlock) shouldBe a[Right[_, Seq[Receipt]]] + blockExecution.executeAndValidateBlock(fullBlock) shouldBe a[Right[_, Seq[Receipt]]] fullBlock.body.transactionList shouldBe Seq(signedTransaction.tx) fullBlock.header.extraData shouldBe headerExtraData } @@ -231,7 +231,7 @@ class BlockGeneratorSpec extends AnyFlatSpec with Matchers with ScalaCheckProper fullBlock.header, blockchain.getBlockHeaderByHash ) shouldBe Right(BlockHeaderValid) - blockExecution.executeBlock(fullBlock) shouldBe a[Right[_, Seq[Receipt]]] + blockExecution.executeAndValidateBlock(fullBlock) shouldBe a[Right[_, Seq[Receipt]]] fullBlock.body.transactionList shouldBe Seq(generalTx) fullBlock.header.extraData shouldBe headerExtraData } @@ -289,7 +289,7 @@ class BlockGeneratorSpec extends AnyFlatSpec with Matchers with ScalaCheckProper val generatedBlock = blockGenerator.generateBlock(bestBlock, Seq(generalTx), Address(testAddress), blockGenerator.emptyX) - blockExecution.executeBlock(generatedBlock.block, true) shouldBe a[Right[_, Seq[Receipt]]] + blockExecution.executeAndValidateBlock(generatedBlock.block, true) shouldBe a[Right[_, Seq[Receipt]]] } it should "generate block after eip155 and allow both chain specific and general transactions" in new TestSetup { @@ -315,7 +315,7 @@ class BlockGeneratorSpec extends AnyFlatSpec with Matchers with ScalaCheckProper validators.blockHeaderValidator.validate(fullBlock.header, blockchain.getBlockHeaderByHash) shouldBe Right( BlockHeaderValid ) - blockExecution.executeBlock(fullBlock) shouldBe a[Right[_, Seq[Receipt]]] + blockExecution.executeAndValidateBlock(fullBlock) shouldBe a[Right[_, Seq[Receipt]]] fullBlock.body.transactionList shouldBe Seq(signedTransaction.tx, generalTx) fullBlock.header.extraData shouldBe headerExtraData } @@ -344,7 +344,7 @@ class BlockGeneratorSpec extends AnyFlatSpec with Matchers with ScalaCheckProper validators.blockHeaderValidator.validate(fullBlock.header, blockchain.getBlockHeaderByHash) shouldBe Right( BlockHeaderValid ) - blockExecution.executeBlock(fullBlock) shouldBe a[Right[_, Seq[Receipt]]] + blockExecution.executeAndValidateBlock(fullBlock) shouldBe a[Right[_, Seq[Receipt]]] fullBlock.body.transactionList shouldBe Seq(signedTransaction.tx, nextTransaction) fullBlock.header.extraData shouldBe headerExtraData } @@ -386,7 +386,7 @@ class BlockGeneratorSpec extends AnyFlatSpec with Matchers with ScalaCheckProper validators.blockHeaderValidator.validate(fullBlock.header, blockchain.getBlockHeaderByHash) shouldBe Right( BlockHeaderValid ) - blockExecution.executeBlock(fullBlock) shouldBe a[Right[_, Seq[Receipt]]] + blockExecution.executeAndValidateBlock(fullBlock) shouldBe a[Right[_, Seq[Receipt]]] fullBlock.body.transactionList shouldBe Seq(signedTransaction.tx, nextTransaction) fullBlock.header.extraData shouldBe headerExtraData } @@ -415,7 +415,7 @@ class BlockGeneratorSpec extends AnyFlatSpec with Matchers with ScalaCheckProper validators.blockHeaderValidator.validate(fullBlock.header, blockchain.getBlockHeaderByHash) shouldBe Right( BlockHeaderValid ) - blockExecution.executeBlock(fullBlock) shouldBe a[Right[_, Seq[Receipt]]] + blockExecution.executeAndValidateBlock(fullBlock) shouldBe a[Right[_, Seq[Receipt]]] fullBlock.body.transactionList shouldBe Seq(signedTransaction.tx) fullBlock.header.extraData shouldBe headerExtraData } diff --git a/src/test/scala/io/iohk/ethereum/domain/BlockchainSpec.scala b/src/test/scala/io/iohk/ethereum/domain/BlockchainSpec.scala index cd578af892..d4220d6d6e 100644 --- a/src/test/scala/io/iohk/ethereum/domain/BlockchainSpec.scala +++ b/src/test/scala/io/iohk/ethereum/domain/BlockchainSpec.scala @@ -56,7 +56,7 @@ class BlockchainSpec extends AnyFlatSpec with Matchers { val parent = Fixtures.Blocks.Genesis.block blockchain.storeBlock(parent) - val validBlock = CheckpointingTestHelpers.createBlockWithCheckpoint(parent.header, checkpoint) + val validBlock = new CheckpointBlockGenerator().generate(parent, checkpoint) blockchain.save(validBlock, Seq.empty, BigInt(0), saveAsBestBlock = true) diff --git a/src/test/scala/io/iohk/ethereum/jsonrpc/JsonRpcControllerEthSpec.scala b/src/test/scala/io/iohk/ethereum/jsonrpc/JsonRpcControllerEthSpec.scala index 702198883c..4ffe998a54 100644 --- a/src/test/scala/io/iohk/ethereum/jsonrpc/JsonRpcControllerEthSpec.scala +++ b/src/test/scala/io/iohk/ethereum/jsonrpc/JsonRpcControllerEthSpec.scala @@ -12,12 +12,8 @@ import io.iohk.ethereum.crypto.kec256 import io.iohk.ethereum.domain._ import io.iohk.ethereum.jsonrpc.EthService._ import io.iohk.ethereum.jsonrpc.FilterManager.LogFilterLogs -import io.iohk.ethereum.jsonrpc.serialization.JsonSerializers.{ - OptionNoneToJNullSerializer, - QuantitiesSerializer, - UnformattedDataJsonSerializer -} import io.iohk.ethereum.jsonrpc.PersonalService._ +import io.iohk.ethereum.jsonrpc.serialization.JsonSerializers.{OptionNoneToJNullSerializer, QuantitiesSerializer, UnformattedDataJsonSerializer} import io.iohk.ethereum.ommers.OmmersPool import io.iohk.ethereum.ommers.OmmersPool.Ommers import io.iohk.ethereum.testing.ActorsTesting.simpleAutoPilot @@ -110,6 +106,48 @@ class JsonRpcControllerEthSpec response should haveResult(expectedBlockResponse) } + it should "handle eth_getBlockByHash request (block with checkpoint)" in new JsonRpcControllerFixture { + val blockToRequest = blockWithCheckpoint + val blockTd = blockToRequest.header.difficulty + + blockchain + .storeBlock(blockToRequest) + .and(blockchain.storeTotalDifficulty(blockToRequest.header.hash, blockTd)) + .commit() + + val request = newJsonRpcRequest( + "eth_getBlockByHash", + List(JString(s"0x${blockToRequest.header.hashAsHexString}"), JBool(false)) + ) + val response = jsonRpcController.handleRequest(request).futureValue + + val expectedBlockResponse = + Extraction.decompose(BlockResponse(blockToRequest, fullTxs = false, totalDifficulty = Some(blockTd))) + + response should haveResult(expectedBlockResponse) + } + + it should "handle eth_getBlockByHash request (block with treasuryOptOut)" in new JsonRpcControllerFixture { + val blockToRequest = blockWithTreasuryOptOut + val blockTd = blockToRequest.header.difficulty + + blockchain + .storeBlock(blockToRequest) + .and(blockchain.storeTotalDifficulty(blockToRequest.header.hash, blockTd)) + .commit() + + val request = newJsonRpcRequest( + "eth_getBlockByHash", + List(JString(s"0x${blockToRequest.header.hashAsHexString}"), JBool(false)) + ) + val response = jsonRpcController.handleRequest(request).futureValue + + val expectedBlockResponse = + Extraction.decompose(BlockResponse(blockToRequest, fullTxs = false, totalDifficulty = Some(blockTd))) + + response should haveResult(expectedBlockResponse) + } + it should "handle eth_getBlockByNumber request" in new JsonRpcControllerFixture { val blockToRequest = Block(Fixtures.Blocks.Block3125369.header, Fixtures.Blocks.Block3125369.body) val blockTd = blockToRequest.header.difficulty @@ -131,6 +169,48 @@ class JsonRpcControllerEthSpec response should haveResult(expectedBlockResponse) } + it should "handle eth_getBlockByNumber request (block with treasuryOptOut)" in new JsonRpcControllerFixture { + val blockToRequest = blockWithTreasuryOptOut + val blockTd = blockToRequest.header.difficulty + + blockchain + .storeBlock(blockToRequest) + .and(blockchain.storeTotalDifficulty(blockToRequest.header.hash, blockTd)) + .commit() + + val request = newJsonRpcRequest( + "eth_getBlockByNumber", + List(JString(s"0x${Hex.toHexString(blockToRequest.header.number.toByteArray)}"), JBool(false)) + ) + val response = jsonRpcController.handleRequest(request).futureValue + + val expectedBlockResponse = + Extraction.decompose(BlockResponse(blockToRequest, fullTxs = false, totalDifficulty = Some(blockTd))) + + response should haveResult(expectedBlockResponse) + } + + it should "handle eth_getBlockByNumber request (block with checkpoint)" in new JsonRpcControllerFixture { + val blockToRequest = blockWithCheckpoint + val blockTd = blockToRequest.header.difficulty + + blockchain + .storeBlock(blockToRequest) + .and(blockchain.storeTotalDifficulty(blockToRequest.header.hash, blockTd)) + .commit() + + val request = newJsonRpcRequest( + "eth_getBlockByNumber", + List(JString(s"0x${Hex.toHexString(blockToRequest.header.number.toByteArray)}"), JBool(false)) + ) + val response = jsonRpcController.handleRequest(request).futureValue + + val expectedBlockResponse = + Extraction.decompose(BlockResponse(blockToRequest, fullTxs = false, totalDifficulty = Some(blockTd))) + + response should haveResult(expectedBlockResponse) + } + it should "handle eth_getUncleByBlockHashAndIndex request" in new JsonRpcControllerFixture { val uncle = Fixtures.Blocks.DaoForkBlock.header val blockToRequest = Block(Fixtures.Blocks.Block3125369.header, BlockBody(Nil, Seq(uncle))) diff --git a/src/test/scala/io/iohk/ethereum/jsonrpc/JsonRpcControllerFixture.scala b/src/test/scala/io/iohk/ethereum/jsonrpc/JsonRpcControllerFixture.scala index 576695fc6c..e36ed1267d 100644 --- a/src/test/scala/io/iohk/ethereum/jsonrpc/JsonRpcControllerFixture.scala +++ b/src/test/scala/io/iohk/ethereum/jsonrpc/JsonRpcControllerFixture.scala @@ -3,13 +3,16 @@ package io.iohk.ethereum.jsonrpc import akka.actor.ActorSystem import akka.testkit.TestProbe import akka.util.ByteString -import io.iohk.ethereum.{Fixtures, Timeouts} +import io.iohk.ethereum.{Fixtures, ObjectGenerators, Timeouts} import io.iohk.ethereum.blockchain.sync.EphemBlockchainTestSetup +import io.iohk.ethereum.checkpointing.CheckpointingTestHelpers +import io.iohk.ethereum.consensus.blocks.CheckpointBlockGenerator import io.iohk.ethereum.consensus.{ConsensusConfigs, TestConsensus} import io.iohk.ethereum.consensus.ethash.blocks.EthashBlockGenerator import io.iohk.ethereum.consensus.ethash.validators.ValidatorsExecutor import io.iohk.ethereum.crypto.ECDSASignature import io.iohk.ethereum.db.storage.AppStateStorage +import io.iohk.ethereum.domain.BlockHeader.HeaderExtraFields.HefPostEcip1098 import io.iohk.ethereum.domain.{Block, BlockBody, SignedTransaction} import io.iohk.ethereum.jsonrpc.JsonRpcController.JsonRpcConfig import io.iohk.ethereum.keystore.KeyStore @@ -113,6 +116,12 @@ class JsonRpcControllerFixture(implicit system: ActorSystem) unixTimestamp = 0 ) + val checkpoint = ObjectGenerators.fakeCheckpointGen(2, 5).sample.get + val checkpointBlockGenerator = new CheckpointBlockGenerator() + val blockWithCheckpoint = checkpointBlockGenerator.generate(Fixtures.Blocks.Block3125369.block, checkpoint) + val blockWithTreasuryOptOut = + Block(Fixtures.Blocks.Block3125369.header.copy(extraFields = HefPostEcip1098(true)), Fixtures.Blocks.Block3125369.body) + val parentBlock = Block(blockHeader.copy(number = 1), BlockBody.empty) val r: ByteString = ByteString(Hex.decode("a3f20717a250c2b0b729b7e5becbff67fdaef7e0699da4de7ca5895b02a170a1")) diff --git a/src/test/scala/io/iohk/ethereum/ledger/BlockExecutionSpec.scala b/src/test/scala/io/iohk/ethereum/ledger/BlockExecutionSpec.scala index cb923f1c24..47ac0e3b96 100644 --- a/src/test/scala/io/iohk/ethereum/ledger/BlockExecutionSpec.scala +++ b/src/test/scala/io/iohk/ethereum/ledger/BlockExecutionSpec.scala @@ -1,17 +1,19 @@ package io.iohk.ethereum.ledger import akka.util.ByteString -import io.iohk.ethereum.Mocks -import io.iohk.ethereum.Mocks.{MockVM, MockValidatorsFailOnSpecificBlockNumber} +import io.iohk.ethereum.Mocks.{MockVM, MockValidatorsAlwaysSucceed, MockValidatorsFailOnSpecificBlockNumber} +import io.iohk.ethereum.checkpointing.CheckpointingTestHelpers import io.iohk.ethereum.consensus.TestConsensus +import io.iohk.ethereum.consensus.blocks.CheckpointBlockGenerator import io.iohk.ethereum.crypto.ECDSASignature import io.iohk.ethereum.domain._ import io.iohk.ethereum.ledger.Ledger.BlockResult import io.iohk.ethereum.vm.OutOfGas -import org.scalatest.prop.TableFor4 -import org.scalatestplus.scalacheck.ScalaCheckPropertyChecks +import io.iohk.ethereum.{Mocks, ObjectGenerators} import org.scalatest.matchers.should.Matchers +import org.scalatest.prop.TableFor4 import org.scalatest.wordspec.AnyWordSpec +import org.scalatestplus.scalacheck.ScalaCheckPropertyChecks // scalastyle:off magic.number class BlockExecutionSpec extends AnyWordSpec with Matchers with ScalaCheckPropertyChecks { @@ -48,7 +50,7 @@ class BlockExecutionSpec extends AnyWordSpec with Matchers with ScalaCheckProper val blockExecution = new BlockExecution(blockchain, blockchainConfig, newConsensus.blockPreparator, blockValidation) - val (blocks, error) = blockExecution.executeBlocks(List(block1, block2), defaultBlockHeader.difficulty) + val (blocks, error) = blockExecution.executeAndValidateBlocks(List(block1, block2), defaultBlockHeader.difficulty) // No block should be executed if first one has invalid transactions blocks.isEmpty shouldBe true @@ -82,13 +84,35 @@ class BlockExecutionSpec extends AnyWordSpec with Matchers with ScalaCheckProper val blockExecution = new BlockExecution(blockchain, blockchainConfig, newConsensus.blockPreparator, blockValidation) - val (blocks, error) = blockExecution.executeBlocks(List(block1, block2), defaultBlockHeader.difficulty) + val (blocks, error) = blockExecution.executeAndValidateBlocks(List(block1, block2), defaultBlockHeader.difficulty) // Only first block should be executed blocks.size shouldBe 1 blocks.head.block shouldBe block1 error.isDefined shouldBe true } + + "block with checkpoint and without txs" in new BlockchainSetup { + val checkpoint = ObjectGenerators.fakeCheckpointGen(2, 5).sample.get + val blockWithCheckpoint = new CheckpointBlockGenerator().generate(Block(validBlockParentHeader, validBlockBodyWithNoTxs), checkpoint) + val treasuryAccountBefore = blockchain.getAccount(blockchainConfig.treasuryAddress, blockWithCheckpoint.number) + + val mockValidators = MockValidatorsAlwaysSucceed + val newConsensus: TestConsensus = consensus.withVM(vm).withValidators(mockValidators) + val blockValidation = new BlockValidation(newConsensus, blockchain, BlockQueue(blockchain, syncConfig)) + val blockExecution = + new BlockExecution(blockchain, blockchainConfig, newConsensus.blockPreparator, blockValidation) + + val (blocks, error) = blockExecution.executeAndValidateBlocks(List(blockWithCheckpoint), defaultBlockHeader.difficulty) + val beneficiaryAccount = blockchain.getAccount(Address(blockWithCheckpoint.header.beneficiary), blockWithCheckpoint.number) + val treasuryAccountAfter = blockchain.getAccount(blockchainConfig.treasuryAddress, blockWithCheckpoint.number) + + beneficiaryAccount.isDefined shouldBe false + treasuryAccountBefore shouldBe treasuryAccountAfter + blocks.size shouldBe 1 + blocks.head.block shouldBe blockWithCheckpoint + error.isDefined shouldBe false + } } "correctly run executeBlockTransactions" when { diff --git a/src/test/scala/io/iohk/ethereum/ledger/BlockImportSpec.scala b/src/test/scala/io/iohk/ethereum/ledger/BlockImportSpec.scala index 4eabf10923..4f29449312 100644 --- a/src/test/scala/io/iohk/ethereum/ledger/BlockImportSpec.scala +++ b/src/test/scala/io/iohk/ethereum/ledger/BlockImportSpec.scala @@ -112,7 +112,7 @@ class BlockImportSpec extends AnyFlatSpec with Matchers with ScalaFutures { val blockData2 = BlockData(newBlock2, Seq.empty[Receipt], newTd2) val blockData3 = BlockData(newBlock3, Seq.empty[Receipt], newTd3) - (ledgerWithMockedBlockExecution.blockExecution.executeBlocks _) + (ledgerWithMockedBlockExecution.blockExecution.executeAndValidateBlocks _) .expects(newBranch, *) .returning((List(blockData2, blockData3), None)) @@ -157,7 +157,7 @@ class BlockImportSpec extends AnyFlatSpec with Matchers with ScalaFutures { val blockData2 = BlockData(newBlock2, Seq.empty[Receipt], newTd2) val blockData3 = BlockData(newBlock3, Seq.empty[Receipt], newTd3) - (ledgerWithMockedBlockExecution.blockExecution.executeBlocks _) + (ledgerWithMockedBlockExecution.blockExecution.executeAndValidateBlocks _) .expects(newBranch, *) .returning((List(blockData2), Some(execError))) @@ -260,7 +260,7 @@ class BlockImportSpec extends AnyFlatSpec with Matchers with ScalaFutures { val blockData2 = BlockData(newBlock2, Seq.empty[Receipt], newTd2) val blockData3 = BlockData(newBlock3WithOmmer, Seq.empty[Receipt], newTd3) - (ledgerWithMockedBlockExecution.blockExecution.executeBlocks _) + (ledgerWithMockedBlockExecution.blockExecution.executeAndValidateBlocks _) .expects(newBranch, *) .returning((List(blockData2, blockData3), None)) @@ -288,7 +288,7 @@ class BlockImportSpec extends AnyFlatSpec with Matchers with ScalaFutures { blockchain.save(parentBlock, Nil, tdParent, saveAsBestBlock = true) blockchain.save(regularBlock, Nil, tdRegular, saveAsBestBlock = true) - (ledgerWithMockedBlockExecution.blockExecution.executeBlocks _) + (ledgerWithMockedBlockExecution.blockExecution.executeAndValidateBlocks _) .expects(List(checkpointBlock), *) .returning((List(BlockData(checkpointBlock, Nil, tdCheckpoint)), None)) diff --git a/src/test/scala/io/iohk/ethereum/ledger/LedgerSpec.scala b/src/test/scala/io/iohk/ethereum/ledger/LedgerSpec.scala index 613f1dee91..e78af425de 100644 --- a/src/test/scala/io/iohk/ethereum/ledger/LedgerSpec.scala +++ b/src/test/scala/io/iohk/ethereum/ledger/LedgerSpec.scala @@ -74,7 +74,7 @@ class LedgerSpec extends AnyFlatSpec with ScalaCheckPropertyChecks with Matchers ) val block = Block(blockHeader, blockBodyWithOmmers) - val blockExecResult = ledger.blockExecution.executeBlock(block) + val blockExecResult = ledger.blockExecution.executeAndValidateBlock(block) assert(blockExecResult.isRight) } } @@ -121,7 +121,7 @@ class LedgerSpec extends AnyFlatSpec with ScalaCheckPropertyChecks with Matchers assert(seqFailingValidators.forall { validators => val ledger = newTestLedger(validators = validators) - val blockExecResult = ledger.blockExecution.executeBlock(block) + val blockExecResult = ledger.blockExecution.executeAndValidateBlock(block) blockExecResult.left.forall { case e: ValidationBeforeExecError => true @@ -172,7 +172,7 @@ class LedgerSpec extends AnyFlatSpec with ScalaCheckPropertyChecks with Matchers val blockHeader: BlockHeader = validBlockHeader.copy(gasUsed = cumulativeGasUsedBlock, stateRoot = stateRootHash) val block = Block(blockHeader, validBlockBodyWithNoTxs) - val blockExecResult = ledger.blockExecution.executeBlock(block) + val blockExecResult = ledger.blockExecution.executeAndValidateBlock(block) assert(blockExecResult match { case Left(_: ValidationAfterExecError) => true @@ -271,7 +271,7 @@ class LedgerSpec extends AnyFlatSpec with ScalaCheckPropertyChecks with Matchers val blockWithCorrectStateAndGasUsed = block.copy( header = block.header.copy(stateRoot = blockExpectedStateRoot, gasUsed = gasUsedReceipt2) ) - assert(ledger.blockExecution.executeBlock(blockWithCorrectStateAndGasUsed).isRight) + assert(ledger.blockExecution.executeAndValidateBlock(blockWithCorrectStateAndGasUsed).isRight) } }