{-# LANGUAGE DeriveAnyClass        #-}
{-# LANGUAGE DeriveGeneric         #-}
{-# LANGUAGE FlexibleInstances     #-}
{-# LANGUAGE GADTs                 #-}
{-# LANGUAGE LambdaCase            #-}
{-# LANGUAGE MultiParamTypeClasses #-}
{-# LANGUAGE OverloadedStrings     #-}
{-# LANGUAGE ScopedTypeVariables   #-}
{-# LANGUAGE StandaloneDeriving    #-}
{-# LANGUAGE TypeFamilies          #-}

{-# OPTIONS_GHC -Wno-orphans #-}

module Ouroboros.Consensus.Byron.Ledger.Serialisation (
    -- * Data family instances
    NestedCtxt_ (..)
  , RawBoundaryHeader
  , RawHeader
    -- * Serialisation
  , byronBlockEncodingOverhead
  , decodeByronBlock
  , decodeByronBoundaryBlock
  , decodeByronBoundaryHeader
  , decodeByronHeaderHash
  , decodeByronRegularBlock
  , decodeByronRegularHeader
  , encodeByronBlock
  , encodeByronBoundaryHeader
  , encodeByronHeaderHash
  , encodeByronRegularHeader
    -- * Support for on-disk format
  , byronBinaryBlockInfo
    -- * Unsized header
  , addV1Envelope
  , decodeUnsizedHeader
  , dropV1Envelope
  , encodeUnsizedHeader
  , fakeByronBlockSizeHint
  ) where

import           Control.Monad.Except
import qualified Data.ByteString as Strict
import qualified Data.ByteString.Lazy as Lazy
import           Data.Word (Word32)

import qualified Codec.CBOR.Encoding as CBOR
import           Codec.Serialise (Serialise (..))

import           Cardano.Binary

import qualified Cardano.Chain.Block as CC
import qualified Cardano.Chain.Slotting as CC

import           Ouroboros.Network.DeltaQ (SizeInBytes)

import           Ouroboros.Consensus.Block

import           Ouroboros.Consensus.Storage.Common (BinaryBlockInfo (..))

import           Ouroboros.Consensus.Byron.Ledger.Block
import           Ouroboros.Consensus.Byron.Ledger.Orphans ()

{-------------------------------------------------------------------------------
  Serialise instances

  Mostly we don't depend on Serialise, but use explicit functions instead.
-------------------------------------------------------------------------------}

instance Serialise ByronHash where
  decode :: Decoder s ByronHash
decode = Decoder s ByronHash
forall s. Decoder s (HeaderHash ByronBlock)
decodeByronHeaderHash
  encode :: ByronHash -> Encoding
encode = HeaderHash ByronBlock -> Encoding
ByronHash -> Encoding
encodeByronHeaderHash

{-------------------------------------------------------------------------------
  Type synonyms
-------------------------------------------------------------------------------}

type RawBoundaryHeader = CC.ABoundaryHeader Strict.ByteString
type RawHeader         = CC.AHeader         Strict.ByteString

{-------------------------------------------------------------------------------
  Nested contents
-------------------------------------------------------------------------------}

-- | Since the Byron header does not contain the size, we include it in the
-- nested type instead.
data instance NestedCtxt_ ByronBlock f a where
  CtxtByronRegular ::
       !SizeInBytes
    -> NestedCtxt_ ByronBlock Header RawHeader

  -- | In order to reconstruct 'Header ByronBlock' we need the 'SlotNo'
  --
  -- We could compute that using 'EpochSlots', but we don't have that available
  -- here.
  CtxtByronBoundary ::
       !SizeInBytes
    -> NestedCtxt_ ByronBlock Header (SlotNo, RawBoundaryHeader)

deriving instance Show (NestedCtxt_ ByronBlock f a)

instance SameDepIndex (NestedCtxt_ ByronBlock f) where
  sameDepIndex :: NestedCtxt_ ByronBlock f a
-> NestedCtxt_ ByronBlock f b -> Maybe (a :~: b)
sameDepIndex (CtxtByronRegular size) (CtxtByronRegular size') = do
      Bool -> Maybe ()
forall (f :: * -> *). Alternative f => Bool -> f ()
guard (SizeInBytes
size SizeInBytes -> SizeInBytes -> Bool
forall a. Eq a => a -> a -> Bool
== SizeInBytes
size')
      (a :~: a) -> Maybe (a :~: a)
forall (m :: * -> *) a. Monad m => a -> m a
return a :~: a
forall k (a :: k). a :~: a
Refl
  sameDepIndex (CtxtByronBoundary size) (CtxtByronBoundary size') = do
      Bool -> Maybe ()
forall (f :: * -> *). Alternative f => Bool -> f ()
guard (SizeInBytes
size SizeInBytes -> SizeInBytes -> Bool
forall a. Eq a => a -> a -> Bool
== SizeInBytes
size')
      (a :~: a) -> Maybe (a :~: a)
forall (m :: * -> *) a. Monad m => a -> m a
return a :~: a
forall k (a :: k). a :~: a
Refl
  sameDepIndex NestedCtxt_ ByronBlock f a
_ NestedCtxt_ ByronBlock f b
_ =
      Maybe (a :~: b)
forall a. Maybe a
Nothing

instance HasNestedContent Header ByronBlock where
  unnest :: Header ByronBlock -> DepPair (NestedCtxt Header ByronBlock)
unnest Header ByronBlock
hdr = case Header ByronBlock -> ABlockOrBoundaryHdr ByteString
byronHeaderRaw Header ByronBlock
hdr of
      CC.ABOBBoundaryHdr ABoundaryHeader ByteString
h -> NestedCtxt Header ByronBlock (SlotNo, ABoundaryHeader ByteString)
-> (SlotNo, ABoundaryHeader ByteString)
-> DepPair (NestedCtxt Header ByronBlock)
forall (f :: * -> *) a. f a -> a -> DepPair f
DepPair (NestedCtxt_ ByronBlock Header (SlotNo, ABoundaryHeader ByteString)
-> NestedCtxt
     Header ByronBlock (SlotNo, ABoundaryHeader ByteString)
forall (f :: * -> *) blk a.
NestedCtxt_ blk f a -> NestedCtxt f blk a
NestedCtxt (SizeInBytes
-> NestedCtxt_
     ByronBlock Header (SlotNo, ABoundaryHeader ByteString)
CtxtByronBoundary SizeInBytes
blockSize)) (SlotNo
slotNo, ABoundaryHeader ByteString
h)
      CC.ABOBBlockHdr    AHeader ByteString
h -> NestedCtxt Header ByronBlock (AHeader ByteString)
-> AHeader ByteString -> DepPair (NestedCtxt Header ByronBlock)
forall (f :: * -> *) a. f a -> a -> DepPair f
DepPair (NestedCtxt_ ByronBlock Header (AHeader ByteString)
-> NestedCtxt Header ByronBlock (AHeader ByteString)
forall (f :: * -> *) blk a.
NestedCtxt_ blk f a -> NestedCtxt f blk a
NestedCtxt (SizeInBytes -> NestedCtxt_ ByronBlock Header (AHeader ByteString)
CtxtByronRegular  SizeInBytes
blockSize)) AHeader ByteString
h
    where
      blockSize :: SizeInBytes
blockSize = Header ByronBlock -> SizeInBytes
byronHeaderBlockSizeHint Header ByronBlock
hdr
      slotNo :: SlotNo
slotNo    = Header ByronBlock -> SlotNo
byronHeaderSlotNo        Header ByronBlock
hdr

  nest :: DepPair (NestedCtxt Header ByronBlock) -> Header ByronBlock
nest = \case
      DepPair (NestedCtxt (CtxtByronBoundary blockSize)) (slotNo, h) ->
        SlotNo
-> ABoundaryHeader ByteString -> SizeInBytes -> Header ByronBlock
mkBoundaryByronHeader SlotNo
slotNo ABoundaryHeader ByteString
h SizeInBytes
blockSize
      DepPair (NestedCtxt (CtxtByronRegular blockSize)) a
h ->
        AHeader ByteString -> SizeInBytes -> Header ByronBlock
mkRegularByronHeader a
AHeader ByteString
h SizeInBytes
blockSize

{-------------------------------------------------------------------------------
  Serialisation
-------------------------------------------------------------------------------}

-- | The Byron block encoding overhead size in bytes.
--
-- This encompasses the overhead in bytes for everything that is encoded
-- within a Byron block, excluding the actual generalized transactions
-- (transactions, delegation certificates, update votes, and update
-- proposals).
byronBlockEncodingOverhead :: Word32
byronBlockEncodingOverhead :: SizeInBytes
byronBlockEncodingOverhead =
    SizeInBytes
blockHeaderOverhead SizeInBytes -> SizeInBytes -> SizeInBytes
forall a. Num a => a -> a -> a
+ SizeInBytes
blockBodyOverhead SizeInBytes -> SizeInBytes -> SizeInBytes
forall a. Num a => a -> a -> a
+ SizeInBytes
safetyMargin
  where
    -- The maximum block header size.
    blockHeaderOverhead :: SizeInBytes
blockHeaderOverhead = SizeInBytes
650

    -- The block body overhead excluding the actual generalized transactions.
    blockBodyOverhead :: SizeInBytes
blockBodyOverhead = SizeInBytes
1 {- ABody: encodeListLen 4 -}
                      SizeInBytes -> SizeInBytes -> SizeInBytes
forall a. Num a => a -> a -> a
+ SizeInBytes
2 {- TxPayload: list -}
                      SizeInBytes -> SizeInBytes -> SizeInBytes
forall a. Num a => a -> a -> a
+ SizeInBytes
1 {- SscPayload: encodeListLen 2 -}
                      SizeInBytes -> SizeInBytes -> SizeInBytes
forall a. Num a => a -> a -> a
+ SizeInBytes
1 {- SscPayload: Word8 -}
                      SizeInBytes -> SizeInBytes -> SizeInBytes
forall a. Num a => a -> a -> a
+ SizeInBytes
1 {- SscPayload: mempty :: Set () -}
                      SizeInBytes -> SizeInBytes -> SizeInBytes
forall a. Num a => a -> a -> a
+ SizeInBytes
2 {- Delegation.Payload: list -}
                      SizeInBytes -> SizeInBytes -> SizeInBytes
forall a. Num a => a -> a -> a
+ SizeInBytes
1 {- Update.Payload: encodeListLen 2 -}
                      SizeInBytes -> SizeInBytes -> SizeInBytes
forall a. Num a => a -> a -> a
+ SizeInBytes
1 {- Update.Payload: Maybe AProposal -}
                      SizeInBytes -> SizeInBytes -> SizeInBytes
forall a. Num a => a -> a -> a
+ SizeInBytes
2 {- Update.Payload: list of AVote -}

    -- Just for safety.
    safetyMargin :: SizeInBytes
safetyMargin = SizeInBytes
1024

encodeByronHeaderHash :: HeaderHash ByronBlock -> Encoding
encodeByronHeaderHash :: HeaderHash ByronBlock -> Encoding
encodeByronHeaderHash = HeaderHash ByronBlock -> Encoding
forall a. ToCBOR a => a -> Encoding
toCBOR

decodeByronHeaderHash :: Decoder s (HeaderHash ByronBlock)
decodeByronHeaderHash :: Decoder s (HeaderHash ByronBlock)
decodeByronHeaderHash = Decoder s (HeaderHash ByronBlock)
forall a s. FromCBOR a => Decoder s a
fromCBOR

-- | Encode a block
--
-- Should be backwards compatible with legacy (cardano-sl) nodes.
--
-- Implementation note: the decoder uses 'CC.fromCBORABlockOrBoundary', which
-- has inverse 'CC.toCBORABlockOrBoundary'. This encoder is intended to be
-- binary compatible with 'CC.toCBORABlockOrBoundary', but does not use it and
-- instead takes advantage of the annotations (using 'encodePreEncoded').
encodeByronBlock :: ByronBlock -> Encoding
encodeByronBlock :: ByronBlock -> Encoding
encodeByronBlock ByronBlock
blk = [Encoding] -> Encoding
forall a. Monoid a => [a] -> a
mconcat [
      Word -> Encoding
CBOR.encodeListLen Word
2
    , case ByronBlock -> ABlockOrBoundary ByteString
byronBlockRaw ByronBlock
blk of
        CC.ABOBBoundary ABoundaryBlock ByteString
b -> [Encoding] -> Encoding
forall a. Monoid a => [a] -> a
mconcat [
            Word -> Encoding
CBOR.encodeWord Word
0
          , ByteString -> Encoding
CBOR.encodePreEncoded (ByteString -> Encoding) -> ByteString -> Encoding
forall a b. (a -> b) -> a -> b
$ ABoundaryBlock ByteString -> ByteString
forall a. ABoundaryBlock a -> a
CC.boundaryAnnotation ABoundaryBlock ByteString
b
          ]
        CC.ABOBBlock ABlock ByteString
b -> [Encoding] -> Encoding
forall a. Monoid a => [a] -> a
mconcat [
            Word -> Encoding
CBOR.encodeWord Word
1
          , ByteString -> Encoding
CBOR.encodePreEncoded (ByteString -> Encoding) -> ByteString -> Encoding
forall a b. (a -> b) -> a -> b
$ ABlock ByteString -> ByteString
forall a. ABlock a -> a
CC.blockAnnotation ABlock ByteString
b
          ]
    ]

-- | Inverse of 'encodeByronBlock'
decodeByronBlock :: CC.EpochSlots -> Decoder s (Lazy.ByteString -> ByronBlock)
decodeByronBlock :: EpochSlots -> Decoder s (ByteString -> ByronBlock)
decodeByronBlock EpochSlots
epochSlots =
    (ByteString -> ABlockOrBoundary ByteSpan -> ByronBlock)
-> ABlockOrBoundary ByteSpan -> ByteString -> ByronBlock
forall a b c. (a -> b -> c) -> b -> a -> c
flip (\ByteString
bs -> EpochSlots -> ABlockOrBoundary ByteString -> ByronBlock
mkByronBlock EpochSlots
epochSlots
               (ABlockOrBoundary ByteString -> ByronBlock)
-> (ABlockOrBoundary ByteSpan -> ABlockOrBoundary ByteString)
-> ABlockOrBoundary ByteSpan
-> ByronBlock
forall b c a. (b -> c) -> (a -> b) -> a -> c
. ByteString
-> ABlockOrBoundary ByteSpan -> ABlockOrBoundary ByteString
forall (f :: * -> *).
Functor f =>
ByteString -> f ByteSpan -> f ByteString
annotationBytes ByteString
bs)
    (ABlockOrBoundary ByteSpan -> ByteString -> ByronBlock)
-> Decoder s (ABlockOrBoundary ByteSpan)
-> Decoder s (ByteString -> ByronBlock)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> EpochSlots -> Decoder s (ABlockOrBoundary ByteSpan)
forall s. EpochSlots -> Decoder s (ABlockOrBoundary ByteSpan)
CC.fromCBORABlockOrBoundary EpochSlots
epochSlots

-- | Decoder for a regular (non-EBB) Byron block.
--
-- PRECONDITION: the 'Lazy.ByteString' given as argument to the decoder is the
-- same as the one that is decoded.
--
-- This is a wrapper for 'CC.fromCBORABlock'.
--
-- Use 'decodeByronBlock' when you can, this function is provided for use by
-- the hard-fork combinator.
decodeByronRegularBlock :: CC.EpochSlots
                        -> Decoder s (Lazy.ByteString -> ByronBlock)
decodeByronRegularBlock :: EpochSlots -> Decoder s (ByteString -> ByronBlock)
decodeByronRegularBlock EpochSlots
epochSlots =
    (ByteString -> ABlock ByteSpan -> ByronBlock)
-> ABlock ByteSpan -> ByteString -> ByronBlock
forall a b c. (a -> b -> c) -> b -> a -> c
flip (\ByteString
bs -> EpochSlots -> ABlockOrBoundary ByteString -> ByronBlock
mkByronBlock EpochSlots
epochSlots
               (ABlockOrBoundary ByteString -> ByronBlock)
-> (ABlock ByteSpan -> ABlockOrBoundary ByteString)
-> ABlock ByteSpan
-> ByronBlock
forall b c a. (b -> c) -> (a -> b) -> a -> c
. ByteString
-> ABlockOrBoundary ByteSpan -> ABlockOrBoundary ByteString
forall (f :: * -> *).
Functor f =>
ByteString -> f ByteSpan -> f ByteString
annotationBytes ByteString
bs
               (ABlockOrBoundary ByteSpan -> ABlockOrBoundary ByteString)
-> (ABlock ByteSpan -> ABlockOrBoundary ByteSpan)
-> ABlock ByteSpan
-> ABlockOrBoundary ByteString
forall b c a. (b -> c) -> (a -> b) -> a -> c
. ABlock ByteSpan -> ABlockOrBoundary ByteSpan
forall a. ABlock a -> ABlockOrBoundary a
CC.ABOBBlock)
    (ABlock ByteSpan -> ByteString -> ByronBlock)
-> Decoder s (ABlock ByteSpan)
-> Decoder s (ByteString -> ByronBlock)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> EpochSlots -> Decoder s (ABlock ByteSpan)
forall s. EpochSlots -> Decoder s (ABlock ByteSpan)
CC.fromCBORABlock EpochSlots
epochSlots

-- | Decoder for a boundary Byron block.
--
-- PRECONDITION: the 'Lazy.ByteString' given as argument to the decoder is the
-- same as the one that is decoded.
--
-- This is a wrapper for 'CC.fromCBORABoundaryBlock'.
--
-- Use 'decodeByronBlock' when you can, this function is provided for use by
-- the hard-fork combinator.
decodeByronBoundaryBlock :: CC.EpochSlots
                         -> Decoder s (Lazy.ByteString -> ByronBlock)
decodeByronBoundaryBlock :: EpochSlots -> Decoder s (ByteString -> ByronBlock)
decodeByronBoundaryBlock EpochSlots
epochSlots =
    (ByteString -> ABoundaryBlock ByteSpan -> ByronBlock)
-> ABoundaryBlock ByteSpan -> ByteString -> ByronBlock
forall a b c. (a -> b -> c) -> b -> a -> c
flip (\ByteString
bs -> EpochSlots -> ABlockOrBoundary ByteString -> ByronBlock
mkByronBlock EpochSlots
epochSlots
               (ABlockOrBoundary ByteString -> ByronBlock)
-> (ABoundaryBlock ByteSpan -> ABlockOrBoundary ByteString)
-> ABoundaryBlock ByteSpan
-> ByronBlock
forall b c a. (b -> c) -> (a -> b) -> a -> c
. ByteString
-> ABlockOrBoundary ByteSpan -> ABlockOrBoundary ByteString
forall (f :: * -> *).
Functor f =>
ByteString -> f ByteSpan -> f ByteString
annotationBytes ByteString
bs
               (ABlockOrBoundary ByteSpan -> ABlockOrBoundary ByteString)
-> (ABoundaryBlock ByteSpan -> ABlockOrBoundary ByteSpan)
-> ABoundaryBlock ByteSpan
-> ABlockOrBoundary ByteString
forall b c a. (b -> c) -> (a -> b) -> a -> c
. ABoundaryBlock ByteSpan -> ABlockOrBoundary ByteSpan
forall a. ABoundaryBlock a -> ABlockOrBoundary a
CC.ABOBBoundary)
    (ABoundaryBlock ByteSpan -> ByteString -> ByronBlock)
-> Decoder s (ABoundaryBlock ByteSpan)
-> Decoder s (ByteString -> ByronBlock)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Decoder s (ABoundaryBlock ByteSpan)
forall s. Decoder s (ABoundaryBlock ByteSpan)
CC.fromCBORABoundaryBlock

-- | Encodes a raw Byron header /without/ a tag indicating whether it's a
-- regular header or an EBB header.
--
-- Uses the annotation, so cheap.
encodeByronRegularHeader :: RawHeader -> Encoding
encodeByronRegularHeader :: AHeader ByteString -> Encoding
encodeByronRegularHeader = ByteString -> Encoding
CBOR.encodePreEncoded (ByteString -> Encoding)
-> (AHeader ByteString -> ByteString)
-> AHeader ByteString
-> Encoding
forall b c a. (b -> c) -> (a -> b) -> a -> c
. AHeader ByteString -> ByteString
forall a. AHeader a -> a
CC.headerAnnotation

-- | Inverse of 'encodeByronRegularHeader'
decodeByronRegularHeader
  :: CC.EpochSlots
  -> Decoder s (Lazy.ByteString -> RawHeader)
decodeByronRegularHeader :: EpochSlots -> Decoder s (ByteString -> AHeader ByteString)
decodeByronRegularHeader EpochSlots
epochSlots =
    (ByteString -> AHeader ByteSpan -> AHeader ByteString)
-> AHeader ByteSpan -> ByteString -> AHeader ByteString
forall a b c. (a -> b -> c) -> b -> a -> c
flip ByteString -> AHeader ByteSpan -> AHeader ByteString
forall (f :: * -> *).
Functor f =>
ByteString -> f ByteSpan -> f ByteString
annotationBytes (AHeader ByteSpan -> ByteString -> AHeader ByteString)
-> Decoder s (AHeader ByteSpan)
-> Decoder s (ByteString -> AHeader ByteString)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> EpochSlots -> Decoder s (AHeader ByteSpan)
forall s. EpochSlots -> Decoder s (AHeader ByteSpan)
CC.fromCBORAHeader EpochSlots
epochSlots

-- | Encodes a raw Byron EBB header /without/ a tag indicating whether it's a
-- regular header or an EBB header.
--
-- Uses the annotation, so cheap.
encodeByronBoundaryHeader :: RawBoundaryHeader -> Encoding
encodeByronBoundaryHeader :: ABoundaryHeader ByteString -> Encoding
encodeByronBoundaryHeader = ByteString -> Encoding
CBOR.encodePreEncoded (ByteString -> Encoding)
-> (ABoundaryHeader ByteString -> ByteString)
-> ABoundaryHeader ByteString
-> Encoding
forall b c a. (b -> c) -> (a -> b) -> a -> c
. ABoundaryHeader ByteString -> ByteString
forall a. ABoundaryHeader a -> a
CC.boundaryHeaderAnnotation

-- | Inverse of 'encodeByronBoundaryHeader'
decodeByronBoundaryHeader :: Decoder s (Lazy.ByteString -> RawBoundaryHeader)
decodeByronBoundaryHeader :: Decoder s (ByteString -> ABoundaryHeader ByteString)
decodeByronBoundaryHeader =
    (ByteString
 -> ABoundaryHeader ByteSpan -> ABoundaryHeader ByteString)
-> ABoundaryHeader ByteSpan
-> ByteString
-> ABoundaryHeader ByteString
forall a b c. (a -> b -> c) -> b -> a -> c
flip ByteString
-> ABoundaryHeader ByteSpan -> ABoundaryHeader ByteString
forall (f :: * -> *).
Functor f =>
ByteString -> f ByteSpan -> f ByteString
annotationBytes (ABoundaryHeader ByteSpan
 -> ByteString -> ABoundaryHeader ByteString)
-> Decoder s (ABoundaryHeader ByteSpan)
-> Decoder s (ByteString -> ABoundaryHeader ByteString)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Decoder s (ABoundaryHeader ByteSpan)
forall s. Decoder s (ABoundaryHeader ByteSpan)
CC.fromCBORABoundaryHeader

-- | The 'BinaryBlockInfo' of the given 'ByronBlock'.
--
-- NOTE: the bytestring obtained by slicing the serialised block using the
-- header offset and size will correspond to the /header annotation/, but not
-- to the serialised header, as we add an envelope ('encodeListLen' + tag)
-- around a header in 'encodeByronHeader'. This envelope must thus still be
-- added to the sliced bytestring before it can be deserialised using
-- 'decodeByronHeader'.
byronBinaryBlockInfo :: ByronBlock -> BinaryBlockInfo
byronBinaryBlockInfo :: ByronBlock -> BinaryBlockInfo
byronBinaryBlockInfo ByronBlock
blk = BinaryBlockInfo :: Word16 -> Word16 -> BinaryBlockInfo
BinaryBlockInfo
    { headerOffset :: Word16
headerOffset = Word16
1 {- 'encodeListLen' of the outer 'Either' envelope -}
                   Word16 -> Word16 -> Word16
forall a. Num a => a -> a -> a
+ Word16
1 {- the tag -}
                   Word16 -> Word16 -> Word16
forall a. Num a => a -> a -> a
+ Word16
1 {- 'encodeListLen' of the block: header + body + ...  -}
      -- Compute the length of the annotated header
    , headerSize :: Word16
headerSize   = Int -> Word16
forall a b. (Integral a, Num b) => a -> b
fromIntegral (Int -> Word16) -> Int -> Word16
forall a b. (a -> b) -> a -> b
$ ByteString -> Int
Strict.length (ByteString -> Int) -> ByteString -> Int
forall a b. (a -> b) -> a -> b
$ case ByronBlock -> ABlockOrBoundary ByteString
byronBlockRaw ByronBlock
blk of
        CC.ABOBBoundary ABoundaryBlock ByteString
b -> ABoundaryHeader ByteString -> ByteString
forall a. ABoundaryHeader a -> a
CC.boundaryHeaderAnnotation (ABoundaryHeader ByteString -> ByteString)
-> ABoundaryHeader ByteString -> ByteString
forall a b. (a -> b) -> a -> b
$ ABoundaryBlock ByteString -> ABoundaryHeader ByteString
forall a. ABoundaryBlock a -> ABoundaryHeader a
CC.boundaryHeader ABoundaryBlock ByteString
b
        CC.ABOBBlock    ABlock ByteString
b -> AHeader ByteString -> ByteString
forall a. AHeader a -> a
CC.headerAnnotation         (AHeader ByteString -> ByteString)
-> AHeader ByteString -> ByteString
forall a b. (a -> b) -> a -> b
$ ABlock ByteString -> AHeader ByteString
forall a. ABlock a -> AHeader a
CC.blockHeader    ABlock ByteString
b
    }

{-------------------------------------------------------------------------------
  V1 envelope: unsized header

  These are auxiliary functions for encoding/decoding the Byron header.
-------------------------------------------------------------------------------}

-- | A 'CC.ABlockOrBoundary' is a CBOR 2-tuple of a 'Word' (0 = EBB, 1 =
-- regular block) and block/ebb payload. This function returns the bytes that
-- should be prepended to the payload, i.e., the byte indicating it's a CBOR
-- 2-tuple and the 'Word' indicating whether its an EBB or regular block.
isEbbEnvelope :: IsEBB -> Lazy.ByteString
isEbbEnvelope :: IsEBB -> ByteString
isEbbEnvelope = \case
  IsEBB
IsEBB    -> ByteString
"\130\NUL"
  IsEBB
IsNotEBB -> ByteString
"\130\SOH"

addV1Envelope ::
     (SomeSecond (NestedCtxt Header) ByronBlock, Lazy.ByteString)
  -> Lazy.ByteString
addV1Envelope :: (SomeSecond (NestedCtxt Header) ByronBlock, ByteString)
-> ByteString
addV1Envelope (SomeSecond (NestedCtxt NestedCtxt_ ByronBlock Header b
ctxt), ByteString
bs) = ByteString
isEbbTag ByteString -> ByteString -> ByteString
forall a. Semigroup a => a -> a -> a
<> ByteString
bs
  where
    isEbbTag :: ByteString
isEbbTag = case NestedCtxt_ ByronBlock Header b
ctxt of
      CtxtByronBoundary {} -> IsEBB -> ByteString
isEbbEnvelope IsEBB
IsEBB
      CtxtByronRegular  {} -> IsEBB -> ByteString
isEbbEnvelope IsEBB
IsNotEBB

-- | Drop the V1 EBB-or-regular-header envelope and reconstruct the context.
-- Since we don't know the block size, use 'fakeByronBlockSizeHint'.
dropV1Envelope ::
     Lazy.ByteString
  -> Except String (SomeSecond (NestedCtxt Header) ByronBlock, Lazy.ByteString)
dropV1Envelope :: ByteString
-> Except
     String (SomeSecond (NestedCtxt Header) ByronBlock, ByteString)
dropV1Envelope ByteString
bs = case Int64 -> ByteString -> (ByteString, ByteString)
Lazy.splitAt Int64
2 ByteString
bs of
    (ByteString
prefix, ByteString
suffix)
      | ByteString
prefix ByteString -> ByteString -> Bool
forall a. Eq a => a -> a -> Bool
== IsEBB -> ByteString
isEbbEnvelope IsEBB
IsEBB
      -> (SomeSecond (NestedCtxt Header) ByronBlock, ByteString)
-> Except
     String (SomeSecond (NestedCtxt Header) ByronBlock, ByteString)
forall (m :: * -> *) a. Monad m => a -> m a
return ( NestedCtxt Header ByronBlock (SlotNo, ABoundaryHeader ByteString)
-> SomeSecond (NestedCtxt Header) ByronBlock
forall (f :: * -> * -> *) a b. f a b -> SomeSecond f a
SomeSecond (NestedCtxt Header ByronBlock (SlotNo, ABoundaryHeader ByteString)
 -> SomeSecond (NestedCtxt Header) ByronBlock)
-> (NestedCtxt_
      ByronBlock Header (SlotNo, ABoundaryHeader ByteString)
    -> NestedCtxt
         Header ByronBlock (SlotNo, ABoundaryHeader ByteString))
-> NestedCtxt_
     ByronBlock Header (SlotNo, ABoundaryHeader ByteString)
-> SomeSecond (NestedCtxt Header) ByronBlock
forall b c a. (b -> c) -> (a -> b) -> a -> c
. NestedCtxt_ ByronBlock Header (SlotNo, ABoundaryHeader ByteString)
-> NestedCtxt
     Header ByronBlock (SlotNo, ABoundaryHeader ByteString)
forall (f :: * -> *) blk a.
NestedCtxt_ blk f a -> NestedCtxt f blk a
NestedCtxt (NestedCtxt_ ByronBlock Header (SlotNo, ABoundaryHeader ByteString)
 -> SomeSecond (NestedCtxt Header) ByronBlock)
-> NestedCtxt_
     ByronBlock Header (SlotNo, ABoundaryHeader ByteString)
-> SomeSecond (NestedCtxt Header) ByronBlock
forall a b. (a -> b) -> a -> b
$ SizeInBytes
-> NestedCtxt_
     ByronBlock Header (SlotNo, ABoundaryHeader ByteString)
CtxtByronBoundary SizeInBytes
fakeByronBlockSizeHint
                , ByteString
suffix
                )
      | ByteString
prefix ByteString -> ByteString -> Bool
forall a. Eq a => a -> a -> Bool
== IsEBB -> ByteString
isEbbEnvelope IsEBB
IsNotEBB
      -> (SomeSecond (NestedCtxt Header) ByronBlock, ByteString)
-> Except
     String (SomeSecond (NestedCtxt Header) ByronBlock, ByteString)
forall (m :: * -> *) a. Monad m => a -> m a
return ( NestedCtxt Header ByronBlock (AHeader ByteString)
-> SomeSecond (NestedCtxt Header) ByronBlock
forall (f :: * -> * -> *) a b. f a b -> SomeSecond f a
SomeSecond (NestedCtxt Header ByronBlock (AHeader ByteString)
 -> SomeSecond (NestedCtxt Header) ByronBlock)
-> (NestedCtxt_ ByronBlock Header (AHeader ByteString)
    -> NestedCtxt Header ByronBlock (AHeader ByteString))
-> NestedCtxt_ ByronBlock Header (AHeader ByteString)
-> SomeSecond (NestedCtxt Header) ByronBlock
forall b c a. (b -> c) -> (a -> b) -> a -> c
. NestedCtxt_ ByronBlock Header (AHeader ByteString)
-> NestedCtxt Header ByronBlock (AHeader ByteString)
forall (f :: * -> *) blk a.
NestedCtxt_ blk f a -> NestedCtxt f blk a
NestedCtxt (NestedCtxt_ ByronBlock Header (AHeader ByteString)
 -> SomeSecond (NestedCtxt Header) ByronBlock)
-> NestedCtxt_ ByronBlock Header (AHeader ByteString)
-> SomeSecond (NestedCtxt Header) ByronBlock
forall a b. (a -> b) -> a -> b
$ SizeInBytes -> NestedCtxt_ ByronBlock Header (AHeader ByteString)
CtxtByronRegular SizeInBytes
fakeByronBlockSizeHint
                , ByteString
suffix
                )
      | Bool
otherwise
      -> String
-> Except
     String (SomeSecond (NestedCtxt Header) ByronBlock, ByteString)
forall e (m :: * -> *) a. MonadError e m => e -> m a
throwError String
"decodeUnsized: invalid prefix"

-- | Fake size (used in compatibility mode)
fakeByronBlockSizeHint :: SizeInBytes
fakeByronBlockSizeHint :: SizeInBytes
fakeByronBlockSizeHint = SizeInBytes
2000

-- | Encode an unsized header
--
-- Does /not/ have to backwards compatible with legacy (cardano-sl) nodes
-- (which never send or store these headers), but should be inverse to
-- 'decodeSizedHeader', and moreover uses 'fromCBORABlockOrBoundaryHdr' from
-- cardano-ledger-byron, and so we don't have too much choice in this encoder.
encodeUnsizedHeader :: UnsizedHeader -> Encoding
encodeUnsizedHeader :: UnsizedHeader -> Encoding
encodeUnsizedHeader (UnsizedHeader ABlockOrBoundaryHdr ByteString
raw SlotNo
_ ByronHash
_) = ABlockOrBoundaryHdr ByteString -> Encoding
CC.toCBORABlockOrBoundaryHdr ABlockOrBoundaryHdr ByteString
raw

-- | Inverse of 'encodeSizedHeader'
decodeUnsizedHeader :: CC.EpochSlots
                    -> Decoder s (Lazy.ByteString -> UnsizedHeader)
decodeUnsizedHeader :: EpochSlots -> Decoder s (ByteString -> UnsizedHeader)
decodeUnsizedHeader EpochSlots
epochSlots =
    ABlockOrBoundaryHdr ByteSpan -> ByteString -> UnsizedHeader
fillInByteString (ABlockOrBoundaryHdr ByteSpan -> ByteString -> UnsizedHeader)
-> Decoder s (ABlockOrBoundaryHdr ByteSpan)
-> Decoder s (ByteString -> UnsizedHeader)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> EpochSlots -> Decoder s (ABlockOrBoundaryHdr ByteSpan)
forall s. EpochSlots -> Decoder s (ABlockOrBoundaryHdr ByteSpan)
CC.fromCBORABlockOrBoundaryHdr EpochSlots
epochSlots
  where
    fillInByteString :: CC.ABlockOrBoundaryHdr ByteSpan
                     -> Lazy.ByteString
                     -> UnsizedHeader
    fillInByteString :: ABlockOrBoundaryHdr ByteSpan -> ByteString -> UnsizedHeader
fillInByteString ABlockOrBoundaryHdr ByteSpan
it ByteString
theBytes = EpochSlots -> ABlockOrBoundaryHdr ByteString -> UnsizedHeader
mkUnsizedHeader EpochSlots
epochSlots (ABlockOrBoundaryHdr ByteString -> UnsizedHeader)
-> ABlockOrBoundaryHdr ByteString -> UnsizedHeader
forall a b. (a -> b) -> a -> b
$
      ByteString -> ByteString
Lazy.toStrict (ByteString -> ByteString)
-> (ByteSpan -> ByteString) -> ByteSpan -> ByteString
forall b c a. (b -> c) -> (a -> b) -> a -> c
. ByteString -> ByteSpan -> ByteString
slice ByteString
theBytes (ByteSpan -> ByteString)
-> ABlockOrBoundaryHdr ByteSpan -> ABlockOrBoundaryHdr ByteString
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> ABlockOrBoundaryHdr ByteSpan
it