From 4bc917c41b7782774f893d88eec7c1696daf58dd Mon Sep 17 00:00:00 2001 From: Leonor Boga Date: Tue, 22 Jun 2021 16:59:11 +0200 Subject: [PATCH 1/6] ETCM-944 - Refactoring - extract methods getWorldStateProxy and getReadOnlyWorldStateProxy from Blockchain --- .../ethereum/ledger/BlockImporterItSpec.scala | 25 +++--- .../ethereum/sync/util/CommonFakePeer.scala | 12 +-- .../sync/util/RegularSyncItSpecUtils.scala | 35 ++++++--- .../ethereum/txExecTest/ContractTest.scala | 50 ++++-------- .../ethereum/txExecTest/ECIP1017Test.scala | 8 +- .../iohk/ethereum/txExecTest/ForksTest.scala | 8 +- .../txExecTest/util/DumpChainApp.scala | 22 ++---- .../txExecTest/util/FixtureProvider.scala | 2 +- .../ethereum/consensus/ConsensusBuilder.scala | 11 ++- .../blocks/BlockGeneratorSkeleton.scala | 6 +- .../blocks/NoOmmersBlockGenerator.scala | 15 +++- .../ethereum/consensus/pow/PoWConsensus.scala | 19 +++-- .../pow/blocks/PoWBlockGenerator.scala | 5 +- .../RestrictedPoWBlockGeneratorImpl.scala | 4 + .../io/iohk/ethereum/domain/Blockchain.scala | 68 ++++------------ .../ethereum/jsonrpc/EthUserService.scala | 10 ++- .../iohk/ethereum/ledger/BlockExecution.scala | 28 +++---- .../ethereum/ledger/BlockPreparator.scala | 12 ++- .../ledger/InMemoryWorldStateProxy.scala | 18 +++++ .../io/iohk/ethereum/ledger/Ledger.scala | 14 +++- .../io/iohk/ethereum/ledger/StxLedger.scala | 14 +++- .../ethereum/nodebuilder/NodeBuilder.scala | 16 +++- .../testmode/TestBlockchainBuilder.scala | 20 +---- .../testmode/TestModeComponentsProvider.scala | 6 +- .../testmode/TestModeServiceBuilder.scala | 2 + .../ethereum/testmode/TestmodeConsensus.scala | 11 ++- .../blockchain/sync/ScenarioSetup.scala | 3 + .../blockchain/sync/StateSyncUtils.scala | 19 +++-- .../sync/regular/RegularSyncFixtures.scala | 9 ++- .../consensus/blocks/BlockGeneratorSpec.scala | 24 +++++- .../consensus/pow/MinerSpecSetup.scala | 4 +- .../consensus/pow/PoWConsensusSpec.scala | 8 ++ .../jsonrpc/EthBlocksServiceSpec.scala | 8 +- .../ethereum/jsonrpc/EthInfoServiceSpec.scala | 21 ++--- .../jsonrpc/EthMiningServiceSpec.scala | 9 ++- .../jsonrpc/EthProofServiceSpec.scala | 1 + .../ethereum/jsonrpc/EthUserServiceSpec.scala | 1 + .../jsonrpc/JsonRpcControllerFixture.scala | 14 ++-- .../ethereum/ledger/BlockExecutionSpec.scala | 72 +++++++++++++---- .../ethereum/ledger/BlockImportSpec.scala | 60 +++++++++------ .../ethereum/ledger/BlockRewardSpec.scala | 17 ++-- .../ethereum/ledger/DeleteAccountsSpec.scala | 24 +++--- .../ledger/DeleteTouchedAccountsSpec.scala | 73 ++++++++---------- .../ledger/InMemoryWorldStateProxySpec.scala | 63 +++++++++------ .../io/iohk/ethereum/ledger/LedgerSpec.scala | 6 +- .../ethereum/ledger/LedgerTestSetup.scala | 77 ++++++++++++++----- .../iohk/ethereum/ledger/StxLedgerSpec.scala | 16 +++- 47 files changed, 585 insertions(+), 385 deletions(-) diff --git a/src/it/scala/io/iohk/ethereum/ledger/BlockImporterItSpec.scala b/src/it/scala/io/iohk/ethereum/ledger/BlockImporterItSpec.scala index 28204de3fb..f0fba2cb7e 100644 --- a/src/it/scala/io/iohk/ethereum/ledger/BlockImporterItSpec.scala +++ b/src/it/scala/io/iohk/ethereum/ledger/BlockImporterItSpec.scala @@ -59,14 +59,15 @@ class BlockImporterItSpec val pendingTransactionsManagerProbe = TestProbe() val supervisor = TestProbe() - val emptyWorld: InMemoryWorldStateProxy = - blockchain.getWorldStateProxy( - -1, - UInt256.Zero, - ByteString(MerklePatriciaTrie.EmptyRootHash), - noEmptyAccounts = false, - ethCompatibleStorage = true - ) + val emptyWorld = InMemoryWorldStateProxy( + storagesInstance.storages.evmCodeStorage, + blockchain.getBackingStorage(-1), + blockchain, + blockchainConfig.accountStartNonce, + ByteString(MerklePatriciaTrie.EmptyRootHash), + noEmptyAccounts = false, + ethCompatibleStorage = true + ) override protected lazy val successValidators: Validators = new Mocks.MockValidatorsAlwaysSucceed { override val ommersValidator: OmmersValidator = ( @@ -82,7 +83,13 @@ class BlockImporterItSpec override lazy val ledger = new TestLedgerImpl(successValidators) { override private[ledger] lazy val blockExecution = - new BlockExecution(blockchain, blockchainConfig, consensus.blockPreparator, blockValidation) { + new BlockExecution( + blockchain, + storagesInstance.storages.evmCodeStorage, + blockchainConfig, + consensus.blockPreparator, + blockValidation + ) { override def executeAndValidateBlock( block: Block, alreadyValidated: Boolean = false diff --git a/src/it/scala/io/iohk/ethereum/sync/util/CommonFakePeer.scala b/src/it/scala/io/iohk/ethereum/sync/util/CommonFakePeer.scala index c7a9f08ac0..ff9eb57650 100644 --- a/src/it/scala/io/iohk/ethereum/sync/util/CommonFakePeer.scala +++ b/src/it/scala/io/iohk/ethereum/sync/util/CommonFakePeer.scala @@ -14,7 +14,7 @@ import io.iohk.ethereum.db.components.{RocksDbDataSourceComponent, Storages} import io.iohk.ethereum.db.dataSource.{RocksDbConfig, RocksDbDataSource} import io.iohk.ethereum.db.storage.pruning.{ArchivePruning, PruningMode} import io.iohk.ethereum.db.storage.{AppStateStorage, Namespaces} -import io.iohk.ethereum.domain.{Block, Blockchain, BlockchainImpl, ChainWeight} +import io.iohk.ethereum.domain.{Block, Blockchain, BlockchainImpl, ChainWeight, UInt256} import io.iohk.ethereum.security.SecureRandomBuilder import io.iohk.ethereum.ledger.InMemoryWorldStateProxy import io.iohk.ethereum.mpt.MerklePatriciaTrie @@ -241,10 +241,12 @@ abstract class CommonFakePeer(peerName: String, fakePeerCustomConfig: FakePeerCu ) private def getMptForBlock(block: Block) = { - bl.getWorldStateProxy( - blockNumber = block.number, - accountStartNonce = blockchainConfig.accountStartNonce, - stateRootHash = block.header.stateRoot, + InMemoryWorldStateProxy( + storagesInstance.storages.evmCodeStorage, + bl.getBackingStorage(block.number), + bl, + blockchainConfig.accountStartNonce, + block.header.stateRoot, noEmptyAccounts = EvmConfig.forBlock(block.number, blockchainConfig).noEmptyAccounts, ethCompatibleStorage = blockchainConfig.ethCompatibleStorage ) 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 a7f8bc11d0..13501b9f87 100644 --- a/src/it/scala/io/iohk/ethereum/sync/util/RegularSyncItSpecUtils.scala +++ b/src/it/scala/io/iohk/ethereum/sync/util/RegularSyncItSpecUtils.scala @@ -32,11 +32,11 @@ import io.iohk.ethereum.sync.util.SyncCommonItSpecUtils.FakePeerCustomConfig.def import io.iohk.ethereum.sync.util.SyncCommonItSpecUtils._ import io.iohk.ethereum.transactions.PendingTransactionsManager import io.iohk.ethereum.utils._ -import io.iohk.ethereum.vm.EvmConfig import monix.eval.Task import monix.execution.Scheduler import akka.actor.typed.scaladsl.adapter._ import io.iohk.ethereum.blockchain.sync.regular.BlockFetcher.AdaptedMessageFromEventBus +import io.iohk.ethereum.mpt.MerklePatriciaTrie import scala.concurrent.duration._ object RegularSyncItSpecUtils { @@ -61,7 +61,15 @@ object RegularSyncItSpecUtils { val fullConfig = FullConsensusConfig(consensusConfig, specificConfig) val vm = VmSetup.vm(VmConfig(config), blockchainConfig, testMode = false) val consensus = - PoWConsensus(vm, bl, blockchainConfig, fullConfig, ValidatorsExecutorAlwaysSucceed, NoAdditionalPoWData) + PoWConsensus( + vm, + storagesInstance.storages.evmCodeStorage, + bl, + blockchainConfig, + fullConfig, + ValidatorsExecutorAlwaysSucceed, + NoAdditionalPoWData + ) consensus } @@ -73,7 +81,14 @@ object RegularSyncItSpecUtils { ) lazy val ledger: Ledger = - new LedgerImpl(bl, blockchainConfig, syncConfig, buildEthashConsensus(), Scheduler.global) + new LedgerImpl( + bl, + storagesInstance.storages.evmCodeStorage, + blockchainConfig, + syncConfig, + buildEthashConsensus(), + Scheduler.global + ) lazy val ommersPool: ActorRef = system.actorOf(OmmersPool.props(bl, 1), "ommers-pool") @@ -197,12 +212,14 @@ object RegularSyncItSpecUtils { } private def getMptForBlock(block: Block) = { - bl.getWorldStateProxy( - blockNumber = block.number, - accountStartNonce = blockchainConfig.accountStartNonce, - stateRootHash = block.header.stateRoot, - noEmptyAccounts = EvmConfig.forBlock(block.number, blockchainConfig).noEmptyAccounts, - ethCompatibleStorage = blockchainConfig.ethCompatibleStorage + InMemoryWorldStateProxy( + storagesInstance.storages.evmCodeStorage, + bl.getBackingStorage(block.number), + bl, + UInt256.Zero, + ByteString(MerklePatriciaTrie.EmptyRootHash), + noEmptyAccounts = false, + ethCompatibleStorage = true ) } diff --git a/src/it/scala/io/iohk/ethereum/txExecTest/ContractTest.scala b/src/it/scala/io/iohk/ethereum/txExecTest/ContractTest.scala index af70413bc6..66c7eb1d2a 100644 --- a/src/it/scala/io/iohk/ethereum/txExecTest/ContractTest.scala +++ b/src/it/scala/io/iohk/ethereum/txExecTest/ContractTest.scala @@ -1,8 +1,7 @@ package io.iohk.ethereum.txExecTest import java.util.concurrent.Executors - -import io.iohk.ethereum.domain.Receipt +import io.iohk.ethereum.domain.{BlockchainImpl, BlockchainStorages, Receipt} import io.iohk.ethereum.ledger.{BlockExecution, BlockQueue, BlockValidation, Ledger} import io.iohk.ethereum.txExecTest.util.FixtureProvider import io.iohk.ethereum.utils.Config @@ -18,47 +17,28 @@ class ContractTest extends AnyFlatSpec with Matchers { val ec = ExecutionContext.fromExecutor(Executors.newFixedThreadPool(4)) val noErrors = a[Right[_, Seq[Receipt]]] - "Ledger" should "transfer ether" in new ScenarioSetup { + "Ledger" should "execute and validate" in new ScenarioSetup { val fixtures: FixtureProvider.Fixture = FixtureProvider.loadFixtures("/txExecTest/purchaseContract") - - val testBlockchainStorages = FixtureProvider.prepareStorages(0, fixtures) + val testBlockchainStorages = FixtureProvider.prepareStorages(2, fixtures) //block only with ether transfers val blockValidation = new BlockValidation(consensus, blockchain, BlockQueue(blockchain, syncConfig)) - val blockExecution = new BlockExecution(blockchain, blockchainConfig, consensus.blockPreparator, blockValidation) + val blockExecution = new BlockExecution( + blockchain, + testBlockchainStorages.evmCodeStorage, + blockchainConfig, + consensus.blockPreparator, + blockValidation + ) + + // transfer ether blockExecution.executeAndValidateBlock(fixtures.blockByNumber(1)) shouldBe noErrors - } - - it should "deploy contract" in new ScenarioSetup { - val fixtures: FixtureProvider.Fixture = FixtureProvider.loadFixtures("/txExecTest/purchaseContract") - val testBlockchainStorages = FixtureProvider.prepareStorages(1, fixtures) - - //contract creation - val blockValidation = new BlockValidation(consensus, blockchain, BlockQueue(blockchain, syncConfig)) - val blockExecution = new BlockExecution(blockchain, blockchainConfig, consensus.blockPreparator, blockValidation) + // deploy contract blockExecution.executeAndValidateBlock(fixtures.blockByNumber(2)) shouldBe noErrors - } - - it should "execute contract call" in new ScenarioSetup { - val fixtures: FixtureProvider.Fixture = FixtureProvider.loadFixtures("/txExecTest/purchaseContract") - val testBlockchainStorages = FixtureProvider.prepareStorages(2, fixtures) - - //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.executeAndValidateBlock(fixtures.blockByNumber(3)) shouldBe noErrors - } - - it should "execute contract that pays 2 accounts" in new ScenarioSetup { - val fixtures: FixtureProvider.Fixture = FixtureProvider.loadFixtures("/txExecTest/purchaseContract") - - val testBlockchainStorages = FixtureProvider.prepareStorages(2, fixtures) - - //block contains contract paying 2 accounts - val blockValidation = new BlockValidation(consensus, blockchain, BlockQueue(blockchain, syncConfig)) - val blockExecution = new BlockExecution(blockchain, blockchainConfig, consensus.blockPreparator, blockValidation) + // execute contract call + // execute contract that pays 2 accounts 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 b0dfebf4f4..ee7df37759 100644 --- a/src/it/scala/io/iohk/ethereum/txExecTest/ECIP1017Test.scala +++ b/src/it/scala/io/iohk/ethereum/txExecTest/ECIP1017Test.scala @@ -78,7 +78,13 @@ class ECIP1017Test extends AnyFlatSpec with Matchers { val storages = FixtureProvider.prepareStorages(blockToExecute - 1, fixtures) val blockchain = BlockchainImpl(storages) val blockValidation = new BlockValidation(consensus, blockchain, BlockQueue(blockchain, syncConfig)) - val blockExecution = new BlockExecution(blockchain, blockchainConfig, consensus.blockPreparator, blockValidation) + val blockExecution = new BlockExecution( + blockchain, + testBlockchainStorages.evmCodeStorage, + blockchainConfig, + consensus.blockPreparator, + blockValidation + ) 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 d41958b548..fb3cfd0979 100644 --- a/src/it/scala/io/iohk/ethereum/txExecTest/ForksTest.scala +++ b/src/it/scala/io/iohk/ethereum/txExecTest/ForksTest.scala @@ -69,7 +69,13 @@ class ForksTest extends AnyFlatSpec with Matchers { val storages = FixtureProvider.prepareStorages(blockToExecute - 1, fixtures) val blockchain = BlockchainImpl(storages) val blockValidation = new BlockValidation(consensus, blockchain, BlockQueue(blockchain, syncConfig)) - val blockExecution = new BlockExecution(blockchain, blockchainConfig, consensus.blockPreparator, blockValidation) + val blockExecution = new BlockExecution( + blockchain, + testBlockchainStorages.evmCodeStorage, + blockchainConfig, + consensus.blockPreparator, + blockValidation + ) blockExecution.executeAndValidateBlock(fixtures.blockByNumber(blockToExecute)) shouldBe noErrors } } diff --git a/src/it/scala/io/iohk/ethereum/txExecTest/util/DumpChainApp.scala b/src/it/scala/io/iohk/ethereum/txExecTest/util/DumpChainApp.scala index 1a23b177e5..128ea79ed5 100644 --- a/src/it/scala/io/iohk/ethereum/txExecTest/util/DumpChainApp.scala +++ b/src/it/scala/io/iohk/ethereum/txExecTest/util/DumpChainApp.scala @@ -12,7 +12,7 @@ import io.iohk.ethereum.db.dataSource.{DataSourceBatchUpdate, RocksDbDataSource} import io.iohk.ethereum.db.storage.NodeStorage.{NodeEncoded, NodeHash} import io.iohk.ethereum.db.storage.TransactionMappingStorage.TransactionLocation import io.iohk.ethereum.db.storage.pruning.{ArchivePruning, PruningMode} -import io.iohk.ethereum.db.storage.{AppStateStorage, StateStorage} +import io.iohk.ethereum.db.storage.{AppStateStorage, MptStorage, StateStorage} import io.iohk.ethereum.domain.BlockHeader.HeaderExtraFields.HefEmpty import io.iohk.ethereum.domain.{Blockchain, UInt256, _} import io.iohk.ethereum.jsonrpc.ProofService.{EmptyStorageValueProof, StorageProof, StorageProofKey, StorageValueProof} @@ -190,22 +190,6 @@ class BlockchainMock(genesisHash: ByteString) extends Blockchain { override type S = InMemoryWorldStateProxyStorage override type WS = InMemoryWorldStateProxy - override def getWorldStateProxy( - blockNumber: BigInt, - accountStartNonce: UInt256, - stateRootHash: ByteString, - noEmptyAccounts: Boolean, - ethCompatibleStorage: Boolean - ): InMemoryWorldStateProxy = ??? - - override def getReadOnlyWorldStateProxy( - blockNumber: Option[BigInt], - accountStartNonce: UInt256, - stateRootHash: ByteString, - noEmptyAccounts: Boolean, - ethCompatibleStorage: Boolean - ): InMemoryWorldStateProxy = ??? - def getBestBlockNumber(): BigInt = ??? def saveBlockNumber(number: BigInt, hash: NodeHash): Unit = ??? @@ -217,4 +201,8 @@ class BlockchainMock(genesisHash: ByteString) extends Blockchain { override def save(block: Block, receipts: Seq[Receipt], weight: ChainWeight, saveAsBestBlock: Boolean): Unit = ??? override def getLatestCheckpointBlockNumber(): BigInt = ??? + + override def getBackingStorage(blockNumber: BigInt): MptStorage = ??? + + override def getReadOnlyStorage(): MptStorage = ??? } diff --git a/src/it/scala/io/iohk/ethereum/txExecTest/util/FixtureProvider.scala b/src/it/scala/io/iohk/ethereum/txExecTest/util/FixtureProvider.scala index ac7b85c1e7..ed14effedc 100644 --- a/src/it/scala/io/iohk/ethereum/txExecTest/util/FixtureProvider.scala +++ b/src/it/scala/io/iohk/ethereum/txExecTest/util/FixtureProvider.scala @@ -65,13 +65,13 @@ object FixtureProvider { val blocksToInclude = fixtures.blockByNumber.toSeq.sortBy { case (number, _) => number }.takeWhile { case (number, _) => number <= blockNumber } - val blockchain = BlockchainImpl(storages) blocksToInclude.foreach { case (_, block) => val receiptsUpdates = fixtures.receipts .get(block.header.hash) .map(r => storages.receiptStorage.put(block.header.hash, r)) .getOrElse(storages.receiptStorage.emptyBatchUpdate) + storages.blockBodiesStorage .put(block.header.hash, fixtures.blockBodies(block.header.hash)) .and(storages.blockHeadersStorage.put(block.header.hash, fixtures.blockHeaders(block.header.hash))) diff --git a/src/main/scala/io/iohk/ethereum/consensus/ConsensusBuilder.scala b/src/main/scala/io/iohk/ethereum/consensus/ConsensusBuilder.scala index 8aff608503..6b13d80f5b 100644 --- a/src/main/scala/io/iohk/ethereum/consensus/ConsensusBuilder.scala +++ b/src/main/scala/io/iohk/ethereum/consensus/ConsensusBuilder.scala @@ -19,6 +19,7 @@ trait ConsensusBuilder { */ trait StdConsensusBuilder extends ConsensusBuilder { self: VmBuilder + with StorageBuilder with BlockchainBuilder with BlockchainConfigBuilder with ConsensusConfigBuilder @@ -43,7 +44,15 @@ trait StdConsensusBuilder extends ConsensusBuilder { case Protocol.PoW | Protocol.MockedPow => NoAdditionalPoWData case Protocol.RestrictedPoW => RestrictedPoWMinerData(nodeKey) } - val consensus = PoWConsensus(vm, blockchain, blockchainConfig, fullConfig, validators, additionalPoWData) + val consensus = PoWConsensus( + vm, + storagesInstance.storages.evmCodeStorage, + blockchain, + blockchainConfig, + fullConfig, + validators, + additionalPoWData + ) consensus } diff --git a/src/main/scala/io/iohk/ethereum/consensus/blocks/BlockGeneratorSkeleton.scala b/src/main/scala/io/iohk/ethereum/consensus/blocks/BlockGeneratorSkeleton.scala index f296f1636f..570cbf9c77 100644 --- a/src/main/scala/io/iohk/ethereum/consensus/blocks/BlockGeneratorSkeleton.scala +++ b/src/main/scala/io/iohk/ethereum/consensus/blocks/BlockGeneratorSkeleton.scala @@ -8,7 +8,7 @@ import io.iohk.ethereum.consensus.pow.blocks.Ommers import io.iohk.ethereum.consensus.validators.std.MptListValidator.intByteArraySerializable import io.iohk.ethereum.crypto.kec256 import io.iohk.ethereum.db.dataSource.EphemDataSource -import io.iohk.ethereum.db.storage.StateStorage +import io.iohk.ethereum.db.storage.{EvmCodeStorage, StateStorage} import io.iohk.ethereum.domain._ import io.iohk.ethereum.domain.BlockHeader.HeaderExtraFields._ import io.iohk.ethereum.consensus.pow.blocks.OmmersSeqEnc @@ -22,7 +22,6 @@ import io.iohk.ethereum.utils.ByteUtils.or * This is a skeleton for a generic [[io.iohk.ethereum.consensus.blocks.BlockGenerator BlockGenerator]]. */ abstract class BlockGeneratorSkeleton( - blockchain: Blockchain, blockchainConfig: BlockchainConfig, consensusConfig: ConsensusConfig, difficultyCalc: DifficultyCalculator, @@ -82,6 +81,7 @@ abstract class BlockGeneratorSkeleton( ): BlockHeader protected def prepareBlock( + evmCodeStorage: EvmCodeStorage, parent: Block, transactions: Seq[SignedTransaction], beneficiary: Address, @@ -97,7 +97,7 @@ abstract class BlockGeneratorSkeleton( val body = newBlockBody(transactionsForBlock, x) val block = Block(header, body) - blockPreparator.prepareBlock(block, parent.header, initialWorldStateBeforeExecution) match { + blockPreparator.prepareBlock(evmCodeStorage, block, parent.header, initialWorldStateBeforeExecution) match { case PreparedBlock(prepareBlock, BlockResult(_, gasUsed, receipts), stateRoot, updatedWorld) => val receiptsLogs: Seq[Array[Byte]] = BloomFilter.EmptyBloomFilter.toArray +: receipts.map(_.logsBloomFilter.toArray) diff --git a/src/main/scala/io/iohk/ethereum/consensus/blocks/NoOmmersBlockGenerator.scala b/src/main/scala/io/iohk/ethereum/consensus/blocks/NoOmmersBlockGenerator.scala index 13a36f71ee..241f32dde0 100644 --- a/src/main/scala/io/iohk/ethereum/consensus/blocks/NoOmmersBlockGenerator.scala +++ b/src/main/scala/io/iohk/ethereum/consensus/blocks/NoOmmersBlockGenerator.scala @@ -6,16 +6,16 @@ import io.iohk.ethereum.domain._ import io.iohk.ethereum.ledger.{BlockPreparator, InMemoryWorldStateProxy} import io.iohk.ethereum.utils.BlockchainConfig import io.iohk.ethereum.consensus.ConsensusMetrics +import io.iohk.ethereum.db.storage.EvmCodeStorage abstract class NoOmmersBlockGenerator( - blockchain: Blockchain, + evmCodeStorage: EvmCodeStorage, blockchainConfig: BlockchainConfig, consensusConfig: ConsensusConfig, blockPreparator: BlockPreparator, difficultyCalc: DifficultyCalculator, blockTimestampProvider: BlockTimestampProvider = DefaultBlockTimestampProvider ) extends BlockGeneratorSkeleton( - blockchain, blockchainConfig, consensusConfig, difficultyCalc, @@ -50,7 +50,16 @@ abstract class NoOmmersBlockGenerator( val blockNumber = pHeader.number + 1 val prepared = - prepareBlock(parent, transactions, beneficiary, blockNumber, blockPreparator, x, initialWorldStateBeforeExecution) + prepareBlock( + evmCodeStorage, + parent, + transactions, + beneficiary, + blockNumber, + blockPreparator, + x, + initialWorldStateBeforeExecution + ) cache.updateAndGet((t: List[PendingBlockAndState]) => (prepared :: t).take(blockCacheSize)) prepared diff --git a/src/main/scala/io/iohk/ethereum/consensus/pow/PoWConsensus.scala b/src/main/scala/io/iohk/ethereum/consensus/pow/PoWConsensus.scala index 2312154165..91dcfbefe5 100644 --- a/src/main/scala/io/iohk/ethereum/consensus/pow/PoWConsensus.scala +++ b/src/main/scala/io/iohk/ethereum/consensus/pow/PoWConsensus.scala @@ -15,7 +15,8 @@ import io.iohk.ethereum.consensus.pow.miners.MockedMiner.MockedMinerResponses.Mi import io.iohk.ethereum.consensus.pow.miners.{MinerProtocol, MockedMiner} import io.iohk.ethereum.consensus.pow.validators.ValidatorsExecutor import io.iohk.ethereum.consensus.validators.Validators -import io.iohk.ethereum.domain.BlockchainImpl +import io.iohk.ethereum.db.storage.EvmCodeStorage +import io.iohk.ethereum.domain.{Blockchain, BlockchainImpl} import io.iohk.ethereum.jsonrpc.AkkaTaskOps.TaskActorOps import io.iohk.ethereum.ledger.BlockPreparator import io.iohk.ethereum.ledger.Ledger.VMImpl @@ -30,6 +31,7 @@ import scala.concurrent.duration._ */ class PoWConsensus private ( val vm: VMImpl, + evmCodeStorage: EvmCodeStorage, blockchain: BlockchainImpl, val blockchainConfig: BlockchainConfig, val config: FullConsensusConfig[EthashConfig], @@ -160,6 +162,7 @@ class PoWConsensus private ( ) new PoWBlockGeneratorImpl( + evmCodeStorage = evmCodeStorage, validators = _validators, blockchain = blockchain, blockchainConfig = blockchainConfig, @@ -182,6 +185,7 @@ class PoWConsensus private ( new PoWConsensus( vm = vm, + evmCodeStorage = evmCodeStorage, blockchain = blockchain, blockchainConfig = blockchainConfig, config = config, @@ -198,6 +202,7 @@ class PoWConsensus private ( def withVM(vm: VMImpl): PoWConsensus = new PoWConsensus( vm = vm, + evmCodeStorage = evmCodeStorage, blockchain = blockchain, blockchainConfig = blockchainConfig, config = config, @@ -209,13 +214,14 @@ class PoWConsensus private ( /** Internal API, used for testing */ def withBlockGenerator(blockGenerator: TestBlockGenerator): PoWConsensus = new PoWConsensus( + evmCodeStorage = evmCodeStorage, vm = vm, blockchain = blockchain, blockchainConfig = blockchainConfig, config = config, validators = validators, blockGenerator = blockGenerator.asInstanceOf[PoWBlockGenerator], - difficultyCalculator + difficultyCalculator = difficultyCalculator ) } @@ -223,25 +229,24 @@ class PoWConsensus private ( object PoWConsensus { def apply( vm: VMImpl, + evmCodeStorage: EvmCodeStorage, blockchain: BlockchainImpl, blockchainConfig: BlockchainConfig, config: FullConsensusConfig[EthashConfig], validators: ValidatorsExecutor, additionalEthashProtocolData: AdditionalPoWProtocolData ): PoWConsensus = { - val difficultyCalculator = DifficultyCalculator(blockchainConfig) - val blockPreparator = new BlockPreparator( vm = vm, signedTxValidator = validators.signedTransactionValidator, blockchain = blockchain, blockchainConfig = blockchainConfig ) - val blockGenerator = additionalEthashProtocolData match { case RestrictedPoWMinerData(key) => new RestrictedPoWBlockGeneratorImpl( + evmCodeStorage = evmCodeStorage, validators = validators, blockchain = blockchain, blockchainConfig = blockchainConfig, @@ -250,9 +255,9 @@ object PoWConsensus { difficultyCalc = difficultyCalculator, minerKeyPair = key ) - case NoAdditionalPoWData => new PoWBlockGeneratorImpl( + evmCodeStorage = evmCodeStorage, validators = validators, blockchain = blockchain, blockchainConfig = blockchainConfig, @@ -261,9 +266,9 @@ object PoWConsensus { difficultyCalc = difficultyCalculator ) } - new PoWConsensus( vm = vm, + evmCodeStorage = evmCodeStorage, blockchain = blockchain, blockchainConfig = blockchainConfig, config = config, diff --git a/src/main/scala/io/iohk/ethereum/consensus/pow/blocks/PoWBlockGenerator.scala b/src/main/scala/io/iohk/ethereum/consensus/pow/blocks/PoWBlockGenerator.scala index 130abfbf26..7a7ef4147c 100644 --- a/src/main/scala/io/iohk/ethereum/consensus/pow/blocks/PoWBlockGenerator.scala +++ b/src/main/scala/io/iohk/ethereum/consensus/pow/blocks/PoWBlockGenerator.scala @@ -11,6 +11,7 @@ import io.iohk.ethereum.domain._ import io.iohk.ethereum.ledger.{BlockPreparator, InMemoryWorldStateProxy} import io.iohk.ethereum.utils.BlockchainConfig import io.iohk.ethereum.consensus.ConsensusMetrics +import io.iohk.ethereum.db.storage.EvmCodeStorage /** Internal API, used for testing (especially mocks) */ trait PoWBlockGenerator extends TestBlockGenerator { @@ -23,6 +24,7 @@ trait PoWBlockGenerator extends TestBlockGenerator { } class PoWBlockGeneratorImpl( + evmCodeStorage: EvmCodeStorage, validators: ValidatorsExecutor, blockchain: Blockchain, blockchainConfig: BlockchainConfig, @@ -31,7 +33,6 @@ class PoWBlockGeneratorImpl( difficultyCalc: DifficultyCalculator, blockTimestampProvider: BlockTimestampProvider = DefaultBlockTimestampProvider ) extends BlockGeneratorSkeleton( - blockchain, blockchainConfig, consensusConfig, difficultyCalc, @@ -86,6 +87,7 @@ class PoWBlockGeneratorImpl( } val prepared = prepareBlock( + evmCodeStorage, parent, transactions, beneficiary, @@ -104,6 +106,7 @@ class PoWBlockGeneratorImpl( def withBlockTimestampProvider(blockTimestampProvider: BlockTimestampProvider): PoWBlockGeneratorImpl = new PoWBlockGeneratorImpl( + evmCodeStorage, validators, blockchain, blockchainConfig, diff --git a/src/main/scala/io/iohk/ethereum/consensus/pow/blocks/RestrictedPoWBlockGeneratorImpl.scala b/src/main/scala/io/iohk/ethereum/consensus/pow/blocks/RestrictedPoWBlockGeneratorImpl.scala index 04635c2939..ee2ec0ed05 100644 --- a/src/main/scala/io/iohk/ethereum/consensus/pow/blocks/RestrictedPoWBlockGeneratorImpl.scala +++ b/src/main/scala/io/iohk/ethereum/consensus/pow/blocks/RestrictedPoWBlockGeneratorImpl.scala @@ -10,8 +10,10 @@ import io.iohk.ethereum.ledger.{BlockPreparator, InMemoryWorldStateProxy} import io.iohk.ethereum.utils.BlockchainConfig import org.bouncycastle.crypto.AsymmetricCipherKeyPair import io.iohk.ethereum.consensus.ConsensusMetrics +import io.iohk.ethereum.db.storage.EvmCodeStorage class RestrictedPoWBlockGeneratorImpl( + evmCodeStorage: EvmCodeStorage, validators: ValidatorsExecutor, blockchain: Blockchain, blockchainConfig: BlockchainConfig, @@ -21,6 +23,7 @@ class RestrictedPoWBlockGeneratorImpl( minerKeyPair: AsymmetricCipherKeyPair, blockTimestampProvider: BlockTimestampProvider = DefaultBlockTimestampProvider ) extends PoWBlockGeneratorImpl( + evmCodeStorage, validators, blockchain, blockchainConfig, @@ -46,6 +49,7 @@ class RestrictedPoWBlockGeneratorImpl( case Right(_) => ommers } val prepared = prepareBlock( + evmCodeStorage, parent, transactions, beneficiary, diff --git a/src/main/scala/io/iohk/ethereum/domain/Blockchain.scala b/src/main/scala/io/iohk/ethereum/domain/Blockchain.scala index 844986dabb..7de454d9be 100644 --- a/src/main/scala/io/iohk/ethereum/domain/Blockchain.scala +++ b/src/main/scala/io/iohk/ethereum/domain/Blockchain.scala @@ -107,6 +107,19 @@ trait Blockchain { ethCompatibleStorage: Boolean ): StorageProof + /** + * Get the MptStorage + * @param blockNumber + * @return MptStorage + */ + def getBackingStorage(blockNumber: BigInt): MptStorage + + /** Get the MptStorage for read-only + * + * @return MptStorage + */ + def getReadOnlyStorage(): MptStorage + /** * Returns the receipts based on a block hash * @param blockhash @@ -184,22 +197,6 @@ trait Blockchain { def genesisBlock: Block = getBlockByNumber(0).get - def getWorldStateProxy( - blockNumber: BigInt, - accountStartNonce: UInt256, - stateRootHash: ByteString, - noEmptyAccounts: Boolean, - ethCompatibleStorage: Boolean - ): WS - - def getReadOnlyWorldStateProxy( - blockNumber: Option[BigInt], - accountStartNonce: UInt256, - stateRootHash: ByteString, - noEmptyAccounts: Boolean, - ethCompatibleStorage: Boolean - ): WS - /** * Strict check if given block hash is in chain * Using any of getXXXByHash is not always accurate - after restart the best block is often lower than before restart @@ -325,6 +322,10 @@ class BlockchainImpl( StorageProof(position, value, proof) } + def getBackingStorage(blockNumber: BigInt): MptStorage = stateStorage.getBackingStorage(blockNumber) + + def getReadOnlyStorage(): MptStorage = stateStorage.getReadOnlyStorage + private def persistBestBlocksData(): Unit = { val currentBestBlockNumber = getBestBlockNumber() val currentBestCheckpointNumber = getLatestCheckpointBlockNumber() @@ -534,41 +535,6 @@ class BlockchainImpl( override type S = InMemoryWorldStateProxyStorage override type WS = InMemoryWorldStateProxy - - override def getWorldStateProxy( - blockNumber: BigInt, - accountStartNonce: UInt256, - stateRootHash: ByteString, - noEmptyAccounts: Boolean, - ethCompatibleStorage: Boolean - ): InMemoryWorldStateProxy = - InMemoryWorldStateProxy( - evmCodeStorage, - stateStorage.getBackingStorage(blockNumber), - accountStartNonce, - (number: BigInt) => getBlockHeaderByNumber(number).map(_.hash), - stateRootHash, - noEmptyAccounts, - ethCompatibleStorage - ) - - //FIXME Maybe we can use this one in regular execution too and persist underlying storage when block execution is successful - override def getReadOnlyWorldStateProxy( - blockNumber: Option[BigInt], - accountStartNonce: UInt256, - stateRootHash: ByteString, - noEmptyAccounts: Boolean, - ethCompatibleStorage: Boolean - ): InMemoryWorldStateProxy = - InMemoryWorldStateProxy( - evmCodeStorage, - stateStorage.getReadOnlyStorage, - accountStartNonce, - (number: BigInt) => getBlockHeaderByNumber(number).map(_.hash), - stateRootHash, - noEmptyAccounts = noEmptyAccounts, - ethCompatibleStorage = ethCompatibleStorage - ) } trait BlockchainStorages { diff --git a/src/main/scala/io/iohk/ethereum/jsonrpc/EthUserService.scala b/src/main/scala/io/iohk/ethereum/jsonrpc/EthUserService.scala index 151ecb05b8..6242154701 100644 --- a/src/main/scala/io/iohk/ethereum/jsonrpc/EthUserService.scala +++ b/src/main/scala/io/iohk/ethereum/jsonrpc/EthUserService.scala @@ -1,8 +1,9 @@ package io.iohk.ethereum.jsonrpc import akka.util.ByteString +import io.iohk.ethereum.db.storage.EvmCodeStorage import io.iohk.ethereum.domain._ -import io.iohk.ethereum.ledger.Ledger +import io.iohk.ethereum.ledger.{InMemoryWorldStateProxy, Ledger} import io.iohk.ethereum.mpt.MerklePatriciaTrie.MissingNodeException import io.iohk.ethereum.utils.BlockchainConfig import monix.eval.Task @@ -22,6 +23,7 @@ object EthUserService { class EthUserService( val blockchain: Blockchain, + evmCodeStorage: EvmCodeStorage, val ledger: Ledger, blockchainConfig: BlockchainConfig ) extends ResolveBlock { @@ -30,8 +32,10 @@ class EthUserService( def getCode(req: GetCodeRequest): ServiceResponse[GetCodeResponse] = { Task { resolveBlock(req.block).map { case ResolvedBlock(block, _) => - val world = blockchain.getWorldStateProxy( - block.header.number, + val world = InMemoryWorldStateProxy( + evmCodeStorage, + blockchain.getBackingStorage(block.header.number), + blockchain, blockchainConfig.accountStartNonce, block.header.stateRoot, noEmptyAccounts = false, diff --git a/src/main/scala/io/iohk/ethereum/ledger/BlockExecution.scala b/src/main/scala/io/iohk/ethereum/ledger/BlockExecution.scala index d89852b96d..31dc6504c5 100644 --- a/src/main/scala/io/iohk/ethereum/ledger/BlockExecution.scala +++ b/src/main/scala/io/iohk/ethereum/ledger/BlockExecution.scala @@ -3,15 +3,17 @@ package io.iohk.ethereum.ledger import io.iohk.ethereum.domain._ import io.iohk.ethereum.ledger.BlockExecutionError.MissingParentError import io.iohk.ethereum.ledger.Ledger.BlockResult -import io.iohk.ethereum.utils.{BlockchainConfig, DaoForkConfig, Logger} +import io.iohk.ethereum.utils.{BlockchainConfig, ByteStringUtils, DaoForkConfig, Logger} import io.iohk.ethereum.vm.EvmConfig import scala.annotation.tailrec import cats.implicits._ +import io.iohk.ethereum.db.storage.EvmCodeStorage import io.iohk.ethereum.mpt.MerklePatriciaTrie.MPTException class BlockExecution( blockchain: BlockchainImpl, + evmCodeStorage: EvmCodeStorage, blockchainConfig: BlockchainConfig, blockPreparator: BlockPreparator, blockValidation: BlockValidation @@ -57,10 +59,19 @@ class BlockExecution( /** Executes a block (executes transactions and pays rewards) */ private def executeBlock(block: Block): Either[BlockExecutionError, BlockResult] = { for { - parent <- blockchain + parentHeader <- blockchain .getBlockHeaderByHash(block.header.parentHash) .toRight(MissingParentError) // Should not never occur because validated earlier - execResult <- executeBlockTransactions(block, parent) + initialWorld = InMemoryWorldStateProxy( + evmCodeStorage = evmCodeStorage, + blockchain.getBackingStorage(block.header.number), + blockchain, + accountStartNonce = blockchainConfig.accountStartNonce, + stateRootHash = parentHeader.stateRoot, + noEmptyAccounts = EvmConfig.forBlock(parentHeader.number, blockchainConfig).noEmptyAccounts, + ethCompatibleStorage = blockchainConfig.ethCompatibleStorage + ) + execResult <- executeBlockTransactions(block, initialWorld) worldToPersist <- Either .catchOnly[MPTException](blockPreparator.payBlockReward(block, execResult.worldState)) .leftMap(BlockExecutionError.MPTError.apply) @@ -75,18 +86,9 @@ class BlockExecution( */ protected[ledger] def executeBlockTransactions( block: Block, - parent: BlockHeader + initialWorld: InMemoryWorldStateProxy ): Either[BlockExecutionError, BlockResult] = { - val parentStateRoot = parent.stateRoot val blockHeaderNumber = block.header.number - val initialWorld = blockchain.getWorldStateProxy( - blockNumber = blockHeaderNumber, - accountStartNonce = blockchainConfig.accountStartNonce, - stateRootHash = parentStateRoot, - noEmptyAccounts = EvmConfig.forBlock(blockHeaderNumber, blockchainConfig).noEmptyAccounts, - ethCompatibleStorage = blockchainConfig.ethCompatibleStorage - ) - executeBlockTransactions(block, blockHeaderNumber, initialWorld) } diff --git a/src/main/scala/io/iohk/ethereum/ledger/BlockPreparator.scala b/src/main/scala/io/iohk/ethereum/ledger/BlockPreparator.scala index 3520c56ca7..ca95460c4f 100644 --- a/src/main/scala/io/iohk/ethereum/ledger/BlockPreparator.scala +++ b/src/main/scala/io/iohk/ethereum/ledger/BlockPreparator.scala @@ -2,6 +2,7 @@ package io.iohk.ethereum.ledger import io.iohk.ethereum.consensus.validators.SignedTransactionError.TransactionSignatureError import io.iohk.ethereum.consensus.validators.SignedTransactionValidator +import io.iohk.ethereum.db.storage.EvmCodeStorage import io.iohk.ethereum.domain.UInt256._ import io.iohk.ethereum.domain._ import io.iohk.ethereum.ledger.BlockExecutionError.{StateBeforeFailure, TxsExecutionError} @@ -370,6 +371,7 @@ class BlockPreparator( } def prepareBlock( + evmCodeStorage: EvmCodeStorage, block: Block, parent: BlockHeader, initialWorldStateBeforeExecution: Option[InMemoryWorldStateProxy] @@ -377,10 +379,12 @@ class BlockPreparator( val initialWorld = initialWorldStateBeforeExecution.getOrElse( - blockchain.getReadOnlyWorldStateProxy( - None, - blockchainConfig.accountStartNonce, - parent.stateRoot, + InMemoryWorldStateProxy( + evmCodeStorage = evmCodeStorage, + mptStorage = blockchain.getReadOnlyStorage(), + blockchain = blockchain, + accountStartNonce = blockchainConfig.accountStartNonce, + stateRootHash = parent.stateRoot, noEmptyAccounts = EvmConfig.forBlock(block.header.number, blockchainConfig).noEmptyAccounts, ethCompatibleStorage = blockchainConfig.ethCompatibleStorage ) diff --git a/src/main/scala/io/iohk/ethereum/ledger/InMemoryWorldStateProxy.scala b/src/main/scala/io/iohk/ethereum/ledger/InMemoryWorldStateProxy.scala index 9a761546b2..2bc162e430 100644 --- a/src/main/scala/io/iohk/ethereum/ledger/InMemoryWorldStateProxy.scala +++ b/src/main/scala/io/iohk/ethereum/ledger/InMemoryWorldStateProxy.scala @@ -14,6 +14,24 @@ object InMemoryWorldStateProxy { import Account._ def apply( + evmCodeStorage: EvmCodeStorage, + mptStorage: MptStorage, + blockchain: Blockchain, + accountStartNonce: UInt256, + stateRootHash: ByteString, + noEmptyAccounts: Boolean, + ethCompatibleStorage: Boolean + ): InMemoryWorldStateProxy = InMemoryWorldStateProxy.apply( + evmCodeStorage, + mptStorage, + accountStartNonce, + (number: BigInt) => blockchain.getBlockHeaderByNumber(number).map(_.hash), + stateRootHash, + noEmptyAccounts, + ethCompatibleStorage + ) + + private def apply( evmCodeStorage: EvmCodeStorage, nodesKeyValueStorage: MptStorage, accountStartNonce: UInt256, diff --git a/src/main/scala/io/iohk/ethereum/ledger/Ledger.scala b/src/main/scala/io/iohk/ethereum/ledger/Ledger.scala index 65e4b6860f..d7b3de0f9c 100644 --- a/src/main/scala/io/iohk/ethereum/ledger/Ledger.scala +++ b/src/main/scala/io/iohk/ethereum/ledger/Ledger.scala @@ -3,6 +3,7 @@ package io.iohk.ethereum.ledger import akka.util.ByteString import cats.data.NonEmptyList import io.iohk.ethereum.consensus.Consensus +import io.iohk.ethereum.db.storage.EvmCodeStorage import io.iohk.ethereum.domain._ import io.iohk.ethereum.utils.Config.SyncConfig import io.iohk.ethereum.utils.{BlockchainConfig, Logger} @@ -71,6 +72,7 @@ trait Ledger { */ class LedgerImpl( blockchain: BlockchainImpl, + evmCodeStorage: EvmCodeStorage, blockQueue: BlockQueue, blockchainConfig: BlockchainConfig, theConsensus: Consensus, @@ -80,11 +82,19 @@ class LedgerImpl( def this( blockchain: BlockchainImpl, + evmCodeStorage: EvmCodeStorage, blockchainConfig: BlockchainConfig, syncConfig: SyncConfig, theConsensus: Consensus, validationContext: Scheduler - ) = this(blockchain, BlockQueue(blockchain, syncConfig), blockchainConfig, theConsensus, validationContext) + ) = this( + blockchain, + evmCodeStorage, + BlockQueue(blockchain, syncConfig), + blockchainConfig, + theConsensus, + validationContext + ) val consensus: Consensus = theConsensus @@ -94,7 +104,7 @@ class LedgerImpl( private[ledger] lazy val blockValidation = new BlockValidation(consensus, blockchain, blockQueue) private[ledger] lazy val blockExecution = - new BlockExecution(blockchain, blockchainConfig, consensus.blockPreparator, blockValidation) + new BlockExecution(blockchain, evmCodeStorage, blockchainConfig, consensus.blockPreparator, blockValidation) private[ledger] val branchResolution = new BranchResolution(blockchain) private[ledger] val blockImport = new BlockImport( diff --git a/src/main/scala/io/iohk/ethereum/ledger/StxLedger.scala b/src/main/scala/io/iohk/ethereum/ledger/StxLedger.scala index 8d46ad0470..38838f8f7b 100644 --- a/src/main/scala/io/iohk/ethereum/ledger/StxLedger.scala +++ b/src/main/scala/io/iohk/ethereum/ledger/StxLedger.scala @@ -1,11 +1,17 @@ package io.iohk.ethereum.ledger +import io.iohk.ethereum.db.storage.EvmCodeStorage import io.iohk.ethereum.domain.{Account, BlockHeader, BlockchainImpl, SignedTransactionWithSender} import io.iohk.ethereum.ledger.Ledger.TxResult import io.iohk.ethereum.utils.BlockchainConfig import io.iohk.ethereum.vm.EvmConfig -class StxLedger(blockchain: BlockchainImpl, blockchainConfig: BlockchainConfig, blockPreparator: BlockPreparator) { +class StxLedger( + blockchain: BlockchainImpl, + evmCodeStorage: EvmCodeStorage, + blockchainConfig: BlockchainConfig, + blockPreparator: BlockPreparator +) { def simulateTransaction( stx: SignedTransactionWithSender, @@ -15,8 +21,10 @@ class StxLedger(blockchain: BlockchainImpl, blockchainConfig: BlockchainConfig, val tx = stx.tx val world1 = world.getOrElse( - blockchain.getReadOnlyWorldStateProxy( - blockNumber = None, + InMemoryWorldStateProxy( + evmCodeStorage = evmCodeStorage, + mptStorage = blockchain.getReadOnlyStorage(), + blockchain = blockchain, accountStartNonce = blockchainConfig.accountStartNonce, stateRootHash = blockHeader.stateRoot, noEmptyAccounts = EvmConfig.forBlock(blockHeader.number, blockchainConfig).noEmptyAccounts, diff --git a/src/main/scala/io/iohk/ethereum/nodebuilder/NodeBuilder.scala b/src/main/scala/io/iohk/ethereum/nodebuilder/NodeBuilder.scala index 6eb63a95ab..0414147ae7 100644 --- a/src/main/scala/io/iohk/ethereum/nodebuilder/NodeBuilder.scala +++ b/src/main/scala/io/iohk/ethereum/nodebuilder/NodeBuilder.scala @@ -466,10 +466,11 @@ trait EthBlocksServiceBuilder { } trait EthUserServiceBuilder { - self: BlockchainBuilder with BlockchainConfigBuilder with LedgerBuilder => + self: BlockchainBuilder with BlockchainConfigBuilder with LedgerBuilder with StorageBuilder => lazy val ethUserService = new EthUserService( blockchain, + storagesInstance.storages.evmCodeStorage, ledger, blockchainConfig ) @@ -672,6 +673,7 @@ trait LedgerBuilder { trait StdLedgerBuilder extends LedgerBuilder { self: BlockchainConfigBuilder with BlockchainBuilder + with StorageBuilder with SyncConfigBuilder with ConsensusBuilder with ActorSystemBuilder => @@ -685,11 +687,19 @@ trait StdLedgerBuilder extends LedgerBuilder { * so a refactoring should probably take that into account. */ protected def newLedger(): LedgerImpl = - new LedgerImpl(blockchain, blockchainConfig, syncConfig, consensus, scheduler) + new LedgerImpl( + blockchain, + storagesInstance.storages.evmCodeStorage, + blockchainConfig, + syncConfig, + consensus, + scheduler + ) override lazy val ledger: Ledger = newLedger() - override lazy val stxLedger: StxLedger = new StxLedger(blockchain, blockchainConfig, consensus.blockPreparator) + override lazy val stxLedger: StxLedger = + new StxLedger(blockchain, storagesInstance.storages.evmCodeStorage, blockchainConfig, consensus.blockPreparator) } trait CheckpointBlockGeneratorBuilder { diff --git a/src/main/scala/io/iohk/ethereum/testmode/TestBlockchainBuilder.scala b/src/main/scala/io/iohk/ethereum/testmode/TestBlockchainBuilder.scala index 6f66ca6edc..3c93274817 100644 --- a/src/main/scala/io/iohk/ethereum/testmode/TestBlockchainBuilder.scala +++ b/src/main/scala/io/iohk/ethereum/testmode/TestBlockchainBuilder.scala @@ -24,25 +24,7 @@ trait TestBlockchainBuilder extends BlockchainBuilder { transactionMappingStorage = storages.transactionMappingStorage, appStateStorage = storages.appStateStorage, stateStorage = storages.stateStorage - ) { - override def getWorldStateProxy( - blockNumber: BigInt, - accountStartNonce: UInt256, - stateRootHash: ByteString, - noEmptyAccounts: Boolean, - ethCompatibleStorage: Boolean - ): InMemoryWorldStateProxy = - TestModeWorldStateProxy( - evmCodeStorage, - stateStorage.getBackingStorage(blockNumber), - accountStartNonce, - (number: BigInt) => getBlockHeaderByNumber(number).map(_.hash), - stateRootHash, - noEmptyAccounts, - ethCompatibleStorage, - key => preimages.put(crypto.kec256(key.bytes), key) - ) - } + ) } } diff --git a/src/main/scala/io/iohk/ethereum/testmode/TestModeComponentsProvider.scala b/src/main/scala/io/iohk/ethereum/testmode/TestModeComponentsProvider.scala index f13214ee01..14079ac010 100644 --- a/src/main/scala/io/iohk/ethereum/testmode/TestModeComponentsProvider.scala +++ b/src/main/scala/io/iohk/ethereum/testmode/TestModeComponentsProvider.scala @@ -2,6 +2,7 @@ package io.iohk.ethereum.testmode import io.iohk.ethereum.consensus.difficulty.DifficultyCalculator import io.iohk.ethereum.consensus.{Consensus, ConsensusConfig} +import io.iohk.ethereum.db.storage.EvmCodeStorage import io.iohk.ethereum.domain.BlockchainImpl import io.iohk.ethereum.ledger.Ledger.VMImpl import io.iohk.ethereum.ledger.{Ledger, LedgerImpl, StxLedger} @@ -12,6 +13,7 @@ import monix.execution.Scheduler /** Provides a ledger or consensus instances with modifiable blockchain config (used in test mode). */ class TestModeComponentsProvider( blockchain: BlockchainImpl, + evmCodeStorage: EvmCodeStorage, syncConfig: SyncConfig, validationExecutionContext: Scheduler, consensusConfig: ConsensusConfig, @@ -22,13 +24,14 @@ class TestModeComponentsProvider( def ledger(blockchainConfig: BlockchainConfig, sealEngine: SealEngineType): Ledger = new LedgerImpl( blockchain, + evmCodeStorage, blockchainConfig, syncConfig, consensus(blockchainConfig, sealEngine), validationExecutionContext ) def stxLedger(blockchainConfig: BlockchainConfig, sealEngine: SealEngineType): StxLedger = - new StxLedger(blockchain, blockchainConfig, consensus(blockchainConfig, sealEngine).blockPreparator) + new StxLedger(blockchain, evmCodeStorage, blockchainConfig, consensus(blockchainConfig, sealEngine).blockPreparator) def consensus( blockchainConfig: BlockchainConfig, sealEngine: SealEngineType, @@ -36,6 +39,7 @@ class TestModeComponentsProvider( ): TestmodeConsensus = new TestmodeConsensus( vm, + evmCodeStorage, blockchain, blockchainConfig, consensusConfig, diff --git a/src/main/scala/io/iohk/ethereum/testmode/TestModeServiceBuilder.scala b/src/main/scala/io/iohk/ethereum/testmode/TestModeServiceBuilder.scala index a1742c6a07..bb90ff0db7 100644 --- a/src/main/scala/io/iohk/ethereum/testmode/TestModeServiceBuilder.scala +++ b/src/main/scala/io/iohk/ethereum/testmode/TestModeServiceBuilder.scala @@ -12,6 +12,7 @@ import monix.execution.Scheduler trait TestModeServiceBuilder extends LedgerBuilder { self: BlockchainConfigBuilder + with StorageBuilder with TestBlockchainBuilder with SyncConfigBuilder with ConsensusBuilder @@ -24,6 +25,7 @@ trait TestModeServiceBuilder extends LedgerBuilder { lazy val testModeComponentsProvider: TestModeComponentsProvider = new TestModeComponentsProvider( blockchain, + storagesInstance.storages.evmCodeStorage, syncConfig, scheduler, consensusConfig, diff --git a/src/main/scala/io/iohk/ethereum/testmode/TestmodeConsensus.scala b/src/main/scala/io/iohk/ethereum/testmode/TestmodeConsensus.scala index c4446e5f75..8df97310d5 100644 --- a/src/main/scala/io/iohk/ethereum/testmode/TestmodeConsensus.scala +++ b/src/main/scala/io/iohk/ethereum/testmode/TestmodeConsensus.scala @@ -16,9 +16,11 @@ import io.iohk.ethereum.nodebuilder._ import io.iohk.ethereum.utils.BlockchainConfig import monix.eval.Task import io.iohk.ethereum.consensus.pow.validators.ValidatorsExecutor +import io.iohk.ethereum.db.storage.EvmCodeStorage class TestmodeConsensus( override val vm: VMImpl, + evmCodeStorage: EvmCodeStorage, blockchain: BlockchainImpl, blockchainConfig: BlockchainConfig, consensusConfig: ConsensusConfig, @@ -86,7 +88,7 @@ class TestmodeConsensus( override def blockGenerator: NoOmmersBlockGenerator = new NoOmmersBlockGenerator( - blockchain, + evmCodeStorage, blockchainConfig, consensusConfig, blockPreparator, @@ -114,10 +116,15 @@ class TestmodeConsensus( } trait TestmodeConsensusBuilder extends ConsensusBuilder { - self: VmBuilder with TestBlockchainBuilder with BlockchainConfigBuilder with ConsensusConfigBuilder => + self: VmBuilder + with TestBlockchainBuilder + with BlockchainConfigBuilder + with ConsensusConfigBuilder + with StorageBuilder => override lazy val consensus = new TestmodeConsensus( vm, + storagesInstance.storages.evmCodeStorage, blockchain, blockchainConfig, consensusConfig, 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 1b9fc70cf2..c400ef6a85 100644 --- a/src/test/scala/io/iohk/ethereum/blockchain/sync/ScenarioSetup.scala +++ b/src/test/scala/io/iohk/ethereum/blockchain/sync/ScenarioSetup.scala @@ -86,6 +86,7 @@ trait ScenarioSetup extends StdTestConsensusBuilder with SyncConfigBuilder with protected def newTestLedger(consensus: Consensus): LedgerImpl = new LedgerImpl( blockchain = blockchain, + evmCodeStorage = storagesInstance.storages.evmCodeStorage, blockchainConfig = blockchainConfig, syncConfig = syncConfig, theConsensus = consensus, @@ -95,6 +96,7 @@ trait ScenarioSetup extends StdTestConsensusBuilder with SyncConfigBuilder with protected def newTestLedger(blockchain: BlockchainImpl): LedgerImpl = new LedgerImpl( blockchain = blockchain, + evmCodeStorage = storagesInstance.storages.evmCodeStorage, blockchainConfig = blockchainConfig, syncConfig = syncConfig, theConsensus = consensus, @@ -104,6 +106,7 @@ trait ScenarioSetup extends StdTestConsensusBuilder with SyncConfigBuilder with protected def newTestLedger(blockchain: BlockchainImpl, blockchainConfig: BlockchainConfig): LedgerImpl = new LedgerImpl( blockchain = blockchain, + evmCodeStorage = storagesInstance.storages.evmCodeStorage, blockchainConfig = blockchainConfig, syncConfig = syncConfig, theConsensus = consensus, diff --git a/src/test/scala/io/iohk/ethereum/blockchain/sync/StateSyncUtils.scala b/src/test/scala/io/iohk/ethereum/blockchain/sync/StateSyncUtils.scala index 809c60f49e..8dce4d4d3c 100644 --- a/src/test/scala/io/iohk/ethereum/blockchain/sync/StateSyncUtils.scala +++ b/src/test/scala/io/iohk/ethereum/blockchain/sync/StateSyncUtils.scala @@ -32,22 +32,21 @@ object StateSyncUtils extends EphemBlockchainTestSetup { } def buildWorld(accountData: Seq[MptNodeData], existingTree: Option[ByteString] = None): ByteString = { - val init: InMemoryWorldStateProxy = blockchain - .getWorldStateProxy( - blockNumber = 1, - accountStartNonce = blockchainConfig.accountStartNonce, - stateRootHash = existingTree.getOrElse(ByteString(MerklePatriciaTrie.EmptyRootHash)), - noEmptyAccounts = true, - ethCompatibleStorage = blockchainConfig.ethCompatibleStorage - ) - .asInstanceOf[InMemoryWorldStateProxy] + val init = InMemoryWorldStateProxy( + storagesInstance.storages.evmCodeStorage, + blockchain.getBackingStorage(1), + blockchain, + blockchainConfig.accountStartNonce, + existingTree.getOrElse(ByteString(MerklePatriciaTrie.EmptyRootHash)), + noEmptyAccounts = true, + ethCompatibleStorage = blockchainConfig.ethCompatibleStorage + ) val modifiedWorld = accountData.foldLeft(init) { case (world, data) => val storage = world.getStorage(data.accountAddress) val modifiedStorage = data.accountStorage.foldLeft(storage) { case (s, v) => s.store(v._1, v._2) } - val code = world.getCode(data.accountAddress) val worldWithAccAndStorage = world .saveAccount(data.accountAddress, Account.empty().copy(balance = data.accountBalance)) .saveStorage(data.accountAddress, modifiedStorage) 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 961e11cb96..b36449a56d 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 @@ -158,7 +158,14 @@ trait RegularSyncFixtures { self: Matchers with AsyncMockFactory => .timeout(remainingOrDefault) class TestLedgerImpl - extends LedgerImpl(blockchain, blockchainConfig, syncConfig, consensus, Scheduler(system.dispatcher)) { + extends LedgerImpl( + blockchain, + storagesInstance.storages.evmCodeStorage, + blockchainConfig, + syncConfig, + consensus, + Scheduler(system.dispatcher) + ) { protected val results = mutable.Map[ByteString, Task[BlockImportResult]]() protected val importedBlocksSet = mutable.Set[Block]() private val importedBlocksSubject = ReplaySubject[Block]() 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 888beff75c..e36a5287a9 100644 --- a/src/test/scala/io/iohk/ethereum/consensus/blocks/BlockGeneratorSpec.scala +++ b/src/test/scala/io/iohk/ethereum/consensus/blocks/BlockGeneratorSpec.scala @@ -243,7 +243,13 @@ class BlockGeneratorSpec extends AnyFlatSpec with Matchers with ScalaCheckProper ) override lazy val blockExecution = - new BlockExecution(blockchain, blockchainConfig, consensus.blockPreparator, blockValidation) + new BlockExecution( + blockchain, + storagesInstance.storages.evmCodeStorage, + blockchainConfig, + consensus.blockPreparator, + blockValidation + ) val generalTx = SignedTransaction.sign(transaction, keyPair, None).tx val specificTx = @@ -319,7 +325,13 @@ class BlockGeneratorSpec extends AnyFlatSpec with Matchers with ScalaCheckProper ) override lazy val blockExecution = - new BlockExecution(blockchain, blockchainConfig, consensus.blockPreparator, blockValidation) + new BlockExecution( + blockchain, + storagesInstance.storages.evmCodeStorage, + blockchainConfig, + consensus.blockPreparator, + blockValidation + ) val transaction1 = Transaction( nonce = 0, @@ -710,7 +722,13 @@ class BlockGeneratorSpec extends AnyFlatSpec with Matchers with ScalaCheckProper lazy val blockValidation = new BlockValidation(consensus, blockchain, BlockQueue(blockchain, syncConfig)) lazy val blockExecution = - new BlockExecution(blockchain, blockchainConfig, consensus.blockPreparator, blockValidation) + new BlockExecution( + blockchain, + storagesInstance.storages.evmCodeStorage, + blockchainConfig, + consensus.blockPreparator, + blockValidation + ) // FIXME: the change in gas limit voting strategy caused the hardcoded nonce and mixHash in this file to be invalid // The gas limit of all the generated blocks has to be set to the old strategy of increasing as much as possible diff --git a/src/test/scala/io/iohk/ethereum/consensus/pow/MinerSpecSetup.scala b/src/test/scala/io/iohk/ethereum/consensus/pow/MinerSpecSetup.scala index 5f7b5f8b23..cb0a1a2e37 100644 --- a/src/test/scala/io/iohk/ethereum/consensus/pow/MinerSpecSetup.scala +++ b/src/test/scala/io/iohk/ethereum/consensus/pow/MinerSpecSetup.scala @@ -11,6 +11,7 @@ import io.iohk.ethereum.consensus.blocks.{PendingBlock, PendingBlockAndState} import io.iohk.ethereum.consensus.pow.blocks.PoWBlockGenerator import io.iohk.ethereum.consensus.pow.difficulty.EthashDifficultyCalculator import io.iohk.ethereum.consensus.pow.validators.ValidatorsExecutor +import io.iohk.ethereum.db.storage.{EvmCodeStorage, MptStorage} import io.iohk.ethereum.domain._ import io.iohk.ethereum.jsonrpc.EthMiningService import io.iohk.ethereum.jsonrpc.EthMiningService.SubmitHashRateResponse @@ -41,6 +42,7 @@ trait MinerSpecSetup extends ConsensusConfigBuilder with MockFactory { val fakeWorld = mock[InMemoryWorldStateProxy] val blockGenerator: PoWBlockGenerator = mock[PoWBlockGenerator] val ethMiningService: EthMiningService = mock[EthMiningService] + val evmCodeStorage: EvmCodeStorage = mock[EvmCodeStorage] lazy val vm: VMImpl = new VMImpl @@ -76,7 +78,7 @@ trait MinerSpecSetup extends ConsensusConfigBuilder with MockFactory { val validators = ValidatorsExecutor(blockchainConfig, consensusConfig.protocol) val additionalPoWData = NoAdditionalPoWData - PoWConsensus(vm, blockchain, blockchainConfig, fullConfig, validators, additionalPoWData) + PoWConsensus(vm, evmCodeStorage, blockchain, blockchainConfig, fullConfig, validators, additionalPoWData) } protected def setBlockForMining(parentBlock: Block, transactions: Seq[SignedTransaction] = Seq(txToMine)): Block = { diff --git a/src/test/scala/io/iohk/ethereum/consensus/pow/PoWConsensusSpec.scala b/src/test/scala/io/iohk/ethereum/consensus/pow/PoWConsensusSpec.scala index f3d305d480..1d9d6a439c 100644 --- a/src/test/scala/io/iohk/ethereum/consensus/pow/PoWConsensusSpec.scala +++ b/src/test/scala/io/iohk/ethereum/consensus/pow/PoWConsensusSpec.scala @@ -9,6 +9,7 @@ import io.iohk.ethereum.consensus.Protocol.{NoAdditionalPoWData, RestrictedPoWMi import io.iohk.ethereum.consensus.pow.blocks.{PoWBlockGeneratorImpl, RestrictedPoWBlockGeneratorImpl} import io.iohk.ethereum.consensus.pow.validators.ValidatorsExecutor import io.iohk.ethereum.consensus.{ConsensusConfigs, FullConsensusConfig, Protocol} +import io.iohk.ethereum.db.storage.EvmCodeStorage import io.iohk.ethereum.domain.BlockchainImpl import io.iohk.ethereum.nodebuilder.StdNode import org.bouncycastle.crypto.AsymmetricCipherKeyPair @@ -25,6 +26,7 @@ class PoWConsensusSpec "PoWConsensus" should "use NoAdditionalPoWData block generator for PoWBlockGeneratorImpl" in new TestSetup { val powConsensus = PoWConsensus( vm, + storagesInstance.storages.evmCodeStorage, blockchain, blockchainConfig, ConsensusConfigs.fullConsensusConfig, @@ -40,6 +42,7 @@ class PoWConsensusSpec val powConsensus = PoWConsensus( vm, + evmCodeStorage, blockchain, blockchainConfig, ConsensusConfigs.fullConsensusConfig, @@ -56,6 +59,7 @@ class PoWConsensusSpec val powConsensus = PoWConsensus( vm, + evmCodeStorage, blockchain, blockchainConfig, fullConsensusConfig, @@ -74,6 +78,7 @@ class PoWConsensusSpec val powConsensus = PoWConsensus( vm, + evmCodeStorage, blockchain, blockchainConfig, fullConsensusConfig, @@ -92,6 +97,7 @@ class PoWConsensusSpec val powConsensus = PoWConsensus( vm, + evmCodeStorage, blockchain, blockchainConfig, fullConsensusConfig, @@ -110,6 +116,7 @@ class PoWConsensusSpec val powConsensus = PoWConsensus( vm, + evmCodeStorage, blockchain, blockchainConfig, fullConsensusConfig, @@ -124,6 +131,7 @@ class PoWConsensusSpec trait TestSetup extends ScenarioSetup with MockFactory { override lazy val blockchain: BlockchainImpl = mock[BlockchainImpl] + val evmCodeStorage: EvmCodeStorage = mock[EvmCodeStorage] val validator: ValidatorsExecutor = successValidators.asInstanceOf[ValidatorsExecutor] } diff --git a/src/test/scala/io/iohk/ethereum/jsonrpc/EthBlocksServiceSpec.scala b/src/test/scala/io/iohk/ethereum/jsonrpc/EthBlocksServiceSpec.scala index 86ced7b597..26407cd409 100644 --- a/src/test/scala/io/iohk/ethereum/jsonrpc/EthBlocksServiceSpec.scala +++ b/src/test/scala/io/iohk/ethereum/jsonrpc/EthBlocksServiceSpec.scala @@ -10,7 +10,7 @@ import io.iohk.ethereum.consensus.{ConsensusConfigs, TestConsensus} import io.iohk.ethereum.db.storage.AppStateStorage import io.iohk.ethereum.domain.{Block, BlockBody, ChainWeight, UInt256} import io.iohk.ethereum.jsonrpc.EthBlocksService._ -import io.iohk.ethereum.ledger.Ledger +import io.iohk.ethereum.ledger.{InMemoryWorldStateProxy, Ledger} import io.iohk.ethereum.{Fixtures, NormalPatience, WithActorSystemShutDown} import monix.execution.Scheduler.Implicits.global import org.scalactic.TypeCheckedTripleEquals @@ -429,8 +429,10 @@ class EthBlocksServiceSpec val uncleWeight = ChainWeight.totalDifficultyOnly(uncle.difficulty) val blockToRequestWithUncles = blockToRequest.copy(body = BlockBody(Nil, Seq(uncle))) - val fakeWorld = blockchain.getReadOnlyWorldStateProxy( - None, + val fakeWorld = InMemoryWorldStateProxy( + storagesInstance.storages.evmCodeStorage, + blockchain.getBackingStorage(-1), + blockchain, UInt256.Zero, ByteString.empty, noEmptyAccounts = false, diff --git a/src/test/scala/io/iohk/ethereum/jsonrpc/EthInfoServiceSpec.scala b/src/test/scala/io/iohk/ethereum/jsonrpc/EthInfoServiceSpec.scala index a7e25ce9c5..6da8285681 100644 --- a/src/test/scala/io/iohk/ethereum/jsonrpc/EthInfoServiceSpec.scala +++ b/src/test/scala/io/iohk/ethereum/jsonrpc/EthInfoServiceSpec.scala @@ -9,11 +9,11 @@ import io.iohk.ethereum.blockchain.sync.{EphemBlockchainTestSetup, SyncProtocol} import io.iohk.ethereum.consensus._ import io.iohk.ethereum.consensus.pow.blocks.PoWBlockGenerator import io.iohk.ethereum.db.storage.AppStateStorage -import io.iohk.ethereum.domain.{Block, BlockchainImpl, UInt256, _} +import io.iohk.ethereum.domain.{Block, UInt256, _} import io.iohk.ethereum.jsonrpc.EthInfoService.{ProtocolVersionRequest, _} import io.iohk.ethereum.keystore.KeyStore import io.iohk.ethereum.ledger.Ledger.TxResult -import io.iohk.ethereum.ledger.{Ledger, StxLedger} +import io.iohk.ethereum.ledger.{InMemoryWorldStateProxy, Ledger, StxLedger} import io.iohk.ethereum.network.p2p.messages.Capability import io.iohk.ethereum.testing.ActorsTesting.simpleAutoPilot import monix.execution.Scheduler.Implicits.global @@ -95,14 +95,17 @@ class EthServiceSpec blockchain.storeBlock(blockToRequest).commit() blockchain.saveBestKnownBlocks(blockToRequest.header.number) - val txResult = TxResult( - BlockchainImpl(storagesInstance.storages) - .getWorldStateProxy(-1, UInt256.Zero, ByteString.empty, noEmptyAccounts = false, ethCompatibleStorage = true), - 123, - Nil, - ByteString("return_value"), - None + val worldStateProxy = InMemoryWorldStateProxy( + storagesInstance.storages.evmCodeStorage, + blockchain.getBackingStorage(-1), + blockchain, + UInt256.Zero, + ByteString.empty, + noEmptyAccounts = false, + ethCompatibleStorage = true ) + + val txResult = TxResult(worldStateProxy, 123, Nil, ByteString("return_value"), None) (stxLedger.simulateTransaction _).expects(*, *, *).returning(txResult) val tx = CallTx( diff --git a/src/test/scala/io/iohk/ethereum/jsonrpc/EthMiningServiceSpec.scala b/src/test/scala/io/iohk/ethereum/jsonrpc/EthMiningServiceSpec.scala index 7f293233ff..3bc7199654 100644 --- a/src/test/scala/io/iohk/ethereum/jsonrpc/EthMiningServiceSpec.scala +++ b/src/test/scala/io/iohk/ethereum/jsonrpc/EthMiningServiceSpec.scala @@ -15,7 +15,7 @@ import io.iohk.ethereum.domain.BlockHeader.getEncodedWithoutNonce import io.iohk.ethereum.domain.{Block, BlockBody, BlockHeader, ChainWeight, UInt256} import io.iohk.ethereum.jsonrpc.EthMiningService._ import io.iohk.ethereum.jsonrpc.server.controllers.JsonRpcBaseController.JsonRpcConfig -import io.iohk.ethereum.ledger.{Ledger, StxLedger} +import io.iohk.ethereum.ledger.{InMemoryWorldStateProxy, Ledger} import io.iohk.ethereum.mpt.MerklePatriciaTrie import io.iohk.ethereum.nodebuilder.ApisBuilder import io.iohk.ethereum.ommers.OmmersPool @@ -236,6 +236,7 @@ class EthMiningServiceSpec lazy val difficultyCalc = new EthashDifficultyCalculator(blockchainConfig) lazy val restrictedGenerator = new RestrictedPoWBlockGeneratorImpl( + evmCodeStorage = storagesInstance.storages.evmCodeStorage, validators = MockValidatorsAlwaysSucceed, blockchain = blockchain, blockchainConfig = blockchainConfig, @@ -303,8 +304,10 @@ class EthMiningServiceSpec val powHash = ByteString(kec256(getEncodedWithoutNonce(block.header))) val target = ByteString((BigInt(2).pow(256) / difficulty).toByteArray) - val fakeWorld = blockchain.getReadOnlyWorldStateProxy( - None, + val fakeWorld = InMemoryWorldStateProxy( + storagesInstance.storages.evmCodeStorage, + blockchain.getReadOnlyStorage(), + blockchain, UInt256.Zero, ByteString.empty, noEmptyAccounts = false, diff --git a/src/test/scala/io/iohk/ethereum/jsonrpc/EthProofServiceSpec.scala b/src/test/scala/io/iohk/ethereum/jsonrpc/EthProofServiceSpec.scala index 3344cca062..d7734489b1 100644 --- a/src/test/scala/io/iohk/ethereum/jsonrpc/EthProofServiceSpec.scala +++ b/src/test/scala/io/iohk/ethereum/jsonrpc/EthProofServiceSpec.scala @@ -274,6 +274,7 @@ class EthProofServiceSpec val ethUserService = new EthUserService( blockchain, + storagesInstance.storages.evmCodeStorage, ledger, blockchainConfig ) diff --git a/src/test/scala/io/iohk/ethereum/jsonrpc/EthUserServiceSpec.scala b/src/test/scala/io/iohk/ethereum/jsonrpc/EthUserServiceSpec.scala index e6c782c988..1adc5f5ebc 100644 --- a/src/test/scala/io/iohk/ethereum/jsonrpc/EthUserServiceSpec.scala +++ b/src/test/scala/io/iohk/ethereum/jsonrpc/EthUserServiceSpec.scala @@ -139,6 +139,7 @@ class EthUserServiceSpec override lazy val ledger = mock[Ledger] lazy val ethUserService = new EthUserService( blockchain, + storagesInstance.storages.evmCodeStorage, ledger, blockchainConfig ) diff --git a/src/test/scala/io/iohk/ethereum/jsonrpc/JsonRpcControllerFixture.scala b/src/test/scala/io/iohk/ethereum/jsonrpc/JsonRpcControllerFixture.scala index 4bdc36256f..e52625ebd2 100644 --- a/src/test/scala/io/iohk/ethereum/jsonrpc/JsonRpcControllerFixture.scala +++ b/src/test/scala/io/iohk/ethereum/jsonrpc/JsonRpcControllerFixture.scala @@ -11,10 +11,11 @@ import io.iohk.ethereum.consensus.{ConsensusConfigs, TestConsensus} import io.iohk.ethereum.crypto.ECDSASignature import io.iohk.ethereum.db.storage.AppStateStorage import io.iohk.ethereum.domain.BlockHeader.HeaderExtraFields.HefEmpty -import io.iohk.ethereum.domain.{Block, BlockBody, SignedTransaction, UInt256} +import io.iohk.ethereum.domain.{Block, BlockBody, SignedTransaction} import io.iohk.ethereum.jsonrpc.server.controllers.JsonRpcBaseController.JsonRpcConfig import io.iohk.ethereum.keystore.KeyStore -import io.iohk.ethereum.ledger.{BloomFilter, Ledger, StxLedger} +import io.iohk.ethereum.ledger.{BloomFilter, InMemoryWorldStateProxy, Ledger, StxLedger} +import io.iohk.ethereum.mpt.MerklePatriciaTrie import io.iohk.ethereum.network.p2p.messages.Capability import io.iohk.ethereum.nodebuilder.ApisBuilder import io.iohk.ethereum.utils.{Config, FilterConfig} @@ -107,6 +108,7 @@ class JsonRpcControllerFixture(implicit system: ActorSystem) val ethUserService = new EthUserService( blockchain, + storagesInstance.storages.evmCodeStorage, ledger, blockchainConfig ) @@ -172,9 +174,11 @@ class JsonRpcControllerFixture(implicit system: ActorSystem) def newJsonRpcRequest(method: String) = JsonRpcRequest("2.0", method, None, Some(JInt(1))) - val fakeWorld = blockchain.getReadOnlyWorldStateProxy( - None, - UInt256.Zero, + val fakeWorld = InMemoryWorldStateProxy( + storagesInstance.storages.evmCodeStorage, + blockchain.getReadOnlyStorage(), + blockchain, + blockchainConfig.accountStartNonce, ByteString.empty, noEmptyAccounts = false, ethCompatibleStorage = true diff --git a/src/test/scala/io/iohk/ethereum/ledger/BlockExecutionSpec.scala b/src/test/scala/io/iohk/ethereum/ledger/BlockExecutionSpec.scala index a9efd039d1..1bb1b583c2 100644 --- a/src/test/scala/io/iohk/ethereum/ledger/BlockExecutionSpec.scala +++ b/src/test/scala/io/iohk/ethereum/ledger/BlockExecutionSpec.scala @@ -5,9 +5,9 @@ import io.iohk.ethereum.Mocks.{MockVM, MockValidatorsAlwaysSucceed, MockValidato 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.domain.{Block, _} import io.iohk.ethereum.ledger.Ledger.BlockResult -import io.iohk.ethereum.vm.OutOfGas +import io.iohk.ethereum.vm.{EvmConfig, OutOfGas} import io.iohk.ethereum.{BlockHelpers, Mocks, ObjectGenerators} import org.scalatest.matchers.should.Matchers import org.scalatest.prop.TableFor4 @@ -47,7 +47,13 @@ class BlockExecutionSpec extends AnyWordSpec with Matchers with ScalaCheckProper 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) + new BlockExecution( + blockchain, + blockchainStorages.evmCodeStorage, + blockchainConfig, + newConsensus.blockPreparator, + blockValidation + ) val (blocks, error) = blockExecution.executeAndValidateBlocks(List(block1, block2), defaultChainWeight) @@ -81,7 +87,13 @@ class BlockExecutionSpec extends AnyWordSpec with Matchers with ScalaCheckProper val newConsensus: TestConsensus = consensus.withVM(mockVm).withValidators(mockValidators) val blockValidation = new BlockValidation(newConsensus, blockchain, BlockQueue(blockchain, syncConfig)) val blockExecution = - new BlockExecution(blockchain, blockchainConfig, newConsensus.blockPreparator, blockValidation) + new BlockExecution( + blockchain, + blockchainStorages.evmCodeStorage, + blockchainConfig, + newConsensus.blockPreparator, + blockValidation + ) val (blocks, error) = blockExecution.executeAndValidateBlocks(List(block1, block2), defaultChainWeight) @@ -108,7 +120,13 @@ class BlockExecutionSpec extends AnyWordSpec with Matchers with ScalaCheckProper val newConsensus: TestConsensus = consensus.withVM(mockVm).withValidators(mockValidators) val blockValidation = new BlockValidation(newConsensus, blockchain, BlockQueue(blockchain, syncConfig)) val blockExecution = - new BlockExecution(blockchain, blockchainConfig, newConsensus.blockPreparator, blockValidation) + new BlockExecution( + blockchain, + blockchainStorages.evmCodeStorage, + blockchainConfig, + newConsensus.blockPreparator, + blockValidation + ) val (blocks, error) = blockExecution.executeAndValidateBlocks(chain, defaultChainWeight) @@ -127,7 +145,13 @@ class BlockExecutionSpec extends AnyWordSpec with Matchers with ScalaCheckProper 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) + new BlockExecution( + blockchain, + blockchainStorages.evmCodeStorage, + blockchainConfig, + newConsensus.blockPreparator, + blockValidation + ) val (blocks, error) = blockExecution.executeAndValidateBlocks(List(blockWithCheckpoint), defaultChainWeight) @@ -148,7 +172,7 @@ class BlockExecutionSpec extends AnyWordSpec with Matchers with ScalaCheckProper val block = Block(validBlockHeader, validBlockBodyWithNoTxs) val txsExecResult: Either[BlockExecutionError, BlockResult] = - blockExecution.executeBlockTransactions(block, validBlockParentHeader) + blockExecution.executeBlockTransactions(block, initialWorld) txsExecResult.isRight shouldBe true @@ -178,10 +202,16 @@ class BlockExecutionSpec extends AnyWordSpec with Matchers with ScalaCheckProper val blockValidation = new BlockValidation(newConsensus, blockchain, BlockQueue(blockchain, syncConfig)) val blockExecution = - new BlockExecution(blockchain, blockchainConfig, newConsensus.blockPreparator, blockValidation) + new BlockExecution( + blockchain, + blockchainStorages.evmCodeStorage, + blockchainConfig, + newConsensus.blockPreparator, + blockValidation + ) val txsExecResult: Either[BlockExecutionError, BlockResult] = - blockExecution.executeBlockTransactions(block, validBlockParentHeader) + blockExecution.executeBlockTransactions(block, initialWorld) txsExecResult.isRight shouldBe true val BlockResult(resultingWorldState, resultingGasUsed, resultingReceipts) = txsExecResult.toOption.get @@ -246,9 +276,15 @@ class BlockExecutionSpec extends AnyWordSpec with Matchers with ScalaCheckProper val newConsensus = consensus.withValidators(mockValidators).withVM(mockVm) val blockValidation = new BlockValidation(newConsensus, blockchain, BlockQueue(blockchain, syncConfig)) val blockExecution = - new BlockExecution(blockchain, blockchainConfig, newConsensus.blockPreparator, blockValidation) + new BlockExecution( + blockchain, + blockchainStorages.evmCodeStorage, + blockchainConfig, + newConsensus.blockPreparator, + blockValidation + ) - val txsExecResult = blockExecution.executeBlockTransactions(block, validBlockParentHeader) + val txsExecResult = blockExecution.executeBlockTransactions(block, initialWorld) txsExecResult.isRight shouldBe txValidAccordingToValidators if (txsExecResult.isRight) { @@ -286,7 +322,7 @@ class BlockExecutionSpec extends AnyWordSpec with Matchers with ScalaCheckProper val block = Block(validBlockHeader, blockBodyWithTxs) val txsExecResult: Either[BlockExecutionError, BlockResult] = - blockExecution.executeBlockTransactions(block, validBlockParentHeader) + blockExecution.executeBlockTransactions(block, initialWorld) txsExecResult.isLeft shouldBe true } @@ -298,7 +334,7 @@ class BlockExecutionSpec extends AnyWordSpec with Matchers with ScalaCheckProper val block = Block(validBlockHeader, blockBodyWithTxs) val txsExecResult: Either[BlockExecutionError, BlockResult] = - blockExecution.executeBlockTransactions(block, validBlockParentHeader) + blockExecution.executeBlockTransactions(block, initialWorld) txsExecResult.isLeft shouldBe true } @@ -306,9 +342,13 @@ class BlockExecutionSpec extends AnyWordSpec with Matchers with ScalaCheckProper } trait BlockExecutionTestSetup extends BlockchainSetup { - val blockValidation = new BlockValidation(consensus, blockchain, BlockQueue(blockchain, syncConfig)) - val blockExecution = new BlockExecution(blockchain, blockchainConfig, consensus.blockPreparator, blockValidation) - + val blockExecution = new BlockExecution( + blockchain, + blockchainStorages.evmCodeStorage, + blockchainConfig, + consensus.blockPreparator, + blockValidation + ) } } diff --git a/src/test/scala/io/iohk/ethereum/ledger/BlockImportSpec.scala b/src/test/scala/io/iohk/ethereum/ledger/BlockImportSpec.scala index b90878ced6..af63abfa99 100644 --- a/src/test/scala/io/iohk/ethereum/ledger/BlockImportSpec.scala +++ b/src/test/scala/io/iohk/ethereum/ledger/BlockImportSpec.scala @@ -6,15 +6,17 @@ import io.iohk.ethereum.Mocks.MockValidatorsAlwaysSucceed import io.iohk.ethereum.consensus._ 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.MerklePatriciaTrie +import io.iohk.ethereum.mpt.{LeafNode, MerklePatriciaTrie} import org.scalatest.concurrent.ScalaFutures -import scala.concurrent.duration._ -import scala.language.postfixOps 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 { override implicit val patienceConfig: PatienceConfig = @@ -46,14 +48,6 @@ class BlockImportSpec extends AnyFlatSpec with Matchers with ScalaFutures { val newWeight = currentWeight.increaseTotalDifficulty(difficulty) val blockData = BlockData(block, Seq.empty[Receipt], newWeight) - val emptyWorld: InMemoryWorldStateProxy = BlockchainImpl(storagesInstance.storages) - .getWorldStateProxy( - -1, - UInt256.Zero, - ByteString(MerklePatriciaTrie.EmptyRootHash), - noEmptyAccounts = false, - ethCompatibleStorage = true - ) // Just to bypass metrics needs (blockchain.getBlockByHash _).expects(*).returning(None) @@ -62,7 +56,7 @@ class BlockImportSpec extends AnyFlatSpec with Matchers with ScalaFutures { (blockQueue.getBranch _).expects(hash, true).returning(List(block)) (blockchain.getBlockHeaderByHash _).expects(*).returning(Some(block.header)) - (blockchain.getWorldStateProxy _).expects(*, *, *, *, *).returning(emptyWorld) + (blockchain.getBackingStorage _).expects(*).returning(storagesInstance.storages.stateStorage.getBackingStorage(6)) expectBlockSaved(block, Seq.empty[Receipt], newWeight, saveAsBestBlock = true) @@ -84,20 +78,22 @@ class BlockImportSpec extends AnyFlatSpec with Matchers with ScalaFutures { .returning(Some(Leaf(hash, currentWeight.increase(block.header)))) (blockQueue.getBranch _).expects(hash, true).returning(List(block)) - val emptyWorld: InMemoryWorldStateProxy = BlockchainImpl(storagesInstance.storages) - .getWorldStateProxy( - -1, - UInt256.Zero, - ByteString(MerklePatriciaTrie.EmptyRootHash), - noEmptyAccounts = false, - ethCompatibleStorage = true - ) + val mptStorage = mock[MptStorage] + val mptNode = LeafNode( + ByteString(MerklePatriciaTrie.EmptyRootHash), + ByteString(MerklePatriciaTrie.EmptyRootHash), + Some(MerklePatriciaTrie.EmptyRootHash), + Some(MerklePatriciaTrie.EmptyRootHash) + ) (blockchain.getBlockHeaderByHash _).expects(*).returning(Some(block.header)) - (blockchain.getWorldStateProxy _).expects(*, *, *, *, *).returning(emptyWorld) - (blockQueue.removeSubtree _).expects(hash) + (blockchain.getBlockHeaderByNumber _).expects(*).returning(Some(block.header)) + (blockchain.getBackingStorage _).expects(*).returning(mptStorage) + (mptStorage.get _).expects(*).returning(mptNode) + + (blockQueue.removeSubtree _).expects(*) - whenReady(ledger.importBlock(block).runToFuture) { _ shouldBe a[BlockImportFailed] } + whenReady(failingLedger.importBlock(block).runToFuture) { _ shouldBe a[BlockImportFailed] } } // scalastyle:off magic.number @@ -239,7 +235,14 @@ class BlockImportSpec extends AnyFlatSpec with Matchers with ScalaFutures { val newConsensus: TestConsensus = consensus.withValidators(validators).withVM(new Mocks.MockVM()) val ledgerWithMockedValidators = - new LedgerImpl(blockchain, blockQueue, blockchainConfig, newConsensus, scheduler) + new LedgerImpl( + blockchain, + storagesInstance.storages.evmCodeStorage, + blockQueue, + blockchainConfig, + newConsensus, + scheduler + ) val newBlock: Block = getBlock(number = bestNum + 1) setBlockExists(newBlock, inChain = false, inQueue = false) @@ -261,7 +264,14 @@ class BlockImportSpec extends AnyFlatSpec with Matchers with ScalaFutures { val newConsensus: TestConsensus = consensus.withValidators(validators).withVM(new Mocks.MockVM()) val ledgerWithMockedValidators = - new LedgerImpl(blockchain, blockQueue, blockchainConfig, newConsensus, scheduler) + new LedgerImpl( + blockchain, + storagesInstance.storages.evmCodeStorage, + blockQueue, + blockchainConfig, + newConsensus, + scheduler + ) val newBlock: Block = getBlock(number = bestNum + 1) setBlockExists(newBlock, inChain = false, inQueue = false) diff --git a/src/test/scala/io/iohk/ethereum/ledger/BlockRewardSpec.scala b/src/test/scala/io/iohk/ethereum/ledger/BlockRewardSpec.scala index ec274ffe0c..cf6d59fcb6 100644 --- a/src/test/scala/io/iohk/ethereum/ledger/BlockRewardSpec.scala +++ b/src/test/scala/io/iohk/ethereum/ledger/BlockRewardSpec.scala @@ -191,14 +191,15 @@ class BlockRewardSpec extends AnyFlatSpec with Matchers with ScalaCheckPropertyC val ommerFiveBlocksDifferenceReward = BigInt("1875000000000000000") val afterByzantiumNewBlockReward: BigInt = BigInt(10).pow(18) * 3 - val worldState: InMemoryWorldStateProxy = BlockchainImpl(storagesInstance.storages) - .getWorldStateProxy( - -1, - UInt256.Zero, - ByteString(MerklePatriciaTrie.EmptyRootHash), - noEmptyAccounts = false, - ethCompatibleStorage = true - ) + val worldState = InMemoryWorldStateProxy( + storagesInstance.storages.evmCodeStorage, + blockchain.getBackingStorage(-1), + blockchain, + UInt256.Zero, + ByteString(MerklePatriciaTrie.EmptyRootHash), + noEmptyAccounts = false, + ethCompatibleStorage = true + ) .saveAccount(validAccountAddress, Account(balance = 10)) .saveAccount(validAccountAddress2, Account(balance = 20)) .saveAccount(validAccountAddress3, Account(balance = 30)) diff --git a/src/test/scala/io/iohk/ethereum/ledger/DeleteAccountsSpec.scala b/src/test/scala/io/iohk/ethereum/ledger/DeleteAccountsSpec.scala index 9b75488c03..ac07be207e 100644 --- a/src/test/scala/io/iohk/ethereum/ledger/DeleteAccountsSpec.scala +++ b/src/test/scala/io/iohk/ethereum/ledger/DeleteAccountsSpec.scala @@ -65,18 +65,18 @@ class DeleteAccountsSpec extends AnyFlatSpec with Matchers with MockFactory { val accountAddresses = Set(validAccountAddress, validAccountAddress2, validAccountAddress3) - val worldStateWithoutPersist: InMemoryWorldStateProxy = - BlockchainImpl(storagesInstance.storages) - .getWorldStateProxy( - -1, - UInt256.Zero, - ByteString(MerklePatriciaTrie.EmptyRootHash), - noEmptyAccounts = false, - ethCompatibleStorage = true - ) - .saveAccount(validAccountAddress, Account(balance = 10)) - .saveAccount(validAccountAddress2, Account(balance = 20)) - .saveAccount(validAccountAddress3, Account(balance = 30)) + val worldStateWithoutPersist = InMemoryWorldStateProxy( + storagesInstance.storages.evmCodeStorage, + blockchain.getBackingStorage(-1), + blockchain, + UInt256.Zero, + ByteString(MerklePatriciaTrie.EmptyRootHash), + noEmptyAccounts = false, + ethCompatibleStorage = true + ) + .saveAccount(validAccountAddress, Account(balance = 10)) + .saveAccount(validAccountAddress2, Account(balance = 20)) + .saveAccount(validAccountAddress3, Account(balance = 30)) val worldState = InMemoryWorldStateProxy.persistState(worldStateWithoutPersist) } diff --git a/src/test/scala/io/iohk/ethereum/ledger/DeleteTouchedAccountsSpec.scala b/src/test/scala/io/iohk/ethereum/ledger/DeleteTouchedAccountsSpec.scala index 9e368d06a1..be86000c29 100644 --- a/src/test/scala/io/iohk/ethereum/ledger/DeleteTouchedAccountsSpec.scala +++ b/src/test/scala/io/iohk/ethereum/ledger/DeleteTouchedAccountsSpec.scala @@ -3,26 +3,21 @@ package io.iohk.ethereum.ledger import akka.util.ByteString import io.iohk.ethereum.Mocks.MockVM import io.iohk.ethereum.blockchain.sync.EphemBlockchainTestSetup -import io.iohk.ethereum.domain.{Account, Address, BlockchainImpl, UInt256} +import io.iohk.ethereum.domain.{Account, Address, UInt256} import io.iohk.ethereum.ledger.Ledger.VMImpl import io.iohk.ethereum.mpt.MerklePatriciaTrie import io.iohk.ethereum.utils.Config import io.iohk.ethereum.utils.Config.SyncConfig import io.iohk.ethereum.vm.{BlockchainConfigForEvm, EvmConfig} -import org.scalamock.scalatest.MockFactory import org.scalatest.flatspec.AnyFlatSpec import org.scalatest.matchers.should.Matchers -class DeleteTouchedAccountsSpec extends AnyFlatSpec with Matchers with MockFactory { +class DeleteTouchedAccountsSpec extends AnyFlatSpec with Matchers { val blockchainConfig = Config.blockchains.blockchainConfig val syncConfig = SyncConfig(Config.config) - // FIXME Delete - // val blockchain = mock[BlockchainImpl] - it should "delete no accounts when there are no touched accounts" in new TestSetup { - val newWorld = InMemoryWorldStateProxy.persistState(consensus.blockPreparator.deleteEmptyTouchedAccounts(worldStatePostEIP161)) accountAddresses.foreach { a => assert(newWorld.getAccount(a).isDefined) } @@ -30,7 +25,6 @@ class DeleteTouchedAccountsSpec extends AnyFlatSpec with Matchers with MockFacto } it should "delete no accounts when there are no empty touched accounts" in new TestSetup { - val worldAfterTransfer = worldStatePostEIP161.transfer(validAccountAddress, validAccountAddress2, transferBalance) worldAfterTransfer.touchedAccounts.size shouldEqual 2 @@ -40,7 +34,6 @@ class DeleteTouchedAccountsSpec extends AnyFlatSpec with Matchers with MockFacto } it should "delete touched empty account" in new TestSetup { - val worldAfterTransfer = worldStatePostEIP161.transfer(validAccountAddress, validEmptyAccountAddress, zeroTransferBalance) worldAfterTransfer.touchedAccounts.size shouldEqual 2 @@ -54,7 +47,6 @@ class DeleteTouchedAccountsSpec extends AnyFlatSpec with Matchers with MockFacto } it should "delete touched empty account after transfer to self" in new TestSetup { - val worldAfterTransfer = worldStatePostEIP161.transfer(validEmptyAccountAddress, validEmptyAccountAddress, zeroTransferBalance) worldAfterTransfer.touchedAccounts.size shouldEqual 1 @@ -68,7 +60,6 @@ class DeleteTouchedAccountsSpec extends AnyFlatSpec with Matchers with MockFacto } it should "not mark for deletion and delete any account pre EIP161" in new TestSetup { - val worldAfterTransfer = worldStatePreEIP161.transfer(validAccountAddress, validEmptyAccountAddress, zeroTransferBalance) worldAfterTransfer.touchedAccounts.size shouldEqual 0 @@ -87,7 +78,6 @@ class DeleteTouchedAccountsSpec extends AnyFlatSpec with Matchers with MockFacto } it should "delete multiple touched empty accounts" in new TestSetup { - val worldAfterTransfer = worldStatePostEIP161.transfer(validAccountAddress, validEmptyAccountAddress, zeroTransferBalance) worldAfterTransfer.touchedAccounts.size shouldEqual 2 @@ -112,7 +102,6 @@ class DeleteTouchedAccountsSpec extends AnyFlatSpec with Matchers with MockFacto } it should "not delete touched new account resulting from contract creation (initialised)" in new TestSetup { - val worldAfterInitAndTransfer = worldStatePostEIP161 .initialiseAccount(validCreatedAccountAddress) @@ -159,35 +148,35 @@ class DeleteTouchedAccountsSpec extends AnyFlatSpec with Matchers with MockFacto validEmptyAccountAddress1 ) - val worldStateWithoutPersist: InMemoryWorldStateProxy = - BlockchainImpl(storagesInstance.storages) - .getWorldStateProxy( - -1, - UInt256.Zero, - ByteString(MerklePatriciaTrie.EmptyRootHash), - postEip161Config.noEmptyAccounts, - ethCompatibleStorage = true - ) - .saveAccount(validAccountAddress, Account(balance = validAccountBalance)) - .saveAccount(validAccountAddress2, Account(balance = 20)) - .saveAccount(validAccountAddress3, Account(balance = 30)) - .saveAccount(validEmptyAccountAddress, Account.empty()) - .saveAccount(validEmptyAccountAddress1, Account.empty()) - - val worldStateWithoutPersistPreEIP161: InMemoryWorldStateProxy = - BlockchainImpl(storagesInstance.storages) - .getWorldStateProxy( - -1, - UInt256.Zero, - ByteString(MerklePatriciaTrie.EmptyRootHash), - postEip160Config.noEmptyAccounts, - ethCompatibleStorage = true - ) - .saveAccount(validAccountAddress, Account(balance = validAccountBalance)) - .saveAccount(validAccountAddress2, Account(balance = 20)) - .saveAccount(validAccountAddress3, Account(balance = 30)) - .saveAccount(validEmptyAccountAddress, Account.empty()) - .saveAccount(validEmptyAccountAddress1, Account.empty()) + val worldStateWithoutPersist = InMemoryWorldStateProxy( + storagesInstance.storages.evmCodeStorage, + blockchain.getBackingStorage(-1), + blockchain, + UInt256.Zero, + ByteString(MerklePatriciaTrie.EmptyRootHash), + noEmptyAccounts = postEip161Config.noEmptyAccounts, + ethCompatibleStorage = true + ) + .saveAccount(validAccountAddress, Account(balance = validAccountBalance)) + .saveAccount(validAccountAddress2, Account(balance = 20)) + .saveAccount(validAccountAddress3, Account(balance = 30)) + .saveAccount(validEmptyAccountAddress, Account.empty()) + .saveAccount(validEmptyAccountAddress1, Account.empty()) + + val worldStateWithoutPersistPreEIP161 = InMemoryWorldStateProxy( + storagesInstance.storages.evmCodeStorage, + blockchain.getBackingStorage(-1), + blockchain, + UInt256.Zero, + ByteString(MerklePatriciaTrie.EmptyRootHash), + noEmptyAccounts = postEip160Config.noEmptyAccounts, + ethCompatibleStorage = true + ) + .saveAccount(validAccountAddress, Account(balance = validAccountBalance)) + .saveAccount(validAccountAddress2, Account(balance = 20)) + .saveAccount(validAccountAddress3, Account(balance = 30)) + .saveAccount(validEmptyAccountAddress, Account.empty()) + .saveAccount(validEmptyAccountAddress1, Account.empty()) val transferBalance = 5 val zeroTransferBalance = 0 diff --git a/src/test/scala/io/iohk/ethereum/ledger/InMemoryWorldStateProxySpec.scala b/src/test/scala/io/iohk/ethereum/ledger/InMemoryWorldStateProxySpec.scala index e9c1d27498..62b2aa1f42 100644 --- a/src/test/scala/io/iohk/ethereum/ledger/InMemoryWorldStateProxySpec.scala +++ b/src/test/scala/io/iohk/ethereum/ledger/InMemoryWorldStateProxySpec.scala @@ -2,7 +2,7 @@ package io.iohk.ethereum.ledger import akka.util.ByteString import io.iohk.ethereum.blockchain.sync.EphemBlockchainTestSetup -import io.iohk.ethereum.domain.{Account, Address, BlockchainImpl, UInt256} +import io.iohk.ethereum.domain.{Account, Address, UInt256} import io.iohk.ethereum.mpt.MerklePatriciaTrie import io.iohk.ethereum.mpt.MerklePatriciaTrie.MPTException import io.iohk.ethereum.vm.{EvmConfig, Generators} @@ -123,13 +123,16 @@ class InMemoryWorldStateProxySpec extends AnyFlatSpec with Matchers { validateInitialWorld(persistedWorldState) // Create a new WS instance based on storages and new root state and check - val newWorldState = BlockchainImpl(storagesInstance.storages).getWorldStateProxy( - -1, + val newWorldState = InMemoryWorldStateProxy( + storagesInstance.storages.evmCodeStorage, + blockchain.getBackingStorage(-1), + blockchain, UInt256.Zero, persistedWorldState.stateRootHash, noEmptyAccounts = true, ethCompatibleStorage = true ) + validateInitialWorld(newWorldState) // Update this new WS check everything is ok @@ -252,14 +255,15 @@ class InMemoryWorldStateProxySpec extends AnyFlatSpec with Matchers { val persistedWorldStateWithAnAccount = InMemoryWorldStateProxy.persistState(worldStateWithAnAccount) - val readWorldState = - blockchain.getReadOnlyWorldStateProxy( - None, - UInt256.Zero, - persistedWorldStateWithAnAccount.stateRootHash, - noEmptyAccounts = false, - ethCompatibleStorage = false - ) + val readWorldState = InMemoryWorldStateProxy( + storagesInstance.storages.evmCodeStorage, + blockchain.getReadOnlyStorage(), + blockchain, + UInt256.Zero, + persistedWorldStateWithAnAccount.stateRootHash, + noEmptyAccounts = false, + ethCompatibleStorage = false + ) readWorldState.getAccount(address1) shouldEqual Some(account) @@ -273,13 +277,16 @@ class InMemoryWorldStateProxySpec extends AnyFlatSpec with Matchers { ) assertThrows[MPTException] { - val newReadWorld = blockchain.getReadOnlyWorldStateProxy( - None, + val newReadWorld = InMemoryWorldStateProxy( + storagesInstance.storages.evmCodeStorage, + blockchain.getReadOnlyStorage(), + blockchain, UInt256.Zero, changedReadWorld.stateRootHash, noEmptyAccounts = false, ethCompatibleStorage = false ) + newReadWorld.getAccount(address1) shouldEqual Some(changedAccount) } @@ -299,8 +306,10 @@ class InMemoryWorldStateProxySpec extends AnyFlatSpec with Matchers { .saveStorage(alreadyExistingAddress, worldState.getStorage(alreadyExistingAddress).store(0, 1)) ) - val world2 = blockchain.getWorldStateProxy( - -1, + val world2 = InMemoryWorldStateProxy( + storagesInstance.storages.evmCodeStorage, + blockchain.getBackingStorage(-1), + blockchain, UInt256.Zero, world1.stateRootHash, noEmptyAccounts = false, @@ -323,16 +332,20 @@ class InMemoryWorldStateProxySpec extends AnyFlatSpec with Matchers { trait TestSetup extends EphemBlockchainTestSetup { val postEip161Config = EvmConfig.PostEIP161ConfigBuilder(io.iohk.ethereum.vm.Fixtures.blockchainConfig) - val worldState = - blockchain.getWorldStateProxy( - -1, - UInt256.Zero, - ByteString(MerklePatriciaTrie.EmptyRootHash), - noEmptyAccounts = false, - ethCompatibleStorage = true - ) - val postEIP161WorldState = blockchain.getWorldStateProxy( - -1, + val worldState = InMemoryWorldStateProxy( + storagesInstance.storages.evmCodeStorage, + blockchain.getBackingStorage(-1), + blockchain, + UInt256.Zero, + ByteString(MerklePatriciaTrie.EmptyRootHash), + noEmptyAccounts = false, + ethCompatibleStorage = true + ) + + val postEIP161WorldState = InMemoryWorldStateProxy( + storagesInstance.storages.evmCodeStorage, + blockchain.getBackingStorage(-1), + blockchain, UInt256.Zero, ByteString(MerklePatriciaTrie.EmptyRootHash), noEmptyAccounts = postEip161Config.noEmptyAccounts, diff --git a/src/test/scala/io/iohk/ethereum/ledger/LedgerSpec.scala b/src/test/scala/io/iohk/ethereum/ledger/LedgerSpec.scala index 9f8beec3a2..52b5e00f20 100644 --- a/src/test/scala/io/iohk/ethereum/ledger/LedgerSpec.scala +++ b/src/test/scala/io/iohk/ethereum/ledger/LedgerSpec.scala @@ -217,7 +217,7 @@ class LedgerSpec extends AnyFlatSpec with ScalaCheckPropertyChecks with Matchers val validBlockBodyWithTxs: BlockBody = validBlockBodyWithNoTxs.copy(transactionList = Seq(stx1.tx, stx2.tx)) val block = Block(validBlockHeader, validBlockBodyWithTxs) - val txsExecResult = ledger.blockExecution.executeBlockTransactions(block, validBlockParentHeader) + val txsExecResult = ledger.blockExecution.executeBlockTransactions(block, initialWorld) assert(txsExecResult.isRight) val BlockResult(resultingWorldState, resultingGasUsed, resultingReceipts) = txsExecResult.toOption.get @@ -298,7 +298,7 @@ class LedgerSpec extends AnyFlatSpec with ScalaCheckPropertyChecks with Matchers // We don't care about block txs in this test ledger.blockExecution.executeBlockTransactions( proDaoBlock.copy(body = proDaoBlock.body.copy(transactionList = Seq.empty)), - parentBlockHeader + initialWorld ) } @@ -313,7 +313,7 @@ class LedgerSpec extends AnyFlatSpec with ScalaCheckPropertyChecks with Matchers // We don't care about block txs in this test ledger.blockExecution.executeBlockTransactions( proDaoBlock.copy(body = proDaoBlock.body.copy(transactionList = Seq.empty)), - parentBlockHeader + initialWorld ) } diff --git a/src/test/scala/io/iohk/ethereum/ledger/LedgerTestSetup.scala b/src/test/scala/io/iohk/ethereum/ledger/LedgerTestSetup.scala index f852918e40..a2ff95b777 100644 --- a/src/test/scala/io/iohk/ethereum/ledger/LedgerTestSetup.scala +++ b/src/test/scala/io/iohk/ethereum/ledger/LedgerTestSetup.scala @@ -12,7 +12,7 @@ import io.iohk.ethereum.consensus.{GetBlockHeaderByHash, GetNBlocksBack, TestCon import io.iohk.ethereum.crypto.{generateKeyPair, kec256} import io.iohk.ethereum.domain._ import io.iohk.ethereum.ledger.BlockExecutionError.ValidationAfterExecError -import io.iohk.ethereum.ledger.Ledger.{PC, PR, VMImpl} +import io.iohk.ethereum.ledger.Ledger.{BlockResult, PC, PR, VMImpl} import io.iohk.ethereum.mpt.MerklePatriciaTrie import io.iohk.ethereum.security.SecureRandomBuilder import io.iohk.ethereum.utils.Config.SyncConfig @@ -82,14 +82,15 @@ trait TestSetup extends SecureRandomBuilder with EphemBlockchainTestSetup { val defaultGasLimit: UInt256 = 1000000 val defaultValue: BigInt = 1000 - val emptyWorld: InMemoryWorldStateProxy = BlockchainImpl(storagesInstance.storages) - .getWorldStateProxy( - -1, - UInt256.Zero, - ByteString(MerklePatriciaTrie.EmptyRootHash), - noEmptyAccounts = false, - ethCompatibleStorage = true - ) + val emptyWorld: InMemoryWorldStateProxy = InMemoryWorldStateProxy( + storagesInstance.storages.evmCodeStorage, + blockchain.getBackingStorage(-1), + blockchain, + UInt256.Zero, + ByteString(MerklePatriciaTrie.EmptyRootHash), + noEmptyAccounts = false, + ethCompatibleStorage = true + ) val worldWithMinerAndOriginAccounts: InMemoryWorldStateProxy = InMemoryWorldStateProxy.persistState( emptyWorld @@ -134,13 +135,16 @@ trait TestSetup extends SecureRandomBuilder with EphemBlockchainTestSetup { blockchainStorages: BlockchainStorages, changes: Seq[(Address, Changes)] ): ByteString = { - val initialWorld = BlockchainImpl(blockchainStorages).getWorldStateProxy( - -1, + val initialWorld = InMemoryWorldStateProxy( + storagesInstance.storages.evmCodeStorage, + blockchain.getBackingStorage(-1), + blockchain, UInt256.Zero, stateRootHash, noEmptyAccounts = false, ethCompatibleStorage = true ) + val newWorld = changes.foldLeft[InMemoryWorldStateProxy](initialWorld) { case (recWorld, (address, change)) => change match { case UpdateBalance(balanceIncrease) => @@ -233,15 +237,9 @@ trait DaoForkTestSetup extends TestSetup with MockFactory { (testBlockchain.getBlockHeaderByHash _) .expects(proDaoBlock.header.parentHash) .returning(Some(parentBlockHeader)) - (testBlockchain.getWorldStateProxy _) - .expects( - proDaoBlock.header.number, - proDaoBlockchainConfig.accountStartNonce, - Fixtures.Blocks.DaoParentBlock.header.stateRoot, - false, - true - ) - .returning(worldState) + (testBlockchain.getBackingStorage _) + .expects(*) + .returning(storagesInstance.storages.stateStorage.getBackingStorage(1920000)) } trait BinarySimulationChopSetup { @@ -270,16 +268,53 @@ trait TestSetupWithVmAndValidators extends EphemBlockchainTestSetup { implicit val schedulerContext: Scheduler = Scheduler.fixedPool("ledger-test-pool", 4) - class TestLedgerImpl(validators: Validators)(implicit testContext: Scheduler) + class FailingTestLedgerImpl(validators: Validators)(implicit testContext: Scheduler) extends LedgerImpl( blockchain, + storagesInstance.storages.evmCodeStorage, blockQueue, blockchainConfig, consensus.withValidators(validators).withVM(new Mocks.MockVM()), testContext ) + class TestLedgerImpl(validators: Validators)(implicit testContext: Scheduler) + extends LedgerImpl( + blockchain, + storagesInstance.storages.evmCodeStorage, + blockQueue, + blockchainConfig, + consensus.withValidators(validators).withVM(new Mocks.MockVM()), + testContext + ) { + override private[ledger] lazy val blockExecution = + new BlockExecution( + blockchain, + storagesInstance.storages.evmCodeStorage, + blockchainConfig, + consensus.blockPreparator, + blockValidation + ) { + override def executeAndValidateBlock( + block: Block, + alreadyValidated: Boolean = false + ): Either[BlockExecutionError, Seq[Receipt]] = { + val emptyWorld = InMemoryWorldStateProxy( + storagesInstance.storages.evmCodeStorage, + blockchain.getBackingStorage(-1), + blockchain, + blockchainConfig.accountStartNonce, + ByteString(MerklePatriciaTrie.EmptyRootHash), + noEmptyAccounts = false, + ethCompatibleStorage = true + ) + Right(BlockResult(emptyWorld).receipts) + } + } + } + override lazy val ledger = new TestLedgerImpl(successValidators) + lazy val failingLedger = new FailingTestLedgerImpl(successValidators) def randomHash(): ByteString = ObjectGenerators.byteStringOfLengthNGen(32).sample.get diff --git a/src/test/scala/io/iohk/ethereum/ledger/StxLedgerSpec.scala b/src/test/scala/io/iohk/ethereum/ledger/StxLedgerSpec.scala index 14a867cc75..e46e3bc2bc 100644 --- a/src/test/scala/io/iohk/ethereum/ledger/StxLedgerSpec.scala +++ b/src/test/scala/io/iohk/ethereum/ledger/StxLedgerSpec.scala @@ -81,7 +81,12 @@ class StxLedgerSpec extends AnyFlatSpec with Matchers with Logger { val newBlock: Block = genesisBlock.copy(header = block.header.copy(number = 1, parentHash = genesisHash)) val preparedBlock: Ledger.PreparedBlock = - consensus.blockPreparator.prepareBlock(newBlock, genesisBlock.header, None) + consensus.blockPreparator.prepareBlock( + storagesInstance.storages.evmCodeStorage, + newBlock, + genesisBlock.header, + None + ) val preparedWorld: InMemoryWorldStateProxy = preparedBlock.updatedWorld val header: BlockHeader = preparedBlock.block.header.copy(number = 1, stateRoot = preparedBlock.stateRootHash) @@ -143,11 +148,14 @@ trait ScenarioSetup extends EphemBlockchainTestSetup { treasuryAddress = Address(0) ) - override lazy val stxLedger = new StxLedger(blockchain, blockchainConfig, consensus.blockPreparator) + override lazy val stxLedger = + new StxLedger(blockchain, storagesInstance.storages.evmCodeStorage, blockchainConfig, consensus.blockPreparator) val emptyWorld: InMemoryWorldStateProxy = - blockchain.getWorldStateProxy( - -1, + InMemoryWorldStateProxy( + storagesInstance.storages.evmCodeStorage, + blockchain.getBackingStorage(-1), + blockchain, UInt256.Zero, ByteString(MerklePatriciaTrie.EmptyRootHash), noEmptyAccounts = false, From 2678ffb65f957bb35f6c72cee4e34f5516fe7be7 Mon Sep 17 00:00:00 2001 From: Leonor Boga Date: Wed, 23 Jun 2021 09:01:41 +0200 Subject: [PATCH 2/6] ETCM-944 Rename test class to SchedulerStateSpec --- .../blockchain/sync/SchedulerStateSpec.scala | 42 +++++++++++++++++++ 1 file changed, 42 insertions(+) create mode 100644 src/test/scala/io/iohk/ethereum/blockchain/sync/SchedulerStateSpec.scala diff --git a/src/test/scala/io/iohk/ethereum/blockchain/sync/SchedulerStateSpec.scala b/src/test/scala/io/iohk/ethereum/blockchain/sync/SchedulerStateSpec.scala new file mode 100644 index 0000000000..9e7e7487c2 --- /dev/null +++ b/src/test/scala/io/iohk/ethereum/blockchain/sync/SchedulerStateSpec.scala @@ -0,0 +1,42 @@ +package io.iohk.ethereum.blockchain.sync + +import akka.util.ByteString +import io.iohk.ethereum.blockchain.sync.fast.SyncStateScheduler.{SchedulerState, StateNode, StateNodeRequest} +import org.scalatest.flatspec.AnyFlatSpec +import org.scalatest.matchers.must.Matchers + +class SchedulerStateSpec extends AnyFlatSpec with Matchers { + "SchedulerState" should "schedule node hashes for retrieval" in new TestSetup { + val stateWithRequest = schedulerState.schedule(request1) + assert(stateWithRequest != schedulerState) + assert(stateWithRequest.getPendingRequestByHash(request1.nodeHash).contains(request1)) + } + + it should "return enqueued elements in depth order" in new TestSetup { + val stateWithRequests = schedulerState.schedule(request2).schedule(request3).schedule(request1).schedule(request4) + assert(stateWithRequests != schedulerState) + val (allMissingElements, newState) = stateWithRequests.getAllMissingHashes + assert(allMissingElements == reqestsInDepthOrder.map(_.nodeHash)) + val (allMissingElements1, newState1) = newState.getAllMissingHashes + assert(allMissingElements1.isEmpty) + } + + it should "return at most n enqueued elements in depth order" in new TestSetup { + val stateWithRequests = schedulerState.schedule(request2).schedule(request3).schedule(request1).schedule(request4) + assert(stateWithRequests != schedulerState) + val (twoMissingElements, newState) = stateWithRequests.getMissingHashes(2) + assert(twoMissingElements == reqestsInDepthOrder.take(2).map(_.nodeHash)) + val (allMissingElements1, newState1) = newState.getAllMissingHashes + assert(allMissingElements1.size == 2) + } + + trait TestSetup extends EphemBlockchainTestSetup { + val schedulerState = SchedulerState() + val request1 = StateNodeRequest(ByteString(1), None, StateNode, Seq(), 1, 0) + val request2 = StateNodeRequest(ByteString(2), None, StateNode, Seq(), 2, 0) + val request3 = StateNodeRequest(ByteString(3), None, StateNode, Seq(), 3, 0) + val request4 = StateNodeRequest(ByteString(4), None, StateNode, Seq(), 4, 0) + + val reqestsInDepthOrder = List(request4, request3, request2, request1) + } +} From 671578af897d9fc68df5fbcc72a6c4c4b8ad0f9c Mon Sep 17 00:00:00 2001 From: Leonor Boga Date: Wed, 23 Jun 2021 09:53:56 +0200 Subject: [PATCH 3/6] ETCM-944 Fix tests --- .../ethereum/ledger/BlockImporterItSpec.scala | 5 +-- .../ethereum/txExecTest/ContractTest.scala | 13 +++--- .../ethereum/txExecTest/ECIP1017Test.scala | 15 ++----- .../iohk/ethereum/txExecTest/ForksTest.scala | 12 +----- .../ethereum/txExecTest/ScenarioSetup.scala | 5 +-- .../txExecTest/util/FixtureProvider.scala | 6 ++- .../blockchain/sync/StateSyncSpec.scala | 5 +-- .../blockchain/sync/StateSyncUtils.scala | 2 +- .../sync/SyncSchedulerStateSpec.scala | 42 ------------------- .../consensus/pow/PoWConsensusSpec.scala | 2 +- .../ethereum/ledger/BlockValidationSpec.scala | 11 ++--- .../ethereum/ledger/LedgerTestSetup.scala | 4 +- 12 files changed, 29 insertions(+), 93 deletions(-) delete mode 100644 src/test/scala/io/iohk/ethereum/blockchain/sync/SyncSchedulerStateSpec.scala diff --git a/src/it/scala/io/iohk/ethereum/ledger/BlockImporterItSpec.scala b/src/it/scala/io/iohk/ethereum/ledger/BlockImporterItSpec.scala index f0fba2cb7e..a8a792a019 100644 --- a/src/it/scala/io/iohk/ethereum/ledger/BlockImporterItSpec.scala +++ b/src/it/scala/io/iohk/ethereum/ledger/BlockImporterItSpec.scala @@ -141,7 +141,7 @@ class BlockImporterItSpec "BlockImporter" should "not discard blocks of the main chain if the reorganisation failed" in { //ledger with not mocked blockExecution - val ledger = new TestLedgerImpl(successValidators) + val ledger = new TestLedgerImplNotMockedBlockExecution(successValidators) val blockImporter = system.actorOf( BlockImporter.props( fetcherProbe.ref, @@ -253,7 +253,7 @@ class BlockImporterItSpec val newBlock: Block = getBlock(genesisBlock.number + 5, difficulty = 104, parent = parent.header.hash) val invalidBlock = newBlock.copy(header = newBlock.header.copy(beneficiary = Address(111).bytes)) - val ledger = new TestLedgerImpl(successValidators) + val ledger = new TestLedgerImplNotMockedBlockExecution(successValidators) val blockImporter = system.actorOf( BlockImporter.props( fetcherProbe.ref, @@ -271,7 +271,6 @@ class BlockImporterItSpec blockImporter ! BlockFetcher.PickedBlocks(NonEmptyList.fromListUnsafe(List(invalidBlock))) eventually { - val msg = fetcherProbe .fishForMessage(Timeouts.longTimeout) { case BlockFetcher.FetchStateNode(_, _) => true diff --git a/src/it/scala/io/iohk/ethereum/txExecTest/ContractTest.scala b/src/it/scala/io/iohk/ethereum/txExecTest/ContractTest.scala index 66c7eb1d2a..f4aadbd764 100644 --- a/src/it/scala/io/iohk/ethereum/txExecTest/ContractTest.scala +++ b/src/it/scala/io/iohk/ethereum/txExecTest/ContractTest.scala @@ -1,25 +1,22 @@ package io.iohk.ethereum.txExecTest -import java.util.concurrent.Executors -import io.iohk.ethereum.domain.{BlockchainImpl, BlockchainStorages, Receipt} -import io.iohk.ethereum.ledger.{BlockExecution, BlockQueue, BlockValidation, Ledger} +import io.iohk.ethereum.domain.{BlockchainImpl, Receipt} +import io.iohk.ethereum.ledger.{BlockExecution, BlockQueue, BlockValidation} import io.iohk.ethereum.txExecTest.util.FixtureProvider import io.iohk.ethereum.utils.Config import org.scalatest.flatspec.AnyFlatSpec import org.scalatest.matchers.should.Matchers -import scala.concurrent.ExecutionContext - class ContractTest extends AnyFlatSpec with Matchers { val blockchainConfig = Config.blockchains.blockchainConfig val syncConfig = Config.SyncConfig(Config.config) - val vm = new Ledger.VMImpl - val ec = ExecutionContext.fromExecutor(Executors.newFixedThreadPool(4)) val noErrors = a[Right[_, Seq[Receipt]]] "Ledger" should "execute and validate" in new ScenarioSetup { val fixtures: FixtureProvider.Fixture = FixtureProvider.loadFixtures("/txExecTest/purchaseContract") - val testBlockchainStorages = FixtureProvider.prepareStorages(2, fixtures) + lazy val testBlockchainStorages = FixtureProvider.prepareStorages(2, fixtures) + + override lazy val blockchain = BlockchainImpl(testBlockchainStorages) //block only with ether transfers val blockValidation = new BlockValidation(consensus, blockchain, BlockQueue(blockchain, syncConfig)) diff --git a/src/it/scala/io/iohk/ethereum/txExecTest/ECIP1017Test.scala b/src/it/scala/io/iohk/ethereum/txExecTest/ECIP1017Test.scala index ee7df37759..ceb71b1dd8 100644 --- a/src/it/scala/io/iohk/ethereum/txExecTest/ECIP1017Test.scala +++ b/src/it/scala/io/iohk/ethereum/txExecTest/ECIP1017Test.scala @@ -1,15 +1,12 @@ package io.iohk.ethereum.txExecTest -import java.util.concurrent.Executors -import io.iohk.ethereum.domain.{Address, BlockchainImpl, Receipt, UInt256} -import io.iohk.ethereum.ledger._ +import io.iohk.ethereum.domain.{Address, Receipt, UInt256} +import io.iohk.ethereum.ledger.{BlockExecution, BlockQueue, BlockValidation} import io.iohk.ethereum.txExecTest.util.FixtureProvider import io.iohk.ethereum.utils.{BlockchainConfig, ForkBlockNumbers, MonetaryPolicyConfig} import org.scalatest.flatspec.AnyFlatSpec import org.scalatest.matchers.should.Matchers -import scala.concurrent.ExecutionContext - class ECIP1017Test extends AnyFlatSpec with Matchers { val EraDuration = 3 @@ -53,13 +50,9 @@ class ECIP1017Test extends AnyFlatSpec with Matchers { gasTieBreaker = false, treasuryAddress = Address(0) ) - val ec = ExecutionContext.fromExecutor(Executors.newFixedThreadPool(4)) - val noErrors = a[Right[_, Seq[Receipt]]] } - val vm = new Ledger.VMImpl - /** * Tests the block reward calculation through out all the monetary policy through all the eras till block * mining reward goes to zero. Block mining reward is tested till era 200 (that starts at block number 602) @@ -72,11 +65,9 @@ class ECIP1017Test extends AnyFlatSpec with Matchers { val startBlock = 1 val endBlock = 602 - protected val testBlockchainStorages = FixtureProvider.prepareStorages(startBlock, fixtures) + protected val testBlockchainStorages = FixtureProvider.prepareStorages(endBlock, fixtures) (startBlock to endBlock) foreach { blockToExecute => - val storages = FixtureProvider.prepareStorages(blockToExecute - 1, fixtures) - val blockchain = BlockchainImpl(storages) val blockValidation = new BlockValidation(consensus, blockchain, BlockQueue(blockchain, syncConfig)) val blockExecution = new BlockExecution( blockchain, diff --git a/src/it/scala/io/iohk/ethereum/txExecTest/ForksTest.scala b/src/it/scala/io/iohk/ethereum/txExecTest/ForksTest.scala index fb3cfd0979..6318796c0a 100644 --- a/src/it/scala/io/iohk/ethereum/txExecTest/ForksTest.scala +++ b/src/it/scala/io/iohk/ethereum/txExecTest/ForksTest.scala @@ -1,16 +1,12 @@ package io.iohk.ethereum.txExecTest -import java.util.concurrent.Executors -import io.iohk.ethereum.domain.{Address, BlockchainImpl, Receipt, UInt256} +import io.iohk.ethereum.domain.{Address, Receipt, UInt256} import io.iohk.ethereum.ledger.{BlockExecution, BlockQueue, BlockValidation} import io.iohk.ethereum.txExecTest.util.FixtureProvider import io.iohk.ethereum.utils.{BlockchainConfig, ForkBlockNumbers, MonetaryPolicyConfig} import org.scalatest.flatspec.AnyFlatSpec import org.scalatest.matchers.should.Matchers -import scala.concurrent.ExecutionContext - -// scalastyle:off magic.number class ForksTest extends AnyFlatSpec with Matchers { trait TestSetup extends ScenarioSetup { @@ -52,9 +48,7 @@ class ForksTest extends AnyFlatSpec with Matchers { ethCompatibleStorage = true, treasuryAddress = Address(0) ) - val noErrors = a[Right[_, Seq[Receipt]]] - val ec = ExecutionContext.fromExecutor(Executors.newFixedThreadPool(4)) } "Ledger" should "execute blocks with respect to forks" in new TestSetup { @@ -63,11 +57,9 @@ class ForksTest extends AnyFlatSpec with Matchers { val startBlock = 1 val endBlock = 11 - protected val testBlockchainStorages = FixtureProvider.prepareStorages(startBlock, fixtures) + protected val testBlockchainStorages = FixtureProvider.prepareStorages(endBlock, fixtures) (startBlock to endBlock) foreach { blockToExecute => - val storages = FixtureProvider.prepareStorages(blockToExecute - 1, fixtures) - val blockchain = BlockchainImpl(storages) val blockValidation = new BlockValidation(consensus, blockchain, BlockQueue(blockchain, syncConfig)) val blockExecution = new BlockExecution( blockchain, diff --git a/src/it/scala/io/iohk/ethereum/txExecTest/ScenarioSetup.scala b/src/it/scala/io/iohk/ethereum/txExecTest/ScenarioSetup.scala index bb47d54797..4ca7a15366 100644 --- a/src/it/scala/io/iohk/ethereum/txExecTest/ScenarioSetup.scala +++ b/src/it/scala/io/iohk/ethereum/txExecTest/ScenarioSetup.scala @@ -1,12 +1,11 @@ package io.iohk.ethereum.txExecTest -import io.iohk.ethereum.blockchain.sync +import io.iohk.ethereum.blockchain.sync.EphemBlockchainTestSetup import io.iohk.ethereum.domain.{BlockchainImpl, BlockchainStorages} import io.iohk.ethereum.ledger.Ledger.VMImpl -trait ScenarioSetup extends sync.ScenarioSetup { +trait ScenarioSetup extends EphemBlockchainTestSetup { protected val testBlockchainStorages: BlockchainStorages - override lazy val blockchain: BlockchainImpl = BlockchainImpl(testBlockchainStorages) override lazy val vm: VMImpl = new VMImpl } diff --git a/src/it/scala/io/iohk/ethereum/txExecTest/util/FixtureProvider.scala b/src/it/scala/io/iohk/ethereum/txExecTest/util/FixtureProvider.scala index ed14effedc..287d8a1d31 100644 --- a/src/it/scala/io/iohk/ethereum/txExecTest/util/FixtureProvider.scala +++ b/src/it/scala/io/iohk/ethereum/txExecTest/util/FixtureProvider.scala @@ -205,6 +205,8 @@ object FixtureProvider { ) } - private def withClose[A, B <: Closeable](closeable: B)(f: B => A): A = try { f(closeable) } - finally { closeable.close() } + private def withClose[A, B <: Closeable](closeable: B)(f: B => A): A = { + try { f(closeable) } + finally { closeable.close() } + } } 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 f6e4c03847..2eda43b956 100644 --- a/src/test/scala/io/iohk/ethereum/blockchain/sync/StateSyncSpec.scala +++ b/src/test/scala/io/iohk/ethereum/blockchain/sync/StateSyncSpec.scala @@ -8,7 +8,6 @@ import akka.testkit.TestActor.AutoPilot import akka.testkit.{TestKit, TestProbe} import akka.util.ByteString import io.iohk.ethereum.blockchain.sync.StateSyncUtils.{MptNodeData, TrieProvider} -import io.iohk.ethereum.blockchain.sync.fast.SyncStateSchedulerActor._ import io.iohk.ethereum.blockchain.sync.fast.{SyncStateScheduler, SyncStateSchedulerActor} import io.iohk.ethereum.blockchain.sync.fast.SyncStateSchedulerActor.{ RestartRequested, @@ -17,7 +16,6 @@ import io.iohk.ethereum.blockchain.sync.fast.SyncStateSchedulerActor.{ StateSyncStats, WaitingForNewTargetBlock } -import io.iohk.ethereum.db.dataSource.RocksDbDataSource.IterationError import io.iohk.ethereum.domain.{Address, BlockchainImpl, ChainWeight} import io.iohk.ethereum.network.EtcPeerManagerActor._ import io.iohk.ethereum.network.PeerEventBusActor.PeerEvent.MessageFromPeer @@ -28,7 +26,6 @@ import io.iohk.ethereum.network.{Peer, PeerId} import io.iohk.ethereum.utils.Config import io.iohk.ethereum.{Fixtures, ObjectGenerators, WithActorSystemShutDown} import monix.execution.Scheduler -import monix.reactive.Observable import org.scalactic.anyvals.PosInt import org.scalatest.BeforeAndAfterAll import org.scalatest.flatspec.AnyFlatSpecLike @@ -154,7 +151,7 @@ class StateSyncSpec bestBlockHash = peerStatus.bestHash ) - val trieProvider = new TrieProvider(blockchain, storagesInstance.storages.evmCodeStorage, blockchainConfig) + val trieProvider = new TrieProvider(blockchain, getNewStorages.storages.evmCodeStorage, blockchainConfig) val peersMap = (1 to 8).map { i => ( diff --git a/src/test/scala/io/iohk/ethereum/blockchain/sync/StateSyncUtils.scala b/src/test/scala/io/iohk/ethereum/blockchain/sync/StateSyncUtils.scala index 8dce4d4d3c..7c5ccb9eda 100644 --- a/src/test/scala/io/iohk/ethereum/blockchain/sync/StateSyncUtils.scala +++ b/src/test/scala/io/iohk/ethereum/blockchain/sync/StateSyncUtils.scala @@ -33,7 +33,7 @@ object StateSyncUtils extends EphemBlockchainTestSetup { def buildWorld(accountData: Seq[MptNodeData], existingTree: Option[ByteString] = None): ByteString = { val init = InMemoryWorldStateProxy( - storagesInstance.storages.evmCodeStorage, + evmCodeStorage, blockchain.getBackingStorage(1), blockchain, blockchainConfig.accountStartNonce, diff --git a/src/test/scala/io/iohk/ethereum/blockchain/sync/SyncSchedulerStateSpec.scala b/src/test/scala/io/iohk/ethereum/blockchain/sync/SyncSchedulerStateSpec.scala deleted file mode 100644 index 3fe747163c..0000000000 --- a/src/test/scala/io/iohk/ethereum/blockchain/sync/SyncSchedulerStateSpec.scala +++ /dev/null @@ -1,42 +0,0 @@ -package io.iohk.ethereum.blockchain.sync - -import akka.util.ByteString -import io.iohk.ethereum.blockchain.sync.fast.SyncStateScheduler.{SchedulerState, StateNode, StateNodeRequest} -import org.scalatest.flatspec.AnyFlatSpec -import org.scalatest.matchers.must.Matchers - -class SyncSchedulerStateSpec extends AnyFlatSpec with Matchers { - "SyncSchedulerState" should "schedule node hashes for retrieval" in new TestSetup { - val stateWithRequest = schedulerState.schedule(request1) - assert(stateWithRequest != schedulerState) - assert(stateWithRequest.getPendingRequestByHash(request1.nodeHash).contains(request1)) - } - - it should "return enqueued elements in depth order" in new TestSetup { - val stateWithRequests = schedulerState.schedule(request2).schedule(request3).schedule(request1).schedule(request4) - assert(stateWithRequests != schedulerState) - val (allMissingElements, newState) = stateWithRequests.getAllMissingHashes - assert(allMissingElements == reqestsInDepthOrder.map(_.nodeHash)) - val (allMissingElements1, newState1) = newState.getAllMissingHashes - assert(allMissingElements1.isEmpty) - } - - it should "return at most n enqueued elements in depth order" in new TestSetup { - val stateWithRequests = schedulerState.schedule(request2).schedule(request3).schedule(request1).schedule(request4) - assert(stateWithRequests != schedulerState) - val (twoMissingElements, newState) = stateWithRequests.getMissingHashes(2) - assert(twoMissingElements == reqestsInDepthOrder.take(2).map(_.nodeHash)) - val (allMissingElements1, newState1) = newState.getAllMissingHashes - assert(allMissingElements1.size == 2) - } - - trait TestSetup extends EphemBlockchainTestSetup { - val schedulerState = SchedulerState() - val request1 = StateNodeRequest(ByteString(1), None, StateNode, Seq(), 1, 0) - val request2 = StateNodeRequest(ByteString(2), None, StateNode, Seq(), 2, 0) - val request3 = StateNodeRequest(ByteString(3), None, StateNode, Seq(), 3, 0) - val request4 = StateNodeRequest(ByteString(4), None, StateNode, Seq(), 4, 0) - - val reqestsInDepthOrder = List(request4, request3, request2, request1) - } -} diff --git a/src/test/scala/io/iohk/ethereum/consensus/pow/PoWConsensusSpec.scala b/src/test/scala/io/iohk/ethereum/consensus/pow/PoWConsensusSpec.scala index 1d9d6a439c..68e5376130 100644 --- a/src/test/scala/io/iohk/ethereum/consensus/pow/PoWConsensusSpec.scala +++ b/src/test/scala/io/iohk/ethereum/consensus/pow/PoWConsensusSpec.scala @@ -129,7 +129,7 @@ class PoWConsensusSpec powConsensus.minerCoordinatorRef.isDefined shouldBe true } - trait TestSetup extends ScenarioSetup with MockFactory { + trait TestSetup extends EphemBlockchainTestSetup with MockFactory { override lazy val blockchain: BlockchainImpl = mock[BlockchainImpl] val evmCodeStorage: EvmCodeStorage = mock[EvmCodeStorage] val validator: ValidatorsExecutor = successValidators.asInstanceOf[ValidatorsExecutor] diff --git a/src/test/scala/io/iohk/ethereum/ledger/BlockValidationSpec.scala b/src/test/scala/io/iohk/ethereum/ledger/BlockValidationSpec.scala index 0a59620006..751bd62058 100644 --- a/src/test/scala/io/iohk/ethereum/ledger/BlockValidationSpec.scala +++ b/src/test/scala/io/iohk/ethereum/ledger/BlockValidationSpec.scala @@ -11,26 +11,27 @@ import org.scalatest.wordspec.AnyWordSpec import io.iohk.ethereum.utils.ByteStringUtils._ class BlockValidationSpec extends AnyWordSpec with Matchers with MockFactory { + import BlockValidationTestSetup._ "BlockValidation" should { "validate block after execution" when { - "report valid results from execution as correct" in new BlockValidationTestSetup { + "report valid results from execution as correct" in { blockValidation.validateBlockAfterExecution(block, stateRootHash, receipts, gasUsed) shouldBe Right( BlockExecutionSuccess ) } - "report as invalid a block that doesn't have the correct gas used" in new BlockValidationTestSetup { + "report as invalid a block that doesn't have the correct gas used" in { val invalidGasUsed: BigInt = gasUsed + 1 blockValidation.validateBlockAfterExecution(block, stateRootHash, receipts, invalidGasUsed).isLeft shouldBe true } - "report as invalid a block that doesn't have the correct state root hash" in new BlockValidationTestSetup { + "report as invalid a block that doesn't have the correct state root hash" in { val invalidStateRootHash: ByteString = concatByteStrings((stateRootHash.head + 1).toByte, stateRootHash.tail) blockValidation.validateBlockAfterExecution(block, invalidStateRootHash, receipts, gasUsed).isLeft shouldBe true } - "report as invalid a block that doesn't have the correct receipts information" in new BlockValidationTestSetup { + "report as invalid a block that doesn't have the correct receipts information" in { val invalidReceipts: Seq[Receipt] = Seq.empty[Receipt] blockValidation.validateBlockAfterExecution(block, stateRootHash, invalidReceipts, gasUsed).isLeft shouldBe true } @@ -38,7 +39,7 @@ class BlockValidationSpec extends AnyWordSpec with Matchers with MockFactory { } // scalastyle:off magic.number - trait BlockValidationTestSetup { + object BlockValidationTestSetup { private val setup = new io.iohk.ethereum.blockchain.sync.ScenarioSetup { override lazy val blockchain: BlockchainImpl = mock[BlockchainImpl] diff --git a/src/test/scala/io/iohk/ethereum/ledger/LedgerTestSetup.scala b/src/test/scala/io/iohk/ethereum/ledger/LedgerTestSetup.scala index a2ff95b777..2ab7f0c694 100644 --- a/src/test/scala/io/iohk/ethereum/ledger/LedgerTestSetup.scala +++ b/src/test/scala/io/iohk/ethereum/ledger/LedgerTestSetup.scala @@ -268,7 +268,7 @@ trait TestSetupWithVmAndValidators extends EphemBlockchainTestSetup { implicit val schedulerContext: Scheduler = Scheduler.fixedPool("ledger-test-pool", 4) - class FailingTestLedgerImpl(validators: Validators)(implicit testContext: Scheduler) + class TestLedgerImplNotMockedBlockExecution(validators: Validators)(implicit testContext: Scheduler) extends LedgerImpl( blockchain, storagesInstance.storages.evmCodeStorage, @@ -314,7 +314,7 @@ trait TestSetupWithVmAndValidators extends EphemBlockchainTestSetup { } override lazy val ledger = new TestLedgerImpl(successValidators) - lazy val failingLedger = new FailingTestLedgerImpl(successValidators) + lazy val failingLedger = new TestLedgerImplNotMockedBlockExecution(successValidators) def randomHash(): ByteString = ObjectGenerators.byteStringOfLengthNGen(32).sample.get From 52d1b3cf189a632d6dd0eb3af07b6c72ba8db134 Mon Sep 17 00:00:00 2001 From: Leonor Boga Date: Wed, 23 Jun 2021 16:07:42 +0200 Subject: [PATCH 4/6] ETCM-944 Pass getBlockHashByNumber field directly to InMemoryWorldStateProxy --- .../iohk/ethereum/ledger/BlockImporterItSpec.scala | 2 +- .../io/iohk/ethereum/sync/util/CommonFakePeer.scala | 2 +- .../ethereum/sync/util/RegularSyncItSpecUtils.scala | 2 +- src/main/resources/conf/base.conf | 2 +- .../io/iohk/ethereum/jsonrpc/EthUserService.scala | 2 +- .../io/iohk/ethereum/ledger/BlockExecution.scala | 2 +- .../io/iohk/ethereum/ledger/BlockPreparator.scala | 2 +- .../ethereum/ledger/InMemoryWorldStateProxy.scala | 4 ++-- .../scala/io/iohk/ethereum/ledger/StxLedger.scala | 2 +- .../ethereum/blockchain/sync/StateSyncUtils.scala | 2 +- .../iohk/ethereum/jsonrpc/EthBlocksServiceSpec.scala | 2 +- .../iohk/ethereum/jsonrpc/EthInfoServiceSpec.scala | 2 +- .../iohk/ethereum/jsonrpc/EthMiningServiceSpec.scala | 2 +- .../ethereum/jsonrpc/JsonRpcControllerFixture.scala | 2 +- .../io/iohk/ethereum/ledger/BlockRewardSpec.scala | 2 +- .../io/iohk/ethereum/ledger/DeleteAccountsSpec.scala | 2 +- .../ethereum/ledger/DeleteTouchedAccountsSpec.scala | 4 ++-- .../ledger/InMemoryWorldStateProxySpec.scala | 12 ++++++------ .../io/iohk/ethereum/ledger/LedgerTestSetup.scala | 6 +++--- .../io/iohk/ethereum/ledger/StxLedgerSpec.scala | 2 +- 20 files changed, 29 insertions(+), 29 deletions(-) diff --git a/src/it/scala/io/iohk/ethereum/ledger/BlockImporterItSpec.scala b/src/it/scala/io/iohk/ethereum/ledger/BlockImporterItSpec.scala index a8a792a019..06b12fb1fa 100644 --- a/src/it/scala/io/iohk/ethereum/ledger/BlockImporterItSpec.scala +++ b/src/it/scala/io/iohk/ethereum/ledger/BlockImporterItSpec.scala @@ -62,7 +62,7 @@ class BlockImporterItSpec val emptyWorld = InMemoryWorldStateProxy( storagesInstance.storages.evmCodeStorage, blockchain.getBackingStorage(-1), - blockchain, + (number: BigInt) => blockchain.getBlockHeaderByNumber(number).map(_.hash), blockchainConfig.accountStartNonce, ByteString(MerklePatriciaTrie.EmptyRootHash), noEmptyAccounts = false, diff --git a/src/it/scala/io/iohk/ethereum/sync/util/CommonFakePeer.scala b/src/it/scala/io/iohk/ethereum/sync/util/CommonFakePeer.scala index ff9eb57650..f6d1c15d13 100644 --- a/src/it/scala/io/iohk/ethereum/sync/util/CommonFakePeer.scala +++ b/src/it/scala/io/iohk/ethereum/sync/util/CommonFakePeer.scala @@ -244,7 +244,7 @@ abstract class CommonFakePeer(peerName: String, fakePeerCustomConfig: FakePeerCu InMemoryWorldStateProxy( storagesInstance.storages.evmCodeStorage, bl.getBackingStorage(block.number), - bl, + (number: BigInt) => bl.getBlockHeaderByNumber(number).map(_.hash), blockchainConfig.accountStartNonce, block.header.stateRoot, noEmptyAccounts = EvmConfig.forBlock(block.number, blockchainConfig).noEmptyAccounts, 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 13501b9f87..ccb98240e5 100644 --- a/src/it/scala/io/iohk/ethereum/sync/util/RegularSyncItSpecUtils.scala +++ b/src/it/scala/io/iohk/ethereum/sync/util/RegularSyncItSpecUtils.scala @@ -215,7 +215,7 @@ object RegularSyncItSpecUtils { InMemoryWorldStateProxy( storagesInstance.storages.evmCodeStorage, bl.getBackingStorage(block.number), - bl, + (number: BigInt) => bl.getBlockHeaderByNumber(number).map(_.hash), UInt256.Zero, ByteString(MerklePatriciaTrie.EmptyRootHash), noEmptyAccounts = false, diff --git a/src/main/resources/conf/base.conf b/src/main/resources/conf/base.conf index 8e4a31f21c..a170d25f83 100644 --- a/src/main/resources/conf/base.conf +++ b/src/main/resources/conf/base.conf @@ -386,7 +386,7 @@ mantis { nodes-per-request = 384 # Minimum number of peers required to start fast-sync (by determining the pivot block) - min-peers-to-choose-pivot-block = 3 + min-peers-to-choose-pivot-block = 1 # Number of additional peers used to determine pivot block during fast-sync # Number of peers used to reach consensus = min-peers-to-choose-pivot-block + peers-to-choose-pivot-block-margin diff --git a/src/main/scala/io/iohk/ethereum/jsonrpc/EthUserService.scala b/src/main/scala/io/iohk/ethereum/jsonrpc/EthUserService.scala index 6242154701..d652d72d88 100644 --- a/src/main/scala/io/iohk/ethereum/jsonrpc/EthUserService.scala +++ b/src/main/scala/io/iohk/ethereum/jsonrpc/EthUserService.scala @@ -35,7 +35,7 @@ class EthUserService( val world = InMemoryWorldStateProxy( evmCodeStorage, blockchain.getBackingStorage(block.header.number), - blockchain, + (number: BigInt) => blockchain.getBlockHeaderByNumber(number).map(_.hash), blockchainConfig.accountStartNonce, block.header.stateRoot, noEmptyAccounts = false, diff --git a/src/main/scala/io/iohk/ethereum/ledger/BlockExecution.scala b/src/main/scala/io/iohk/ethereum/ledger/BlockExecution.scala index 31dc6504c5..17e2e07333 100644 --- a/src/main/scala/io/iohk/ethereum/ledger/BlockExecution.scala +++ b/src/main/scala/io/iohk/ethereum/ledger/BlockExecution.scala @@ -65,7 +65,7 @@ class BlockExecution( initialWorld = InMemoryWorldStateProxy( evmCodeStorage = evmCodeStorage, blockchain.getBackingStorage(block.header.number), - blockchain, + (number: BigInt) => blockchain.getBlockHeaderByNumber(number).map(_.hash), accountStartNonce = blockchainConfig.accountStartNonce, stateRootHash = parentHeader.stateRoot, noEmptyAccounts = EvmConfig.forBlock(parentHeader.number, blockchainConfig).noEmptyAccounts, diff --git a/src/main/scala/io/iohk/ethereum/ledger/BlockPreparator.scala b/src/main/scala/io/iohk/ethereum/ledger/BlockPreparator.scala index ca95460c4f..4f0a017a8d 100644 --- a/src/main/scala/io/iohk/ethereum/ledger/BlockPreparator.scala +++ b/src/main/scala/io/iohk/ethereum/ledger/BlockPreparator.scala @@ -382,7 +382,7 @@ class BlockPreparator( InMemoryWorldStateProxy( evmCodeStorage = evmCodeStorage, mptStorage = blockchain.getReadOnlyStorage(), - blockchain = blockchain, + getBlockHashByNumber = (number: BigInt) => blockchain.getBlockHeaderByNumber(number).map(_.hash), accountStartNonce = blockchainConfig.accountStartNonce, stateRootHash = parent.stateRoot, noEmptyAccounts = EvmConfig.forBlock(block.header.number, blockchainConfig).noEmptyAccounts, diff --git a/src/main/scala/io/iohk/ethereum/ledger/InMemoryWorldStateProxy.scala b/src/main/scala/io/iohk/ethereum/ledger/InMemoryWorldStateProxy.scala index 2bc162e430..4e3c808aa5 100644 --- a/src/main/scala/io/iohk/ethereum/ledger/InMemoryWorldStateProxy.scala +++ b/src/main/scala/io/iohk/ethereum/ledger/InMemoryWorldStateProxy.scala @@ -16,7 +16,7 @@ object InMemoryWorldStateProxy { def apply( evmCodeStorage: EvmCodeStorage, mptStorage: MptStorage, - blockchain: Blockchain, + getBlockHashByNumber: BigInt => Option[ByteString], accountStartNonce: UInt256, stateRootHash: ByteString, noEmptyAccounts: Boolean, @@ -25,7 +25,7 @@ object InMemoryWorldStateProxy { evmCodeStorage, mptStorage, accountStartNonce, - (number: BigInt) => blockchain.getBlockHeaderByNumber(number).map(_.hash), + getBlockHashByNumber, stateRootHash, noEmptyAccounts, ethCompatibleStorage diff --git a/src/main/scala/io/iohk/ethereum/ledger/StxLedger.scala b/src/main/scala/io/iohk/ethereum/ledger/StxLedger.scala index 38838f8f7b..8d27bef055 100644 --- a/src/main/scala/io/iohk/ethereum/ledger/StxLedger.scala +++ b/src/main/scala/io/iohk/ethereum/ledger/StxLedger.scala @@ -24,7 +24,7 @@ class StxLedger( InMemoryWorldStateProxy( evmCodeStorage = evmCodeStorage, mptStorage = blockchain.getReadOnlyStorage(), - blockchain = blockchain, + getBlockHashByNumber = (number: BigInt) => blockchain.getBlockHeaderByNumber(number).map(_.hash), accountStartNonce = blockchainConfig.accountStartNonce, stateRootHash = blockHeader.stateRoot, noEmptyAccounts = EvmConfig.forBlock(blockHeader.number, blockchainConfig).noEmptyAccounts, diff --git a/src/test/scala/io/iohk/ethereum/blockchain/sync/StateSyncUtils.scala b/src/test/scala/io/iohk/ethereum/blockchain/sync/StateSyncUtils.scala index 7c5ccb9eda..0aca412563 100644 --- a/src/test/scala/io/iohk/ethereum/blockchain/sync/StateSyncUtils.scala +++ b/src/test/scala/io/iohk/ethereum/blockchain/sync/StateSyncUtils.scala @@ -35,7 +35,7 @@ object StateSyncUtils extends EphemBlockchainTestSetup { val init = InMemoryWorldStateProxy( evmCodeStorage, blockchain.getBackingStorage(1), - blockchain, + (number: BigInt) => blockchain.getBlockHeaderByNumber(number).map(_.hash), blockchainConfig.accountStartNonce, existingTree.getOrElse(ByteString(MerklePatriciaTrie.EmptyRootHash)), noEmptyAccounts = true, diff --git a/src/test/scala/io/iohk/ethereum/jsonrpc/EthBlocksServiceSpec.scala b/src/test/scala/io/iohk/ethereum/jsonrpc/EthBlocksServiceSpec.scala index 26407cd409..08fe56debd 100644 --- a/src/test/scala/io/iohk/ethereum/jsonrpc/EthBlocksServiceSpec.scala +++ b/src/test/scala/io/iohk/ethereum/jsonrpc/EthBlocksServiceSpec.scala @@ -432,7 +432,7 @@ class EthBlocksServiceSpec val fakeWorld = InMemoryWorldStateProxy( storagesInstance.storages.evmCodeStorage, blockchain.getBackingStorage(-1), - blockchain, + (number: BigInt) => blockchain.getBlockHeaderByNumber(number).map(_.hash), UInt256.Zero, ByteString.empty, noEmptyAccounts = false, diff --git a/src/test/scala/io/iohk/ethereum/jsonrpc/EthInfoServiceSpec.scala b/src/test/scala/io/iohk/ethereum/jsonrpc/EthInfoServiceSpec.scala index 6da8285681..9fb4ecfd9a 100644 --- a/src/test/scala/io/iohk/ethereum/jsonrpc/EthInfoServiceSpec.scala +++ b/src/test/scala/io/iohk/ethereum/jsonrpc/EthInfoServiceSpec.scala @@ -98,7 +98,7 @@ class EthServiceSpec val worldStateProxy = InMemoryWorldStateProxy( storagesInstance.storages.evmCodeStorage, blockchain.getBackingStorage(-1), - blockchain, + (number: BigInt) => blockchain.getBlockHeaderByNumber(number).map(_.hash), UInt256.Zero, ByteString.empty, noEmptyAccounts = false, diff --git a/src/test/scala/io/iohk/ethereum/jsonrpc/EthMiningServiceSpec.scala b/src/test/scala/io/iohk/ethereum/jsonrpc/EthMiningServiceSpec.scala index 3bc7199654..03419c305b 100644 --- a/src/test/scala/io/iohk/ethereum/jsonrpc/EthMiningServiceSpec.scala +++ b/src/test/scala/io/iohk/ethereum/jsonrpc/EthMiningServiceSpec.scala @@ -307,7 +307,7 @@ class EthMiningServiceSpec val fakeWorld = InMemoryWorldStateProxy( storagesInstance.storages.evmCodeStorage, blockchain.getReadOnlyStorage(), - blockchain, + (number: BigInt) => blockchain.getBlockHeaderByNumber(number).map(_.hash), UInt256.Zero, ByteString.empty, noEmptyAccounts = false, diff --git a/src/test/scala/io/iohk/ethereum/jsonrpc/JsonRpcControllerFixture.scala b/src/test/scala/io/iohk/ethereum/jsonrpc/JsonRpcControllerFixture.scala index e52625ebd2..2b9812043e 100644 --- a/src/test/scala/io/iohk/ethereum/jsonrpc/JsonRpcControllerFixture.scala +++ b/src/test/scala/io/iohk/ethereum/jsonrpc/JsonRpcControllerFixture.scala @@ -177,7 +177,7 @@ class JsonRpcControllerFixture(implicit system: ActorSystem) val fakeWorld = InMemoryWorldStateProxy( storagesInstance.storages.evmCodeStorage, blockchain.getReadOnlyStorage(), - blockchain, + (number: BigInt) => blockchain.getBlockHeaderByNumber(number).map(_.hash), blockchainConfig.accountStartNonce, ByteString.empty, noEmptyAccounts = false, diff --git a/src/test/scala/io/iohk/ethereum/ledger/BlockRewardSpec.scala b/src/test/scala/io/iohk/ethereum/ledger/BlockRewardSpec.scala index cf6d59fcb6..6320e8c456 100644 --- a/src/test/scala/io/iohk/ethereum/ledger/BlockRewardSpec.scala +++ b/src/test/scala/io/iohk/ethereum/ledger/BlockRewardSpec.scala @@ -194,7 +194,7 @@ class BlockRewardSpec extends AnyFlatSpec with Matchers with ScalaCheckPropertyC val worldState = InMemoryWorldStateProxy( storagesInstance.storages.evmCodeStorage, blockchain.getBackingStorage(-1), - blockchain, + (number: BigInt) => blockchain.getBlockHeaderByNumber(number).map(_.hash), UInt256.Zero, ByteString(MerklePatriciaTrie.EmptyRootHash), noEmptyAccounts = false, diff --git a/src/test/scala/io/iohk/ethereum/ledger/DeleteAccountsSpec.scala b/src/test/scala/io/iohk/ethereum/ledger/DeleteAccountsSpec.scala index ac07be207e..ebdeffa491 100644 --- a/src/test/scala/io/iohk/ethereum/ledger/DeleteAccountsSpec.scala +++ b/src/test/scala/io/iohk/ethereum/ledger/DeleteAccountsSpec.scala @@ -68,7 +68,7 @@ class DeleteAccountsSpec extends AnyFlatSpec with Matchers with MockFactory { val worldStateWithoutPersist = InMemoryWorldStateProxy( storagesInstance.storages.evmCodeStorage, blockchain.getBackingStorage(-1), - blockchain, + (number: BigInt) => blockchain.getBlockHeaderByNumber(number).map(_.hash), UInt256.Zero, ByteString(MerklePatriciaTrie.EmptyRootHash), noEmptyAccounts = false, diff --git a/src/test/scala/io/iohk/ethereum/ledger/DeleteTouchedAccountsSpec.scala b/src/test/scala/io/iohk/ethereum/ledger/DeleteTouchedAccountsSpec.scala index be86000c29..bdb06e8abe 100644 --- a/src/test/scala/io/iohk/ethereum/ledger/DeleteTouchedAccountsSpec.scala +++ b/src/test/scala/io/iohk/ethereum/ledger/DeleteTouchedAccountsSpec.scala @@ -151,7 +151,7 @@ class DeleteTouchedAccountsSpec extends AnyFlatSpec with Matchers { val worldStateWithoutPersist = InMemoryWorldStateProxy( storagesInstance.storages.evmCodeStorage, blockchain.getBackingStorage(-1), - blockchain, + (number: BigInt) => blockchain.getBlockHeaderByNumber(number).map(_.hash), UInt256.Zero, ByteString(MerklePatriciaTrie.EmptyRootHash), noEmptyAccounts = postEip161Config.noEmptyAccounts, @@ -166,7 +166,7 @@ class DeleteTouchedAccountsSpec extends AnyFlatSpec with Matchers { val worldStateWithoutPersistPreEIP161 = InMemoryWorldStateProxy( storagesInstance.storages.evmCodeStorage, blockchain.getBackingStorage(-1), - blockchain, + (number: BigInt) => blockchain.getBlockHeaderByNumber(number).map(_.hash), UInt256.Zero, ByteString(MerklePatriciaTrie.EmptyRootHash), noEmptyAccounts = postEip160Config.noEmptyAccounts, diff --git a/src/test/scala/io/iohk/ethereum/ledger/InMemoryWorldStateProxySpec.scala b/src/test/scala/io/iohk/ethereum/ledger/InMemoryWorldStateProxySpec.scala index 62b2aa1f42..de28cf567e 100644 --- a/src/test/scala/io/iohk/ethereum/ledger/InMemoryWorldStateProxySpec.scala +++ b/src/test/scala/io/iohk/ethereum/ledger/InMemoryWorldStateProxySpec.scala @@ -126,7 +126,7 @@ class InMemoryWorldStateProxySpec extends AnyFlatSpec with Matchers { val newWorldState = InMemoryWorldStateProxy( storagesInstance.storages.evmCodeStorage, blockchain.getBackingStorage(-1), - blockchain, + (number: BigInt) => blockchain.getBlockHeaderByNumber(number).map(_.hash), UInt256.Zero, persistedWorldState.stateRootHash, noEmptyAccounts = true, @@ -258,7 +258,7 @@ class InMemoryWorldStateProxySpec extends AnyFlatSpec with Matchers { val readWorldState = InMemoryWorldStateProxy( storagesInstance.storages.evmCodeStorage, blockchain.getReadOnlyStorage(), - blockchain, + (number: BigInt) => blockchain.getBlockHeaderByNumber(number).map(_.hash), UInt256.Zero, persistedWorldStateWithAnAccount.stateRootHash, noEmptyAccounts = false, @@ -280,7 +280,7 @@ class InMemoryWorldStateProxySpec extends AnyFlatSpec with Matchers { val newReadWorld = InMemoryWorldStateProxy( storagesInstance.storages.evmCodeStorage, blockchain.getReadOnlyStorage(), - blockchain, + (number: BigInt) => blockchain.getBlockHeaderByNumber(number).map(_.hash), UInt256.Zero, changedReadWorld.stateRootHash, noEmptyAccounts = false, @@ -309,7 +309,7 @@ class InMemoryWorldStateProxySpec extends AnyFlatSpec with Matchers { val world2 = InMemoryWorldStateProxy( storagesInstance.storages.evmCodeStorage, blockchain.getBackingStorage(-1), - blockchain, + (number: BigInt) => blockchain.getBlockHeaderByNumber(number).map(_.hash), UInt256.Zero, world1.stateRootHash, noEmptyAccounts = false, @@ -335,7 +335,7 @@ class InMemoryWorldStateProxySpec extends AnyFlatSpec with Matchers { val worldState = InMemoryWorldStateProxy( storagesInstance.storages.evmCodeStorage, blockchain.getBackingStorage(-1), - blockchain, + (number: BigInt) => blockchain.getBlockHeaderByNumber(number).map(_.hash), UInt256.Zero, ByteString(MerklePatriciaTrie.EmptyRootHash), noEmptyAccounts = false, @@ -345,7 +345,7 @@ class InMemoryWorldStateProxySpec extends AnyFlatSpec with Matchers { val postEIP161WorldState = InMemoryWorldStateProxy( storagesInstance.storages.evmCodeStorage, blockchain.getBackingStorage(-1), - blockchain, + (number: BigInt) => blockchain.getBlockHeaderByNumber(number).map(_.hash), UInt256.Zero, ByteString(MerklePatriciaTrie.EmptyRootHash), noEmptyAccounts = postEip161Config.noEmptyAccounts, diff --git a/src/test/scala/io/iohk/ethereum/ledger/LedgerTestSetup.scala b/src/test/scala/io/iohk/ethereum/ledger/LedgerTestSetup.scala index 2ab7f0c694..586510b1f3 100644 --- a/src/test/scala/io/iohk/ethereum/ledger/LedgerTestSetup.scala +++ b/src/test/scala/io/iohk/ethereum/ledger/LedgerTestSetup.scala @@ -85,7 +85,7 @@ trait TestSetup extends SecureRandomBuilder with EphemBlockchainTestSetup { val emptyWorld: InMemoryWorldStateProxy = InMemoryWorldStateProxy( storagesInstance.storages.evmCodeStorage, blockchain.getBackingStorage(-1), - blockchain, + (number: BigInt) => blockchain.getBlockHeaderByNumber(number).map(_.hash), UInt256.Zero, ByteString(MerklePatriciaTrie.EmptyRootHash), noEmptyAccounts = false, @@ -138,7 +138,7 @@ trait TestSetup extends SecureRandomBuilder with EphemBlockchainTestSetup { val initialWorld = InMemoryWorldStateProxy( storagesInstance.storages.evmCodeStorage, blockchain.getBackingStorage(-1), - blockchain, + (number: BigInt) => blockchain.getBlockHeaderByNumber(number).map(_.hash), UInt256.Zero, stateRootHash, noEmptyAccounts = false, @@ -302,7 +302,7 @@ trait TestSetupWithVmAndValidators extends EphemBlockchainTestSetup { val emptyWorld = InMemoryWorldStateProxy( storagesInstance.storages.evmCodeStorage, blockchain.getBackingStorage(-1), - blockchain, + (number: BigInt) => blockchain.getBlockHeaderByNumber(number).map(_.hash), blockchainConfig.accountStartNonce, ByteString(MerklePatriciaTrie.EmptyRootHash), noEmptyAccounts = false, diff --git a/src/test/scala/io/iohk/ethereum/ledger/StxLedgerSpec.scala b/src/test/scala/io/iohk/ethereum/ledger/StxLedgerSpec.scala index e46e3bc2bc..950f6921d9 100644 --- a/src/test/scala/io/iohk/ethereum/ledger/StxLedgerSpec.scala +++ b/src/test/scala/io/iohk/ethereum/ledger/StxLedgerSpec.scala @@ -155,7 +155,7 @@ trait ScenarioSetup extends EphemBlockchainTestSetup { InMemoryWorldStateProxy( storagesInstance.storages.evmCodeStorage, blockchain.getBackingStorage(-1), - blockchain, + (number: BigInt) => blockchain.getBlockHeaderByNumber(number).map(_.hash), UInt256.Zero, ByteString(MerklePatriciaTrie.EmptyRootHash), noEmptyAccounts = false, From ce5fe8cc98fab9c422c674007180d4c2a0e6c57b Mon Sep 17 00:00:00 2001 From: Leonor Boga Date: Fri, 25 Jun 2021 13:22:15 +0200 Subject: [PATCH 5/6] ETCM-944 Rollback min-peers-to-choose-pivor-block to 3 --- src/main/resources/conf/base.conf | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/resources/conf/base.conf b/src/main/resources/conf/base.conf index a170d25f83..8e4a31f21c 100644 --- a/src/main/resources/conf/base.conf +++ b/src/main/resources/conf/base.conf @@ -386,7 +386,7 @@ mantis { nodes-per-request = 384 # Minimum number of peers required to start fast-sync (by determining the pivot block) - min-peers-to-choose-pivot-block = 1 + min-peers-to-choose-pivot-block = 3 # Number of additional peers used to determine pivot block during fast-sync # Number of peers used to reach consensus = min-peers-to-choose-pivot-block + peers-to-choose-pivot-block-margin From 11c05a1219f20d59fc57149a53ec34657276873d Mon Sep 17 00:00:00 2001 From: Leonor Boga Date: Fri, 25 Jun 2021 15:51:46 +0200 Subject: [PATCH 6/6] ETCM-944 Rename some methods --- .../iohk/ethereum/ledger/BlockImporterItSpec.scala | 2 +- .../io/iohk/ethereum/sync/util/CommonFakePeer.scala | 2 +- .../ethereum/sync/util/RegularSyncItSpecUtils.scala | 2 +- .../iohk/ethereum/txExecTest/util/DumpChainApp.scala | 4 ++-- .../scala/io/iohk/ethereum/domain/Blockchain.scala | 8 ++++---- .../io/iohk/ethereum/jsonrpc/EthUserService.scala | 2 +- .../io/iohk/ethereum/ledger/BlockExecution.scala | 2 +- .../io/iohk/ethereum/ledger/BlockPreparator.scala | 2 +- .../scala/io/iohk/ethereum/ledger/StxLedger.scala | 2 +- .../ethereum/blockchain/sync/StateSyncUtils.scala | 2 +- .../iohk/ethereum/jsonrpc/EthBlocksServiceSpec.scala | 2 +- .../iohk/ethereum/jsonrpc/EthInfoServiceSpec.scala | 2 +- .../iohk/ethereum/jsonrpc/EthMiningServiceSpec.scala | 2 +- .../ethereum/jsonrpc/JsonRpcControllerFixture.scala | 2 +- .../io/iohk/ethereum/ledger/BlockImportSpec.scala | 6 ++++-- .../io/iohk/ethereum/ledger/BlockRewardSpec.scala | 2 +- .../io/iohk/ethereum/ledger/DeleteAccountsSpec.scala | 2 +- .../ethereum/ledger/DeleteTouchedAccountsSpec.scala | 4 ++-- .../ledger/InMemoryWorldStateProxySpec.scala | 12 ++++++------ .../io/iohk/ethereum/ledger/LedgerTestSetup.scala | 8 ++++---- .../io/iohk/ethereum/ledger/StxLedgerSpec.scala | 2 +- 21 files changed, 37 insertions(+), 35 deletions(-) diff --git a/src/it/scala/io/iohk/ethereum/ledger/BlockImporterItSpec.scala b/src/it/scala/io/iohk/ethereum/ledger/BlockImporterItSpec.scala index 06b12fb1fa..3a42a6697d 100644 --- a/src/it/scala/io/iohk/ethereum/ledger/BlockImporterItSpec.scala +++ b/src/it/scala/io/iohk/ethereum/ledger/BlockImporterItSpec.scala @@ -61,7 +61,7 @@ class BlockImporterItSpec val emptyWorld = InMemoryWorldStateProxy( storagesInstance.storages.evmCodeStorage, - blockchain.getBackingStorage(-1), + blockchain.getBackingMptStorage(-1), (number: BigInt) => blockchain.getBlockHeaderByNumber(number).map(_.hash), blockchainConfig.accountStartNonce, ByteString(MerklePatriciaTrie.EmptyRootHash), diff --git a/src/it/scala/io/iohk/ethereum/sync/util/CommonFakePeer.scala b/src/it/scala/io/iohk/ethereum/sync/util/CommonFakePeer.scala index f6d1c15d13..ab43f8a64a 100644 --- a/src/it/scala/io/iohk/ethereum/sync/util/CommonFakePeer.scala +++ b/src/it/scala/io/iohk/ethereum/sync/util/CommonFakePeer.scala @@ -243,7 +243,7 @@ abstract class CommonFakePeer(peerName: String, fakePeerCustomConfig: FakePeerCu private def getMptForBlock(block: Block) = { InMemoryWorldStateProxy( storagesInstance.storages.evmCodeStorage, - bl.getBackingStorage(block.number), + bl.getBackingMptStorage(block.number), (number: BigInt) => bl.getBlockHeaderByNumber(number).map(_.hash), blockchainConfig.accountStartNonce, block.header.stateRoot, 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 ccb98240e5..f18e2ad7fc 100644 --- a/src/it/scala/io/iohk/ethereum/sync/util/RegularSyncItSpecUtils.scala +++ b/src/it/scala/io/iohk/ethereum/sync/util/RegularSyncItSpecUtils.scala @@ -214,7 +214,7 @@ object RegularSyncItSpecUtils { private def getMptForBlock(block: Block) = { InMemoryWorldStateProxy( storagesInstance.storages.evmCodeStorage, - bl.getBackingStorage(block.number), + bl.getBackingMptStorage(block.number), (number: BigInt) => bl.getBlockHeaderByNumber(number).map(_.hash), UInt256.Zero, ByteString(MerklePatriciaTrie.EmptyRootHash), diff --git a/src/it/scala/io/iohk/ethereum/txExecTest/util/DumpChainApp.scala b/src/it/scala/io/iohk/ethereum/txExecTest/util/DumpChainApp.scala index 128ea79ed5..8cf9bfb3f0 100644 --- a/src/it/scala/io/iohk/ethereum/txExecTest/util/DumpChainApp.scala +++ b/src/it/scala/io/iohk/ethereum/txExecTest/util/DumpChainApp.scala @@ -202,7 +202,7 @@ class BlockchainMock(genesisHash: ByteString) extends Blockchain { override def getLatestCheckpointBlockNumber(): BigInt = ??? - override def getBackingStorage(blockNumber: BigInt): MptStorage = ??? + override def getBackingMptStorage(blockNumber: BigInt): MptStorage = ??? - override def getReadOnlyStorage(): MptStorage = ??? + override def getReadOnlyMptStorage(): MptStorage = ??? } diff --git a/src/main/scala/io/iohk/ethereum/domain/Blockchain.scala b/src/main/scala/io/iohk/ethereum/domain/Blockchain.scala index 7de454d9be..b49118c7f3 100644 --- a/src/main/scala/io/iohk/ethereum/domain/Blockchain.scala +++ b/src/main/scala/io/iohk/ethereum/domain/Blockchain.scala @@ -112,13 +112,13 @@ trait Blockchain { * @param blockNumber * @return MptStorage */ - def getBackingStorage(blockNumber: BigInt): MptStorage + def getBackingMptStorage(blockNumber: BigInt): MptStorage /** Get the MptStorage for read-only * * @return MptStorage */ - def getReadOnlyStorage(): MptStorage + def getReadOnlyMptStorage(): MptStorage /** * Returns the receipts based on a block hash @@ -322,9 +322,9 @@ class BlockchainImpl( StorageProof(position, value, proof) } - def getBackingStorage(blockNumber: BigInt): MptStorage = stateStorage.getBackingStorage(blockNumber) + def getBackingMptStorage(blockNumber: BigInt): MptStorage = stateStorage.getBackingStorage(blockNumber) - def getReadOnlyStorage(): MptStorage = stateStorage.getReadOnlyStorage + def getReadOnlyMptStorage(): MptStorage = stateStorage.getReadOnlyStorage private def persistBestBlocksData(): Unit = { val currentBestBlockNumber = getBestBlockNumber() diff --git a/src/main/scala/io/iohk/ethereum/jsonrpc/EthUserService.scala b/src/main/scala/io/iohk/ethereum/jsonrpc/EthUserService.scala index d652d72d88..227f2d9c1c 100644 --- a/src/main/scala/io/iohk/ethereum/jsonrpc/EthUserService.scala +++ b/src/main/scala/io/iohk/ethereum/jsonrpc/EthUserService.scala @@ -34,7 +34,7 @@ class EthUserService( resolveBlock(req.block).map { case ResolvedBlock(block, _) => val world = InMemoryWorldStateProxy( evmCodeStorage, - blockchain.getBackingStorage(block.header.number), + blockchain.getBackingMptStorage(block.header.number), (number: BigInt) => blockchain.getBlockHeaderByNumber(number).map(_.hash), blockchainConfig.accountStartNonce, block.header.stateRoot, diff --git a/src/main/scala/io/iohk/ethereum/ledger/BlockExecution.scala b/src/main/scala/io/iohk/ethereum/ledger/BlockExecution.scala index 17e2e07333..47b6cf8409 100644 --- a/src/main/scala/io/iohk/ethereum/ledger/BlockExecution.scala +++ b/src/main/scala/io/iohk/ethereum/ledger/BlockExecution.scala @@ -64,7 +64,7 @@ class BlockExecution( .toRight(MissingParentError) // Should not never occur because validated earlier initialWorld = InMemoryWorldStateProxy( evmCodeStorage = evmCodeStorage, - blockchain.getBackingStorage(block.header.number), + blockchain.getBackingMptStorage(block.header.number), (number: BigInt) => blockchain.getBlockHeaderByNumber(number).map(_.hash), accountStartNonce = blockchainConfig.accountStartNonce, stateRootHash = parentHeader.stateRoot, diff --git a/src/main/scala/io/iohk/ethereum/ledger/BlockPreparator.scala b/src/main/scala/io/iohk/ethereum/ledger/BlockPreparator.scala index 4f0a017a8d..740b3be8b3 100644 --- a/src/main/scala/io/iohk/ethereum/ledger/BlockPreparator.scala +++ b/src/main/scala/io/iohk/ethereum/ledger/BlockPreparator.scala @@ -381,7 +381,7 @@ class BlockPreparator( initialWorldStateBeforeExecution.getOrElse( InMemoryWorldStateProxy( evmCodeStorage = evmCodeStorage, - mptStorage = blockchain.getReadOnlyStorage(), + mptStorage = blockchain.getReadOnlyMptStorage(), getBlockHashByNumber = (number: BigInt) => blockchain.getBlockHeaderByNumber(number).map(_.hash), accountStartNonce = blockchainConfig.accountStartNonce, stateRootHash = parent.stateRoot, diff --git a/src/main/scala/io/iohk/ethereum/ledger/StxLedger.scala b/src/main/scala/io/iohk/ethereum/ledger/StxLedger.scala index 8d27bef055..181be1d1ec 100644 --- a/src/main/scala/io/iohk/ethereum/ledger/StxLedger.scala +++ b/src/main/scala/io/iohk/ethereum/ledger/StxLedger.scala @@ -23,7 +23,7 @@ class StxLedger( val world1 = world.getOrElse( InMemoryWorldStateProxy( evmCodeStorage = evmCodeStorage, - mptStorage = blockchain.getReadOnlyStorage(), + mptStorage = blockchain.getReadOnlyMptStorage(), getBlockHashByNumber = (number: BigInt) => blockchain.getBlockHeaderByNumber(number).map(_.hash), accountStartNonce = blockchainConfig.accountStartNonce, stateRootHash = blockHeader.stateRoot, diff --git a/src/test/scala/io/iohk/ethereum/blockchain/sync/StateSyncUtils.scala b/src/test/scala/io/iohk/ethereum/blockchain/sync/StateSyncUtils.scala index 0aca412563..0082f691dc 100644 --- a/src/test/scala/io/iohk/ethereum/blockchain/sync/StateSyncUtils.scala +++ b/src/test/scala/io/iohk/ethereum/blockchain/sync/StateSyncUtils.scala @@ -34,7 +34,7 @@ object StateSyncUtils extends EphemBlockchainTestSetup { def buildWorld(accountData: Seq[MptNodeData], existingTree: Option[ByteString] = None): ByteString = { val init = InMemoryWorldStateProxy( evmCodeStorage, - blockchain.getBackingStorage(1), + blockchain.getBackingMptStorage(1), (number: BigInt) => blockchain.getBlockHeaderByNumber(number).map(_.hash), blockchainConfig.accountStartNonce, existingTree.getOrElse(ByteString(MerklePatriciaTrie.EmptyRootHash)), diff --git a/src/test/scala/io/iohk/ethereum/jsonrpc/EthBlocksServiceSpec.scala b/src/test/scala/io/iohk/ethereum/jsonrpc/EthBlocksServiceSpec.scala index 08fe56debd..61f9923982 100644 --- a/src/test/scala/io/iohk/ethereum/jsonrpc/EthBlocksServiceSpec.scala +++ b/src/test/scala/io/iohk/ethereum/jsonrpc/EthBlocksServiceSpec.scala @@ -431,7 +431,7 @@ class EthBlocksServiceSpec val fakeWorld = InMemoryWorldStateProxy( storagesInstance.storages.evmCodeStorage, - blockchain.getBackingStorage(-1), + blockchain.getBackingMptStorage(-1), (number: BigInt) => blockchain.getBlockHeaderByNumber(number).map(_.hash), UInt256.Zero, ByteString.empty, diff --git a/src/test/scala/io/iohk/ethereum/jsonrpc/EthInfoServiceSpec.scala b/src/test/scala/io/iohk/ethereum/jsonrpc/EthInfoServiceSpec.scala index 9fb4ecfd9a..d3ba9f6338 100644 --- a/src/test/scala/io/iohk/ethereum/jsonrpc/EthInfoServiceSpec.scala +++ b/src/test/scala/io/iohk/ethereum/jsonrpc/EthInfoServiceSpec.scala @@ -97,7 +97,7 @@ class EthServiceSpec val worldStateProxy = InMemoryWorldStateProxy( storagesInstance.storages.evmCodeStorage, - blockchain.getBackingStorage(-1), + blockchain.getBackingMptStorage(-1), (number: BigInt) => blockchain.getBlockHeaderByNumber(number).map(_.hash), UInt256.Zero, ByteString.empty, diff --git a/src/test/scala/io/iohk/ethereum/jsonrpc/EthMiningServiceSpec.scala b/src/test/scala/io/iohk/ethereum/jsonrpc/EthMiningServiceSpec.scala index 03419c305b..e5e1ec65ea 100644 --- a/src/test/scala/io/iohk/ethereum/jsonrpc/EthMiningServiceSpec.scala +++ b/src/test/scala/io/iohk/ethereum/jsonrpc/EthMiningServiceSpec.scala @@ -306,7 +306,7 @@ class EthMiningServiceSpec val fakeWorld = InMemoryWorldStateProxy( storagesInstance.storages.evmCodeStorage, - blockchain.getReadOnlyStorage(), + blockchain.getReadOnlyMptStorage(), (number: BigInt) => blockchain.getBlockHeaderByNumber(number).map(_.hash), UInt256.Zero, ByteString.empty, diff --git a/src/test/scala/io/iohk/ethereum/jsonrpc/JsonRpcControllerFixture.scala b/src/test/scala/io/iohk/ethereum/jsonrpc/JsonRpcControllerFixture.scala index 2b9812043e..6eaf9a6bef 100644 --- a/src/test/scala/io/iohk/ethereum/jsonrpc/JsonRpcControllerFixture.scala +++ b/src/test/scala/io/iohk/ethereum/jsonrpc/JsonRpcControllerFixture.scala @@ -176,7 +176,7 @@ class JsonRpcControllerFixture(implicit system: ActorSystem) val fakeWorld = InMemoryWorldStateProxy( storagesInstance.storages.evmCodeStorage, - blockchain.getReadOnlyStorage(), + blockchain.getReadOnlyMptStorage(), (number: BigInt) => blockchain.getBlockHeaderByNumber(number).map(_.hash), blockchainConfig.accountStartNonce, ByteString.empty, diff --git a/src/test/scala/io/iohk/ethereum/ledger/BlockImportSpec.scala b/src/test/scala/io/iohk/ethereum/ledger/BlockImportSpec.scala index af63abfa99..cf95a5882c 100644 --- a/src/test/scala/io/iohk/ethereum/ledger/BlockImportSpec.scala +++ b/src/test/scala/io/iohk/ethereum/ledger/BlockImportSpec.scala @@ -56,7 +56,9 @@ class BlockImportSpec extends AnyFlatSpec with Matchers with ScalaFutures { (blockQueue.getBranch _).expects(hash, true).returning(List(block)) (blockchain.getBlockHeaderByHash _).expects(*).returning(Some(block.header)) - (blockchain.getBackingStorage _).expects(*).returning(storagesInstance.storages.stateStorage.getBackingStorage(6)) + (blockchain.getBackingMptStorage _) + .expects(*) + .returning(storagesInstance.storages.stateStorage.getBackingStorage(6)) expectBlockSaved(block, Seq.empty[Receipt], newWeight, saveAsBestBlock = true) @@ -88,7 +90,7 @@ class BlockImportSpec extends AnyFlatSpec with Matchers with ScalaFutures { (blockchain.getBlockHeaderByHash _).expects(*).returning(Some(block.header)) (blockchain.getBlockHeaderByNumber _).expects(*).returning(Some(block.header)) - (blockchain.getBackingStorage _).expects(*).returning(mptStorage) + (blockchain.getBackingMptStorage _).expects(*).returning(mptStorage) (mptStorage.get _).expects(*).returning(mptNode) (blockQueue.removeSubtree _).expects(*) diff --git a/src/test/scala/io/iohk/ethereum/ledger/BlockRewardSpec.scala b/src/test/scala/io/iohk/ethereum/ledger/BlockRewardSpec.scala index 6320e8c456..effec2bb73 100644 --- a/src/test/scala/io/iohk/ethereum/ledger/BlockRewardSpec.scala +++ b/src/test/scala/io/iohk/ethereum/ledger/BlockRewardSpec.scala @@ -193,7 +193,7 @@ class BlockRewardSpec extends AnyFlatSpec with Matchers with ScalaCheckPropertyC val worldState = InMemoryWorldStateProxy( storagesInstance.storages.evmCodeStorage, - blockchain.getBackingStorage(-1), + blockchain.getBackingMptStorage(-1), (number: BigInt) => blockchain.getBlockHeaderByNumber(number).map(_.hash), UInt256.Zero, ByteString(MerklePatriciaTrie.EmptyRootHash), diff --git a/src/test/scala/io/iohk/ethereum/ledger/DeleteAccountsSpec.scala b/src/test/scala/io/iohk/ethereum/ledger/DeleteAccountsSpec.scala index ebdeffa491..366cfe734d 100644 --- a/src/test/scala/io/iohk/ethereum/ledger/DeleteAccountsSpec.scala +++ b/src/test/scala/io/iohk/ethereum/ledger/DeleteAccountsSpec.scala @@ -67,7 +67,7 @@ class DeleteAccountsSpec extends AnyFlatSpec with Matchers with MockFactory { val worldStateWithoutPersist = InMemoryWorldStateProxy( storagesInstance.storages.evmCodeStorage, - blockchain.getBackingStorage(-1), + blockchain.getBackingMptStorage(-1), (number: BigInt) => blockchain.getBlockHeaderByNumber(number).map(_.hash), UInt256.Zero, ByteString(MerklePatriciaTrie.EmptyRootHash), diff --git a/src/test/scala/io/iohk/ethereum/ledger/DeleteTouchedAccountsSpec.scala b/src/test/scala/io/iohk/ethereum/ledger/DeleteTouchedAccountsSpec.scala index bdb06e8abe..d431a98423 100644 --- a/src/test/scala/io/iohk/ethereum/ledger/DeleteTouchedAccountsSpec.scala +++ b/src/test/scala/io/iohk/ethereum/ledger/DeleteTouchedAccountsSpec.scala @@ -150,7 +150,7 @@ class DeleteTouchedAccountsSpec extends AnyFlatSpec with Matchers { val worldStateWithoutPersist = InMemoryWorldStateProxy( storagesInstance.storages.evmCodeStorage, - blockchain.getBackingStorage(-1), + blockchain.getBackingMptStorage(-1), (number: BigInt) => blockchain.getBlockHeaderByNumber(number).map(_.hash), UInt256.Zero, ByteString(MerklePatriciaTrie.EmptyRootHash), @@ -165,7 +165,7 @@ class DeleteTouchedAccountsSpec extends AnyFlatSpec with Matchers { val worldStateWithoutPersistPreEIP161 = InMemoryWorldStateProxy( storagesInstance.storages.evmCodeStorage, - blockchain.getBackingStorage(-1), + blockchain.getBackingMptStorage(-1), (number: BigInt) => blockchain.getBlockHeaderByNumber(number).map(_.hash), UInt256.Zero, ByteString(MerklePatriciaTrie.EmptyRootHash), diff --git a/src/test/scala/io/iohk/ethereum/ledger/InMemoryWorldStateProxySpec.scala b/src/test/scala/io/iohk/ethereum/ledger/InMemoryWorldStateProxySpec.scala index de28cf567e..de7761c1ff 100644 --- a/src/test/scala/io/iohk/ethereum/ledger/InMemoryWorldStateProxySpec.scala +++ b/src/test/scala/io/iohk/ethereum/ledger/InMemoryWorldStateProxySpec.scala @@ -125,7 +125,7 @@ class InMemoryWorldStateProxySpec extends AnyFlatSpec with Matchers { // Create a new WS instance based on storages and new root state and check val newWorldState = InMemoryWorldStateProxy( storagesInstance.storages.evmCodeStorage, - blockchain.getBackingStorage(-1), + blockchain.getBackingMptStorage(-1), (number: BigInt) => blockchain.getBlockHeaderByNumber(number).map(_.hash), UInt256.Zero, persistedWorldState.stateRootHash, @@ -257,7 +257,7 @@ class InMemoryWorldStateProxySpec extends AnyFlatSpec with Matchers { val readWorldState = InMemoryWorldStateProxy( storagesInstance.storages.evmCodeStorage, - blockchain.getReadOnlyStorage(), + blockchain.getReadOnlyMptStorage(), (number: BigInt) => blockchain.getBlockHeaderByNumber(number).map(_.hash), UInt256.Zero, persistedWorldStateWithAnAccount.stateRootHash, @@ -279,7 +279,7 @@ class InMemoryWorldStateProxySpec extends AnyFlatSpec with Matchers { assertThrows[MPTException] { val newReadWorld = InMemoryWorldStateProxy( storagesInstance.storages.evmCodeStorage, - blockchain.getReadOnlyStorage(), + blockchain.getReadOnlyMptStorage(), (number: BigInt) => blockchain.getBlockHeaderByNumber(number).map(_.hash), UInt256.Zero, changedReadWorld.stateRootHash, @@ -308,7 +308,7 @@ class InMemoryWorldStateProxySpec extends AnyFlatSpec with Matchers { val world2 = InMemoryWorldStateProxy( storagesInstance.storages.evmCodeStorage, - blockchain.getBackingStorage(-1), + blockchain.getBackingMptStorage(-1), (number: BigInt) => blockchain.getBlockHeaderByNumber(number).map(_.hash), UInt256.Zero, world1.stateRootHash, @@ -334,7 +334,7 @@ class InMemoryWorldStateProxySpec extends AnyFlatSpec with Matchers { val worldState = InMemoryWorldStateProxy( storagesInstance.storages.evmCodeStorage, - blockchain.getBackingStorage(-1), + blockchain.getBackingMptStorage(-1), (number: BigInt) => blockchain.getBlockHeaderByNumber(number).map(_.hash), UInt256.Zero, ByteString(MerklePatriciaTrie.EmptyRootHash), @@ -344,7 +344,7 @@ class InMemoryWorldStateProxySpec extends AnyFlatSpec with Matchers { val postEIP161WorldState = InMemoryWorldStateProxy( storagesInstance.storages.evmCodeStorage, - blockchain.getBackingStorage(-1), + blockchain.getBackingMptStorage(-1), (number: BigInt) => blockchain.getBlockHeaderByNumber(number).map(_.hash), UInt256.Zero, ByteString(MerklePatriciaTrie.EmptyRootHash), diff --git a/src/test/scala/io/iohk/ethereum/ledger/LedgerTestSetup.scala b/src/test/scala/io/iohk/ethereum/ledger/LedgerTestSetup.scala index 586510b1f3..be8f3f6d22 100644 --- a/src/test/scala/io/iohk/ethereum/ledger/LedgerTestSetup.scala +++ b/src/test/scala/io/iohk/ethereum/ledger/LedgerTestSetup.scala @@ -84,7 +84,7 @@ trait TestSetup extends SecureRandomBuilder with EphemBlockchainTestSetup { val emptyWorld: InMemoryWorldStateProxy = InMemoryWorldStateProxy( storagesInstance.storages.evmCodeStorage, - blockchain.getBackingStorage(-1), + blockchain.getBackingMptStorage(-1), (number: BigInt) => blockchain.getBlockHeaderByNumber(number).map(_.hash), UInt256.Zero, ByteString(MerklePatriciaTrie.EmptyRootHash), @@ -137,7 +137,7 @@ trait TestSetup extends SecureRandomBuilder with EphemBlockchainTestSetup { ): ByteString = { val initialWorld = InMemoryWorldStateProxy( storagesInstance.storages.evmCodeStorage, - blockchain.getBackingStorage(-1), + blockchain.getBackingMptStorage(-1), (number: BigInt) => blockchain.getBlockHeaderByNumber(number).map(_.hash), UInt256.Zero, stateRootHash, @@ -237,7 +237,7 @@ trait DaoForkTestSetup extends TestSetup with MockFactory { (testBlockchain.getBlockHeaderByHash _) .expects(proDaoBlock.header.parentHash) .returning(Some(parentBlockHeader)) - (testBlockchain.getBackingStorage _) + (testBlockchain.getBackingMptStorage _) .expects(*) .returning(storagesInstance.storages.stateStorage.getBackingStorage(1920000)) } @@ -301,7 +301,7 @@ trait TestSetupWithVmAndValidators extends EphemBlockchainTestSetup { ): Either[BlockExecutionError, Seq[Receipt]] = { val emptyWorld = InMemoryWorldStateProxy( storagesInstance.storages.evmCodeStorage, - blockchain.getBackingStorage(-1), + blockchain.getBackingMptStorage(-1), (number: BigInt) => blockchain.getBlockHeaderByNumber(number).map(_.hash), blockchainConfig.accountStartNonce, ByteString(MerklePatriciaTrie.EmptyRootHash), diff --git a/src/test/scala/io/iohk/ethereum/ledger/StxLedgerSpec.scala b/src/test/scala/io/iohk/ethereum/ledger/StxLedgerSpec.scala index 950f6921d9..8768b58cfe 100644 --- a/src/test/scala/io/iohk/ethereum/ledger/StxLedgerSpec.scala +++ b/src/test/scala/io/iohk/ethereum/ledger/StxLedgerSpec.scala @@ -154,7 +154,7 @@ trait ScenarioSetup extends EphemBlockchainTestSetup { val emptyWorld: InMemoryWorldStateProxy = InMemoryWorldStateProxy( storagesInstance.storages.evmCodeStorage, - blockchain.getBackingStorage(-1), + blockchain.getBackingMptStorage(-1), (number: BigInt) => blockchain.getBlockHeaderByNumber(number).map(_.hash), UInt256.Zero, ByteString(MerklePatriciaTrie.EmptyRootHash),