Skip to content

Commit 04bf2bc

Browse files
author
Leonor Boga
committed
ETCM-963 Refactoring write layer: move "save" from Blockchain class to BlockchainWriter
1 parent fe41195 commit 04bf2bc

Some content is hidden

Large Commits have some content hidden by default. Use the searchbox below for content that may be hidden.

48 files changed

+344
-285
lines changed

src/it/scala/io/iohk/ethereum/ledger/BlockImporterItSpec.scala

Lines changed: 16 additions & 15 deletions
Original file line numberDiff line numberDiff line change
@@ -63,7 +63,7 @@ class BlockImporterItSpec
6363
)
6464
val genesisWeight: ChainWeight = ChainWeight.zero.increase(genesis.header)
6565

66-
blockchain.save(genesis, Seq(), genesisWeight, saveAsBestBlock = true)
66+
blockchainWriter.save(genesis, Seq(), genesisWeight, saveAsBestBlock = true)
6767

6868
lazy val checkpointBlockGenerator: CheckpointBlockGenerator = new CheckpointBlockGenerator
6969

@@ -101,6 +101,7 @@ class BlockImporterItSpec
101101
new BlockExecution(
102102
blockchain,
103103
blockchainReader,
104+
blockchainWriter,
104105
storagesInstance.storages.evmCodeStorage,
105106
blockchainConfig,
106107
consensus.blockPreparator,
@@ -147,10 +148,10 @@ class BlockImporterItSpec
147148
val oldWeight4: ChainWeight = oldWeight3.increase(oldBlock4.header)
148149

149150
//saving initial main chain
150-
blockchain.save(block1, Nil, weight1, saveAsBestBlock = true)
151-
blockchain.save(oldBlock2, Nil, oldWeight2, saveAsBestBlock = true)
152-
blockchain.save(oldBlock3, Nil, oldWeight3, saveAsBestBlock = true)
153-
blockchain.save(oldBlock4, Nil, oldWeight4, saveAsBestBlock = true)
151+
blockchainWriter.save(block1, Nil, weight1, saveAsBestBlock = true)
152+
blockchainWriter.save(oldBlock2, Nil, oldWeight2, saveAsBestBlock = true)
153+
blockchainWriter.save(oldBlock3, Nil, oldWeight3, saveAsBestBlock = true)
154+
blockchainWriter.save(oldBlock4, Nil, oldWeight4, saveAsBestBlock = true)
154155

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

185186
//returning discarded initial chain
186-
blockchain.save(oldBlock2, Nil, oldWeight2, saveAsBestBlock = true)
187-
blockchain.save(oldBlock3, Nil, oldWeight3, saveAsBestBlock = true)
188-
blockchain.save(oldBlock4, Nil, oldWeight4, saveAsBestBlock = true)
187+
blockchainWriter.save(oldBlock2, Nil, oldWeight2, saveAsBestBlock = true)
188+
blockchainWriter.save(oldBlock3, Nil, oldWeight3, saveAsBestBlock = true)
189+
blockchainWriter.save(oldBlock4, Nil, oldWeight4, saveAsBestBlock = true)
189190

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

@@ -206,12 +207,12 @@ class BlockImporterItSpec
206207
ommers = Seq(oldBlock4.header)
207208
)
208209

209-
blockchain.save(oldBlock2, Nil, oldWeight2, saveAsBestBlock = true)
210-
blockchain.save(oldBlock3, Nil, oldWeight3, saveAsBestBlock = true)
211-
blockchain.save(oldBlock4, Nil, oldWeight4, saveAsBestBlock = true)
210+
blockchainWriter.save(oldBlock2, Nil, oldWeight2, saveAsBestBlock = true)
211+
blockchainWriter.save(oldBlock3, Nil, oldWeight3, saveAsBestBlock = true)
212+
blockchainWriter.save(oldBlock4, Nil, oldWeight4, saveAsBestBlock = true)
212213
// simulation of node restart
213214
blockchain.saveBestKnownBlocks(blockchain.getBestBlockNumber() - 1)
214-
blockchain.save(newBlock4ParentOldBlock3, Nil, newBlock4WeightParentOldBlock3, saveAsBestBlock = true)
215+
blockchainWriter.save(newBlock4ParentOldBlock3, Nil, newBlock4WeightParentOldBlock3, saveAsBestBlock = true)
215216

216217
//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
217218
blockImporter ! BlockFetcher.PickedBlocks(NonEmptyList.fromListUnsafe(List(newBlock5ParentOldBlock4)))
@@ -223,7 +224,7 @@ class BlockImporterItSpec
223224

