{-# LANGUAGE DataKinds #-}
{-# LANGUAGE DeriveAnyClass #-}
{-# LANGUAGE DeriveGeneric #-}
{-# LANGUAGE FlexibleInstances #-}
{-# LANGUAGE MultiParamTypeClasses #-}
{-# LANGUAGE NamedFieldPuns #-}
{-# LANGUAGE OverloadedStrings #-}
{-# LANGUAGE RankNTypes #-}
{-# LANGUAGE ScopedTypeVariables #-}
{-# LANGUAGE StandaloneDeriving #-}
{-# LANGUAGE TypeApplications #-}
{-# LANGUAGE TypeFamilies #-}
{-# LANGUAGE TypeOperators #-}
{-# LANGUAGE UndecidableInstances #-}
{-# LANGUAGE UndecidableSuperClasses #-}
{-# LANGUAGE ViewPatterns #-}

module Ouroboros.Consensus.Protocol.Praos (
    ConsensusConfig (..)
  , Praos
  , PraosCannotForge (..)
  , PraosCrypto
  , PraosFields (..)
  , PraosIsLeader (..)
  , PraosParams (..)
  , PraosState (..)
  , PraosToSign (..)
  , PraosValidationErr (..)
  , Ticked (..)
  , forgePraosFields
  , praosCheckCanForge
    -- * For testing purposes
  , doValidateKESSignature
  , doValidateVRFSignature
  ) where

import           Cardano.Binary (FromCBOR (..), ToCBOR (..), enforceSize)
import qualified Cardano.Crypto.DSIGN as DSIGN
import qualified Cardano.Crypto.KES as KES
import           Cardano.Crypto.VRF (hashVerKeyVRF)
import qualified Cardano.Crypto.VRF as VRF
import           Cardano.Ledger.BaseTypes (ActiveSlotCoeff, Nonce, (⭒))
import qualified Cardano.Ledger.BaseTypes as SL
import qualified Cardano.Ledger.Chain as SL
import           Cardano.Ledger.Crypto (Crypto, DSIGN, KES, StandardCrypto, VRF)
import           Cardano.Ledger.Hashes (EraIndependentTxBody)
import           Cardano.Ledger.Keys (KeyHash, KeyRole (BlockIssuer),
                     VKey (VKey), coerceKeyRole, hashKey)
import qualified Cardano.Ledger.Keys as SL
import           Cardano.Ledger.PoolDistr
                     (IndividualPoolStake (IndividualPoolStake))
import qualified Cardano.Ledger.PoolDistr as SL
import           Cardano.Ledger.Slot (Duration (Duration), (+*))
import qualified Cardano.Protocol.TPraos.API as SL
import           Cardano.Protocol.TPraos.BHeader (BoundedNatural (bvValue),
                     checkLeaderNatValue, prevHashToNonce)
import           Cardano.Protocol.TPraos.OCert (KESPeriod (KESPeriod),
                     OCert (OCert), OCertSignable)
import qualified Cardano.Protocol.TPraos.OCert as OCert
import qualified Cardano.Protocol.TPraos.Rules.Prtcl as SL
import qualified Cardano.Protocol.TPraos.Rules.Tickn as SL
import           Cardano.Slotting.EpochInfo (EpochInfo, epochInfoEpoch,
                     epochInfoFirst, hoistEpochInfo)
import           Cardano.Slotting.Slot (EpochNo (EpochNo), SlotNo (SlotNo),
                     WithOrigin, unSlotNo)
import qualified Codec.CBOR.Encoding as CBOR
import           Codec.Serialise (Serialise (decode, encode))
import           Control.Exception (throw)
import           Control.Monad (unless)
import           Control.Monad.Except (Except, runExcept, throwError)
import           Data.Coerce (coerce)
import           Data.Functor.Identity (runIdentity)
import           Data.Map.Strict (Map)
import qualified Data.Map.Strict as Map
import           Data.Proxy (Proxy (Proxy))
import qualified Data.Set as Set
import           Data.Word (Word64)
import           GHC.Generics (Generic)
import           NoThunks.Class (NoThunks)
import           Numeric.Natural (Natural)
import           Ouroboros.Consensus.Block (WithOrigin (NotOrigin))
import qualified Ouroboros.Consensus.HardFork.History as History
import           Ouroboros.Consensus.Protocol.Abstract
import           Ouroboros.Consensus.Protocol.Ledger.HotKey (HotKey)
import qualified Ouroboros.Consensus.Protocol.Ledger.HotKey as HotKey
import           Ouroboros.Consensus.Protocol.Ledger.Util (isNewEpoch)
import           Ouroboros.Consensus.Protocol.Praos.Common
import           Ouroboros.Consensus.Protocol.Praos.Header (HeaderBody)
import qualified Ouroboros.Consensus.Protocol.Praos.Views as Views
import           Ouroboros.Consensus.Protocol.Praos.VRF (InputVRF, mkInputVRF,
                     vrfLeaderValue, vrfNonceValue)
import           Ouroboros.Consensus.Protocol.TPraos
                     (ConsensusConfig (TPraosConfig, tpraosEpochInfo, tpraosParams),
                     TPraos,
                     TPraosState (tpraosStateChainDepState, tpraosStateLastSlot))
import           Ouroboros.Consensus.Ticked (Ticked)
import           Ouroboros.Consensus.Util.Versioned (VersionDecoder (Decode),
                     decodeVersion, encodeVersion)

data Praos c

class
  ( Crypto c,
    DSIGN.Signable (DSIGN c) (OCertSignable c),
    DSIGN.Signable (DSIGN c) (SL.Hash c EraIndependentTxBody),
    KES.Signable (KES c) (HeaderBody c),
    VRF.Signable (VRF c) InputVRF
  ) =>
  PraosCrypto c

instance PraosCrypto StandardCrypto

{-------------------------------------------------------------------------------
  Fields required by Praos in the header
-------------------------------------------------------------------------------}

data PraosFields c toSign = PraosFields
  { forall c toSign. PraosFields c toSign -> SignedKES c toSign
praosSignature :: SL.SignedKES c toSign,
    forall c toSign. PraosFields c toSign -> toSign
praosToSign    :: toSign
  }
  deriving ((forall x. PraosFields c toSign -> Rep (PraosFields c toSign) x)
-> (forall x. Rep (PraosFields c toSign) x -> PraosFields c toSign)
-> Generic (PraosFields c toSign)
forall x. Rep (PraosFields c toSign) x -> PraosFields c toSign
forall x. PraosFields c toSign -> Rep (PraosFields c toSign) x
forall a.
(forall x. a -> Rep a x) -> (forall x. Rep a x -> a) -> Generic a
forall c toSign x.
Rep (PraosFields c toSign) x -> PraosFields c toSign
forall c toSign x.
PraosFields c toSign -> Rep (PraosFields c toSign) x
$cfrom :: forall c toSign x.
PraosFields c toSign -> Rep (PraosFields c toSign) x
from :: forall x. PraosFields c toSign -> Rep (PraosFields c toSign) x
$cto :: forall c toSign x.
Rep (PraosFields c toSign) x -> PraosFields c toSign
to :: forall x. Rep (PraosFields c toSign) x -> PraosFields c toSign
Generic)

deriving instance
  (NoThunks toSign, PraosCrypto c) =>
  NoThunks (PraosFields c toSign)

deriving instance
  (Show toSign, PraosCrypto c) =>
  Show (PraosFields c toSign)

-- | Fields arising from praos execution which must be included in
-- the block signature.
data PraosToSign c = PraosToSign
  { -- | Verification key for the issuer of this block.
    forall c. PraosToSign c -> VKey 'BlockIssuer c
praosToSignIssuerVK :: SL.VKey 'SL.BlockIssuer c,
    forall c. PraosToSign c -> VerKeyVRF c
praosToSignVrfVK    :: SL.VerKeyVRF c,
    -- | Verifiable random value. This is used both to prove the issuer is
    -- eligible to issue a block, and to contribute to the evolving nonce.
    forall c. PraosToSign c -> CertifiedVRF c InputVRF
praosToSignVrfRes   :: SL.CertifiedVRF c InputVRF,
    -- | Lightweight delegation certificate mapping the cold (DSIGN) key to
    -- the online KES key.
    forall c. PraosToSign c -> OCert c
praosToSignOCert    :: OCert.OCert c
  }
  deriving ((forall x. PraosToSign c -> Rep (PraosToSign c) x)
-> (forall x. Rep (PraosToSign c) x -> PraosToSign c)
-> Generic (PraosToSign c)
forall x. Rep (PraosToSign c) x -> PraosToSign c
forall x. PraosToSign c -> Rep (PraosToSign c) x
forall a.
(forall x. a -> Rep a x) -> (forall x. Rep a x -> a) -> Generic a
forall c x. Rep (PraosToSign c) x -> PraosToSign c
forall c x. PraosToSign c -> Rep (PraosToSign c) x
$cfrom :: forall c x. PraosToSign c -> Rep (PraosToSign c) x
from :: forall x. PraosToSign c -> Rep (PraosToSign c) x
$cto :: forall c x. Rep (PraosToSign c) x -> PraosToSign c
to :: forall x. Rep (PraosToSign c) x -> PraosToSign c
Generic)

instance PraosCrypto c => NoThunks (PraosToSign c)

deriving instance PraosCrypto c => Show (PraosToSign c)

forgePraosFields ::
  ( PraosCrypto c,
    SL.KESignable c toSign,
    Monad m
  ) =>
  HotKey c m ->
  CanBeLeader (Praos c) ->
  IsLeader (Praos c) ->
  (PraosToSign c -> toSign) ->
  m (PraosFields c toSign)
forgePraosFields :: forall c toSign (m :: * -> *).
(PraosCrypto c, KESignable c toSign, Monad m) =>
HotKey c m
-> CanBeLeader (Praos c)
-> IsLeader (Praos c)
-> (PraosToSign c -> toSign)
-> m (PraosFields c toSign)
forgePraosFields
  HotKey c m
hotKey
  PraosCanBeLeader
    { VKey 'BlockIssuer c
praosCanBeLeaderColdVerKey :: VKey 'BlockIssuer c
praosCanBeLeaderColdVerKey :: forall c. PraosCanBeLeader c -> VKey 'BlockIssuer c
praosCanBeLeaderColdVerKey,
      SignKeyVRF c
praosCanBeLeaderSignKeyVRF :: SignKeyVRF c
praosCanBeLeaderSignKeyVRF :: forall c. PraosCanBeLeader c -> SignKeyVRF c
praosCanBeLeaderSignKeyVRF,
      OCert c
praosCanBeLeaderOpCert :: OCert c
praosCanBeLeaderOpCert :: forall c. PraosCanBeLeader c -> OCert c
praosCanBeLeaderOpCert
    }
  PraosIsLeader {CertifiedVRF c InputVRF
praosIsLeaderVrfRes :: CertifiedVRF c InputVRF
praosIsLeaderVrfRes :: forall c. PraosIsLeader c -> CertifiedVRF c InputVRF
praosIsLeaderVrfRes}
  PraosToSign c -> toSign
mkToSign = do
    SignedKES (KES c) toSign
signature <- HotKey c m -> toSign -> m (SignedKES (KES c) toSign)
forall c toSign (m :: * -> *).
(KESignable c toSign, HasCallStack) =>
HotKey c m -> toSign -> m (SignedKES c toSign)
HotKey.sign HotKey c m
hotKey toSign
toSign
    PraosFields c toSign -> m (PraosFields c toSign)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return
      PraosFields
        { praosSignature :: SignedKES (KES c) toSign
praosSignature = SignedKES (KES c) toSign
signature,
          praosToSign :: toSign
praosToSign = toSign
toSign
        }
    where
      toSign :: toSign
toSign = PraosToSign c -> toSign
mkToSign PraosToSign c
signedFields

      signedFields :: PraosToSign c
signedFields =
        PraosToSign
          { praosToSignIssuerVK :: VKey 'BlockIssuer c
praosToSignIssuerVK = VKey 'BlockIssuer c
praosCanBeLeaderColdVerKey,
            praosToSignVrfVK :: VerKeyVRF c
praosToSignVrfVK = SignKeyVRF c -> VerKeyVRF c
forall v. VRFAlgorithm v => SignKeyVRF v -> VerKeyVRF v
VRF.deriveVerKeyVRF SignKeyVRF c
praosCanBeLeaderSignKeyVRF,
            praosToSignVrfRes :: CertifiedVRF c InputVRF
praosToSignVrfRes = CertifiedVRF c InputVRF
praosIsLeaderVrfRes,
            praosToSignOCert :: OCert c
praosToSignOCert = OCert c
praosCanBeLeaderOpCert
          }

{-------------------------------------------------------------------------------
  Protocol proper
-------------------------------------------------------------------------------}

-- | Praos parameters that are node independent
data PraosParams = PraosParams
  { -- | See 'Globals.slotsPerKESPeriod'.
    PraosParams -> Word64
praosSlotsPerKESPeriod             :: !Word64,
    -- | Active slots coefficient. This parameter represents the proportion
    -- of slots in which blocks should be issued. This can be interpreted as
    -- the probability that a party holding all the stake will be elected as
    -- leader for a given slot.
    PraosParams -> ActiveSlotCoeff
praosLeaderF                       :: !SL.ActiveSlotCoeff,
    -- | See 'Globals.securityParameter'.
    PraosParams -> SecurityParam
praosSecurityParam                 :: !SecurityParam,
    -- | Maximum number of KES iterations, see 'Globals.maxKESEvo'.
    PraosParams -> Word64
praosMaxKESEvo                     :: !Word64,
    -- | All blocks invalid after this protocol version, see
    -- 'Globals.maxMajorPV'.
    PraosParams -> MaxMajorProtVer
praosMaxMajorPV                    :: !MaxMajorProtVer,
    -- | The number of slots before the start of an epoch where the
    -- corresponding epoch nonce is snapshotted. This has to be at least one
    -- stability window such that the nonce is stable at the beginning of the
    -- epoch. Ouroboros Genesis requires this to be even larger, see
    -- 'SL.computeRandomnessStabilisationWindow'.
    PraosParams -> Word64
praosRandomnessStabilisationWindow :: !Word64
  }
  deriving ((forall x. PraosParams -> Rep PraosParams x)
-> (forall x. Rep PraosParams x -> PraosParams)
-> Generic PraosParams
forall x. Rep PraosParams x -> PraosParams
forall x. PraosParams -> Rep PraosParams x
forall a.
(forall x. a -> Rep a x) -> (forall x. Rep a x -> a) -> Generic a
$cfrom :: forall x. PraosParams -> Rep PraosParams x
from :: forall x. PraosParams -> Rep PraosParams x
$cto :: forall x. Rep PraosParams x -> PraosParams
to :: forall x. Rep PraosParams x -> PraosParams
Generic, Context -> PraosParams -> IO (Maybe ThunkInfo)
Proxy PraosParams -> String
(Context -> PraosParams -> IO (Maybe ThunkInfo))
-> (Context -> PraosParams -> IO (Maybe ThunkInfo))
-> (Proxy PraosParams -> String)
-> NoThunks PraosParams
forall a.
(Context -> a -> IO (Maybe ThunkInfo))
-> (Context -> a -> IO (Maybe ThunkInfo))
-> (Proxy a -> String)
-> NoThunks a
$cnoThunks :: Context -> PraosParams -> IO (Maybe ThunkInfo)
noThunks :: Context -> PraosParams -> IO (Maybe ThunkInfo)
$cwNoThunks :: Context -> PraosParams -> IO (Maybe ThunkInfo)
wNoThunks :: Context -> PraosParams -> IO (Maybe ThunkInfo)
$cshowTypeOf :: Proxy PraosParams -> String
showTypeOf :: Proxy PraosParams -> String
NoThunks)

-- | Assembled proof that the issuer has the right to issue a block in the
-- selected slot.
newtype PraosIsLeader c = PraosIsLeader
  { forall c. PraosIsLeader c -> CertifiedVRF c InputVRF
praosIsLeaderVrfRes :: SL.CertifiedVRF c InputVRF
  }
  deriving ((forall x. PraosIsLeader c -> Rep (PraosIsLeader c) x)
-> (forall x. Rep (PraosIsLeader c) x -> PraosIsLeader c)
-> Generic (PraosIsLeader c)
forall x. Rep (PraosIsLeader c) x -> PraosIsLeader c
forall x. PraosIsLeader c -> Rep (PraosIsLeader c) x
forall a.
(forall x. a -> Rep a x) -> (forall x. Rep a x -> a) -> Generic a
forall c x. Rep (PraosIsLeader c) x -> PraosIsLeader c
forall c x. PraosIsLeader c -> Rep (PraosIsLeader c) x
$cfrom :: forall c x. PraosIsLeader c -> Rep (PraosIsLeader c) x
from :: forall x. PraosIsLeader c -> Rep (PraosIsLeader c) x
$cto :: forall c x. Rep (PraosIsLeader c) x -> PraosIsLeader c
to :: forall x. Rep (PraosIsLeader c) x -> PraosIsLeader c
Generic)

instance PraosCrypto c => NoThunks (PraosIsLeader c)

-- | Static configuration
data instance ConsensusConfig (Praos c) = PraosConfig
  { forall c. ConsensusConfig (Praos c) -> PraosParams
praosParams :: !PraosParams,
    forall c.
ConsensusConfig (Praos c)
-> EpochInfo (Except PastHorizonException)
praosEpochInfo :: !(EpochInfo (Except History.PastHorizonException))
    -- it's useful for this record to be EpochInfo and one other thing,
    -- because the one other thing can then be used as the
    -- PartialConsensConfig in the HFC instance.
  }
  deriving ((forall x.
 ConsensusConfig (Praos c) -> Rep (ConsensusConfig (Praos c)) x)
-> (forall x.
    Rep (ConsensusConfig (Praos c)) x -> ConsensusConfig (Praos c))
-> Generic (ConsensusConfig (Praos c))
forall x.
Rep (ConsensusConfig (Praos c)) x -> ConsensusConfig (Praos c)
forall x.
ConsensusConfig (Praos c) -> Rep (ConsensusConfig (Praos c)) x
forall a.
(forall x. a -> Rep a x) -> (forall x. Rep a x -> a) -> Generic a
forall c x.
Rep (ConsensusConfig (Praos c)) x -> ConsensusConfig (Praos c)
forall c x.
ConsensusConfig (Praos c) -> Rep (ConsensusConfig (Praos c)) x
$cfrom :: forall c x.
ConsensusConfig (Praos c) -> Rep (ConsensusConfig (Praos c)) x
from :: forall x.
ConsensusConfig (Praos c) -> Rep (ConsensusConfig (Praos c)) x
$cto :: forall c x.
Rep (ConsensusConfig (Praos c)) x -> ConsensusConfig (Praos c)
to :: forall x.
Rep (ConsensusConfig (Praos c)) x -> ConsensusConfig (Praos c)
Generic)

instance PraosCrypto c => NoThunks (ConsensusConfig (Praos c))

type PraosValidateView c = Views.HeaderView c

{-------------------------------------------------------------------------------
  ConsensusProtocol
-------------------------------------------------------------------------------}

-- | Praos consensus state.
--
-- We track the last slot and the counters for operational certificates, as well
-- as a series of nonces which get updated in different ways over the course of
-- an epoch.
data PraosState c = PraosState
  { forall c. PraosState c -> WithOrigin SlotNo
praosStateLastSlot            :: !(WithOrigin SlotNo),
    -- | Operation Certificate counters
    forall c. PraosState c -> Map (KeyHash 'BlockIssuer c) Word64
praosStateOCertCounters       :: !(Map (KeyHash 'BlockIssuer c) Word64),
    -- | Evolving nonce
    forall c. PraosState c -> Nonce
praosStateEvolvingNonce       :: !Nonce,
    -- | Candidate nonce
    forall c. PraosState c -> Nonce
praosStateCandidateNonce      :: !Nonce,
    -- | Epoch nonce
    forall c. PraosState c -> Nonce
praosStateEpochNonce          :: !Nonce,
    -- | Nonce constructed from the hash of the previous block
    forall c. PraosState c -> Nonce
praosStateLabNonce            :: !Nonce,
    -- | Nonce corresponding to the LAB nonce of the last block of the previous
    -- epoch
    forall c. PraosState c -> Nonce
praosStateLastEpochBlockNonce :: !Nonce
  }
  deriving ((forall x. PraosState c -> Rep (PraosState c) x)
-> (forall x. Rep (PraosState c) x -> PraosState c)
-> Generic (PraosState c)
forall x. Rep (PraosState c) x -> PraosState c
forall x. PraosState c -> Rep (PraosState c) x
forall a.
(forall x. a -> Rep a x) -> (forall x. Rep a x -> a) -> Generic a
forall c x. Rep (PraosState c) x -> PraosState c
forall c x. PraosState c -> Rep (PraosState c) x
$cfrom :: forall c x. PraosState c -> Rep (PraosState c) x
from :: forall x. PraosState c -> Rep (PraosState c) x
$cto :: forall c x. Rep (PraosState c) x -> PraosState c
to :: forall x. Rep (PraosState c) x -> PraosState c
Generic, Int -> PraosState c -> ShowS
[PraosState c] -> ShowS
PraosState c -> String
(Int -> PraosState c -> ShowS)
-> (PraosState c -> String)
-> ([PraosState c] -> ShowS)
-> Show (PraosState c)
forall c. Int -> PraosState c -> ShowS
forall c. [PraosState c] -> ShowS
forall c. PraosState c -> String
forall a.
(Int -> a -> ShowS) -> (a -> String) -> ([a] -> ShowS) -> Show a
$cshowsPrec :: forall c. Int -> PraosState c -> ShowS
showsPrec :: Int -> PraosState c -> ShowS
$cshow :: forall c. PraosState c -> String
show :: PraosState c -> String
$cshowList :: forall c. [PraosState c] -> ShowS
showList :: [PraosState c] -> ShowS
Show, PraosState c -> PraosState c -> Bool
(PraosState c -> PraosState c -> Bool)
-> (PraosState c -> PraosState c -> Bool) -> Eq (PraosState c)
forall c. PraosState c -> PraosState c -> Bool
forall a. (a -> a -> Bool) -> (a -> a -> Bool) -> Eq a
$c== :: forall c. PraosState c -> PraosState c -> Bool
== :: PraosState c -> PraosState c -> Bool
$c/= :: forall c. PraosState c -> PraosState c -> Bool
/= :: PraosState c -> PraosState c -> Bool
Eq)

instance PraosCrypto c => NoThunks (PraosState c)

instance PraosCrypto c => ToCBOR (PraosState c) where
  toCBOR :: PraosState c -> Encoding
toCBOR = PraosState c -> Encoding
forall a. Serialise a => a -> Encoding
encode

instance PraosCrypto c => FromCBOR (PraosState c) where
  fromCBOR :: forall s. Decoder s (PraosState c)
fromCBOR = Decoder s (PraosState c)
forall s. Decoder s (PraosState c)
forall a s. Serialise a => Decoder s a
decode

instance PraosCrypto c => Serialise (PraosState c) where
  encode :: PraosState c -> Encoding
encode
    PraosState
      { WithOrigin SlotNo
praosStateLastSlot :: forall c. PraosState c -> WithOrigin SlotNo
praosStateLastSlot :: WithOrigin SlotNo
praosStateLastSlot,
        Map (KeyHash 'BlockIssuer c) Word64
praosStateOCertCounters :: forall c. PraosState c -> Map (KeyHash 'BlockIssuer c) Word64
praosStateOCertCounters :: Map (KeyHash 'BlockIssuer c) Word64
praosStateOCertCounters,
        Nonce
praosStateEvolvingNonce :: forall c. PraosState c -> Nonce
praosStateEvolvingNonce :: Nonce
praosStateEvolvingNonce,
        Nonce
praosStateCandidateNonce :: forall c. PraosState c -> Nonce
praosStateCandidateNonce :: Nonce
praosStateCandidateNonce,
        Nonce
praosStateEpochNonce :: forall c. PraosState c -> Nonce
praosStateEpochNonce :: Nonce
praosStateEpochNonce,
        Nonce
praosStateLabNonce :: forall c. PraosState c -> Nonce
praosStateLabNonce :: Nonce
praosStateLabNonce,
        Nonce
praosStateLastEpochBlockNonce :: forall c. PraosState c -> Nonce
praosStateLastEpochBlockNonce :: Nonce
praosStateLastEpochBlockNonce
      } =
      VersionNumber -> Encoding -> Encoding
encodeVersion VersionNumber
0 (Encoding -> Encoding) -> Encoding -> Encoding
forall a b. (a -> b) -> a -> b
$
        [Encoding] -> Encoding
forall a. Monoid a => [a] -> a
mconcat
          [ Word -> Encoding
CBOR.encodeListLen Word
7,
            WithOrigin SlotNo -> Encoding
forall a. ToCBOR a => a -> Encoding
toCBOR WithOrigin SlotNo
praosStateLastSlot,
            Map (KeyHash 'BlockIssuer c) Word64 -> Encoding
forall a. ToCBOR a => a -> Encoding
toCBOR Map (KeyHash 'BlockIssuer c) Word64
praosStateOCertCounters,
            Nonce -> Encoding
forall a. ToCBOR a => a -> Encoding
toCBOR Nonce
praosStateEvolvingNonce,
            Nonce -> Encoding
forall a. ToCBOR a => a -> Encoding
toCBOR Nonce
praosStateCandidateNonce,
            Nonce -> Encoding
forall a. ToCBOR a => a -> Encoding
toCBOR Nonce
praosStateEpochNonce,
            Nonce -> Encoding
forall a. ToCBOR a => a -> Encoding
toCBOR Nonce
praosStateLabNonce,
            Nonce -> Encoding
forall a. ToCBOR a => a -> Encoding
toCBOR Nonce
praosStateLastEpochBlockNonce
          ]

  decode :: forall s. Decoder s (PraosState c)
decode =
    [(VersionNumber, VersionDecoder (PraosState c))]
-> forall s. Decoder s (PraosState c)
forall a.
[(VersionNumber, VersionDecoder a)] -> forall s. Decoder s a
decodeVersion
      [(VersionNumber
0, (forall s. Decoder s (PraosState c))
-> VersionDecoder (PraosState c)
forall a. (forall s. Decoder s a) -> VersionDecoder a
Decode Decoder s (PraosState c)
forall s. Decoder s (PraosState c)
decodePraosState)]
    where
      decodePraosState :: Decoder s (PraosState c)
decodePraosState = do
        Text -> Int -> Decoder s ()
forall s. Text -> Int -> Decoder s ()
enforceSize Text
"PraosState" Int
7
        WithOrigin SlotNo
-> Map (KeyHash 'BlockIssuer c) Word64
-> Nonce
-> Nonce
-> Nonce
-> Nonce
-> Nonce
-> PraosState c
forall c.
WithOrigin SlotNo
-> Map (KeyHash 'BlockIssuer c) Word64
-> Nonce
-> Nonce
-> Nonce
-> Nonce
-> Nonce
-> PraosState c
PraosState
          (WithOrigin SlotNo
 -> Map (KeyHash 'BlockIssuer c) Word64
 -> Nonce
 -> Nonce
 -> Nonce
 -> Nonce
 -> Nonce
 -> PraosState c)
-> Decoder s (WithOrigin SlotNo)
-> Decoder
     s
     (Map (KeyHash 'BlockIssuer c) Word64
      -> Nonce -> Nonce -> Nonce -> Nonce -> Nonce -> PraosState c)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Decoder s (WithOrigin SlotNo)
forall s. Decoder s (WithOrigin SlotNo)
forall a s. FromCBOR a => Decoder s a
fromCBOR
          Decoder
  s
  (Map (KeyHash 'BlockIssuer c) Word64
   -> Nonce -> Nonce -> Nonce -> Nonce -> Nonce -> PraosState c)
-> Decoder s (Map (KeyHash 'BlockIssuer c) Word64)
-> Decoder
     s (Nonce -> Nonce -> Nonce -> Nonce -> Nonce -> PraosState c)
forall a b. Decoder s (a -> b) -> Decoder s a -> Decoder s b
forall (f :: * -> *) a b. Applicative f => f (a -> b) -> f a -> f b
<*> Decoder s (Map (KeyHash 'BlockIssuer c) Word64)
forall s. Decoder s (Map (KeyHash 'BlockIssuer c) Word64)
forall a s. FromCBOR a => Decoder s a
fromCBOR
          Decoder
  s (Nonce -> Nonce -> Nonce -> Nonce -> Nonce -> PraosState c)
-> Decoder s Nonce
-> Decoder s (Nonce -> Nonce -> Nonce -> Nonce -> PraosState c)
forall a b. Decoder s (a -> b) -> Decoder s a -> Decoder s b
forall (f :: * -> *) a b. Applicative f => f (a -> b) -> f a -> f b
<*> Decoder s Nonce
forall s. Decoder s Nonce
forall a s. FromCBOR a => Decoder s a
fromCBOR
          Decoder s (Nonce -> Nonce -> Nonce -> Nonce -> PraosState c)
-> Decoder s Nonce
-> Decoder s (Nonce -> Nonce -> Nonce -> PraosState c)
forall a b. Decoder s (a -> b) -> Decoder s a -> Decoder s b
forall (f :: * -> *) a b. Applicative f => f (a -> b) -> f a -> f b
<*> Decoder s Nonce
forall s. Decoder s Nonce
forall a s. FromCBOR a => Decoder s a
fromCBOR
          Decoder s (Nonce -> Nonce -> Nonce -> PraosState c)
-> Decoder s Nonce -> Decoder s (Nonce -> Nonce -> PraosState c)
forall a b. Decoder s (a -> b) -> Decoder s a -> Decoder s b
forall (f :: * -> *) a b. Applicative f => f (a -> b) -> f a -> f b
<*> Decoder s Nonce
forall s. Decoder s Nonce
forall a s. FromCBOR a => Decoder s a
fromCBOR
          Decoder s (Nonce -> Nonce -> PraosState c)
-> Decoder s Nonce -> Decoder s (Nonce -> PraosState c)
forall a b. Decoder s (a -> b) -> Decoder s a -> Decoder s b
forall (f :: * -> *) a b. Applicative f => f (a -> b) -> f a -> f b
<*> Decoder s Nonce
forall s. Decoder s Nonce
forall a s. FromCBOR a => Decoder s a
fromCBOR
          Decoder s (Nonce -> PraosState c)
-> Decoder s Nonce -> Decoder s (PraosState c)
forall a b. Decoder s (a -> b) -> Decoder s a -> Decoder s b
forall (f :: * -> *) a b. Applicative f => f (a -> b) -> f a -> f b
<*> Decoder s Nonce
forall s. Decoder s Nonce
forall a s. FromCBOR a => Decoder s a
fromCBOR

data instance Ticked (PraosState c) = TickedPraosState
  { forall c. Ticked (PraosState c) -> PraosState c
tickedPraosStateChainDepState :: PraosState c,
    forall c. Ticked (PraosState c) -> LedgerView c
tickedPraosStateLedgerView :: Views.LedgerView c
  }

-- | Errors which we might encounter
data PraosValidationErr c
  = VRFKeyUnknown
      !(KeyHash SL.StakePool c) -- unknown VRF keyhash (not registered)
  | VRFKeyWrongVRFKey
      !(KeyHash SL.StakePool c) -- KeyHash of block issuer
      !(SL.Hash c (SL.VerKeyVRF c)) -- VRF KeyHash registered with stake pool
      !(SL.Hash c (SL.VerKeyVRF c)) -- VRF KeyHash from Header
  | VRFKeyBadProof
      !SlotNo -- Slot used for VRF calculation
      !Nonce -- Epoch nonce used for VRF calculation
      !(VRF.CertifiedVRF (VRF c) InputVRF) -- VRF calculated nonce value
  | VRFLeaderValueTooBig Natural Rational ActiveSlotCoeff
  | KESBeforeStartOCERT
      !KESPeriod -- OCert Start KES Period
      !KESPeriod -- Current KES Period
  | KESAfterEndOCERT
      !KESPeriod -- Current KES Period
      !KESPeriod -- OCert Start KES Period
      !Word64 -- Max KES Key Evolutions
  | CounterTooSmallOCERT
      !Word64 -- last KES counter used
      !Word64 -- current KES counter
  | -- | The KES counter has been incremented by more than 1
    CounterOverIncrementedOCERT
      !Word64 -- last KES counter used
      !Word64 -- current KES counter
  | InvalidSignatureOCERT
      !Word64 -- OCert counter
      !KESPeriod -- OCert KES period
      !String -- DSIGN error message
  | InvalidKesSignatureOCERT
      !Word -- current KES Period
      !Word -- KES start period
      !Word -- expected KES evolutions
      !Word64 -- max KES evolutions
      !String -- error message given by Consensus Layer
  | NoCounterForKeyHashOCERT
      !(KeyHash 'BlockIssuer c) -- stake pool key hash
  deriving ((forall x. PraosValidationErr c -> Rep (PraosValidationErr c) x)
-> (forall x. Rep (PraosValidationErr c) x -> PraosValidationErr c)
-> Generic (PraosValidationErr c)
forall x. Rep (PraosValidationErr c) x -> PraosValidationErr c
forall x. PraosValidationErr c -> Rep (PraosValidationErr c) x
forall a.
(forall x. a -> Rep a x) -> (forall x. Rep a x -> a) -> Generic a
forall c x. Rep (PraosValidationErr c) x -> PraosValidationErr c
forall c x. PraosValidationErr c -> Rep (PraosValidationErr c) x
$cfrom :: forall c x. PraosValidationErr c -> Rep (PraosValidationErr c) x
from :: forall x. PraosValidationErr c -> Rep (PraosValidationErr c) x
$cto :: forall c x. Rep (PraosValidationErr c) x -> PraosValidationErr c
to :: forall x. Rep (PraosValidationErr c) x -> PraosValidationErr c
Generic)

deriving instance PraosCrypto c => Eq (PraosValidationErr c)

deriving instance PraosCrypto c => NoThunks (PraosValidationErr c)

deriving instance PraosCrypto c => Show (PraosValidationErr c)

instance PraosCrypto c => ConsensusProtocol (Praos c) where
  type ChainDepState (Praos c) = PraosState c
  type IsLeader (Praos c) = PraosIsLeader c
  type CanBeLeader (Praos c) = PraosCanBeLeader c
  type SelectView (Praos c) = PraosChainSelectView c
  type LedgerView (Praos c) = Views.LedgerView c
  type ValidationErr (Praos c) = PraosValidationErr c
  type ValidateView (Praos c) = PraosValidateView c

  protocolSecurityParam :: ConsensusConfig (Praos c) -> SecurityParam
protocolSecurityParam = PraosParams -> SecurityParam
praosSecurityParam (PraosParams -> SecurityParam)
-> (ConsensusConfig (Praos c) -> PraosParams)
-> ConsensusConfig (Praos c)
-> SecurityParam
forall b c a. (b -> c) -> (a -> b) -> a -> c
. ConsensusConfig (Praos c) -> PraosParams
forall c. ConsensusConfig (Praos c) -> PraosParams
praosParams

  checkIsLeader :: HasCallStack =>
ConsensusConfig (Praos c)
-> CanBeLeader (Praos c)
-> SlotNo
-> Ticked (ChainDepState (Praos c))
-> Maybe (IsLeader (Praos c))
checkIsLeader
    ConsensusConfig (Praos c)
cfg
    PraosCanBeLeader
      { SignKeyVRF c
praosCanBeLeaderSignKeyVRF :: forall c. PraosCanBeLeader c -> SignKeyVRF c
praosCanBeLeaderSignKeyVRF :: SignKeyVRF c
praosCanBeLeaderSignKeyVRF,
        VKey 'BlockIssuer c
praosCanBeLeaderColdVerKey :: forall c. PraosCanBeLeader c -> VKey 'BlockIssuer c
praosCanBeLeaderColdVerKey :: VKey 'BlockIssuer c
praosCanBeLeaderColdVerKey
      }
    SlotNo
slot
    Ticked (ChainDepState (Praos c))
cs =
      if ConsensusConfig (Praos c)
-> LedgerView (Praos c)
-> KeyHash 'StakePool c
-> CertifiedVRF (VRF c) InputVRF
-> Bool
forall c.
PraosCrypto c =>
ConsensusConfig (Praos c)
-> LedgerView (Praos c)
-> KeyHash 'StakePool c
-> CertifiedVRF (VRF c) InputVRF
-> Bool
meetsLeaderThreshold ConsensusConfig (Praos c)
cfg LedgerView (Praos c)
LedgerView c
lv (KeyHash 'BlockIssuer c -> KeyHash 'StakePool c
forall (r :: KeyRole) c (r' :: KeyRole).
KeyHash r c -> KeyHash r' c
forall (a :: KeyRole -> * -> *) (r :: KeyRole) c (r' :: KeyRole).
HasKeyRole a =>
a r c -> a r' c
SL.coerceKeyRole KeyHash 'BlockIssuer c
vkhCold) CertifiedVRF (VRF c) InputVRF
rho
        then
          PraosIsLeader c -> Maybe (PraosIsLeader c)
forall a. a -> Maybe a
Just
            PraosIsLeader
              { praosIsLeaderVrfRes :: CertifiedVRF (VRF c) InputVRF
praosIsLeaderVrfRes = CertifiedVRF (VRF c) InputVRF -> CertifiedVRF (VRF c) InputVRF
forall a b. Coercible a b => a -> b
coerce CertifiedVRF (VRF c) InputVRF
rho
              }
        else Maybe (IsLeader (Praos c))
Maybe (PraosIsLeader c)
forall a. Maybe a
Nothing
      where
        chainState :: PraosState c
chainState = Ticked (PraosState c) -> PraosState c
forall c. Ticked (PraosState c) -> PraosState c
tickedPraosStateChainDepState Ticked (ChainDepState (Praos c))
Ticked (PraosState c)
cs
        lv :: LedgerView c
lv = Ticked (PraosState c) -> LedgerView c
forall c. Ticked (PraosState c) -> LedgerView c
tickedPraosStateLedgerView Ticked (ChainDepState (Praos c))
Ticked (PraosState c)
cs
        eta0 :: Nonce
eta0 = PraosState c -> Nonce
forall c. PraosState c -> Nonce
praosStateEpochNonce PraosState c
chainState
        vkhCold :: KeyHash 'BlockIssuer c
vkhCold = VKey 'BlockIssuer c -> KeyHash 'BlockIssuer c
forall c (kd :: KeyRole). Crypto c => VKey kd c -> KeyHash kd c
SL.hashKey VKey 'BlockIssuer c
praosCanBeLeaderColdVerKey
        rho' :: InputVRF
rho' = SlotNo -> Nonce -> InputVRF
mkInputVRF SlotNo
slot Nonce
eta0

        rho :: CertifiedVRF (VRF c) InputVRF
rho = ContextVRF (VRF c)
-> InputVRF -> SignKeyVRF c -> CertifiedVRF (VRF c) InputVRF
forall v a.
(VRFAlgorithm v, Signable v a) =>
ContextVRF v -> a -> SignKeyVRF v -> CertifiedVRF v a
VRF.evalCertified () InputVRF
rho' SignKeyVRF c
praosCanBeLeaderSignKeyVRF

  -- Updating the chain dependent state for Praos.
  --
  -- If we are not in a new epoch, then nothing happens. If we are in a new
  -- epoch, we do two things:
  -- - Update the epoch nonce to the combination of the candidate nonce and the
  --   nonce derived from the last block of the previous epoch.
  -- - Update the "last block of previous epoch" nonce to the nonce derived from
  --   the last applied block.
  tickChainDepState :: ConsensusConfig (Praos c)
-> LedgerView (Praos c)
-> SlotNo
-> ChainDepState (Praos c)
-> Ticked (ChainDepState (Praos c))
tickChainDepState
    PraosConfig {EpochInfo (Except PastHorizonException)
praosEpochInfo :: forall c.
ConsensusConfig (Praos c)
-> EpochInfo (Except PastHorizonException)
praosEpochInfo :: EpochInfo (Except PastHorizonException)
praosEpochInfo}
    LedgerView (Praos c)
lv
    SlotNo
slot
    ChainDepState (Praos c)
st =
      TickedPraosState
        { tickedPraosStateChainDepState :: PraosState c
tickedPraosStateChainDepState = PraosState c
st',
          tickedPraosStateLedgerView :: LedgerView c
tickedPraosStateLedgerView = LedgerView (Praos c)
LedgerView c
lv
        }
      where
        newEpoch :: Bool
newEpoch =
          EpochInfo Identity -> WithOrigin SlotNo -> SlotNo -> Bool
isNewEpoch
            (EpochInfo (Except PastHorizonException) -> EpochInfo Identity
History.toPureEpochInfo EpochInfo (Except PastHorizonException)
praosEpochInfo)
            (PraosState c -> WithOrigin SlotNo
forall c. PraosState c -> WithOrigin SlotNo
praosStateLastSlot ChainDepState (Praos c)
PraosState c
st)
            SlotNo
slot
        st' :: PraosState c
st' =
          if Bool
newEpoch
            then
              ChainDepState (Praos c)
st
                { praosStateEpochNonce =
                    praosStateCandidateNonce st
                       praosStateLastEpochBlockNonce st,
                  praosStateLastEpochBlockNonce = praosStateLabNonce st
                }
            else ChainDepState (Praos c)
PraosState c
st

  -- Validate and update the chain dependent state as a result of processing a
  -- new header.
  --
  -- This consists of:
  -- - Validate the VRF checks
  -- - Validate the KES checks
  -- - Call 'reupdateChainDepState'
  --
  updateChainDepState :: HasCallStack =>
ConsensusConfig (Praos c)
-> ValidateView (Praos c)
-> SlotNo
-> Ticked (ChainDepState (Praos c))
-> Except (ValidationErr (Praos c)) (ChainDepState (Praos c))
updateChainDepState
    cfg :: ConsensusConfig (Praos c)
cfg@( PraosConfig
            PraosParams {ActiveSlotCoeff
praosLeaderF :: PraosParams -> ActiveSlotCoeff
praosLeaderF :: ActiveSlotCoeff
praosLeaderF}
            EpochInfo (Except PastHorizonException)
_
          )
    ValidateView (Praos c)
b
    SlotNo
slot
    Ticked (ChainDepState (Praos c))
tcs = do
      -- First, we check the KES signature, which validates that the issuer is
      -- in fact who they say they are.
      ConsensusConfig (Praos c)
-> LedgerView (Praos c)
-> Map (KeyHash 'BlockIssuer c) Word64
-> HeaderView c
-> Except (PraosValidationErr c) ()
forall c.
PraosCrypto c =>
ConsensusConfig (Praos c)
-> LedgerView (Praos c)
-> Map (KeyHash 'BlockIssuer c) Word64
-> HeaderView c
-> Except (PraosValidationErr c) ()
validateKESSignature ConsensusConfig (Praos c)
cfg LedgerView (Praos c)
LedgerView c
lv (PraosState c -> Map (KeyHash 'BlockIssuer c) Word64
forall c. PraosState c -> Map (KeyHash 'BlockIssuer c) Word64
praosStateOCertCounters PraosState c
cs) ValidateView (Praos c)
HeaderView c
b
      -- Then we examing the VRF proof, which confirms that they have the
      -- right to issue in this slot.
      Nonce
-> LedgerView c
-> ActiveSlotCoeff
-> HeaderView c
-> Except (PraosValidationErr c) ()
forall c.
PraosCrypto c =>
Nonce
-> LedgerView c
-> ActiveSlotCoeff
-> HeaderView c
-> Except (PraosValidationErr c) ()
validateVRFSignature (PraosState c -> Nonce
forall c. PraosState c -> Nonce
praosStateEpochNonce PraosState c
cs) LedgerView c
lv ActiveSlotCoeff
praosLeaderF ValidateView (Praos c)
HeaderView c
b
      -- Finally, we apply the changes from this header to the chain state.
      PraosState c
-> ExceptT (PraosValidationErr c) Identity (PraosState c)
forall a. a -> ExceptT (PraosValidationErr c) Identity a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (PraosState c
 -> ExceptT (PraosValidationErr c) Identity (PraosState c))
-> PraosState c
-> ExceptT (PraosValidationErr c) Identity (PraosState c)
forall a b. (a -> b) -> a -> b
$ ConsensusConfig (Praos c)
-> ValidateView (Praos c)
-> SlotNo
-> Ticked (ChainDepState (Praos c))
-> ChainDepState (Praos c)
forall p.
(ConsensusProtocol p, HasCallStack) =>
ConsensusConfig p
-> ValidateView p
-> SlotNo
-> Ticked (ChainDepState p)
-> ChainDepState p
reupdateChainDepState ConsensusConfig (Praos c)
cfg ValidateView (Praos c)
b SlotNo
slot Ticked (ChainDepState (Praos c))
tcs
      where
        lv :: LedgerView c
lv = Ticked (PraosState c) -> LedgerView c
forall c. Ticked (PraosState c) -> LedgerView c
tickedPraosStateLedgerView Ticked (ChainDepState (Praos c))
Ticked (PraosState c)
tcs
        cs :: PraosState c
cs = Ticked (PraosState c) -> PraosState c
forall c. Ticked (PraosState c) -> PraosState c
tickedPraosStateChainDepState Ticked (ChainDepState (Praos c))
Ticked (PraosState c)
tcs

  -- Re-update the chain dependent state as a result of processing a header.
  --
  -- This consists of:
  -- - Update the last applied block hash.
  -- - Update the evolving and (potentially) candidate nonces based on the
  --   position in the epoch.
  -- - Update the operational certificate counter.
  reupdateChainDepState :: HasCallStack =>
ConsensusConfig (Praos c)
-> ValidateView (Praos c)
-> SlotNo
-> Ticked (ChainDepState (Praos c))
-> ChainDepState (Praos c)
reupdateChainDepState
    _cfg :: ConsensusConfig (Praos c)
_cfg@( PraosConfig
             PraosParams {Word64
praosRandomnessStabilisationWindow :: PraosParams -> Word64
praosRandomnessStabilisationWindow :: Word64
praosRandomnessStabilisationWindow}
             EpochInfo (Except PastHorizonException)
ei
           )
    ValidateView (Praos c)
b
    SlotNo
slot
    Ticked (ChainDepState (Praos c))
tcs =
      PraosState c
cs
        { praosStateLastSlot = NotOrigin slot,
          praosStateLabNonce = prevHashToNonce (Views.hvPrevHash b),
          praosStateEvolvingNonce = newEvolvingNonce,
          praosStateCandidateNonce =
            if slot +* Duration praosRandomnessStabilisationWindow < firstSlotNextEpoch
              then newEvolvingNonce
              else praosStateCandidateNonce cs,
          praosStateOCertCounters =
            Map.insert hk n $ praosStateOCertCounters cs
        }
      where
        epochInfoWithErr :: EpochInfo Identity
epochInfoWithErr =
          (forall a. Except PastHorizonException a -> Identity a)
-> EpochInfo (Except PastHorizonException) -> EpochInfo Identity
forall (m :: * -> *) (n :: * -> *).
(forall a. m a -> n a) -> EpochInfo m -> EpochInfo n
hoistEpochInfo
            ((PastHorizonException -> Identity a)
-> (a -> Identity a) -> Either PastHorizonException a -> Identity a
forall a c b. (a -> c) -> (b -> c) -> Either a b -> c
either PastHorizonException -> Identity a
forall a e. Exception e => e -> a
throw a -> Identity a
forall a. a -> Identity a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (Either PastHorizonException a -> Identity a)
-> (Except PastHorizonException a -> Either PastHorizonException a)
-> Except PastHorizonException a
-> Identity a
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Except PastHorizonException a -> Either PastHorizonException a
forall e a. Except e a -> Either e a
runExcept)
            EpochInfo (Except PastHorizonException)
ei
        firstSlotNextEpoch :: SlotNo
firstSlotNextEpoch = Identity SlotNo -> SlotNo
forall a. Identity a -> a
runIdentity (Identity SlotNo -> SlotNo) -> Identity SlotNo -> SlotNo
forall a b. (a -> b) -> a -> b
$ do
          EpochNo Word64
currentEpochNo <- EpochInfo Identity -> SlotNo -> Identity EpochNo
forall (m :: * -> *).
HasCallStack =>
EpochInfo m -> SlotNo -> m EpochNo
epochInfoEpoch EpochInfo Identity
epochInfoWithErr SlotNo
slot
          let nextEpoch :: EpochNo
nextEpoch = Word64 -> EpochNo
EpochNo (Word64 -> EpochNo) -> Word64 -> EpochNo
forall a b. (a -> b) -> a -> b
$ Word64
currentEpochNo Word64 -> Word64 -> Word64
forall a. Num a => a -> a -> a
+ Word64
1
          EpochInfo Identity -> EpochNo -> Identity SlotNo
forall (m :: * -> *).
HasCallStack =>
EpochInfo m -> EpochNo -> m SlotNo
epochInfoFirst EpochInfo Identity
epochInfoWithErr EpochNo
nextEpoch
        cs :: PraosState c
cs = Ticked (PraosState c) -> PraosState c
forall c. Ticked (PraosState c) -> PraosState c
tickedPraosStateChainDepState Ticked (ChainDepState (Praos c))
Ticked (PraosState c)
tcs
        eta :: Nonce
eta = Proxy c -> CertifiedVRF (VRF c) InputVRF -> Nonce
forall c (proxy :: * -> *).
Crypto c =>
proxy c -> CertifiedVRF (VRF c) InputVRF -> Nonce
vrfNonceValue (forall t. Proxy t
forall {k} (t :: k). Proxy t
Proxy @c) (CertifiedVRF (VRF c) InputVRF -> Nonce)
-> CertifiedVRF (VRF c) InputVRF -> Nonce
forall a b. (a -> b) -> a -> b
$ HeaderView c -> CertifiedVRF (VRF c) InputVRF
forall crypto.
HeaderView crypto -> CertifiedVRF (VRF crypto) InputVRF
Views.hvVrfRes ValidateView (Praos c)
HeaderView c
b
        newEvolvingNonce :: Nonce
newEvolvingNonce = PraosState c -> Nonce
forall c. PraosState c -> Nonce
praosStateEvolvingNonce PraosState c
cs Nonce -> Nonce -> Nonce
 Nonce
eta
        OCert VerKeyKES c
_ Word64
n KESPeriod
_ SignedDSIGN c (OCertSignable c)
_ = HeaderView c -> OCert c
forall crypto. HeaderView crypto -> OCert crypto
Views.hvOCert ValidateView (Praos c)
HeaderView c
b
        hk :: KeyHash 'BlockIssuer c
hk = VKey 'BlockIssuer c -> KeyHash 'BlockIssuer c
forall c (kd :: KeyRole). Crypto c => VKey kd c -> KeyHash kd c
hashKey (VKey 'BlockIssuer c -> KeyHash 'BlockIssuer c)
-> VKey 'BlockIssuer c -> KeyHash 'BlockIssuer c
forall a b. (a -> b) -> a -> b
$ HeaderView c -> VKey 'BlockIssuer c
forall crypto. HeaderView crypto -> VKey 'BlockIssuer crypto
Views.hvVK ValidateView (Praos c)
HeaderView c
b

-- | Check whether this node meets the leader threshold to issue a block.
meetsLeaderThreshold ::
  forall c.
  PraosCrypto c =>
  ConsensusConfig (Praos c) ->
  LedgerView (Praos c) ->
  SL.KeyHash 'SL.StakePool c ->
  VRF.CertifiedVRF (VRF c) InputVRF ->
  Bool
meetsLeaderThreshold :: forall c.
PraosCrypto c =>
ConsensusConfig (Praos c)
-> LedgerView (Praos c)
-> KeyHash 'StakePool c
-> CertifiedVRF (VRF c) InputVRF
-> Bool
meetsLeaderThreshold
  PraosConfig {PraosParams
praosParams :: forall c. ConsensusConfig (Praos c) -> PraosParams
praosParams :: PraosParams
praosParams}
  Views.LedgerView {PoolDistr c
lvPoolDistr :: PoolDistr c
lvPoolDistr :: forall crypto. LedgerView crypto -> PoolDistr crypto
Views.lvPoolDistr}
  KeyHash 'StakePool c
keyHash
  CertifiedVRF (VRF c) InputVRF
rho =
    BoundedNatural -> Rational -> ActiveSlotCoeff -> Bool
checkLeaderNatValue
      (Proxy c -> CertifiedVRF (VRF c) InputVRF -> BoundedNatural
forall c (proxy :: * -> *).
Crypto c =>
proxy c -> CertifiedVRF (VRF c) InputVRF -> BoundedNatural
vrfLeaderValue (forall t. Proxy t
forall {k} (t :: k). Proxy t
Proxy @c) CertifiedVRF (VRF c) InputVRF
rho)
      Rational
r
      (PraosParams -> ActiveSlotCoeff
praosLeaderF PraosParams
praosParams)
    where
      SL.PoolDistr Map (KeyHash 'StakePool c) (IndividualPoolStake c)
poolDistr CompactForm Coin
_totalActiveStake = PoolDistr c
lvPoolDistr
      r :: Rational
r =
        Rational
-> (IndividualPoolStake c -> Rational)
-> Maybe (IndividualPoolStake c)
-> Rational
forall b a. b -> (a -> b) -> Maybe a -> b
maybe Rational
0 IndividualPoolStake c -> Rational
forall c. IndividualPoolStake c -> Rational
SL.individualPoolStake (Maybe (IndividualPoolStake c) -> Rational)
-> Maybe (IndividualPoolStake c) -> Rational
forall a b. (a -> b) -> a -> b
$
          KeyHash 'StakePool c
-> Map (KeyHash 'StakePool c) (IndividualPoolStake c)
-> Maybe (IndividualPoolStake c)
forall k a. Ord k => k -> Map k a -> Maybe a
Map.lookup KeyHash 'StakePool c
keyHash Map (KeyHash 'StakePool c) (IndividualPoolStake c)
poolDistr

validateVRFSignature ::
  forall c.
  ( PraosCrypto c
  ) =>
  Nonce ->
  Views.LedgerView c ->
  ActiveSlotCoeff ->
  Views.HeaderView c ->
  Except (PraosValidationErr c) ()
validateVRFSignature :: forall c.
PraosCrypto c =>
Nonce
-> LedgerView c
-> ActiveSlotCoeff
-> HeaderView c
-> Except (PraosValidationErr c) ()
validateVRFSignature Nonce
eta0 (LedgerView c -> PoolDistr c
forall crypto. LedgerView crypto -> PoolDistr crypto
Views.lvPoolDistr -> SL.PoolDistr Map (KeyHash 'StakePool c) (IndividualPoolStake c)
pd CompactForm Coin
_) =
  Nonce
-> Map (KeyHash 'StakePool c) (IndividualPoolStake c)
-> ActiveSlotCoeff
-> HeaderView c
-> Except (PraosValidationErr c) ()
forall c.
PraosCrypto c =>
Nonce
-> Map (KeyHash 'StakePool c) (IndividualPoolStake c)
-> ActiveSlotCoeff
-> HeaderView c
-> Except (PraosValidationErr c) ()
doValidateVRFSignature Nonce
eta0 Map (KeyHash 'StakePool c) (IndividualPoolStake c)
pd

-- NOTE: this function is much easier to test than 'validateVRFSignature' because we don't need
-- to construct a 'PraosConfig' nor 'LedgerView' to test it.
doValidateVRFSignature ::
  forall c.
  PraosCrypto c =>
  Nonce ->
  Map (KeyHash SL.StakePool c) (IndividualPoolStake c) ->
  ActiveSlotCoeff ->
  Views.HeaderView c ->
  Except (PraosValidationErr c) ()
doValidateVRFSignature :: forall c.
PraosCrypto c =>
Nonce
-> Map (KeyHash 'StakePool c) (IndividualPoolStake c)
-> ActiveSlotCoeff
-> HeaderView c
-> Except (PraosValidationErr c) ()
doValidateVRFSignature Nonce
eta0 Map (KeyHash 'StakePool c) (IndividualPoolStake c)
pd ActiveSlotCoeff
f HeaderView c
b = do
  case KeyHash 'StakePool c
-> Map (KeyHash 'StakePool c) (IndividualPoolStake c)
-> Maybe (IndividualPoolStake c)
forall k a. Ord k => k -> Map k a -> Maybe a
Map.lookup KeyHash 'StakePool c
hk Map (KeyHash 'StakePool c) (IndividualPoolStake c)
pd of
    Maybe (IndividualPoolStake c)
Nothing -> PraosValidationErr c -> Except (PraosValidationErr c) ()
forall a.
PraosValidationErr c -> ExceptT (PraosValidationErr c) Identity a
forall e (m :: * -> *) a. MonadError e m => e -> m a
throwError (PraosValidationErr c -> Except (PraosValidationErr c) ())
-> PraosValidationErr c -> Except (PraosValidationErr c) ()
forall a b. (a -> b) -> a -> b
$ KeyHash 'StakePool c -> PraosValidationErr c
forall c. KeyHash 'StakePool c -> PraosValidationErr c
VRFKeyUnknown KeyHash 'StakePool c
hk
    Just (IndividualPoolStake Rational
sigma CompactForm Coin
_totalPoolStake Hash (HASH c) (VerKeyVRF (VRF c))
vrfHK) -> do
      Hash (HASH c) (VerKeyVRF (VRF c))
vrfHK Hash (HASH c) (VerKeyVRF (VRF c))
-> Hash (HASH c) (VerKeyVRF (VRF c)) -> Bool
forall a. Eq a => a -> a -> Bool
== VerKeyVRF (VRF c) -> Hash (HASH c) (VerKeyVRF (VRF c))
forall h.
HashAlgorithm h =>
VerKeyVRF (VRF c) -> Hash h (VerKeyVRF (VRF c))
forall v h.
(VRFAlgorithm v, HashAlgorithm h) =>
VerKeyVRF v -> Hash h (VerKeyVRF v)
hashVerKeyVRF VerKeyVRF (VRF c)
vrfK
        Bool -> PraosValidationErr c -> Except (PraosValidationErr c) ()
forall e. Bool -> e -> Except e ()
?! KeyHash 'StakePool c
-> Hash (HASH c) (VerKeyVRF (VRF c))
-> Hash (HASH c) (VerKeyVRF (VRF c))
-> PraosValidationErr c
forall c.
KeyHash 'StakePool c
-> Hash c (VerKeyVRF c)
-> Hash c (VerKeyVRF c)
-> PraosValidationErr c
VRFKeyWrongVRFKey KeyHash 'StakePool c
hk Hash (HASH c) (VerKeyVRF (VRF c))
vrfHK (VerKeyVRF (VRF c) -> Hash (HASH c) (VerKeyVRF (VRF c))
forall h.
HashAlgorithm h =>
VerKeyVRF (VRF c) -> Hash h (VerKeyVRF (VRF c))
forall v h.
(VRFAlgorithm v, HashAlgorithm h) =>
VerKeyVRF v -> Hash h (VerKeyVRF v)
hashVerKeyVRF VerKeyVRF (VRF c)
vrfK)
      ContextVRF (VRF c)
-> VerKeyVRF (VRF c)
-> InputVRF
-> CertifiedVRF (VRF c) InputVRF
-> Bool
forall v a.
(VRFAlgorithm v, Signable v a) =>
ContextVRF v -> VerKeyVRF v -> a -> CertifiedVRF v a -> Bool
VRF.verifyCertified
        ()
        VerKeyVRF (VRF c)
vrfK
        (SlotNo -> Nonce -> InputVRF
mkInputVRF SlotNo
slot Nonce
eta0)
        CertifiedVRF (VRF c) InputVRF
vrfCert
        Bool -> PraosValidationErr c -> Except (PraosValidationErr c) ()
forall e. Bool -> e -> Except e ()
?! SlotNo
-> Nonce -> CertifiedVRF (VRF c) InputVRF -> PraosValidationErr c
forall c.
SlotNo
-> Nonce -> CertifiedVRF (VRF c) InputVRF -> PraosValidationErr c
VRFKeyBadProof SlotNo
slot Nonce
eta0 CertifiedVRF (VRF c) InputVRF
vrfCert
      BoundedNatural -> Rational -> ActiveSlotCoeff -> Bool
checkLeaderNatValue BoundedNatural
vrfLeaderVal Rational
sigma ActiveSlotCoeff
f
        Bool -> PraosValidationErr c -> Except (PraosValidationErr c) ()
forall e. Bool -> e -> Except e ()
?! Natural -> Rational -> ActiveSlotCoeff -> PraosValidationErr c
forall c.
Natural -> Rational -> ActiveSlotCoeff -> PraosValidationErr c
VRFLeaderValueTooBig (BoundedNatural -> Natural
bvValue BoundedNatural
vrfLeaderVal) Rational
sigma ActiveSlotCoeff
f
  where
    hk :: KeyHash 'StakePool c
hk = KeyHash 'BlockIssuer c -> KeyHash 'StakePool c
forall (r :: KeyRole) c (r' :: KeyRole).
KeyHash r c -> KeyHash r' c
forall (a :: KeyRole -> * -> *) (r :: KeyRole) c (r' :: KeyRole).
HasKeyRole a =>
a r c -> a r' c
coerceKeyRole (KeyHash 'BlockIssuer c -> KeyHash 'StakePool c)
-> (HeaderView c -> KeyHash 'BlockIssuer c)
-> HeaderView c
-> KeyHash 'StakePool c
forall b c a. (b -> c) -> (a -> b) -> a -> c
. VKey 'BlockIssuer c -> KeyHash 'BlockIssuer c
forall c (kd :: KeyRole). Crypto c => VKey kd c -> KeyHash kd c
hashKey (VKey 'BlockIssuer c -> KeyHash 'BlockIssuer c)
-> (HeaderView c -> VKey 'BlockIssuer c)
-> HeaderView c
-> KeyHash 'BlockIssuer c
forall b c a. (b -> c) -> (a -> b) -> a -> c
. HeaderView c -> VKey 'BlockIssuer c
forall crypto. HeaderView crypto -> VKey 'BlockIssuer crypto
Views.hvVK (HeaderView c -> KeyHash 'StakePool c)
-> HeaderView c -> KeyHash 'StakePool c
forall a b. (a -> b) -> a -> b
$ HeaderView c
b
    vrfK :: VerKeyVRF (VRF c)
vrfK = HeaderView c -> VerKeyVRF (VRF c)
forall crypto. HeaderView crypto -> VerKeyVRF (VRF crypto)
Views.hvVrfVK HeaderView c
b
    vrfCert :: CertifiedVRF (VRF c) InputVRF
vrfCert = HeaderView c -> CertifiedVRF (VRF c) InputVRF
forall crypto.
HeaderView crypto -> CertifiedVRF (VRF crypto) InputVRF
Views.hvVrfRes HeaderView c
b
    vrfLeaderVal :: BoundedNatural
vrfLeaderVal = Proxy c -> CertifiedVRF (VRF c) InputVRF -> BoundedNatural
forall c (proxy :: * -> *).
Crypto c =>
proxy c -> CertifiedVRF (VRF c) InputVRF -> BoundedNatural
vrfLeaderValue (forall t. Proxy t
forall {k} (t :: k). Proxy t
Proxy @c) CertifiedVRF (VRF c) InputVRF
vrfCert
    slot :: SlotNo
slot = HeaderView c -> SlotNo
forall crypto. HeaderView crypto -> SlotNo
Views.hvSlotNo HeaderView c
b

validateKESSignature ::
  PraosCrypto c =>
  ConsensusConfig (Praos c) ->
  LedgerView (Praos c) ->
  Map (KeyHash 'BlockIssuer c) Word64 ->
  Views.HeaderView c ->
  Except (PraosValidationErr c) ()
validateKESSignature :: forall c.
PraosCrypto c =>
ConsensusConfig (Praos c)
-> LedgerView (Praos c)
-> Map (KeyHash 'BlockIssuer c) Word64
-> HeaderView c
-> Except (PraosValidationErr c) ()
validateKESSignature
  _cfg :: ConsensusConfig (Praos c)
_cfg@( PraosConfig
              PraosParams{Word64
praosMaxKESEvo :: PraosParams -> Word64
praosMaxKESEvo :: Word64
praosMaxKESEvo, Word64
praosSlotsPerKESPeriod :: PraosParams -> Word64
praosSlotsPerKESPeriod :: Word64
praosSlotsPerKESPeriod}
              EpochInfo (Except PastHorizonException)
_ei
          )
  Views.LedgerView{lvPoolDistr :: forall crypto. LedgerView crypto -> PoolDistr crypto
Views.lvPoolDistr = SL.PoolDistr Map (KeyHash 'StakePool c) (IndividualPoolStake c)
lvPoolDistr CompactForm Coin
_totalActiveStake}
  Map (KeyHash 'BlockIssuer c) Word64
ocertCounters =
    Word64
-> Word64
-> Map (KeyHash 'StakePool c) (IndividualPoolStake c)
-> Map (KeyHash 'BlockIssuer c) Word64
-> HeaderView c
-> Except (PraosValidationErr c) ()
forall c.
PraosCrypto c =>
Word64
-> Word64
-> Map (KeyHash 'StakePool c) (IndividualPoolStake c)
-> Map (KeyHash 'BlockIssuer c) Word64
-> HeaderView c
-> Except (PraosValidationErr c) ()
doValidateKESSignature Word64
praosMaxKESEvo Word64
praosSlotsPerKESPeriod Map (KeyHash 'StakePool c) (IndividualPoolStake c)
lvPoolDistr Map (KeyHash 'BlockIssuer c) Word64
ocertCounters

-- NOTE: This function is much easier to test than 'validateKESSignature' because we don't need to
-- construct a 'PraosConfig' nor 'LedgerView' to test it.
doValidateKESSignature ::
  PraosCrypto c =>
  Word64 ->
  Word64 ->
  Map (KeyHash SL.StakePool c) (IndividualPoolStake c) ->
  Map (KeyHash BlockIssuer c) Word64 ->
  Views.HeaderView c ->
  Except (PraosValidationErr c) ()
doValidateKESSignature :: forall c.
PraosCrypto c =>
Word64
-> Word64
-> Map (KeyHash 'StakePool c) (IndividualPoolStake c)
-> Map (KeyHash 'BlockIssuer c) Word64
-> HeaderView c
-> Except (PraosValidationErr c) ()
doValidateKESSignature Word64
praosMaxKESEvo Word64
praosSlotsPerKESPeriod Map (KeyHash 'StakePool c) (IndividualPoolStake c)
stakeDistribution Map (KeyHash 'BlockIssuer c) Word64
ocertCounters HeaderView c
b =
  do
    KESPeriod
c0 KESPeriod -> KESPeriod -> Bool
forall a. Ord a => a -> a -> Bool
<= KESPeriod
kp Bool -> PraosValidationErr c -> Except (PraosValidationErr c) ()
forall e. Bool -> e -> Except e ()
?! KESPeriod -> KESPeriod -> PraosValidationErr c
forall c. KESPeriod -> KESPeriod -> PraosValidationErr c
KESBeforeStartOCERT KESPeriod
c0 KESPeriod
kp
    Word
kp_ Word -> Word -> Bool
forall a. Ord a => a -> a -> Bool
< Word
c0_ Word -> Word -> Word
forall a. Num a => a -> a -> a
+ Word64 -> Word
forall a b. (Integral a, Num b) => a -> b
fromIntegral Word64
praosMaxKESEvo Bool -> PraosValidationErr c -> Except (PraosValidationErr c) ()
forall e. Bool -> e -> Except e ()
?! KESPeriod -> KESPeriod -> Word64 -> PraosValidationErr c
forall c. KESPeriod -> KESPeriod -> Word64 -> PraosValidationErr c
KESAfterEndOCERT KESPeriod
kp KESPeriod
c0 Word64
praosMaxKESEvo

    let t :: Word
t = if Word
kp_ Word -> Word -> Bool
forall a. Ord a => a -> a -> Bool
>= Word
c0_ then Word
kp_ Word -> Word -> Word
forall a. Num a => a -> a -> a
- Word
c0_ else Word
0
    -- this is required to prevent an arithmetic underflow, in the case of kp_ <
    -- c0_ we get the above `KESBeforeStartOCERT` failure in the transition.

    ContextDSIGN (DSIGN c)
-> VerKeyDSIGN (DSIGN c)
-> OCertSignable c
-> SignedDSIGN (DSIGN c) (OCertSignable c)
-> Either String ()
forall v a.
(DSIGNAlgorithm v, Signable v a, HasCallStack) =>
ContextDSIGN v
-> VerKeyDSIGN v -> a -> SignedDSIGN v a -> Either String ()
DSIGN.verifySignedDSIGN () VerKeyDSIGN (DSIGN c)
vkcold (OCert c -> OCertSignable c
forall c. OCert c -> OCertSignable c
OCert.ocertToSignable OCert c
oc) SignedDSIGN (DSIGN c) (OCertSignable c)
tau Either String ()
-> (String -> PraosValidationErr c)
-> Except (PraosValidationErr c) ()
forall e1 a e2. Either e1 a -> (e1 -> e2) -> Except e2 ()
?!:
      Word64 -> KESPeriod -> String -> PraosValidationErr c
forall c. Word64 -> KESPeriod -> String -> PraosValidationErr c
InvalidSignatureOCERT Word64
n KESPeriod
c0
    ContextKES (KES c)
-> VerKeyKES (KES c)
-> Word
-> HeaderBody c
-> SignedKES (KES c) (HeaderBody c)
-> Either String ()
forall v a.
(KESAlgorithm v, Signable v a) =>
ContextKES v
-> VerKeyKES v -> Word -> a -> SignedKES v a -> Either String ()
KES.verifySignedKES () VerKeyKES (KES c)
vk_hot Word
t (HeaderView c -> HeaderBody c
forall crypto. HeaderView crypto -> HeaderBody crypto
Views.hvSigned HeaderView c
b) (HeaderView c -> SignedKES (KES c) (HeaderBody c)
forall crypto.
HeaderView crypto -> SignedKES (KES crypto) (HeaderBody crypto)
Views.hvSignature HeaderView c
b) Either String ()
-> (String -> PraosValidationErr c)
-> Except (PraosValidationErr c) ()
forall e1 a e2. Either e1 a -> (e1 -> e2) -> Except e2 ()
?!:
      Word -> Word -> Word -> Word64 -> String -> PraosValidationErr c
forall c.
Word -> Word -> Word -> Word64 -> String -> PraosValidationErr c
InvalidKesSignatureOCERT Word
kp_ Word
c0_ Word
t Word64
praosMaxKESEvo

    case Maybe Word64
currentIssueNo of
      Maybe Word64
Nothing -> do
        PraosValidationErr c -> Except (PraosValidationErr c) ()
forall a.
PraosValidationErr c -> ExceptT (PraosValidationErr c) Identity a
forall e (m :: * -> *) a. MonadError e m => e -> m a
throwError (PraosValidationErr c -> Except (PraosValidationErr c) ())
-> PraosValidationErr c -> Except (PraosValidationErr c) ()
forall a b. (a -> b) -> a -> b
$ KeyHash 'BlockIssuer c -> PraosValidationErr c
forall c. KeyHash 'BlockIssuer c -> PraosValidationErr c
NoCounterForKeyHashOCERT KeyHash 'BlockIssuer c
hk
      Just Word64
m -> do
        Word64
m Word64 -> Word64 -> Bool
forall a. Ord a => a -> a -> Bool
<= Word64
n Bool -> PraosValidationErr c -> Except (PraosValidationErr c) ()
forall e. Bool -> e -> Except e ()
?! Word64 -> Word64 -> PraosValidationErr c
forall c. Word64 -> Word64 -> PraosValidationErr c
CounterTooSmallOCERT Word64
m Word64
n
        Word64
n Word64 -> Word64 -> Bool
forall a. Ord a => a -> a -> Bool
<= Word64
m Word64 -> Word64 -> Word64
forall a. Num a => a -> a -> a
+ Word64
1 Bool -> PraosValidationErr c -> Except (PraosValidationErr c) ()
forall e. Bool -> e -> Except e ()
?! Word64 -> Word64 -> PraosValidationErr c
forall c. Word64 -> Word64 -> PraosValidationErr c
CounterOverIncrementedOCERT Word64
m Word64
n
    where
      oc :: OCert c
oc@(OCert VerKeyKES (KES c)
vk_hot Word64
n c0 :: KESPeriod
c0@(KESPeriod Word
c0_) SignedDSIGN (DSIGN c) (OCertSignable c)
tau) = HeaderView c -> OCert c
forall crypto. HeaderView crypto -> OCert crypto
Views.hvOCert HeaderView c
b
      (VKey VerKeyDSIGN (DSIGN c)
vkcold) = HeaderView c -> VKey 'BlockIssuer c
forall crypto. HeaderView crypto -> VKey 'BlockIssuer crypto
Views.hvVK HeaderView c
b
      SlotNo Word64
s = HeaderView c -> SlotNo
forall crypto. HeaderView crypto -> SlotNo
Views.hvSlotNo HeaderView c
b
      hk :: KeyHash 'BlockIssuer c
hk = VKey 'BlockIssuer c -> KeyHash 'BlockIssuer c
forall c (kd :: KeyRole). Crypto c => VKey kd c -> KeyHash kd c
hashKey (VKey 'BlockIssuer c -> KeyHash 'BlockIssuer c)
-> VKey 'BlockIssuer c -> KeyHash 'BlockIssuer c
forall a b. (a -> b) -> a -> b
$ HeaderView c -> VKey 'BlockIssuer c
forall crypto. HeaderView crypto -> VKey 'BlockIssuer crypto
Views.hvVK HeaderView c
b
      kp :: KESPeriod
kp@(KESPeriod Word
kp_) =
        if Word64
praosSlotsPerKESPeriod Word64 -> Word64 -> Bool
forall a. Eq a => a -> a -> Bool
== Word64
0
          then String -> KESPeriod
forall a. HasCallStack => String -> a
error String
"kesPeriod: slots per KES period was set to zero"
          else Word -> KESPeriod
KESPeriod (Word -> KESPeriod) -> (Word64 -> Word) -> Word64 -> KESPeriod
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Word64 -> Word
forall a b. (Integral a, Num b) => a -> b
fromIntegral (Word64 -> KESPeriod) -> Word64 -> KESPeriod
forall a b. (a -> b) -> a -> b
$ Word64
s Word64 -> Word64 -> Word64
forall a. Integral a => a -> a -> a
`div` Word64
praosSlotsPerKESPeriod

      currentIssueNo :: Maybe Word64
      currentIssueNo :: Maybe Word64
currentIssueNo
        | KeyHash 'BlockIssuer c
-> Map (KeyHash 'BlockIssuer c) Word64 -> Bool
forall k a. Ord k => k -> Map k a -> Bool
Map.member KeyHash 'BlockIssuer c
hk Map (KeyHash 'BlockIssuer c) Word64
ocertCounters = KeyHash 'BlockIssuer c
-> Map (KeyHash 'BlockIssuer c) Word64 -> Maybe Word64
forall k a. Ord k => k -> Map k a -> Maybe a
Map.lookup KeyHash 'BlockIssuer c
hk Map (KeyHash 'BlockIssuer c) Word64
ocertCounters
        | KeyHash 'StakePool c -> Set (KeyHash 'StakePool c) -> Bool
forall a. Ord a => a -> Set a -> Bool
Set.member (KeyHash 'BlockIssuer c -> KeyHash 'StakePool c
forall (r :: KeyRole) c (r' :: KeyRole).
KeyHash r c -> KeyHash r' c
forall (a :: KeyRole -> * -> *) (r :: KeyRole) c (r' :: KeyRole).
HasKeyRole a =>
a r c -> a r' c
coerceKeyRole KeyHash 'BlockIssuer c
hk) (Map (KeyHash 'StakePool c) (IndividualPoolStake c)
-> Set (KeyHash 'StakePool c)
forall k a. Map k a -> Set k
Map.keysSet Map (KeyHash 'StakePool c) (IndividualPoolStake c)
stakeDistribution) =
          Word64 -> Maybe Word64
forall a. a -> Maybe a
Just Word64
0
        | Bool
otherwise = Maybe Word64
forall a. Maybe a
Nothing

{-------------------------------------------------------------------------------
  CannotForge
-------------------------------------------------------------------------------}

-- | Expresses that, whilst we believe ourselves to be a leader for this slot,
-- we are nonetheless unable to forge a block.
data PraosCannotForge c
  = -- | The KES key in our operational certificate can't be used because the
    -- current (wall clock) period is before the start period of the key.
    -- current KES period.
    --
    -- Note: the opposite case, i.e., the wall clock period being after the
    -- end period of the key, is caught when trying to update the key in
    -- 'updateForgeState'.
    PraosCannotForgeKeyNotUsableYet
      !OCert.KESPeriod
      -- ^ Current KES period according to the wallclock slot, i.e., the KES
      -- period in which we want to use the key.
      !OCert.KESPeriod
      -- ^ Start KES period of the KES key.
  deriving ((forall x. PraosCannotForge c -> Rep (PraosCannotForge c) x)
-> (forall x. Rep (PraosCannotForge c) x -> PraosCannotForge c)
-> Generic (PraosCannotForge c)
forall x. Rep (PraosCannotForge c) x -> PraosCannotForge c
forall x. PraosCannotForge c -> Rep (PraosCannotForge c) x
forall a.
(forall x. a -> Rep a x) -> (forall x. Rep a x -> a) -> Generic a
forall c x. Rep (PraosCannotForge c) x -> PraosCannotForge c
forall c x. PraosCannotForge c -> Rep (PraosCannotForge c) x
$cfrom :: forall c x. PraosCannotForge c -> Rep (PraosCannotForge c) x
from :: forall x. PraosCannotForge c -> Rep (PraosCannotForge c) x
$cto :: forall c x. Rep (PraosCannotForge c) x -> PraosCannotForge c
to :: forall x. Rep (PraosCannotForge c) x -> PraosCannotForge c
Generic)

deriving instance PraosCrypto c => Show (PraosCannotForge c)

praosCheckCanForge ::
  ConsensusConfig (Praos c) ->
  SlotNo ->
  HotKey.KESInfo ->
  Either (PraosCannotForge c) ()
praosCheckCanForge :: forall c.
ConsensusConfig (Praos c)
-> SlotNo -> KESInfo -> Either (PraosCannotForge c) ()
praosCheckCanForge
  PraosConfig {PraosParams
praosParams :: forall c. ConsensusConfig (Praos c) -> PraosParams
praosParams :: PraosParams
praosParams}
  SlotNo
curSlot
  KESInfo
kesInfo
    | let startPeriod :: KESPeriod
startPeriod = KESInfo -> KESPeriod
HotKey.kesStartPeriod KESInfo
kesInfo,
      KESPeriod
startPeriod KESPeriod -> KESPeriod -> Bool
forall a. Ord a => a -> a -> Bool
> KESPeriod
wallclockPeriod =
      PraosCannotForge c -> Either (PraosCannotForge c) ()
forall a. PraosCannotForge c -> Either (PraosCannotForge c) a
forall e (m :: * -> *) a. MonadError e m => e -> m a
throwError (PraosCannotForge c -> Either (PraosCannotForge c) ())
-> PraosCannotForge c -> Either (PraosCannotForge c) ()
forall a b. (a -> b) -> a -> b
$ KESPeriod -> KESPeriod -> PraosCannotForge c
forall c. KESPeriod -> KESPeriod -> PraosCannotForge c
PraosCannotForgeKeyNotUsableYet KESPeriod
wallclockPeriod KESPeriod
startPeriod
    | Bool
otherwise =
      () -> Either (PraosCannotForge c) ()
forall a. a -> Either (PraosCannotForge c) a
forall (m :: * -> *) a. Monad m => a -> m a
return ()
    where
      -- The current wallclock KES period
      wallclockPeriod :: OCert.KESPeriod
      wallclockPeriod :: KESPeriod
wallclockPeriod =
        Word -> KESPeriod
OCert.KESPeriod (Word -> KESPeriod) -> Word -> KESPeriod
forall a b. (a -> b) -> a -> b
$
          Word64 -> Word
forall a b. (Integral a, Num b) => a -> b
fromIntegral (Word64 -> Word) -> Word64 -> Word
forall a b. (a -> b) -> a -> b
$
            SlotNo -> Word64
unSlotNo SlotNo
curSlot Word64 -> Word64 -> Word64
forall a. Integral a => a -> a -> a
`div` PraosParams -> Word64
praosSlotsPerKESPeriod PraosParams
praosParams


{-------------------------------------------------------------------------------
  PraosProtocolSupportsNode
-------------------------------------------------------------------------------}

instance PraosCrypto c => PraosProtocolSupportsNode (Praos c) where
  type PraosProtocolSupportsNodeCrypto (Praos c) = c

  getPraosNonces :: forall (proxy :: * -> *).
proxy (Praos c) -> ChainDepState (Praos c) -> PraosNonces
getPraosNonces proxy (Praos c)
_prx ChainDepState (Praos c)
cdst =
      PraosNonces {
          candidateNonce :: Nonce
candidateNonce   = Nonce
praosStateCandidateNonce
        , epochNonce :: Nonce
epochNonce       = Nonce
praosStateEpochNonce
        , evolvingNonce :: Nonce
evolvingNonce    = Nonce
praosStateEvolvingNonce
        , labNonce :: Nonce
labNonce         = Nonce
praosStateLabNonce
        , previousLabNonce :: Nonce
previousLabNonce = Nonce
praosStateLastEpochBlockNonce
        }
    where
      PraosState {
          Nonce
praosStateCandidateNonce :: forall c. PraosState c -> Nonce
praosStateCandidateNonce :: Nonce
praosStateCandidateNonce
        , Nonce
praosStateEpochNonce :: forall c. PraosState c -> Nonce
praosStateEpochNonce :: Nonce
praosStateEpochNonce
        , Nonce
praosStateEvolvingNonce :: forall c. PraosState c -> Nonce
praosStateEvolvingNonce :: Nonce
praosStateEvolvingNonce
        , Nonce
praosStateLabNonce :: forall c. PraosState c -> Nonce
praosStateLabNonce :: Nonce
praosStateLabNonce
        , Nonce
praosStateLastEpochBlockNonce :: forall c. PraosState c -> Nonce
praosStateLastEpochBlockNonce :: Nonce
praosStateLastEpochBlockNonce
        } = ChainDepState (Praos c)
cdst

  getOpCertCounters :: forall (proxy :: * -> *).
proxy (Praos c)
-> ChainDepState (Praos c)
-> Map
     (KeyHash 'BlockIssuer (PraosProtocolSupportsNodeCrypto (Praos c)))
     Word64
getOpCertCounters proxy (Praos c)
_prx ChainDepState (Praos c)
cdst =
      Map (KeyHash 'BlockIssuer c) Word64
Map
  (KeyHash 'BlockIssuer (PraosProtocolSupportsNodeCrypto (Praos c)))
  Word64
praosStateOCertCounters
    where
      PraosState {
          Map (KeyHash 'BlockIssuer c) Word64
praosStateOCertCounters :: forall c. PraosState c -> Map (KeyHash 'BlockIssuer c) Word64
praosStateOCertCounters :: Map (KeyHash 'BlockIssuer c) Word64
praosStateOCertCounters
        } = ChainDepState (Praos c)
cdst

{-------------------------------------------------------------------------------
  Translation from transitional Praos
-------------------------------------------------------------------------------}

-- | We can translate between TPraos and Praos, provided:
--
-- - They share the same HASH algorithm
-- - They share the same ADDRHASH algorithm
-- - They share the same DSIGN verification keys
-- - They share the same VRF verification keys
instance
  ( c1 ~ c2 ) =>
  TranslateProto (TPraos c1) (Praos c2)
  where
  translateLedgerView :: Proxy (TPraos c1, Praos c2)
-> LedgerView (TPraos c1) -> LedgerView (Praos c2)
translateLedgerView Proxy (TPraos c1, Praos c2)
_ SL.LedgerView {PoolDistr c1
lvPoolDistr :: PoolDistr c1
lvPoolDistr :: forall c. LedgerView c -> PoolDistr c
SL.lvPoolDistr, ChainChecksPParams
lvChainChecks :: ChainChecksPParams
lvChainChecks :: forall c. LedgerView c -> ChainChecksPParams
SL.lvChainChecks} =
      Views.LedgerView
        { lvPoolDistr :: PoolDistr c2
Views.lvPoolDistr = PoolDistr c1 -> PoolDistr c2
coercePoolDistr PoolDistr c1
lvPoolDistr,
          lvMaxHeaderSize :: Word16
Views.lvMaxHeaderSize = ChainChecksPParams -> Word16
SL.ccMaxBHSize ChainChecksPParams
lvChainChecks,
          lvMaxBodySize :: Word32
Views.lvMaxBodySize = ChainChecksPParams -> Word32
SL.ccMaxBBSize ChainChecksPParams
lvChainChecks,
          lvProtocolVersion :: ProtVer
Views.lvProtocolVersion = ChainChecksPParams -> ProtVer
SL.ccProtocolVersion ChainChecksPParams
lvChainChecks
        }
      where
        coercePoolDistr :: SL.PoolDistr c1 -> SL.PoolDistr c2
        coercePoolDistr :: PoolDistr c1 -> PoolDistr c2
coercePoolDistr (SL.PoolDistr Map (KeyHash 'StakePool c1) (IndividualPoolStake c1)
m CompactForm Coin
totalActiveStake) =
          Map (KeyHash 'StakePool c2) (IndividualPoolStake c2)
-> CompactForm Coin -> PoolDistr c2
forall c.
Map (KeyHash 'StakePool c) (IndividualPoolStake c)
-> CompactForm Coin -> PoolDistr c
SL.PoolDistr
            ((KeyHash 'StakePool c1 -> KeyHash 'StakePool c2)
-> Map (KeyHash 'StakePool c1) (IndividualPoolStake c2)
-> Map (KeyHash 'StakePool c2) (IndividualPoolStake c2)
forall k1 k2 a. (k1 -> k2) -> Map k1 a -> Map k2 a
Map.mapKeysMonotonic KeyHash 'StakePool c1 -> KeyHash 'StakePool c2
forall a b. Coercible a b => a -> b
coerce ((IndividualPoolStake c1 -> IndividualPoolStake c2)
-> Map (KeyHash 'StakePool c1) (IndividualPoolStake c1)
-> Map (KeyHash 'StakePool c1) (IndividualPoolStake c2)
forall a b k. (a -> b) -> Map k a -> Map k b
Map.map IndividualPoolStake c1 -> IndividualPoolStake c2
coerceIndividualPoolStake Map (KeyHash 'StakePool c1) (IndividualPoolStake c1)
m))
            CompactForm Coin
totalActiveStake
        coerceIndividualPoolStake :: SL.IndividualPoolStake c1 -> SL.IndividualPoolStake c2
        coerceIndividualPoolStake :: IndividualPoolStake c1 -> IndividualPoolStake c2
coerceIndividualPoolStake (SL.IndividualPoolStake Rational
stake CompactForm Coin
totalStake Hash c1 (VerKeyVRF c1)
vrf) =
          Rational
-> CompactForm Coin
-> Hash (HASH c2) (VerKeyVRF (VRF c2))
-> IndividualPoolStake c2
forall c.
Rational
-> CompactForm Coin
-> Hash c (VerKeyVRF c)
-> IndividualPoolStake c
SL.IndividualPoolStake Rational
stake CompactForm Coin
totalStake (Hash (HASH c2) (VerKeyVRF (VRF c2))
-> Hash (HASH c2) (VerKeyVRF (VRF c2))
forall a b. Coercible a b => a -> b
coerce Hash c1 (VerKeyVRF c1)
Hash (HASH c2) (VerKeyVRF (VRF c2))
vrf)

  translateChainDepState :: Proxy (TPraos c1, Praos c2)
-> ChainDepState (TPraos c1) -> ChainDepState (Praos c2)
translateChainDepState Proxy (TPraos c1, Praos c2)
_ ChainDepState (TPraos c1)
tpState =
    PraosState
      { praosStateLastSlot :: WithOrigin SlotNo
praosStateLastSlot = TPraosState c2 -> WithOrigin SlotNo
forall c. TPraosState c -> WithOrigin SlotNo
tpraosStateLastSlot ChainDepState (TPraos c1)
TPraosState c2
tpState,
        praosStateOCertCounters :: Map (KeyHash 'BlockIssuer c2) Word64
praosStateOCertCounters = (KeyHash 'BlockIssuer c2 -> KeyHash 'BlockIssuer c2)
-> Map (KeyHash 'BlockIssuer c2) Word64
-> Map (KeyHash 'BlockIssuer c2) Word64
forall k1 k2 a. (k1 -> k2) -> Map k1 a -> Map k2 a
Map.mapKeysMonotonic KeyHash 'BlockIssuer c2 -> KeyHash 'BlockIssuer c2
forall a b. Coercible a b => a -> b
coerce Map (KeyHash 'BlockIssuer c2) Word64
certCounters,
        praosStateEvolvingNonce :: Nonce
praosStateEvolvingNonce = Nonce
evolvingNonce,
        praosStateCandidateNonce :: Nonce
praosStateCandidateNonce = Nonce
candidateNonce,
        praosStateEpochNonce :: Nonce
praosStateEpochNonce = TicknState -> Nonce
SL.ticknStateEpochNonce TicknState
csTickn,
        praosStateLabNonce :: Nonce
praosStateLabNonce = Nonce
csLabNonce,
        praosStateLastEpochBlockNonce :: Nonce
praosStateLastEpochBlockNonce = TicknState -> Nonce
SL.ticknStatePrevHashNonce TicknState
csTickn
      }
    where
      SL.ChainDepState {PrtclState c2
csProtocol :: PrtclState c2
csProtocol :: forall c. ChainDepState c -> PrtclState c
SL.csProtocol, TicknState
csTickn :: TicknState
csTickn :: forall c. ChainDepState c -> TicknState
SL.csTickn, Nonce
csLabNonce :: Nonce
csLabNonce :: forall c. ChainDepState c -> Nonce
SL.csLabNonce} =
        TPraosState c2 -> ChainDepState c2
forall c. TPraosState c -> ChainDepState c
tpraosStateChainDepState ChainDepState (TPraos c1)
TPraosState c2
tpState
      SL.PrtclState Map (KeyHash 'BlockIssuer c2) Word64
certCounters Nonce
evolvingNonce Nonce
candidateNonce =
        PrtclState c2
csProtocol

{-------------------------------------------------------------------------------
  Util
-------------------------------------------------------------------------------}

-- | Check value and raise error if it is false.
(?!) :: Bool -> e -> Except e ()
Bool
a ?! :: forall e. Bool -> e -> Except e ()
?! e
b = Bool -> ExceptT e Identity () -> ExceptT e Identity ()
forall (f :: * -> *). Applicative f => Bool -> f () -> f ()
unless Bool
a (ExceptT e Identity () -> ExceptT e Identity ())
-> ExceptT e Identity () -> ExceptT e Identity ()
forall a b. (a -> b) -> a -> b
$ e -> ExceptT e Identity ()
forall a. e -> ExceptT e Identity a
forall e (m :: * -> *) a. MonadError e m => e -> m a
throwError e
b

infix 1 ?!

(?!:) :: Either e1 a -> (e1 -> e2) -> Except e2 ()
(Right a
_) ?!: :: forall e1 a e2. Either e1 a -> (e1 -> e2) -> Except e2 ()
?!: e1 -> e2
_ = () -> ExceptT e2 Identity ()
forall a. a -> ExceptT e2 Identity a
forall (f :: * -> *) a. Applicative f => a -> f a
pure ()
(Left e1
e1) ?!: e1 -> e2
f = e2 -> ExceptT e2 Identity ()
forall a. e2 -> ExceptT e2 Identity a
forall e (m :: * -> *) a. MonadError e m => e -> m a
throwError (e2 -> ExceptT e2 Identity ()) -> e2 -> ExceptT e2 Identity ()
forall a b. (a -> b) -> a -> b
$ e1 -> e2
f e1
e1

infix 1 ?!: