Skip to content

Commit

Permalink
PM-3353: Typed Quorum Certificates.
Browse files Browse the repository at this point in the history
  • Loading branch information
aakoshh committed Jun 18, 2021
1 parent 9ba0c58 commit 8af2945
Show file tree
Hide file tree
Showing 32 changed files with 303 additions and 140 deletions.
@@ -1,7 +1,7 @@
package io.iohk.metronome.checkpointing.models

import cats.data.NonEmptyList
import io.iohk.metronome.hotstuff.consensus.basic.QuorumCertificate
import io.iohk.metronome.hotstuff.consensus.basic.{QuorumCertificate, Phase}
import io.iohk.metronome.checkpointing.CheckpointingAgreement

/** The Checkpoint Certificate is a proof of the BFT agreement
Expand All @@ -26,5 +26,5 @@ case class CheckpointCertificate(
// Proof that `checkpoint` is part of `headers.last.contentMerkleRoot`.
proof: MerkleTree.Proof,
// Commit Q.C. over `headers.head`.
commitQC: QuorumCertificate[CheckpointingAgreement]
commitQC: QuorumCertificate[CheckpointingAgreement, Phase.Commit]
)
Expand Up @@ -17,6 +17,7 @@ import io.iohk.metronome.hotstuff.consensus.basic.{
QuorumCertificate
}
import scodec.bits.{BitVector, ByteVector}
import scala.reflect.ClassTag

object RLPCodecs {
implicit val rlpBitVector: RLPCodec[BitVector] =
Expand Down Expand Up @@ -173,10 +174,10 @@ object RLPCodecs {
: RLPCodec[CheckpointingAgreement.GroupSignature] =
deriveLabelledGenericRLPCodec

// Derviation doesn't seem to work on generic case class.
// Derivation doesn't seem to work on generic case class.
implicit val rlpQuorumCertificate
: RLPCodec[QuorumCertificate[CheckpointingAgreement]] =
RLPCodec.instance[QuorumCertificate[CheckpointingAgreement]](
: RLPCodec[QuorumCertificate[CheckpointingAgreement, VotingPhase]] =
RLPCodec.instance[QuorumCertificate[CheckpointingAgreement, VotingPhase]](
{ case QuorumCertificate(phase, viewNumber, blockHash, signature) =>
RLPList(
RLPEncoder.encode(phase),
Expand All @@ -186,7 +187,7 @@ object RLPCodecs {
)
},
{ case RLPList(phase, viewNumber, blockHash, signature) =>
QuorumCertificate[CheckpointingAgreement](
QuorumCertificate[CheckpointingAgreement, VotingPhase](
phase.decodeAs[VotingPhase]("phase"),
viewNumber.decodeAs[ViewNumber]("viewNumber"),
blockHash.decodeAs[CheckpointingAgreement.Hash]("blockHash"),
Expand All @@ -195,6 +196,27 @@ object RLPCodecs {
}
)

def rlpQuorumCertificate[P <: VotingPhase: ClassTag]
: RLPCodec[QuorumCertificate[CheckpointingAgreement, P]] = {
val ct = implicitly[ClassTag[P]]
rlpQuorumCertificate.xmap[QuorumCertificate[CheckpointingAgreement, P]](
qc =>
ct.unapply(qc.phase) match {
case Some(_) => qc.coerce[P]
case None =>
RLPException.decodeError(
"QuorumCertificate",
s"expected phase ${ct.runtimeClass.getSimpleName}, got ${qc.phase}"
)
},
qc => qc.coerce[VotingPhase]
)
}

implicit val rlpQuorumCertificateCommit
: RLPCodec[QuorumCertificate[CheckpointingAgreement, Phase.Commit]] =
rlpQuorumCertificate[Phase.Commit]

implicit val rlpCheckpointCertificate: RLPCodec[CheckpointCertificate] =
deriveLabelledGenericRLPCodec
}
Expand Up @@ -113,7 +113,7 @@ object ArbitraryInstances

viewNumber <- Gen.posNum[Long].map(x => ViewNumber(x + n))
signature <- arbitrary[CheckpointingAgreement.GSig]
commitQC = QuorumCertificate[CheckpointingAgreement](
commitQC = QuorumCertificate[CheckpointingAgreement, Phase.Commit](
phase = Phase.Commit,
viewNumber = viewNumber,
blockHash = headers.head.hash,
Expand Down
Expand Up @@ -168,7 +168,7 @@ class RLPCodecsSpec extends AnyFlatSpec with Matchers {
leafIndex = 2,
siblingPath = Vector(sample[MerkleTree.Hash], sample[MerkleTree.Hash])
),
commitQC = QuorumCertificate[CheckpointingAgreement](
commitQC = QuorumCertificate[CheckpointingAgreement, Phase.Commit](
phase = Phase.Commit,
viewNumber = ViewNumber(10),
blockHash = sample[Block.Header.Hash],
Expand Down
Expand Up @@ -339,7 +339,7 @@ trait RobotComposition {
protected def makeViewStateStorage(genesis: RobotBlock)(implicit
storeRunner: KVStoreRunner[Task, NS]
) = Resource.liftF {
val genesisQC = QuorumCertificate[RobotAgreement](
val genesisQC = QuorumCertificate[RobotAgreement, Phase.Prepare](
phase = Phase.Prepare,
viewNumber = ViewNumber(0),
blockHash = genesis.hash,
Expand Down
Expand Up @@ -22,7 +22,7 @@ class RobotSigning(
*/
override def validate(
federation: Federation[RobotAgreement.PKey],
quorumCertificate: QuorumCertificate[RobotAgreement]
quorumCertificate: QuorumCertificate[RobotAgreement, _]
): Boolean =
if (quorumCertificate.blockHash == genesisHash) {
quorumCertificate.signature.sig.isEmpty
Expand Down
Expand Up @@ -10,7 +10,7 @@ import io.iohk.metronome.examples.robot.RobotAgreement
import io.iohk.metronome.examples.robot.models.{RobotBlock, Robot}
import io.iohk.metronome.examples.robot.service.messages.RobotMessage
import io.iohk.metronome.examples.robot.service.tracing.RobotTracers
import io.iohk.metronome.hotstuff.consensus.basic.QuorumCertificate
import io.iohk.metronome.hotstuff.consensus.basic.{QuorumCertificate, Phase}
import io.iohk.metronome.hotstuff.service.ApplicationService
import io.iohk.metronome.hotstuff.service.storage.{
BlockStorage,
Expand Down Expand Up @@ -38,7 +38,7 @@ class RobotService[F[_]: Sync: Timer, N](

/** Make a random valid move on top of the last block. */
override def createBlock(
highQC: QuorumCertificate[RobotAgreement]
highQC: QuorumCertificate[RobotAgreement, Phase.Prepare]
): F[Option[RobotBlock]] = {
val parentState = for {
parent <- OptionT {
Expand Down Expand Up @@ -146,7 +146,7 @@ class RobotService[F[_]: Sync: Timer, N](
/** Execute the next block in the queue, store the resulting state. */
override def executeBlock(
block: RobotBlock,
commitQC: QuorumCertificate[RobotAgreement],
commitQC: QuorumCertificate[RobotAgreement, Phase.Commit],
commitPath: NonEmptyList[RobotAgreement.Hash]
): F[Boolean] =
projectState(block.parentHash).flatMap {
Expand Down
Expand Up @@ -43,7 +43,7 @@ object Effect {
*/
case class CreateBlock[A <: Agreement](
viewNumber: ViewNumber,
highQC: QuorumCertificate[A]
highQC: QuorumCertificate[A, Phase.Prepare]
) extends Effect[A]

/** Once the Prepare Q.C. has been established for a block,
Expand All @@ -67,7 +67,7 @@ object Effect {
*/
case class ExecuteBlocks[A <: Agreement](
lastExecutedBlockHash: A#Hash,
quorumCertificate: QuorumCertificate[A]
quorumCertificate: QuorumCertificate[A, Phase.Commit]
) extends Effect[A]

}
Expand Up @@ -21,6 +21,6 @@ object Event {
viewNumber: ViewNumber,
block: A#Block,
// The certificate which the block extended.
highQC: QuorumCertificate[A]
highQC: QuorumCertificate[A, Phase.Prepare]
) extends Event[A]
}
Expand Up @@ -24,7 +24,7 @@ object Message {
case class Prepare[A <: Agreement](
viewNumber: ViewNumber,
block: A#Block,
highQC: QuorumCertificate[A]
highQC: QuorumCertificate[A, Phase.Prepare]
) extends LeaderMessage[A]

/** Having received one of the leader messages, the replica
Expand Down Expand Up @@ -56,14 +56,14 @@ object Message {
*/
case class Quorum[A <: Agreement](
viewNumber: ViewNumber,
quorumCertificate: QuorumCertificate[A]
quorumCertificate: QuorumCertificate[A, VotingPhase]
) extends LeaderMessage[A]

/** At the end of the round, replicas send the `NewView` message
* to the next leader with the last Prepare Q.C.
*/
case class NewView[A <: Agreement](
viewNumber: ViewNumber,
prepareQC: QuorumCertificate[A]
prepareQC: QuorumCertificate[A, Phase.Prepare]
) extends ReplicaMessage[A]
}
Expand Up @@ -46,4 +46,8 @@ object Phase {
case object PreCommit extends VotingPhase
case object Commit extends VotingPhase
case object Decide extends Phase

type Prepare = Prepare.type
type PreCommit = PreCommit.type
type Commit = Commit.type
}
Expand Up @@ -32,7 +32,7 @@ object ProtocolError {
/** The Q.C. signature doesn't match the content. */
case class InvalidQuorumCertificate[A <: Agreement](
sender: A#PKey,
quorumCertificate: QuorumCertificate[A]
quorumCertificate: QuorumCertificate[A, VotingPhase]
) extends ProtocolError[A]

/** The block in the prepare message doesn't extend the previous Q.C. */
Expand Down
Expand Up @@ -42,11 +42,11 @@ case class ProtocolState[A <: Agreement: Block: Signing](
signingKey: A#SKey,
federation: Federation[A#PKey],
// Highest QC for which a replica voted Pre-Commit, because it received a Prepare Q.C. from the leader.
prepareQC: QuorumCertificate[A],
prepareQC: QuorumCertificate[A, Phase.Prepare],
// Locked QC, for which a replica voted Commit, because it received a Pre-Commit Q.C. from leader.
lockedQC: QuorumCertificate[A],
lockedQC: QuorumCertificate[A, Phase.PreCommit],
// Commit QC, which a replica received in the Decide phase, and then executed the block in it.
commitQC: QuorumCertificate[A],
commitQC: QuorumCertificate[A, Phase.Commit],
// The block the federation is currently voting on.
preparedBlock: A#Block,
// Timeout for the view, so that it can be adjusted next time if necessary.
Expand Down Expand Up @@ -222,7 +222,7 @@ case class ProtocolState[A <: Agreement: Block: Signing](
SaveBlock(preparedBlock)
)
val next = moveTo(Phase.Commit).copy(
prepareQC = m.quorumCertificate
prepareQC = m.quorumCertificate.coerce[Phase.Prepare]
)
next -> effects
}
Expand All @@ -238,7 +238,7 @@ case class ProtocolState[A <: Agreement: Block: Signing](
sendVote(Phase.Commit, m.quorumCertificate.blockHash)
)
val next = moveTo(Phase.Decide).copy(
lockedQC = m.quorumCertificate
lockedQC = m.quorumCertificate.coerce[Phase.PreCommit]
)
next -> effects
}
Expand All @@ -252,12 +252,14 @@ case class ProtocolState[A <: Agreement: Block: Signing](
handleQuorum(e, Phase.Commit) { m =>
handleNextView(NextView(viewNumber)) match {
case (next, effects) =>
val commitQC = m.quorumCertificate.coerce[Phase.Commit]

val withExec = ExecuteBlocks(
lastExecutedBlockHash,
m.quorumCertificate
commitQC
) +: effects

val withLast = next.copy(commitQC = m.quorumCertificate)
val withLast = next.copy(commitQC = commitQC)

withLast -> withExec
}
Expand Down Expand Up @@ -417,7 +419,10 @@ case class ProtocolState[A <: Agreement: Block: Signing](
* which means each leader is expected to create max 1 block
* on top of the previous high Q.C.
*/
private def isExtension(block: A#Block, qc: QuorumCertificate[A]): Boolean =
private def isExtension(
block: A#Block,
qc: QuorumCertificate[A, _]
): Boolean =
qc.blockHash == Block[A].parentBlockHash(block)

/** Register a new vote; if there are enough to form a new Q.C.,
Expand Down
Expand Up @@ -2,13 +2,49 @@ package io.iohk.metronome.hotstuff.consensus.basic

import io.iohk.metronome.crypto.GroupSignature
import io.iohk.metronome.hotstuff.consensus.ViewNumber
import scala.reflect.ClassTag

/** A Quorum Certifcate (QC) over a tuple (message-type, view-number, block-hash) is a data type
* that combines a collection of signatures for the same tuple signed by (n − f) replicas.
*/
case class QuorumCertificate[A <: Agreement](
phase: VotingPhase,
case class QuorumCertificate[A <: Agreement, +P <: VotingPhase](
phase: P,
viewNumber: ViewNumber,
blockHash: A#Hash,
signature: GroupSignature[A#PKey, (VotingPhase, ViewNumber, A#Hash), A#GSig]
)
) {
def coerce[V <: VotingPhase](implicit
ct: ClassTag[V]
): QuorumCertificate[A, V] = {
// The following assertion is not always true in testing.
// This is a remnant of the fact that originally the `QuorumCertificate` was not generic in P,
// and tests generate invalid certificates, which the code is supposed to detect.
// We can coerce into the wrong type, but accessing the `phase` on such an instance would lead
// to a `ClassCastException`. In practice the codecs will reject such messages.

// assert(ct.unapply(phase).isDefined)
this.asInstanceOf[QuorumCertificate[A, V]]
}

protected[basic] def withPhase[V <: VotingPhase](phase: V) =
copy[A, V](phase = phase)

protected[basic] def withViewNumber(viewNumber: ViewNumber) =
copy[A, P](viewNumber = viewNumber)

protected[basic] def withBlockHash(blockHash: A#Hash) =
copy[A, P](blockHash = blockHash)

protected[basic] def withSignature(
signature: GroupSignature[
A#PKey,
(VotingPhase, ViewNumber, A#Hash),
A#GSig
]
) =
copy[A, P](signature = signature)

// Sometimes when we have just `QuorumCertificate[A, _]` the compiler
// can't prove that `.phase` is a `VotingPhase` and not just `$1`.
protected[basic] def votingPhase: VotingPhase = phase
}
Expand Up @@ -50,12 +50,12 @@ trait Signing[A <: Agreement] {

def validate(
federation: Federation[A#PKey],
quorumCertificate: QuorumCertificate[A]
quorumCertificate: QuorumCertificate[A, _]
): Boolean =
validate(
federation,
quorumCertificate.signature,
quorumCertificate.phase,
quorumCertificate.votingPhase,
quorumCertificate.viewNumber,
quorumCertificate.blockHash
)
Expand Down

0 comments on commit 8af2945

Please sign in to comment.