224225
val checkpoint = ObjectGenerators.fakeCheckpointGen(3, 3).sample.get
225226
val oldBlock5WithCheckpoint: Block = checkpointBlockGenerator.generate(oldBlock4, checkpoint)
226-
blockchain.save(oldBlock5WithCheckpoint, Nil, oldWeight4, saveAsBestBlock = true)
227+
blockchainWriter.save(oldBlock5WithCheckpoint, Nil, oldWeight4, saveAsBestBlock = true)
227228

228229
val newBranch = List(newBlock2, newBlock3)
229230

@@ -237,7 +238,7 @@ class BlockImporterItSpec
237238

238239
val checkpoint = ObjectGenerators.fakeCheckpointGen(3, 3).sample.get
239240
val newBlock4WithCheckpoint: Block = checkpointBlockGenerator.generate(newBlock3, checkpoint)
240-
blockchain.save(newBlock4WithCheckpoint, Nil, newWeight3, saveAsBestBlock = true)
241+
blockchainWriter.save(newBlock4WithCheckpoint, Nil, newWeight3, saveAsBestBlock = true)
241242

242243
val newBranch = List(newBlock4WithCheckpoint)
243244

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

256-
blockchain.save(newBlock5, Nil, newWeight5, saveAsBestBlock = true)
257+
blockchainWriter.save(newBlock5, Nil, newWeight5, saveAsBestBlock = true)
257258

