From 8b44f025c7644403e31400f0f76570e68f425f91 Mon Sep 17 00:00:00 2001 From: Sergey Nazarov Date: Thu, 13 May 2021 17:02:48 +0300 Subject: [PATCH] Fixed VRF value at current height (#3474) --- .../com/wavesplatform/events/events.scala | 6 +- .../events/BlockchainUpdateTriggersSpec.scala | 312 ------------------ .../wavesplatform/events/EventsHelpers.scala | 39 --- .../events/GenesisBlockUpdateSpec.scala | 35 -- .../wavesplatform/events/WithBlockchain.scala | 46 --- .../scala/com/wavesplatform/Importer.scala | 5 +- .../scala/com/wavesplatform/block/Block.scala | 6 +- .../state/BlockchainUpdaterImpl.scala | 40 ++- .../state/diffs/BlockDiffer.scala | 11 +- .../diffs/invoke/InvokeDiffsCommon.scala | 2 +- .../state/reader/CompositeBlockchain.scala | 74 +++-- .../com/wavesplatform/utx/UtxPoolImpl.scala | 2 +- .../wavesplatform/utx/UtxPriorityPool.scala | 2 +- .../com/wavesplatform/db/WithState.scala | 10 +- .../history/BlockRewardSpec.scala | 2 +- .../com/wavesplatform/history/Domain.scala | 2 +- .../state/BlockchainUpdaterImplSpec.scala | 226 +++++++------ .../state/CompositeBlockchainSpec.scala | 15 - .../diffs/BlockDifferDetailedDiffTest.scala | 2 +- .../state/diffs/CommonValidationTest.scala | 6 +- .../diffs/ExchangeTransactionDiffTest.scala | 4 +- .../diffs/ReissueTransactionDiffTest.scala | 2 +- .../state/diffs/ScriptsCountTest.scala | 2 +- 23 files changed, 230 insertions(+), 621 deletions(-) delete mode 100644 grpc-server/src/test/scala/com/wavesplatform/events/BlockchainUpdateTriggersSpec.scala delete mode 100644 grpc-server/src/test/scala/com/wavesplatform/events/EventsHelpers.scala delete mode 100644 grpc-server/src/test/scala/com/wavesplatform/events/GenesisBlockUpdateSpec.scala delete mode 100644 grpc-server/src/test/scala/com/wavesplatform/events/WithBlockchain.scala delete mode 100644 node/src/test/scala/com/wavesplatform/state/CompositeBlockchainSpec.scala diff --git a/grpc-server/src/main/scala/com/wavesplatform/events/events.scala b/grpc-server/src/main/scala/com/wavesplatform/events/events.scala index 30e5a041059..1ce868c4a3b 100644 --- a/grpc-server/src/main/scala/com/wavesplatform/events/events.scala +++ b/grpc-server/src/main/scala/com/wavesplatform/events/events.scala @@ -350,7 +350,7 @@ object StateUpdate { def atomic(blockchainBeforeWithMinerReward: Blockchain, diff: Diff): StateUpdate = { val blockchain = blockchainBeforeWithMinerReward - val blockchainAfter = CompositeBlockchain(blockchain, Some(diff)) + val blockchainAfter = CompositeBlockchain(blockchain, diff) val PortfolioUpdates(updatedBalances, updatedLeaseBalances) = DiffToStateApplier.portfolios(blockchain, diff) @@ -484,11 +484,11 @@ object StateUpdate { .foldLeft((Seq.empty[StateUpdate], parentDiff)) { case ((updates, accDiff), txDiff) => ( - updates :+ atomic(CompositeBlockchain(blockchainBeforeWithMinerReward, Some(accDiff)), txDiff), + updates :+ atomic(CompositeBlockchain(blockchainBeforeWithMinerReward, accDiff), txDiff), accDiff.combine(txDiff) ) } - val blockchainAfter = CompositeBlockchain(blockchainBeforeWithMinerReward, Some(totalDiff)) + val blockchainAfter = CompositeBlockchain(blockchainBeforeWithMinerReward, totalDiff) val metadata = transactionsMetadata(blockchainAfter, totalDiff) val refAssets = referencedAssets(blockchainAfter, txsStateUpdates) (parentStateUpdateWithMinerReward, txsStateUpdates, metadata, refAssets) diff --git a/grpc-server/src/test/scala/com/wavesplatform/events/BlockchainUpdateTriggersSpec.scala b/grpc-server/src/test/scala/com/wavesplatform/events/BlockchainUpdateTriggersSpec.scala deleted file mode 100644 index 85791bf7580..00000000000 --- a/grpc-server/src/test/scala/com/wavesplatform/events/BlockchainUpdateTriggersSpec.scala +++ /dev/null @@ -1,312 +0,0 @@ -//package com.wavesplatform.events -// -//import com.wavesplatform.account.KeyPair -//import com.wavesplatform.block.MicroBlock -//import com.wavesplatform.common.state.ByteStr -//import com.wavesplatform.common.utils.EitherExt2 -//import com.wavesplatform.features.EstimatorProvider -//import com.wavesplatform.history.Domain.BlockchainUpdaterExt -//import com.wavesplatform.lagonaki.mocks.TestBlock -//import com.wavesplatform.lang.script.Script -//import com.wavesplatform.protobuf.utils.PBImplicitConversions.PBByteStringOps -//import com.wavesplatform.settings.{Constants, WavesSettings} -//import com.wavesplatform.state.diffs.ENOUGH_AMT -//import com.wavesplatform.state.{Blockchain, Diff, NewTransactionInfo} -//import com.wavesplatform.transaction.Asset.Waves -//import com.wavesplatform.transaction.assets.IssueTransaction -//import com.wavesplatform.transaction.transfer.TransferTransaction -//import com.wavesplatform.transaction.{BlockchainUpdater, DataTransaction, GenesisTransaction, Transaction} -//import com.wavesplatform.{BlockGen, TestHelpers, crypto, state} -//import org.scalacheck.Gen -//import org.scalatest.{FreeSpec, Matchers} -//import org.scalatestplus.scalacheck.ScalaCheckPropertyChecks -// -//class BlockchainUpdateTriggersSpec extends FreeSpec with Matchers with BlockGen with ScalaCheckPropertyChecks with EventsHelpers { -// private val WAVES_AMOUNT = Constants.UnitsInWave * Constants.TotalWaves -// -// override protected def settings: WavesSettings = TestHelpers.enableNG(super.settings) -// -// // add a genesis block to the blockchain -// private val master: KeyPair = accountGen.sample.get -// private val rich: KeyPair = accountGen.sample.get -// private val initialAmount: Long = WAVES_AMOUNT / 2 -// private val genesis = TestBlock.create( -// 0, -// Seq( -// GenesisTransaction.create(master.toAddress, initialAmount, 0).explicitGet(), -// GenesisTransaction.create(rich.toAddress, initialAmount, 0).explicitGet() -// ), -// master -// ) -// override protected def initBlockchain(blockchainUpdater: Blockchain with BlockchainUpdater): Unit = { -// blockchainUpdater.processBlock(genesis).explicitGet() -// super.initBlockchain(blockchainUpdater) -// } -// -// private val sigGen: Gen[ByteStr] = bytes64gen.map(ByteStr.apply) -// private val heightGen: Gen[Int] = Gen.choose(1, 1000) -// private val assetAmtGen: Gen[Long] = Gen.oneOf(Gen.const[Long](1), Gen.choose[Long](2, ENOUGH_AMT)) -// -// private def microBlockGen(txs: Seq[Transaction], signer: KeyPair): Gen[MicroBlock] = -// for { -// sig <- byteArrayGen(crypto.SignatureLength).map(ByteStr.apply) -// mb = MicroBlock.buildAndSign(3.toByte, signer, txs, genesis.signature, sig).explicitGet() -// } yield mb -// -// /** -// * Tests the assertion both for transactions added in a block and in a microblock -// */ -// private def testTxsStateUpdates[A](txs: Seq[Transaction])(assertion: Seq[StateUpdate] => A): Unit = { -// val b = blockGen(txs, master).sample.get -// assertion(appendBlock(b).transactionStateUpdates) -// -// val mb = microBlockGen(txs, master).sample.get -// assertion(appendMicroBlock(mb).transactionStateUpdates) -// } -// -// private def isNFT(tx: IssueTransaction): Boolean = tx.quantity == 1 && tx.decimals == 0 && !tx.reissuable -// -// "updated Waves amount is calculated correctly for miner reward" in forAll { -// for { -// b <- blockGen(Seq.empty, master) -// reward <- Gen.option(Gen.choose(1L, 1000000L)) -// ba = appendBlock(b, reward) -// } yield (reward, ba) -// } { -// case (reward, BlockAppended(_, _, _, updatedWavesAmount, _, _)) => -// updatedWavesAmount shouldBe WAVES_AMOUNT + reward.getOrElse(0L) -// } -// -// "rollbacks correctly" - { -// "block" in forAll(sigGen, heightGen) { (sig, height) => -// produceEvent(_.onRollback(sig, height)) match { -// case RollbackCompleted(toId, toHeight) => -// toId shouldBe sig -// toHeight shouldBe height -// case _ => fail() -// } -// } -// -// "microblock" in forAll(sigGen, heightGen) { (sig, height) => -// produceEvent(_.onMicroBlockRollback(sig, height)) match { -// case MicroBlockRollbackCompleted(toId, toHeight) => -// toId shouldBe sig -// toHeight shouldBe height -// case _ => fail() -// } -// } -// } -// -// "appends correctly" - { -// "empty block" in forAll { -// for { -// b <- blockGen(Seq.empty, master) -// ba = appendBlock(b) -// } yield (b, ba) -// } { -// case (b, BlockAppended(toId, toHeight, block, _, _, _)) => -// toId shouldBe b.signature -// toHeight shouldBe blockchain.height + 1 -// -// block.signature shouldBe b.signature -// block.transactionData shouldBe b.transactionData -// } -// -// "microblock with one transaction" in forAll { -// for { -// tx <- dataTransactionGen(0, sender = Some(rich)) -// mb <- microBlockGen(Seq(tx), master) -// mba = appendMicroBlock(mb) -// } yield (mb, mba) -// } { -// case (mb, MicroBlockAppended(toId, toHeight, microBlock, _, _)) => -// toId shouldBe mb.totalResBlockSig -// toHeight shouldBe blockchain.height -// -// microBlock.signature shouldBe mb.signature -// microBlock.transactionData shouldBe mb.transactionData -// } -// -// "including correct miner rewards for" - { -// "block" in forAll { -// for { -// miner <- accountGen -// tx <- dataTransactionGen(0, sender = Some(rich)) -// mb <- blockGen(Seq(tx), miner) -// ba = appendBlock(mb) -// } yield (tx, miner, ba.blockStateUpdate) -// } { case (tx, miner, su) => su.balances.find(_._1 == miner.publicKey.toAddress).get._3 shouldBe 0.4 * tx.fee } -// -// "microblock, giving reward to a key block miner" in forAll { -// for { -// tx <- dataTransactionGen(0, sender = Some(rich)) -// mb <- microBlockGen(Seq(tx), master) -// mba = appendMicroBlock(mb) -// } yield (tx, mba.microBlockStateUpdate) -// } { -// case (tx, su) => -// su.balances.find(_._1 == master.publicKey.toAddress).get._3 shouldBe (0.4 * tx.fee + initialAmount) -// } -// } -// -// "block/microblock with balance updates from transfer txs" in forAll { -// for { -// sender <- accountGen -// recipient <- accountGen -// amount <- Gen.choose(1L, initialAmount - Constants.UnitsInWave) -// master2sender <- transferGeneratorPV2(1, master, sender.toAddress, amount) -// fee <- Gen.choose(1, master2sender.amount - 1) -// sender2recipient = TransferTransaction.selfSigned(2.toByte, sender, recipient.toAddress, Waves, master2sender.amount - fee, Waves, fee, ByteStr.empty, 2) -// .explicitGet() -// } yield (sender, recipient, master2sender, sender2recipient) -// } { -// case (sender, recipient, master2sender, sender2recipient) => -// testTxsStateUpdates(Seq(master2sender, sender2recipient)) { transactionStateUpdates => -// transactionStateUpdates.last.balances.find(_._1 == sender.publicKey.toAddress).get._3 shouldBe 0 -// -// transactionStateUpdates.last.balances.find(_._1 == recipient.publicKey.toAddress).get._3 shouldBe -// sender2recipient.amount -// } -// } -// -// "block/microblock with a data transaction" in forAll(dataTransactionGen(DataTransaction.MaxEntryCount, sender = Some(rich))) { tx => -// testTxsStateUpdates(Seq(tx)) { -// _.head.dataEntries.map(_._2).sortBy(_.key) shouldBe tx.data.sortBy(_.key) -// } -// } -// -// "blocks/microblocks with correct asset state updates by" - { -// "issue transaction" in forAll(issueV2TransactionGen(Gen.const(master))) { tx => -// testTxsStateUpdates(Seq(tx)) { upds => -// val AssetStateUpdate(asset, decimals, name, description, reissuable, volume, script, sponsorship, nft, assetExistedBefore) = -// upds.head.assets.head -// asset.id shouldBe tx.id() -// name shouldBe tx.name.byteStr -// description shouldBe tx.description.byteStr -// decimals shouldBe tx.decimals -// reissuable shouldBe tx.reissuable -// volume.toLong shouldBe tx.quantity -// script.map(_.script) shouldBe tx.script -// nft shouldBe isNFT(tx) -// sponsorship shouldBe None -// assetExistedBefore shouldBe false -// } -// } -// -// "reissue transaction" in forAll { -// for { -// issueAmt <- assetAmtGen -// reissueAmt <- assetAmtGen -// (issue, reissue, _) <- issueReissueBurnGeneratorP(issueAmt, reissueAmt, 1, master).suchThat(_._1.reissuable) -// } yield (issue, reissue) -// } { -// case (issue, reissue) => -// testTxsStateUpdates(Seq(issue, reissue)) { upds => -// val issueUpd = upds.head.assets.head -// val reissueUpd = upds.last.assets.head -// -// reissueUpd shouldBe issueUpd.copy( -// volume = BigInt(issue.quantity) + BigInt(reissue.quantity), -// reissuable = reissue.reissuable, -// assetExistedBefore = !issueUpd.assetExistedBefore -// ) -// } -// } -// -// "burn transaction" in forAll { -// for { -// issueAmt <- assetAmtGen -// burnAmt <- Gen.choose(1, issueAmt) -// (issue, _, burn) <- issueReissueBurnGeneratorP(issueAmt, 1, burnAmt, master) -// } yield (issue, burn) -// } { -// case (issue, burn) => -// testTxsStateUpdates(Seq(issue, burn)) { upds => -// val issueUpd = upds.head.assets.head -// val burnUpd = upds.last.assets.head -// -// burnUpd shouldBe issueUpd.copy( -// volume = BigInt(issue.quantity) - BigInt(burn.quantity), -// assetExistedBefore = !issueUpd.assetExistedBefore -// ) -// } -// } -// -// "set asset script transaction" in forAll(issueAndSetAssetScriptGen(master)) { -// case (issue, setAssetScript) => -// testTxsStateUpdates(Seq(issue, setAssetScript)) { upds => -// val issueUpd = upds.head.assets.head -// val scriptUpd = upds.last.assets.head -// -// scriptUpd shouldBe issueUpd.copy( -// script = setAssetScript.script.map( -// s => -// state.AssetScriptInfo( -// s, -// Script.estimate(s, EstimatorProvider.EstimatorBlockchainExt(blockchain).estimator, useContractVerifierLimit = false).explicitGet() -// ) -// ), -// assetExistedBefore = !issueUpd.assetExistedBefore -// ) -// } -// } -// -// "sponsor fee transaction " in forAll(sponsorFeeCancelSponsorFeeGen(master)) { -// case (issue, startSponsorship, _, cancelSponsorship) => -// testTxsStateUpdates(Seq(issue, startSponsorship, cancelSponsorship)) { upds => -// val issueUpd = upds.head.assets.head -// val startSponsorshipUpd = upds(1).assets.head -// val cancelSponsorshipUpd = upds(2).assets.head -// -// startSponsorshipUpd shouldBe issueUpd.copy( -// sponsorship = startSponsorship.minSponsoredAssetFee, -// assetExistedBefore = !issueUpd.assetExistedBefore -// ) -// -// cancelSponsorshipUpd shouldBe startSponsorshipUpd.copy( -// sponsorship = cancelSponsorship.minSponsoredAssetFee -// ) -// } -// } -// -// "invokeScript transaction (diff emulated by issue, reussue and burn txs)" in forAll { -// for { -// issueAmt <- assetAmtGen -// reissueAmt <- assetAmtGen -// (issue, reissue, _) <- issueReissueBurnGeneratorP(issueAmt, reissueAmt, 1, master).suchThat(_._1.reissuable) -// invoke <- invokeScriptGen(Gen.const(Seq.empty)) -// } yield (issue, reissue, invoke) -// } { -// case (issue, reissue, invoke) => -// // create a block with issue and reissue txs, getting their diffs -// val assetsDummyBlock = TestBlock.create(master, Seq(issue, reissue)) -// val assetsDummyBlockDiff = detailedDiffFromBlock(assetsDummyBlock) -// -// val invokeBlock = TestBlock.create(master, Seq(invoke)) -// // merge issue/reissue diffs as if they were produced by a single invoke -// val invokeTxDiff = assetsDummyBlockDiff.transactionDiffs -// .foldLeft(Diff.empty)(Diff.diffMonoid.combine) -// .copy(transactions = Map(invoke.id() -> NewTransactionInfo(invoke, Set(master.toAddress), true))) -// val invokeBlockDetailedDiff = assetsDummyBlockDiff.copy(transactionDiffs = Seq(invokeTxDiff)) -// -// produceEvent(_.onProcessBlock(invokeBlock, invokeBlockDetailedDiff, None, blockchain)) match { -// case ba: BlockAppended => -// val AssetStateUpdate(asset, decimals, name, description, reissuable, volume, script, sponsorship, nft, assetExistedBefore) = -// ba.transactionStateUpdates.head.assets.head -// -// asset.id shouldBe issue.assetId -// decimals shouldBe issue.decimals -// name shouldBe issue.name.byteStr -// description shouldBe issue.description.byteStr -// reissuable shouldBe reissue.reissuable -// volume shouldBe (BigInt(issue.quantity) + BigInt(reissue.quantity)) -// script shouldBe issue.script -// sponsorship shouldBe None -// nft shouldBe isNFT(issue) -// assetExistedBefore shouldBe false -// case _ => fail() -// } -// } -// } -// } -//} diff --git a/grpc-server/src/test/scala/com/wavesplatform/events/EventsHelpers.scala b/grpc-server/src/test/scala/com/wavesplatform/events/EventsHelpers.scala deleted file mode 100644 index ff241d297b2..00000000000 --- a/grpc-server/src/test/scala/com/wavesplatform/events/EventsHelpers.scala +++ /dev/null @@ -1,39 +0,0 @@ -//package com.wavesplatform.events -// -//import com.wavesplatform.block.{Block, MicroBlock} -//import com.wavesplatform.common.utils.EitherExt2 -//import com.wavesplatform.mining.MiningConstraint -//import com.wavesplatform.state.diffs.BlockDiffer -//import com.wavesplatform.state.diffs.BlockDiffer.DetailedDiff -//import monix.execution.Scheduler.Implicits.global -//import monix.reactive.subjects.ReplaySubject -//import org.scalatest.Suite -// -//import scala.concurrent.duration._ -// -//private[events] trait EventsHelpers extends WithBlockchain { _: Suite => -// protected def produceEvent(useTrigger: BlockchainUpdateTriggers => Unit): BlockchainUpdated = { -// val evts = ReplaySubject[BlockchainUpdated]() -// val t = new BlockchainUpdateTriggersImpl(evts) -// useTrigger(t) -// evts.onComplete() -// evts.toListL.runSyncUnsafe(500.milliseconds).head -// } -// -// protected def detailedDiffFromBlock(b: Block): DetailedDiff = -// BlockDiffer.fromBlock(blockchain, None, b, MiningConstraint.Unlimited, verify = false).explicitGet().detailedDiff -// -// protected def appendBlock(b: Block, minerReward: Option[Long] = None): BlockAppended = -// produceEvent(_.onProcessBlock(b, detailedDiffFromBlock(b), minerReward, blockchain)) match { -// case ba: BlockAppended => ba -// case _ => fail() -// } -// -// protected def appendMicroBlock(mb: MicroBlock): MicroBlockAppended = { -// val dd = BlockDiffer.fromMicroBlock(blockchain, Some(0), mb, 1, MiningConstraint.Unlimited, verify = false).explicitGet().detailedDiff -// produceEvent(_.onProcessMicroBlock(mb, dd, blockchain, mb.totalResBlockSig)) match { -// case mba: MicroBlockAppended => mba -// case _ => fail() -// } -// } -//} diff --git a/grpc-server/src/test/scala/com/wavesplatform/events/GenesisBlockUpdateSpec.scala b/grpc-server/src/test/scala/com/wavesplatform/events/GenesisBlockUpdateSpec.scala deleted file mode 100644 index 6301b70a6fb..00000000000 --- a/grpc-server/src/test/scala/com/wavesplatform/events/GenesisBlockUpdateSpec.scala +++ /dev/null @@ -1,35 +0,0 @@ -//package com.wavesplatform.events -// -//import com.wavesplatform.common.utils.EitherExt2 -//import com.wavesplatform.settings.WavesSettings -//import com.wavesplatform.state.diffs.ENOUGH_AMT -//import com.wavesplatform.transaction.GenesisTransaction -//import com.wavesplatform.{BlockGen, TestHelpers} -//import org.scalacheck.Gen -//import org.scalatest.{FreeSpec, Matchers} -//import org.scalatestplus.scalacheck.ScalaCheckPropertyChecks -// -//class GenesisBlockUpdateSpec extends FreeSpec with Matchers with BlockGen with ScalaCheckPropertyChecks with EventsHelpers { -// override protected def settings: WavesSettings = TestHelpers.enableNG(super.settings) -// -// val genesisAppendWithWavesAmountGen: Gen[(BlockAppended, Long)] = for { -// master <- accountGen -// wavesAmount <- Gen.choose(1L, ENOUGH_AMT) -// gt = GenesisTransaction.create(master.toAddress, wavesAmount, 0).explicitGet() -// b <- blockGen(Seq(gt), master) -// ba = appendBlock(b) -// } yield (ba, wavesAmount) -// -// "on genesis block append" - { -// "master address balance gets correctly updated" in forAll(genesisAppendWithWavesAmountGen) { -// case (BlockAppended(_, _, _, _, _, upds), wavesAmount) => -// upds.head.balances.head._3 shouldBe wavesAmount -// } -// -// "updated Waves amount is calculated correctly" in forAll(genesisAppendWithWavesAmountGen) { -// case (BlockAppended(_, _, _, updatedWavesAmount, _, _), wavesAmount) => -// updatedWavesAmount shouldBe wavesAmount -// } -// } -// -//} diff --git a/grpc-server/src/test/scala/com/wavesplatform/events/WithBlockchain.scala b/grpc-server/src/test/scala/com/wavesplatform/events/WithBlockchain.scala deleted file mode 100644 index 13e454c3373..00000000000 --- a/grpc-server/src/test/scala/com/wavesplatform/events/WithBlockchain.scala +++ /dev/null @@ -1,46 +0,0 @@ -//package com.wavesplatform.events -// -//import java.nio.file.Files -// -//import com.wavesplatform.database.TestStorageFactory -//import com.wavesplatform.settings.{WavesSettings, loadConfig} -//import com.wavesplatform.state.Blockchain -//import com.wavesplatform.transaction.BlockchainUpdater -//import com.wavesplatform.{NTPTime, TestHelpers, database} -//import monix.reactive.Observer -//import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, Suite} -// -//trait WithBlockchain extends BeforeAndAfterEach with BeforeAndAfterAll with NTPTime { _: Suite => -// protected def settings: WavesSettings = WavesSettings.fromRootConfig(loadConfig(None)) -// -// private val path = Files.createTempDirectory("leveldb-test") -// private val db = database.openDB(path.toAbsolutePath.toString) -// private val (bcu, _) = TestStorageFactory( -// settings, -// db, -// ntpTime, -// Observer.stopped, -// BlockchainUpdateTriggers.noop -// ) -// -// protected def blockchain: Blockchain = bcu -// -// /** -// * Override this method to do some initialization actions with -// * the blockchain before it becomes read-only -// * @param blockchainUpdater a blockchain to add something to (genesis, some blocks, etc.) -// */ -// protected def initBlockchain(blockchainUpdater: Blockchain with BlockchainUpdater): Unit = () -// -// override protected def beforeAll(): Unit = { -// initBlockchain(bcu) -// super.beforeAll() -// } -// -// override def afterAll(): Unit = { -// bcu.shutdown() -// db.close() -// TestHelpers.deleteRecursively(path) -// super.afterAll() -// } -//} diff --git a/node/src/main/scala/com/wavesplatform/Importer.scala b/node/src/main/scala/com/wavesplatform/Importer.scala index 32e5a1b5402..7beee1226e3 100644 --- a/node/src/main/scala/com/wavesplatform/Importer.scala +++ b/node/src/main/scala/com/wavesplatform/Importer.scala @@ -98,10 +98,7 @@ object Importer extends ScorexLogging { } } - def loadSettings(file: Option[File]): WavesSettings = { - val settings = Application.loadApplicationConfig(file) - settings.copy(dbSettings = settings.dbSettings.copy(useBloomFilter = true)) - } + def loadSettings(file: Option[File]): WavesSettings = Application.loadApplicationConfig(file) private[this] var triggers = Seq.empty[BlockchainUpdateTriggers] diff --git a/node/src/main/scala/com/wavesplatform/block/Block.scala b/node/src/main/scala/com/wavesplatform/block/Block.scala index 7e3e38c4222..c35643f7ee6 100644 --- a/node/src/main/scala/com/wavesplatform/block/Block.scala +++ b/node/src/main/scala/com/wavesplatform/block/Block.scala @@ -29,7 +29,9 @@ case class BlockHeader( featureVotes: Seq[Short], rewardVote: Long, transactionsRoot: ByteStr -) +) { + val score: Coeval[BigInt] = Coeval.evalOnce((BigInt("18446744073709551616") / baseTarget).ensuring(_ > 0)) +} case class Block( header: BlockHeader, @@ -45,7 +47,7 @@ case class Block( val bytes: Coeval[Array[Byte]] = Coeval.evalOnce(BlockSerializer.toBytes(this)) val json: Coeval[JsObject] = Coeval.evalOnce(BlockSerializer.toJson(this)) - val blockScore: Coeval[BigInt] = Coeval.evalOnce((BigInt("18446744073709551616") / header.baseTarget).ensuring(_ > 0)) + val blockScore: Coeval[BigInt] = header.score val bodyBytes: Coeval[Array[Byte]] = Coeval.evalOnce { if (header.version < Block.ProtoBlockVersion) copy(signature = ByteStr.empty).bytes() diff --git a/node/src/main/scala/com/wavesplatform/state/BlockchainUpdaterImpl.scala b/node/src/main/scala/com/wavesplatform/state/BlockchainUpdaterImpl.scala index 99c17c9f68c..964f7ec69f6 100644 --- a/node/src/main/scala/com/wavesplatform/state/BlockchainUpdaterImpl.scala +++ b/node/src/main/scala/com/wavesplatform/state/BlockchainUpdaterImpl.scala @@ -6,8 +6,8 @@ import cats.implicits._ import cats.kernel.Monoid import com.wavesplatform.account.{Address, Alias} import com.wavesplatform.api.BlockMeta -import com.wavesplatform.block.{Block, MicroBlock, SignedBlockHeader} import com.wavesplatform.block.Block.BlockId +import com.wavesplatform.block.{Block, MicroBlock, SignedBlockHeader} import com.wavesplatform.common.state.ByteStr import com.wavesplatform.database.Storage import com.wavesplatform.events.BlockchainUpdateTriggers @@ -18,15 +18,15 @@ import com.wavesplatform.mining.{Miner, MiningConstraint, MiningConstraints} import com.wavesplatform.settings.{BlockchainSettings, WavesSettings} import com.wavesplatform.state.diffs.BlockDiffer import com.wavesplatform.state.reader.{CompositeBlockchain, LeaseDetails} -import com.wavesplatform.transaction._ import com.wavesplatform.transaction.Asset.{IssuedAsset, Waves} import com.wavesplatform.transaction.TxValidationError.{BlockAppendError, GenericError, MicroBlockAppendError} +import com.wavesplatform.transaction._ import com.wavesplatform.transaction.lease._ import com.wavesplatform.transaction.transfer.TransferTransaction -import com.wavesplatform.utils.{forceStopApplication, ScorexLogging, Time, UnsupportedFeature} +import com.wavesplatform.utils.{ScorexLogging, Time, UnsupportedFeature, forceStopApplication} import kamon.Kamon -import monix.reactive.{Observable, Observer} import monix.reactive.subjects.ReplaySubject +import monix.reactive.{Observable, Observer} class BlockchainUpdaterImpl( leveldb: Blockchain with Storage, @@ -206,17 +206,18 @@ class BlockchainUpdaterImpl( val miningConstraints = MiningConstraints(leveldb, height) val reward = nextReward() - val referencedBlockchain = CompositeBlockchain(leveldb, carry = leveldb.carryFee, reward = reward) + val referencedBlockchain = CompositeBlockchain(leveldb, reward) BlockDiffer .fromBlock( referencedBlockchain, leveldb.lastBlock, block, miningConstraints.total, + hitSource, verify ) .map { r => - val updatedBlockchain = CompositeBlockchain(leveldb, Some(r.diff), Some(block), r.carry, reward, Some(hitSource)) + val updatedBlockchain = CompositeBlockchain(leveldb, r.diff, block, hitSource, r.carry, reward) miner.scheduleMining(Some(updatedBlockchain)) blockchainUpdateTriggers.onProcessBlock(block, r.detailedDiff, reward, referencedBlockchain) Option((r, Nil, reward, hitSource)) @@ -230,13 +231,14 @@ class BlockchainUpdaterImpl( blockchainUpdateTriggers.onRollback(this, ng.base.header.reference, leveldb.height) - val referencedBlockchain = CompositeBlockchain(leveldb, carry = leveldb.carryFee, reward = ng.reward) + val referencedBlockchain = CompositeBlockchain(leveldb, ng.reward) BlockDiffer .fromBlock( referencedBlockchain, leveldb.lastBlock, block, miningConstraints.total, + hitSource, verify ) .map { r => @@ -259,13 +261,14 @@ class BlockchainUpdaterImpl( blockchainUpdateTriggers.onRollback(this, ng.base.header.reference, leveldb.height) - val referencedBlockchain = CompositeBlockchain(leveldb, carry = leveldb.carryFee, reward = ng.reward) + val referencedBlockchain = CompositeBlockchain(leveldb, ng.reward) BlockDiffer .fromBlock( referencedBlockchain, leveldb.lastBlock, block, miningConstraints.total, + hitSource, verify ) .map { r => @@ -307,13 +310,21 @@ class BlockchainUpdaterImpl( val liquidDiffWithCancelledLeases = ng.cancelExpiredLeases(referencedLiquidDiff) val referencedBlockchain = - CompositeBlockchain(leveldb, Some(liquidDiffWithCancelledLeases), Some(referencedForgedBlock), carry, reward) + CompositeBlockchain( + leveldb, + liquidDiffWithCancelledLeases, + referencedForgedBlock, + ng.hitSource, + carry, + reward + ) val maybeDiff = BlockDiffer .fromBlock( referencedBlockchain, Some(referencedForgedBlock), block, constraint, + hitSource, verify ) @@ -321,11 +332,11 @@ class BlockchainUpdaterImpl( differResult => val tempBlockchain = CompositeBlockchain( referencedBlockchain, - Some(differResult.diff), - Some(block), + differResult.diff, + block, + hitSource, differResult.carry, - reward, - Some(hitSource) + reward ) miner.scheduleMining(Some(tempBlockchain)) @@ -663,7 +674,8 @@ class BlockchainUpdaterImpl( } override def balanceSnapshots(address: Address, from: Int, to: Option[BlockId]): Seq[BalanceSnapshot] = readLock { - CompositeBlockchain(leveldb, to.fold(ngState.map(_.bestLiquidDiff))(id => ngState.map(_.diffFor(id)._1))) + to.fold(ngState.map(_.bestLiquidDiff))(id => ngState.map(_.diffFor(id)._1)) + .fold[Blockchain](leveldb)(CompositeBlockchain(leveldb, _)) .balanceSnapshots(address, from, to) } diff --git a/node/src/main/scala/com/wavesplatform/state/diffs/BlockDiffer.scala b/node/src/main/scala/com/wavesplatform/state/diffs/BlockDiffer.scala index 28cf0088d1f..117e7ad6c31 100644 --- a/node/src/main/scala/com/wavesplatform/state/diffs/BlockDiffer.scala +++ b/node/src/main/scala/com/wavesplatform/state/diffs/BlockDiffer.scala @@ -4,6 +4,7 @@ import cats.implicits._ import cats.kernel.Monoid import cats.syntax.either.catsSyntaxEitherId import com.wavesplatform.block.{Block, MicroBlock} +import com.wavesplatform.common.state.ByteStr import com.wavesplatform.features.BlockchainFeatures import com.wavesplatform.lang.ValidationError import com.wavesplatform.mining.MiningConstraint @@ -31,16 +32,18 @@ object BlockDiffer extends ScorexLogging { maybePrevBlock: Option[Block], block: Block, constraint: MiningConstraint, + hitSource: ByteStr, verify: Boolean = true ): Either[ValidationError, Result] = - fromBlockTraced(blockchain, maybePrevBlock, block, constraint, verify).resultE + fromBlockTraced(blockchain, maybePrevBlock, block, constraint, hitSource, verify).resultE def fromBlockTraced( blockchain: Blockchain, maybePrevBlock: Option[Block], block: Block, constraint: MiningConstraint, - verify: Boolean = true + hitSource: ByteStr, + verify: Boolean ): TracedResult[ValidationError, Result] = { val stateHeight = blockchain.height @@ -73,7 +76,7 @@ object BlockDiffer extends ScorexLogging { for { _ <- TracedResult(Either.cond(!verify || block.signatureValid(), (), GenericError(s"Block $block has invalid signature"))) r <- apply( - CompositeBlockchain(blockchain, newBlock = Some(block)), + CompositeBlockchain(blockchain, Diff.empty, block, hitSource, 0, None), constraint, maybePrevBlock.map(_.header.timestamp), Diff.empty.copy(portfolios = Map(block.sender.toAddress -> (minerReward |+| initialFeeFromThisBlock |+| feeFromPreviousBlock))), @@ -154,7 +157,7 @@ object BlockDiffer extends ScorexLogging { .foldLeft(TracedResult(Result(initDiff, 0L, 0L, initConstraint, DetailedDiff(initDiff, Nil)).asRight[ValidationError])) { case (acc @ TracedResult(Left(_), _), _) => acc case (TracedResult(Right(Result(currDiff, carryFee, currTotalFee, currConstraint, DetailedDiff(parentDiff, txDiffs))), _), tx) => - val currBlockchain = CompositeBlockchain(blockchain, Some(currDiff)) + val currBlockchain = CompositeBlockchain(blockchain, currDiff) txDiffer(currBlockchain, tx).flatMap { thisTxDiff => val updatedConstraint = updateConstraint(currConstraint, currBlockchain, tx, thisTxDiff) if (updatedConstraint.isOverfilled) diff --git a/node/src/main/scala/com/wavesplatform/state/diffs/invoke/InvokeDiffsCommon.scala b/node/src/main/scala/com/wavesplatform/state/diffs/invoke/InvokeDiffsCommon.scala index 17f068f863b..54ae5a00c47 100644 --- a/node/src/main/scala/com/wavesplatform/state/diffs/invoke/InvokeDiffsCommon.scala +++ b/node/src/main/scala/com/wavesplatform/state/diffs/invoke/InvokeDiffsCommon.scala @@ -303,7 +303,7 @@ object InvokeDiffsCommon { if (remainingLimit < Int.MaxValue) remainingLimit - curDiff.scriptsComplexity.toInt else remainingLimit - val blockchain = CompositeBlockchain(sblockchain, Some(curDiff)) + val blockchain = CompositeBlockchain(sblockchain, curDiff) val actionSender = Recipient.Address(ByteStr(tx.dAppAddressOrAlias.bytes)) def applyTransfer(transfer: AssetTransfer, pk: PublicKey): TracedResult[FailedTransactionError, Diff] = { diff --git a/node/src/main/scala/com/wavesplatform/state/reader/CompositeBlockchain.scala b/node/src/main/scala/com/wavesplatform/state/reader/CompositeBlockchain.scala index 34171f5a203..50a1997108a 100644 --- a/node/src/main/scala/com/wavesplatform/state/reader/CompositeBlockchain.scala +++ b/node/src/main/scala/com/wavesplatform/state/reader/CompositeBlockchain.scala @@ -17,13 +17,12 @@ import com.wavesplatform.transaction.lease.LeaseTransaction import com.wavesplatform.transaction.transfer.TransferTransaction import com.wavesplatform.transaction.{Asset, Transaction} -final case class CompositeBlockchain( +final class CompositeBlockchain private ( inner: Blockchain, maybeDiff: Option[Diff] = None, - newBlock: Option[Block] = None, + blockMeta: Option[(SignedBlockHeader, ByteStr)] = None, carry: Long = 0, - reward: Option[Long] = None, - hitSource: Option[ByteStr] = None + reward: Option[Long] = None ) extends Blockchain { override val settings: BlockchainSettings = inner.settings @@ -32,9 +31,8 @@ final case class CompositeBlockchain( override def balance(address: Address, assetId: Asset): Long = inner.balance(address, assetId) + diff.portfolios.getOrElse(address, Portfolio.empty).balanceOf(assetId) - override def leaseBalance(address: Address): LeaseBalance = { + override def leaseBalance(address: Address): LeaseBalance = cats.Monoid.combine(inner.leaseBalance(address), diff.portfolios.getOrElse(address, Portfolio.empty).lease) - } override def assetScript(asset: IssuedAsset): Option[AssetScriptInfo] = maybeDiff @@ -52,14 +50,13 @@ final case class CompositeBlockchain( } } - override def transferById(id: ByteStr): Option[(Int, TransferTransaction)] = { + override def transferById(id: ByteStr): Option[(Int, TransferTransaction)] = diff.transactions .get(id) .collect { case NewTransactionInfo(tx: TransferTransaction, _, true) => (height, tx) } .orElse(inner.transferById(id)) - } override def transactionInfo(id: ByteStr): Option[(Int, Transaction, Boolean)] = diff.transactions @@ -73,7 +70,7 @@ final case class CompositeBlockchain( .map(info => (this.height, info.applied)) .orElse(inner.transactionMeta(id)) - override def height: Int = inner.height + newBlock.fold(0)(_ => 1) + override def height: Int = inner.height + blockMeta.fold(0)(_ => 1) override def resolveAlias(alias: Alias): Either[ValidationError, Address] = inner.resolveAlias(alias) match { case l @ Left(AliasIsDisabled(_)) => l @@ -86,7 +83,7 @@ final case class CompositeBlockchain( override def filledVolumeAndFee(orderId: ByteStr): VolumeAndFee = diff.orderFills.get(orderId).orEmpty.combine(inner.filledVolumeAndFee(orderId)) - override def balanceAtHeight(address: Address, h: Int, assetId: Asset = Waves): Option[(Int, Long)] = { + override def balanceAtHeight(address: Address, h: Int, assetId: Asset = Waves): Option[(Int, Long)] = if (maybeDiff.isEmpty || h < this.height) { inner.balanceAtHeight(address, h, assetId) } else { @@ -94,9 +91,8 @@ final case class CompositeBlockchain( val bs = height -> balance Some(bs) } - } - override def balanceSnapshots(address: Address, from: Int, to: Option[BlockId]): Seq[BalanceSnapshot] = { + override def balanceSnapshots(address: Address, from: Int, to: Option[BlockId]): Seq[BalanceSnapshot] = if (maybeDiff.isEmpty || to.exists(id => inner.heightOf(id).isDefined)) { inner.balanceSnapshots(address, from, to) } else { @@ -105,40 +101,35 @@ final case class CompositeBlockchain( val bs = BalanceSnapshot(height, Portfolio(balance, lease, Map.empty)) if (inner.height > 0 && from < this.height) bs +: inner.balanceSnapshots(address, from, to) else Seq(bs) } - } - override def accountScript(address: Address): Option[AccountScriptInfo] = { + override def accountScript(address: Address): Option[AccountScriptInfo] = diff.scripts.get(address) match { case None => inner.accountScript(address) case Some(None) => None case Some(Some(scr)) => Some(scr) } - } - override def hasAccountScript(address: Address): Boolean = { + override def hasAccountScript(address: Address): Boolean = diff.scripts.get(address) match { case None => inner.hasAccountScript(address) case Some(None) => false case Some(Some(_)) => true } - } - override def accountData(acc: Address, key: String): Option[DataEntry[_]] = { - val diffData = diff.accountData.get(acc).orEmpty - diffData.data.get(key).orElse(inner.accountData(acc, key)).filterNot(_.isEmpty) - } + override def accountData(acc: Address, key: String): Option[DataEntry[_]] = + diff.accountData.get(acc).orEmpty.data.get(key).orElse(inner.accountData(acc, key)).filterNot(_.isEmpty) override def carryFee: Long = carry - override def score: BigInt = newBlock.fold(BigInt(0))(_.blockScore()) + inner.score + override def score: BigInt = blockMeta.fold(BigInt(0))(_._1.header.score()) + inner.score override def blockHeader(height: Int): Option[SignedBlockHeader] = - newBlock match { - case Some(b) if this.height == height => Some(SignedBlockHeader(b.header, b.signature)) - case _ => inner.blockHeader(height) + blockMeta match { + case Some((header, _)) if this.height == height => Some(header) + case _ => inner.blockHeader(height) } - override def heightOf(blockId: ByteStr): Option[Int] = newBlock.filter(_.id() == blockId).map(_ => height) orElse inner.heightOf(blockId) + override def heightOf(blockId: ByteStr): Option[Int] = blockMeta.filter(_._1.id() == blockId).map(_ => height) orElse inner.heightOf(blockId) /** Features related */ override def approvedFeatures: Map[Short, Int] = inner.approvedFeatures @@ -154,12 +145,37 @@ final case class CompositeBlockchain( override def wavesAmount(height: Int): BigInt = inner.wavesAmount(height) + BigInt(reward.getOrElse(0L)) - override def hitSource(height: Int): Option[ByteStr] = hitSource.filter(_ => this.height == height) orElse inner.hitSource(height) + override def hitSource(height: Int): Option[ByteStr] = + blockMeta + .collect { case (_, hitSource) if this.height == height => hitSource } + .orElse(inner.hitSource(height)) } object CompositeBlockchain { - def apply(blockchain: Blockchain, ngState: NgState): CompositeBlockchain = - CompositeBlockchain(blockchain, Some(ngState.bestLiquidDiff), Some(ngState.bestLiquidBlock), ngState.carryFee, ngState.reward) + def apply(inner: Blockchain, ngState: NgState): CompositeBlockchain = + new CompositeBlockchain( + inner, + Some(ngState.bestLiquidDiff), + Some(SignedBlockHeader(ngState.bestLiquidBlock.header, ngState.bestLiquidBlock.signature) -> ngState.hitSource), + ngState.carryFee, + ngState.reward + ) + + def apply(inner: Blockchain, reward: Option[Long]): CompositeBlockchain = + new CompositeBlockchain(inner, carry = inner.carryFee, reward = reward) + + def apply(inner: Blockchain, diff: Diff): CompositeBlockchain = + new CompositeBlockchain(inner, Some(diff)) + + def apply( + inner: Blockchain, + diff: Diff, + newBlock: Block, + hitSource: ByteStr, + carry: Long, + reward: Option[Long] + ): CompositeBlockchain = + new CompositeBlockchain(inner, Some(diff), Some(SignedBlockHeader(newBlock.header, newBlock.signature) -> hitSource), carry, reward) private def assetDescription( asset: IssuedAsset, diff --git a/node/src/main/scala/com/wavesplatform/utx/UtxPoolImpl.scala b/node/src/main/scala/com/wavesplatform/utx/UtxPoolImpl.scala index 9838836b6df..04fdf74d776 100644 --- a/node/src/main/scala/com/wavesplatform/utx/UtxPoolImpl.scala +++ b/node/src/main/scala/com/wavesplatform/utx/UtxPoolImpl.scala @@ -341,7 +341,7 @@ class UtxPoolImpl( val newScriptedAddresses = scriptedAddresses(tx) if (!priority && r.checkedAddresses.intersect(newScriptedAddresses).nonEmpty) r else { - val updatedBlockchain = CompositeBlockchain(blockchain, Some(r.totalDiff)) + val updatedBlockchain = CompositeBlockchain(blockchain, r.totalDiff) val newCheckedAddresses = newScriptedAddresses ++ r.checkedAddresses differ(updatedBlockchain, tx).resultE match { case Right(newDiff) => diff --git a/node/src/main/scala/com/wavesplatform/utx/UtxPriorityPool.scala b/node/src/main/scala/com/wavesplatform/utx/UtxPriorityPool.scala index f067a2079ff..482fc1b9fde 100644 --- a/node/src/main/scala/com/wavesplatform/utx/UtxPriorityPool.scala +++ b/node/src/main/scala/com/wavesplatform/utx/UtxPriorityPool.scala @@ -27,7 +27,7 @@ final class UtxPriorityPool(base: Blockchain) extends ScorexLogging with Optimis def priorityTransactions: Seq[Transaction] = priorityDiffs.flatMap(_.diff.transactionsValues) def priorityTransactionIds: Seq[ByteStr] = priorityTransactions.map(_.id()) - def compositeBlockchain: CompositeBlockchain = CompositeBlockchain(base, Some(priorityDiffsCombined)) + def compositeBlockchain: CompositeBlockchain = CompositeBlockchain(base, priorityDiffsCombined) def lockedWrite[T](f: => T): T = this.writeLock(f) diff --git a/node/src/test/scala/com/wavesplatform/db/WithState.scala b/node/src/test/scala/com/wavesplatform/db/WithState.scala index dc2a116e959..2142346bd11 100644 --- a/node/src/test/scala/com/wavesplatform/db/WithState.scala +++ b/node/src/test/scala/com/wavesplatform/db/WithState.scala @@ -72,7 +72,7 @@ trait WithState extends DBCacheSettings with Matchers with NTPTime { _: Suite => def assertDiffEi(preconditions: Seq[Block], block: Block, state: LevelDBWriter)( assertion: Either[ValidationError, Diff] => Unit ): Unit = { - def differ(blockchain: Blockchain, b: Block) = BlockDiffer.fromBlock(blockchain, None, b, MiningConstraint.Unlimited) + def differ(blockchain: Blockchain, b: Block) = BlockDiffer.fromBlock(blockchain, None, b, MiningConstraint.Unlimited, b.header.generationSignature) preconditions.foreach { precondition => val BlockDiffer.Result(preconditionDiff, preconditionFees, totalFee, _, _) = differ(state, precondition).explicitGet() @@ -85,7 +85,7 @@ trait WithState extends DBCacheSettings with Matchers with NTPTime { _: Suite => def assertDiffEiTraced(preconditions: Seq[Block], block: Block, fs: FunctionalitySettings = TFS.Enabled)( assertion: TracedResult[ValidationError, Diff] => Unit ): Unit = withLevelDBWriter(fs) { state => - def differ(blockchain: Blockchain, b: Block) = BlockDiffer.fromBlockTraced(blockchain, None, b, MiningConstraint.Unlimited) + def differ(blockchain: Blockchain, b: Block) = BlockDiffer.fromBlockTraced(blockchain, None, b, MiningConstraint.Unlimited, b.header.generationSignature, verify = true) preconditions.foreach { precondition => val BlockDiffer.Result(preconditionDiff, preconditionFees, totalFee, _, _) = differ(state, precondition).resultE.explicitGet() @@ -99,7 +99,7 @@ trait WithState extends DBCacheSettings with Matchers with NTPTime { _: Suite => assertion: (Diff, Blockchain) => Unit ): Unit = withLevelDBWriter(fs) { state => def differ(blockchain: Blockchain, prevBlock: Option[Block], b: Block): Either[ValidationError, BlockDiffer.Result] = - BlockDiffer.fromBlock(blockchain, if (withNg) prevBlock else None, b, MiningConstraint.Unlimited) + BlockDiffer.fromBlock(blockchain, if (withNg) prevBlock else None, b, MiningConstraint.Unlimited, b.header.generationSignature) preconditions.foldLeft[Option[Block]](None) { (prevBlock, curBlock) => val BlockDiffer.Result(diff, fees, totalFee, _, _) = differ(state, prevBlock, curBlock).explicitGet() @@ -108,7 +108,7 @@ trait WithState extends DBCacheSettings with Matchers with NTPTime { _: Suite => } val BlockDiffer.Result(diff, fees, totalFee, _, _) = differ(state, preconditions.lastOption, block).explicitGet() - val cb = CompositeBlockchain(state, Some(diff)) + val cb = CompositeBlockchain(state, diff) assertion(diff, cb) state.append(diff, fees, totalFee, None, block.header.generationSignature, block) @@ -127,7 +127,7 @@ trait WithState extends DBCacheSettings with Matchers with NTPTime { _: Suite => def assertDiffAndState(fs: FunctionalitySettings)(test: (Seq[Transaction] => Either[ValidationError, Unit]) => Unit): Unit = withLevelDBWriter(fs) { state => - def differ(blockchain: Blockchain, b: Block) = BlockDiffer.fromBlock(blockchain, None, b, MiningConstraint.Unlimited) + def differ(blockchain: Blockchain, b: Block) = BlockDiffer.fromBlock(blockchain, None, b, MiningConstraint.Unlimited, b.header.generationSignature) test(txs => { val nextHeight = state.height + 1 diff --git a/node/src/test/scala/com/wavesplatform/history/BlockRewardSpec.scala b/node/src/test/scala/com/wavesplatform/history/BlockRewardSpec.scala index 48e05f7139b..61e873a77f5 100644 --- a/node/src/test/scala/com/wavesplatform/history/BlockRewardSpec.scala +++ b/node/src/test/scala/com/wavesplatform/history/BlockRewardSpec.scala @@ -256,7 +256,7 @@ class BlockRewardSpec } yield (miner1, miner2, Seq(genesisBlock, b2, b3, b4), b5, m5s) def differ(blockchain: Blockchain, prevBlock: Option[Block], b: Block): BlockDiffer.Result = - BlockDiffer.fromBlock(blockchain, prevBlock, b, MiningConstraint.Unlimited: MiningConstraint).explicitGet() + BlockDiffer.fromBlock(blockchain, prevBlock, b, MiningConstraint.Unlimited: MiningConstraint, b.header.generationSignature).explicitGet() "when NG state is empty" in forAll(ngEmptyScenario) { case (miner1, miner2, b2s, b3, m3s) => diff --git a/node/src/test/scala/com/wavesplatform/history/Domain.scala b/node/src/test/scala/com/wavesplatform/history/Domain.scala index 97975e933ce..f9a7e8dd1b4 100644 --- a/node/src/test/scala/com/wavesplatform/history/Domain.scala +++ b/node/src/test/scala/com/wavesplatform/history/Domain.scala @@ -115,7 +115,7 @@ case class Domain(db: DB, blockchainUpdater: BlockchainUpdaterImpl, levelDBWrite timestamp = timestamp, reference = reference, baseTarget = blockchainUpdater.lastBlockHeader.fold(60L)(_.header.baseTarget), - generationSignature = com.wavesplatform.history.generationSignature, + generationSignature = com.wavesplatform.history.correctGenerationSignature(version), txs = txs, featureVotes = Nil, rewardVote = -1L, diff --git a/node/src/test/scala/com/wavesplatform/state/BlockchainUpdaterImplSpec.scala b/node/src/test/scala/com/wavesplatform/state/BlockchainUpdaterImplSpec.scala index fb4c78314e5..5fe280fbf54 100644 --- a/node/src/test/scala/com/wavesplatform/state/BlockchainUpdaterImplSpec.scala +++ b/node/src/test/scala/com/wavesplatform/state/BlockchainUpdaterImplSpec.scala @@ -1,33 +1,39 @@ package com.wavesplatform.state +import com.google.common.primitives.Longs import com.typesafe.config.ConfigFactory import com.wavesplatform.TestHelpers.enableNG import com.wavesplatform.account.{Address, KeyPair} import com.wavesplatform.block.Block import com.wavesplatform.common.state.ByteStr import com.wavesplatform.common.utils.EitherExt2 -import com.wavesplatform.db.DBCacheSettings +import com.wavesplatform.db.{DBCacheSettings, WithDomain} import com.wavesplatform.events.BlockchainUpdateTriggers +import com.wavesplatform.features.BlockchainFeatures import com.wavesplatform.history.Domain.BlockchainUpdaterExt import com.wavesplatform.history.{chainBaseAndMicro, randomSig} import com.wavesplatform.lagonaki.mocks.TestBlock -import com.wavesplatform.settings.{TestFunctionalitySettings, WavesSettings, loadConfig} +import com.wavesplatform.lang.v1.estimator.v2.ScriptEstimatorV2 +import com.wavesplatform.settings.{WavesSettings, loadConfig} import com.wavesplatform.state.diffs.ENOUGH_AMT -import com.wavesplatform.state.utils._ import com.wavesplatform.transaction.Asset.Waves +import com.wavesplatform.transaction.smart.script.ScriptCompiler +import com.wavesplatform.transaction.smart.{InvokeScriptTransaction, SetScriptTransaction} import com.wavesplatform.transaction.transfer.TransferTransaction import com.wavesplatform.transaction.{GenesisTransaction, Transaction} import com.wavesplatform.utils.Time -import com.wavesplatform.{EitherMatchers, NTPTime, RequestGen, WithDB} +import com.wavesplatform.{EitherMatchers, NTPTime, RequestGen} import org.scalacheck.Gen import org.scalamock.scalatest.MockFactory import org.scalatest.{FreeSpec, Matchers} +import scala.util.Random + class BlockchainUpdaterImplSpec extends FreeSpec with Matchers with EitherMatchers - with WithDB + with WithDomain with RequestGen with NTPTime with DBCacheSettings @@ -36,8 +42,7 @@ class BlockchainUpdaterImplSpec private val FEE_AMT = 1000000L // default settings, no NG - private lazy val functionalitySettings = TestFunctionalitySettings.Stub - private lazy val wavesSettings = WavesSettings.fromRootConfig(loadConfig(ConfigFactory.load())) + private lazy val wavesSettings = WavesSettings.fromRootConfig(loadConfig(ConfigFactory.load())) def baseTest( gen: Time => Gen[(KeyPair, Seq[Block])], @@ -45,25 +50,16 @@ class BlockchainUpdaterImplSpec triggers: BlockchainUpdateTriggers = BlockchainUpdateTriggers.noop )( f: (BlockchainUpdaterImpl, KeyPair) => Unit - ): Unit = { - val (fs, settings) = - if (enableNg) (enableNG(functionalitySettings), enableNG(wavesSettings)) else (functionalitySettings, wavesSettings) - - val defaultWriter = TestLevelDB.withFunctionalitySettings(db, ignoreSpendableBalanceChanged, fs) - val bcu = new BlockchainUpdaterImpl(defaultWriter, ignoreSpendableBalanceChanged, settings, ntpTime, triggers, (_, _) => Seq.empty) - try { - val (account, blocks) = gen(ntpTime).sample.get + ): Unit = withDomain(if (enableNg) enableNG(wavesSettings) else wavesSettings) { d => + d.triggers = d.triggers :+ triggers - blocks.foreach { block => - bcu.processBlock(block) should beRight - } + val (account, blocks) = gen(ntpTime).sample.get - bcu.shutdown() - f(bcu, account) - } finally { - bcu.shutdown() - db.close() + blocks.foreach { block => + d.appendBlock(block) } + + f(d.blockchainUpdater, account) } def createTransfer(master: KeyPair, recipient: Address, ts: Long): TransferTransaction = { @@ -179,7 +175,7 @@ class BlockchainUpdaterImplSpec baseTest(time => commonPreconditions(time.correctedTime()), enableNg = true, triggersMock)((_, _) => ()) } - "block, then 2 microblocks, then block referencing previous microblock" in { + "block, then 2 microblocks, then block referencing previous microblock" in withDomain(enableNG(wavesSettings)) { d => def preconditions(ts: Long): Gen[(Transaction, Seq[Transaction])] = for { master <- accountGen @@ -196,84 +192,114 @@ class BlockchainUpdaterImplSpec val triggersMock = mock[BlockchainUpdateTriggers] - val defaultWriter = - TestLevelDB.withFunctionalitySettings(db, ignoreSpendableBalanceChanged, enableNG(functionalitySettings)) - val bcu = - new BlockchainUpdaterImpl(defaultWriter, ignoreSpendableBalanceChanged, enableNG(wavesSettings), ntpTime, triggersMock, (_, _) => Seq.empty) - - try { - val (genesis, transfers) = preconditions(0).sample.get - val (block1, microBlocks1And2) = chainBaseAndMicro(randomSig, genesis, Seq(transfers.take(2), Seq(transfers(2)))) - val (block2, microBlock3) = chainBaseAndMicro(microBlocks1And2.head.totalResBlockSig, transfers(3), Seq(Seq(transfers(4)))) - - inSequence { - // genesis - (triggersMock.onProcessBlock _) - .expects(where { (block, diff, _, bc) => - bc.height == 0 && - block.transactionData.length == 1 && - diff.parentDiff.portfolios.head._2.balance == 0 && - diff.transactionDiffs.head.portfolios.head._2.balance == ENOUGH_AMT - }) - .once() - - // microblock 1 - (triggersMock.onProcessMicroBlock _) - .expects(where { (microBlock, diff, bc, _, _) => - bc.height == 1 && - microBlock.transactionData.length == 2 && - // miner reward, no NG — all txs fees - diff.parentDiff.portfolios.size == 1 && - diff.parentDiff.portfolios.head._2.balance == FEE_AMT * 2 * 0.4 - }) - .once() - - // microblock 2 - (triggersMock.onProcessMicroBlock _) - .expects(where { (microBlock, diff, bc, _, _) => - bc.height == 1 && - microBlock.transactionData.length == 1 && - // miner reward, no NG — all txs fees - diff.parentDiff.portfolios.size == 1 && - diff.parentDiff.portfolios.head._2.balance == FEE_AMT * 0.4 - }) - .once() - - // rollback microblock - (triggersMock.onMicroBlockRollback _) - .expects(where { (_, toSig) => - toSig == microBlocks1And2.head.totalResBlockSig - }) - .once() - - // next keyblock - (triggersMock.onProcessBlock _) - .expects(where { (block, _, _, bc) => - bc.height == 1 && - block.header.reference == microBlocks1And2.head.totalResBlockSig - }) - .once() - - // microblock 3 - (triggersMock.onProcessMicroBlock _) - .expects(where { (microBlock, _, bc, _, _) => - bc.height == 2 && microBlock.reference == block2.signature - }) - .once() - } - - bcu.processBlock(block1) should beRight - bcu.processMicroBlock(microBlocks1And2.head) should beRight - bcu.processMicroBlock(microBlocks1And2.last) should beRight - bcu.processBlock(block2) should beRight // this should remove previous microblock - bcu.processMicroBlock(microBlock3.head) should beRight - bcu.shutdown() - } finally { - bcu.shutdown() - db.close() + d.triggers = d.triggers :+ triggersMock + + val (genesis, transfers) = preconditions(0).sample.get + val (block1, microBlocks1And2) = chainBaseAndMicro(randomSig, genesis, Seq(transfers.take(2), Seq(transfers(2)))) + val (block2, microBlock3) = chainBaseAndMicro(microBlocks1And2.head.totalResBlockSig, transfers(3), Seq(Seq(transfers(4)))) + + inSequence { + // genesis + (triggersMock.onProcessBlock _) + .expects(where { (block, diff, _, bc) => + bc.height == 0 && + block.transactionData.length == 1 && + diff.parentDiff.portfolios.head._2.balance == 0 && + diff.transactionDiffs.head.portfolios.head._2.balance == ENOUGH_AMT + }) + .once() + + // microblock 1 + (triggersMock.onProcessMicroBlock _) + .expects(where { (microBlock, diff, bc, _, _) => + bc.height == 1 && + microBlock.transactionData.length == 2 && + // miner reward, no NG — all txs fees + diff.parentDiff.portfolios.size == 1 && + diff.parentDiff.portfolios.head._2.balance == FEE_AMT * 2 * 0.4 + }) + .once() + + // microblock 2 + (triggersMock.onProcessMicroBlock _) + .expects(where { (microBlock, diff, bc, _, _) => + bc.height == 1 && + microBlock.transactionData.length == 1 && + // miner reward, no NG — all txs fees + diff.parentDiff.portfolios.size == 1 && + diff.parentDiff.portfolios.head._2.balance == FEE_AMT * 0.4 + }) + .once() + + // rollback microblock + (triggersMock.onMicroBlockRollback _) + .expects(where { (_, toSig) => + toSig == microBlocks1And2.head.totalResBlockSig + }) + .once() + + // next keyblock + (triggersMock.onProcessBlock _) + .expects(where { (block, _, _, bc) => + bc.height == 1 && + block.header.reference == microBlocks1And2.head.totalResBlockSig + }) + .once() + + // microblock 3 + (triggersMock.onProcessMicroBlock _) + .expects(where { (microBlock, _, bc, _, _) => + bc.height == 2 && microBlock.reference == block2.signature + }) + .once() } + + d.blockchainUpdater.processBlock(block1) should beRight + d.blockchainUpdater.processMicroBlock(microBlocks1And2.head) should beRight + d.blockchainUpdater.processMicroBlock(microBlocks1And2.last) should beRight + d.blockchainUpdater.processBlock(block2) should beRight // this should remove previous microblock + d.blockchainUpdater.processMicroBlock(microBlock3.head) should beRight + d.blockchainUpdater.shutdown() } } - } + "VRF" in withDomain( + domainSettingsWithFeatures( + BlockchainFeatures.NG, + BlockchainFeatures.BlockV5, + BlockchainFeatures.Ride4DApps + ) + ) { d => + + val script = ScriptCompiler.compile(""" + |{-# STDLIB_VERSION 4 #-} + |{-# SCRIPT_TYPE ACCOUNT #-} + |{-# CONTENT_TYPE DAPP #-} + | + |@Callable(i) + |func default() = { + | [ + | BinaryEntry("vrf", value(value(blockInfoByHeight(height)).vrf)) + | ] + |} + |""".stripMargin, ScriptEstimatorV2).explicitGet()._1 + + val dapp = KeyPair(Longs.toByteArray(Random.nextLong())) + val sender = KeyPair(Longs.toByteArray(Random.nextLong())) + + d.appendBlock( + GenesisTransaction.create(dapp.toAddress, 10_00000000, ntpTime.getTimestamp()).explicitGet(), + GenesisTransaction.create(sender.toAddress, 10_00000000, ntpTime.getTimestamp()).explicitGet() + ) + + d.appendBlock( + SetScriptTransaction.selfSigned(2.toByte, dapp, Some(script), 500_0000L, ntpTime.getTimestamp()).explicitGet() + ) + + val invoke = + InvokeScriptTransaction.selfSigned(3.toByte, sender, dapp.toAddress, None, Seq.empty, 50_0000L, Waves, ntpTime.getTimestamp()).explicitGet() + + d.appendBlock(d.createBlock(5.toByte, Seq(invoke))) + } + } } diff --git a/node/src/test/scala/com/wavesplatform/state/CompositeBlockchainSpec.scala b/node/src/test/scala/com/wavesplatform/state/CompositeBlockchainSpec.scala deleted file mode 100644 index ea0fca7ffad..00000000000 --- a/node/src/test/scala/com/wavesplatform/state/CompositeBlockchainSpec.scala +++ /dev/null @@ -1,15 +0,0 @@ -package com.wavesplatform.state - -import com.wavesplatform.state.reader.CompositeBlockchain -import com.wavesplatform.utils.EmptyBlockchain -import com.wavesplatform.{BlockGen, NoShrink} -import org.scalatest.{FreeSpec, Matchers} -import org.scalatestplus.scalacheck.{ScalaCheckPropertyChecks => PropertyChecks} - -class CompositeBlockchainSpec extends FreeSpec with Matchers with PropertyChecks with BlockGen with NoShrink { - "blockHeaderAndSize at current height is last block" in forAll(randomSignerBlockGen) { block => - val comp = CompositeBlockchain(EmptyBlockchain, newBlock = Some(block)) - - comp.blockHeader(comp.height).map(_.id()) should contain(block.id()) - } -} diff --git a/node/src/test/scala/com/wavesplatform/state/diffs/BlockDifferDetailedDiffTest.scala b/node/src/test/scala/com/wavesplatform/state/diffs/BlockDifferDetailedDiffTest.scala index f888541bb25..8841bbc3e3c 100644 --- a/node/src/test/scala/com/wavesplatform/state/diffs/BlockDifferDetailedDiffTest.scala +++ b/node/src/test/scala/com/wavesplatform/state/diffs/BlockDifferDetailedDiffTest.scala @@ -23,7 +23,7 @@ class BlockDifferDetailedDiffTest extends FreeSpec with Matchers with PropertyCh ): Unit = withLevelDBWriter(fs) { state => def differ(blockchain: Blockchain, prevBlock: Option[Block], b: Block) = - BlockDiffer.fromBlock(blockchain, prevBlock, b, MiningConstraint.Unlimited) + BlockDiffer.fromBlock(blockchain, prevBlock, b, MiningConstraint.Unlimited, b.header.generationSignature) preconditions.foldLeft[Option[Block]](None) { (prevBlock, curBlock) => val BlockDiffer.Result(diff, fees, totalFee, _, _) = differ(state, prevBlock, curBlock).explicitGet() diff --git a/node/src/test/scala/com/wavesplatform/state/diffs/CommonValidationTest.scala b/node/src/test/scala/com/wavesplatform/state/diffs/CommonValidationTest.scala index c08f26c6a69..9b811ad3bd6 100644 --- a/node/src/test/scala/com/wavesplatform/state/diffs/CommonValidationTest.scala +++ b/node/src/test/scala/com/wavesplatform/state/diffs/CommonValidationTest.scala @@ -55,7 +55,7 @@ class CommonValidationTest extends PropSpec with PropertyChecks with Matchers wi case (genesisBlock, transferTx) => withLevelDBWriter(settings) { blockchain => val BlockDiffer.Result(preconditionDiff, preconditionFees, totalFee, _, _) = - BlockDiffer.fromBlock(blockchain, None, genesisBlock, MiningConstraint.Unlimited).explicitGet() + BlockDiffer.fromBlock(blockchain, None, genesisBlock, MiningConstraint.Unlimited, genesisBlock.header.generationSignature).explicitGet() blockchain.append(preconditionDiff, preconditionFees, totalFee, None, genesisBlock.header.generationSignature, genesisBlock) f(FeeValidation(blockchain, transferTx)) @@ -78,7 +78,7 @@ class CommonValidationTest extends PropSpec with PropertyChecks with Matchers wi case (genesisBlock, transferTx) => withLevelDBWriter(settings) { blockchain => val BlockDiffer.Result(preconditionDiff, preconditionFees, totalFee, _, _) = - BlockDiffer.fromBlock(blockchain, None, genesisBlock, MiningConstraint.Unlimited).explicitGet() + BlockDiffer.fromBlock(blockchain, None, genesisBlock, MiningConstraint.Unlimited, genesisBlock.header.generationSignature).explicitGet() blockchain.append(preconditionDiff, preconditionFees, totalFee, None, genesisBlock.header.generationSignature, genesisBlock) f(FeeValidation(blockchain, transferTx)) @@ -200,7 +200,7 @@ class CommonValidationTest extends PropSpec with PropertyChecks with Matchers wi case (genesisBlock, transferTx) => withLevelDBWriter(settings) { blockchain => val BlockDiffer.Result(preconditionDiff, preconditionFees, totalFee, _, _) = - BlockDiffer.fromBlock(blockchain, None, genesisBlock, MiningConstraint.Unlimited).explicitGet() + BlockDiffer.fromBlock(blockchain, None, genesisBlock, MiningConstraint.Unlimited, genesisBlock.header.generationSignature).explicitGet() blockchain.append(preconditionDiff, preconditionFees, totalFee, None, genesisBlock.header.generationSignature, genesisBlock) f(FeeValidation(blockchain, transferTx)) diff --git a/node/src/test/scala/com/wavesplatform/state/diffs/ExchangeTransactionDiffTest.scala b/node/src/test/scala/com/wavesplatform/state/diffs/ExchangeTransactionDiffTest.scala index ad6b94082cd..735f4c0b85a 100644 --- a/node/src/test/scala/com/wavesplatform/state/diffs/ExchangeTransactionDiffTest.scala +++ b/node/src/test/scala/com/wavesplatform/state/diffs/ExchangeTransactionDiffTest.scala @@ -1178,7 +1178,7 @@ class ExchangeTransactionDiffTest ) { d => d.appendBlock(Seq(amountAssetIssue, priceAssetIssue, order1FeeAssetIssue, order2FeeAssetIssue).distinct: _*) val newBlock = d.createBlock(2.toByte, Seq(exchange)) - val diff = BlockDiffer.fromBlock(d.blockchainUpdater, Some(d.lastBlock), newBlock, MiningConstraint.Unlimited).explicitGet() + val diff = BlockDiffer.fromBlock(d.blockchainUpdater, Some(d.lastBlock), newBlock, MiningConstraint.Unlimited, newBlock.header.generationSignature).explicitGet() diff.diff.scriptsComplexity shouldBe complexity } } @@ -1252,7 +1252,7 @@ class ExchangeTransactionDiffTest ) { d => d.appendBlock(Seq(tradeableAssetIssue, feeAssetIssue).distinct: _*) val newBlock = d.createBlock(2.toByte, Seq(exchange)) - val diff = BlockDiffer.fromBlock(d.blockchainUpdater, Some(d.lastBlock), newBlock, MiningConstraint.Unlimited).explicitGet() + val diff = BlockDiffer.fromBlock(d.blockchainUpdater, Some(d.lastBlock), newBlock, MiningConstraint.Unlimited, newBlock.header.generationSignature).explicitGet() diff.diff.scriptsComplexity shouldBe complexity val feeUnits = FeeValidation.getMinFee(d.blockchainUpdater, exchange).explicitGet().minFeeInWaves / FeeValidation.FeeUnit diff --git a/node/src/test/scala/com/wavesplatform/state/diffs/ReissueTransactionDiffTest.scala b/node/src/test/scala/com/wavesplatform/state/diffs/ReissueTransactionDiffTest.scala index aa4825bbac3..b3a678c5d0e 100644 --- a/node/src/test/scala/com/wavesplatform/state/diffs/ReissueTransactionDiffTest.scala +++ b/node/src/test/scala/com/wavesplatform/state/diffs/ReissueTransactionDiffTest.scala @@ -88,7 +88,7 @@ class ReissueTransactionDiffTest withLevelDBWriter(fs) { blockchain => preconditions.foreach { block => val BlockDiffer.Result(preconditionDiff, preconditionFees, totalFee, _, _) = - BlockDiffer.fromBlock(blockchain, blockchain.lastBlock, block, MiningConstraint.Unlimited).explicitGet() + BlockDiffer.fromBlock(blockchain, blockchain.lastBlock, block, MiningConstraint.Unlimited, block.header.generationSignature).explicitGet() blockchain.append(preconditionDiff, preconditionFees, totalFee, None, block.header.generationSignature, block) } f((FeeValidation(blockchain, txs._1), FeeValidation(blockchain, txs._2), FeeValidation(blockchain, txs._3))) diff --git a/node/src/test/scala/com/wavesplatform/state/diffs/ScriptsCountTest.scala b/node/src/test/scala/com/wavesplatform/state/diffs/ScriptsCountTest.scala index ad7f25da586..6979e39007f 100644 --- a/node/src/test/scala/com/wavesplatform/state/diffs/ScriptsCountTest.scala +++ b/node/src/test/scala/com/wavesplatform/state/diffs/ScriptsCountTest.scala @@ -216,7 +216,7 @@ class ScriptsCountTest extends PropSpec with PropertyChecks with WithState with assertDiffAndState(Nil, TestBlock.create(Seq(genesis)), fs) { case (_, state) => txs.foldLeft(Diff.empty) { (diff, tx) => - val newState = CompositeBlockchain(state, Some(diff)) + val newState = CompositeBlockchain(state, diff) val newDiff = TransactionDiffer(Some(tx.timestamp), tx.timestamp)(newState, tx).resultE.explicitGet() val oldRuns = ScriptsCountTest.calculateLegacy(newState, tx) if (newDiff.scriptsRun != oldRuns) throw new IllegalArgumentException(s"$tx ${newDiff.scriptsRun} != $oldRuns")