{-# LANGUAGE DataKinds #-}
{-# LANGUAGE DefaultSignatures #-}
{-# LANGUAGE FlexibleContexts #-}
{-# LANGUAGE FlexibleInstances #-}
{-# LANGUAGE OverloadedStrings #-}
{-# LANGUAGE RankNTypes #-}
{-# LANGUAGE RecordWildCards #-}
{-# LANGUAGE ScopedTypeVariables #-}
{-# LANGUAGE TypeApplications #-}
{-# OPTIONS_GHC -Wno-orphans #-}

-- | Golden tests infrastructure.
--
-- Golden tests are implemented using
-- __[tasty-golden](https://github.com/UnkindPartition/tasty-golden)__.
--
-- When adding a new golden test, running the test suite locally will generate
-- the golden files. These files should be checked in as CI will fail if there
-- are missing golden files.
--
-- Failing a golden test suite when the corresponding golden files are not found
-- is done via the @--no-create@ flag, which surprisingly is opt-in. In our
-- @nix@ infrastructure, this flag for CI is set in @nix/haskell.nix@.
--
-- > testFlags = lib.mkForce [ "--no-create" ];
--
-- In particular, if we introduce golden tests in new suites, we need to add
-- a line in the nix configuration above similar to the previous ones.
module Test.Util.Serialisation.Golden
  ( ToGoldenDirectory (..)
  , goldenTest_SerialiseDisk
  , goldenTest_SerialiseNodeToClient
  , goldenTest_SerialiseNodeToNode
  , goldenTest_all
  ) where

import Cardano.Prelude (forceElemsToWHNF)
import Codec.CBOR.Encoding (Encoding)
import Codec.CBOR.FlatTerm (TermToken (..))
import qualified Codec.CBOR.FlatTerm as CBOR
import qualified Codec.CBOR.Write as CBOR
import Codec.Serialise (encode)
import Control.Exception (SomeException, evaluate, try)
import Data.Bifunctor (first)
import qualified Data.ByteString as Strict
import qualified Data.ByteString.UTF8 as BS.UTF8
import Data.List (nub)
import qualified Data.Map.Strict as Map
import Data.Proxy (Proxy (..))
import Data.TreeDiff
import GHC.Stack (HasCallStack)
import Ouroboros.Consensus.Block (CodecConfig)
import Ouroboros.Consensus.Ledger.Extended (encodeDiskExtLedgerState)
import Ouroboros.Consensus.Ledger.Query
  ( QueryVersion
  , nodeToClientVersionToQueryVersion
  )
import Ouroboros.Consensus.Ledger.Tables (valuesMKEncoder)
import Ouroboros.Consensus.Node.NetworkProtocolVersion
  ( HasNetworkProtocolVersion (..)
  , SupportedNetworkProtocolVersion (..)
  )
import Ouroboros.Consensus.Node.Run
  ( SerialiseDiskConstraints
  , SerialiseNodeToClientConstraints
  , SerialiseNodeToNodeConstraints
  )
import Ouroboros.Consensus.Node.Serialisation
  ( SerialiseBlockQueryResult (..)
  , SerialiseNodeToClient (..)
  , SerialiseNodeToNode (..)
  )
import Ouroboros.Consensus.Storage.Serialisation (EncodeDisk (..))
import Ouroboros.Consensus.Util.CBOR (decodeAsFlatTerm)
import Ouroboros.Consensus.Util.Condense (Condense (..))
import System.Directory (createDirectoryIfMissing)
import System.FilePath (takeDirectory, (</>))
import Test.Cardano.Binary.TreeDiff (CBORBytes (..))
import Test.Tasty
import Test.Tasty.Golden.Advanced (goldenTest)
import Test.Util.Serialisation.Examples (Examples (..), Labelled)
import Test.Util.Serialisation.SomeResult (SomeResult (..))

{-------------------------------------------------------------------------------
  Golden test
-------------------------------------------------------------------------------}

-- | Golden test for CBOR output. When the output doesn't match the golden
-- one, we show an 'ediff' of the 'FlatTerm' output of both.
--
-- Exceptions: when forcing an encoding throws an exception, we 'show' the
-- exception and use that as the output.
goldenTestCBOR ::
  TestName ->
  a ->
  (a -> Encoding) ->
  -- | Path to the file containing the golden output
  FilePath ->
  TestTree
goldenTestCBOR :: forall a. TestName -> a -> (a -> Encoding) -> TestName -> TestTree
goldenTestCBOR TestName
testName a
example a -> Encoding
enc TestName
goldenFile =
  TestName
-> IO ByteString
-> IO ByteString
-> (ByteString -> ByteString -> IO (Maybe TestName))
-> (ByteString -> IO ())
-> TestTree
forall a.
TestName
-> IO a
-> IO a
-> (a -> a -> IO (Maybe TestName))
-> (a -> IO ())
-> TestTree
goldenTest
    TestName
testName
    (TestName -> IO ByteString
Strict.readFile TestName
goldenFile)
    ((SomeException -> ByteString)
-> (ByteString -> ByteString)
-> Either SomeException ByteString
-> ByteString
forall a c b. (a -> c) -> (b -> c) -> Either a b -> c
either SomeException -> ByteString
exceptionToByteString ByteString -> ByteString
forall a. a -> a
id (Either SomeException ByteString -> ByteString)
-> IO (Either SomeException ByteString) -> IO ByteString
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> IO ByteString -> IO (Either SomeException ByteString)
forall e a. Exception e => IO a -> IO (Either e a)
try (ByteString -> IO ByteString
forall a. a -> IO a
evaluate ByteString
actualValue))
    ByteString -> ByteString -> IO (Maybe TestName)
diff
    ByteString -> IO ()
updateGoldenFile
 where
  -- Copied from tasty-golden because it isn't exported
  updateGoldenFile :: Strict.ByteString -> IO ()
  updateGoldenFile :: ByteString -> IO ()
updateGoldenFile ByteString
bytes = do
    let dir :: TestName
dir = TestName -> TestName
takeDirectory TestName
goldenFile
    Bool -> TestName -> IO ()
createDirectoryIfMissing Bool
True TestName
dir
    TestName -> ByteString -> IO ()
Strict.writeFile TestName
goldenFile ByteString
bytes

  actualValue :: Strict.ByteString
  actualValue :: ByteString
actualValue = Encoding -> ByteString
CBOR.toStrictByteString (a -> Encoding
enc a
example)

  exceptionToByteString :: SomeException -> Strict.ByteString
  exceptionToByteString :: SomeException -> ByteString
exceptionToByteString = TestName -> ByteString
BS.UTF8.fromString (TestName -> ByteString)
-> (SomeException -> TestName) -> SomeException -> ByteString
forall b c a. (b -> c) -> (a -> b) -> a -> c
. SomeException -> TestName
forall a. Show a => a -> TestName
show

  -- \| Use 'ediff' ('ToExpr') to diff the 'FlatTerm' representation.
  diff :: Strict.ByteString -> Strict.ByteString -> IO (Maybe String)
  diff :: ByteString -> ByteString -> IO (Maybe TestName)
diff ByteString
golden ByteString
actual = do
    actualRes <-
      (Either SomeException FlatTerm -> Either SomeException FlatTerm)
-> IO (Either SomeException FlatTerm)
-> IO (Either SomeException FlatTerm)
forall a b. (a -> b) -> IO a -> IO b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap ((SomeException -> SomeException)
-> Either SomeException FlatTerm -> Either SomeException FlatTerm
forall a b c. (a -> b) -> Either a c -> Either b c
forall (p :: * -> * -> *) a b c.
Bifunctor p =>
(a -> b) -> p a c -> p b c
first (\(SomeException
e :: SomeException) -> SomeException
e))
        (IO (Either SomeException FlatTerm)
 -> IO (Either SomeException FlatTerm))
-> (a -> IO (Either SomeException FlatTerm))
-> a
-> IO (Either SomeException FlatTerm)
forall b c a. (b -> c) -> (a -> b) -> a -> c
. IO FlatTerm -> IO (Either SomeException FlatTerm)
forall e a. Exception e => IO a -> IO (Either e a)
try
        (IO FlatTerm -> IO (Either SomeException FlatTerm))
-> (a -> IO FlatTerm) -> a -> IO (Either SomeException FlatTerm)
forall b c a. (b -> c) -> (a -> b) -> a -> c
. FlatTerm -> IO FlatTerm
forall a. a -> IO a
evaluate
        (FlatTerm -> IO FlatTerm) -> (a -> FlatTerm) -> a -> IO FlatTerm
forall b c a. (b -> c) -> (a -> b) -> a -> c
. FlatTerm -> FlatTerm
forall (t :: * -> *) a. Foldable t => t a -> t a
forceElemsToWHNF
        (FlatTerm -> FlatTerm) -> (a -> FlatTerm) -> a -> FlatTerm
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Encoding -> FlatTerm
CBOR.toFlatTerm
        (Encoding -> FlatTerm) -> (a -> Encoding) -> a -> FlatTerm
forall b c a. (b -> c) -> (a -> b) -> a -> c
. a -> Encoding
enc
        (a -> IO (Either SomeException FlatTerm))
-> a -> IO (Either SomeException FlatTerm)
forall a b. (a -> b) -> a -> b
$ a
example
    return $ case (actualRes, decodeAsFlatTerm golden) of
      (Left SomeException
e, Right FlatTerm
goldenFlatTerm)
        -- Encoder threw an exception and the golden output was valid
        -- CBOR. However, sometimes the 'show'n exception is also valid
        -- CBOR. So if the exception and the golden output match, the test
        -- passes.
        | SomeException -> ByteString
exceptionToByteString SomeException
e ByteString -> ByteString -> Bool
forall a. Eq a => a -> a -> Bool
== ByteString
golden -> Maybe TestName
forall a. Maybe a
Nothing
        | Bool
otherwise ->
            TestName -> Maybe TestName
forall a. a -> Maybe a
Just (TestName -> Maybe TestName) -> TestName -> Maybe TestName
forall a b. (a -> b) -> a -> b
$
              [TestName] -> TestName
unlines
                [ TestName
"Exception thrown by encoder doesn't match the golden CBOR output"
                , TestName
"Exception:"
                , SomeException -> TestName
forall a. Show a => a -> TestName
show SomeException
e
                , TestName
"Golden term:"
                , FlatTerm -> TestName
forall a. Condense a => a -> TestName
condense FlatTerm
goldenFlatTerm
                ]
      (Left SomeException
e, Left DeserialiseFailure
_)
        -- Encoder threw an exception. The golden output was not valid
        -- CBOR and the bytestrings match: we expected the exception
        | SomeException -> ByteString
exceptionToByteString SomeException
e ByteString -> ByteString -> Bool
forall a. Eq a => a -> a -> Bool
== ByteString
golden -> Maybe TestName
forall a. Maybe a
Nothing
        | Bool
otherwise ->
            TestName -> Maybe TestName
forall a. a -> Maybe a
Just (TestName -> Maybe TestName) -> TestName -> Maybe TestName
forall a b. (a -> b) -> a -> b
$
              [TestName] -> TestName
unlines
                [ TestName
"Exception thrown by encoder doesn't match the golden output"
                , TestName
"Exception:"
                , SomeException -> TestName
forall a. Show a => a -> TestName
show SomeException
e
                , TestName
"Golden output:"
                , ByteString -> TestName
BS.UTF8.toString ByteString
golden
                ]
      (Right FlatTerm
_actualFlatTerm, Right FlatTerm
_goldenFlatTerm)
        | ByteString
actual ByteString -> ByteString -> Bool
forall a. Eq a => a -> a -> Bool
== ByteString
golden -> Maybe TestName
forall a. Maybe a
Nothing
        | Bool
otherwise ->
            TestName -> Maybe TestName
forall a. a -> Maybe a
Just (TestName -> Maybe TestName) -> TestName -> Maybe TestName
forall a b. (a -> b) -> a -> b
$
              [TestName] -> TestName
unlines
                [ TestName
"Golden term /= actual term, diff golden actual:"
                , Doc -> TestName
forall a. Show a => a -> TestName
show (Edit EditExpr -> Doc
ansiWlEditExpr (CBORBytes -> CBORBytes -> Edit EditExpr
forall a. ToExpr a => a -> a -> Edit EditExpr
ediff (ByteString -> CBORBytes
CBORBytes ByteString
golden) (ByteString -> CBORBytes
CBORBytes ByteString
actual)))
                ]
      (Right FlatTerm
actualFlatTerm, Left DeserialiseFailure
e) ->
        TestName -> Maybe TestName
forall a. a -> Maybe a
Just (TestName -> Maybe TestName) -> TestName -> Maybe TestName
forall a b. (a -> b) -> a -> b
$
          [TestName] -> TestName
unlines
            [ TestName
"Golden output /= actual term:"
            , TestName
"Golden output is not valid CBOR:"
            , ByteString -> TestName
BS.UTF8.toString ByteString
golden
            , TestName
"Exception: "
            , DeserialiseFailure -> TestName
forall a. Show a => a -> TestName
show DeserialiseFailure
e
            , TestName
"Actual term:"
            , FlatTerm -> TestName
forall a. Condense a => a -> TestName
condense FlatTerm
actualFlatTerm
            ]

goldenTests ::
  HasCallStack =>
  TestName ->
  Labelled a ->
  (a -> Encoding) ->
  -- | Folder containing the golden files
  FilePath ->
  TestTree
goldenTests :: forall a.
HasCallStack =>
TestName -> Labelled a -> (a -> Encoding) -> TestName -> TestTree
goldenTests TestName
testName Labelled a
examples a -> Encoding
enc TestName
goldenFolder
  | [Maybe TestName] -> [Maybe TestName]
forall a. Eq a => [a] -> [a]
nub [Maybe TestName]
labels [Maybe TestName] -> [Maybe TestName] -> Bool
forall a. Eq a => a -> a -> Bool
/= [Maybe TestName]
labels =
      TestName -> TestTree
forall a. HasCallStack => TestName -> a
error (TestName -> TestTree) -> TestName -> TestTree
forall a b. (a -> b) -> a -> b
$ TestName
"Examples with the same label for " TestName -> TestName -> TestName
forall a. Semigroup a => a -> a -> a
<> TestName
testName
  | [(Maybe TestName
Nothing, a
example)] <- Labelled a
examples =
      -- If there's just a single unlabelled example, no need for grouping,
      -- which makes the output more verbose.
      TestName -> a -> (a -> Encoding) -> TestName -> TestTree
forall a. TestName -> a -> (a -> Encoding) -> TestName -> TestTree
goldenTestCBOR TestName
testName a
example a -> Encoding
enc (TestName
goldenFolder TestName -> TestName -> TestName
</> TestName
testName)
  | Bool
otherwise =
      TestName -> [TestTree] -> TestTree
testGroup
        TestName
testName
        [ TestName -> a -> (a -> Encoding) -> TestName -> TestTree
forall a. TestName -> a -> (a -> Encoding) -> TestName -> TestTree
goldenTestCBOR TestName
testName' a
example a -> Encoding
enc (TestName
goldenFolder TestName -> TestName -> TestName
</> TestName
testName')
        | (Maybe TestName
mbLabel, a
example) <- Labelled a
examples
        , let testName' :: TestName
testName' = case Maybe TestName
mbLabel of
                Maybe TestName
Nothing -> TestName
testName
                Just TestName
label -> TestName
testName TestName -> TestName -> TestName
forall a. Semigroup a => a -> a -> a
<> TestName
"_" TestName -> TestName -> TestName
forall a. Semigroup a => a -> a -> a
<> TestName
label
        ]
 where
  labels :: [Maybe String]
  labels :: [Maybe TestName]
labels = ((Maybe TestName, a) -> Maybe TestName)
-> Labelled a -> [Maybe TestName]
forall a b. (a -> b) -> [a] -> [b]
map (Maybe TestName, a) -> Maybe TestName
forall a b. (a, b) -> a
fst Labelled a
examples

goldenTests' ::
  HasCallStack =>
  TestName ->
  Labelled (a, a -> Encoding) ->
  -- | Folder containing the golden files
  FilePath ->
  TestTree
goldenTests' :: forall a.
HasCallStack =>
TestName -> Labelled (a, a -> Encoding) -> TestName -> TestTree
goldenTests' TestName
testName Labelled (a, a -> Encoding)
examples TestName
goldenFolder
  | [Maybe TestName] -> [Maybe TestName]
forall a. Eq a => [a] -> [a]
nub [Maybe TestName]
labels [Maybe TestName] -> [Maybe TestName] -> Bool
forall a. Eq a => a -> a -> Bool
/= [Maybe TestName]
labels =
      TestName -> TestTree
forall a. HasCallStack => TestName -> a
error (TestName -> TestTree) -> TestName -> TestTree
forall a b. (a -> b) -> a -> b
$ TestName
"Examples with the same label for " TestName -> TestName -> TestName
forall a. Semigroup a => a -> a -> a
<> TestName
testName
  | [(Maybe TestName
Nothing, (a
example, a -> Encoding
exampleEncoder))] <- Labelled (a, a -> Encoding)
examples =
      -- If there's just a single unlabelled example, no need for grouping,
      -- which makes the output more verbose.
      TestName -> a -> (a -> Encoding) -> TestName -> TestTree
forall a. TestName -> a -> (a -> Encoding) -> TestName -> TestTree
goldenTestCBOR TestName
testName a
example a -> Encoding
exampleEncoder (TestName
goldenFolder TestName -> TestName -> TestName
</> TestName
testName)
  | Bool
otherwise =
      TestName -> [TestTree] -> TestTree
testGroup
        TestName
testName
        [ TestName -> a -> (a -> Encoding) -> TestName -> TestTree
forall a. TestName -> a -> (a -> Encoding) -> TestName -> TestTree
goldenTestCBOR TestName
testName' a
example a -> Encoding
exampleEncoder (TestName
goldenFolder TestName -> TestName -> TestName
</> TestName
testName')
        | (Maybe TestName
mbLabel, (a
example, a -> Encoding
exampleEncoder)) <- Labelled (a, a -> Encoding)
examples
        , let testName' :: TestName
testName' = case Maybe TestName
mbLabel of
                Maybe TestName
Nothing -> TestName
testName
                Just TestName
label -> TestName
testName TestName -> TestName -> TestName
forall a. Semigroup a => a -> a -> a
<> TestName
"_" TestName -> TestName -> TestName
forall a. Semigroup a => a -> a -> a
<> TestName
label
        ]
 where
  labels :: [Maybe String]
  labels :: [Maybe TestName]
labels = ((Maybe TestName, (a, a -> Encoding)) -> Maybe TestName)
-> Labelled (a, a -> Encoding) -> [Maybe TestName]
forall a b. (a -> b) -> [a] -> [b]
map (Maybe TestName, (a, a -> Encoding)) -> Maybe TestName
forall a b. (a, b) -> a
fst Labelled (a, a -> Encoding)
examples

{-------------------------------------------------------------------------------
  Skeletons
-------------------------------------------------------------------------------}

-- | Convert 'a' to a 'FilePath' that can be used as the directory containing
-- golden output files.
--
-- This class allows overriding the 'Show' in cases where that output is not
-- suitable to be used as a directory.
--
-- For example, the 'Show' output for a hard fork enabled 'NodeToNodeVersion'
-- will contain colons, asterisks, spaces, parentheses, ... and other
-- characters that we don't want to use for a directory name. For instance
-- colons cannot be used in Windows file/folder names.
class ToGoldenDirectory a where
  toGoldenDirectory :: a -> FilePath
  default toGoldenDirectory :: Show a => a -> FilePath
  toGoldenDirectory = a -> TestName
forall a. Show a => a -> TestName
show

-- | Golden tests for all things we serialise to disk and send across the
-- network.
--
-- Exceptions: when an encoder throws an exception, which can happen when
-- serialising a Shelley header in combination with
-- 'CardanoNodeToNodeVersion1', we 'show' the exception and use that as the
-- output.
goldenTest_all ::
  ( SerialiseDiskConstraints blk
  , SerialiseNodeToNodeConstraints blk
  , SerialiseNodeToClientConstraints blk
  , SupportedNetworkProtocolVersion blk
  , ToGoldenDirectory (BlockNodeToNodeVersion blk)
  , ToGoldenDirectory (QueryVersion, BlockNodeToClientVersion blk)
  , HasCallStack
  ) =>
  CodecConfig blk ->
  -- | Path relative to the root of the repository that contains the golden
  -- files
  FilePath ->
  Examples blk ->
  TestTree
goldenTest_all :: forall blk.
(SerialiseDiskConstraints blk, SerialiseNodeToNodeConstraints blk,
 SerialiseNodeToClientConstraints blk,
 SupportedNetworkProtocolVersion blk,
 ToGoldenDirectory (BlockNodeToNodeVersion blk),
 ToGoldenDirectory (QueryVersion, BlockNodeToClientVersion blk),
 HasCallStack) =>
CodecConfig blk -> TestName -> Examples blk -> TestTree
goldenTest_all CodecConfig blk
codecConfig TestName
goldenDir Examples blk
examples =
  TestName -> [TestTree] -> TestTree
testGroup
    TestName
"Golden tests"
    [ CodecConfig blk -> TestName -> Examples blk -> TestTree
forall blk.
(SerialiseDiskConstraints blk, HasCallStack) =>
CodecConfig blk -> TestName -> Examples blk -> TestTree
goldenTest_SerialiseDisk CodecConfig blk
codecConfig TestName
goldenDir Examples blk
examples
    , CodecConfig blk -> TestName -> Examples blk -> TestTree
forall blk.
(SerialiseNodeToNodeConstraints blk,
 SupportedNetworkProtocolVersion blk,
 ToGoldenDirectory (BlockNodeToNodeVersion blk), HasCallStack) =>
CodecConfig blk -> TestName -> Examples blk -> TestTree
goldenTest_SerialiseNodeToNode CodecConfig blk
codecConfig TestName
goldenDir Examples blk
examples
    , CodecConfig blk -> TestName -> Examples blk -> TestTree
forall blk.
(SerialiseNodeToClientConstraints blk,
 SupportedNetworkProtocolVersion blk,
 ToGoldenDirectory (QueryVersion, BlockNodeToClientVersion blk),
 HasCallStack) =>
CodecConfig blk -> TestName -> Examples blk -> TestTree
goldenTest_SerialiseNodeToClient CodecConfig blk
codecConfig TestName
goldenDir Examples blk
examples
    ]

-- TODO how can we ensure that we have a test for each constraint listed in
-- 'SerialiseDiskConstraints'?
goldenTest_SerialiseDisk ::
  forall blk.
  ( SerialiseDiskConstraints blk
  , HasCallStack
  ) =>
  CodecConfig blk ->
  FilePath ->
  Examples blk ->
  TestTree
goldenTest_SerialiseDisk :: forall blk.
(SerialiseDiskConstraints blk, HasCallStack) =>
CodecConfig blk -> TestName -> Examples blk -> TestTree
goldenTest_SerialiseDisk CodecConfig blk
codecConfig TestName
goldenDir Examples{Labelled blk
Labelled SlotNo
Labelled (HeaderHash blk)
Labelled (Header blk)
Labelled (LedgerTables (LedgerState blk) ValuesMK)
Labelled (ChainDepState (BlockProtocol blk))
Labelled (LedgerState blk EmptyMK)
Labelled (LedgerConfig blk)
Labelled (GenTxId blk)
Labelled (ApplyTxErr blk)
Labelled (GenTx blk)
Labelled (AnnTip blk)
Labelled (SerialisedHeader blk)
Labelled (ExtLedgerState blk EmptyMK)
Labelled (SomeBlockQuery (BlockQuery blk))
Labelled (Serialised blk)
Labelled (SomeResult blk)
exampleBlock :: Labelled blk
exampleSerialisedBlock :: Labelled (Serialised blk)
exampleHeader :: Labelled (Header blk)
exampleSerialisedHeader :: Labelled (SerialisedHeader blk)
exampleHeaderHash :: Labelled (HeaderHash blk)
exampleGenTx :: Labelled (GenTx blk)
exampleGenTxId :: Labelled (GenTxId blk)
exampleApplyTxErr :: Labelled (ApplyTxErr blk)
exampleQuery :: Labelled (SomeBlockQuery (BlockQuery blk))
exampleResult :: Labelled (SomeResult blk)
exampleAnnTip :: Labelled (AnnTip blk)
exampleLedgerState :: Labelled (LedgerState blk EmptyMK)
exampleChainDepState :: Labelled (ChainDepState (BlockProtocol blk))
exampleExtLedgerState :: Labelled (ExtLedgerState blk EmptyMK)
exampleSlotNo :: Labelled SlotNo
exampleLedgerConfig :: Labelled (LedgerConfig blk)
exampleLedgerTables :: Labelled (LedgerTables (LedgerState blk) ValuesMK)
exampleLedgerTables :: forall blk.
Examples blk -> Labelled (LedgerTables (LedgerState blk) ValuesMK)
exampleLedgerConfig :: forall blk. Examples blk -> Labelled (LedgerConfig blk)
exampleSlotNo :: forall blk. Examples blk -> Labelled SlotNo
exampleExtLedgerState :: forall blk. Examples blk -> Labelled (ExtLedgerState blk EmptyMK)
exampleChainDepState :: forall blk.
Examples blk -> Labelled (ChainDepState (BlockProtocol blk))
exampleLedgerState :: forall blk. Examples blk -> Labelled (LedgerState blk EmptyMK)
exampleAnnTip :: forall blk. Examples blk -> Labelled (AnnTip blk)
exampleResult :: forall blk. Examples blk -> Labelled (SomeResult blk)
exampleQuery :: forall blk.
Examples blk -> Labelled (SomeBlockQuery (BlockQuery blk))
exampleApplyTxErr :: forall blk. Examples blk -> Labelled (ApplyTxErr blk)
exampleGenTxId :: forall blk. Examples blk -> Labelled (GenTxId blk)
exampleGenTx :: forall blk. Examples blk -> Labelled (GenTx blk)
exampleHeaderHash :: forall blk. Examples blk -> Labelled (HeaderHash blk)
exampleSerialisedHeader :: forall blk. Examples blk -> Labelled (SerialisedHeader blk)
exampleHeader :: forall blk. Examples blk -> Labelled (Header blk)
exampleSerialisedBlock :: forall blk. Examples blk -> Labelled (Serialised blk)
exampleBlock :: forall blk. Examples blk -> Labelled blk
..} =
  TestName -> [TestTree] -> TestTree
testGroup
    TestName
"SerialiseDisk"
    [ TestName -> Labelled blk -> (blk -> Encoding) -> TestTree
forall a. TestName -> Labelled a -> (a -> Encoding) -> TestTree
test TestName
"Block" Labelled blk
exampleBlock (CodecConfig blk -> blk -> Encoding
forall blk a. EncodeDisk blk a => CodecConfig blk -> a -> Encoding
encodeDisk CodecConfig blk
codecConfig)
    , TestName
-> Labelled (HeaderHash blk)
-> (HeaderHash blk -> Encoding)
-> TestTree
forall a. TestName -> Labelled a -> (a -> Encoding) -> TestTree
test TestName
"HeaderHash" Labelled (HeaderHash blk)
exampleHeaderHash HeaderHash blk -> Encoding
forall a. Serialise a => a -> Encoding
encode
    , TestName
-> Labelled (LedgerState blk EmptyMK)
-> (LedgerState blk EmptyMK -> Encoding)
-> TestTree
forall a. TestName -> Labelled a -> (a -> Encoding) -> TestTree
test TestName
"LedgerState" Labelled (LedgerState blk EmptyMK)
exampleLedgerState (CodecConfig blk -> LedgerState blk EmptyMK -> Encoding
forall blk a. EncodeDisk blk a => CodecConfig blk -> a -> Encoding
encodeDisk CodecConfig blk
codecConfig)
    , TestName
-> Labelled (AnnTip blk) -> (AnnTip blk -> Encoding) -> TestTree
forall a. TestName -> Labelled a -> (a -> Encoding) -> TestTree
test TestName
"AnnTip" Labelled (AnnTip blk)
exampleAnnTip (CodecConfig blk -> AnnTip blk -> Encoding
forall blk a. EncodeDisk blk a => CodecConfig blk -> a -> Encoding
encodeDisk CodecConfig blk
codecConfig)
    , TestName
-> Labelled (ChainDepState (BlockProtocol blk))
-> (ChainDepState (BlockProtocol blk) -> Encoding)
-> TestTree
forall a. TestName -> Labelled a -> (a -> Encoding) -> TestTree
test TestName
"ChainDepState" Labelled (ChainDepState (BlockProtocol blk))
exampleChainDepState (CodecConfig blk -> ChainDepState (BlockProtocol blk) -> Encoding
forall blk a. EncodeDisk blk a => CodecConfig blk -> a -> Encoding
encodeDisk CodecConfig blk
codecConfig)
    , TestName
-> Labelled (ExtLedgerState blk EmptyMK)
-> (ExtLedgerState blk EmptyMK -> Encoding)
-> TestTree
forall a. TestName -> Labelled a -> (a -> Encoding) -> TestTree
test TestName
"ExtLedgerState" Labelled (ExtLedgerState blk EmptyMK)
exampleExtLedgerState ExtLedgerState blk EmptyMK -> Encoding
encodeExt
    , TestTree
testLedgerTables
    ]
 where
  test :: TestName -> Labelled a -> (a -> Encoding) -> TestTree
  test :: forall a. TestName -> Labelled a -> (a -> Encoding) -> TestTree
test TestName
testName Labelled a
exampleValues a -> Encoding
enc =
    TestName -> Labelled a -> (a -> Encoding) -> TestName -> TestTree
forall a.
HasCallStack =>
TestName -> Labelled a -> (a -> Encoding) -> TestName -> TestTree
goldenTests
      TestName
testName
      Labelled a
exampleValues
      a -> Encoding
enc
      (TestName
goldenDir TestName -> TestName -> TestName
</> TestName
"disk")

  testLedgerTables :: TestTree
  testLedgerTables :: TestTree
testLedgerTables =
    TestName
-> Labelled
     (LedgerTables (LedgerState blk) ValuesMK,
      LedgerTables (LedgerState blk) ValuesMK -> Encoding)
-> TestName
-> TestTree
forall a.
HasCallStack =>
TestName -> Labelled (a, a -> Encoding) -> TestName -> TestTree
goldenTests'
      TestName
"LedgerTables"
      ( ((Maybe TestName, LedgerTables (LedgerState blk) ValuesMK)
 -> (Maybe TestName, LedgerState blk EmptyMK)
 -> (Maybe TestName,
     (LedgerTables (LedgerState blk) ValuesMK,
      LedgerTables (LedgerState blk) ValuesMK -> Encoding)))
-> Labelled (LedgerTables (LedgerState blk) ValuesMK)
-> Labelled (LedgerState blk EmptyMK)
-> Labelled
     (LedgerTables (LedgerState blk) ValuesMK,
      LedgerTables (LedgerState blk) ValuesMK -> Encoding)
forall a b c. (a -> b -> c) -> [a] -> [b] -> [c]
zipWith
          (\(Maybe TestName
lbl, LedgerTables (LedgerState blk) ValuesMK
tbs) (Maybe TestName
_, LedgerState blk EmptyMK
st) -> (Maybe TestName
lbl, (LedgerTables (LedgerState blk) ValuesMK
tbs, LedgerState blk EmptyMK
-> LedgerTables (LedgerState blk) ValuesMK -> Encoding
forall (l :: LedgerStateKind).
SerializeTablesWithHint l =>
l EmptyMK -> LedgerTables l ValuesMK -> Encoding
valuesMKEncoder LedgerState blk EmptyMK
st)))
          Labelled (LedgerTables (LedgerState blk) ValuesMK)
exampleLedgerTables
          Labelled (LedgerState blk EmptyMK)
exampleLedgerState
      )
      (TestName
goldenDir TestName -> TestName -> TestName
</> TestName
"disk")

  encodeExt :: ExtLedgerState blk EmptyMK -> Encoding
encodeExt = 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

-- TODO how can we ensure that we have a test for each constraint listed in
-- 'SerialiseNodeToNodeConstraints'?
goldenTest_SerialiseNodeToNode ::
  forall blk.
  ( SerialiseNodeToNodeConstraints blk
  , SupportedNetworkProtocolVersion blk
  , ToGoldenDirectory (BlockNodeToNodeVersion blk)
  , HasCallStack
  ) =>
  CodecConfig blk ->
  FilePath ->
  Examples blk ->
  TestTree
goldenTest_SerialiseNodeToNode :: forall blk.
(SerialiseNodeToNodeConstraints blk,
 SupportedNetworkProtocolVersion blk,
 ToGoldenDirectory (BlockNodeToNodeVersion blk), HasCallStack) =>
CodecConfig blk -> TestName -> Examples blk -> TestTree
goldenTest_SerialiseNodeToNode CodecConfig blk
codecConfig TestName
goldenDir Examples{Labelled blk
Labelled SlotNo
Labelled (HeaderHash blk)
Labelled (Header blk)
Labelled (LedgerTables (LedgerState blk) ValuesMK)
Labelled (ChainDepState (BlockProtocol blk))
Labelled (LedgerState blk EmptyMK)
Labelled (LedgerConfig blk)
Labelled (GenTxId blk)
Labelled (ApplyTxErr blk)
Labelled (GenTx blk)
Labelled (AnnTip blk)
Labelled (SerialisedHeader blk)
Labelled (ExtLedgerState blk EmptyMK)
Labelled (SomeBlockQuery (BlockQuery blk))
Labelled (Serialised blk)
Labelled (SomeResult blk)
exampleLedgerTables :: forall blk.
Examples blk -> Labelled (LedgerTables (LedgerState blk) ValuesMK)
exampleLedgerConfig :: forall blk. Examples blk -> Labelled (LedgerConfig blk)
exampleSlotNo :: forall blk. Examples blk -> Labelled SlotNo
exampleExtLedgerState :: forall blk. Examples blk -> Labelled (ExtLedgerState blk EmptyMK)
exampleChainDepState :: forall blk.
Examples blk -> Labelled (ChainDepState (BlockProtocol blk))
exampleLedgerState :: forall blk. Examples blk -> Labelled (LedgerState blk EmptyMK)
exampleAnnTip :: forall blk. Examples blk -> Labelled (AnnTip blk)
exampleResult :: forall blk. Examples blk -> Labelled (SomeResult blk)
exampleQuery :: forall blk.
Examples blk -> Labelled (SomeBlockQuery (BlockQuery blk))
exampleApplyTxErr :: forall blk. Examples blk -> Labelled (ApplyTxErr blk)
exampleGenTxId :: forall blk. Examples blk -> Labelled (GenTxId blk)
exampleGenTx :: forall blk. Examples blk -> Labelled (GenTx blk)
exampleHeaderHash :: forall blk. Examples blk -> Labelled (HeaderHash blk)
exampleSerialisedHeader :: forall blk. Examples blk -> Labelled (SerialisedHeader blk)
exampleHeader :: forall blk. Examples blk -> Labelled (Header blk)
exampleSerialisedBlock :: forall blk. Examples blk -> Labelled (Serialised blk)
exampleBlock :: forall blk. Examples blk -> Labelled blk
exampleBlock :: Labelled blk
exampleSerialisedBlock :: Labelled (Serialised blk)
exampleHeader :: Labelled (Header blk)
exampleSerialisedHeader :: Labelled (SerialisedHeader blk)
exampleHeaderHash :: Labelled (HeaderHash blk)
exampleGenTx :: Labelled (GenTx blk)
exampleGenTxId :: Labelled (GenTxId blk)
exampleApplyTxErr :: Labelled (ApplyTxErr blk)
exampleQuery :: Labelled (SomeBlockQuery (BlockQuery blk))
exampleResult :: Labelled (SomeResult blk)
exampleAnnTip :: Labelled (AnnTip blk)
exampleLedgerState :: Labelled (LedgerState blk EmptyMK)
exampleChainDepState :: Labelled (ChainDepState (BlockProtocol blk))
exampleExtLedgerState :: Labelled (ExtLedgerState blk EmptyMK)
exampleSlotNo :: Labelled SlotNo
exampleLedgerConfig :: Labelled (LedgerConfig blk)
exampleLedgerTables :: Labelled (LedgerTables (LedgerState blk) ValuesMK)
..} =
  TestName -> [TestTree] -> TestTree
testGroup
    TestName
"SerialiseNodeToNode"
    [ BlockNodeToNodeVersion blk -> TestTree
testVersion BlockNodeToNodeVersion blk
version
    | BlockNodeToNodeVersion blk
version <- [BlockNodeToNodeVersion blk] -> [BlockNodeToNodeVersion blk]
forall a. Eq a => [a] -> [a]
nub ([BlockNodeToNodeVersion blk] -> [BlockNodeToNodeVersion blk])
-> [BlockNodeToNodeVersion blk] -> [BlockNodeToNodeVersion blk]
forall a b. (a -> b) -> a -> b
$ Map NodeToNodeVersion (BlockNodeToNodeVersion blk)
-> [BlockNodeToNodeVersion blk]
forall k a. Map k a -> [a]
Map.elems (Map NodeToNodeVersion (BlockNodeToNodeVersion blk)
 -> [BlockNodeToNodeVersion blk])
-> Map NodeToNodeVersion (BlockNodeToNodeVersion blk)
-> [BlockNodeToNodeVersion blk]
forall a b. (a -> b) -> a -> b
$ Proxy blk -> Map NodeToNodeVersion (BlockNodeToNodeVersion blk)
forall blk.
SupportedNetworkProtocolVersion blk =>
Proxy blk -> Map NodeToNodeVersion (BlockNodeToNodeVersion blk)
supportedNodeToNodeVersions (Proxy blk -> Map NodeToNodeVersion (BlockNodeToNodeVersion blk))
-> Proxy blk -> Map NodeToNodeVersion (BlockNodeToNodeVersion blk)
forall a b. (a -> b) -> a -> b
$ forall t. Proxy t
forall {k} (t :: k). Proxy t
Proxy @blk
    ]
 where
  testVersion :: BlockNodeToNodeVersion blk -> TestTree
  testVersion :: BlockNodeToNodeVersion blk -> TestTree
testVersion BlockNodeToNodeVersion blk
version =
    TestName -> [TestTree] -> TestTree
testGroup
      (BlockNodeToNodeVersion blk -> TestName
forall a. ToGoldenDirectory a => a -> TestName
toGoldenDirectory BlockNodeToNodeVersion blk
version)
      [ TestName -> Labelled blk -> TestTree
forall a.
SerialiseNodeToNode blk a =>
TestName -> Labelled a -> TestTree
test TestName
"Block" Labelled blk
exampleBlock
      , TestName -> Labelled (Header blk) -> TestTree
forall a.
SerialiseNodeToNode blk a =>
TestName -> Labelled a -> TestTree
test TestName
"Header" Labelled (Header blk)
exampleHeader
      , TestName -> Labelled (Serialised blk) -> TestTree
forall a.
SerialiseNodeToNode blk a =>
TestName -> Labelled a -> TestTree
test TestName
"SerialisedBlock" Labelled (Serialised blk)
exampleSerialisedBlock
      , TestName -> Labelled (SerialisedHeader blk) -> TestTree
forall a.
SerialiseNodeToNode blk a =>
TestName -> Labelled a -> TestTree
test TestName
"SerialisedHeader" Labelled (SerialisedHeader blk)
exampleSerialisedHeader
      , TestName -> Labelled (GenTx blk) -> TestTree
forall a.
SerialiseNodeToNode blk a =>
TestName -> Labelled a -> TestTree
test TestName
"GenTx" Labelled (GenTx blk)
exampleGenTx
      , TestName -> Labelled (GenTxId blk) -> TestTree
forall a.
SerialiseNodeToNode blk a =>
TestName -> Labelled a -> TestTree
test TestName
"GenTxId" Labelled (GenTxId blk)
exampleGenTxId
      ]
   where
    test :: SerialiseNodeToNode blk a => TestName -> Labelled a -> TestTree
    test :: forall a.
SerialiseNodeToNode blk a =>
TestName -> Labelled a -> TestTree
test TestName
testName Labelled a
exampleValues =
      TestName -> Labelled a -> (a -> Encoding) -> TestName -> TestTree
forall a.
HasCallStack =>
TestName -> Labelled a -> (a -> Encoding) -> TestName -> TestTree
goldenTests
        TestName
testName
        Labelled a
exampleValues
        (CodecConfig blk -> BlockNodeToNodeVersion blk -> a -> Encoding
forall blk a.
SerialiseNodeToNode blk a =>
CodecConfig blk -> BlockNodeToNodeVersion blk -> a -> Encoding
encodeNodeToNode CodecConfig blk
codecConfig BlockNodeToNodeVersion blk
version)
        (TestName
goldenDir TestName -> TestName -> TestName
</> BlockNodeToNodeVersion blk -> TestName
forall a. ToGoldenDirectory a => a -> TestName
toGoldenDirectory BlockNodeToNodeVersion blk
version)

-- TODO how can we ensure that we have a test for each constraint listed in
-- 'SerialiseNodeToClientConstraints'?
goldenTest_SerialiseNodeToClient ::
  forall blk.
  ( SerialiseNodeToClientConstraints blk
  , SupportedNetworkProtocolVersion blk
  , ToGoldenDirectory (QueryVersion, BlockNodeToClientVersion blk)
  , HasCallStack
  ) =>
  CodecConfig blk ->
  FilePath ->
  Examples blk ->
  TestTree
goldenTest_SerialiseNodeToClient :: forall blk.
(SerialiseNodeToClientConstraints blk,
 SupportedNetworkProtocolVersion blk,
 ToGoldenDirectory (QueryVersion, BlockNodeToClientVersion blk),
 HasCallStack) =>
CodecConfig blk -> TestName -> Examples blk -> TestTree
goldenTest_SerialiseNodeToClient CodecConfig blk
codecConfig TestName
goldenDir Examples{Labelled blk
Labelled SlotNo
Labelled (HeaderHash blk)
Labelled (Header blk)
Labelled (LedgerTables (LedgerState blk) ValuesMK)
Labelled (ChainDepState (BlockProtocol blk))
Labelled (LedgerState blk EmptyMK)
Labelled (LedgerCfg (LedgerState blk))
Labelled (GenTxId blk)
Labelled (ApplyTxErr blk)
Labelled (GenTx blk)
Labelled (AnnTip blk)
Labelled (SerialisedHeader blk)
Labelled (ExtLedgerState blk EmptyMK)
Labelled (SomeBlockQuery (BlockQuery blk))
Labelled (Serialised blk)
Labelled (SomeResult blk)
exampleLedgerTables :: forall blk.
Examples blk -> Labelled (LedgerTables (LedgerState blk) ValuesMK)
exampleLedgerConfig :: forall blk. Examples blk -> Labelled (LedgerConfig blk)
exampleSlotNo :: forall blk. Examples blk -> Labelled SlotNo
exampleExtLedgerState :: forall blk. Examples blk -> Labelled (ExtLedgerState blk EmptyMK)
exampleChainDepState :: forall blk.
Examples blk -> Labelled (ChainDepState (BlockProtocol blk))
exampleLedgerState :: forall blk. Examples blk -> Labelled (LedgerState blk EmptyMK)
exampleAnnTip :: forall blk. Examples blk -> Labelled (AnnTip blk)
exampleResult :: forall blk. Examples blk -> Labelled (SomeResult blk)
exampleQuery :: forall blk.
Examples blk -> Labelled (SomeBlockQuery (BlockQuery blk))
exampleApplyTxErr :: forall blk. Examples blk -> Labelled (ApplyTxErr blk)
exampleGenTxId :: forall blk. Examples blk -> Labelled (GenTxId blk)
exampleGenTx :: forall blk. Examples blk -> Labelled (GenTx blk)
exampleHeaderHash :: forall blk. Examples blk -> Labelled (HeaderHash blk)
exampleSerialisedHeader :: forall blk. Examples blk -> Labelled (SerialisedHeader blk)
exampleHeader :: forall blk. Examples blk -> Labelled (Header blk)
exampleSerialisedBlock :: forall blk. Examples blk -> Labelled (Serialised blk)
exampleBlock :: forall blk. Examples blk -> Labelled blk
exampleBlock :: Labelled blk
exampleSerialisedBlock :: Labelled (Serialised blk)
exampleHeader :: Labelled (Header blk)
exampleSerialisedHeader :: Labelled (SerialisedHeader blk)
exampleHeaderHash :: Labelled (HeaderHash blk)
exampleGenTx :: Labelled (GenTx blk)
exampleGenTxId :: Labelled (GenTxId blk)
exampleApplyTxErr :: Labelled (ApplyTxErr blk)
exampleQuery :: Labelled (SomeBlockQuery (BlockQuery blk))
exampleResult :: Labelled (SomeResult blk)
exampleAnnTip :: Labelled (AnnTip blk)
exampleLedgerState :: Labelled (LedgerState blk EmptyMK)
exampleChainDepState :: Labelled (ChainDepState (BlockProtocol blk))
exampleExtLedgerState :: Labelled (ExtLedgerState blk EmptyMK)
exampleSlotNo :: Labelled SlotNo
exampleLedgerConfig :: Labelled (LedgerCfg (LedgerState blk))
exampleLedgerTables :: Labelled (LedgerTables (LedgerState blk) ValuesMK)
..} =
  TestName -> [TestTree] -> TestTree
testGroup
    TestName
"SerialiseNodeToClient"
    [ (QueryVersion, BlockNodeToClientVersion blk) -> TestTree
testVersion (QueryVersion
queryVersion, BlockNodeToClientVersion blk
blockVersion)
    | (QueryVersion
queryVersion, BlockNodeToClientVersion blk
blockVersion) <-
        [(QueryVersion, BlockNodeToClientVersion blk)]
-> [(QueryVersion, BlockNodeToClientVersion blk)]
forall a. Eq a => [a] -> [a]
nub ([(QueryVersion, BlockNodeToClientVersion blk)]
 -> [(QueryVersion, BlockNodeToClientVersion blk)])
-> (Map NodeToClientVersion (BlockNodeToClientVersion blk)
    -> [(QueryVersion, BlockNodeToClientVersion blk)])
-> Map NodeToClientVersion (BlockNodeToClientVersion blk)
-> [(QueryVersion, BlockNodeToClientVersion blk)]
forall b c a. (b -> c) -> (a -> b) -> a -> c
. ((NodeToClientVersion, BlockNodeToClientVersion blk)
 -> (QueryVersion, BlockNodeToClientVersion blk))
-> [(NodeToClientVersion, BlockNodeToClientVersion blk)]
-> [(QueryVersion, BlockNodeToClientVersion blk)]
forall a b. (a -> b) -> [a] -> [b]
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap ((NodeToClientVersion -> QueryVersion)
-> (NodeToClientVersion, BlockNodeToClientVersion blk)
-> (QueryVersion, BlockNodeToClientVersion blk)
forall a b c. (a -> b) -> (a, c) -> (b, c)
forall (p :: * -> * -> *) a b c.
Bifunctor p =>
(a -> b) -> p a c -> p b c
first NodeToClientVersion -> QueryVersion
nodeToClientVersionToQueryVersion) ([(NodeToClientVersion, BlockNodeToClientVersion blk)]
 -> [(QueryVersion, BlockNodeToClientVersion blk)])
-> (Map NodeToClientVersion (BlockNodeToClientVersion blk)
    -> [(NodeToClientVersion, BlockNodeToClientVersion blk)])
-> Map NodeToClientVersion (BlockNodeToClientVersion blk)
-> [(QueryVersion, BlockNodeToClientVersion blk)]
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Map NodeToClientVersion (BlockNodeToClientVersion blk)
-> [(NodeToClientVersion, BlockNodeToClientVersion blk)]
forall k a. Map k a -> [(k, a)]
Map.toList (Map NodeToClientVersion (BlockNodeToClientVersion blk)
 -> [(QueryVersion, BlockNodeToClientVersion blk)])
-> Map NodeToClientVersion (BlockNodeToClientVersion blk)
-> [(QueryVersion, BlockNodeToClientVersion blk)]
forall a b. (a -> b) -> a -> b
$
          Proxy blk -> Map NodeToClientVersion (BlockNodeToClientVersion blk)
forall blk.
SupportedNetworkProtocolVersion blk =>
Proxy blk -> Map NodeToClientVersion (BlockNodeToClientVersion blk)
supportedNodeToClientVersions (forall t. Proxy t
forall {k} (t :: k). Proxy t
Proxy @blk)
    ]
 where
  testVersion :: (QueryVersion, BlockNodeToClientVersion blk) -> TestTree
  testVersion :: (QueryVersion, BlockNodeToClientVersion blk) -> TestTree
testVersion versions :: (QueryVersion, BlockNodeToClientVersion blk)
versions@(QueryVersion
_, BlockNodeToClientVersion blk
blockVersion) =
    TestName -> [TestTree] -> TestTree
testGroup
      ((QueryVersion, BlockNodeToClientVersion blk) -> TestName
forall a. ToGoldenDirectory a => a -> TestName
toGoldenDirectory (QueryVersion, BlockNodeToClientVersion blk)
versions)
      [ TestName -> Labelled blk -> (blk -> Encoding) -> TestTree
forall a. TestName -> Labelled a -> (a -> Encoding) -> TestTree
test TestName
"Block" Labelled blk
exampleBlock blk -> Encoding
forall a. SerialiseNodeToClient blk a => a -> Encoding
enc'
      , TestName
-> Labelled (Serialised blk)
-> (Serialised blk -> Encoding)
-> TestTree
forall a. TestName -> Labelled a -> (a -> Encoding) -> TestTree
test TestName
"SerialisedBlock" Labelled (Serialised blk)
exampleSerialisedBlock Serialised blk -> Encoding
forall a. SerialiseNodeToClient blk a => a -> Encoding
enc'
      , TestName
-> Labelled (GenTx blk) -> (GenTx blk -> Encoding) -> TestTree
forall a. TestName -> Labelled a -> (a -> Encoding) -> TestTree
test TestName
"GenTx" Labelled (GenTx blk)
exampleGenTx GenTx blk -> Encoding
forall a. SerialiseNodeToClient blk a => a -> Encoding
enc'
      , TestName
-> Labelled (GenTxId blk) -> (GenTxId blk -> Encoding) -> TestTree
forall a. TestName -> Labelled a -> (a -> Encoding) -> TestTree
test TestName
"GenTxId" Labelled (GenTxId blk)
exampleGenTxId GenTxId blk -> Encoding
forall a. SerialiseNodeToClient blk a => a -> Encoding
enc'
      , TestName
-> Labelled (ApplyTxErr blk)
-> (ApplyTxErr blk -> Encoding)
-> TestTree
forall a. TestName -> Labelled a -> (a -> Encoding) -> TestTree
test TestName
"ApplyTxErr" Labelled (ApplyTxErr blk)
exampleApplyTxErr ApplyTxErr blk -> Encoding
forall a. SerialiseNodeToClient blk a => a -> Encoding
enc'
      , TestName
-> Labelled (SomeBlockQuery (BlockQuery blk))
-> (SomeBlockQuery (BlockQuery blk) -> Encoding)
-> TestTree
forall a. TestName -> Labelled a -> (a -> Encoding) -> TestTree
test TestName
"Query" Labelled (SomeBlockQuery (BlockQuery blk))
exampleQuery SomeBlockQuery (BlockQuery blk) -> Encoding
forall a. SerialiseNodeToClient blk a => a -> Encoding
enc'
      , TestName -> Labelled SlotNo -> (SlotNo -> Encoding) -> TestTree
forall a. TestName -> Labelled a -> (a -> Encoding) -> TestTree
test TestName
"SlotNo" Labelled SlotNo
exampleSlotNo SlotNo -> Encoding
forall a. SerialiseNodeToClient blk a => a -> Encoding
enc'
      , TestName
-> Labelled (LedgerCfg (LedgerState blk))
-> (LedgerCfg (LedgerState blk) -> Encoding)
-> TestTree
forall a. TestName -> Labelled a -> (a -> Encoding) -> TestTree
test TestName
"LedgerConfig" Labelled (LedgerCfg (LedgerState blk))
exampleLedgerConfig LedgerCfg (LedgerState blk) -> Encoding
forall a. SerialiseNodeToClient blk a => a -> Encoding
enc'
      , TestName
-> Labelled (SomeResult blk)
-> (SomeResult blk -> Encoding)
-> TestTree
forall a. TestName -> Labelled a -> (a -> Encoding) -> TestTree
test TestName
"Result" Labelled (SomeResult blk)
exampleResult SomeResult blk -> Encoding
encRes
      ]
   where
    enc' :: SerialiseNodeToClient blk a => a -> Encoding
    enc' :: forall a. SerialiseNodeToClient blk a => a -> Encoding
enc' = CodecConfig blk -> BlockNodeToClientVersion blk -> a -> Encoding
forall blk a.
SerialiseNodeToClient blk a =>
CodecConfig blk -> BlockNodeToClientVersion blk -> a -> Encoding
encodeNodeToClient CodecConfig blk
codecConfig BlockNodeToClientVersion blk
blockVersion

    encRes :: SomeResult blk -> Encoding
    encRes :: SomeResult blk -> Encoding
encRes (SomeResult BlockQuery blk fp result
q result
r) = 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
codecConfig BlockNodeToClientVersion blk
blockVersion BlockQuery blk fp result
q result
r

    test :: TestName -> Labelled a -> (a -> Encoding) -> TestTree
    test :: forall a. TestName -> Labelled a -> (a -> Encoding) -> TestTree
test TestName
testName Labelled a
exampleValues a -> Encoding
enc =
      TestName -> Labelled a -> (a -> Encoding) -> TestName -> TestTree
forall a.
HasCallStack =>
TestName -> Labelled a -> (a -> Encoding) -> TestName -> TestTree
goldenTests
        TestName
testName
        Labelled a
exampleValues
        a -> Encoding
enc
        (TestName
goldenDir TestName -> TestName -> TestName
</> (QueryVersion, BlockNodeToClientVersion blk) -> TestName
forall a. ToGoldenDirectory a => a -> TestName
toGoldenDirectory (QueryVersion, BlockNodeToClientVersion blk)
versions)

{-------------------------------------------------------------------------------
  FlatTerm
-------------------------------------------------------------------------------}

instance Condense TermToken where
  condense :: TermToken -> TestName
condense = TermToken -> TestName
forall a. Show a => a -> TestName
show