Skip to content

Commit 526eb96

Browse files
author
Aurélien Richez
authored
[ETCM-1044] add best block hash into storage (#1077)
* Adds bestBlock hash in app state * use getBlockByHash to get the best block number instead of getBlockByNumber * Adds a first test in BlockchainReaderSpec
1 parent 5dc67e6 commit 526eb96

24 files changed

+183
-111
lines changed

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

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -107,7 +107,7 @@ class BlockImporterItSpec
107107
blockchainWriter.save(oldBlock3, Nil, oldWeight3, saveAsBestBlock = true)
108108
blockchainWriter.save(oldBlock4, Nil, oldWeight4, saveAsBestBlock = true)
109109
// simulation of node restart
110-
blockchain.saveBestKnownBlocks(blockchainReader.getBestBlockNumber() - 1)
110+
blockchain.saveBestKnownBlocks(oldBlock3.header.hash, oldBlock3.number)
111111
blockchainWriter.save(newBlock4ParentOldBlock3, Nil, newBlock4WeightParentOldBlock3, saveAsBestBlock = true)
112112

113113
//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

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

Lines changed: 6 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -196,7 +196,11 @@ class BlockchainMock(genesisHash: ByteString) extends Blockchain {
196196

197197
def getBestBlockNumber(): BigInt = ???
198198

199-
def saveBestKnownBlocks(bestBlockNumber: BigInt, latestCheckpointNumber: Option[BigInt] = None): Unit = ???
199+
override def saveBestKnownBlocks(
200+
bestBlockhash: ByteString,
201+
bestBlockNumber: BigInt,
202+
latestCheckpointNumber: Option[BigInt] = None
203+
): Unit = ???
200204

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

@@ -205,4 +209,5 @@ class BlockchainMock(genesisHash: ByteString) extends Blockchain {
205209
override def getBackingMptStorage(blockNumber: BigInt): MptStorage = ???
206210

207211
override def getReadOnlyMptStorage(): MptStorage = ???
212+
208213
}

src/main/scala/io/iohk/ethereum/blockchain/sync/fast/FastSync.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -941,7 +941,7 @@ class FastSync(
941941
}
942942

943943
def assignBlockchainWork(peerWithInfo: PeerWithInfo): Unit = {
944-
val PeerWithInfo(peer, peerInfo) = peerWithInfo
944+
val PeerWithInfo(peer, _) = peerWithInfo
945945
log.debug(s"Assigning blockchain work for peer [{}]", peer.id.value)
946946
if (syncState.receiptsQueue.nonEmpty) {
947947
requestReceipts(peer)

src/main/scala/io/iohk/ethereum/consensus/ConsensusImpl.scala

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -302,9 +302,9 @@ class ConsensusImpl(
302302
case BlockData(block, _, _) if block.hasCheckpoint => block.number
303303
}.maximumOption
304304

305-
val bestNumber = oldBranch.last.block.header.number
306-
blockchain.saveBestKnownBlocks(bestNumber, checkpointNumber)
307-
executedBlocks.foreach(data => blockQueue.enqueueBlock(data.block, bestNumber))
305+
val bestHeader = oldBranch.last.block.header
306+
blockchain.saveBestKnownBlocks(bestHeader.hash, bestHeader.number, checkpointNumber)
307+
executedBlocks.foreach(data => blockQueue.enqueueBlock(data.block, bestHeader.number))
308308

309309
newBranch.diff(executedBlocks.map(_.block)).headOption.foreach { block =>
310310
blockQueue.removeSubtree(block.header.hash)

src/main/scala/io/iohk/ethereum/db/storage/AppStateStorage.scala

Lines changed: 16 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -2,11 +2,15 @@ package io.iohk.ethereum.db.storage
22

33
import java.math.BigInteger
44

5+
import akka.util.ByteString
6+
57
import scala.collection.immutable.ArraySeq
68

79
import io.iohk.ethereum.db.dataSource.DataSource
810
import io.iohk.ethereum.db.dataSource.DataSourceBatchUpdate
911
import io.iohk.ethereum.db.storage.AppStateStorage._
12+
import io.iohk.ethereum.domain.appstate.BlockInfo
13+
import io.iohk.ethereum.utils.Hex
1014

1115
/** This class is used to store app state variables
1216
* Key: see AppStateStorage.Keys
@@ -27,6 +31,16 @@ class AppStateStorage(val dataSource: DataSource) extends TransactionalKeyValueS
2731
def getBestBlockNumber(): BigInt =
2832
getBigInt(Keys.BestBlockNumber)
2933

34+
def getBestBlockInfo(): BlockInfo =
35+
BlockInfo( // TODO ETCM-1090 provide the genesis hash as default
36+
get(Keys.BestBlockHash).map(v => ByteString(Hex.decode(v))).getOrElse(ByteString.empty),
37+
getBigInt(Keys.BestBlockNumber)
38+
)
39+
40+
def putBestBlockInfo(b: BlockInfo): DataSourceBatchUpdate =
41+
put(Keys.BestBlockNumber, b.number.toString)
42+
.and(put(Keys.BestBlockHash, Hex.toHexString(b.hash.toArray)))
43+
3044
def putBestBlockNumber(bestBlockNumber: BigInt): DataSourceBatchUpdate =
3145
put(Keys.BestBlockNumber, bestBlockNumber.toString)
3246

@@ -72,9 +86,11 @@ object AppStateStorage {
7286

7387
object Keys {
7488
val BestBlockNumber = "BestBlockNumber"
89+
val BestBlockHash = "BestBlockHash"
7590
val FastSyncDone = "FastSyncDone"
7691
val EstimatedHighestBlock = "EstimatedHighestBlock"
7792
val SyncStartingBlock = "SyncStartingBlock"
7893
val LatestCheckpointBlockNumber = "LatestCheckpointBlockNumber"
7994
}
95+
8096
}

src/main/scala/io/iohk/ethereum/domain/Blockchain.scala

Lines changed: 27 additions & 14 deletions
Original file line numberDiff line numberDiff line change
@@ -7,6 +7,7 @@ import scala.annotation.tailrec
77
import io.iohk.ethereum.db.dataSource.DataSourceBatchUpdate
88
import io.iohk.ethereum.db.storage._
99
import io.iohk.ethereum.domain
10+
import io.iohk.ethereum.domain.appstate.BlockInfo
1011
import io.iohk.ethereum.jsonrpc.ProofService.StorageProof
1112
import io.iohk.ethereum.ledger.InMemoryWorldStateProxy
1213
import io.iohk.ethereum.ledger.InMemoryWorldStateProxyStorage
@@ -64,7 +65,11 @@ trait Blockchain {
6465

6566
def removeBlock(hash: ByteString): Unit
6667

67-
def saveBestKnownBlocks(bestBlockNumber: BigInt, latestCheckpointNumber: Option[BigInt] = None): Unit
68+
def saveBestKnownBlocks(
69+
bestBlockHash: ByteString,
70+
bestBlockNumber: BigInt,
71+
latestCheckpointNumber: Option[BigInt] = None
72+
): Unit
6873

6974
}
7075

@@ -126,20 +131,28 @@ class BlockchainImpl(
126131

127132
def getReadOnlyMptStorage(): MptStorage = stateStorage.getReadOnlyStorage
128133

129-
override def saveBestKnownBlocks(bestBlockNumber: BigInt, latestCheckpointNumber: Option[BigInt] = None): Unit =
134+
override def saveBestKnownBlocks(
135+
bestBlockHash: ByteString,
136+
bestBlockNumber: BigInt,
137+
latestCheckpointNumber: Option[BigInt] = None
138+
): Unit =
130139
latestCheckpointNumber match {
131140
case Some(number) =>
132-
saveBestKnownBlockAndLatestCheckpointNumber(bestBlockNumber, number)
141+
saveBestKnownBlockAndLatestCheckpointNumber(bestBlockHash, bestBlockNumber, number)
133142
case None =>
134-
saveBestKnownBlock(bestBlockNumber)
143+
saveBestKnownBlock(bestBlockHash, bestBlockNumber)
135144
}
136145

137-
private def saveBestKnownBlock(bestBlockNumber: BigInt): Unit =
138-
appStateStorage.putBestBlockNumber(bestBlockNumber).commit()
146+
private def saveBestKnownBlock(bestBlockHash: ByteString, bestBlockNumber: BigInt): Unit =
147+
appStateStorage.putBestBlockInfo(BlockInfo(bestBlockHash, bestBlockNumber)).commit()
139148

140-
private def saveBestKnownBlockAndLatestCheckpointNumber(number: BigInt, latestCheckpointNumber: BigInt): Unit =
149+
private def saveBestKnownBlockAndLatestCheckpointNumber(
150+
bestBlockHash: ByteString,
151+
number: BigInt,
152+
latestCheckpointNumber: BigInt
153+
): Unit =
141154
appStateStorage
142-
.putBestBlockNumber(number)
155+
.putBestBlockInfo(BlockInfo(bestBlockHash, number))
143156
.and(appStateStorage.putLatestCheckpointBlockNumber(latestCheckpointNumber))
144157
.commit()
145158

@@ -163,15 +176,15 @@ class BlockchainImpl(
163176
log.debug(s"Trying to remove block ${block.idTag}")
164177

165178
val txList = block.body.transactionList
166-
val bestBlockNumber = blockchainReader.getBestBlockNumber()
167179
val latestCheckpointNumber = getLatestCheckpointBlockNumber()
168180

169181
val blockNumberMappingUpdates =
170182
if (blockchainReader.getHashByBlockNumber(blockchainReader.getBestBranch(), block.number).contains(blockHash))
171183
removeBlockNumberMapping(block.number)
172184
else blockNumberMappingStorage.emptyBatchUpdate
173185

174-
val newBestBlockNumber: BigInt = (bestBlockNumber - 1).max(0)
186+
val potentialNewBestBlockNumber: BigInt = (block.number - 1).max(0)
187+
val potentialNewBestBlockHash: ByteString = block.header.parentHash
175188
val newLatestCheckpointNumber: BigInt =
176189
if (block.hasCheckpoint && block.number == latestCheckpointNumber) {
177190
findPreviousCheckpointBlockNumber(block.number, block.number)
@@ -187,8 +200,8 @@ class BlockchainImpl(
187200
into the case of having an incomplete best block and so an inconsistent db
188201
*/
189202
val bestBlockNumberUpdates =
190-
if (appStateStorage.getBestBlockNumber() > newBestBlockNumber)
191-
appStateStorage.putBestBlockNumber(newBestBlockNumber)
203+
if (appStateStorage.getBestBlockNumber() > potentialNewBestBlockNumber)
204+
appStateStorage.putBestBlockInfo(BlockInfo(potentialNewBestBlockHash, potentialNewBestBlockNumber))
192205
else appStateStorage.emptyBatchUpdate
193206
val latestCheckpointNumberUpdates =
194207
if (appStateStorage.getLatestCheckpointBlockNumber() > newLatestCheckpointNumber)
@@ -197,7 +210,7 @@ class BlockchainImpl(
197210

198211
log.debug(
199212
"Persisting app info data into database. Persisted block number is {}. Persisted checkpoint number is {}",
200-
newBestBlockNumber,
213+
potentialNewBestBlockNumber,
201214
newLatestCheckpointNumber
202215
)
203216

@@ -215,7 +228,7 @@ class BlockchainImpl(
215228
log.debug(
216229
"Removed block with hash {}. New best block number - {}, new best checkpoint block number - {}",
217230
ByteStringUtils.hash2string(blockHash),
218-
newBestBlockNumber,
231+
potentialNewBestBlockNumber,
219232
newLatestCheckpointNumber
220233
)
221234
}

src/main/scala/io/iohk/ethereum/domain/BlockchainReader.scala

Lines changed: 12 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -13,6 +13,7 @@ import io.iohk.ethereum.domain.branch.Branch
1313
import io.iohk.ethereum.domain.branch.EmptyBranch
1414
import io.iohk.ethereum.mpt.MerklePatriciaTrie
1515
import io.iohk.ethereum.mpt.MptNode
16+
import io.iohk.ethereum.utils.Hex
1617
import io.iohk.ethereum.utils.Logger
1718

1819
class BlockchainReader(
@@ -84,9 +85,17 @@ class BlockchainReader(
8485

8586
//returns the best known block if it's available in the storage
8687
def getBestBlock(): Option[Block] = {
87-
val bestBlockNumber = getBestBlockNumber()
88-
log.debug("Trying to get best block with number {}", bestBlockNumber)
89-
getBlockByNumber(bestBlockNumber)
88+
val bestKnownBlockinfo = appStateStorage.getBestBlockInfo()
89+
log.debug("Trying to get best block with number {}", bestKnownBlockinfo.number)
90+
val bestBlock = getBlockByHash(bestKnownBlockinfo.hash)
91+
if (bestBlock.isEmpty) {
92+
log.error(
93+
"Best block {} (number: {}) not found in storage.",
94+
Hex.toHexString(bestKnownBlockinfo.hash.toArray),
95+
bestKnownBlockinfo.number
96+
)
97+
}
98+
bestBlock
9099
}
91100

92101
def genesisHeader: BlockHeader =

src/main/scala/io/iohk/ethereum/domain/BlockchainWriter.scala

Lines changed: 3 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -11,6 +11,7 @@ import io.iohk.ethereum.db.storage.ChainWeightStorage
1111
import io.iohk.ethereum.db.storage.ReceiptStorage
1212
import io.iohk.ethereum.db.storage.TransactionMappingStorage
1313
import io.iohk.ethereum.db.storage.TransactionMappingStorage.TransactionLocation
14+
import io.iohk.ethereum.domain.appstate.BlockInfo
1415
import io.iohk.ethereum.utils.Logger
1516

1617
class BlockchainWriter(
@@ -31,14 +32,14 @@ class BlockchainWriter(
3132
block.header.number
3233
)
3334
appStateStorage
34-
.putBestBlockNumber(block.header.number)
35+
.putBestBlockInfo(BlockInfo(block.header.hash, block.header.number))
3536
.and(appStateStorage.putLatestCheckpointBlockNumber(block.header.number))
3637
} else if (saveAsBestBlock) {
3738
log.debug(
3839
"New best known block number - {}",
3940
block.header.number
4041
)
41-
appStateStorage.putBestBlockNumber(block.header.number)
42+
appStateStorage.putBestBlockInfo(BlockInfo(block.header.hash, block.header.number))
4243
} else {
4344
appStateStorage.emptyBatchUpdate
4445
}
Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,5 @@
1+
package io.iohk.ethereum.domain.appstate
2+
3+
import akka.util.ByteString
4+
5+
case class BlockInfo(hash: ByteString, number: BigInt)

src/main/scala/io/iohk/ethereum/nodebuilder/StdNode.scala

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

33
import akka.actor.typed.ActorSystem
4+
import akka.util.ByteString
45

56
import scala.concurrent.Await
67
import scala.concurrent.ExecutionContext.Implicits.global
@@ -18,6 +19,7 @@ import io.iohk.ethereum.network.discovery.PeerDiscoveryManager
1819
import io.iohk.ethereum.nodebuilder.tooling.PeriodicConsistencyCheck
1920
import io.iohk.ethereum.nodebuilder.tooling.StorageConsistencyChecker
2021
import io.iohk.ethereum.utils.Config
22+
import io.iohk.ethereum.utils.Hex
2123

2224
/** A standard node is everything Ethereum prescribes except the mining algorithm,
2325
* which is plugged in dynamically.
@@ -32,6 +34,8 @@ abstract class BaseNode extends Node {
3234
def start(): Unit = {
3335
startMetricsClient()
3436

37+
fixDatabase()
38+
3539
loadGenesisData()
3640

3741
runDBConsistencyCheck()
@@ -132,6 +136,24 @@ abstract class BaseNode extends Node {
132136
tryAndLogFailure(() => Metrics.get().close())
133137
tryAndLogFailure(() => storagesInstance.dataSource.close())
134138
}
139+
140+
def fixDatabase(): Unit = {
141+
// FIXME this is a temporary solution to avoid an incompatibility due to the introduction of the best block hash
142+
// We can remove this fix when we release an incompatible version.
143+
val bestBlockInfo = storagesInstance.storages.appStateStorage.getBestBlockInfo()
144+
if (bestBlockInfo.hash == ByteString.empty && bestBlockInfo.number > 0) {
145+
log.warn("Fixing best block hash into database for block {}", bestBlockInfo.number)
146+
storagesInstance.storages.blockNumberMappingStorage.get(bestBlockInfo.number) match {
147+
case Some(hash) =>
148+
log.warn("Putting {} as the best block hash", Hex.toHexString(hash.toArray))
149+
storagesInstance.storages.appStateStorage.putBestBlockInfo(bestBlockInfo.copy(hash = hash)).commit()
150+
case None =>
151+
log.error("No block found for number {} when trying to fix database", bestBlockInfo.number)
152+
}
153+
154+
}
155+
156+
}
135157
}
136158

137159
class StdNode extends BaseNode with StdMiningBuilder

src/test/scala/io/iohk/ethereum/ObjectGenerators.scala

Lines changed: 0 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -33,8 +33,6 @@ trait ObjectGenerators {
3333

3434
def intGen(min: Int, max: Int): Gen[Int] = Gen.choose(min, max)
3535

36-
def posIntGen(min: Int, max: Int): Gen[Int] = Gen.choose(min, max).suchThat(_ > 0)
37-
3836
def intGen: Gen[Int] = Gen.choose(Int.MinValue, Int.MaxValue)
3937

4038
def longGen: Gen[Long] = Gen.choose(Long.MinValue, Long.MaxValue)

src/test/scala/io/iohk/ethereum/blockchain/sync/SyncStateSchedulerSpec.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -252,7 +252,7 @@ class SyncStateSchedulerSpec
252252
buildScheduler()
253253
val header = Fixtures.Blocks.ValidBlock.header.copy(stateRoot = worldHash, number = 1)
254254
schedulerBlockchainWriter.storeBlockHeader(header).commit()
255-
schedulerBlockchain.saveBestKnownBlocks(1)
255+
schedulerBlockchain.saveBestKnownBlocks(header.hash, 1)
256256
var state = scheduler.initState(worldHash).get
257257
while (state.activeRequest.nonEmpty) {
258258
val (allMissingNodes1, state2) = scheduler.getAllMissingNodes(state)

src/test/scala/io/iohk/ethereum/blockchain/sync/regular/RegularSyncSpec.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -738,7 +738,7 @@ class RegularSyncSpec
738738
goToTop()
739739

740740
val num: BigInt = 42
741-
blockchain.saveBestKnownBlocks(num, Some(num))
741+
blockchain.saveBestKnownBlocks(testBlocks.head.hash, num, Some(num))
742742

743743
etcPeerManager.expectMsg(GetHandshakedPeers)
744744
etcPeerManager.reply(HandshakedPeers(handshakedPeers))

src/test/scala/io/iohk/ethereum/consensus/pow/validators/StdOmmersValidatorSpec.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -464,7 +464,7 @@ class StdOmmersValidatorSpec extends AnyFlatSpec with Matchers with ScalaCheckPr
464464
.and(blockchainWriter.storeBlock(block95))
465465
.and(blockchainWriter.storeBlock(block96))
466466
.commit()
467-
blockchain.saveBestKnownBlocks(block96.number)
467+
blockchain.saveBestKnownBlocks(block96.hash, block96.number)
468468

469469
}
470470
}
Lines changed: 25 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,25 @@
1+
package io.iohk.ethereum.domain
2+
3+
import org.bouncycastle.util.encoders.Hex
4+
import org.scalatest.flatspec.AnyFlatSpec
5+
import org.scalatest.matchers.should.Matchers
6+
import org.scalatestplus.scalacheck.ScalaCheckPropertyChecks
7+
8+
import io.iohk.ethereum.ObjectGenerators
9+
import io.iohk.ethereum.blockchain.sync.EphemBlockchainTestSetup
10+
import io.iohk.ethereum.network.p2p.messages.BaseETH6XMessages.NewBlock
11+
import io.iohk.ethereum.security.SecureRandomBuilder
12+
13+
class BlockchainReaderSpec extends AnyFlatSpec with Matchers with ScalaCheckPropertyChecks with SecureRandomBuilder {
14+
15+
val chainId: Option[Byte] = Hex.decode("3d").headOption
16+
17+
"BlockchainReader" should "be able to get the best block after it was stored by BlockchainWriter" in new EphemBlockchainTestSetup {
18+
forAll(ObjectGenerators.newBlockGen(secureRandom, chainId)) { case NewBlock(block, weight) =>
19+
blockchainWriter.save(block, Nil, ChainWeight(0, weight), true)
20+
21+
blockchainReader.getBestBlock() shouldBe Some(block)
22+
}
23+
}
24+
25+
}

0 commit comments

Comments
 (0)