{-# LANGUAGE DataKinds #-}
{-# LANGUAGE DeriveGeneric #-}
{-# LANGUAGE FlexibleContexts #-}
{-# LANGUAGE FlexibleInstances #-}
{-# LANGUAGE MultiParamTypeClasses #-}
{-# LANGUAGE OverloadedStrings #-}
{-# LANGUAGE RankNTypes #-}
{-# LANGUAGE ScopedTypeVariables #-}
{-# LANGUAGE StandaloneDeriving #-}
{-# LANGUAGE TypeApplications #-}
{-# LANGUAGE TypeFamilies #-}
{-# LANGUAGE UndecidableInstances #-}

-- | Abstract key evolving signatures.
module Cardano.Crypto.KES.Class (
  -- * KES algorithm class
  KESAlgorithm (..),
  genKeyKES,
  updateKES,
  forgetSignKeyKES,
  Period,
  OptimizedKESAlgorithm (..),
  verifyOptimizedKES,

  -- * 'SignKeyWithPeriodKES' wrapper
  SignKeyWithPeriodKES (..),
  updateKESWithPeriod,

  -- * 'SignedKES' wrapper
  SignedKES (..),
  signedKES,
  verifySignedKES,

  -- * CBOR encoding and decoding
  encodeVerKeyKES,
  decodeVerKeyKES,
  encodeSigKES,
  decodeSigKES,
  encodeSignedKES,
  decodeSignedKES,

  -- * Encoded 'Size' expressions
  encodedVerKeyKESSizeExpr,
  encodedSignKeyKESSizeExpr,
  encodedSigKESSizeExpr,

  -- * Raw sizes
  sizeVerKeyKES,
  sizeSigKES,
  sizeSignKeyKES,
  seedSizeKES,

  -- * Unsound APIs
  UnsoundKESAlgorithm (..),
  encodeSignKeyKES,
  decodeSignKeyKES,
  rawDeserialiseSignKeyKES,
  UnsoundPureKESAlgorithm (..),
  unsoundPureSignedKES,
  encodeUnsoundPureSignKeyKES,
  decodeUnsoundPureSignKeyKES,

  -- * Utility functions

  -- These are used between multiple KES implementations. User code will
  -- most likely not need these, but they are required for recursive
  -- definitions of the SumKES algorithms, and can be expressed entirely in
  -- terms of the KES, DSIGN and Hash typeclasses, so we keep them here for
  -- convenience.
  hashPairOfVKeys,
  mungeName,
  unsoundPureSignKeyKESToSoundSignKeyKESViaSer,
)
where

import Control.Monad.Class.MonadST (MonadST)
import Control.Monad.Class.MonadThrow (MonadThrow)
import Control.Monad.Trans.Maybe (MaybeT (..), runMaybeT)
import Data.ByteString (ByteString)
import qualified Data.ByteString as BS
import Data.Kind (Type)
import Data.Proxy (Proxy (..))
import Data.Typeable (Typeable)
import GHC.Exts (Constraint)
import GHC.Generics (Generic)
import GHC.Stack
import GHC.TypeLits (ErrorMessage (..), KnownNat, Nat, TypeError, natVal)
import NoThunks.Class (NoThunks)

import Cardano.Binary (Decoder, Encoding, Size, decodeBytes, encodeBytes, withWordSize)

import Cardano.Crypto.DSIGN.Class (failSizeCheck)
import Cardano.Crypto.Hash.Class (Hash, HashAlgorithm, hashWith)
import Cardano.Crypto.Libsodium (MLockedAllocator, mlockedMalloc)
import Cardano.Crypto.Libsodium.MLockedSeed
import Cardano.Crypto.Seed
import Cardano.Crypto.Util (Empty)

class
  ( Typeable v
  , Show (VerKeyKES v)
  , Eq (VerKeyKES v)
  , Show (SigKES v)
  , Eq (SigKES v)
  , NoThunks (SigKES v)
  , NoThunks (SignKeyKES v)
  , NoThunks (VerKeyKES v)
  , KnownNat (SeedSizeKES v)
  , KnownNat (SizeVerKeyKES v)
  , KnownNat (SizeSignKeyKES v)
  , KnownNat (SizeSigKES v)
  ) =>
  KESAlgorithm v
  where
  --
  -- Key and signature types
  --
  data VerKeyKES v :: Type
  data SigKES v :: Type
  data SignKeyKES v :: Type

  type SeedSizeKES v :: Nat
  type SizeVerKeyKES v :: Nat
  type SizeSignKeyKES v :: Nat
  type SizeSigKES v :: Nat

  --
  -- Metadata and basic key operations
  --
  algorithmNameKES :: proxy v -> String

  hashVerKeyKES :: HashAlgorithm h => VerKeyKES v -> Hash h (VerKeyKES v)
  hashVerKeyKES = forall h a. HashAlgorithm h => (a -> ByteString) -> a -> Hash h a
hashWith forall v. KESAlgorithm v => VerKeyKES v -> ByteString
rawSerialiseVerKeyKES

  -- | Context required to run the KES algorithm
  --
  -- Unit by default (no context required)
  type ContextKES v :: Type

  type ContextKES v = ()

  type Signable v :: Type -> Constraint
  type Signable v = Empty

  --
  -- Core algorithm operations
  --

  -- | Full KES verification. This method checks that the signature itself
  -- checks out (as per 'verifySigKES'), and also makes sure that it matches
  -- the provided VerKey.
  verifyKES ::
    (Signable v a, HasCallStack) =>
    ContextKES v ->
    VerKeyKES v ->
    -- | The /current/ period for the key
    Period ->
    a ->
    SigKES v ->
    Either String ()

  -- | Return the total number of KES periods supported by this algorithm. The
  -- KES algorithm is assumed to support a fixed maximum number of periods, not
  -- a variable number.
  --
  -- Do note that this is the total number of /periods/ not the total number of
  -- evolutions. The difference is off-by-one. For example if there are 2
  -- periods (period 0 and 1) then there is only one evolution.
  totalPeriodsKES ::
    proxy v -> Word

  --
  -- Serialisation/(de)serialisation in fixed-size raw format
  --

  rawSerialiseVerKeyKES :: VerKeyKES v -> ByteString
  rawSerialiseSigKES :: SigKES v -> ByteString

  rawDeserialiseVerKeyKES :: ByteString -> Maybe (VerKeyKES v)
  rawDeserialiseSigKES :: ByteString -> Maybe (SigKES v)

  deriveVerKeyKES :: (MonadST m, MonadThrow m) => SignKeyKES v -> m (VerKeyKES v)

  --
  -- Core algorithm operations
  --

  signKES ::
    forall a m.
    (Signable v a, MonadST m, MonadThrow m) =>
    ContextKES v ->
    -- | The /current/ period for the key
    Period ->
    a ->
    SignKeyKES v ->
    m (SigKES v)

  updateKESWith ::
    (MonadST m, MonadThrow m) =>
    MLockedAllocator m ->
    ContextKES v ->
    SignKeyKES v ->
    -- | The /current/ period for the key, not the target period.
    Period ->
    m (Maybe (SignKeyKES v))

  genKeyKESWith ::
    (MonadST m, MonadThrow m) =>
    MLockedAllocator m ->
    MLockedSeed (SeedSizeKES v) ->
    m (SignKeyKES v)

  --
  -- Secure forgetting
  --

  -- | Forget a signing key synchronously, rather than waiting for GC. In some
  -- non-mock instances this provides a guarantee that the signing key is no
  -- longer in memory.
  --
  -- The precondition is that this key value will not be used again.
  forgetSignKeyKESWith ::
    (MonadST m, MonadThrow m) =>
    MLockedAllocator m ->
    SignKeyKES v ->
    m ()

sizeVerKeyKES :: forall v proxy. KESAlgorithm v => proxy v -> Word
sizeVerKeyKES :: forall v (proxy :: * -> *). KESAlgorithm v => proxy v -> Period
sizeVerKeyKES proxy v
_ = forall a. Num a => Integer -> a
fromInteger (forall (n :: Nat) (proxy :: Nat -> *).
KnownNat n =>
proxy n -> Integer
natVal (forall {k} (t :: k). Proxy t
Proxy @(SizeVerKeyKES v)))

sizeSigKES :: forall v proxy. KESAlgorithm v => proxy v -> Word
sizeSigKES :: forall v (proxy :: * -> *). KESAlgorithm v => proxy v -> Period
sizeSigKES proxy v
_ = forall a. Num a => Integer -> a
fromInteger (forall (n :: Nat) (proxy :: Nat -> *).
KnownNat n =>
proxy n -> Integer
natVal (forall {k} (t :: k). Proxy t
Proxy @(SizeSigKES v)))

sizeSignKeyKES :: forall v proxy. KESAlgorithm v => proxy v -> Word
sizeSignKeyKES :: forall v (proxy :: * -> *). KESAlgorithm v => proxy v -> Period
sizeSignKeyKES proxy v
_ = forall a. Num a => Integer -> a
fromInteger (forall (n :: Nat) (proxy :: Nat -> *).
KnownNat n =>
proxy n -> Integer
natVal (forall {k} (t :: k). Proxy t
Proxy @(SizeSignKeyKES v)))

-- | The upper bound on the 'Seed' size needed by 'genKeyKES'
seedSizeKES :: forall v proxy. KESAlgorithm v => proxy v -> Word
seedSizeKES :: forall v (proxy :: * -> *). KESAlgorithm v => proxy v -> Period
seedSizeKES proxy v
_ = forall a. Num a => Integer -> a
fromInteger (forall (n :: Nat) (proxy :: Nat -> *).
KnownNat n =>
proxy n -> Integer
natVal (forall {k} (t :: k). Proxy t
Proxy @(SeedSizeKES v)))

-- | Forget a signing key synchronously, rather than waiting for GC. In some
-- non-mock instances this provides a guarantee that the signing key is no
-- longer in memory.
--
-- The precondition is that this key value will not be used again.
forgetSignKeyKES ::
  (KESAlgorithm v, MonadST m, MonadThrow m) =>
  SignKeyKES v ->
  m ()
forgetSignKeyKES :: forall v (m :: * -> *).
(KESAlgorithm v, MonadST m, MonadThrow m) =>
SignKeyKES v -> m ()
forgetSignKeyKES = forall v (m :: * -> *).
(KESAlgorithm v, MonadST m, MonadThrow m) =>
MLockedAllocator m -> SignKeyKES v -> m ()
forgetSignKeyKESWith forall (m :: * -> *). MonadST m => MLockedAllocator m
mlockedMalloc

-- | Key generation
genKeyKES ::
  forall v m.
  (KESAlgorithm v, MonadST m, MonadThrow m) =>
  MLockedSeed (SeedSizeKES v) ->
  m (SignKeyKES v)
genKeyKES :: forall v (m :: * -> *).
(KESAlgorithm v, MonadST m, MonadThrow m) =>
MLockedSeed (SeedSizeKES v) -> m (SignKeyKES v)
genKeyKES = forall v (m :: * -> *).
(KESAlgorithm v, MonadST m, MonadThrow m) =>
MLockedAllocator m
-> MLockedSeed (SeedSizeKES v) -> m (SignKeyKES v)
genKeyKESWith forall (m :: * -> *). MonadST m => MLockedAllocator m
mlockedMalloc

-- | Update the KES signature key to the /next/ period, given the /current/
-- period.
--
-- It returns 'Nothing' if the cannot be evolved any further.
--
-- The precondition (to get a 'Just' result) is that the current KES period
-- of the input key is not the last period. The given period must be the
-- current KES period of the input key (not the next or target).
--
-- The postcondition is that in case a key is returned, its current KES
-- period is incremented by one compared to before.
--
-- Note that you must track the current period separately, and to skip to a
-- later period requires repeated use of this function, since it only
-- increments one period at once.
updateKES ::
  forall v m.
  (KESAlgorithm v, MonadST m, MonadThrow m) =>
  ContextKES v ->
  SignKeyKES v ->
  -- | The /current/ period for the key, not the target period.
  Period ->
  m (Maybe (SignKeyKES v))
updateKES :: forall v (m :: * -> *).
(KESAlgorithm v, MonadST m, MonadThrow m) =>
ContextKES v -> SignKeyKES v -> Period -> m (Maybe (SignKeyKES v))
updateKES = forall v (m :: * -> *).
(KESAlgorithm v, MonadST m, MonadThrow m) =>
MLockedAllocator m
-> ContextKES v
-> SignKeyKES v
-> Period
-> m (Maybe (SignKeyKES v))
updateKESWith forall (m :: * -> *). MonadST m => MLockedAllocator m
mlockedMalloc

-- | Pure implementations of the core KES operations. These are unsound, because
-- proper handling of KES secrets (seeds, sign keys) requires mlocking and
-- deterministic erasure (\"secure forgetting\"), which is not possible in pure
-- code.
-- This API is only provided for testing purposes; it must not be used to
-- generate or use real KES keys.
class KESAlgorithm v => UnsoundPureKESAlgorithm v where
  data UnsoundPureSignKeyKES v :: Type

  unsoundPureSignKES ::
    forall a.
    Signable v a =>
    ContextKES v ->
    -- | The /current/ period for the key
    Period ->
    a ->
    UnsoundPureSignKeyKES v ->
    SigKES v

  unsoundPureUpdateKES ::
    ContextKES v ->
    UnsoundPureSignKeyKES v ->
    -- | The /current/ period for the key, not the target period.
    Period ->
    Maybe (UnsoundPureSignKeyKES v)

  unsoundPureGenKeyKES ::
    Seed ->
    UnsoundPureSignKeyKES v

  unsoundPureDeriveVerKeyKES ::
    UnsoundPureSignKeyKES v ->
    VerKeyKES v

  unsoundPureSignKeyKESToSoundSignKeyKES ::
    (MonadST m, MonadThrow m) =>
    UnsoundPureSignKeyKES v ->
    m (SignKeyKES v)

  rawSerialiseUnsoundPureSignKeyKES :: UnsoundPureSignKeyKES v -> ByteString
  rawDeserialiseUnsoundPureSignKeyKES :: ByteString -> Maybe (UnsoundPureSignKeyKES v)

-- | Unsound operations on KES sign keys. These operations violate secure
-- forgetting constraints by leaking secrets to unprotected memory. Consider
-- using the 'DirectSerialise' / 'DirectDeserialise' APIs instead.
class KESAlgorithm v => UnsoundKESAlgorithm v where
  rawDeserialiseSignKeyKESWith ::
    (MonadST m, MonadThrow m) =>
    MLockedAllocator m ->
    ByteString ->
    m (Maybe (SignKeyKES v))

  rawSerialiseSignKeyKES :: (MonadST m, MonadThrow m) => SignKeyKES v -> m ByteString

rawDeserialiseSignKeyKES ::
  (UnsoundKESAlgorithm v, MonadST m, MonadThrow m) =>
  ByteString ->
  m (Maybe (SignKeyKES v))
rawDeserialiseSignKeyKES :: forall v (m :: * -> *).
(UnsoundKESAlgorithm v, MonadST m, MonadThrow m) =>
ByteString -> m (Maybe (SignKeyKES v))
rawDeserialiseSignKeyKES = forall v (m :: * -> *).
(UnsoundKESAlgorithm v, MonadST m, MonadThrow m) =>
MLockedAllocator m -> ByteString -> m (Maybe (SignKeyKES v))
rawDeserialiseSignKeyKESWith forall (m :: * -> *). MonadST m => MLockedAllocator m
mlockedMalloc

-- | Helper function for implementing 'unsoundPureSignKeyKESToSoundSignKeyKES'
-- for KES algorithms that support both 'UnsoundKESAlgorithm' and
-- 'UnsoundPureKESAlgorithm'. For such KES algorithms, unsound sign keys can be
-- marshalled to sound sign keys by serializing and then deserializing them.
unsoundPureSignKeyKESToSoundSignKeyKESViaSer ::
  (MonadST m, MonadThrow m, UnsoundKESAlgorithm k, UnsoundPureKESAlgorithm k) =>
  UnsoundPureSignKeyKES k ->
  m (SignKeyKES k)
unsoundPureSignKeyKESToSoundSignKeyKESViaSer :: forall (m :: * -> *) k.
(MonadST m, MonadThrow m, UnsoundKESAlgorithm k,
 UnsoundPureKESAlgorithm k) =>
UnsoundPureSignKeyKES k -> m (SignKeyKES k)
unsoundPureSignKeyKESToSoundSignKeyKESViaSer UnsoundPureSignKeyKES k
sk =
  forall b a. b -> (a -> b) -> Maybe a -> b
maybe (forall a. HasCallStack => String -> a
error String
"unsoundPureSignKeyKESToSoundSignKeyKES: deserialisation failure") forall (m :: * -> *) a. Monad m => a -> m a
return
    forall (m :: * -> *) a b. Monad m => (a -> m b) -> m a -> m b
=<< (forall v (m :: * -> *).
(UnsoundKESAlgorithm v, MonadST m, MonadThrow m) =>
ByteString -> m (Maybe (SignKeyKES v))
rawDeserialiseSignKeyKES forall b c a. (b -> c) -> (a -> b) -> a -> c
. forall v.
UnsoundPureKESAlgorithm v =>
UnsoundPureSignKeyKES v -> ByteString
rawSerialiseUnsoundPureSignKeyKES forall a b. (a -> b) -> a -> b
$ UnsoundPureSignKeyKES k
sk)

-- | Subclass for KES algorithms that embed a copy of the VerKey into the
-- signature itself, rather than relying on the externally supplied VerKey
-- alone. Some optimizations made in the 'Cardano.Crypto.KES.CompactSingleKES'
-- and 'Cardano.Crypto.KES.CompactSumKES' implementations require this
-- additional interface in order to avoid redundant computations.
class KESAlgorithm v => OptimizedKESAlgorithm v where
  -- | Partial verification: this method only verifies the signature itself,
  -- but it does not check it against any externally-provided VerKey. Use
  -- 'verifyKES' for full KES verification.
  verifySigKES ::
    (Signable v a, HasCallStack) =>
    ContextKES v ->
    -- | The /current/ period for the key
    Period ->
    a ->
    SigKES v ->
    Either String ()

  -- | Extract a VerKey from a SigKES. Note that a VerKey embedded in or
  -- derived from a SigKES is effectively user-supplied, so it is not enough
  -- to validate a SigKES against this VerKey (like 'verifySigKES' does); you
  -- must also compare the VerKey against an externally-provided key that you
  -- want to verify against (see 'verifyKES').
  verKeyFromSigKES ::
    ContextKES v ->
    Period ->
    SigKES v ->
    VerKeyKES v

verifyOptimizedKES ::
  (OptimizedKESAlgorithm v, Signable v a, HasCallStack) =>
  ContextKES v ->
  VerKeyKES v ->
  Period ->
  a ->
  SigKES v ->
  Either String ()
verifyOptimizedKES :: forall v a.
(OptimizedKESAlgorithm v, Signable v a, HasCallStack) =>
ContextKES v
-> VerKeyKES v -> Period -> a -> SigKES v -> Either String ()
verifyOptimizedKES ContextKES v
ctx VerKeyKES v
vk Period
t a
a SigKES v
sig = do
  forall v a.
(OptimizedKESAlgorithm v, Signable v a, HasCallStack) =>
ContextKES v -> Period -> a -> SigKES v -> Either String ()
verifySigKES ContextKES v
ctx Period
t a
a SigKES v
sig
  let vk' :: VerKeyKES v
vk' = forall v.
OptimizedKESAlgorithm v =>
ContextKES v -> Period -> SigKES v -> VerKeyKES v
verKeyFromSigKES ContextKES v
ctx Period
t SigKES v
sig
  if VerKeyKES v
vk' forall a. Eq a => a -> a -> Bool
== VerKeyKES v
vk
    then
      forall (m :: * -> *) a. Monad m => a -> m a
return ()
    else
      forall a b. a -> Either a b
Left String
"KES verification failed"

--
-- Do not provide Ord instances for keys, see #38
--

instance
  ( TypeError ('Text "Ord not supported for signing keys, use the hash instead")
  , Eq (SignKeyKES v)
  ) =>
  Ord (SignKeyKES v)
  where
  compare :: SignKeyKES v -> SignKeyKES v -> Ordering
compare = forall a. HasCallStack => String -> a
error String
"unsupported"

instance
  ( TypeError ('Text "Ord not supported for verification keys, use the hash instead")
  , KESAlgorithm v
  ) =>
  Ord (VerKeyKES v)
  where
  compare :: VerKeyKES v -> VerKeyKES v -> Ordering
compare = forall a. HasCallStack => String -> a
error String
"unsupported"

--
-- Convenient CBOR encoding/decoding
--
-- Implementations in terms of the raw (de)serialise
--

encodeVerKeyKES :: KESAlgorithm v => VerKeyKES v -> Encoding
encodeVerKeyKES :: forall v. KESAlgorithm v => VerKeyKES v -> Encoding
encodeVerKeyKES = ByteString -> Encoding
encodeBytes forall b c a. (b -> c) -> (a -> b) -> a -> c
. forall v. KESAlgorithm v => VerKeyKES v -> ByteString
rawSerialiseVerKeyKES

encodeUnsoundPureSignKeyKES :: UnsoundPureKESAlgorithm v => UnsoundPureSignKeyKES v -> Encoding
encodeUnsoundPureSignKeyKES :: forall v.
UnsoundPureKESAlgorithm v =>
UnsoundPureSignKeyKES v -> Encoding
encodeUnsoundPureSignKeyKES = ByteString -> Encoding
encodeBytes forall b c a. (b -> c) -> (a -> b) -> a -> c
. forall v.
UnsoundPureKESAlgorithm v =>
UnsoundPureSignKeyKES v -> ByteString
rawSerialiseUnsoundPureSignKeyKES

encodeSigKES :: KESAlgorithm v => SigKES v -> Encoding
encodeSigKES :: forall v. KESAlgorithm v => SigKES v -> Encoding
encodeSigKES = ByteString -> Encoding
encodeBytes forall b c a. (b -> c) -> (a -> b) -> a -> c
. forall v. KESAlgorithm v => SigKES v -> ByteString
rawSerialiseSigKES

encodeSignKeyKES ::
  forall v m.
  (UnsoundKESAlgorithm v, MonadST m, MonadThrow m) =>
  SignKeyKES v ->
  m Encoding
encodeSignKeyKES :: forall v (m :: * -> *).
(UnsoundKESAlgorithm v, MonadST m, MonadThrow m) =>
SignKeyKES v -> m Encoding
encodeSignKeyKES = forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap ByteString -> Encoding
encodeBytes forall b c a. (b -> c) -> (a -> b) -> a -> c
. forall v (m :: * -> *).
(UnsoundKESAlgorithm v, MonadST m, MonadThrow m) =>
SignKeyKES v -> m ByteString
rawSerialiseSignKeyKES

decodeVerKeyKES :: forall v s. KESAlgorithm v => Decoder s (VerKeyKES v)
decodeVerKeyKES :: forall v s. KESAlgorithm v => Decoder s (VerKeyKES v)
decodeVerKeyKES = do
  ByteString
bs <- forall s. Decoder s ByteString
decodeBytes
  case forall v. KESAlgorithm v => ByteString -> Maybe (VerKeyKES v)
rawDeserialiseVerKeyKES ByteString
bs of
    Just VerKeyKES v
vk -> forall (m :: * -> *) a. Monad m => a -> m a
return VerKeyKES v
vk
    Maybe (VerKeyKES v)
Nothing -> forall (m :: * -> *) a.
MonadFail m =>
String -> String -> ByteString -> Period -> m a
failSizeCheck String
"decodeVerKeyKES" String
"key" ByteString
bs (forall v (proxy :: * -> *). KESAlgorithm v => proxy v -> Period
sizeVerKeyKES (forall {k} (t :: k). Proxy t
Proxy :: Proxy v))
{-# INLINE decodeVerKeyKES #-}

decodeUnsoundPureSignKeyKES ::
  forall v s. UnsoundPureKESAlgorithm v => Decoder s (UnsoundPureSignKeyKES v)
decodeUnsoundPureSignKeyKES :: forall v s.
UnsoundPureKESAlgorithm v =>
Decoder s (UnsoundPureSignKeyKES v)
decodeUnsoundPureSignKeyKES = do
  ByteString
bs <- forall s. Decoder s ByteString
decodeBytes
  case forall v.
UnsoundPureKESAlgorithm v =>
ByteString -> Maybe (UnsoundPureSignKeyKES v)
rawDeserialiseUnsoundPureSignKeyKES ByteString
bs of
    Just UnsoundPureSignKeyKES v
vk -> forall (m :: * -> *) a. Monad m => a -> m a
return UnsoundPureSignKeyKES v
vk
    Maybe (UnsoundPureSignKeyKES v)
Nothing -> forall (m :: * -> *) a.
MonadFail m =>
String -> String -> ByteString -> Period -> m a
failSizeCheck String
"decodeUnsoundPureSignKeyKES" String
"key" ByteString
bs (forall v (proxy :: * -> *). KESAlgorithm v => proxy v -> Period
sizeSignKeyKES (forall {k} (t :: k). Proxy t
Proxy :: Proxy v))
{-# INLINE decodeUnsoundPureSignKeyKES #-}

decodeSigKES :: forall v s. KESAlgorithm v => Decoder s (SigKES v)
decodeSigKES :: forall v s. KESAlgorithm v => Decoder s (SigKES v)
decodeSigKES = do
  ByteString
bs <- forall s. Decoder s ByteString
decodeBytes
  case forall v. KESAlgorithm v => ByteString -> Maybe (SigKES v)
rawDeserialiseSigKES ByteString
bs of
    Just SigKES v
sig -> forall (m :: * -> *) a. Monad m => a -> m a
return SigKES v
sig
    Maybe (SigKES v)
Nothing -> forall (m :: * -> *) a.
MonadFail m =>
String -> String -> ByteString -> Period -> m a
failSizeCheck String
"decodeSigKES" String
"signature" ByteString
bs (forall v (proxy :: * -> *). KESAlgorithm v => proxy v -> Period
sizeSigKES (forall {k} (t :: k). Proxy t
Proxy :: Proxy v))
{-# INLINE decodeSigKES #-}

decodeSignKeyKES ::
  forall v s m.
  (UnsoundKESAlgorithm v, MonadST m, MonadThrow m) =>
  Decoder s (m (Maybe (SignKeyKES v)))
decodeSignKeyKES :: forall v s (m :: * -> *).
(UnsoundKESAlgorithm v, MonadST m, MonadThrow m) =>
Decoder s (m (Maybe (SignKeyKES v)))
decodeSignKeyKES = do
  ByteString
bs <- forall s. Decoder s ByteString
decodeBytes
  let expected :: Int
expected = forall a b. (Integral a, Num b) => a -> b
fromIntegral (forall v (proxy :: * -> *). KESAlgorithm v => proxy v -> Period
sizeSignKeyKES (forall {k} (t :: k). Proxy t
Proxy @v))
      actual :: Int
actual = ByteString -> Int
BS.length ByteString
bs
  if Int
actual forall a. Eq a => a -> a -> Bool
/= Int
expected
    then
      forall (m :: * -> *) a. MonadFail m => String -> m a
fail
        ( String
"decodeSignKeyKES: wrong length, expected "
            forall a. [a] -> [a] -> [a]
++ forall a. Show a => a -> String
show Int
expected
            forall a. [a] -> [a] -> [a]
++ String
" bytes but got "
            forall a. [a] -> [a] -> [a]
++ forall a. Show a => a -> String
show Int
actual
        )
    else
      forall (m :: * -> *) a. Monad m => a -> m a
return forall a b. (a -> b) -> a -> b
$ forall v (m :: * -> *).
(UnsoundKESAlgorithm v, MonadST m, MonadThrow m) =>
ByteString -> m (Maybe (SignKeyKES v))
rawDeserialiseSignKeyKES ByteString
bs

-- | The KES period. Periods are enumerated from zero.
--
-- Be careful of fencepost errors: if there are 2 periods (period 0 and 1)
-- then there is only one key evolution.
type Period = Word

newtype SignedKES v a = SignedKES {forall v a. SignedKES v a -> SigKES v
getSig :: SigKES v}
  deriving (forall a.
(forall x. a -> Rep a x) -> (forall x. Rep a x -> a) -> Generic a
forall v a x. Rep (SignedKES v a) x -> SignedKES v a
forall v a x. SignedKES v a -> Rep (SignedKES v a) x
$cto :: forall v a x. Rep (SignedKES v a) x -> SignedKES v a
$cfrom :: forall v a x. SignedKES v a -> Rep (SignedKES v a) x
Generic)

deriving instance KESAlgorithm v => Show (SignedKES v a)
deriving instance KESAlgorithm v => Eq (SignedKES v a)

instance KESAlgorithm v => NoThunks (SignedKES v a)

-- use generic instance

signedKES ::
  (KESAlgorithm v, Signable v a, MonadST m, MonadThrow m) =>
  ContextKES v ->
  Period ->
  a ->
  SignKeyKES v ->
  m (SignedKES v a)
signedKES :: forall v a (m :: * -> *).
(KESAlgorithm v, Signable v a, MonadST m, MonadThrow m) =>
ContextKES v -> Period -> a -> SignKeyKES v -> m (SignedKES v a)
signedKES ContextKES v
ctxt Period
time a
a SignKeyKES v
key = forall v a. SigKES v -> SignedKES v a
SignedKES forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> forall v a (m :: * -> *).
(KESAlgorithm v, Signable v a, MonadST m, MonadThrow m) =>
ContextKES v -> Period -> a -> SignKeyKES v -> m (SigKES v)
signKES ContextKES v
ctxt Period
time a
a SignKeyKES v
key

verifySignedKES ::
  (KESAlgorithm v, Signable v a) =>
  ContextKES v ->
  VerKeyKES v ->
  Period ->
  a ->
  SignedKES v a ->
  Either String ()
verifySignedKES :: forall v a.
(KESAlgorithm v, Signable v a) =>
ContextKES v
-> VerKeyKES v -> Period -> a -> SignedKES v a -> Either String ()
verifySignedKES ContextKES v
ctxt VerKeyKES v
vk Period
j a
a (SignedKES SigKES v
sig) = forall v a.
(KESAlgorithm v, Signable v a, HasCallStack) =>
ContextKES v
-> VerKeyKES v -> Period -> a -> SigKES v -> Either String ()
verifyKES ContextKES v
ctxt VerKeyKES v
vk Period
j a
a SigKES v
sig

unsoundPureSignedKES ::
  (UnsoundPureKESAlgorithm v, Signable v a) =>
  ContextKES v ->
  Period ->
  a ->
  UnsoundPureSignKeyKES v ->
  SignedKES v a
unsoundPureSignedKES :: forall v a.
(UnsoundPureKESAlgorithm v, Signable v a) =>
ContextKES v
-> Period -> a -> UnsoundPureSignKeyKES v -> SignedKES v a
unsoundPureSignedKES ContextKES v
ctxt Period
time a
a UnsoundPureSignKeyKES v
key = forall v a. SigKES v -> SignedKES v a
SignedKES forall a b. (a -> b) -> a -> b
$ forall v a.
(UnsoundPureKESAlgorithm v, Signable v a) =>
ContextKES v -> Period -> a -> UnsoundPureSignKeyKES v -> SigKES v
unsoundPureSignKES ContextKES v
ctxt Period
time a
a UnsoundPureSignKeyKES v
key

encodeSignedKES :: KESAlgorithm v => SignedKES v a -> Encoding
encodeSignedKES :: forall v a. KESAlgorithm v => SignedKES v a -> Encoding
encodeSignedKES (SignedKES SigKES v
s) = forall v. KESAlgorithm v => SigKES v -> Encoding
encodeSigKES SigKES v
s

decodeSignedKES :: KESAlgorithm v => Decoder s (SignedKES v a)
decodeSignedKES :: forall v s a. KESAlgorithm v => Decoder s (SignedKES v a)
decodeSignedKES = forall v a. SigKES v -> SignedKES v a
SignedKES forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> forall v s. KESAlgorithm v => Decoder s (SigKES v)
decodeSigKES
{-# INLINE decodeSignedKES #-}

-- | A sign key bundled with its associated period.
data SignKeyWithPeriodKES v
  = SignKeyWithPeriodKES
  { forall v. SignKeyWithPeriodKES v -> SignKeyKES v
skWithoutPeriodKES :: !(SignKeyKES v)
  , forall v. SignKeyWithPeriodKES v -> Period
periodKES :: !Period
  }
  deriving (forall a.
(forall x. a -> Rep a x) -> (forall x. Rep a x -> a) -> Generic a
forall v x.
Rep (SignKeyWithPeriodKES v) x -> SignKeyWithPeriodKES v
forall v x.
SignKeyWithPeriodKES v -> Rep (SignKeyWithPeriodKES v) x
$cto :: forall v x.
Rep (SignKeyWithPeriodKES v) x -> SignKeyWithPeriodKES v
$cfrom :: forall v x.
SignKeyWithPeriodKES v -> Rep (SignKeyWithPeriodKES v) x
Generic)

deriving instance (KESAlgorithm v, Eq (SignKeyKES v)) => Eq (SignKeyWithPeriodKES v)

deriving instance (KESAlgorithm v, Show (SignKeyKES v)) => Show (SignKeyWithPeriodKES v)

instance KESAlgorithm v => NoThunks (SignKeyWithPeriodKES v)

-- use generic instance

updateKESWithPeriod ::
  (KESAlgorithm v, MonadST m, MonadThrow m) =>
  ContextKES v ->
  SignKeyWithPeriodKES v ->
  m (Maybe (SignKeyWithPeriodKES v))
updateKESWithPeriod :: forall v (m :: * -> *).
(KESAlgorithm v, MonadST m, MonadThrow m) =>
ContextKES v
-> SignKeyWithPeriodKES v -> m (Maybe (SignKeyWithPeriodKES v))
updateKESWithPeriod ContextKES v
c (SignKeyWithPeriodKES SignKeyKES v
sk Period
t) = forall (m :: * -> *) a. MaybeT m a -> m (Maybe a)
runMaybeT forall a b. (a -> b) -> a -> b
$ do
  SignKeyKES v
sk' <- forall (m :: * -> *) a. m (Maybe a) -> MaybeT m a
MaybeT forall a b. (a -> b) -> a -> b
$ forall v (m :: * -> *).
(KESAlgorithm v, MonadST m, MonadThrow m) =>
ContextKES v -> SignKeyKES v -> Period -> m (Maybe (SignKeyKES v))
updateKES ContextKES v
c SignKeyKES v
sk Period
t
  forall (m :: * -> *) a. Monad m => a -> m a
return forall a b. (a -> b) -> a -> b
$ forall v. SignKeyKES v -> Period -> SignKeyWithPeriodKES v
SignKeyWithPeriodKES SignKeyKES v
sk' (forall a. Enum a => a -> a
succ Period
t)

--
-- 'Size' expressions for 'ToCBOR' instances.
--

-- | 'Size' expression for 'VerKeyKES' which is using 'sizeVerKeyKES' encoded
-- as 'Size'.
encodedVerKeyKESSizeExpr :: forall v. KESAlgorithm v => Proxy (VerKeyKES v) -> Size
encodedVerKeyKESSizeExpr :: forall v. KESAlgorithm v => Proxy (VerKeyKES v) -> Size
encodedVerKeyKESSizeExpr Proxy (VerKeyKES v)
_proxy =
  -- 'encodeBytes' envelope
  forall a b. (Integral a, Num b) => a -> b
fromIntegral ((forall s a. (Integral s, Integral a) => s -> a
withWordSize :: Word -> Integer) (forall v (proxy :: * -> *). KESAlgorithm v => proxy v -> Period
sizeVerKeyKES (forall {k} (t :: k). Proxy t
Proxy :: Proxy v)))
    -- payload
    forall a. Num a => a -> a -> a
+ forall a b. (Integral a, Num b) => a -> b
fromIntegral (forall v (proxy :: * -> *). KESAlgorithm v => proxy v -> Period
sizeVerKeyKES (forall {k} (t :: k). Proxy t
Proxy :: Proxy v))

-- | 'Size' expression for 'SignKeyKES' which is using 'sizeSignKeyKES' encoded
-- as 'Size'.
encodedSignKeyKESSizeExpr :: forall v. KESAlgorithm v => Proxy (SignKeyKES v) -> Size
encodedSignKeyKESSizeExpr :: forall v. KESAlgorithm v => Proxy (SignKeyKES v) -> Size
encodedSignKeyKESSizeExpr Proxy (SignKeyKES v)
_proxy =
  -- 'encodeBytes' envelope
  forall a b. (Integral a, Num b) => a -> b
fromIntegral ((forall s a. (Integral s, Integral a) => s -> a
withWordSize :: Word -> Integer) (forall v (proxy :: * -> *). KESAlgorithm v => proxy v -> Period
sizeSignKeyKES (forall {k} (t :: k). Proxy t
Proxy @v)))
    -- payload
    forall a. Num a => a -> a -> a
+ forall a b. (Integral a, Num b) => a -> b
fromIntegral (forall v (proxy :: * -> *). KESAlgorithm v => proxy v -> Period
sizeSignKeyKES (forall {k} (t :: k). Proxy t
Proxy :: Proxy v))

-- | 'Size' expression for 'SigKES' which is using 'sizeSigKES' encoded as
-- 'Size'.
encodedSigKESSizeExpr :: forall v. KESAlgorithm v => Proxy (SigKES v) -> Size
encodedSigKESSizeExpr :: forall v. KESAlgorithm v => Proxy (SigKES v) -> Size
encodedSigKESSizeExpr Proxy (SigKES v)
_proxy =
  -- 'encodeBytes' envelope
  forall a b. (Integral a, Num b) => a -> b
fromIntegral ((forall s a. (Integral s, Integral a) => s -> a
withWordSize :: Word -> Integer) (forall v (proxy :: * -> *). KESAlgorithm v => proxy v -> Period
sizeSigKES (forall {k} (t :: k). Proxy t
Proxy :: Proxy v)))
    -- payload
    forall a. Num a => a -> a -> a
+ forall a b. (Integral a, Num b) => a -> b
fromIntegral (forall v (proxy :: * -> *). KESAlgorithm v => proxy v -> Period
sizeSigKES (forall {k} (t :: k). Proxy t
Proxy :: Proxy v))

hashPairOfVKeys ::
  (KESAlgorithm d, HashAlgorithm h) =>
  (VerKeyKES d, VerKeyKES d) ->
  Hash h (VerKeyKES d, VerKeyKES d)
hashPairOfVKeys :: forall d h.
(KESAlgorithm d, HashAlgorithm h) =>
(VerKeyKES d, VerKeyKES d) -> Hash h (VerKeyKES d, VerKeyKES d)
hashPairOfVKeys =
  forall h a. HashAlgorithm h => (a -> ByteString) -> a -> Hash h a
hashWith forall a b. (a -> b) -> a -> b
$ \(VerKeyKES d
a, VerKeyKES d
b) ->
    forall v. KESAlgorithm v => VerKeyKES v -> ByteString
rawSerialiseVerKeyKES VerKeyKES d
a forall a. Semigroup a => a -> a -> a
<> forall v. KESAlgorithm v => VerKeyKES v -> ByteString
rawSerialiseVerKeyKES VerKeyKES d
b

mungeName :: String -> String
mungeName :: ShowS
mungeName String
basename
  | (String
name, Char
'^' : String
nstr) <- forall a. (a -> Bool) -> [a] -> ([a], [a])
span (forall a. Eq a => a -> a -> Bool
/= Char
'^') String
basename
  , [(Period
n, String
"")] <- forall a. Read a => ReadS a
reads String
nstr =
      String
name forall a. [a] -> [a] -> [a]
++ Char
'^' forall a. a -> [a] -> [a]
: forall a. Show a => a -> String
show (Period
n forall a. Num a => a -> a -> a
+ Period
1 :: Word)
  | Bool
otherwise =
      String
basename forall a. [a] -> [a] -> [a]
++ String
"_2^1"