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
Original file line number Diff line number Diff line change
Expand Up @@ -5,6 +5,7 @@ import io.iohk.ethereum.domain.{Blockchain, BlockchainReader}
import monix.eval.Task
import org.bouncycastle.util.encoders.Hex
import io.iohk.ethereum.consensus.Consensus
import io.iohk.ethereum.ledger.BlockQueue

object EthBlocksService {
case class BestBlockNumberRequest()
Expand Down Expand Up @@ -38,7 +39,8 @@ object EthBlocksService {
class EthBlocksService(
val blockchain: Blockchain,
val blockchainReader: BlockchainReader,
val consensus: Consensus
val consensus: Consensus,
val blockQueue: BlockQueue
) extends ResolveBlock {
import EthBlocksService._

Expand Down Expand Up @@ -71,8 +73,8 @@ class EthBlocksService(
*/
def getByBlockHash(request: BlockByBlockHashRequest): ServiceResponse[BlockByBlockHashResponse] = Task {
val BlockByBlockHashRequest(blockHash, fullTxs) = request
val blockOpt = blockchainReader.getBlockByHash(blockHash)
val weight = blockchain.getChainWeightByHash(blockHash)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Are these two changes required by ETS? I wonder what would be a convenient way to refactor this with ledger gone...

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The underlying need was to be able to get a block by hash, no matter if it's on the canonical chain or an alternate one. The solution was to delegate the work to the ledger, which implements the getBlockByHash exactly as it (first look in the canonical blockchain, then into the BlockQueue). The getChainWeightByHash was added to match the getBlockByHash behaviour.

The feature (being able to query a block by its hash, no matter its location) is required by the ETS. After a successful insert, retesteth queries for the insert block by its hash to do some consistency check.
It is not restricted to the ledger, so if any other component provides this after the refactor, let's use the new one.

val blockOpt = blockchainReader.getBlockByHash(blockHash) orElse blockQueue.getBlockByHash(blockHash)
val weight = blockchain.getChainWeightByHash(blockHash) orElse blockQueue.getChainWeightByHash(blockHash)

val blockResponseOpt = blockOpt.map(block => BlockResponse(block, weight, fullTxs = fullTxs))
Right(BlockByBlockHashResponse(blockResponseOpt))
Expand Down
16 changes: 14 additions & 2 deletions src/main/scala/io/iohk/ethereum/jsonrpc/TestService.scala
Original file line number Diff line number Diff line change
Expand Up @@ -136,6 +136,10 @@ class TestService(
def setChainParams(request: SetChainParamsRequest): ServiceResponse[SetChainParamsResponse] = {
currentConfig = buildNewConfig(request.chainParams.blockchainParams)

// clear ledger's cache on test start
// setChainParams is expected to be the first remote call for each test
testModeComponentsProvider.clearState()

val genesisData = GenesisData(
nonce = request.chainParams.genesis.nonce,
mixHash = Some(request.chainParams.genesis.mixHash),
Expand All @@ -158,6 +162,9 @@ class TestService(

// remove current genesis (Try because it may not exist)
Try(blockchain.removeBlock(blockchain.genesisHeader.hash, withState = false))
// TODO clear the storage ? When relaunching some tests on the same running test mantis client,
// we end up with duplicate blocks because they are still present in the storage layer
// for example: bcMultiChainTest/ChainAtoChainB_BlockHash_Istanbul
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

How about when setChainParams is called?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

do you mean put the cleaning into the setChainParams call, or into the setChainParams caller ?


// load the new genesis
val genesisDataLoader = new GenesisDataLoader(blockchain, blockchainReader, stateStorage, currentConfig)
Expand Down Expand Up @@ -276,15 +283,20 @@ class TestService(
testModeComponentsProvider
.blockImport(currentConfig, preimageCache, sealEngine)
.importBlock(value)
.flatMap(handleResult)
.flatMap(handleResult(value))
}
}

private def handleResult(blockImportResult: BlockImportResult): ServiceResponse[ImportRawBlockResponse] = {
private def handleResult(
block: Block
)(blockImportResult: BlockImportResult): ServiceResponse[ImportRawBlockResponse] = {
blockImportResult match {
case BlockImportedToTop(blockImportData) =>
val blockHash = s"0x${ByteStringUtils.hash2string(blockImportData.head.block.header.hash)}"
ImportRawBlockResponse(blockHash).rightNow
case BlockEnqueued | ChainReorganised(_, _, _) =>
val blockHash = s"0x${ByteStringUtils.hash2string(block.hash)}"
ImportRawBlockResponse(blockHash).rightNow
case e =>
log.warn("Block import failed with {}", e)
Task.now(Left(JsonRpcError(-1, "block validation failed!", None)))
Expand Down
17 changes: 17 additions & 0 deletions src/main/scala/io/iohk/ethereum/ledger/BlockQueue.scala
Original file line number Diff line number Diff line change
Expand Up @@ -82,6 +82,14 @@ class BlockQueue(blockchain: Blockchain, val maxQueuedBlockNumberAhead: Int, val
def isQueued(hash: ByteString): Boolean =
blocks.contains(hash)

/**
* Returns the weight of the block corresponding to the hash, or None if not found
* @param hash the block's hash to get the weight from
* @return the weight of the block corresponding to the hash, or None if not found
*/
def getChainWeightByHash(hash: ByteString): Option[ChainWeight] =
blocks.get(hash).flatMap(_.weight)

/**
* Takes a branch going from descendant block upwards to the oldest ancestor
* @param descendant the youngest block to be removed
Expand Down Expand Up @@ -124,6 +132,14 @@ class BlockQueue(blockchain: Blockchain, val maxQueuedBlockNumberAhead: Int, val
parentToChildren -= block.header.hash
}

/**
* Clear the BlockQueue
*/
def clear(): Unit = {
blocks.clear()
parentToChildren.clear()
}

/**
* Removes stale blocks - too old or too young in relation the current best block number
* @param bestBlockNumber - best block number of the main chain
Expand Down Expand Up @@ -193,4 +209,5 @@ class BlockQueue(blockchain: Blockchain, val maxQueuedBlockNumberAhead: Int, val
private def isNumberOutOfRange(blockNumber: BigInt, bestBlockNumber: BigInt): Boolean =
blockNumber - bestBlockNumber > maxQueuedBlockNumberAhead ||
bestBlockNumber - blockNumber > maxQueuedBlockNumberBehind

}
9 changes: 5 additions & 4 deletions src/main/scala/io/iohk/ethereum/nodebuilder/NodeBuilder.scala
Original file line number Diff line number Diff line change
Expand Up @@ -432,8 +432,9 @@ trait TestServiceBuilder {
}

trait TestEthBlockServiceBuilder extends EthBlocksServiceBuilder {
self: TestBlockchainBuilder with TestModeServiceBuilder with ConsensusBuilder =>
override lazy val ethBlocksService = new TestEthBlockServiceWrapper(blockchain, blockchainReader, consensus)
self: TestBlockchainBuilder with TestModeServiceBuilder with ConsensusBuilder with BlockQueueBuilder =>
override lazy val ethBlocksService =
new TestEthBlockServiceWrapper(blockchain, blockchainReader, consensus, blockQueue)
}

trait EthProofServiceBuilder {
Expand Down Expand Up @@ -509,9 +510,9 @@ trait EthTxServiceBuilder {
}

trait EthBlocksServiceBuilder {
self: BlockchainBuilder with ConsensusBuilder =>
self: BlockchainBuilder with ConsensusBuilder with BlockQueueBuilder =>

lazy val ethBlocksService = new EthBlocksService(blockchain, blockchainReader, consensus)
lazy val ethBlocksService = new EthBlocksService(blockchain, blockchainReader, consensus, blockQueue)
}

trait EthUserServiceBuilder {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -6,18 +6,22 @@ import io.iohk.ethereum.jsonrpc.{
BaseBlockResponse,
BaseTransactionResponse,
EthBlocksService,
JsonRpcError,
ServiceResponse,
TransactionData
}
import io.iohk.ethereum.utils.Logger
import io.iohk.ethereum.utils.ByteStringUtils._
import akka.util.ByteString
import io.iohk.ethereum.consensus.Consensus
import io.iohk.ethereum.ledger.BlockQueue

class TestEthBlockServiceWrapper(
blockchain: Blockchain,
blockchainReader: BlockchainReader,
consensus: Consensus
) extends EthBlocksService(blockchain, blockchainReader, consensus)
consensus: Consensus,
blockQueue: BlockQueue
) extends EthBlocksService(blockchain, blockchainReader, consensus, blockQueue)
with Logger {

/**
Expand All @@ -31,10 +35,29 @@ class TestEthBlockServiceWrapper(
): ServiceResponse[EthBlocksService.BlockByBlockHashResponse] = super
.getByBlockHash(request)
.map(
_.map(blockByBlockResponse => {
val fullBlock = blockchainReader.getBlockByNumber(blockByBlockResponse.blockResponse.get.number).get
BlockByBlockHashResponse(blockByBlockResponse.blockResponse.map(response => toEthResponse(fullBlock, response)))
})
_.flatMap {

case BlockByBlockHashResponse(None) =>
Left(JsonRpcError.LogicError(s"EthBlockService: unable to find block for hash ${request.blockHash.toHex}"))

case BlockByBlockHashResponse(Some(baseBlockResponse)) if baseBlockResponse.hash.isEmpty =>
Left(JsonRpcError.LogicError(s"missing hash for block $baseBlockResponse"))

case BlockByBlockHashResponse(Some(baseBlockResponse)) =>
val ethResponseOpt = for {
hash <- baseBlockResponse.hash
fullBlock <- blockchainReader.getBlockByHash(hash) orElse blockQueue.getBlockByHash(hash)
} yield toEthResponse(fullBlock, baseBlockResponse)

ethResponseOpt match {
case None =>
Left(
JsonRpcError.LogicError(s"Ledger: unable to find block for hash=${baseBlockResponse.hash.get.toHex}")
)
case Some(_) =>
Right(BlockByBlockHashResponse(ethResponseOpt))
}
}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is a bit hard to follow, might be nicer as for-comprehension?

)

/**
Expand Down Expand Up @@ -122,9 +145,9 @@ final case class EthTransactionResponse(
gasPrice: BigInt,
gas: BigInt,
input: ByteString,
r: ByteString,
s: ByteString,
v: ByteString
r: BigInt,
s: BigInt,
v: BigInt
) extends BaseTransactionResponse

object EthTransactionResponse {
Expand All @@ -149,8 +172,8 @@ object EthTransactionResponse {
gasPrice = stx.tx.gasPrice,
gas = stx.tx.gasLimit,
input = stx.tx.payload,
r = UInt256(stx.signature.r).bytes,
s = UInt256(stx.signature.s).bytes,
v = ByteString(stx.signature.v)
r = stx.signature.r,
s = stx.signature.s,
v = stx.signature.v
)
}
Original file line number Diff line number Diff line change
Expand Up @@ -2,7 +2,7 @@ package io.iohk.ethereum.testmode

import akka.util.ByteString
import io.iohk.ethereum.consensus.difficulty.DifficultyCalculator
import io.iohk.ethereum.consensus.{Consensus, ConsensusConfig}
import io.iohk.ethereum.consensus.ConsensusConfig
import io.iohk.ethereum.crypto
import io.iohk.ethereum.db.storage.EvmCodeStorage
import io.iohk.ethereum.domain.{BlockchainImpl, BlockchainReader, UInt256}
Expand All @@ -13,9 +13,10 @@ import io.iohk.ethereum.utils.Config.SyncConfig
import monix.execution.Scheduler
import io.iohk.ethereum.ledger.BlockImport
import io.iohk.ethereum.ledger.BlockValidation
import io.iohk.ethereum.ledger.BlockExecution
import io.iohk.ethereum.ledger.BlockQueue

import scala.collection.immutable.HashMap

/** Provides a ledger or consensus instances with modifiable blockchain config (used in test mode). */
class TestModeComponentsProvider(
blockchain: BlockchainImpl,
Expand All @@ -28,14 +29,19 @@ class TestModeComponentsProvider(
vm: VMImpl
) {

// private var cache = HashMap.empty[(BlockchainConfig, SealEngineType), BlockImport]
private val internalBlockQueue = BlockQueue(blockchain, syncConfig)

def blockQueue(): BlockQueue = internalBlockQueue

def blockImport(
blockchainConfig: BlockchainConfig,
preimageCache: collection.concurrent.Map[ByteString, UInt256],
sealEngine: SealEngineType
): BlockImport = {
val blockQueue = BlockQueue(blockchain, syncConfig)
// val blockQueue = BlockQueue(blockchain, syncConfig)
val consensuz = consensus(blockchainConfig, sealEngine)
val blockValidation = new BlockValidation(consensuz, blockchainReader, blockQueue)
val blockValidation = new BlockValidation(consensuz, blockchainReader, internalBlockQueue)
val blockExecution =
new TestModeBlockExecution(
blockchain,
Expand All @@ -50,13 +56,22 @@ class TestModeComponentsProvider(
new BlockImport(
blockchain,
blockchainReader,
blockQueue,
internalBlockQueue,
blockValidation,
blockExecution,
validationExecutionContext
)
}

/**
* Clear the internal builder state
*/
def clearState(): Unit = {
// blockQueue = BlockQueue(blockchain, syncConfig)
// cache = cache.empty
internalBlockQueue.clear()
}

def stxLedger(blockchainConfig: BlockchainConfig, sealEngine: SealEngineType): StxLedger =
new StxLedger(
blockchain,
Expand Down
Original file line number Diff line number Diff line change
@@ -1,13 +1,9 @@
package io.iohk.ethereum.testmode

import akka.util.ByteString
import cats.data.NonEmptyList
import io.iohk.ethereum.consensus.difficulty.DifficultyCalculator
import io.iohk.ethereum.consensus.{Consensus, ConsensusBuilder, ConsensusConfigBuilder}
import io.iohk.ethereum.domain._
import io.iohk.ethereum.ledger._
import io.iohk.ethereum.nodebuilder.{ActorSystemBuilder, _}
import monix.eval.Task
import monix.execution.Scheduler

trait TestModeServiceBuilder extends StxLedgerBuilder {
Expand All @@ -18,6 +14,7 @@ trait TestModeServiceBuilder extends StxLedgerBuilder {
with ConsensusBuilder
with ActorSystemBuilder
with ConsensusConfigBuilder
with BlockQueueBuilder
with VmBuilder =>

val scheduler = Scheduler(system.dispatchers.lookup("validation-context"))
Expand All @@ -34,6 +31,26 @@ trait TestModeServiceBuilder extends StxLedgerBuilder {
vm
)

override lazy val blockQueue: BlockQueue = testModeComponentsProvider.blockQueue();

//<<<<<<< HEAD
//=======
// private def testLedger: Ledger = testModeComponentsProvider.ledger(blockchainConfig, SealEngineType.NoReward)
//
// class TestLedgerProxy extends Ledger {
// override def consensus: Consensus = testLedger.consensus
// override def checkBlockStatus(blockHash: ByteString): BlockStatus = testLedger.checkBlockStatus(blockHash)
// override def getBlockByHash(hash: ByteString): Option[Block] = testLedger.getBlockByHash(hash)
// override def importBlock(block: Block)(implicit
// blockExecutionScheduler: Scheduler
// ): Task[BlockImportResult] = testLedger.importBlock(block)
// override def resolveBranch(headers: NonEmptyList[BlockHeader]): BranchResolutionResult =
// testLedger.resolveBranch(headers)
// override def getChainWeightByHash(hash: ByteString): Option[ChainWeight] = testLedger.getChainWeightByHash(hash)
// }
//
// override lazy val ledger: Ledger = new TestLedgerProxy
//>>>>>>> f521a3125 ([ETCM-927] enhance BlockchainTests/ValidBlocks/bcMultiChainTest/ChainAtoChainB_difficultyB test)
override lazy val stxLedger: StxLedger =
testModeComponentsProvider.stxLedger(blockchainConfig, SealEngineType.NoReward)
}
Original file line number Diff line number Diff line change
Expand Up @@ -397,13 +397,15 @@ class EthBlocksServiceSpec
class TestSetup(implicit system: ActorSystem) extends MockFactory with EphemBlockchainTestSetup {
val blockGenerator = mock[PoWBlockGenerator]
val appStateStorage = mock[AppStateStorage]

override lazy val consensus: TestConsensus = buildTestConsensus().withBlockGenerator(blockGenerator)
override lazy val consensusConfig = ConsensusConfigs.consensusConfig

lazy val ethBlocksService = new EthBlocksService(
blockchain,
blockchainReader,
consensus
consensus,
blockQueue
)

val blockToRequest = Block(Fixtures.Blocks.Block3125369.header, Fixtures.Blocks.Block3125369.body)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,6 @@ 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, InMemoryWorldStateProxy, 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}
Expand Down Expand Up @@ -45,6 +44,7 @@ class JsonRpcControllerFixture(implicit system: ActorSystem)
val blockGenerator = mock[PoWBlockGenerator]

val syncingController = TestProbe()

override lazy val stxLedger = mock[StxLedger]
override lazy val validators = mock[ValidatorsExecutor]
(() => validators.signedTransactionValidator)
Expand Down Expand Up @@ -101,7 +101,7 @@ class JsonRpcControllerFixture(implicit system: ActorSystem)
getTransactionFromPoolTimeout
)

val ethBlocksService = new EthBlocksService(blockchain, blockchainReader, consensus)
val ethBlocksService = new EthBlocksService(blockchain, blockchainReader, consensus, blockQueue)

val ethTxService = new EthTxService(
blockchain,
Expand Down