{-# LANGUAGE ConstraintKinds #-}
{-# LANGUAGE DataKinds #-}
{-# LANGUAGE DeriveGeneric #-}
{-# LANGUAGE DeriveTraversable #-}
{-# LANGUAGE DerivingVia #-}
{-# LANGUAGE FlexibleContexts #-}
{-# LANGUAGE FlexibleInstances #-}
{-# LANGUAGE GADTs #-}
{-# LANGUAGE OverloadedStrings #-}
{-# LANGUAGE QuantifiedConstraints #-}
{-# LANGUAGE RankNTypes #-}
{-# LANGUAGE ScopedTypeVariables #-}
{-# LANGUAGE TypeApplications #-}
{-# LANGUAGE TypeOperators #-}
{-# LANGUAGE UndecidableInstances #-}

module Test.Util.Serialisation.Roundtrip (
    -- * Basic test helpers
    roundtrip
  , roundtrip'
  , roundtripAnd
    -- * Test skeleton
  , Arbitrary'
  , Coherent (..)
  , WithVersion (..)
  , prop_hashSize
  , roundtrip_ConvertRawHash
  , roundtrip_SerialiseDisk
  , roundtrip_SerialiseNodeToClient
  , roundtrip_SerialiseNodeToNode
  , roundtrip_all
  , roundtrip_all_skipping
  , roundtrip_envelopes
    -- ** Exclusion of CBOR validity tests
  , ShouldCheckCBORValidity (CheckCBORValidity, DoNotCheckCBORValidity)
    -- * Roundtrip tests for 'Example's
  , examplesRoundtrip
  ) where

import           Codec.CBOR.Decoding (Decoder)
import           Codec.CBOR.Encoding (Encoding)
import           Codec.CBOR.FlatTerm (toFlatTerm, validFlatTerm)
import           Codec.CBOR.Read (DeserialiseFailure, deserialiseFromBytes)
import           Codec.CBOR.Write (toLazyByteString)
import           Codec.Serialise (decode, encode)
import           Control.Arrow (left)
import           Control.Monad (unless, when)
import qualified Data.ByteString.Base16.Lazy as Base16
import qualified Data.ByteString.Lazy as Lazy
import qualified Data.ByteString.Lazy.Char8 as Char8
import qualified Data.ByteString.Short as Short
import           Data.Constraint
import           Data.Function (on)
import           Data.Maybe (fromMaybe)
import qualified Data.Text.Lazy as T
import           Data.Typeable
import           GHC.Generics (Generic)
import           Ouroboros.Consensus.Block
import           Ouroboros.Consensus.HeaderValidation (AnnTip)
import           Ouroboros.Consensus.Ledger.Basics
import           Ouroboros.Consensus.Ledger.Extended (decodeDiskExtLedgerState,
                     encodeDiskExtLedgerState)
import           Ouroboros.Consensus.Ledger.Query
import qualified Ouroboros.Consensus.Ledger.Query as Query
import           Ouroboros.Consensus.Ledger.SupportsMempool (ApplyTxErr, GenTx,
                     GenTxId)
import           Ouroboros.Consensus.Ledger.SupportsProtocol
                     (LedgerSupportsProtocol)
import           Ouroboros.Consensus.Node.NetworkProtocolVersion
import           Ouroboros.Consensus.Node.Run (SerialiseNodeToClientConstraints,
                     SerialiseNodeToNodeConstraints (..))
import           Ouroboros.Consensus.Node.Serialisation
import           Ouroboros.Consensus.Protocol.Abstract (ChainDepState)
import           Ouroboros.Consensus.Storage.ChainDB (SerialiseDiskConstraints)
import           Ouroboros.Consensus.Storage.Serialisation
import           Ouroboros.Network.Block (Serialised (..), fromSerialised,
                     mkSerialised)
import           Quiet (Quiet (..))
import           Test.Tasty
import           Test.Tasty.QuickCheck
import           Test.Util.Orphans.Arbitrary ()
import           Test.Util.Serialisation.Examples (Examples (..), Labelled)
import           Test.Util.Serialisation.SomeResult (SomeResult (..))
import           Test.Util.TestEnv (adjustQuickCheckTests)
import           Text.Pretty.Simple (pShow)

{------------------------------------------------------------------------------
  Basic test helpers
------------------------------------------------------------------------------}

roundtrip :: (Eq a, Show a)
          => (a -> Encoding)
          -> (forall s. Decoder s a)
          -> a
          -> Property
roundtrip :: forall a.
(Eq a, Show a) =>
(a -> Encoding) -> (forall s. Decoder s a) -> a -> Property
roundtrip a -> Encoding
enc forall s. Decoder s a
dec = (a -> Encoding)
-> (forall s. Decoder s (ByteString -> a)) -> a -> Property
forall a.
(Eq a, Show a) =>
(a -> Encoding)
-> (forall s. Decoder s (ByteString -> a)) -> a -> Property
roundtrip' a -> Encoding
enc (a -> ByteString -> a
forall a b. a -> b -> a
const (a -> ByteString -> a)
-> Decoder s a -> Decoder s (ByteString -> a)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Decoder s a
forall s. Decoder s a
dec)

-- | Perform roundtrip tests, checking the validity of the encoded CBOR.
--
-- See 'roundtripAnd'
--
roundtrip' :: forall a.
              (Eq a, Show a)
           => (a -> Encoding)  -- ^ @enc@
           -> (forall s. Decoder s (Lazy.ByteString -> a))
           -> a
           -> Property
roundtrip' :: forall a.
(Eq a, Show a) =>
(a -> Encoding)
-> (forall s. Decoder s (ByteString -> a)) -> a -> Property
roundtrip' = ShouldCheckCBORValidity
-> (a -> Encoding)
-> (forall s. Decoder s (ByteString -> a))
-> a
-> Property
forall a.
(Eq a, Show a) =>
ShouldCheckCBORValidity
-> (a -> Encoding)
-> (forall s. Decoder s (ByteString -> a))
-> a
-> Property
roundtripAnd ShouldCheckCBORValidity
CheckCBORValidity

data ShouldCheckCBORValidity = CheckCBORValidity | DoNotCheckCBORValidity
  deriving (ShouldCheckCBORValidity -> ShouldCheckCBORValidity -> Bool
(ShouldCheckCBORValidity -> ShouldCheckCBORValidity -> Bool)
-> (ShouldCheckCBORValidity -> ShouldCheckCBORValidity -> Bool)
-> Eq ShouldCheckCBORValidity
forall a. (a -> a -> Bool) -> (a -> a -> Bool) -> Eq a
$c== :: ShouldCheckCBORValidity -> ShouldCheckCBORValidity -> Bool
== :: ShouldCheckCBORValidity -> ShouldCheckCBORValidity -> Bool
$c/= :: ShouldCheckCBORValidity -> ShouldCheckCBORValidity -> Bool
/= :: ShouldCheckCBORValidity -> ShouldCheckCBORValidity -> Bool
Eq, Int -> ShouldCheckCBORValidity -> ShowS
[ShouldCheckCBORValidity] -> ShowS
ShouldCheckCBORValidity -> String
(Int -> ShouldCheckCBORValidity -> ShowS)
-> (ShouldCheckCBORValidity -> String)
-> ([ShouldCheckCBORValidity] -> ShowS)
-> Show ShouldCheckCBORValidity
forall a.
(Int -> a -> ShowS) -> (a -> String) -> ([a] -> ShowS) -> Show a
$cshowsPrec :: Int -> ShouldCheckCBORValidity -> ShowS
showsPrec :: Int -> ShouldCheckCBORValidity -> ShowS
$cshow :: ShouldCheckCBORValidity -> String
show :: ShouldCheckCBORValidity -> String
$cshowList :: [ShouldCheckCBORValidity] -> ShowS
showList :: [ShouldCheckCBORValidity] -> ShowS
Show)

-- | Roundtrip property for values annotated with their serialized form
--
-- If 'CheckCBORValidity' is passed, then we check that the encoded
-- CBOR is valid using 'validFlatTerm'. In general we want to check
-- this, however there are cases where legacy encoders do not produce
-- valid CBOR but we need to keep them for backwards compatibility. In
-- such cases, the option to skip this check
-- ('DoNotCheckCBORValidity') can be used.
--
-- NOTE: Suppose @a@ consists of a pair of the unannotated value @a'@ and some
-- 'Lazy.ByteString'. The roundtrip property will fail if that
-- 'Lazy.ByteString' encoding is not equal to @enc a'@. One way in which this
-- might happen is if the annotation is not canonical CBOR, but @enc@ does
-- produce canonical CBOR.
roundtripAnd :: forall a.
              (Eq a, Show a)
           => ShouldCheckCBORValidity
           -> (a -> Encoding)  -- ^ @enc@
           -> (forall s. Decoder s (Lazy.ByteString -> a))
           -> a
           -> Property
roundtripAnd :: forall a.
(Eq a, Show a) =>
ShouldCheckCBORValidity
-> (a -> Encoding)
-> (forall s. Decoder s (ByteString -> a))
-> a
-> Property
roundtripAnd ShouldCheckCBORValidity
check a -> Encoding
enc forall s. Decoder s (ByteString -> a)
dec a
a = Either String () -> Property
checkRoundtripResult (Either String () -> Property) -> Either String () -> Property
forall a b. (a -> b) -> a -> b
$ do
    let enc_a :: Encoding
enc_a = a -> Encoding
enc a
a
        bs :: ByteString
bs    = Encoding -> ByteString
toLazyByteString Encoding
enc_a

    Bool -> Either String () -> Either String ()
forall (f :: * -> *). Applicative f => Bool -> f () -> f ()
when (ShouldCheckCBORValidity
check ShouldCheckCBORValidity -> ShouldCheckCBORValidity -> Bool
forall a. Eq a => a -> a -> Bool
== ShouldCheckCBORValidity
CheckCBORValidity) (Either String () -> Either String ())
-> Either String () -> Either String ()
forall a b. (a -> b) -> a -> b
$
      (FlatTerm -> Bool
validFlatTerm (Encoding -> FlatTerm
toFlatTerm Encoding
enc_a)          Bool -> String -> Either String ()
?!       String
"Encoded flat term is not valid: " String -> ShowS
forall a. Semigroup a => a -> a -> a
<> Encoding -> String
forall a. Show a => a -> String
show Encoding
enc_a)
    (bsRem, a' ) <- (forall s. Decoder s (ByteString -> a))
-> ByteString
-> Either DeserialiseFailure (ByteString, ByteString -> a)
forall a.
(forall s. Decoder s a)
-> ByteString -> Either DeserialiseFailure (ByteString, a)
deserialiseFromBytes Decoder s (ByteString -> a)
forall s. Decoder s (ByteString -> a)
dec ByteString
bs Either DeserialiseFailure (ByteString, ByteString -> a)
-> (DeserialiseFailure -> String)
-> Either String (ByteString, ByteString -> a)
`onError` ByteString -> DeserialiseFailure -> String
showByteString ByteString
bs
    Lazy.null bsRem                              ?!       "Left-over bytes: " <> toBase16 bsRem
    a == a' bs                                   ?!        pShowNeq a (a' bs)
  where
    (?!) :: Bool -> String -> Either String ()
    Bool
cond ?! :: Bool -> String -> Either String ()
?! String
msg = Bool -> Either String () -> Either String ()
forall (f :: * -> *). Applicative f => Bool -> f () -> f ()
unless Bool
cond (Either String () -> Either String ())
-> Either String () -> Either String ()
forall a b. (a -> b) -> a -> b
$ String -> Either String ()
forall a b. a -> Either a b
Left String
msg
    infix 1 ?!

    pShowNeq :: a -> a -> String
pShowNeq a
x a
y = Text -> String
T.unpack (a -> Text
forall a. Show a => a -> Text
pShow a
x) String -> ShowS
forall a. Semigroup a => a -> a -> a
<> String
"\n \t/= \n"  String -> ShowS
forall a. Semigroup a => a -> a -> a
<> Text -> String
T.unpack (a -> Text
forall a. Show a => a -> Text
pShow a
y)

    onError ::
          Either DeserialiseFailure (Char8.ByteString, Char8.ByteString -> a)
      -> (DeserialiseFailure -> String)
      -> Either String (Char8.ByteString, Char8.ByteString -> a)
    onError :: Either DeserialiseFailure (ByteString, ByteString -> a)
-> (DeserialiseFailure -> String)
-> Either String (ByteString, ByteString -> a)
onError Either DeserialiseFailure (ByteString, ByteString -> a)
result DeserialiseFailure -> String
showDeserialiseFailure =
      (DeserialiseFailure -> String)
-> Either DeserialiseFailure (ByteString, ByteString -> a)
-> Either String (ByteString, ByteString -> a)
forall b c d. (b -> c) -> Either b d -> Either c d
forall (a :: * -> * -> *) b c d.
ArrowChoice a =>
a b c -> a (Either b d) (Either c d)
left DeserialiseFailure -> String
showDeserialiseFailure Either DeserialiseFailure (ByteString, ByteString -> a)
result

    showByteString ::
         Char8.ByteString
      -> DeserialiseFailure
      -> String
    showByteString :: ByteString -> DeserialiseFailure -> String
showByteString ByteString
bs DeserialiseFailure
deserialiseFailure =
      DeserialiseFailure -> String
forall a. Show a => a -> String
show DeserialiseFailure
deserialiseFailure String -> ShowS
forall a. Semigroup a => a -> a -> a
<> String
"\n" String -> ShowS
forall a. Semigroup a => a -> a -> a
<> String
"When deserialising " String -> ShowS
forall a. Semigroup a => a -> a -> a
<> ByteString -> String
toBase16 ByteString
bs

    toBase16 :: Lazy.ByteString -> String
    toBase16 :: ByteString -> String
toBase16 = ByteString -> String
Char8.unpack (ByteString -> String)
-> (ByteString -> ByteString) -> ByteString -> String
forall b c a. (b -> c) -> (a -> b) -> a -> c
. ByteString -> ByteString
Base16.encode

    checkRoundtripResult :: Either String () -> Property
    checkRoundtripResult :: Either String () -> Property
checkRoundtripResult (Left String
str) = String -> Bool -> Property
forall prop. Testable prop => String -> prop -> Property
counterexample String
str Bool
False
    checkRoundtripResult (Right ()) = () -> Property
forall prop. Testable prop => prop -> Property
property ()

roundtripComparingEncoding ::
     (a -> Encoding)
  -> (forall s. Decoder s a)
  -> a
  -> Property
roundtripComparingEncoding :: forall a.
(a -> Encoding) -> (forall s. Decoder s a) -> a -> Property
roundtripComparingEncoding a -> Encoding
enc forall s. Decoder s a
dec = (a -> Encoding)
-> (forall s. Decoder s (ByteString -> a)) -> a -> Property
forall a.
(a -> Encoding)
-> (forall s. Decoder s (ByteString -> a)) -> a -> Property
roundtripComparingEncoding' a -> Encoding
enc (a -> ByteString -> a
forall a b. a -> b -> a
const (a -> ByteString -> a)
-> Decoder s a -> Decoder s (ByteString -> a)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Decoder s a
forall s. Decoder s a
dec)

-- | Like 'roundtrip'', but checks for equality of the encoding (i.e. the byte
-- string) instead of the @a@ values using @Eq a@. This is useful When we don't
-- have an @Eq a@ instance.
roundtripComparingEncoding' ::
    (a -> Encoding)  -- ^ @enc@
  -> (forall s. Decoder s (Lazy.ByteString -> a))
  -> a
  -> Property
roundtripComparingEncoding' :: forall a.
(a -> Encoding)
-> (forall s. Decoder s (ByteString -> a)) -> a -> Property
roundtripComparingEncoding' a -> Encoding
enc forall s. Decoder s (ByteString -> a)
dec a
a = case (forall s. Decoder s (ByteString -> a))
-> ByteString
-> Either DeserialiseFailure (ByteString, ByteString -> a)
forall a.
(forall s. Decoder s a)
-> ByteString -> Either DeserialiseFailure (ByteString, a)
deserialiseFromBytes Decoder s (ByteString -> a)
forall s. Decoder s (ByteString -> a)
dec ByteString
bs of
    Right (ByteString
remainingBytes, ByteString -> a
a')
      | let bs' :: ByteString
bs' = Encoding -> ByteString
toLazyByteString (a -> Encoding
enc (ByteString -> a
a' ByteString
bs))
      , ByteString -> Bool
Lazy.null ByteString
remainingBytes
      -> ByteString
bs ByteString -> ByteString -> Property
forall a. (Eq a, Show a) => a -> a -> Property
=== ByteString
bs'
      | Bool
otherwise
      -> String -> Bool -> Property
forall prop. Testable prop => String -> prop -> Property
counterexample (String
"left-over bytes: " String -> ShowS
forall a. Semigroup a => a -> a -> a
<> ByteString -> String
toBase16 ByteString
remainingBytes) Bool
False
    Left DeserialiseFailure
e
      -> String -> Property -> Property
forall prop. Testable prop => String -> prop -> Property
counterexample (DeserialiseFailure -> String
forall a. Show a => a -> String
show DeserialiseFailure
e) (Property -> Property) -> Property -> Property
forall a b. (a -> b) -> a -> b
$
         String -> Bool -> Property
forall prop. Testable prop => String -> prop -> Property
counterexample (ByteString -> String
toBase16 ByteString
bs) Bool
False
  where
    bs :: ByteString
bs = Encoding -> ByteString
toLazyByteString (a -> Encoding
enc a
a)

    toBase16 :: Lazy.ByteString -> String
    toBase16 :: ByteString -> String
toBase16 = ByteString -> String
Char8.unpack (ByteString -> String)
-> (ByteString -> ByteString) -> ByteString -> String
forall b c a. (b -> c) -> (a -> b) -> a -> c
. ByteString -> ByteString
Base16.encode

{------------------------------------------------------------------------------
  Test skeleton
------------------------------------------------------------------------------}

-- | Constraints needed in practice for something to be passed in as an
-- 'Arbitrary' argument to a QuickCheck property.
type Arbitrary' a = (Arbitrary a, Eq a, Show a)

-- | All roundtrip tests
roundtrip_all
  :: forall blk.
     ( SerialiseDiskConstraints         blk
     , SerialiseNodeToNodeConstraints   blk
     , SerialiseNodeToClientConstraints blk

     , Show (BlockNodeToNodeVersion   blk)

     , StandardHash blk
     , GetHeader    blk

     , Arbitrary' blk
     , Arbitrary' (Header blk)
     , Arbitrary' (HeaderHash blk)
     , Arbitrary' (LedgerState blk EmptyMK)
     , Arbitrary' (AnnTip blk)
     , Arbitrary' (ChainDepState (BlockProtocol blk))

     , ArbitraryWithVersion (BlockNodeToNodeVersion blk) blk
     , ArbitraryWithVersion (BlockNodeToNodeVersion blk) (Coherent blk)
     , ArbitraryWithVersion (BlockNodeToNodeVersion blk) (Header blk)
     , ArbitraryWithVersion (BlockNodeToNodeVersion blk) (GenTx blk)
     , ArbitraryWithVersion (BlockNodeToNodeVersion blk) (GenTxId blk)
     , ArbitraryWithVersion (BlockNodeToNodeVersion blk) (SomeSecond (NestedCtxt Header) blk)

     , ArbitraryWithVersion (BlockNodeToClientVersion blk) blk
     , ArbitraryWithVersion (BlockNodeToClientVersion blk) (GenTx blk)
     , ArbitraryWithVersion (BlockNodeToClientVersion blk) (ApplyTxErr blk)
     , ArbitraryWithVersion (BlockNodeToClientVersion blk) (SomeBlockQuery (BlockQuery blk))
     , ArbitraryWithVersion (BlockNodeToClientVersion blk) (SomeResult blk)
     , Arbitrary (WithVersion (BlockNodeToClientVersion blk) (LedgerConfig blk))
     , ArbitraryWithVersion (QueryVersion, BlockNodeToClientVersion blk) (SomeSecond Query blk)

     , Show (BlockNodeToClientVersion blk)
     , BlockSupportsLedgerQuery blk
     )
  => CodecConfig blk
  -> (forall a. NestedCtxt_ blk Header a -> Dict (Eq a, Show a))
  -> TestTree
roundtrip_all :: forall blk.
(SerialiseDiskConstraints blk, SerialiseNodeToNodeConstraints blk,
 SerialiseNodeToClientConstraints blk,
 Show (BlockNodeToNodeVersion blk), StandardHash blk, GetHeader blk,
 Arbitrary' blk, Arbitrary' (Header blk),
 Arbitrary' (HeaderHash blk), Arbitrary' (LedgerState blk EmptyMK),
 Arbitrary' (AnnTip blk),
 Arbitrary' (ChainDepState (BlockProtocol blk)),
 ArbitraryWithVersion (BlockNodeToNodeVersion blk) blk,
 ArbitraryWithVersion (BlockNodeToNodeVersion blk) (Coherent blk),
 ArbitraryWithVersion (BlockNodeToNodeVersion blk) (Header blk),
 ArbitraryWithVersion (BlockNodeToNodeVersion blk) (GenTx blk),
 ArbitraryWithVersion (BlockNodeToNodeVersion blk) (GenTxId blk),
 ArbitraryWithVersion
   (BlockNodeToNodeVersion blk) (SomeSecond (NestedCtxt Header) blk),
 ArbitraryWithVersion (BlockNodeToClientVersion blk) blk,
 ArbitraryWithVersion (BlockNodeToClientVersion blk) (GenTx blk),
 ArbitraryWithVersion
   (BlockNodeToClientVersion blk) (ApplyTxErr blk),
 ArbitraryWithVersion
   (BlockNodeToClientVersion blk) (SomeBlockQuery (BlockQuery blk)),
 ArbitraryWithVersion
   (BlockNodeToClientVersion blk) (SomeResult blk),
 Arbitrary
   (WithVersion (BlockNodeToClientVersion blk) (LedgerConfig blk)),
 ArbitraryWithVersion
   (QueryVersion, BlockNodeToClientVersion blk)
   (SomeSecond Query blk),
 Show (BlockNodeToClientVersion blk),
 BlockSupportsLedgerQuery blk) =>
CodecConfig blk
-> (forall a. NestedCtxt_ blk Header a -> Dict (Eq a, Show a))
-> TestTree
roundtrip_all = (String -> ShouldCheckCBORValidity)
-> CodecConfig blk
-> (forall a. NestedCtxt_ blk Header a -> Dict (Eq a, Show a))
-> TestTree
forall blk.
(SerialiseDiskConstraints blk, SerialiseNodeToNodeConstraints blk,
 SerialiseNodeToClientConstraints blk,
 Show (BlockNodeToNodeVersion blk), StandardHash blk, GetHeader blk,
 Arbitrary' blk, Arbitrary' (Header blk),
 Arbitrary' (HeaderHash blk), Arbitrary' (LedgerState blk EmptyMK),
 Arbitrary' (AnnTip blk),
 Arbitrary' (ChainDepState (BlockProtocol blk)),
 ArbitraryWithVersion (BlockNodeToNodeVersion blk) blk,
 ArbitraryWithVersion (BlockNodeToNodeVersion blk) (Coherent blk),
 ArbitraryWithVersion (BlockNodeToNodeVersion blk) (Header blk),
 ArbitraryWithVersion (BlockNodeToNodeVersion blk) (GenTx blk),
 ArbitraryWithVersion (BlockNodeToNodeVersion blk) (GenTxId blk),
 ArbitraryWithVersion
   (BlockNodeToNodeVersion blk) (SomeSecond (NestedCtxt Header) blk),
 ArbitraryWithVersion (BlockNodeToClientVersion blk) blk,
 ArbitraryWithVersion (BlockNodeToClientVersion blk) (GenTx blk),
 ArbitraryWithVersion
   (BlockNodeToClientVersion blk) (ApplyTxErr blk),
 ArbitraryWithVersion
   (BlockNodeToClientVersion blk) (SomeBlockQuery (BlockQuery blk)),
 ArbitraryWithVersion
   (BlockNodeToClientVersion blk) (SomeResult blk),
 Arbitrary
   (WithVersion (BlockNodeToClientVersion blk) (LedgerConfig blk)),
 ArbitraryWithVersion
   (QueryVersion, BlockNodeToClientVersion blk)
   (SomeSecond Query blk),
 Show (BlockNodeToClientVersion blk),
 BlockSupportsLedgerQuery blk) =>
(String -> ShouldCheckCBORValidity)
-> CodecConfig blk
-> (forall a. NestedCtxt_ blk Header a -> Dict (Eq a, Show a))
-> TestTree
roundtrip_all_skipping (ShouldCheckCBORValidity -> String -> ShouldCheckCBORValidity
forall a b. a -> b -> a
const ShouldCheckCBORValidity
CheckCBORValidity)

-- | All roundtrip tests, skipping the specified CBOR validity tests.
--
-- TODO: the exclusion rule should only be considered for blocks __before__ Conway!
--
-- The 'TestName' corresponds to the name of the roundtrip property
-- being tested. At the moment we consider for exclusion:
--
-- - Node to client tests due to
--   [this issue](https://github.com/IntersectMBO/cardano-ledger/issues/3800).
--
roundtrip_all_skipping
  :: forall blk.
     ( SerialiseDiskConstraints         blk
     , SerialiseNodeToNodeConstraints   blk
     , SerialiseNodeToClientConstraints blk

     , Show (BlockNodeToNodeVersion   blk)

     , StandardHash blk
     , GetHeader    blk

     , Arbitrary' blk
     , Arbitrary' (Header blk)
     , Arbitrary' (HeaderHash blk)
     , Arbitrary' (LedgerState blk EmptyMK)
     , Arbitrary' (AnnTip blk)
     , Arbitrary' (ChainDepState (BlockProtocol blk))

     , ArbitraryWithVersion (BlockNodeToNodeVersion blk) blk
     , ArbitraryWithVersion (BlockNodeToNodeVersion blk) (Coherent blk)
     , ArbitraryWithVersion (BlockNodeToNodeVersion blk) (Header blk)
     , ArbitraryWithVersion (BlockNodeToNodeVersion blk) (GenTx blk)
     , ArbitraryWithVersion (BlockNodeToNodeVersion blk) (GenTxId blk)
     , ArbitraryWithVersion (BlockNodeToNodeVersion blk) (SomeSecond (NestedCtxt Header) blk)

     , ArbitraryWithVersion (BlockNodeToClientVersion blk) blk
     , ArbitraryWithVersion (BlockNodeToClientVersion blk) (GenTx blk)
     , ArbitraryWithVersion (BlockNodeToClientVersion blk) (ApplyTxErr blk)
     , ArbitraryWithVersion (BlockNodeToClientVersion blk) (SomeBlockQuery (BlockQuery blk))
     , ArbitraryWithVersion (BlockNodeToClientVersion blk) (SomeResult blk)
     , Arbitrary (WithVersion (BlockNodeToClientVersion blk) (LedgerConfig blk))
     , ArbitraryWithVersion (QueryVersion, BlockNodeToClientVersion blk) (SomeSecond Query blk)

     , Show (BlockNodeToClientVersion blk)
     , BlockSupportsLedgerQuery blk
     )
  => (TestName -> ShouldCheckCBORValidity)
  -> CodecConfig blk
  -> (forall a. NestedCtxt_ blk Header a -> Dict (Eq a, Show a))
  -> TestTree
roundtrip_all_skipping :: forall blk.
(SerialiseDiskConstraints blk, SerialiseNodeToNodeConstraints blk,
 SerialiseNodeToClientConstraints blk,
 Show (BlockNodeToNodeVersion blk), StandardHash blk, GetHeader blk,
 Arbitrary' blk, Arbitrary' (Header blk),
 Arbitrary' (HeaderHash blk), Arbitrary' (LedgerState blk EmptyMK),
 Arbitrary' (AnnTip blk),
 Arbitrary' (ChainDepState (BlockProtocol blk)),
 ArbitraryWithVersion (BlockNodeToNodeVersion blk) blk,
 ArbitraryWithVersion (BlockNodeToNodeVersion blk) (Coherent blk),
 ArbitraryWithVersion (BlockNodeToNodeVersion blk) (Header blk),
 ArbitraryWithVersion (BlockNodeToNodeVersion blk) (GenTx blk),
 ArbitraryWithVersion (BlockNodeToNodeVersion blk) (GenTxId blk),
 ArbitraryWithVersion
   (BlockNodeToNodeVersion blk) (SomeSecond (NestedCtxt Header) blk),
 ArbitraryWithVersion (BlockNodeToClientVersion blk) blk,
 ArbitraryWithVersion (BlockNodeToClientVersion blk) (GenTx blk),
 ArbitraryWithVersion
   (BlockNodeToClientVersion blk) (ApplyTxErr blk),
 ArbitraryWithVersion
   (BlockNodeToClientVersion blk) (SomeBlockQuery (BlockQuery blk)),
 ArbitraryWithVersion
   (BlockNodeToClientVersion blk) (SomeResult blk),
 Arbitrary
   (WithVersion (BlockNodeToClientVersion blk) (LedgerConfig blk)),
 ArbitraryWithVersion
   (QueryVersion, BlockNodeToClientVersion blk)
   (SomeSecond Query blk),
 Show (BlockNodeToClientVersion blk),
 BlockSupportsLedgerQuery blk) =>
(String -> ShouldCheckCBORValidity)
-> CodecConfig blk
-> (forall a. NestedCtxt_ blk Header a -> Dict (Eq a, Show a))
-> TestTree
roundtrip_all_skipping String -> ShouldCheckCBORValidity
shouldCheckCBORvalidity CodecConfig blk
ccfg forall a. NestedCtxt_ blk Header a -> Dict (Eq a, Show a)
dictNestedHdr =
    String -> [TestTree] -> TestTree
testGroup String
"Roundtrip" [
        String -> [TestTree] -> TestTree
testGroup String
"SerialiseDisk"         ([TestTree] -> TestTree) -> [TestTree] -> TestTree
forall a b. (a -> b) -> a -> b
$ CodecConfig blk
-> (forall a. NestedCtxt_ blk Header a -> Dict (Eq a, Show a))
-> [TestTree]
forall blk.
(SerialiseDiskConstraints blk, Arbitrary' blk,
 Arbitrary' (Header blk), Arbitrary' (LedgerState blk EmptyMK),
 Arbitrary' (AnnTip blk),
 Arbitrary' (ChainDepState (BlockProtocol blk))) =>
CodecConfig blk
-> (forall a. NestedCtxt_ blk Header a -> Dict (Eq a, Show a))
-> [TestTree]
roundtrip_SerialiseDisk         CodecConfig blk
ccfg NestedCtxt_ blk Header a -> Dict (Eq a, Show a)
forall a. NestedCtxt_ blk Header a -> Dict (Eq a, Show a)
dictNestedHdr
      , String -> [TestTree] -> TestTree
testGroup String
"SerialiseNodeToNode"   ([TestTree] -> TestTree) -> [TestTree] -> TestTree
forall a b. (a -> b) -> a -> b
$ CodecConfig blk -> [TestTree]
forall blk.
(SerialiseNodeToNodeConstraints blk,
 Show (BlockNodeToNodeVersion blk),
 ArbitraryWithVersion (BlockNodeToNodeVersion blk) blk,
 ArbitraryWithVersion (BlockNodeToNodeVersion blk) (Header blk),
 ArbitraryWithVersion (BlockNodeToNodeVersion blk) (GenTx blk),
 ArbitraryWithVersion (BlockNodeToNodeVersion blk) (GenTxId blk),
 EncodeDisk blk blk, DecodeDisk blk (ByteString -> blk),
 HasNestedContent Header blk, EncodeDiskDep (NestedCtxt Header) blk,
 DecodeDiskDep (NestedCtxt Header) blk) =>
CodecConfig blk -> [TestTree]
roundtrip_SerialiseNodeToNode   CodecConfig blk
ccfg
      , String -> [TestTree] -> TestTree
testGroup String
"SerialiseNodeToClient" ([TestTree] -> TestTree) -> [TestTree] -> TestTree
forall a b. (a -> b) -> a -> b
$ (String -> ShouldCheckCBORValidity)
-> CodecConfig blk -> [TestTree]
forall blk.
(SerialiseNodeToClientConstraints blk,
 ArbitraryWithVersion (BlockNodeToClientVersion blk) blk,
 ArbitraryWithVersion (BlockNodeToClientVersion blk) (GenTx blk),
 ArbitraryWithVersion
   (BlockNodeToClientVersion blk) (ApplyTxErr blk),
 ArbitraryWithVersion
   (BlockNodeToClientVersion blk) (SomeBlockQuery (BlockQuery blk)),
 ArbitraryWithVersion
   (BlockNodeToClientVersion blk) (SomeResult blk),
 Arbitrary
   (WithVersion (BlockNodeToClientVersion blk) (LedgerConfig blk)),
 ArbitraryWithVersion
   (QueryVersion, BlockNodeToClientVersion blk)
   (SomeSecond Query blk),
 Show (BlockNodeToClientVersion blk), BlockSupportsLedgerQuery blk,
 EncodeDisk blk blk, DecodeDisk blk (ByteString -> blk)) =>
(String -> ShouldCheckCBORValidity)
-> CodecConfig blk -> [TestTree]
roundtrip_SerialiseNodeToClient
                                                    String -> ShouldCheckCBORValidity
shouldCheckCBORvalidity CodecConfig blk
ccfg
      , String
-> (WithVersion
      (BlockNodeToNodeVersion blk) (SomeSecond (NestedCtxt Header) blk)
    -> Property)
-> TestTree
forall a. Testable a => String -> a -> TestTree
testProperty String
"envelopes"          ((WithVersion
    (BlockNodeToNodeVersion blk) (SomeSecond (NestedCtxt Header) blk)
  -> Property)
 -> TestTree)
-> (WithVersion
      (BlockNodeToNodeVersion blk) (SomeSecond (NestedCtxt Header) blk)
    -> Property)
-> TestTree
forall a b. (a -> b) -> a -> b
$ CodecConfig blk
-> WithVersion
     (BlockNodeToNodeVersion blk) (SomeSecond (NestedCtxt Header) blk)
-> Property
forall blk.
(SerialiseNodeToNode blk (SerialisedHeader blk),
 HasNestedContent Header blk) =>
CodecConfig blk
-> WithVersion
     (BlockNodeToNodeVersion blk) (SomeSecond (NestedCtxt Header) blk)
-> Property
roundtrip_envelopes             CodecConfig blk
ccfg
      , String -> (HeaderHash blk -> Property) -> TestTree
forall a. Testable a => String -> a -> TestTree
testProperty String
"ConvertRawHash"     ((HeaderHash blk -> Property) -> TestTree)
-> (HeaderHash blk -> Property) -> TestTree
forall a b. (a -> b) -> a -> b
$ Proxy blk -> HeaderHash blk -> Property
forall blk.
(StandardHash blk, ConvertRawHash blk) =>
Proxy blk -> HeaderHash blk -> Property
roundtrip_ConvertRawHash        (forall t. Proxy t
forall {k} (t :: k). Proxy t
Proxy @blk)
      , String -> (HeaderHash blk -> Property) -> TestTree
forall a. Testable a => String -> a -> TestTree
testProperty String
"hashSize"           ((HeaderHash blk -> Property) -> TestTree)
-> (HeaderHash blk -> Property) -> TestTree
forall a b. (a -> b) -> a -> b
$ Proxy blk -> HeaderHash blk -> Property
forall blk.
ConvertRawHash blk =>
Proxy blk -> HeaderHash blk -> Property
prop_hashSize                   (forall t. Proxy t
forall {k} (t :: k). Proxy t
Proxy @blk)
      , String
-> (WithVersion (BlockNodeToNodeVersion blk) (Coherent blk)
    -> Property)
-> TestTree
forall a. Testable a => String -> a -> TestTree
testProperty String
"estimateBlockSize"  ((WithVersion (BlockNodeToNodeVersion blk) (Coherent blk)
  -> Property)
 -> TestTree)
-> (WithVersion (BlockNodeToNodeVersion blk) (Coherent blk)
    -> Property)
-> TestTree
forall a b. (a -> b) -> a -> b
$ CodecConfig blk
-> WithVersion (BlockNodeToNodeVersion blk) (Coherent blk)
-> Property
forall blk.
(SerialiseNodeToNodeConstraints blk, GetHeader blk) =>
CodecConfig blk
-> WithVersion (BlockNodeToNodeVersion blk) (Coherent blk)
-> Property
prop_estimateBlockSize          CodecConfig blk
ccfg
      ]

-- TODO how can we ensure that we have a test for each constraint listed in
-- 'SerialiseDiskConstraints'?
roundtrip_SerialiseDisk
  :: forall blk.
     ( SerialiseDiskConstraints blk
     , Arbitrary' blk
     , Arbitrary' (Header blk)
     , Arbitrary' (LedgerState blk EmptyMK)
     , Arbitrary' (AnnTip blk)
     , Arbitrary' (ChainDepState (BlockProtocol blk))
     )
  => CodecConfig blk
  -> (forall a. NestedCtxt_ blk Header a -> Dict (Eq a, Show a))
  -> [TestTree]
roundtrip_SerialiseDisk :: forall blk.
(SerialiseDiskConstraints blk, Arbitrary' blk,
 Arbitrary' (Header blk), Arbitrary' (LedgerState blk EmptyMK),
 Arbitrary' (AnnTip blk),
 Arbitrary' (ChainDepState (BlockProtocol blk))) =>
CodecConfig blk
-> (forall a. NestedCtxt_ blk Header a -> Dict (Eq a, Show a))
-> [TestTree]
roundtrip_SerialiseDisk CodecConfig blk
ccfg forall a. NestedCtxt_ blk Header a -> Dict (Eq a, Show a)
dictNestedHdr =
    [ String -> (blk -> Property) -> TestTree
forall a. Testable a => String -> a -> TestTree
testProperty String
"roundtrip block" ((blk -> Property) -> TestTree) -> (blk -> Property) -> TestTree
forall a b. (a -> b) -> a -> b
$
        forall a.
(Eq a, Show a) =>
(a -> Encoding)
-> (forall s. Decoder s (ByteString -> a)) -> a -> Property
roundtrip' @blk (CodecConfig blk -> blk -> Encoding
forall blk a. EncodeDisk blk a => CodecConfig blk -> a -> Encoding
encodeDisk CodecConfig blk
ccfg) (CodecConfig blk -> forall s. Decoder s (ByteString -> blk)
forall blk a.
DecodeDisk blk a =>
CodecConfig blk -> forall s. Decoder s a
decodeDisk CodecConfig blk
ccfg)
    , String -> (Header blk -> Property) -> TestTree
forall a. Testable a => String -> a -> TestTree
testProperty String
"roundtrip Header" ((Header blk -> Property) -> TestTree)
-> (Header blk -> Property) -> TestTree
forall a b. (a -> b) -> a -> b
$ \Header blk
hdr ->
        case Header blk -> DepPair (NestedCtxt Header blk)
forall (f :: * -> *) blk.
HasNestedContent f blk =>
f blk -> DepPair (NestedCtxt f blk)
unnest Header blk
hdr of
          DepPair NestedCtxt Header blk a
ctxt a
nestedHdr -> case NestedCtxt_ blk Header a -> Dict (Eq a, Show a)
forall a. NestedCtxt_ blk Header a -> Dict (Eq a, Show a)
dictNestedHdr (NestedCtxt Header blk a -> NestedCtxt_ blk Header a
forall (f :: * -> *) blk a.
NestedCtxt f blk a -> NestedCtxt_ blk f a
flipNestedCtxt NestedCtxt Header blk a
ctxt) of
            Dict (Eq a, Show a)
Dict ->
              (a -> Encoding)
-> (forall s. Decoder s (ByteString -> a)) -> a -> Property
forall a.
(Eq a, Show a) =>
(a -> Encoding)
-> (forall s. Decoder s (ByteString -> a)) -> a -> Property
roundtrip'
                (CodecConfig blk -> NestedCtxt Header blk a -> a -> Encoding
forall a.
CodecConfig blk -> NestedCtxt Header blk a -> a -> Encoding
forall (f :: * -> * -> *) blk a.
EncodeDiskDep f blk =>
CodecConfig blk -> f blk a -> a -> Encoding
encodeDiskDep CodecConfig blk
ccfg NestedCtxt Header blk a
ctxt)
                (CodecConfig blk
-> NestedCtxt Header blk a -> forall s. Decoder s (ByteString -> a)
forall a.
CodecConfig blk
-> NestedCtxt Header blk a -> forall s. Decoder s (ByteString -> a)
forall (f :: * -> * -> *) blk a.
DecodeDiskDep f blk =>
CodecConfig blk -> f blk a -> forall s. Decoder s (ByteString -> a)
decodeDiskDep CodecConfig blk
ccfg NestedCtxt Header blk a
ctxt)
                a
nestedHdr
      -- Since the 'LedgerState' is a large data structure, we lower the
      -- number of tests to avoid slowing down the testsuite too much
    , (Int -> Int) -> TestTree -> TestTree
adjustQuickCheckTests (Int -> Int -> Int
forall a. Integral a => a -> a -> a
`div` Int
10) (TestTree -> TestTree) -> TestTree -> TestTree
forall a b. (a -> b) -> a -> b
$
      Proxy (LedgerState blk EmptyMK) -> String -> TestTree
forall a.
(Arbitrary' a, EncodeDisk blk a, DecodeDisk blk a) =>
Proxy a -> String -> TestTree
rt (forall t. Proxy t
forall {k} (t :: k). Proxy t
Proxy @(LedgerState blk EmptyMK)) String
"LedgerState"
    , Proxy (AnnTip blk) -> String -> TestTree
forall a.
(Arbitrary' a, EncodeDisk blk a, DecodeDisk blk a) =>
Proxy a -> String -> TestTree
rt (forall t. Proxy t
forall {k} (t :: k). Proxy t
Proxy @(AnnTip blk)) String
"AnnTip"
    , Proxy (ChainDepState (BlockProtocol blk)) -> String -> TestTree
forall a.
(Arbitrary' a, EncodeDisk blk a, DecodeDisk blk a) =>
Proxy a -> String -> TestTree
rt (forall t. Proxy t
forall {k} (t :: k). Proxy t
Proxy @(ChainDepState (BlockProtocol blk))) String
"ChainDepState"
    ]
  where
    rt :: forall a. (Arbitrary' a, EncodeDisk blk a, DecodeDisk blk a)
       => Proxy a -> String -> TestTree
    rt :: forall a.
(Arbitrary' a, EncodeDisk blk a, DecodeDisk blk a) =>
Proxy a -> String -> TestTree
rt Proxy a
_ String
name =
      String -> (a -> Property) -> TestTree
forall a. Testable a => String -> a -> TestTree
testProperty (String
"roundtrip " String -> ShowS
forall a. Semigroup a => a -> a -> a
<> String
name) ((a -> Property) -> TestTree) -> (a -> Property) -> TestTree
forall a b. (a -> b) -> a -> b
$
        forall a.
(Eq a, Show a) =>
(a -> Encoding) -> (forall s. Decoder s a) -> a -> Property
roundtrip @a
          (CodecConfig blk -> a -> Encoding
forall blk a. EncodeDisk blk a => CodecConfig blk -> a -> Encoding
encodeDisk CodecConfig blk
ccfg)
          (CodecConfig blk -> forall s. Decoder s a
forall blk a.
DecodeDisk blk a =>
CodecConfig blk -> forall s. Decoder s a
decodeDisk CodecConfig blk
ccfg)

-- | Used to generate arbitrary values for the serialisation roundtrip tests.
-- As the serialisation format can change with the version, not all arbitrary
-- values of the type might be valid for each version.
--
-- For example, a certain constructor can only be used after a certain version
-- and can thus not be generated for any prior versions.
data WithVersion v a = WithVersion v a
  deriving (WithVersion v a -> WithVersion v a -> Bool
(WithVersion v a -> WithVersion v a -> Bool)
-> (WithVersion v a -> WithVersion v a -> Bool)
-> Eq (WithVersion v a)
forall a. (a -> a -> Bool) -> (a -> a -> Bool) -> Eq a
forall v a.
(Eq v, Eq a) =>
WithVersion v a -> WithVersion v a -> Bool
$c== :: forall v a.
(Eq v, Eq a) =>
WithVersion v a -> WithVersion v a -> Bool
== :: WithVersion v a -> WithVersion v a -> Bool
$c/= :: forall v a.
(Eq v, Eq a) =>
WithVersion v a -> WithVersion v a -> Bool
/= :: WithVersion v a -> WithVersion v a -> Bool
Eq, Int -> WithVersion v a -> ShowS
[WithVersion v a] -> ShowS
WithVersion v a -> String
(Int -> WithVersion v a -> ShowS)
-> (WithVersion v a -> String)
-> ([WithVersion v a] -> ShowS)
-> Show (WithVersion v a)
forall a.
(Int -> a -> ShowS) -> (a -> String) -> ([a] -> ShowS) -> Show a
forall v a. (Show v, Show a) => Int -> WithVersion v a -> ShowS
forall v a. (Show v, Show a) => [WithVersion v a] -> ShowS
forall v a. (Show v, Show a) => WithVersion v a -> String
$cshowsPrec :: forall v a. (Show v, Show a) => Int -> WithVersion v a -> ShowS
showsPrec :: Int -> WithVersion v a -> ShowS
$cshow :: forall v a. (Show v, Show a) => WithVersion v a -> String
show :: WithVersion v a -> String
$cshowList :: forall v a. (Show v, Show a) => [WithVersion v a] -> ShowS
showList :: [WithVersion v a] -> ShowS
Show, (forall a b. (a -> b) -> WithVersion v a -> WithVersion v b)
-> (forall a b. a -> WithVersion v b -> WithVersion v a)
-> Functor (WithVersion v)
forall a b. a -> WithVersion v b -> WithVersion v a
forall a b. (a -> b) -> WithVersion v a -> WithVersion v b
forall v a b. a -> WithVersion v b -> WithVersion v a
forall v a b. (a -> b) -> WithVersion v a -> WithVersion v b
forall (f :: * -> *).
(forall a b. (a -> b) -> f a -> f b)
-> (forall a b. a -> f b -> f a) -> Functor f
$cfmap :: forall v a b. (a -> b) -> WithVersion v a -> WithVersion v b
fmap :: forall a b. (a -> b) -> WithVersion v a -> WithVersion v b
$c<$ :: forall v a b. a -> WithVersion v b -> WithVersion v a
<$ :: forall a b. a -> WithVersion v b -> WithVersion v a
Functor, (forall m. Monoid m => WithVersion v m -> m)
-> (forall m a. Monoid m => (a -> m) -> WithVersion v a -> m)
-> (forall m a. Monoid m => (a -> m) -> WithVersion v a -> m)
-> (forall a b. (a -> b -> b) -> b -> WithVersion v a -> b)
-> (forall a b. (a -> b -> b) -> b -> WithVersion v a -> b)
-> (forall b a. (b -> a -> b) -> b -> WithVersion v a -> b)
-> (forall b a. (b -> a -> b) -> b -> WithVersion v a -> b)
-> (forall a. (a -> a -> a) -> WithVersion v a -> a)
-> (forall a. (a -> a -> a) -> WithVersion v a -> a)
-> (forall a. WithVersion v a -> [a])
-> (forall a. WithVersion v a -> Bool)
-> (forall a. WithVersion v a -> Int)
-> (forall a. Eq a => a -> WithVersion v a -> Bool)
-> (forall a. Ord a => WithVersion v a -> a)
-> (forall a. Ord a => WithVersion v a -> a)
-> (forall a. Num a => WithVersion v a -> a)
-> (forall a. Num a => WithVersion v a -> a)
-> Foldable (WithVersion v)
forall a. Eq a => a -> WithVersion v a -> Bool
forall a. Num a => WithVersion v a -> a
forall a. Ord a => WithVersion v a -> a
forall m. Monoid m => WithVersion v m -> m
forall a. WithVersion v a -> Bool
forall a. WithVersion v a -> Int
forall a. WithVersion v a -> [a]
forall a. (a -> a -> a) -> WithVersion v a -> a
forall v a. Eq a => a -> WithVersion v a -> Bool
forall v a. Num a => WithVersion v a -> a
forall v a. Ord a => WithVersion v a -> a
forall m a. Monoid m => (a -> m) -> WithVersion v a -> m
forall v m. Monoid m => WithVersion v m -> m
forall v a. WithVersion v a -> Bool
forall v a. WithVersion v a -> Int
forall v a. WithVersion v a -> [a]
forall b a. (b -> a -> b) -> b -> WithVersion v a -> b
forall a b. (a -> b -> b) -> b -> WithVersion v a -> b
forall v a. (a -> a -> a) -> WithVersion v a -> a
forall v m a. Monoid m => (a -> m) -> WithVersion v a -> m
forall v b a. (b -> a -> b) -> b -> WithVersion v a -> b
forall v a b. (a -> b -> b) -> b -> WithVersion v a -> b
forall (t :: * -> *).
(forall m. Monoid m => t m -> m)
-> (forall m a. Monoid m => (a -> m) -> t a -> m)
-> (forall m a. Monoid m => (a -> m) -> t a -> m)
-> (forall a b. (a -> b -> b) -> b -> t a -> b)
-> (forall a b. (a -> b -> b) -> b -> t a -> b)
-> (forall b a. (b -> a -> b) -> b -> t a -> b)
-> (forall b a. (b -> a -> b) -> b -> t a -> b)
-> (forall a. (a -> a -> a) -> t a -> a)
-> (forall a. (a -> a -> a) -> t a -> a)
-> (forall a. t a -> [a])
-> (forall a. t a -> Bool)
-> (forall a. t a -> Int)
-> (forall a. Eq a => a -> t a -> Bool)
-> (forall a. Ord a => t a -> a)
-> (forall a. Ord a => t a -> a)
-> (forall a. Num a => t a -> a)
-> (forall a. Num a => t a -> a)
-> Foldable t
$cfold :: forall v m. Monoid m => WithVersion v m -> m
fold :: forall m. Monoid m => WithVersion v m -> m
$cfoldMap :: forall v m a. Monoid m => (a -> m) -> WithVersion v a -> m
foldMap :: forall m a. Monoid m => (a -> m) -> WithVersion v a -> m
$cfoldMap' :: forall v m a. Monoid m => (a -> m) -> WithVersion v a -> m
foldMap' :: forall m a. Monoid m => (a -> m) -> WithVersion v a -> m
$cfoldr :: forall v a b. (a -> b -> b) -> b -> WithVersion v a -> b
foldr :: forall a b. (a -> b -> b) -> b -> WithVersion v a -> b
$cfoldr' :: forall v a b. (a -> b -> b) -> b -> WithVersion v a -> b
foldr' :: forall a b. (a -> b -> b) -> b -> WithVersion v a -> b
$cfoldl :: forall v b a. (b -> a -> b) -> b -> WithVersion v a -> b
foldl :: forall b a. (b -> a -> b) -> b -> WithVersion v a -> b
$cfoldl' :: forall v b a. (b -> a -> b) -> b -> WithVersion v a -> b
foldl' :: forall b a. (b -> a -> b) -> b -> WithVersion v a -> b
$cfoldr1 :: forall v a. (a -> a -> a) -> WithVersion v a -> a
foldr1 :: forall a. (a -> a -> a) -> WithVersion v a -> a
$cfoldl1 :: forall v a. (a -> a -> a) -> WithVersion v a -> a
foldl1 :: forall a. (a -> a -> a) -> WithVersion v a -> a
$ctoList :: forall v a. WithVersion v a -> [a]
toList :: forall a. WithVersion v a -> [a]
$cnull :: forall v a. WithVersion v a -> Bool
null :: forall a. WithVersion v a -> Bool
$clength :: forall v a. WithVersion v a -> Int
length :: forall a. WithVersion v a -> Int
$celem :: forall v a. Eq a => a -> WithVersion v a -> Bool
elem :: forall a. Eq a => a -> WithVersion v a -> Bool
$cmaximum :: forall v a. Ord a => WithVersion v a -> a
maximum :: forall a. Ord a => WithVersion v a -> a
$cminimum :: forall v a. Ord a => WithVersion v a -> a
minimum :: forall a. Ord a => WithVersion v a -> a
$csum :: forall v a. Num a => WithVersion v a -> a
sum :: forall a. Num a => WithVersion v a -> a
$cproduct :: forall v a. Num a => WithVersion v a -> a
product :: forall a. Num a => WithVersion v a -> a
Foldable, Functor (WithVersion v)
Foldable (WithVersion v)
(Functor (WithVersion v), Foldable (WithVersion v)) =>
(forall (f :: * -> *) a b.
 Applicative f =>
 (a -> f b) -> WithVersion v a -> f (WithVersion v b))
-> (forall (f :: * -> *) a.
    Applicative f =>
    WithVersion v (f a) -> f (WithVersion v a))
-> (forall (m :: * -> *) a b.
    Monad m =>
    (a -> m b) -> WithVersion v a -> m (WithVersion v b))
-> (forall (m :: * -> *) a.
    Monad m =>
    WithVersion v (m a) -> m (WithVersion v a))
-> Traversable (WithVersion v)
forall v. Functor (WithVersion v)
forall v. Foldable (WithVersion v)
forall v (m :: * -> *) a.
Monad m =>
WithVersion v (m a) -> m (WithVersion v a)
forall v (f :: * -> *) a.
Applicative f =>
WithVersion v (f a) -> f (WithVersion v a)
forall v (m :: * -> *) a b.
Monad m =>
(a -> m b) -> WithVersion v a -> m (WithVersion v b)
forall v (f :: * -> *) a b.
Applicative f =>
(a -> f b) -> WithVersion v a -> f (WithVersion v b)
forall (t :: * -> *).
(Functor t, Foldable t) =>
(forall (f :: * -> *) a b.
 Applicative f =>
 (a -> f b) -> t a -> f (t b))
-> (forall (f :: * -> *) a. Applicative f => t (f a) -> f (t a))
-> (forall (m :: * -> *) a b.
    Monad m =>
    (a -> m b) -> t a -> m (t b))
-> (forall (m :: * -> *) a. Monad m => t (m a) -> m (t a))
-> Traversable t
forall (m :: * -> *) a.
Monad m =>
WithVersion v (m a) -> m (WithVersion v a)
forall (f :: * -> *) a.
Applicative f =>
WithVersion v (f a) -> f (WithVersion v a)
forall (m :: * -> *) a b.
Monad m =>
(a -> m b) -> WithVersion v a -> m (WithVersion v b)
forall (f :: * -> *) a b.
Applicative f =>
(a -> f b) -> WithVersion v a -> f (WithVersion v b)
$ctraverse :: forall v (f :: * -> *) a b.
Applicative f =>
(a -> f b) -> WithVersion v a -> f (WithVersion v b)
traverse :: forall (f :: * -> *) a b.
Applicative f =>
(a -> f b) -> WithVersion v a -> f (WithVersion v b)
$csequenceA :: forall v (f :: * -> *) a.
Applicative f =>
WithVersion v (f a) -> f (WithVersion v a)
sequenceA :: forall (f :: * -> *) a.
Applicative f =>
WithVersion v (f a) -> f (WithVersion v a)
$cmapM :: forall v (m :: * -> *) a b.
Monad m =>
(a -> m b) -> WithVersion v a -> m (WithVersion v b)
mapM :: forall (m :: * -> *) a b.
Monad m =>
(a -> m b) -> WithVersion v a -> m (WithVersion v b)
$csequence :: forall v (m :: * -> *) a.
Monad m =>
WithVersion v (m a) -> m (WithVersion v a)
sequence :: forall (m :: * -> *) a.
Monad m =>
WithVersion v (m a) -> m (WithVersion v a)
Traversable)

-- | Similar to @Arbitrary'@, but with an 'Arbitrary' instasnce for
-- @('WithVersion' v a)@.
type ArbitraryWithVersion v a = (Arbitrary (WithVersion v a), Eq a, Show a)

instance ( blockVersion ~ BlockNodeToClientVersion blk
         , Arbitrary blockVersion
         , Arbitrary (WithVersion (BlockNodeToClientVersion blk) (SomeBlockQuery (BlockQuery blk)))
         )
      => Arbitrary (WithVersion (QueryVersion, blockVersion) (SomeSecond Query blk)) where
  arbitrary :: Gen
  (WithVersion (QueryVersion, blockVersion) (SomeSecond Query blk))
arbitrary = do
    queryVersion <- Gen QueryVersion
forall a. Arbitrary a => Gen a
arbitrary
    case queryVersion of
      -- This case statement will cause a warning when we add a new top
      -- level query and hence a new QueryVersion. In that case we should
      -- support such top level `Query` constructors in this Arbitrary instance.
      QueryVersion
Query.QueryVersion1 -> Gen
  (WithVersion (QueryVersion, blockVersion) (SomeSecond Query blk))
genTopLevelQuery1
      QueryVersion
Query.QueryVersion2 -> Gen
  (WithVersion (QueryVersion, blockVersion) (SomeSecond Query blk))
genTopLevelQuery2
      QueryVersion
Query.QueryVersion3 -> Gen
  (WithVersion (QueryVersion, blockVersion) (SomeSecond Query blk))
genTopLevelQuery3
    where
      mkEntry ::
           QueryVersion
        -> Query blk query
        -> Gen
            (WithVersion (QueryVersion, blockVersion) (SomeSecond Query blk))
      mkEntry :: forall query.
QueryVersion
-> Query blk query
-> Gen
     (WithVersion (QueryVersion, blockVersion) (SomeSecond Query blk))
mkEntry QueryVersion
qv Query blk query
q = do
        blockV <- Gen blockVersion
forall a. Arbitrary a => Gen a
arbitrary
        return (WithVersion (qv, blockV) (SomeSecond q))

      genTopLevelQuery1 :: Gen
  (WithVersion (QueryVersion, blockVersion) (SomeSecond Query blk))
genTopLevelQuery1 =
        let version :: QueryVersion
version = QueryVersion
Query.QueryVersion1
        in  [(Int,
  Gen
    (WithVersion (QueryVersion, blockVersion) (SomeSecond Query blk)))]
-> Gen
     (WithVersion (QueryVersion, blockVersion) (SomeSecond Query blk))
forall a. HasCallStack => [(Int, Gen a)] -> Gen a
frequency
              [ (Int
15, QueryVersion
-> Gen
     (WithVersion (QueryVersion, blockVersion) (SomeSecond Query blk))
arbitraryBlockQuery QueryVersion
version    )
              , (Int
1 , QueryVersion
-> Query blk SystemStart
-> Gen
     (WithVersion (QueryVersion, blockVersion) (SomeSecond Query blk))
forall query.
QueryVersion
-> Query blk query
-> Gen
     (WithVersion (QueryVersion, blockVersion) (SomeSecond Query blk))
mkEntry QueryVersion
version Query blk SystemStart
forall blk. Query blk SystemStart
GetSystemStart )
              ]

      genTopLevelQuery2 :: Gen
  (WithVersion (QueryVersion, blockVersion) (SomeSecond Query blk))
genTopLevelQuery2 =
        let version :: QueryVersion
version = QueryVersion
Query.QueryVersion2
        in  [(Int,
  Gen
    (WithVersion (QueryVersion, blockVersion) (SomeSecond Query blk)))]
-> Gen
     (WithVersion (QueryVersion, blockVersion) (SomeSecond Query blk))
forall a. HasCallStack => [(Int, Gen a)] -> Gen a
frequency
              [ (Int
15, QueryVersion
-> Gen
     (WithVersion (QueryVersion, blockVersion) (SomeSecond Query blk))
arbitraryBlockQuery QueryVersion
version    )
              , (Int
1 , QueryVersion
-> Query blk SystemStart
-> Gen
     (WithVersion (QueryVersion, blockVersion) (SomeSecond Query blk))
forall query.
QueryVersion
-> Query blk query
-> Gen
     (WithVersion (QueryVersion, blockVersion) (SomeSecond Query blk))
mkEntry QueryVersion
version Query blk SystemStart
forall blk. Query blk SystemStart
GetSystemStart )
              , (Int
1 , QueryVersion
-> Query blk (WithOrigin BlockNo)
-> Gen
     (WithVersion (QueryVersion, blockVersion) (SomeSecond Query blk))
forall query.
QueryVersion
-> Query blk query
-> Gen
     (WithVersion (QueryVersion, blockVersion) (SomeSecond Query blk))
mkEntry QueryVersion
version Query blk (WithOrigin BlockNo)
forall blk. Query blk (WithOrigin BlockNo)
GetChainBlockNo)
              , (Int
1 , QueryVersion
-> Query blk (Point blk)
-> Gen
     (WithVersion (QueryVersion, blockVersion) (SomeSecond Query blk))
forall query.
QueryVersion
-> Query blk query
-> Gen
     (WithVersion (QueryVersion, blockVersion) (SomeSecond Query blk))
mkEntry QueryVersion
version Query blk (Point blk)
forall blk. Query blk (Point blk)
GetChainPoint  )
              ]

      genTopLevelQuery3 :: Gen
  (WithVersion (QueryVersion, blockVersion) (SomeSecond Query blk))
genTopLevelQuery3 =
        let version :: QueryVersion
version = QueryVersion
Query.QueryVersion3
        in  [(Int,
  Gen
    (WithVersion (QueryVersion, blockVersion) (SomeSecond Query blk)))]
-> Gen
     (WithVersion (QueryVersion, blockVersion) (SomeSecond Query blk))
forall a. HasCallStack => [(Int, Gen a)] -> Gen a
frequency
              [ (Int
15, QueryVersion
-> Gen
     (WithVersion (QueryVersion, blockVersion) (SomeSecond Query blk))
arbitraryBlockQuery QueryVersion
version      )
              , (Int
1 , QueryVersion
-> Query blk SystemStart
-> Gen
     (WithVersion (QueryVersion, blockVersion) (SomeSecond Query blk))
forall query.
QueryVersion
-> Query blk query
-> Gen
     (WithVersion (QueryVersion, blockVersion) (SomeSecond Query blk))
mkEntry QueryVersion
version Query blk SystemStart
forall blk. Query blk SystemStart
GetSystemStart   )
              , (Int
1 , QueryVersion
-> Query blk (WithOrigin BlockNo)
-> Gen
     (WithVersion (QueryVersion, blockVersion) (SomeSecond Query blk))
forall query.
QueryVersion
-> Query blk query
-> Gen
     (WithVersion (QueryVersion, blockVersion) (SomeSecond Query blk))
mkEntry QueryVersion
version Query blk (WithOrigin BlockNo)
forall blk. Query blk (WithOrigin BlockNo)
GetChainBlockNo  )
              , (Int
1 , QueryVersion
-> Query blk (Point blk)
-> Gen
     (WithVersion (QueryVersion, blockVersion) (SomeSecond Query blk))
forall query.
QueryVersion
-> Query blk query
-> Gen
     (WithVersion (QueryVersion, blockVersion) (SomeSecond Query blk))
mkEntry QueryVersion
version Query blk (Point blk)
forall blk. Query blk (Point blk)
GetChainPoint    )
              , (Int
1 , QueryVersion
-> Query blk (LedgerCfg (LedgerState blk))
-> Gen
     (WithVersion (QueryVersion, blockVersion) (SomeSecond Query blk))
forall query.
QueryVersion
-> Query blk query
-> Gen
     (WithVersion (QueryVersion, blockVersion) (SomeSecond Query blk))
mkEntry QueryVersion
version Query blk (LedgerCfg (LedgerState blk))
forall blk. Query blk (LedgerCfg (LedgerState blk))
DebugLedgerConfig)
              ]

      arbitraryBlockQuery :: QueryVersion
                          -> Gen (WithVersion (QueryVersion, blockVersion)
                                              (SomeSecond Query blk))
      arbitraryBlockQuery :: QueryVersion
-> Gen
     (WithVersion (QueryVersion, blockVersion) (SomeSecond Query blk))
arbitraryBlockQuery QueryVersion
queryVersion = do
        WithVersion blockV (SomeBlockQuery someBlockQuery) <- Gen (WithVersion blockVersion (SomeBlockQuery (BlockQuery blk)))
forall a. Arbitrary a => Gen a
arbitrary
        return (WithVersion (queryVersion, blockV)
                            (SomeSecond (BlockQuery someBlockQuery)))

-- | This is @OVERLAPPABLE@ because we have to override the default behaviour
-- for e.g. 'Query's.
instance {-# OVERLAPPABLE #-} (Arbitrary version, Arbitrary a)
      => Arbitrary (WithVersion version a) where
  arbitrary :: Gen (WithVersion version a)
arbitrary = version -> a -> WithVersion version a
forall v a. v -> a -> WithVersion v a
WithVersion (version -> a -> WithVersion version a)
-> Gen version -> Gen (a -> WithVersion version a)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Gen version
forall a. Arbitrary a => Gen a
arbitrary Gen (a -> WithVersion version a)
-> Gen a -> Gen (WithVersion version a)
forall a b. Gen (a -> b) -> Gen a -> Gen b
forall (f :: * -> *) a b. Applicative f => f (a -> b) -> f a -> f b
<*> Gen a
forall a. Arbitrary a => Gen a
arbitrary

-- | Used to generate slightly less arbitrary values
--
-- Like some other QuickCheck modifiers, the exact meaning is
-- context-dependent. The original motivating example is that some of our
-- serialization-adjacent properties require that the generated block contains
-- a header and a body that match, ie are /coherent/.
newtype Coherent a = Coherent { forall a. Coherent a -> a
getCoherent :: a }
  deriving (Coherent a -> Coherent a -> Bool
(Coherent a -> Coherent a -> Bool)
-> (Coherent a -> Coherent a -> Bool) -> Eq (Coherent a)
forall a. Eq a => Coherent a -> Coherent a -> Bool
forall a. (a -> a -> Bool) -> (a -> a -> Bool) -> Eq a
$c== :: forall a. Eq a => Coherent a -> Coherent a -> Bool
== :: Coherent a -> Coherent a -> Bool
$c/= :: forall a. Eq a => Coherent a -> Coherent a -> Bool
/= :: Coherent a -> Coherent a -> Bool
Eq, (forall x. Coherent a -> Rep (Coherent a) x)
-> (forall x. Rep (Coherent a) x -> Coherent a)
-> Generic (Coherent a)
forall x. Rep (Coherent a) x -> Coherent a
forall x. Coherent a -> Rep (Coherent a) x
forall a.
(forall x. a -> Rep a x) -> (forall x. Rep a x -> a) -> Generic a
forall a x. Rep (Coherent a) x -> Coherent a
forall a x. Coherent a -> Rep (Coherent a) x
$cfrom :: forall a x. Coherent a -> Rep (Coherent a) x
from :: forall x. Coherent a -> Rep (Coherent a) x
$cto :: forall a x. Rep (Coherent a) x -> Coherent a
to :: forall x. Rep (Coherent a) x -> Coherent a
Generic)
  deriving (Int -> Coherent a -> ShowS
[Coherent a] -> ShowS
Coherent a -> String
(Int -> Coherent a -> ShowS)
-> (Coherent a -> String)
-> ([Coherent a] -> ShowS)
-> Show (Coherent a)
forall a. Show a => Int -> Coherent a -> ShowS
forall a. Show a => [Coherent a] -> ShowS
forall a. Show a => Coherent a -> String
forall a.
(Int -> a -> ShowS) -> (a -> String) -> ([a] -> ShowS) -> Show a
$cshowsPrec :: forall a. Show a => Int -> Coherent a -> ShowS
showsPrec :: Int -> Coherent a -> ShowS
$cshow :: forall a. Show a => Coherent a -> String
show :: Coherent a -> String
$cshowList :: forall a. Show a => [Coherent a] -> ShowS
showList :: [Coherent a] -> ShowS
Show) via (Quiet (Coherent a))

-- TODO how can we ensure that we have a test for each constraint listed in
-- 'SerialiseNodeToNodeConstraints'?
roundtrip_SerialiseNodeToNode
  :: forall blk.
     ( SerialiseNodeToNodeConstraints blk
     , Show (BlockNodeToNodeVersion blk)
     , ArbitraryWithVersion (BlockNodeToNodeVersion blk) blk
     , ArbitraryWithVersion (BlockNodeToNodeVersion blk) (Header blk)
     , ArbitraryWithVersion (BlockNodeToNodeVersion blk) (GenTx blk)
     , ArbitraryWithVersion (BlockNodeToNodeVersion blk) (GenTxId blk)

       -- Needed for testing the @Serialised blk@
     , EncodeDisk blk blk
     , DecodeDisk blk (Lazy.ByteString -> blk)
       -- Needed for testing the @Serialised (Header blk)@
     , HasNestedContent Header blk
     , EncodeDiskDep (NestedCtxt Header) blk
     , DecodeDiskDep (NestedCtxt Header) blk
     )
  => CodecConfig blk
  -> [TestTree]
roundtrip_SerialiseNodeToNode :: forall blk.
(SerialiseNodeToNodeConstraints blk,
 Show (BlockNodeToNodeVersion blk),
 ArbitraryWithVersion (BlockNodeToNodeVersion blk) blk,
 ArbitraryWithVersion (BlockNodeToNodeVersion blk) (Header blk),
 ArbitraryWithVersion (BlockNodeToNodeVersion blk) (GenTx blk),
 ArbitraryWithVersion (BlockNodeToNodeVersion blk) (GenTxId blk),
 EncodeDisk blk blk, DecodeDisk blk (ByteString -> blk),
 HasNestedContent Header blk, EncodeDiskDep (NestedCtxt Header) blk,
 DecodeDiskDep (NestedCtxt Header) blk) =>
CodecConfig blk -> [TestTree]
roundtrip_SerialiseNodeToNode CodecConfig blk
ccfg =
    [ Proxy blk -> String -> TestTree
forall a.
(Arbitrary (WithVersion (BlockNodeToNodeVersion blk) a), Eq a,
 Show a, SerialiseNodeToNode blk a) =>
Proxy a -> String -> TestTree
rt (forall t. Proxy t
forall {k} (t :: k). Proxy t
Proxy @blk)              String
"blk"
    , Proxy (Header blk) -> String -> TestTree
forall a.
(Arbitrary (WithVersion (BlockNodeToNodeVersion blk) a), Eq a,
 Show a, SerialiseNodeToNode blk a) =>
Proxy a -> String -> TestTree
rt (forall t. Proxy t
forall {k} (t :: k). Proxy t
Proxy @(Header blk))     String
"Header"
    , Proxy (GenTx blk) -> String -> TestTree
forall a.
(Arbitrary (WithVersion (BlockNodeToNodeVersion blk) a), Eq a,
 Show a, SerialiseNodeToNode blk a) =>
Proxy a -> String -> TestTree
rt (forall t. Proxy t
forall {k} (t :: k). Proxy t
Proxy @(GenTx blk))      String
"GenTx"
    , Proxy (GenTxId blk) -> String -> TestTree
forall a.
(Arbitrary (WithVersion (BlockNodeToNodeVersion blk) a), Eq a,
 Show a, SerialiseNodeToNode blk a) =>
Proxy a -> String -> TestTree
rt (forall t. Proxy t
forall {k} (t :: k). Proxy t
Proxy @(GenTxId blk))    String
"GenTxId"
      -- Roundtrip a @'Serialised' blk@
      --
      -- We generate a random @blk@, convert it to 'Serialised' (using
      -- 'encodeDisk', which doesn't add CBOR-in-CBOR), encode it (adding
      -- CBOR-in-CBOR), decode that 'Serialised' and convert (using
      -- 'decodeNodeToNode') it to a @blk@ again.
    , String
-> (WithVersion (BlockNodeToNodeVersion blk) blk -> Property)
-> TestTree
forall a. Testable a => String -> a -> TestTree
testProperty String
"roundtrip Serialised blk" ((WithVersion (BlockNodeToNodeVersion blk) blk -> Property)
 -> TestTree)
-> (WithVersion (BlockNodeToNodeVersion blk) blk -> Property)
-> TestTree
forall a b. (a -> b) -> a -> b
$
        \(WithVersion BlockNodeToNodeVersion blk
version blk
blk) ->
          forall a.
(Eq a, Show a) =>
(a -> Encoding) -> (forall s. Decoder s a) -> a -> Property
roundtrip @blk
            ((blk -> Encoding)
-> (Serialised blk -> Encoding) -> blk -> Encoding
forall a.
(a -> Encoding) -> (Serialised a -> Encoding) -> a -> Encoding
encodeThroughSerialised (CodecConfig blk -> blk -> Encoding
forall blk a. EncodeDisk blk a => CodecConfig blk -> a -> Encoding
encodeDisk CodecConfig blk
ccfg) (BlockNodeToNodeVersion blk -> Serialised blk -> Encoding
forall a.
SerialiseNodeToNode blk a =>
BlockNodeToNodeVersion blk -> a -> Encoding
enc BlockNodeToNodeVersion blk
version))
            ((forall s. Decoder s (ByteString -> blk))
-> (forall s. Decoder s (Serialised blk))
-> forall s. Decoder s blk
forall a.
(forall s. Decoder s (ByteString -> a))
-> (forall s. Decoder s (Serialised a)) -> forall s. Decoder s a
decodeThroughSerialised (CodecConfig blk -> forall s. Decoder s (ByteString -> blk)
forall blk a.
DecodeDisk blk a =>
CodecConfig blk -> forall s. Decoder s a
decodeDisk CodecConfig blk
ccfg) (BlockNodeToNodeVersion blk -> forall s. Decoder s (Serialised blk)
forall a.
SerialiseNodeToNode blk a =>
BlockNodeToNodeVersion blk -> forall s. Decoder s a
dec BlockNodeToNodeVersion blk
version))
            blk
blk
      -- Same as above but for 'Header'
    , String
-> (WithVersion (BlockNodeToNodeVersion blk) (Header blk)
    -> Property)
-> TestTree
forall a. Testable a => String -> a -> TestTree
testProperty String
"roundtrip Serialised Header" ((WithVersion (BlockNodeToNodeVersion blk) (Header blk)
  -> Property)
 -> TestTree)
-> (WithVersion (BlockNodeToNodeVersion blk) (Header blk)
    -> Property)
-> TestTree
forall a b. (a -> b) -> a -> b
$
        \(WithVersion BlockNodeToNodeVersion blk
version Header blk
hdr) ->
          forall a.
(Eq a, Show a) =>
(a -> Encoding) -> (forall s. Decoder s a) -> a -> Property
roundtrip @(Header blk)
            (BlockNodeToNodeVersion blk -> SerialisedHeader blk -> Encoding
forall a.
SerialiseNodeToNode blk a =>
BlockNodeToNodeVersion blk -> a -> Encoding
enc BlockNodeToNodeVersion blk
version (SerialisedHeader blk -> Encoding)
-> (Header blk -> SerialisedHeader blk) -> Header blk -> Encoding
forall b c a. (b -> c) -> (a -> b) -> a -> c
. GenDepPair Serialised (NestedCtxt Header blk)
-> SerialisedHeader blk
forall blk.
GenDepPair Serialised (NestedCtxt Header blk)
-> SerialisedHeader blk
SerialisedHeaderFromDepPair (GenDepPair Serialised (NestedCtxt Header blk)
 -> SerialisedHeader blk)
-> (Header blk -> GenDepPair Serialised (NestedCtxt Header blk))
-> Header blk
-> SerialisedHeader blk
forall b c a. (b -> c) -> (a -> b) -> a -> c
. CodecConfig blk
-> DepPair (NestedCtxt Header blk)
-> GenDepPair Serialised (NestedCtxt Header blk)
forall (f :: * -> * -> *) blk.
EncodeDiskDep f blk =>
CodecConfig blk -> DepPair (f blk) -> GenDepPair Serialised (f blk)
encodeDepPair CodecConfig blk
ccfg (DepPair (NestedCtxt Header blk)
 -> GenDepPair Serialised (NestedCtxt Header blk))
-> (Header blk -> DepPair (NestedCtxt Header blk))
-> Header blk
-> GenDepPair Serialised (NestedCtxt Header blk)
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Header blk -> DepPair (NestedCtxt Header blk)
forall (f :: * -> *) blk.
HasNestedContent f blk =>
f blk -> DepPair (NestedCtxt f blk)
unnest)
            (DepPair (NestedCtxt Header blk) -> Header blk
forall (f :: * -> *) blk.
HasNestedContent f blk =>
DepPair (NestedCtxt f blk) -> f blk
nest (DepPair (NestedCtxt Header blk) -> Header blk)
-> Decoder s (DepPair (NestedCtxt Header blk))
-> Decoder s (Header blk)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> (CodecConfig blk
-> GenDepPair Serialised (NestedCtxt Header blk)
-> Decoder s (DepPair (NestedCtxt Header blk))
forall (f :: * -> * -> *) blk s.
DecodeDiskDep f blk =>
CodecConfig blk
-> GenDepPair Serialised (f blk) -> Decoder s (DepPair (f blk))
decodeDepPair CodecConfig blk
ccfg (GenDepPair Serialised (NestedCtxt Header blk)
 -> Decoder s (DepPair (NestedCtxt Header blk)))
-> (SerialisedHeader blk
    -> GenDepPair Serialised (NestedCtxt Header blk))
-> SerialisedHeader blk
-> Decoder s (DepPair (NestedCtxt Header blk))
forall b c a. (b -> c) -> (a -> b) -> a -> c
. SerialisedHeader blk
-> GenDepPair Serialised (NestedCtxt Header blk)
forall blk.
SerialisedHeader blk
-> GenDepPair Serialised (NestedCtxt Header blk)
serialisedHeaderToDepPair (SerialisedHeader blk
 -> Decoder s (DepPair (NestedCtxt Header blk)))
-> Decoder s (SerialisedHeader blk)
-> Decoder s (DepPair (NestedCtxt Header blk))
forall (m :: * -> *) a b. Monad m => (a -> m b) -> m a -> m b
=<< BlockNodeToNodeVersion blk
-> forall s. Decoder s (SerialisedHeader blk)
forall a.
SerialiseNodeToNode blk a =>
BlockNodeToNodeVersion blk -> forall s. Decoder s a
dec BlockNodeToNodeVersion blk
version))
            Header blk
hdr
      -- Check the compatibility between 'encodeNodeToNode' for @'Serialised'
      -- blk@ and 'decodeNodeToNode' for @blk@.
    , String
-> (WithVersion (BlockNodeToNodeVersion blk) blk -> Property)
-> TestTree
forall a. Testable a => String -> a -> TestTree
testProperty String
"roundtrip Serialised blk compat 1" ((WithVersion (BlockNodeToNodeVersion blk) blk -> Property)
 -> TestTree)
-> (WithVersion (BlockNodeToNodeVersion blk) blk -> Property)
-> TestTree
forall a b. (a -> b) -> a -> b
$
        \(WithVersion BlockNodeToNodeVersion blk
version blk
blk) ->
          forall a.
(Eq a, Show a) =>
(a -> Encoding) -> (forall s. Decoder s a) -> a -> Property
roundtrip @blk
            ((blk -> Encoding)
-> (Serialised blk -> Encoding) -> blk -> Encoding
forall a.
(a -> Encoding) -> (Serialised a -> Encoding) -> a -> Encoding
encodeThroughSerialised (CodecConfig blk -> blk -> Encoding
forall blk a. EncodeDisk blk a => CodecConfig blk -> a -> Encoding
encodeDisk CodecConfig blk
ccfg) (BlockNodeToNodeVersion blk -> Serialised blk -> Encoding
forall a.
SerialiseNodeToNode blk a =>
BlockNodeToNodeVersion blk -> a -> Encoding
enc BlockNodeToNodeVersion blk
version))
            (BlockNodeToNodeVersion blk -> forall s. Decoder s blk
forall a.
SerialiseNodeToNode blk a =>
BlockNodeToNodeVersion blk -> forall s. Decoder s a
dec BlockNodeToNodeVersion blk
version)
            blk
blk
      -- Check the compatibility between 'encodeNodeToNode' for @blk@ and
      -- 'decodeNodeToNode' for @'Serialised' blk@.
    , String
-> (WithVersion (BlockNodeToNodeVersion blk) blk -> Property)
-> TestTree
forall a. Testable a => String -> a -> TestTree
testProperty String
"roundtrip Serialised blk compat 2" ((WithVersion (BlockNodeToNodeVersion blk) blk -> Property)
 -> TestTree)
-> (WithVersion (BlockNodeToNodeVersion blk) blk -> Property)
-> TestTree
forall a b. (a -> b) -> a -> b
$
        \(WithVersion BlockNodeToNodeVersion blk
version blk
blk) ->
          forall a.
(Eq a, Show a) =>
(a -> Encoding) -> (forall s. Decoder s a) -> a -> Property
roundtrip @blk
            (BlockNodeToNodeVersion blk -> blk -> Encoding
forall a.
SerialiseNodeToNode blk a =>
BlockNodeToNodeVersion blk -> a -> Encoding
enc BlockNodeToNodeVersion blk
version)
            ((forall s. Decoder s (ByteString -> blk))
-> (forall s. Decoder s (Serialised blk))
-> forall s. Decoder s blk
forall a.
(forall s. Decoder s (ByteString -> a))
-> (forall s. Decoder s (Serialised a)) -> forall s. Decoder s a
decodeThroughSerialised (CodecConfig blk -> forall s. Decoder s (ByteString -> blk)
forall blk a.
DecodeDisk blk a =>
CodecConfig blk -> forall s. Decoder s a
decodeDisk CodecConfig blk
ccfg) (BlockNodeToNodeVersion blk -> forall s. Decoder s (Serialised blk)
forall a.
SerialiseNodeToNode blk a =>
BlockNodeToNodeVersion blk -> forall s. Decoder s a
dec BlockNodeToNodeVersion blk
version))
            blk
blk
      -- Same as above but for 'Header'
    , String
-> (WithVersion (BlockNodeToNodeVersion blk) (Header blk)
    -> Property)
-> TestTree
forall a. Testable a => String -> a -> TestTree
testProperty String
"roundtrip Serialised Header compat 1" ((WithVersion (BlockNodeToNodeVersion blk) (Header blk)
  -> Property)
 -> TestTree)
-> (WithVersion (BlockNodeToNodeVersion blk) (Header blk)
    -> Property)
-> TestTree
forall a b. (a -> b) -> a -> b
$
        \(WithVersion BlockNodeToNodeVersion blk
version Header blk
hdr) ->
          forall a.
(Eq a, Show a) =>
(a -> Encoding) -> (forall s. Decoder s a) -> a -> Property
roundtrip @(Header blk)
            (BlockNodeToNodeVersion blk -> SerialisedHeader blk -> Encoding
forall a.
SerialiseNodeToNode blk a =>
BlockNodeToNodeVersion blk -> a -> Encoding
enc BlockNodeToNodeVersion blk
version (SerialisedHeader blk -> Encoding)
-> (Header blk -> SerialisedHeader blk) -> Header blk -> Encoding
forall b c a. (b -> c) -> (a -> b) -> a -> c
. GenDepPair Serialised (NestedCtxt Header blk)
-> SerialisedHeader blk
forall blk.
GenDepPair Serialised (NestedCtxt Header blk)
-> SerialisedHeader blk
SerialisedHeaderFromDepPair (GenDepPair Serialised (NestedCtxt Header blk)
 -> SerialisedHeader blk)
-> (Header blk -> GenDepPair Serialised (NestedCtxt Header blk))
-> Header blk
-> SerialisedHeader blk
forall b c a. (b -> c) -> (a -> b) -> a -> c
. CodecConfig blk
-> DepPair (NestedCtxt Header blk)
-> GenDepPair Serialised (NestedCtxt Header blk)
forall (f :: * -> * -> *) blk.
EncodeDiskDep f blk =>
CodecConfig blk -> DepPair (f blk) -> GenDepPair Serialised (f blk)
encodeDepPair CodecConfig blk
ccfg (DepPair (NestedCtxt Header blk)
 -> GenDepPair Serialised (NestedCtxt Header blk))
-> (Header blk -> DepPair (NestedCtxt Header blk))
-> Header blk
-> GenDepPair Serialised (NestedCtxt Header blk)
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Header blk -> DepPair (NestedCtxt Header blk)
forall (f :: * -> *) blk.
HasNestedContent f blk =>
f blk -> DepPair (NestedCtxt f blk)
unnest)
            (BlockNodeToNodeVersion blk -> forall s. Decoder s (Header blk)
forall a.
SerialiseNodeToNode blk a =>
BlockNodeToNodeVersion blk -> forall s. Decoder s a
dec BlockNodeToNodeVersion blk
version)
            Header blk
hdr
    , String
-> (WithVersion (BlockNodeToNodeVersion blk) (Header blk)
    -> Property)
-> TestTree
forall a. Testable a => String -> a -> TestTree
testProperty String
"roundtrip Serialised Header compat 2" ((WithVersion (BlockNodeToNodeVersion blk) (Header blk)
  -> Property)
 -> TestTree)
-> (WithVersion (BlockNodeToNodeVersion blk) (Header blk)
    -> Property)
-> TestTree
forall a b. (a -> b) -> a -> b
$
        \(WithVersion BlockNodeToNodeVersion blk
version Header blk
hdr) ->
          forall a.
(Eq a, Show a) =>
(a -> Encoding) -> (forall s. Decoder s a) -> a -> Property
roundtrip @(Header blk)
            (BlockNodeToNodeVersion blk -> Header blk -> Encoding
forall a.
SerialiseNodeToNode blk a =>
BlockNodeToNodeVersion blk -> a -> Encoding
enc BlockNodeToNodeVersion blk
version)
            (DepPair (NestedCtxt Header blk) -> Header blk
forall (f :: * -> *) blk.
HasNestedContent f blk =>
DepPair (NestedCtxt f blk) -> f blk
nest (DepPair (NestedCtxt Header blk) -> Header blk)
-> Decoder s (DepPair (NestedCtxt Header blk))
-> Decoder s (Header blk)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> (CodecConfig blk
-> GenDepPair Serialised (NestedCtxt Header blk)
-> Decoder s (DepPair (NestedCtxt Header blk))
forall (f :: * -> * -> *) blk s.
DecodeDiskDep f blk =>
CodecConfig blk
-> GenDepPair Serialised (f blk) -> Decoder s (DepPair (f blk))
decodeDepPair CodecConfig blk
ccfg (GenDepPair Serialised (NestedCtxt Header blk)
 -> Decoder s (DepPair (NestedCtxt Header blk)))
-> (SerialisedHeader blk
    -> GenDepPair Serialised (NestedCtxt Header blk))
-> SerialisedHeader blk
-> Decoder s (DepPair (NestedCtxt Header blk))
forall b c a. (b -> c) -> (a -> b) -> a -> c
. SerialisedHeader blk
-> GenDepPair Serialised (NestedCtxt Header blk)
forall blk.
SerialisedHeader blk
-> GenDepPair Serialised (NestedCtxt Header blk)
serialisedHeaderToDepPair (SerialisedHeader blk
 -> Decoder s (DepPair (NestedCtxt Header blk)))
-> Decoder s (SerialisedHeader blk)
-> Decoder s (DepPair (NestedCtxt Header blk))
forall (m :: * -> *) a b. Monad m => (a -> m b) -> m a -> m b
=<< BlockNodeToNodeVersion blk
-> forall s. Decoder s (SerialisedHeader blk)
forall a.
SerialiseNodeToNode blk a =>
BlockNodeToNodeVersion blk -> forall s. Decoder s a
dec BlockNodeToNodeVersion blk
version))
            Header blk
hdr
    ]
  where
    enc :: SerialiseNodeToNode blk a
        => BlockNodeToNodeVersion blk -> a -> Encoding
    enc :: forall a.
SerialiseNodeToNode blk a =>
BlockNodeToNodeVersion blk -> a -> Encoding
enc = CodecConfig blk -> BlockNodeToNodeVersion blk -> a -> Encoding
forall blk a.
SerialiseNodeToNode blk a =>
CodecConfig blk -> BlockNodeToNodeVersion blk -> a -> Encoding
encodeNodeToNode CodecConfig blk
ccfg

    dec :: SerialiseNodeToNode blk a
        => BlockNodeToNodeVersion blk -> forall s. Decoder s a
    dec :: forall a.
SerialiseNodeToNode blk a =>
BlockNodeToNodeVersion blk -> forall s. Decoder s a
dec = CodecConfig blk
-> BlockNodeToNodeVersion blk -> forall s. Decoder s a
forall blk a.
SerialiseNodeToNode blk a =>
CodecConfig blk
-> BlockNodeToNodeVersion blk -> forall s. Decoder s a
decodeNodeToNode CodecConfig blk
ccfg

    rt
      :: forall a.
         ( Arbitrary (WithVersion (BlockNodeToNodeVersion blk) a)
         , Eq a
         , Show a
         , SerialiseNodeToNode blk a
         )
       => Proxy a -> String -> TestTree
    rt :: forall a.
(Arbitrary (WithVersion (BlockNodeToNodeVersion blk) a), Eq a,
 Show a, SerialiseNodeToNode blk a) =>
Proxy a -> String -> TestTree
rt Proxy a
_ String
name =
      String
-> (WithVersion (BlockNodeToNodeVersion blk) a -> Property)
-> TestTree
forall a. Testable a => String -> a -> TestTree
testProperty (String
"roundtrip " String -> ShowS
forall a. Semigroup a => a -> a -> a
<> String
name) ((WithVersion (BlockNodeToNodeVersion blk) a -> Property)
 -> TestTree)
-> (WithVersion (BlockNodeToNodeVersion blk) a -> Property)
-> TestTree
forall a b. (a -> b) -> a -> b
$ \(WithVersion BlockNodeToNodeVersion blk
version a
x) ->
        forall a.
(Eq a, Show a) =>
(a -> Encoding) -> (forall s. Decoder s a) -> a -> Property
roundtrip @a (BlockNodeToNodeVersion blk -> a -> Encoding
forall a.
SerialiseNodeToNode blk a =>
BlockNodeToNodeVersion blk -> a -> Encoding
enc BlockNodeToNodeVersion blk
version) (BlockNodeToNodeVersion blk -> forall s. Decoder s a
forall a.
SerialiseNodeToNode blk a =>
BlockNodeToNodeVersion blk -> forall s. Decoder s a
dec BlockNodeToNodeVersion blk
version) a
x

-- TODO how can we ensure that we have a test for each constraint listed in
-- 'SerialiseNodeToClientConstraints'?
roundtrip_SerialiseNodeToClient
  :: forall blk.
     ( SerialiseNodeToClientConstraints blk
     , ArbitraryWithVersion (BlockNodeToClientVersion blk) blk
     , ArbitraryWithVersion (BlockNodeToClientVersion blk) (GenTx blk)
     , ArbitraryWithVersion (BlockNodeToClientVersion blk) (ApplyTxErr blk)
     , ArbitraryWithVersion (BlockNodeToClientVersion blk) (SomeBlockQuery (BlockQuery blk))
     , ArbitraryWithVersion (BlockNodeToClientVersion blk) (SomeResult blk)
     , Arbitrary (WithVersion (BlockNodeToClientVersion blk) (LedgerConfig blk))
     , ArbitraryWithVersion (QueryVersion, BlockNodeToClientVersion blk) (SomeSecond Query blk)

     , Show (BlockNodeToClientVersion blk)
     , BlockSupportsLedgerQuery blk
       -- Needed for testing the @Serialised blk@
     , EncodeDisk blk blk
     , DecodeDisk blk (Lazy.ByteString -> blk)
     )
  => (TestName -> ShouldCheckCBORValidity)
  -> CodecConfig blk
  -> [TestTree]
roundtrip_SerialiseNodeToClient :: forall blk.
(SerialiseNodeToClientConstraints blk,
 ArbitraryWithVersion (BlockNodeToClientVersion blk) blk,
 ArbitraryWithVersion (BlockNodeToClientVersion blk) (GenTx blk),
 ArbitraryWithVersion
   (BlockNodeToClientVersion blk) (ApplyTxErr blk),
 ArbitraryWithVersion
   (BlockNodeToClientVersion blk) (SomeBlockQuery (BlockQuery blk)),
 ArbitraryWithVersion
   (BlockNodeToClientVersion blk) (SomeResult blk),
 Arbitrary
   (WithVersion (BlockNodeToClientVersion blk) (LedgerConfig blk)),
 ArbitraryWithVersion
   (QueryVersion, BlockNodeToClientVersion blk)
   (SomeSecond Query blk),
 Show (BlockNodeToClientVersion blk), BlockSupportsLedgerQuery blk,
 EncodeDisk blk blk, DecodeDisk blk (ByteString -> blk)) =>
(String -> ShouldCheckCBORValidity)
-> CodecConfig blk -> [TestTree]
roundtrip_SerialiseNodeToClient String -> ShouldCheckCBORValidity
shouldCheckCBORvalidity CodecConfig blk
ccfg =
    [ Proxy blk -> String -> TestTree
forall a.
(Arbitrary (WithVersion (BlockNodeToClientVersion blk) a), Eq a,
 Show a, SerialiseNodeToClient blk a) =>
Proxy a -> String -> TestTree
rt (forall t. Proxy t
forall {k} (t :: k). Proxy t
Proxy @blk)                         String
"blk"
    , Proxy (GenTx blk) -> String -> TestTree
forall a.
(Arbitrary (WithVersion (BlockNodeToClientVersion blk) a), Eq a,
 Show a, SerialiseNodeToClient blk a) =>
Proxy a -> String -> TestTree
rt (forall t. Proxy t
forall {k} (t :: k). Proxy t
Proxy @(GenTx blk))                 String
"GenTx"
    , Proxy (ApplyTxErr blk) -> String -> TestTree
forall a.
(Arbitrary (WithVersion (BlockNodeToClientVersion blk) a), Eq a,
 Show a, SerialiseNodeToClient blk a) =>
Proxy a -> String -> TestTree
rt (forall t. Proxy t
forall {k} (t :: k). Proxy t
Proxy @(ApplyTxErr blk))            String
"ApplyTxErr"
    , Proxy (SomeBlockQuery (BlockQuery blk)) -> String -> TestTree
forall a.
(Arbitrary (WithVersion (BlockNodeToClientVersion blk) a), Eq a,
 Show a, SerialiseNodeToClient blk a) =>
Proxy a -> String -> TestTree
rt (forall t. Proxy t
forall {k} (t :: k). Proxy t
Proxy @(SomeBlockQuery (BlockQuery blk))) String
"BlockQuery"
    -- Note: Ideally we'd just use 'rt' to test Ledger config, but that would
    -- require an 'Eq' and 'Show' instance for all ledger config types which
    -- we'd like to avoid (as the EpochInfo is a record of functions).
    , String -> Property -> TestTree
forall a. Testable a => String -> a -> TestTree
testProperty String
"roundtrip (comparing encoding) LedgerConfig" (Property -> TestTree) -> Property -> TestTree
forall a b. (a -> b) -> a -> b
$
        Int
-> (Blind
      (WithVersion (BlockNodeToClientVersion blk) (LedgerConfig blk))
    -> Property)
-> Property
forall prop. Testable prop => Int -> prop -> Property
withMaxSuccess Int
20 ((Blind
    (WithVersion (BlockNodeToClientVersion blk) (LedgerConfig blk))
  -> Property)
 -> Property)
-> (Blind
      (WithVersion (BlockNodeToClientVersion blk) (LedgerConfig blk))
    -> Property)
-> Property
forall a b. (a -> b) -> a -> b
$ \(Blind (WithVersion BlockNodeToClientVersion blk
version LedgerConfig blk
a)) ->
          forall a.
(a -> Encoding) -> (forall s. Decoder s a) -> a -> Property
roundtripComparingEncoding @(LedgerConfig blk) (BlockNodeToClientVersion blk -> LedgerConfig blk -> Encoding
forall a.
SerialiseNodeToClient blk a =>
BlockNodeToClientVersion blk -> a -> Encoding
enc BlockNodeToClientVersion blk
version) (BlockNodeToClientVersion blk
-> forall {s}. Decoder s (LedgerConfig blk)
forall a.
SerialiseNodeToClient blk a =>
BlockNodeToClientVersion blk -> forall s. Decoder s a
dec BlockNodeToClientVersion blk
version) LedgerConfig blk
a
    , forall a version.
(Arbitrary (WithVersion version a), Eq a, Show a, Show version) =>
(version -> a -> Encoding)
-> (version -> forall s. Decoder s a) -> String -> TestTree
rtWith
        @(SomeSecond Query blk)
        @(QueryVersion, BlockNodeToClientVersion blk)
        (\(QueryVersion
queryVersion, BlockNodeToClientVersion blk
blockVersion) SomeSecond Query blk
query -> CodecConfig blk
-> QueryVersion
-> BlockNodeToClientVersion blk
-> SomeSecond Query blk
-> Encoding
forall blk.
(SerialiseNodeToClient blk (SomeBlockQuery (BlockQuery blk)),
 Show (SomeSecond Query blk), BlockSupportsLedgerQuery blk,
 Show (BlockNodeToClientVersion blk), Typeable blk) =>
CodecConfig blk
-> QueryVersion
-> BlockNodeToClientVersion blk
-> SomeSecond Query blk
-> Encoding
Query.queryEncodeNodeToClient
                          CodecConfig blk
ccfg
                          QueryVersion
queryVersion
                          BlockNodeToClientVersion blk
blockVersion
                          SomeSecond Query blk
query
        )
        (\(QueryVersion
queryVersion, BlockNodeToClientVersion blk
blockVersion) -> CodecConfig blk
-> QueryVersion
-> BlockNodeToClientVersion blk
-> forall s. Decoder s (SomeSecond Query blk)
forall blk.
SerialiseNodeToClient blk (SomeBlockQuery (BlockQuery blk)) =>
CodecConfig blk
-> QueryVersion
-> BlockNodeToClientVersion blk
-> forall s. Decoder s (SomeSecond Query blk)
Query.queryDecodeNodeToClient
                          CodecConfig blk
ccfg
                          QueryVersion
queryVersion
                          BlockNodeToClientVersion blk
blockVersion
        )
        String
"Query"
      -- See roundtrip_SerialiseNodeToNode for more info
    , let testLabel :: String
testLabel = String
"roundtrip Serialised blk" in
        String
-> (WithVersion (BlockNodeToClientVersion blk) blk -> Property)
-> TestTree
forall a. Testable a => String -> a -> TestTree
testProperty String
testLabel ((WithVersion (BlockNodeToClientVersion blk) blk -> Property)
 -> TestTree)
-> (WithVersion (BlockNodeToClientVersion blk) blk -> Property)
-> TestTree
forall a b. (a -> b) -> a -> b
$
          \(WithVersion BlockNodeToClientVersion blk
version blk
blk) ->
            forall a.
(Eq a, Show a) =>
ShouldCheckCBORValidity
-> (a -> Encoding)
-> (forall s. Decoder s (ByteString -> a))
-> a
-> Property
roundtripAnd @blk
              (String -> ShouldCheckCBORValidity
shouldCheckCBORvalidity String
testLabel)
              ((blk -> Encoding)
-> (Serialised blk -> Encoding) -> blk -> Encoding
forall a.
(a -> Encoding) -> (Serialised a -> Encoding) -> a -> Encoding
encodeThroughSerialised (CodecConfig blk -> blk -> Encoding
forall blk a. EncodeDisk blk a => CodecConfig blk -> a -> Encoding
encodeDisk CodecConfig blk
ccfg) (BlockNodeToClientVersion blk -> Serialised blk -> Encoding
forall a.
SerialiseNodeToClient blk a =>
BlockNodeToClientVersion blk -> a -> Encoding
enc BlockNodeToClientVersion blk
version))
              (blk -> ByteString -> blk
forall a b. a -> b -> a
const (blk -> ByteString -> blk)
-> Decoder s blk -> Decoder s (ByteString -> blk)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> (forall s. Decoder s (ByteString -> blk))
-> (forall s. Decoder s (Serialised blk))
-> forall s. Decoder s blk
forall a.
(forall s. Decoder s (ByteString -> a))
-> (forall s. Decoder s (Serialised a)) -> forall s. Decoder s a
decodeThroughSerialised (CodecConfig blk -> forall s. Decoder s (ByteString -> blk)
forall blk a.
DecodeDisk blk a =>
CodecConfig blk -> forall s. Decoder s a
decodeDisk CodecConfig blk
ccfg) (BlockNodeToClientVersion blk
-> forall s. Decoder s (Serialised blk)
forall a.
SerialiseNodeToClient blk a =>
BlockNodeToClientVersion blk -> forall s. Decoder s a
dec BlockNodeToClientVersion blk
version))
              blk
blk
      -- See roundtrip_SerialiseNodeToNode for more info
    , let testLabel :: String
testLabel = String
"roundtrip Serialised blk compat" in
        String
-> (WithVersion (BlockNodeToClientVersion blk) blk -> Property)
-> TestTree
forall a. Testable a => String -> a -> TestTree
testProperty String
testLabel ((WithVersion (BlockNodeToClientVersion blk) blk -> Property)
 -> TestTree)
-> (WithVersion (BlockNodeToClientVersion blk) blk -> Property)
-> TestTree
forall a b. (a -> b) -> a -> b
$
          \(WithVersion BlockNodeToClientVersion blk
version blk
blk) ->
            forall a.
(Eq a, Show a) =>
ShouldCheckCBORValidity
-> (a -> Encoding)
-> (forall s. Decoder s (ByteString -> a))
-> a
-> Property
roundtripAnd @blk
              (String -> ShouldCheckCBORValidity
shouldCheckCBORvalidity String
testLabel)
              ((blk -> Encoding)
-> (Serialised blk -> Encoding) -> blk -> Encoding
forall a.
(a -> Encoding) -> (Serialised a -> Encoding) -> a -> Encoding
encodeThroughSerialised (CodecConfig blk -> blk -> Encoding
forall blk a. EncodeDisk blk a => CodecConfig blk -> a -> Encoding
encodeDisk CodecConfig blk
ccfg) (BlockNodeToClientVersion blk -> Serialised blk -> Encoding
forall a.
SerialiseNodeToClient blk a =>
BlockNodeToClientVersion blk -> a -> Encoding
enc BlockNodeToClientVersion blk
version))
              (blk -> ByteString -> blk
forall a b. a -> b -> a
const (blk -> ByteString -> blk)
-> Decoder s blk -> Decoder s (ByteString -> blk)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> BlockNodeToClientVersion blk -> forall s. Decoder s blk
forall a.
SerialiseNodeToClient blk a =>
BlockNodeToClientVersion blk -> forall s. Decoder s a
dec BlockNodeToClientVersion blk
version)
              blk
blk
    , let testLabel :: String
testLabel = String
"roundtrip Result" in
        String
-> (WithVersion (BlockNodeToClientVersion blk) (SomeResult blk)
    -> Property)
-> TestTree
forall a. Testable a => String -> a -> TestTree
testProperty String
testLabel ((WithVersion (BlockNodeToClientVersion blk) (SomeResult blk)
  -> Property)
 -> TestTree)
-> (WithVersion (BlockNodeToClientVersion blk) (SomeResult blk)
    -> Property)
-> TestTree
forall a b. (a -> b) -> a -> b
$
          \(WithVersion BlockNodeToClientVersion blk
version (SomeResult BlockQuery blk fp result
query result
result :: SomeResult blk)) ->
            ShouldCheckCBORValidity
-> (result -> Encoding)
-> (forall s. Decoder s (ByteString -> result))
-> result
-> Property
forall a.
(Eq a, Show a) =>
ShouldCheckCBORValidity
-> (a -> Encoding)
-> (forall s. Decoder s (ByteString -> a))
-> a
-> Property
roundtripAnd
              (String -> ShouldCheckCBORValidity
shouldCheckCBORvalidity String
testLabel)
              (CodecConfig blk
-> BlockNodeToClientVersion blk
-> BlockQuery blk fp result
-> result
-> Encoding
forall k blk (query :: * -> k -> * -> *) (fp :: k) result.
SerialiseBlockQueryResult blk query =>
CodecConfig blk
-> BlockNodeToClientVersion blk
-> query blk fp result
-> result
-> Encoding
forall (fp :: QueryFootprint) result.
CodecConfig blk
-> BlockNodeToClientVersion blk
-> BlockQuery blk fp result
-> result
-> Encoding
encodeBlockQueryResult CodecConfig blk
ccfg BlockNodeToClientVersion blk
version BlockQuery blk fp result
query)
              (result -> ByteString -> result
forall a b. a -> b -> a
const (result -> ByteString -> result)
-> Decoder s result -> Decoder s (ByteString -> result)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> CodecConfig blk
-> BlockNodeToClientVersion blk
-> BlockQuery blk fp result
-> forall s. Decoder s result
forall k blk (query :: * -> k -> * -> *) (fp :: k) result.
SerialiseBlockQueryResult blk query =>
CodecConfig blk
-> BlockNodeToClientVersion blk
-> query blk fp result
-> forall s. Decoder s result
forall (fp :: QueryFootprint) result.
CodecConfig blk
-> BlockNodeToClientVersion blk
-> BlockQuery blk fp result
-> forall s. Decoder s result
decodeBlockQueryResult CodecConfig blk
ccfg BlockNodeToClientVersion blk
version BlockQuery blk fp result
query)
              result
result
    ]
  where
    enc :: SerialiseNodeToClient blk a
        => BlockNodeToClientVersion blk -> a -> Encoding
    enc :: forall a.
SerialiseNodeToClient blk a =>
BlockNodeToClientVersion blk -> a -> Encoding
enc = CodecConfig blk -> BlockNodeToClientVersion blk -> a -> Encoding
forall blk a.
SerialiseNodeToClient blk a =>
CodecConfig blk -> BlockNodeToClientVersion blk -> a -> Encoding
encodeNodeToClient CodecConfig blk
ccfg

    dec :: SerialiseNodeToClient blk a
        => BlockNodeToClientVersion blk -> forall s. Decoder s a
    dec :: forall a.
SerialiseNodeToClient blk a =>
BlockNodeToClientVersion blk -> forall s. Decoder s a
dec = CodecConfig blk
-> BlockNodeToClientVersion blk -> forall s. Decoder s a
forall blk a.
SerialiseNodeToClient blk a =>
CodecConfig blk
-> BlockNodeToClientVersion blk -> forall s. Decoder s a
decodeNodeToClient CodecConfig blk
ccfg

    rt
      :: forall a.
         ( Arbitrary (WithVersion (BlockNodeToClientVersion blk) a)
         , Eq a
         , Show a
         , SerialiseNodeToClient blk a
         )
       => Proxy a -> String -> TestTree
    rt :: forall a.
(Arbitrary (WithVersion (BlockNodeToClientVersion blk) a), Eq a,
 Show a, SerialiseNodeToClient blk a) =>
Proxy a -> String -> TestTree
rt Proxy a
_ String
name = (BlockNodeToClientVersion blk -> a -> Encoding)
-> (BlockNodeToClientVersion blk -> forall s. Decoder s a)
-> String
-> TestTree
forall a version.
(Arbitrary (WithVersion version a), Eq a, Show a, Show version) =>
(version -> a -> Encoding)
-> (version -> forall s. Decoder s a) -> String -> TestTree
rtWith (forall a.
SerialiseNodeToClient blk a =>
BlockNodeToClientVersion blk -> a -> Encoding
enc @a) (forall a.
SerialiseNodeToClient blk a =>
BlockNodeToClientVersion blk -> forall s. Decoder s a
dec @a) String
name

    rtWith
      :: forall a version.
         ( Arbitrary (WithVersion version a)
         , Eq a
         , Show a
         , Show version
         )
       => (version -> a -> Encoding)
       -> (version -> forall s. Decoder s a)
       -> String
       -> TestTree
    rtWith :: forall a version.
(Arbitrary (WithVersion version a), Eq a, Show a, Show version) =>
(version -> a -> Encoding)
-> (version -> forall s. Decoder s a) -> String -> TestTree
rtWith version -> a -> Encoding
enc' version -> forall s. Decoder s a
dec' String
name =
        String -> (WithVersion version a -> Property) -> TestTree
forall a. Testable a => String -> a -> TestTree
testProperty (String
"roundtrip " String -> ShowS
forall a. Semigroup a => a -> a -> a
<> String
name) ((WithVersion version a -> Property) -> TestTree)
-> (WithVersion version a -> Property) -> TestTree
forall a b. (a -> b) -> a -> b
$
          \(WithVersion version
version a
a) ->
            forall a.
(Eq a, Show a) =>
ShouldCheckCBORValidity
-> (a -> Encoding)
-> (forall s. Decoder s (ByteString -> a))
-> a
-> Property
roundtripAnd @a (String -> ShouldCheckCBORValidity
shouldCheckCBORvalidity String
testLabel)
                            (version -> a -> Encoding
enc' version
version)
                            (a -> ByteString -> a
forall a b. a -> b -> a
const (a -> ByteString -> a)
-> Decoder s a -> Decoder s (ByteString -> a)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> version -> forall s. Decoder s a
dec' version
version)
                            a
a
      where
        testLabel :: String
testLabel = String
"roundtrip " String -> ShowS
forall a. Semigroup a => a -> a -> a
<> String
name

{-------------------------------------------------------------------------------
  Checking envelopes
-------------------------------------------------------------------------------}

-- | This is similar to the roundtrip tests for headers, except we don't
-- start with a header but some fixed bytestring in the payload. This makes
-- debugging a bit easier as we can focus on just the envelope.
roundtrip_envelopes ::
     forall blk. (
       SerialiseNodeToNode blk (SerialisedHeader blk)
     , HasNestedContent Header blk
     )
  => CodecConfig blk
  -> WithVersion (BlockNodeToNodeVersion blk) (SomeSecond (NestedCtxt Header) blk)
  -> Property
roundtrip_envelopes :: forall blk.
(SerialiseNodeToNode blk (SerialisedHeader blk),
 HasNestedContent Header blk) =>
CodecConfig blk
-> WithVersion
     (BlockNodeToNodeVersion blk) (SomeSecond (NestedCtxt Header) blk)
-> Property
roundtrip_envelopes CodecConfig blk
ccfg (WithVersion BlockNodeToNodeVersion blk
v (SomeSecond NestedCtxt Header blk b
ctxt)) =
    (Base16 (SerialisedHeader blk) -> Encoding)
-> (forall s. Decoder s (Base16 (SerialisedHeader blk)))
-> Base16 (SerialisedHeader blk)
-> Property
forall a.
(Eq a, Show a) =>
(a -> Encoding) -> (forall s. Decoder s a) -> a -> Property
roundtrip
      (CodecConfig blk
-> BlockNodeToNodeVersion blk -> SerialisedHeader blk -> Encoding
forall blk a.
SerialiseNodeToNode blk a =>
CodecConfig blk -> BlockNodeToNodeVersion blk -> a -> Encoding
encodeNodeToNode CodecConfig blk
ccfg BlockNodeToNodeVersion blk
v (SerialisedHeader blk -> Encoding)
-> (Base16 (SerialisedHeader blk) -> SerialisedHeader blk)
-> Base16 (SerialisedHeader blk)
-> Encoding
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Base16 (SerialisedHeader blk) -> SerialisedHeader blk
forall a. Base16 a -> a
unBase16)
      (SerialisedHeader blk -> Base16 (SerialisedHeader blk)
forall a. a -> Base16 a
Base16 (SerialisedHeader blk -> Base16 (SerialisedHeader blk))
-> Decoder s (SerialisedHeader blk)
-> Decoder s (Base16 (SerialisedHeader blk))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> CodecConfig blk
-> BlockNodeToNodeVersion blk
-> forall s. Decoder s (SerialisedHeader blk)
forall blk a.
SerialiseNodeToNode blk a =>
CodecConfig blk
-> BlockNodeToNodeVersion blk -> forall s. Decoder s a
decodeNodeToNode CodecConfig blk
ccfg BlockNodeToNodeVersion blk
v)
      (SerialisedHeader blk -> Base16 (SerialisedHeader blk)
forall a. a -> Base16 a
Base16 SerialisedHeader blk
serialisedHeader)
  where
    serialisedHeader :: SerialisedHeader blk
    serialisedHeader :: SerialisedHeader blk
serialisedHeader = GenDepPair Serialised (NestedCtxt Header blk)
-> SerialisedHeader blk
forall blk.
GenDepPair Serialised (NestedCtxt Header blk)
-> SerialisedHeader blk
SerialisedHeaderFromDepPair (GenDepPair Serialised (NestedCtxt Header blk)
 -> SerialisedHeader blk)
-> GenDepPair Serialised (NestedCtxt Header blk)
-> SerialisedHeader blk
forall a b. (a -> b) -> a -> b
$
        NestedCtxt Header blk b
-> Serialised b -> GenDepPair Serialised (NestedCtxt Header blk)
forall {k} (f :: k -> *) (a :: k) (g :: k -> *).
f a -> g a -> GenDepPair g f
GenDepPair NestedCtxt Header blk b
ctxt (ByteString -> Serialised b
forall {k} (a :: k). ByteString -> Serialised a
Serialised ByteString
bs)

    bs :: Lazy.ByteString
    bs :: ByteString
bs = ByteString
"<PAYLOAD>" -- Something we can easily recognize in test failures

newtype Base16 a = Base16 { forall a. Base16 a -> a
unBase16 :: a }

instance HasNestedContent Header blk => Show (Base16 (SerialisedHeader blk)) where
  show :: Base16 (SerialisedHeader blk) -> String
show = GenDepPair Serialised (NestedCtxt Header blk) -> String
aux (GenDepPair Serialised (NestedCtxt Header blk) -> String)
-> (Base16 (SerialisedHeader blk)
    -> GenDepPair Serialised (NestedCtxt Header blk))
-> Base16 (SerialisedHeader blk)
-> String
forall b c a. (b -> c) -> (a -> b) -> a -> c
. SerialisedHeader blk
-> GenDepPair Serialised (NestedCtxt Header blk)
forall blk.
SerialisedHeader blk
-> GenDepPair Serialised (NestedCtxt Header blk)
serialisedHeaderToDepPair (SerialisedHeader blk
 -> GenDepPair Serialised (NestedCtxt Header blk))
-> (Base16 (SerialisedHeader blk) -> SerialisedHeader blk)
-> Base16 (SerialisedHeader blk)
-> GenDepPair Serialised (NestedCtxt Header blk)
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Base16 (SerialisedHeader blk) -> SerialisedHeader blk
forall a. Base16 a -> a
unBase16
    where
      aux :: GenDepPair Serialised (NestedCtxt Header blk) -> String
      aux :: GenDepPair Serialised (NestedCtxt Header blk) -> String
aux (GenDepPair NestedCtxt Header blk a
ctxt (Serialised ByteString
bs)) =
          String
"(" String -> ShowS
forall a. Semigroup a => a -> a -> a
<> NestedCtxt Header blk a -> String
forall a. Show a => a -> String
show NestedCtxt Header blk a
ctxt String -> ShowS
forall a. Semigroup a => a -> a -> a
<> String
"," String -> ShowS
forall a. Semigroup a => a -> a -> a
<> ByteString -> String
Char8.unpack (ByteString -> ByteString
Base16.encode ByteString
bs) String -> ShowS
forall a. Semigroup a => a -> a -> a
<> String
")"

instance HasNestedContent Header blk => Eq (Base16 (SerialisedHeader blk)) where
  == :: Base16 (SerialisedHeader blk)
-> Base16 (SerialisedHeader blk) -> Bool
(==) = GenDepPair Serialised (NestedCtxt Header blk)
-> GenDepPair Serialised (NestedCtxt Header blk) -> Bool
aux (GenDepPair Serialised (NestedCtxt Header blk)
 -> GenDepPair Serialised (NestedCtxt Header blk) -> Bool)
-> (Base16 (SerialisedHeader blk)
    -> GenDepPair Serialised (NestedCtxt Header blk))
-> Base16 (SerialisedHeader blk)
-> Base16 (SerialisedHeader blk)
-> Bool
forall b c a. (b -> b -> c) -> (a -> b) -> a -> a -> c
`on` (SerialisedHeader blk
-> GenDepPair Serialised (NestedCtxt Header blk)
forall blk.
SerialisedHeader blk
-> GenDepPair Serialised (NestedCtxt Header blk)
serialisedHeaderToDepPair (SerialisedHeader blk
 -> GenDepPair Serialised (NestedCtxt Header blk))
-> (Base16 (SerialisedHeader blk) -> SerialisedHeader blk)
-> Base16 (SerialisedHeader blk)
-> GenDepPair Serialised (NestedCtxt Header blk)
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Base16 (SerialisedHeader blk) -> SerialisedHeader blk
forall a. Base16 a -> a
unBase16)
    where
      aux :: GenDepPair Serialised (NestedCtxt Header blk)
          -> GenDepPair Serialised (NestedCtxt Header blk)
          -> Bool
      aux :: GenDepPair Serialised (NestedCtxt Header blk)
-> GenDepPair Serialised (NestedCtxt Header blk) -> Bool
aux (GenDepPair NestedCtxt Header blk a
ctxt Serialised a
bs) (GenDepPair NestedCtxt Header blk a
ctxt' Serialised a
bs') =
          case NestedCtxt Header blk a
-> NestedCtxt Header blk a -> Maybe (a :~: a)
forall a b.
NestedCtxt Header blk a
-> NestedCtxt Header blk b -> Maybe (a :~: b)
forall k (f :: k -> *) (a :: k) (b :: k).
SameDepIndex f =>
f a -> f b -> Maybe (a :~: b)
sameDepIndex NestedCtxt Header blk a
ctxt NestedCtxt Header blk a
ctxt' of
            Just a :~: a
Refl -> Serialised a
bs Serialised a -> Serialised a -> Bool
forall a. Eq a => a -> a -> Bool
== Serialised a
Serialised a
bs'
            Maybe (a :~: a)
Nothing   -> Bool
False

{-------------------------------------------------------------------------------
  ConvertRawHash
-------------------------------------------------------------------------------}

roundtrip_ConvertRawHash
  :: (StandardHash blk, ConvertRawHash blk)
  => Proxy blk -> HeaderHash blk -> Property
roundtrip_ConvertRawHash :: forall blk.
(StandardHash blk, ConvertRawHash blk) =>
Proxy blk -> HeaderHash blk -> Property
roundtrip_ConvertRawHash Proxy blk
p HeaderHash blk
h =
    HeaderHash blk
h HeaderHash blk -> HeaderHash blk -> Property
forall a. (Eq a, Show a) => a -> a -> Property
=== Proxy blk -> ShortByteString -> HeaderHash blk
forall blk (proxy :: * -> *).
ConvertRawHash blk =>
proxy blk -> ShortByteString -> HeaderHash blk
forall (proxy :: * -> *).
proxy blk -> ShortByteString -> HeaderHash blk
fromShortRawHash Proxy blk
p (Proxy blk -> HeaderHash blk -> ShortByteString
forall blk (proxy :: * -> *).
ConvertRawHash blk =>
proxy blk -> HeaderHash blk -> ShortByteString
forall (proxy :: * -> *).
proxy blk -> HeaderHash blk -> ShortByteString
toShortRawHash Proxy blk
p HeaderHash blk
h)

prop_hashSize
  :: ConvertRawHash blk
  => Proxy blk -> HeaderHash blk -> Property
prop_hashSize :: forall blk.
ConvertRawHash blk =>
Proxy blk -> HeaderHash blk -> Property
prop_hashSize Proxy blk
p HeaderHash blk
h =
    Proxy blk -> Word32
forall blk (proxy :: * -> *).
ConvertRawHash blk =>
proxy blk -> Word32
forall (proxy :: * -> *). proxy blk -> Word32
hashSize Proxy blk
p Word32 -> Word32 -> Property
forall a. (Eq a, Show a) => a -> a -> Property
=== Int -> Word32
forall a b. (Integral a, Num b) => a -> b
fromIntegral (ShortByteString -> Int
Short.length (Proxy blk -> HeaderHash blk -> ShortByteString
forall blk (proxy :: * -> *).
ConvertRawHash blk =>
proxy blk -> HeaderHash blk -> ShortByteString
forall (proxy :: * -> *).
proxy blk -> HeaderHash blk -> ShortByteString
toShortRawHash Proxy blk
p HeaderHash blk
h))

{-------------------------------------------------------------------------------
  estimateBlockSize
-------------------------------------------------------------------------------}

prop_estimateBlockSize ::
     (SerialiseNodeToNodeConstraints blk, GetHeader blk)
  => CodecConfig blk
  -> WithVersion (BlockNodeToNodeVersion blk) (Coherent blk)
  -> Property
prop_estimateBlockSize :: forall blk.
(SerialiseNodeToNodeConstraints blk, GetHeader blk) =>
CodecConfig blk
-> WithVersion (BlockNodeToNodeVersion blk) (Coherent blk)
-> Property
prop_estimateBlockSize CodecConfig blk
ccfg (WithVersion BlockNodeToNodeVersion blk
version (Coherent blk
blk))
  | SizeInBytes
actualBlockSize SizeInBytes -> SizeInBytes -> Bool
forall a. Ord a => a -> a -> Bool
> SizeInBytes
expectedBlockSize
  = String -> Property -> Property
forall prop. Testable prop => String -> prop -> Property
counterexample
      (String
"actualBlockSize > expectedBlockSize: "
         String -> ShowS
forall a. Semigroup a => a -> a -> a
<> SizeInBytes -> String
forall a. Show a => a -> String
show SizeInBytes
actualBlockSize String -> ShowS
forall a. Semigroup a => a -> a -> a
<> String
" > "
         String -> ShowS
forall a. Semigroup a => a -> a -> a
<> SizeInBytes -> String
forall a. Show a => a -> String
show SizeInBytes
expectedBlockSize)
      (Bool -> Property
forall prop. Testable prop => prop -> Property
property Bool
False)
  | SizeInBytes
actualBlockSize SizeInBytes -> SizeInBytes -> Bool
forall a. Ord a => a -> a -> Bool
< SizeInBytes
expectedBlockSize SizeInBytes -> SizeInBytes -> SizeInBytes
forall a. Num a => a -> a -> a
- SizeInBytes
allowedOverestimate
  = String -> Property -> Property
forall prop. Testable prop => String -> prop -> Property
counterexample
      (String
"actualBlockSize < expectedBlockSize - allowedOverestimate: "
         String -> ShowS
forall a. Semigroup a => a -> a -> a
<> SizeInBytes -> String
forall a. Show a => a -> String
show SizeInBytes
actualBlockSize String -> ShowS
forall a. Semigroup a => a -> a -> a
<> String
" > "
         String -> ShowS
forall a. Semigroup a => a -> a -> a
<> SizeInBytes -> String
forall a. Show a => a -> String
show SizeInBytes
expectedBlockSize String -> ShowS
forall a. Semigroup a => a -> a -> a
<> String
" - "
         String -> ShowS
forall a. Semigroup a => a -> a -> a
<> SizeInBytes -> String
forall a. Show a => a -> String
show SizeInBytes
allowedOverestimate)
      (Bool -> Property
forall prop. Testable prop => prop -> Property
property Bool
False)
  | Bool
otherwise
  = Bool -> String -> Property -> Property
forall prop. Testable prop => Bool -> String -> prop -> Property
classify (SizeInBytes
actualBlockSize SizeInBytes -> SizeInBytes -> Bool
forall a. Eq a => a -> a -> Bool
== SizeInBytes
expectedBlockSize) String
"exact"
  (Property -> Property) -> Property -> Property
forall a b. (a -> b) -> a -> b
$ Bool -> String -> Property -> Property
forall prop. Testable prop => Bool -> String -> prop -> Property
classify (SizeInBytes
actualBlockSize SizeInBytes -> SizeInBytes -> Bool
forall a. Ord a => a -> a -> Bool
<  SizeInBytes
expectedBlockSize) String
"overestimate"
  (Property -> Property) -> Property -> Property
forall a b. (a -> b) -> a -> b
$ Bool -> Property
forall prop. Testable prop => prop -> Property
property Bool
True
  where
    allowedOverestimate :: SizeInBytes
    allowedOverestimate :: SizeInBytes
allowedOverestimate = SizeInBytes
10

    actualBlockSize :: SizeInBytes
    actualBlockSize :: SizeInBytes
actualBlockSize =
          Int64 -> SizeInBytes
forall a b. (Integral a, Num b) => a -> b
fromIntegral
        (Int64 -> SizeInBytes) -> (blk -> Int64) -> blk -> SizeInBytes
forall b c a. (b -> c) -> (a -> b) -> a -> c
. ByteString -> Int64
Lazy.length
        (ByteString -> Int64) -> (blk -> ByteString) -> blk -> Int64
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Encoding -> ByteString
toLazyByteString
        (Encoding -> ByteString) -> (blk -> Encoding) -> blk -> ByteString
forall b c a. (b -> c) -> (a -> b) -> a -> c
. CodecConfig blk -> BlockNodeToNodeVersion blk -> blk -> Encoding
forall blk a.
SerialiseNodeToNode blk a =>
CodecConfig blk -> BlockNodeToNodeVersion blk -> a -> Encoding
encodeNodeToNode CodecConfig blk
ccfg BlockNodeToNodeVersion blk
version
        (blk -> SizeInBytes) -> blk -> SizeInBytes
forall a b. (a -> b) -> a -> b
$ blk
blk

    expectedBlockSize :: SizeInBytes
    expectedBlockSize :: SizeInBytes
expectedBlockSize =
          Header blk -> SizeInBytes
forall blk.
SerialiseNodeToNodeConstraints blk =>
Header blk -> SizeInBytes
estimateBlockSize
        (Header blk -> SizeInBytes)
-> (blk -> Header blk) -> blk -> SizeInBytes
forall b c a. (b -> c) -> (a -> b) -> a -> c
. blk -> Header blk
forall blk. GetHeader blk => blk -> Header blk
getHeader
        (blk -> SizeInBytes) -> blk -> SizeInBytes
forall a b. (a -> b) -> a -> b
$ blk
blk

{-------------------------------------------------------------------------------
  Serialised helpers
-------------------------------------------------------------------------------}

encodeThroughSerialised ::
     (a -> Encoding)
  -> (Serialised a -> Encoding)
  -> (a -> Encoding)
encodeThroughSerialised :: forall a.
(a -> Encoding) -> (Serialised a -> Encoding) -> a -> Encoding
encodeThroughSerialised a -> Encoding
enc Serialised a -> Encoding
encSerialised = Serialised a -> Encoding
encSerialised (Serialised a -> Encoding) -> (a -> Serialised a) -> a -> Encoding
forall b c a. (b -> c) -> (a -> b) -> a -> c
. (a -> Encoding) -> a -> Serialised a
forall a. (a -> Encoding) -> a -> Serialised a
mkSerialised a -> Encoding
enc

decodeThroughSerialised ::
     (forall s. Decoder s (Lazy.ByteString -> a))
  -> (forall s. Decoder s (Serialised a))
  -> (forall s. Decoder s a)
decodeThroughSerialised :: forall a.
(forall s. Decoder s (ByteString -> a))
-> (forall s. Decoder s (Serialised a)) -> forall s. Decoder s a
decodeThroughSerialised forall s. Decoder s (ByteString -> a)
dec forall s. Decoder s (Serialised a)
decSerialised = do
    serialised <- Decoder s (Serialised a)
forall s. Decoder s (Serialised a)
decSerialised
    fromSerialised dec serialised

{------------------------------------------------------------------------------
  Roundtrip tests for examples
------------------------------------------------------------------------------}

examplesRoundtrip ::
     forall blk . (SerialiseDiskConstraints blk, Eq blk, Show blk, LedgerSupportsProtocol blk)
  => CodecConfig blk
  -> Examples blk
  -> [TestTree]
examplesRoundtrip :: forall blk.
(SerialiseDiskConstraints blk, Eq blk, Show blk,
 LedgerSupportsProtocol blk) =>
CodecConfig blk -> Examples blk -> [TestTree]
examplesRoundtrip CodecConfig blk
codecConfig Examples blk
examples =
    [ String
-> (blk -> Encoding)
-> (forall s. Decoder s (ByteString -> blk))
-> (Examples blk -> Labelled blk)
-> TestTree
forall a.
(Eq a, Show a) =>
String
-> (a -> Encoding)
-> (forall s. Decoder s (ByteString -> a))
-> (Examples blk -> Labelled a)
-> TestTree
testRoundtripFor String
"Block"                 (CodecConfig blk -> blk -> Encoding
forall blk a. EncodeDisk blk a => CodecConfig blk -> a -> Encoding
encodeDisk CodecConfig blk
codecConfig) (CodecConfig blk -> forall s. Decoder s (ByteString -> blk)
forall blk a.
DecodeDisk blk a =>
CodecConfig blk -> forall s. Decoder s a
decodeDisk CodecConfig blk
codecConfig)           Examples blk -> Labelled blk
forall blk. Examples blk -> Labelled blk
exampleBlock
    , String
-> (HeaderHash blk -> Encoding)
-> (forall s. Decoder s (ByteString -> HeaderHash blk))
-> (Examples blk -> Labelled (HeaderHash blk))
-> TestTree
forall a.
(Eq a, Show a) =>
String
-> (a -> Encoding)
-> (forall s. Decoder s (ByteString -> a))
-> (Examples blk -> Labelled a)
-> TestTree
testRoundtripFor String
"Header hash"           HeaderHash blk -> Encoding
forall a. Serialise a => a -> Encoding
encode                   (HeaderHash blk -> ByteString -> HeaderHash blk
forall a b. a -> b -> a
const (HeaderHash blk -> ByteString -> HeaderHash blk)
-> Decoder s (HeaderHash blk)
-> Decoder s (ByteString -> HeaderHash blk)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Decoder s (HeaderHash blk)
forall s. Decoder s (HeaderHash blk)
forall a s. Serialise a => Decoder s a
decode)                 Examples blk -> Labelled (HeaderHash blk)
forall blk. Examples blk -> Labelled (HeaderHash blk)
exampleHeaderHash
    , String
-> (LedgerState blk EmptyMK -> Encoding)
-> (forall s. Decoder s (ByteString -> LedgerState blk EmptyMK))
-> (Examples blk -> Labelled (LedgerState blk EmptyMK))
-> TestTree
forall a.
(Eq a, Show a) =>
String
-> (a -> Encoding)
-> (forall s. Decoder s (ByteString -> a))
-> (Examples blk -> Labelled a)
-> TestTree
testRoundtripFor String
"Ledger state"          (CodecConfig blk -> LedgerState blk EmptyMK -> Encoding
forall blk a. EncodeDisk blk a => CodecConfig blk -> a -> Encoding
encodeDisk CodecConfig blk
codecConfig) (LedgerState blk EmptyMK -> ByteString -> LedgerState blk EmptyMK
forall a b. a -> b -> a
const (LedgerState blk EmptyMK -> ByteString -> LedgerState blk EmptyMK)
-> Decoder s (LedgerState blk EmptyMK)
-> Decoder s (ByteString -> LedgerState blk EmptyMK)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> CodecConfig blk -> forall s. Decoder s (LedgerState blk EmptyMK)
forall blk a.
DecodeDisk blk a =>
CodecConfig blk -> forall s. Decoder s a
decodeDisk CodecConfig blk
codecConfig) Examples blk -> Labelled (LedgerState blk EmptyMK)
forall blk. Examples blk -> Labelled (LedgerState blk EmptyMK)
exampleLedgerState
    , String
-> (AnnTip blk -> Encoding)
-> (forall s. Decoder s (ByteString -> AnnTip blk))
-> (Examples blk -> Labelled (AnnTip blk))
-> TestTree
forall a.
(Eq a, Show a) =>
String
-> (a -> Encoding)
-> (forall s. Decoder s (ByteString -> a))
-> (Examples blk -> Labelled a)
-> TestTree
testRoundtripFor String
"Annotated tip"         (CodecConfig blk -> AnnTip blk -> Encoding
forall blk a. EncodeDisk blk a => CodecConfig blk -> a -> Encoding
encodeDisk CodecConfig blk
codecConfig) (AnnTip blk -> ByteString -> AnnTip blk
forall a b. a -> b -> a
const (AnnTip blk -> ByteString -> AnnTip blk)
-> Decoder s (AnnTip blk) -> Decoder s (ByteString -> AnnTip blk)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> CodecConfig blk -> forall s. Decoder s (AnnTip blk)
forall blk a.
DecodeDisk blk a =>
CodecConfig blk -> forall s. Decoder s a
decodeDisk CodecConfig blk
codecConfig) Examples blk -> Labelled (AnnTip blk)
forall blk. Examples blk -> Labelled (AnnTip blk)
exampleAnnTip
    , String
-> (ChainDepState (BlockProtocol blk) -> Encoding)
-> (forall s.
    Decoder s (ByteString -> ChainDepState (BlockProtocol blk)))
-> (Examples blk -> Labelled (ChainDepState (BlockProtocol blk)))
-> TestTree
forall a.
(Eq a, Show a) =>
String
-> (a -> Encoding)
-> (forall s. Decoder s (ByteString -> a))
-> (Examples blk -> Labelled a)
-> TestTree
testRoundtripFor String
"Chain dependent state" (CodecConfig blk -> ChainDepState (BlockProtocol blk) -> Encoding
forall blk a. EncodeDisk blk a => CodecConfig blk -> a -> Encoding
encodeDisk CodecConfig blk
codecConfig) (ChainDepState (BlockProtocol blk)
-> ByteString -> ChainDepState (BlockProtocol blk)
forall a b. a -> b -> a
const (ChainDepState (BlockProtocol blk)
 -> ByteString -> ChainDepState (BlockProtocol blk))
-> Decoder s (ChainDepState (BlockProtocol blk))
-> Decoder s (ByteString -> ChainDepState (BlockProtocol blk))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> CodecConfig blk
-> forall s. Decoder s (ChainDepState (BlockProtocol blk))
forall blk a.
DecodeDisk blk a =>
CodecConfig blk -> forall s. Decoder s a
decodeDisk CodecConfig blk
codecConfig) Examples blk -> Labelled (ChainDepState (BlockProtocol blk))
forall blk.
Examples blk -> Labelled (ChainDepState (BlockProtocol blk))
exampleChainDepState
    , String
-> (ExtLedgerState blk EmptyMK -> Encoding)
-> (forall s. Decoder s (ByteString -> ExtLedgerState blk EmptyMK))
-> (Examples blk -> Labelled (ExtLedgerState blk EmptyMK))
-> TestTree
forall a.
(Eq a, Show a) =>
String
-> (a -> Encoding)
-> (forall s. Decoder s (ByteString -> a))
-> (Examples blk -> Labelled a)
-> TestTree
testRoundtripFor String
"Extended ledger state" (CodecConfig blk -> ExtLedgerState blk EmptyMK -> Encoding
forall blk.
(EncodeDisk blk (LedgerState blk EmptyMK),
 EncodeDisk blk (ChainDepState (BlockProtocol blk)),
 EncodeDisk blk (AnnTip blk)) =>
CodecConfig blk -> ExtLedgerState blk EmptyMK -> Encoding
encodeDiskExtLedgerState CodecConfig blk
codecConfig) (ExtLedgerState blk EmptyMK
-> ByteString -> ExtLedgerState blk EmptyMK
forall a b. a -> b -> a
const (ExtLedgerState blk EmptyMK
 -> ByteString -> ExtLedgerState blk EmptyMK)
-> Decoder s (ExtLedgerState blk EmptyMK)
-> Decoder s (ByteString -> ExtLedgerState blk EmptyMK)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> CodecConfig blk -> forall s. Decoder s (ExtLedgerState blk EmptyMK)
forall blk.
(DecodeDisk blk (LedgerState blk EmptyMK),
 DecodeDisk blk (ChainDepState (BlockProtocol blk)),
 DecodeDisk blk (AnnTip blk)) =>
CodecConfig blk -> forall s. Decoder s (ExtLedgerState blk EmptyMK)
decodeDiskExtLedgerState CodecConfig blk
codecConfig)  Examples blk -> Labelled (ExtLedgerState blk EmptyMK)
forall blk. Examples blk -> Labelled (ExtLedgerState blk EmptyMK)
exampleExtLedgerState
    ]
  where
    testRoundtripFor ::
         forall a . (Eq a, Show a)
      => String
      -> (a -> Encoding)
      -> (forall s . Decoder s (Char8.ByteString -> a))
      -> (Examples blk -> Labelled a)
      -> TestTree
    testRoundtripFor :: forall a.
(Eq a, Show a) =>
String
-> (a -> Encoding)
-> (forall s. Decoder s (ByteString -> a))
-> (Examples blk -> Labelled a)
-> TestTree
testRoundtripFor String
testLabel a -> Encoding
enc forall s. Decoder s (ByteString -> a)
dec Examples blk -> Labelled a
field =
        String -> [TestTree] -> TestTree
testGroup String
testLabel
          [ Maybe String -> a -> TestTree
mkTest Maybe String
exampleName a
example
          | (Maybe String
exampleName, a
example) <- Examples blk -> Labelled a
field Examples blk
examples
          ]
      where
        mkTest :: Maybe String -> a -> TestTree
mkTest Maybe String
exampleName a
example =
          String -> Property -> TestTree
forall a. Testable a => String -> a -> TestTree
testProperty (String -> Maybe String -> String
forall a. a -> Maybe a -> a
fromMaybe String
"" Maybe String
exampleName)
              (Property -> TestTree) -> Property -> TestTree
forall a b. (a -> b) -> a -> b
$ Property -> Property
forall prop. Testable prop => prop -> Property
once
              (Property -> Property) -> Property -> Property
forall a b. (a -> b) -> a -> b
$ (a -> Encoding)
-> (forall s. Decoder s (ByteString -> a)) -> a -> Property
forall a.
(Eq a, Show a) =>
(a -> Encoding)
-> (forall s. Decoder s (ByteString -> a)) -> a -> Property
roundtrip' a -> Encoding
enc Decoder s (ByteString -> a)
forall s. Decoder s (ByteString -> a)
dec a
example