Skip to content

Commit

Permalink
Merge pull request #1108 from wavesplatform/node-129-fair-proof-of-stake
Browse files Browse the repository at this point in the history
NODE-129 Fair Proof-of-Stake
  • Loading branch information
phearnot authored May 23, 2018
2 parents 169dbaf + 74bd220 commit b0fb2a4
Show file tree
Hide file tree
Showing 19 changed files with 633 additions and 276 deletions.
23 changes: 12 additions & 11 deletions it/src/main/scala/com/wavesplatform/it/BaseTargetChecker.scala
Original file line number Diff line number Diff line change
Expand Up @@ -3,43 +3,44 @@ package com.wavesplatform.it
import java.time.Instant

import com.typesafe.config.ConfigFactory.{defaultApplication, defaultReference}
import com.wavesplatform.consensus.PoSSelector
import com.wavesplatform.db.openDB
import com.wavesplatform.history.StorageFactory
import com.wavesplatform.settings._
import com.wavesplatform.state.{ByteStr, EitherExt2}
import net.ceedubs.ficus.Ficus._
import scorex.account.PublicKeyAccount
import scorex.block.Block
import scorex.transaction.PoSCalc
import scorex.utils.NTP

object BaseTargetChecker {
def main(args: Array[String]): Unit = {
val startTs = System.currentTimeMillis()
val docker = Docker(getClass)
val docker = Docker(getClass)
val sharedConfig = docker.genesisOverride
.withFallback(docker.configTemplate)
.withFallback(defaultApplication())
.withFallback(defaultReference())
.resolve()
val settings = WavesSettings.fromConfig(sharedConfig)
val fs = settings.blockchainSettings.functionalitySettings
val genesisBlock = Block.genesis(settings.blockchainSettings.genesisSettings).explicitGet()
val db = openDB("/tmp/tmp-db", 1024)
val bu = StorageFactory(settings, db, NTP)
val pos = new PoSSelector(bu, settings.blockchainSettings)
bu.processBlock(genesisBlock)

println(s"Genesis TS = ${Instant.ofEpochMilli(genesisBlock.timestamp)}")

val m = NodeConfigs.Default.map(_.withFallback(sharedConfig)).collect {
case cfg if cfg.as[Boolean]("waves.miner.enable") =>
val publicKey = PublicKeyAccount(cfg.as[ByteStr]("public-key").arr)
val address = publicKey.toAddress
PoSCalc.nextBlockGenerationTime(1, bu, fs, genesisBlock, publicKey) match {
case Right((_, ts)) => f"$address: ${(ts - startTs) * 1e-3}%10.3f s"
case _ => s"$address: n/a"
}

val account = PublicKeyAccount(cfg.as[ByteStr]("public-key").arr)
val address = account.toAddress
val balance = bu.balance(address, None)
val consensus = genesisBlock.consensusData
val timeDelay = pos
.getValidBlockDelay(bu.height, account.publicKey, consensus.baseTarget, balance)
.explicitGet()

f"$address: ${timeDelay * 1e-3}%10.3f s"
}

docker.close()
Expand Down
49 changes: 49 additions & 0 deletions it/src/test/scala/com/wavesplatform/it/sync/FairPoSTestSuite.scala
Original file line number Diff line number Diff line change
@@ -0,0 +1,49 @@
package com.wavesplatform.it.sync

import com.typesafe.config.{Config, ConfigFactory}
import org.scalatest.{CancelAfterFailure, FunSuite}
import com.wavesplatform.it.api.State
import com.wavesplatform.it.api.SyncHttpApi._
import com.wavesplatform.it.transactions.NodesFromDocker
import com.wavesplatform.it.util._
import scala.concurrent.duration._

class FairPoSTestSuite extends FunSuite with CancelAfterFailure with NodesFromDocker {
import FairPoSTestSuite._

override protected def nodeConfigs: Seq[Config] = Configs

private val transferFee = 0.001.waves
private val transferAmount = 1000.waves

test("blockchain grows with FairPoS activated") {
nodes.head.waitForHeight(10, 3.minutes)

val txId = nodes.head.transfer(nodes.head.address, nodes.last.address, transferAmount, transferFee).id
nodes.last.waitForTransaction(txId)

val heightAfterTransfer = nodes.head.height

nodes.head.waitForHeight(heightAfterTransfer + 20, 10.minutes)
}
}

object FairPoSTestSuite {
import com.wavesplatform.it.NodeConfigs._
private val microblockActivationHeight = 0
private val fairPoSActivationHeight = 10

private val config =
ConfigFactory.parseString(s"""
|waves {
| blockchain.custom {
| functionality {
| pre-activated-features {1 = $microblockActivationHeight, 8 = $fairPoSActivationHeight}
| generation-balance-depth-from-50-to-1000-after-height = 1000
| }
| }
| miner.quorum = 1
|}""".stripMargin)

val Configs: Seq[Config] = Default.map(config.withFallback(_)).take(4)
}
11 changes: 9 additions & 2 deletions src/main/scala/com/wavesplatform/Application.scala
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@ import akka.stream.ActorMaterializer
import cats.instances.all._
import com.typesafe.config._
import com.wavesplatform.actor.RootActorSystem
import com.wavesplatform.consensus.PoSSelector
import com.wavesplatform.db.openDB
import com.wavesplatform.features.api.ActivationApiRoute
import com.wavesplatform.history.{CheckpointServiceImpl, StorageFactory}
Expand Down Expand Up @@ -123,19 +124,25 @@ class Application(val actorSystem: ActorSystem, val settings: WavesSettings, con
maybeUtx = Some(utxStorage)

val knownInvalidBlocks = new InvalidBlockStorageImpl(settings.synchronizationSettings.invalidBlocksStorage)

val pos = new PoSSelector(blockchainUpdater, settings.blockchainSettings)

val miner =
if (settings.minerSettings.enable)
new MinerImpl(allChannels, blockchainUpdater, checkpointService, settings, time, utxStorage, wallet, minerScheduler, appenderScheduler)
new MinerImpl(allChannels, blockchainUpdater, checkpointService, settings, time, utxStorage, wallet, pos, minerScheduler, appenderScheduler)
else Miner.Disabled

val processBlock =
BlockAppender(checkpointService, blockchainUpdater, time, utxStorage, settings, allChannels, peerDatabase, miner, appenderScheduler) _
BlockAppender(checkpointService, blockchainUpdater, time, utxStorage, pos, settings, allChannels, peerDatabase, miner, appenderScheduler) _

val processCheckpoint =
CheckpointAppender(checkpointService, blockchainUpdater, blockchainUpdater, peerDatabase, miner, allChannels, appenderScheduler) _

val processFork = ExtensionAppender(
checkpointService,
blockchainUpdater,
utxStorage,
pos,
time,
settings,
knownInvalidBlocks,
Expand Down
4 changes: 3 additions & 1 deletion src/main/scala/com/wavesplatform/Importer.scala
Original file line number Diff line number Diff line change
Expand Up @@ -4,6 +4,7 @@ import java.io._

import com.google.common.primitives.Ints
import com.typesafe.config.ConfigFactory
import com.wavesplatform.consensus.PoSSelector
import com.wavesplatform.db.openDB
import com.wavesplatform.history.{CheckpointServiceImpl, StorageFactory}
import com.wavesplatform.mining.MultiDimensionalMiningConstraint
Expand Down Expand Up @@ -57,8 +58,9 @@ object Importer extends ScorexLogging {
case Success(inputStream) =>
val db = openDB(settings.dataDirectory, settings.levelDbCacheSize)
val blockchainUpdater = StorageFactory(settings, db, NTP)
val pos = new PoSSelector(blockchainUpdater, settings.blockchainSettings)
val checkpoint = new CheckpointServiceImpl(db, settings.checkpointsSettings)
val extAppender = BlockAppender(checkpoint, blockchainUpdater, NTP, utxPoolStub, settings, scheduler) _
val extAppender = BlockAppender(checkpoint, blockchainUpdater, NTP, utxPoolStub, pos, settings, scheduler) _
checkGenesis(settings, blockchainUpdater)
val bis = new BufferedInputStream(inputStream)
var quit = false
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,30 @@
package com.wavesplatform.consensus

import com.wavesplatform.features.BlockchainFeatures
import com.wavesplatform.settings.FunctionalitySettings
import com.wavesplatform.state.Blockchain
import scorex.account.Address
import scorex.block.Block

object GeneratingBalanceProvider {
private val MinimalEffectiveBalanceForGenerator1: Long = 1000000000000L
private val MinimalEffectiveBalanceForGenerator2: Long = 100000000000L
private val FirstDepth = 50
private val SecondDepth = 1000

def isMiningAllowed(blockchain: Blockchain, height: Int, effectiveBalance: Long): Boolean = {
val activated = blockchain.activatedFeatures.get(BlockchainFeatures.SmallerMinimalGeneratingBalance.id).exists(height >= _)
(!activated && effectiveBalance >= MinimalEffectiveBalanceForGenerator1) || (activated && effectiveBalance >= MinimalEffectiveBalanceForGenerator2)
}

def isEffectiveBalanceValid(blockchain: Blockchain, fs: FunctionalitySettings, height: Int, block: Block, effectiveBalance: Long): Boolean =
block.timestamp < fs.minimalGeneratingBalanceAfter || (block.timestamp >= fs.minimalGeneratingBalanceAfter && effectiveBalance >= MinimalEffectiveBalanceForGenerator1) ||
blockchain.activatedFeatures
.get(BlockchainFeatures.SmallerMinimalGeneratingBalance.id)
.exists(height >= _) && effectiveBalance >= MinimalEffectiveBalanceForGenerator2

def balance(blockchain: Blockchain, fs: FunctionalitySettings, height: Int, account: Address): Long = {
val depth = if (height >= fs.generationBalanceDepthFrom50To1000AfterHeight) SecondDepth else FirstDepth
blockchain.effectiveBalance(account, height, depth)
}
}
112 changes: 112 additions & 0 deletions src/main/scala/com/wavesplatform/consensus/PoSCalculator.scala
Original file line number Diff line number Diff line change
@@ -0,0 +1,112 @@
package com.wavesplatform.consensus

import com.wavesplatform.crypto

trait PoSCalculator {
def calculateBaseTarget(targetBlockDelaySeconds: Long,
prevHeight: Int,
prevBaseTarget: Long,
parentTimestamp: Long,
maybeGreatGrandParentTimestamp: Option[Long],
timestamp: Long): Long

def calculateDelay(hit: BigInt, bt: Long, balance: Long): Long
}

object PoSCalculator {
private[consensus] val HitSize: Int = 8
private[consensus] val MinBaseTarget: Long = 9

private[consensus] def generatorSignature(signature: Array[Byte], publicKey: Array[Byte]): Array[Byte] = {
val s = new Array[Byte](crypto.DigestSize * 2)
System.arraycopy(signature, 0, s, 0, crypto.DigestSize)
System.arraycopy(publicKey, 0, s, crypto.DigestSize, crypto.DigestSize)
crypto.fastHash(s)
}

private[consensus] def hit(generatorSignature: Array[Byte]): BigInt = BigInt(1, generatorSignature.take(HitSize).reverse)

private[consensus] def normalize(value: Long, targetBlockDelaySeconds: Long): Double =
value * targetBlockDelaySeconds / (60: Double)

private[consensus] def normalizeBaseTarget(baseTarget: Long, targetBlockDelaySeconds: Long): Long = {
baseTarget
.max(MinBaseTarget)
.min(Long.MaxValue / targetBlockDelaySeconds)
}
}

object NxtPoSCalculator extends PoSCalculator {
protected val MinBlockDelaySeconds = 53
protected val MaxBlockDelaySeconds = 67
protected val BaseTargetGamma = 64
protected val MeanCalculationDepth = 3

import PoSCalculator._

def calculateBaseTarget(targetBlockDelaySeconds: Long,
prevHeight: Int,
prevBaseTarget: Long,
parentTimestamp: Long,
maybeGreatGrandParentTimestamp: Option[Long],
timestamp: Long): Long = {

if (prevHeight % 2 == 0) {
val meanBlockDelay = maybeGreatGrandParentTimestamp.fold(timestamp - parentTimestamp)(ts => (timestamp - ts) / MeanCalculationDepth) / 1000
val minBlockDelay = normalize(MinBlockDelaySeconds, targetBlockDelaySeconds)
val maxBlockDelay = normalize(MaxBlockDelaySeconds, targetBlockDelaySeconds)
val baseTargetGamma = normalize(BaseTargetGamma, targetBlockDelaySeconds)

val baseTarget = (if (meanBlockDelay > targetBlockDelaySeconds) {
prevBaseTarget * Math.min(meanBlockDelay, maxBlockDelay) / targetBlockDelaySeconds
} else {
prevBaseTarget - prevBaseTarget * baseTargetGamma *
(targetBlockDelaySeconds - Math.max(meanBlockDelay, minBlockDelay)) / (targetBlockDelaySeconds * 100)
}).toLong

normalizeBaseTarget(baseTarget, targetBlockDelaySeconds)
} else {
prevBaseTarget
}
}

def calculateDelay(hit: BigInt, bt: Long, balance: Long): Long = ((hit * 1000) / (BigInt(bt) * balance)).toLong

}

object FairPoSCalculator extends PoSCalculator {

import PoSCalculator._

private val MaxSignature: Array[Byte] = Array.fill[Byte](HitSize)(-1)
private val MaxHit: BigDecimal = BigDecimal(BigInt(1, MaxSignature))
private val C1 = 70000
private val C2 = 5E17
private val TMin = 5000

def calculateDelay(hit: BigInt, bt: Long, balance: Long): Long = {
val h = (BigDecimal(hit) / MaxHit).toDouble
val a = TMin + C1 * math.log(1 - C2 * math.log(h) / bt / balance)
a.toLong
}

def calculateBaseTarget(targetBlockDelaySeconds: Long,
prevHeight: Int,
prevBaseTarget: Long,
parentTimestamp: Long,
maybeGreatGrandParentTimestamp: Option[Long],
timestamp: Long): Long = {
val maxDelay = normalize(90, targetBlockDelaySeconds)
val minDelay = normalize(30, targetBlockDelaySeconds)

maybeGreatGrandParentTimestamp match {
case None =>
prevBaseTarget
case Some(ts) =>
val avg = (timestamp - ts) / 3 / 1000
if (avg > maxDelay) prevBaseTarget + math.max(1, prevBaseTarget / 100)
else if (avg < minDelay) prevBaseTarget - math.max(1, prevBaseTarget / 100)
else prevBaseTarget
}
}
}
92 changes: 92 additions & 0 deletions src/main/scala/com/wavesplatform/consensus/PoSSelector.scala
Original file line number Diff line number Diff line change
@@ -0,0 +1,92 @@
package com.wavesplatform.consensus

import cats.implicits._
import com.wavesplatform.features.BlockchainFeatures
import com.wavesplatform.features.FeatureProvider._
import com.wavesplatform.settings.BlockchainSettings
import com.wavesplatform.state.{Blockchain, ByteStr, _}
import scorex.block.Block
import scorex.consensus.nxt.NxtLikeConsensusBlockData
import scorex.transaction.ValidationError
import scorex.transaction.ValidationError.GenericError

import scala.concurrent.duration.FiniteDuration

class PoSSelector(blockchain: Blockchain, settings: BlockchainSettings) {

import PoSCalculator._

protected def pos(height: Int): PoSCalculator =
if (fairPosActivated(height)) FairPoSCalculator
else NxtPoSCalculator

def consensusData(accountPublicKey: Array[Byte],
height: Int,
targetBlockDelay: FiniteDuration,
refBlockBT: Long,
refBlockTS: Long,
greatGrandParentTS: Option[Long],
currentTime: Long): Either[ValidationError, NxtLikeConsensusBlockData] = {
val bt = pos(height).calculateBaseTarget(targetBlockDelay.toSeconds, height, refBlockBT, refBlockTS, greatGrandParentTS, currentTime)
blockchain.lastBlock
.map(_.consensusData.generationSignature.arr)
.map(gs => NxtLikeConsensusBlockData(bt, ByteStr(generatorSignature(gs, accountPublicKey))))
.toRight(GenericError("No blocks in blockchain"))
}

def getValidBlockDelay(height: Int, accountPublicKey: Array[Byte], refBlockBT: Long, balance: Long): Either[ValidationError, Long] = {
val pc = pos(height)

getHit(height, accountPublicKey)
.map(pc.calculateDelay(_, refBlockBT, balance))
.toRight(GenericError("No blocks in blockchain"))
}

def validateBlockDelay(height: Int, block: Block, parent: Block, effectiveBalance: Long): Either[ValidationError, Unit] = {
getValidBlockDelay(height, block.signerData.generator.publicKey, parent.consensusData.baseTarget, effectiveBalance)
.map(_ + parent.timestamp < block.timestamp)
.ensure(GenericError(s"Block time ${block.timestamp} less than expected"))(identity)
.map(_ => ())
}

def validateGeneratorSignature(height: Int, block: Block): Either[ValidationError, Unit] = {
blockchain.lastBlock
.map(b => generatorSignature(b.consensusData.generationSignature.arr, block.signerData.generator.publicKey))
.toRight(GenericError("No blocks in blockchain T.T"))
.ensure(GenericError("Generation signatures doesnot match"))(_ sameElements block.consensusData.generationSignature.arr)
.map(_ => ())
}

def validateBaseTarget(height: Int, block: Block, parent: Block, grandParent: Option[Block]): Either[ValidationError, Unit] = {
val blockBT = block.consensusData.baseTarget
val blockTS = block.timestamp

val expectedBT = pos(height).calculateBaseTarget(
settings.genesisSettings.averageBlockDelay.toSeconds,
height,
parent.consensusData.baseTarget,
parent.timestamp,
grandParent.map(_.timestamp),
blockTS
)

Either.cond(
expectedBT == blockBT,
(),
GenericError(s"declared baseTarget $blockBT does not match calculated baseTarget $expectedBT")
)
}

private def getHit(height: Int, accountPublicKey: Array[Byte]): Option[BigInt] = {
val blockForHit =
if (fairPosActivated(height) && height > 100) blockchain.blockAt(height - 100)
else blockchain.lastBlock

blockForHit.map(b => {
val genSig = b.consensusData.generationSignature.arr
hit(generatorSignature(genSig, accountPublicKey))
})
}

private def fairPosActivated(height: Int): Boolean = blockchain.activatedFeaturesAt(height).contains(BlockchainFeatures.FairPoS.id)
}
Loading

0 comments on commit b0fb2a4

Please sign in to comment.