Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
31 changes: 16 additions & 15 deletions src/it/scala/io/iohk/ethereum/ledger/BlockImporterItSpec.scala
Original file line number Diff line number Diff line change
Expand Up @@ -63,7 +63,7 @@ class BlockImporterItSpec
)
val genesisWeight: ChainWeight = ChainWeight.zero.increase(genesis.header)

blockchain.save(genesis, Seq(), genesisWeight, saveAsBestBlock = true)
blockchainWriter.save(genesis, Seq(), genesisWeight, saveAsBestBlock = true)

lazy val checkpointBlockGenerator: CheckpointBlockGenerator = new CheckpointBlockGenerator

Expand Down Expand Up @@ -101,6 +101,7 @@ class BlockImporterItSpec
new BlockExecution(
blockchain,
blockchainReader,
blockchainWriter,
storagesInstance.storages.evmCodeStorage,
blockchainConfig,
consensus.blockPreparator,
Expand Down Expand Up @@ -147,10 +148,10 @@ class BlockImporterItSpec
val oldWeight4: ChainWeight = oldWeight3.increase(oldBlock4.header)

//saving initial main chain
blockchain.save(block1, Nil, weight1, saveAsBestBlock = true)
blockchain.save(oldBlock2, Nil, oldWeight2, saveAsBestBlock = true)
blockchain.save(oldBlock3, Nil, oldWeight3, saveAsBestBlock = true)
blockchain.save(oldBlock4, Nil, oldWeight4, saveAsBestBlock = true)
blockchainWriter.save(block1, Nil, weight1, saveAsBestBlock = true)
blockchainWriter.save(oldBlock2, Nil, oldWeight2, saveAsBestBlock = true)
blockchainWriter.save(oldBlock3, Nil, oldWeight3, saveAsBestBlock = true)
blockchainWriter.save(oldBlock4, Nil, oldWeight4, saveAsBestBlock = true)

val oldBranch: List[Block] = List(oldBlock2, oldBlock3, oldBlock4)
val newBranch: List[Block] = List(newBlock2, newBlock3)
Expand Down Expand Up @@ -183,9 +184,9 @@ class BlockImporterItSpec
it should "return a correct new best block after reorganising longer chain to a shorter one if its weight is bigger" in {

//returning discarded initial chain
blockchain.save(oldBlock2, Nil, oldWeight2, saveAsBestBlock = true)
blockchain.save(oldBlock3, Nil, oldWeight3, saveAsBestBlock = true)
blockchain.save(oldBlock4, Nil, oldWeight4, saveAsBestBlock = true)
blockchainWriter.save(oldBlock2, Nil, oldWeight2, saveAsBestBlock = true)
blockchainWriter.save(oldBlock3, Nil, oldWeight3, saveAsBestBlock = true)
blockchainWriter.save(oldBlock4, Nil, oldWeight4, saveAsBestBlock = true)

blockImporter ! BlockFetcher.PickedBlocks(NonEmptyList.fromListUnsafe(newBranch))

Expand All @@ -206,12 +207,12 @@ class BlockImporterItSpec
ommers = Seq(oldBlock4.header)
)

blockchain.save(oldBlock2, Nil, oldWeight2, saveAsBestBlock = true)
blockchain.save(oldBlock3, Nil, oldWeight3, saveAsBestBlock = true)
blockchain.save(oldBlock4, Nil, oldWeight4, saveAsBestBlock = true)
blockchainWriter.save(oldBlock2, Nil, oldWeight2, saveAsBestBlock = true)
blockchainWriter.save(oldBlock3, Nil, oldWeight3, saveAsBestBlock = true)
blockchainWriter.save(oldBlock4, Nil, oldWeight4, saveAsBestBlock = true)
// simulation of node restart
blockchain.saveBestKnownBlocks(blockchain.getBestBlockNumber() - 1)
blockchain.save(newBlock4ParentOldBlock3, Nil, newBlock4WeightParentOldBlock3, saveAsBestBlock = true)
blockchainWriter.save(newBlock4ParentOldBlock3, Nil, newBlock4WeightParentOldBlock3, saveAsBestBlock = true)

//not reorganising anymore until oldBlock4(not part of the chain anymore), no block/ommer validation when not part of the chain, resolveBranch is returning UnknownBranch
blockImporter ! BlockFetcher.PickedBlocks(NonEmptyList.fromListUnsafe(List(newBlock5ParentOldBlock4)))
Expand All @@ -223,7 +224,7 @@ class BlockImporterItSpec

val checkpoint = ObjectGenerators.fakeCheckpointGen(3, 3).sample.get
val oldBlock5WithCheckpoint: Block = checkpointBlockGenerator.generate(oldBlock4, checkpoint)
blockchain.save(oldBlock5WithCheckpoint, Nil, oldWeight4, saveAsBestBlock = true)
blockchainWriter.save(oldBlock5WithCheckpoint, Nil, oldWeight4, saveAsBestBlock = true)

val newBranch = List(newBlock2, newBlock3)

Expand All @@ -237,7 +238,7 @@ class BlockImporterItSpec

val checkpoint = ObjectGenerators.fakeCheckpointGen(3, 3).sample.get
val newBlock4WithCheckpoint: Block = checkpointBlockGenerator.generate(newBlock3, checkpoint)
blockchain.save(newBlock4WithCheckpoint, Nil, newWeight3, saveAsBestBlock = true)
blockchainWriter.save(newBlock4WithCheckpoint, Nil, newWeight3, saveAsBestBlock = true)

val newBranch = List(newBlock4WithCheckpoint)

Expand All @@ -253,7 +254,7 @@ class BlockImporterItSpec
val newBlock5: Block = getBlock(genesisBlock.number + 5, difficulty = 104, parent = parent.header.hash)
val newWeight5 = newWeight3.increase(newBlock5.header)

blockchain.save(newBlock5, Nil, newWeight5, saveAsBestBlock = true)
blockchainWriter.save(newBlock5, Nil, newWeight5, saveAsBestBlock = true)

val signatures = CheckpointingTestHelpers.createCheckpointSignatures(
Seq(crypto.generateKeyPair(secureRandom)),
Expand Down
15 changes: 11 additions & 4 deletions src/it/scala/io/iohk/ethereum/sync/util/CommonFakePeer.scala
Original file line number Diff line number Diff line change
Expand Up @@ -39,7 +39,9 @@ import io.iohk.ethereum.db.storage.pruning.PruningMode
import io.iohk.ethereum.domain.Block
import io.iohk.ethereum.domain.Blockchain
import io.iohk.ethereum.domain.BlockchainImpl
import io.iohk.ethereum.domain.BlockchainMetadata
import io.iohk.ethereum.domain.BlockchainReader
import io.iohk.ethereum.domain.BlockchainWriter
import io.iohk.ethereum.domain.ChainWeight
import io.iohk.ethereum.ledger.InMemoryWorldStateProxy
import io.iohk.ethereum.mpt.MerklePatriciaTrie
Expand Down Expand Up @@ -135,8 +137,13 @@ abstract class CommonFakePeer(peerName: String, fakePeerCustomConfig: FakePeerCu
)
)

val blockchainMetadata = new BlockchainMetadata(
storagesInstance.storages.appStateStorage.getBestBlockNumber(),
storagesInstance.storages.appStateStorage.getLatestCheckpointBlockNumber()
)
val blockchainReader: BlockchainReader = BlockchainReader(storagesInstance.storages)
val bl: BlockchainImpl = BlockchainImpl(storagesInstance.storages, blockchainReader)
val blockchainWriter: BlockchainWriter = BlockchainWriter(storagesInstance.storages, blockchainMetadata)
val bl: BlockchainImpl = BlockchainImpl(storagesInstance.storages, blockchainReader, blockchainMetadata)
val evmCodeStorage = storagesInstance.storages.evmCodeStorage

val genesis: Block = Block(
Expand All @@ -145,7 +152,7 @@ abstract class CommonFakePeer(peerName: String, fakePeerCustomConfig: FakePeerCu
)
val genesisWeight: ChainWeight = ChainWeight.zero.increase(genesis.header)

bl.save(genesis, Seq(), genesisWeight, saveAsBestBlock = true)
blockchainWriter.save(genesis, Seq(), genesisWeight, saveAsBestBlock = true)

lazy val nh = nodeStatusHolder

Expand Down Expand Up @@ -346,7 +353,7 @@ abstract class CommonFakePeer(peerName: String, fakePeerCustomConfig: FakePeerCu
val newWeight = ChainWeight.totalDifficultyOnly(1)

broadcastBlock(childBlock, newWeight)
bl.save(childBlock, Seq(), newWeight, saveAsBestBlock = true)
blockchainWriter.save(childBlock, Seq(), newWeight, saveAsBestBlock = true)
}

private def generateValidBlock(
Expand All @@ -357,7 +364,7 @@ abstract class CommonFakePeer(peerName: String, fakePeerCustomConfig: FakePeerCu
val currentWorld = getMptForBlock(currentBestBlock)
val (newBlock, newWeight, _) =
createChildBlock(currentBestBlock, currentWeight, currentWorld)(updateWorldForBlock)
bl.save(newBlock, Seq(), newWeight, saveAsBestBlock = true)
blockchainWriter.save(newBlock, Seq(), newWeight, saveAsBestBlock = true)
broadcastBlock(newBlock, newWeight)
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,7 @@ object FastSyncItSpecUtils {
storagesInstance.storages.appStateStorage,
bl,
blockchainReader,
blockchainWriter,
storagesInstance.storages.evmCodeStorage,
storagesInstance.storages.nodeStorage,
validators,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -96,13 +96,22 @@ object RegularSyncItSpecUtils {
new BlockExecution(
bl,
blockchainReader,
blockchainWriter,
storagesInstance.storages.evmCodeStorage,
blockchainConfig,
consensus.blockPreparator,
blockValidation
)
lazy val blockImport: BlockImport =
new BlockImport(bl, blockchainReader, blockQueue, blockValidation, blockExecution, Scheduler.global)
new BlockImport(
bl,
blockchainReader,
blockchainWriter,
blockQueue,
blockValidation,
blockExecution,
Scheduler.global
)

lazy val ommersPool: ActorRef = system.actorOf(OmmersPool.props(blockchainReader, 1), "ommers-pool")

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@ class ContractTest extends AnyFlatSpec with Matchers {
new BlockExecution(
blockchain,
blockchainReader,
blockchainWriter,
testBlockchainStorages.evmCodeStorage,
blockchainConfig,
consensus.blockPreparator,
Expand Down
10 changes: 9 additions & 1 deletion src/it/scala/io/iohk/ethereum/txExecTest/ECIP1017Test.scala
Original file line number Diff line number Diff line change
Expand Up @@ -6,7 +6,9 @@ import org.scalatest.matchers.should.Matchers

import io.iohk.ethereum.domain.Address
import io.iohk.ethereum.domain.BlockchainImpl
import io.iohk.ethereum.domain.BlockchainMetadata
import io.iohk.ethereum.domain.BlockchainReader
import io.iohk.ethereum.domain.BlockchainWriter
import io.iohk.ethereum.domain.Receipt
import io.iohk.ethereum.domain.UInt256
import io.iohk.ethereum.ledger.BlockExecution
Expand Down Expand Up @@ -78,14 +80,20 @@ class ECIP1017Test extends AnyFlatSpec with Matchers {

(startBlock to endBlock).foreach { blockToExecute =>
val storages = FixtureProvider.prepareStorages(blockToExecute - 1, fixtures)
val blockchainMetadata = new BlockchainMetadata(
storages.appStateStorage.getBestBlockNumber(),
storages.appStateStorage.getLatestCheckpointBlockNumber()
)
val blockchainReader = BlockchainReader(storages)
val blockchain = BlockchainImpl(storages, blockchainReader)
val blockchainWriter = BlockchainWriter(storages, blockchainMetadata)
val blockchain = BlockchainImpl(storages, blockchainReader, blockchainMetadata)
val blockValidation =
new BlockValidation(consensus, blockchainReader, BlockQueue(blockchain, syncConfig))
val blockExecution =
new BlockExecution(
blockchain,
blockchainReader,
blockchainWriter,
testBlockchainStorages.evmCodeStorage,
blockchainConfig,
consensus.blockPreparator,
Expand Down
10 changes: 9 additions & 1 deletion src/it/scala/io/iohk/ethereum/txExecTest/ForksTest.scala
Original file line number Diff line number Diff line change
Expand Up @@ -6,7 +6,9 @@ import org.scalatest.matchers.should.Matchers

import io.iohk.ethereum.domain.Address
import io.iohk.ethereum.domain.BlockchainImpl
import io.iohk.ethereum.domain.BlockchainMetadata
import io.iohk.ethereum.domain.BlockchainReader
import io.iohk.ethereum.domain.BlockchainWriter
import io.iohk.ethereum.domain.Receipt
import io.iohk.ethereum.domain.UInt256
import io.iohk.ethereum.ledger.BlockExecution
Expand Down Expand Up @@ -71,14 +73,20 @@ class ForksTest extends AnyFlatSpec with Matchers {

(startBlock to endBlock).foreach { blockToExecute =>
val storages = FixtureProvider.prepareStorages(blockToExecute - 1, fixtures)
val blockchainMetadata = new BlockchainMetadata(
storages.appStateStorage.getBestBlockNumber(),
storages.appStateStorage.getLatestCheckpointBlockNumber()
)
val blockchainReader = BlockchainReader(storages)
val blockchain = BlockchainImpl(storages, blockchainReader)
val blockchainWriter = BlockchainWriter(storages, blockchainMetadata)
val blockchain = BlockchainImpl(storages, blockchainReader, blockchainMetadata)
val blockValidation =
new BlockValidation(consensus, blockchainReader, BlockQueue(blockchain, syncConfig))
val blockExecution =
new BlockExecution(
blockchain,
blockchainReader,
blockchainWriter,
testBlockchainStorages.evmCodeStorage,
blockchainConfig,
consensus.blockPreparator,
Expand Down
10 changes: 9 additions & 1 deletion src/it/scala/io/iohk/ethereum/txExecTest/ScenarioSetup.scala
Original file line number Diff line number Diff line change
Expand Up @@ -2,14 +2,22 @@ package io.iohk.ethereum.txExecTest

import io.iohk.ethereum.blockchain.sync.EphemBlockchainTestSetup
import io.iohk.ethereum.domain.BlockchainImpl
import io.iohk.ethereum.domain.BlockchainMetadata
import io.iohk.ethereum.domain.BlockchainReader
import io.iohk.ethereum.domain.BlockchainStorages
import io.iohk.ethereum.domain.BlockchainWriter
import io.iohk.ethereum.ledger.VMImpl

trait ScenarioSetup extends EphemBlockchainTestSetup {
protected val testBlockchainStorages: BlockchainStorages

override lazy val blockchainMetadata = new BlockchainMetadata(
testBlockchainStorages.appStateStorage.getBestBlockNumber(),
testBlockchainStorages.appStateStorage.getLatestCheckpointBlockNumber()
)
override lazy val blockchainReader: BlockchainReader = BlockchainReader(testBlockchainStorages)
override lazy val blockchain: BlockchainImpl = BlockchainImpl(testBlockchainStorages, blockchainReader)
override lazy val blockchainWriter: BlockchainWriter = BlockchainWriter(testBlockchainStorages, blockchainMetadata)
override lazy val blockchain: BlockchainImpl =
BlockchainImpl(testBlockchainStorages, blockchainReader, blockchainMetadata)
override lazy val vm: VMImpl = new VMImpl
}
Original file line number Diff line number Diff line change
Expand Up @@ -48,7 +48,6 @@ import io.iohk.ethereum.txExecTest.util.DumpChainActor._
class DumpChainActor(
peerManager: ActorRef,
peerMessageBus: ActorRef,
startBlock: BigInt,
maxBlocks: BigInt,
bootstrapNode: String
) extends Actor {
Expand Down Expand Up @@ -287,12 +286,11 @@ object DumpChainActor {
def props(
peerManager: ActorRef,
peerMessageBus: ActorRef,
startBlock: BigInt,
maxBlocks: BigInt,
bootstrapNode: String
): Props =
Props(
new DumpChainActor(peerManager, peerMessageBus: ActorRef, startBlock: BigInt, maxBlocks: BigInt, bootstrapNode)
new DumpChainActor(peerManager, peerMessageBus: ActorRef, maxBlocks: BigInt, bootstrapNode)
)
val emptyStorage: ByteString = ByteString(
Hex.decode("56e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421")
Expand Down
13 changes: 1 addition & 12 deletions src/it/scala/io/iohk/ethereum/txExecTest/util/DumpChainApp.scala
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@ import io.iohk.ethereum.blockchain.sync.CacheBasedBlacklist
import io.iohk.ethereum.db.components.RocksDbDataSourceComponent
import io.iohk.ethereum.db.components.Storages
import io.iohk.ethereum.db.components.Storages.PruningModeComponent
import io.iohk.ethereum.db.dataSource.DataSourceBatchUpdate
import io.iohk.ethereum.db.storage.AppStateStorage
import io.iohk.ethereum.db.storage.MptStorage
import io.iohk.ethereum.db.storage.NodeStorage.NodeEncoded
Expand Down Expand Up @@ -150,7 +149,7 @@ object DumpChainApp
)
peerManager ! PeerManagerActor.StartConnecting

actorSystem.actorOf(DumpChainActor.props(peerManager, peerMessageBus, startBlock, maxBlocks, node), "dumper")
actorSystem.actorOf(DumpChainActor.props(peerManager, peerMessageBus, maxBlocks, node), "dumper")
}

class BlockchainMock(genesisHash: ByteString) extends Blockchain {
Expand Down Expand Up @@ -185,14 +184,6 @@ class BlockchainMock(genesisHash: ByteString) extends Blockchain {
ethCompatibleStorage: Boolean
): StorageProof = EmptyStorageValueProof(StorageProofKey(position))

override def storeBlockHeader(blockHeader: BlockHeader): DataSourceBatchUpdate = ???

override def storeBlockBody(blockHash: ByteString, blockBody: BlockBody): DataSourceBatchUpdate = ???

override def storeReceipts(blockHash: ByteString, receipts: Seq[Receipt]): DataSourceBatchUpdate = ???

override def storeChainWeight(blockhash: ByteString, chainWeight: ChainWeight): DataSourceBatchUpdate = ???

override def saveNode(nodeHash: NodeHash, nodeEncoded: NodeEncoded, blockNumber: BigInt): Unit = ???

override def removeBlock(hash: ByteString, withState: Boolean = true): Unit = ???
Expand All @@ -215,8 +206,6 @@ class BlockchainMock(genesisHash: ByteString) extends Blockchain {

def getBestBlock(): Option[Block] = ???

override def save(block: Block, receipts: Seq[Receipt], weight: ChainWeight, saveAsBestBlock: Boolean): Unit = ???

override def getLatestCheckpointBlockNumber(): BigInt = ???

override def isInChain(hash: NodeHash): Boolean = ???
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,8 +36,8 @@ import io.iohk.ethereum.utils.BlockchainConfig
import io.iohk.ethereum.utils.Logger

class GenesisDataLoader(
blockchain: Blockchain,
blockchainReader: BlockchainReader,
blockchainWriter: BlockchainWriter,
stateStorage: StateStorage,
blockchainConfig: BlockchainConfig
) extends Logger {
Expand Down Expand Up @@ -122,7 +122,7 @@ class GenesisDataLoader(
case None =>
storage.persist()
stateStorage.forcePersist(GenesisDataLoad)
blockchain.save(
blockchainWriter.save(
Block(header, BlockBody(Nil, Nil)),
Nil,
ChainWeight.totalDifficultyOnly(header.difficulty),
Expand Down Expand Up @@ -162,8 +162,8 @@ class GenesisDataLoader(
)

val storageTrie = storage.foldLeft(emptyTrie) {
case (trie, (key, UInt256.Zero)) => trie
case (trie, (key, value)) => trie.put(key, value)
case (trie, (_, UInt256.Zero)) => trie
case (trie, (key, value)) => trie.put(key, value)
}

ByteString(storageTrie.getRootHash)
Expand Down Expand Up @@ -210,7 +210,7 @@ object GenesisDataLoader {
}

object ByteStringJsonSerializer
extends CustomSerializer[ByteString](formats =>
extends CustomSerializer[ByteString](_ =>
(
{ case jv => deserializeByteString(jv) },
PartialFunction.empty
Expand Down
Loading