258259
val signatures = CheckpointingTestHelpers.createCheckpointSignatures(
259260
Seq(crypto.generateKeyPair(secureRandom)),

src/it/scala/io/iohk/ethereum/sync/util/CommonFakePeer.scala

Lines changed: 16 additions & 14 deletions
Original file line numberDiff line numberDiff line change
@@ -5,16 +5,20 @@ import java.nio.file.Files
55
import java.nio.file.Path
66
import java.time.Clock
77
import java.util.concurrent.atomic.AtomicReference
8+
89
import akka.actor.ActorRef
910
import akka.actor.ActorSystem
1011
import akka.testkit.TestProbe
1112
import akka.util.ByteString
1213
import akka.util.Timeout
14+
1315
import monix.eval.Task
1416

1517
import scala.concurrent.duration.FiniteDuration
1618
import scala.concurrent.duration._
19+
1720
import org.bouncycastle.crypto.AsymmetricCipherKeyPair
21+
1822
import io.iohk.ethereum.Fixtures
1923
import io.iohk.ethereum.Timeouts
2024
import io.iohk.ethereum.blockchain.sync.BlockchainHostActor
@@ -32,15 +36,13 @@ import io.iohk.ethereum.db.storage.AppStateStorage
3236
import io.iohk.ethereum.db.storage.Namespaces
3337
import io.iohk.ethereum.db.storage.pruning.ArchivePruning
3438
import io.iohk.ethereum.db.storage.pruning.PruningMode
35-
import io.iohk.ethereum.domain.{
36-
Block,
37-
Blockchain,
38-
BlockchainImpl,
39-
BlockchainMetadata,
40-
BlockchainReader,
41-
BlockchainWriter,
42-
ChainWeight
43-
}
39+
import io.iohk.ethereum.domain.Block
40+
import io.iohk.ethereum.domain.Blockchain
41+
import io.iohk.ethereum.domain.BlockchainImpl
42+
import io.iohk.ethereum.domain.BlockchainMetadata
43+
import io.iohk.ethereum.domain.BlockchainReader
44+
import io.iohk.ethereum.domain.BlockchainWriter
45+
import io.iohk.ethereum.domain.ChainWeight
4446
import io.iohk.ethereum.ledger.InMemoryWorldStateProxy
4547
import io.iohk.ethereum.mpt.MerklePatriciaTrie
4648
import io.iohk.ethereum.network.EtcPeerManagerActor
@@ -139,8 +141,8 @@ abstract class CommonFakePeer(peerName: String, fakePeerCustomConfig: FakePeerCu
139141
storagesInstance.storages.appStateStorage.getBestBlockNumber(),
140142
storagesInstance.storages.appStateStorage.getLatestCheckpointBlockNumber()
141143
)
142-
val blockchainReader = BlockchainReader(storagesInstance.storages)
143-
val blockchainWriter = BlockchainWriter(storagesInstance.storages, blockchainMetadata)
144+
val blockchainReader: BlockchainReader = BlockchainReader(storagesInstance.storages)
145+
val blockchainWriter: BlockchainWriter = BlockchainWriter(storagesInstance.storages, blockchainMetadata)
144146
val bl: BlockchainImpl = BlockchainImpl(storagesInstance.storages, blockchainReader, blockchainMetadata)
145147
val evmCodeStorage = storagesInstance.storages.evmCodeStorage
146148

@@ -150,7 +152,7 @@ abstract class CommonFakePeer(peerName: String, fakePeerCustomConfig: FakePeerCu
150152
)
151153
val genesisWeight: ChainWeight = ChainWeight.zero.increase(genesis.header)
152154

153-
bl.save(genesis, Seq(), genesisWeight, saveAsBestBlock = true)
155+
blockchainWriter.save(genesis, Seq(), genesisWeight, saveAsBestBlock = true)
154156

155157
lazy val nh = nodeStatusHolder
156158

@@ -351,7 +353,7 @@ abstract class CommonFakePeer(peerName: String, fakePeerCustomConfig: FakePeerCu
351353
val newWeight = ChainWeight.totalDifficultyOnly(1)
352354

353355
broadcastBlock(childBlock, newWeight)
354-
bl.save(childBlock, Seq(), newWeight, saveAsBestBlock = true)
356+
blockchainWriter.save(childBlock, Seq(), newWeight, saveAsBestBlock = true)
355357
}
356358

357359
private def generateValidBlock(
@@ -362,7 +364,7 @@ abstract class CommonFakePeer(peerName: String, fakePeerCustomConfig: FakePeerCu
362364
val currentWorld = getMptForBlock(currentBestBlock)
363365
val (newBlock, newWeight, _) =
364366
createChildBlock(currentBestBlock, currentWeight, currentWorld)(updateWorldForBlock)
365-
bl.save(newBlock, Seq(), newWeight, saveAsBestBlock = true)
367+
blockchainWriter.save(newBlock, Seq(), newWeight, saveAsBestBlock = true)
366368
broadcastBlock(newBlock, newWeight)
367369
}
368370

src/it/scala/io/iohk/ethereum/sync/util/RegularSyncItSpecUtils.scala

Lines changed: 10 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -96,13 +96,22 @@ object RegularSyncItSpecUtils {
9696
new BlockExecution(
9797
bl,
9898
blockchainReader,
99+
blockchainWriter,
99100
storagesInstance.storages.evmCodeStorage,
100101
blockchainConfig,
101102
consensus.blockPreparator,
102103
blockValidation
103104
)
104105
lazy val blockImport: BlockImport =
105-
new BlockImport(bl, blockchainReader, blockQueue, blockValidation, blockExecution, Scheduler.global)
106+
new BlockImport(
107+
bl,
108+
blockchainReader,
109+
blockchainWriter,
110+
blockQueue,
111+
blockValidation,
112+
blockExecution,
113+
Scheduler.global
114+
)
106115

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

src/it/scala/io/iohk/ethereum/txExecTest/ContractTest.scala

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -27,6 +27,7 @@ class ContractTest extends AnyFlatSpec with Matchers {
2727
new BlockExecution(
2828
blockchain,
2929
blockchainReader,
30+
blockchainWriter,
3031
testBlockchainStorages.evmCodeStorage,
3132
blockchainConfig,
3233
consensus.blockPreparator,

src/it/scala/io/iohk/ethereum/txExecTest/ECIP1017Test.scala

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -6,7 +6,9 @@ import org.scalatest.matchers.should.Matchers
66

77
import io.iohk.ethereum.domain.Address
88
import io.iohk.ethereum.domain.BlockchainImpl
9+
import io.iohk.ethereum.domain.BlockchainMetadata
910
import io.iohk.ethereum.domain.BlockchainReader
11+
import io.iohk.ethereum.domain.BlockchainWriter
1012
import io.iohk.ethereum.domain.Receipt
1113
import io.iohk.ethereum.domain.UInt256
1214
import io.iohk.ethereum.ledger.BlockExecution
@@ -83,13 +85,15 @@ class ECIP1017Test extends AnyFlatSpec with Matchers {
8385
storages.appStateStorage.getLatestCheckpointBlockNumber()
8486
)
8587
val blockchainReader = BlockchainReader(storages)
88+
val blockchainWriter = BlockchainWriter(storages, blockchainMetadata)
8689
val blockchain = BlockchainImpl(storages, blockchainReader, blockchainMetadata)
8790
val blockValidation =
8891
new BlockValidation(consensus, blockchainReader, BlockQueue(blockchain, syncConfig))
8992
val blockExecution =
9093
new BlockExecution(
9194
blockchain,
9295
blockchainReader,
96+
blockchainWriter,
9397
testBlockchainStorages.evmCodeStorage,
9498
blockchainConfig,
9599
consensus.blockPreparator,

src/it/scala/io/iohk/ethereum/txExecTest/ForksTest.scala

Lines changed: 5 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -6,7 +6,9 @@ import org.scalatest.matchers.should.Matchers
66

77
import io.iohk.ethereum.domain.Address
88
import io.iohk.ethereum.domain.BlockchainImpl
9+
import io.iohk.ethereum.domain.BlockchainMetadata
910
import io.iohk.ethereum.domain.BlockchainReader
11+
import io.iohk.ethereum.domain.BlockchainWriter
1012
import io.iohk.ethereum.domain.Receipt
1113
import io.iohk.ethereum.domain.UInt256
1214
import io.iohk.ethereum.ledger.BlockExecution
@@ -71,18 +73,20 @@ class ForksTest extends AnyFlatSpec with Matchers {
7173

7274
(startBlock to endBlock).foreach { blockToExecute =>
7375
val storages = FixtureProvider.prepareStorages(blockToExecute - 1, fixtures)
74-
val blockchainReader = BlockchainReader(storages)
7576
val blockchainMetadata = new BlockchainMetadata(
7677
storages.appStateStorage.getBestBlockNumber(),
7778
storages.appStateStorage.getLatestCheckpointBlockNumber()
7879
)
80+
val blockchainReader = BlockchainReader(storages)
81+
val blockchainWriter = BlockchainWriter(storages, blockchainMetadata)
7982
val blockchain = BlockchainImpl(storages, blockchainReader, blockchainMetadata)
8083
val blockValidation =
8184
new BlockValidation(consensus, blockchainReader, BlockQueue(blockchain, syncConfig))
8285
val blockExecution =
8386
new BlockExecution(
8487
blockchain,
8588
blockchainReader,
89+
blockchainWriter,
8690
testBlockchainStorages.evmCodeStorage,
8791
blockchainConfig,
8892
consensus.blockPreparator,

src/it/scala/io/iohk/ethereum/txExecTest/ScenarioSetup.scala

Lines changed: 6 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1,7 +1,11 @@
11
package io.iohk.ethereum.txExecTest
22

33
import io.iohk.ethereum.blockchain.sync.EphemBlockchainTestSetup
4-
import io.iohk.ethereum.domain.{BlockchainImpl, BlockchainMetadata, BlockchainReader, BlockchainStorages}
4+
import io.iohk.ethereum.domain.BlockchainImpl
5+
import io.iohk.ethereum.domain.BlockchainMetadata
6+
import io.iohk.ethereum.domain.BlockchainReader
7+
import io.iohk.ethereum.domain.BlockchainStorages
8+
import io.iohk.ethereum.domain.BlockchainWriter
59
import io.iohk.ethereum.ledger.VMImpl
610

711
trait ScenarioSetup extends EphemBlockchainTestSetup {
@@ -12,6 +16,7 @@ trait ScenarioSetup extends EphemBlockchainTestSetup {
1216
testBlockchainStorages.appStateStorage.getLatestCheckpointBlockNumber()
1317
)
1418
override lazy val blockchainReader: BlockchainReader = BlockchainReader(testBlockchainStorages)
19+
override lazy val blockchainWriter: BlockchainWriter = BlockchainWriter(testBlockchainStorages, blockchainMetadata)
1520
override lazy val blockchain: BlockchainImpl =
1621
BlockchainImpl(testBlockchainStorages, blockchainReader, blockchainMetadata)
1722
override lazy val vm: VMImpl = new VMImpl

src/it/scala/io/iohk/ethereum/txExecTest/util/DumpChainActor.scala

Lines changed: 1 addition & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -48,7 +48,6 @@ import io.iohk.ethereum.txExecTest.util.DumpChainActor._
4848
class DumpChainActor(
4949
peerManager: ActorRef,
5050
peerMessageBus: ActorRef,
51-
startBlock: BigInt,
5251
maxBlocks: BigInt,
5352
bootstrapNode: String
5453
) extends Actor {
@@ -287,12 +286,11 @@ object DumpChainActor {
287286
def props(
288287
peerManager: ActorRef,
289288
peerMessageBus: ActorRef,
290-
startBlock: BigInt,
291289
maxBlocks: BigInt,
292290
bootstrapNode: String
293291
): Props =
294292
Props(
295-
new DumpChainActor(peerManager, peerMessageBus: ActorRef, startBlock: BigInt, maxBlocks: BigInt, bootstrapNode)
293+
new DumpChainActor(peerManager, peerMessageBus: ActorRef, maxBlocks: BigInt, bootstrapNode)
296294
)
297295
val emptyStorage: ByteString = ByteString(
298296
Hex.decode("56e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421")

src/it/scala/io/iohk/ethereum/txExecTest/util/DumpChainApp.scala

Lines changed: 1 addition & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -18,7 +18,6 @@ import io.iohk.ethereum.blockchain.sync.CacheBasedBlacklist
1818
import io.iohk.ethereum.db.components.RocksDbDataSourceComponent
1919
import io.iohk.ethereum.db.components.Storages
2020
import io.iohk.ethereum.db.components.Storages.PruningModeComponent
21-
import io.iohk.ethereum.db.dataSource.DataSourceBatchUpdate
2221
import io.iohk.ethereum.db.storage.AppStateStorage
2322
import io.iohk.ethereum.db.storage.MptStorage
2423
import io.iohk.ethereum.db.storage.NodeStorage.NodeEncoded
@@ -150,7 +149,7 @@ object DumpChainApp
150149
)
151150
peerManager ! PeerManagerActor.StartConnecting
152151

153-
actorSystem.actorOf(DumpChainActor.props(peerManager, peerMessageBus, startBlock, maxBlocks, node), "dumper")
152+
actorSystem.actorOf(DumpChainActor.props(peerManager, peerMessageBus, maxBlocks, node), "dumper")
154153
}
155154

156155
class BlockchainMock(genesisHash: ByteString) extends Blockchain {
@@ -185,14 +184,6 @@ class BlockchainMock(genesisHash: ByteString) extends Blockchain {
185184
ethCompatibleStorage: Boolean
186185
): StorageProof = EmptyStorageValueProof(StorageProofKey(position))
187186

188-
override def storeBlockHeader(blockHeader: BlockHeader): DataSourceBatchUpdate = ???
189-
190-
override def storeBlockBody(blockHash: ByteString, blockBody: BlockBody): DataSourceBatchUpdate = ???
191-
192-
override def storeReceipts(blockHash: ByteString, receipts: Seq[Receipt]): DataSourceBatchUpdate = ???
193-
194-
override def storeChainWeight(blockhash: ByteString, chainWeight: ChainWeight): DataSourceBatchUpdate = ???
195-
196187
override def saveNode(nodeHash: NodeHash, nodeEncoded: NodeEncoded, blockNumber: BigInt): Unit = ???
197188

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

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

218-
override def save(block: Block, receipts: Seq[Receipt], weight: ChainWeight, saveAsBestBlock: Boolean): Unit = ???
219-
220209
override def getLatestCheckpointBlockNumber(): BigInt = ???
221210

222211
override def isInChain(hash: NodeHash): Boolean = ???

src/main/scala/io/iohk/ethereum/blockchain/data/GenesisDataLoader.scala

Lines changed: 5 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -36,8 +36,8 @@ import io.iohk.ethereum.utils.BlockchainConfig
3636
import io.iohk.ethereum.utils.Logger
3737

3838
class GenesisDataLoader(
39-
blockchain: Blockchain,
4039
blockchainReader: BlockchainReader,
40+
blockchainWriter: BlockchainWriter,
4141
stateStorage: StateStorage,
4242
blockchainConfig: BlockchainConfig
4343
) extends Logger {
@@ -122,7 +122,7 @@ class GenesisDataLoader(
122122
case None =>
123123
storage.persist()
124124
stateStorage.forcePersist(GenesisDataLoad)
125-
blockchain.save(
125+
blockchainWriter.save(
126126
Block(header, BlockBody(Nil, Nil)),
127127
Nil,
128128
ChainWeight.totalDifficultyOnly(header.difficulty),
@@ -162,8 +162,8 @@ class GenesisDataLoader(
162162
)
163163

164164
val storageTrie = storage.foldLeft(emptyTrie) {
165-
case (trie, (key, UInt256.Zero)) => trie
166-
case (trie, (key, value)) => trie.put(key, value)
165+
case (trie, (_, UInt256.Zero)) => trie
166+
case (trie, (key, value)) => trie.put(key, value)
167167
}
168168

169169
ByteString(storageTrie.getRootHash)
@@ -210,7 +210,7 @@ object GenesisDataLoader {
210210
}
211211

212212
object ByteStringJsonSerializer
213-
extends CustomSerializer[ByteString](formats =>
213+
extends CustomSerializer[ByteString](_ =>
214214
(
215215
{ case jv => deserializeByteString(jv) },
216216
PartialFunction.empty

0 commit comments

Comments
 (0)