{-# LANGUAGE FlexibleContexts    #-}
{-# LANGUAGE NamedFieldPuns      #-}
{-# LANGUAGE RankNTypes          #-}
{-# LANGUAGE RecordWildCards     #-}
{-# LANGUAGE ScopedTypeVariables #-}

{-# OPTIONS_GHC -Wno-incomplete-uni-patterns #-}

module Test.Ouroboros.Network.BlockFetch
  ( PeerGSVT (..)
  , tests
  ) where

import Test.ChainGenerators (TestChainFork (..))
import Test.QuickCheck
import Test.Tasty (TestTree, testGroup)
import Test.Tasty.HUnit
import Test.Tasty.QuickCheck (testProperty)

import Data.List qualified as List
import Data.Map (Map)
import Data.Map qualified as Map
import Data.Maybe (mapMaybe)
import Data.Proxy (Proxy (..))
import Data.Set (Set)
import Data.Set qualified as Set
import Data.Typeable (Typeable)

import Control.Concurrent.Class.MonadSTM.Strict
import Control.Exception (AssertionFailed (..), throw)
import Control.Monad (unless, void)
import Control.Monad.Class.MonadAsync
import Control.Monad.Class.MonadFork
import Control.Monad.Class.MonadThrow
import Control.Monad.Class.MonadTime.SI (Time (..))
import Control.Monad.Class.MonadTimer.SI
import Control.Monad.IOSim
import Control.Tracer (Tracer (Tracer), contramap, nullTracer)

import Ouroboros.Network.ControlMessage (ControlMessage (..), continueForever)
import Ouroboros.Network.DeltaQ
--TODO: could re-export some of the trace types from more convenient places:
import Ouroboros.Network.AnchoredFragment (AnchoredFragment)
import Ouroboros.Network.AnchoredFragment qualified as AnchoredFragment
import Ouroboros.Network.Block
import Ouroboros.Network.BlockFetch
import Ouroboros.Network.BlockFetch.ClientRegistry
import Ouroboros.Network.BlockFetch.ClientState
import Ouroboros.Network.BlockFetch.DeltaQ
import Ouroboros.Network.BlockFetch.Examples
import Ouroboros.Network.Driver (TraceSendRecv)
import Ouroboros.Network.Mock.Chain qualified as Chain
import Ouroboros.Network.Mock.ConcreteBlock
import Ouroboros.Network.NodeToNode.Version (isPipeliningEnabled)
import Ouroboros.Network.Protocol.BlockFetch.Type (BlockFetch)

import Ouroboros.Network.Testing.Utils


--
-- The list of all tests
--

tests :: TestTree
tests :: TestTree
tests = [Char] -> [TestTree] -> TestTree
testGroup [Char]
"BlockFetch"
  [ [Char] -> (TestChainFork -> Property) -> TestTree
forall a. Testable a => [Char] -> a -> TestTree
testProperty [Char]
"static chains without overlap"
                 TestChainFork -> Property
prop_blockFetchStaticNoOverlap

  , [Char] -> (TestChainFork -> Property) -> TestTree
forall a. Testable a => [Char] -> a -> TestTree
testProperty [Char]
"static chains with overlap"
                 TestChainFork -> Property
prop_blockFetchStaticWithOverlap

  , [Char] -> (([Char] -> IO ()) -> IO ()) -> TestTree
testCaseSteps [Char]
"bracketSyncWithFetchClient"
                  ([Char] -> IO ()) -> IO ()
unit_bracketSyncWithFetchClient

  --TODO: test where for any given delta-Q, check that we do achieve full
  -- pipelining to keep the server busy and get decent enough batching of
  -- requests (testing the high/low watermark mechanism).
  , [Char]
-> (TestChainFork -> Positive SmallDelay -> Property) -> TestTree
forall a. Testable a => [Char] -> a -> TestTree
testProperty [Char]
"termination"
                 TestChainFork -> Positive SmallDelay -> Property
prop_terminate
  , [Char]
-> (Int
    -> Int -> Int -> PeerGSVT -> PeerGSVT -> Bool -> Bool -> Property)
-> TestTree
forall a. Testable a => [Char] -> a -> TestTree
testProperty [Char]
"compare comparePeerGSV" Int
-> Int -> Int -> PeerGSVT -> PeerGSVT -> Bool -> Bool -> Property
prop_comparePeerGSV
  , [Char] -> (Int -> Int -> PeerGSVT -> Bool -> Property) -> TestTree
forall a. Testable a => [Char] -> a -> TestTree
testProperty [Char]
"eq comparePeerGSV" Int -> Int -> PeerGSVT -> Bool -> Property
prop_comparePeerGSVEq
  ]


--
-- Properties
--

-- | In this test we have two candidates chains that are static throughout the
-- run. The two chains share some common prefix (genesis in the degenerate
-- case). The test runs the block fetch logic to download all of both chain
-- candidates.
--
-- In this variant we set up the common prefix of the two candidates as the
-- \"current\" chain. This means the block fetch only has to download the
-- suffixes of the two candidates. This also means that the two suffixes are
-- guaranteed not to have any overlap in their blocks. We rely on this
-- guarantee in this special case to check stronger properties.
--
-- This runs the block fetch and then checks that the trace of the events in
-- that run satisfy the trace properties:
--
-- * 'tracePropertyBlocksRequestedAndRecievedPerPeer'
-- * 'tracePropertyClientStateSanity'
-- * 'tracePropertyInFlight'
--
prop_blockFetchStaticNoOverlap :: TestChainFork -> Property
prop_blockFetchStaticNoOverlap :: TestChainFork -> Property
prop_blockFetchStaticNoOverlap (TestChainFork Chain Block
common Chain Block
fork1 Chain Block
fork2) =
    let trace :: [Example1TraceEvent]
trace = SimTrace () -> [Example1TraceEvent]
forall a b. Typeable b => SimTrace a -> [b]
selectTraceEventsDynamic ((forall s. IOSim s ()) -> SimTrace ()
forall a. (forall s. IOSim s a) -> SimTrace a
runSimTrace IOSim s ()
forall s. IOSim s ()
simulation)

     in [Char] -> Property -> Property
forall prop. Testable prop => [Char] -> prop -> Property
counterexample ([Char]
"\nTrace:\n" [Char] -> [Char] -> [Char]
forall a. [a] -> [a] -> [a]
++ [[Char]] -> [Char]
unlines ((Example1TraceEvent -> [Char]) -> [Example1TraceEvent] -> [[Char]]
forall a b. (a -> b) -> [a] -> [b]
map Example1TraceEvent -> [Char]
forall a. Show a => a -> [Char]
show [Example1TraceEvent]
trace)) (Property -> Property) -> Property -> Property
forall a b. (a -> b) -> a -> b
$

        -- For fetch reqs added and received, we observe exactly the sequence
        -- of blocks we expect, which is the whole fork suffix.
        AnchoredFragment Block
-> AnchoredFragment Block -> [Example1TraceEvent] -> Property
tracePropertyBlocksRequestedAndRecievedPerPeer AnchoredFragment Block
fork1'' AnchoredFragment Block
fork2'' [Example1TraceEvent]
trace

        -- state sanity check
   Property -> Property -> Property
forall prop1 prop2.
(Testable prop1, Testable prop2) =>
prop1 -> prop2 -> Property
.&&. Bool -> Property
forall prop. Testable prop => prop -> Property
property ([Example1TraceEvent] -> Bool
tracePropertyClientStateSanity [Example1TraceEvent]
trace)

        -- check in-flight requests
   Property -> Property -> Property
forall prop1 prop2.
(Testable prop1, Testable prop2) =>
prop1 -> prop2 -> Property
.&&. [Example1TraceEvent] -> Property
tracePropertyInFlight [Example1TraceEvent]
trace

  where
    simulation :: IOSim s ()
    simulation :: forall s. IOSim s ()
simulation =
      Tracer
  (IOSim s) [TraceLabelPeer Int (FetchDecision [Point BlockHeader])]
-> Tracer
     (IOSim s) (TraceLabelPeer Int (TraceFetchClientState BlockHeader))
-> Tracer
     (IOSim s)
     (TraceLabelPeer
        Int (TraceSendRecv (BlockFetch Block (Point Block))))
-> Maybe DiffTime
-> Maybe DiffTime
-> ControlMessageSTM (IOSim s)
-> AnchoredFragment Block
-> [AnchoredFragment Block]
-> IOSim s ()
forall (m :: * -> *).
(MonadSTM m, MonadST m, MonadAsync m, MonadDelay m, MonadFork m,
 MonadTime m, MonadTimer m, MonadMask m, MonadThrow (STM m)) =>
Tracer m [TraceLabelPeer Int (FetchDecision [Point BlockHeader])]
-> Tracer
     m (TraceLabelPeer Int (TraceFetchClientState BlockHeader))
-> Tracer
     m
     (TraceLabelPeer
        Int (TraceSendRecv (BlockFetch Block (Point Block))))
-> Maybe DiffTime
-> Maybe DiffTime
-> ControlMessageSTM m
-> AnchoredFragment Block
-> [AnchoredFragment Block]
-> m ()
blockFetchExample1
        (([TraceLabelPeer Int (FetchDecision [Point BlockHeader])]
 -> Example1TraceEvent)
-> Tracer (IOSim s) Example1TraceEvent
-> Tracer
     (IOSim s) [TraceLabelPeer Int (FetchDecision [Point BlockHeader])]
forall a' a. (a' -> a) -> Tracer (IOSim s) a -> Tracer (IOSim s) a'
forall (f :: * -> *) a' a.
Contravariant f =>
(a' -> a) -> f a -> f a'
contramap [TraceLabelPeer Int (FetchDecision [Point BlockHeader])]
-> Example1TraceEvent
TraceFetchDecision       Tracer (IOSim s) Example1TraceEvent
forall a s. Typeable a => Tracer (IOSim s) a
dynamicTracer)
        ((TraceLabelPeer Int (TraceFetchClientState BlockHeader)
 -> Example1TraceEvent)
-> Tracer (IOSim s) Example1TraceEvent
-> Tracer
     (IOSim s) (TraceLabelPeer Int (TraceFetchClientState BlockHeader))
forall a' a. (a' -> a) -> Tracer (IOSim s) a -> Tracer (IOSim s) a'
forall (f :: * -> *) a' a.
Contravariant f =>
(a' -> a) -> f a -> f a'
contramap TraceLabelPeer Int (TraceFetchClientState BlockHeader)
-> Example1TraceEvent
TraceFetchClientState    Tracer (IOSim s) Example1TraceEvent
forall a s. Typeable a => Tracer (IOSim s) a
dynamicTracer)
        ((TraceLabelPeer
   Int (TraceSendRecv (BlockFetch Block (Point Block)))
 -> Example1TraceEvent)
-> Tracer (IOSim s) Example1TraceEvent
-> Tracer
     (IOSim s)
     (TraceLabelPeer
        Int (TraceSendRecv (BlockFetch Block (Point Block))))
forall a' a. (a' -> a) -> Tracer (IOSim s) a -> Tracer (IOSim s) a'
forall (f :: * -> *) a' a.
Contravariant f =>
(a' -> a) -> f a -> f a'
contramap TraceLabelPeer Int (TraceSendRecv (BlockFetch Block (Point Block)))
-> Example1TraceEvent
TraceFetchClientSendRecv Tracer (IOSim s) Example1TraceEvent
forall a s. Typeable a => Tracer (IOSim s) a
dynamicTracer)
        Maybe DiffTime
forall a. Maybe a
Nothing Maybe DiffTime
forall a. Maybe a
Nothing
        (Proxy (IOSim s) -> ControlMessageSTM (IOSim s)
forall (m :: * -> *) (proxy :: (* -> *) -> *).
Applicative (STM m) =>
proxy m -> ControlMessageSTM m
continueForever (Proxy (IOSim s)
forall {s}. Proxy (IOSim s)
forall {k} (t :: k). Proxy t
Proxy :: Proxy (IOSim s)))
        AnchoredFragment Block
common' [AnchoredFragment Block]
forks

    -- TODO: consider making a specific generator for anchored fragment forks
    common' :: AnchoredFragment Block
common' = Chain Block -> AnchoredFragment Block
chainToAnchoredFragment Chain Block
common
    fork1' :: AnchoredFragment Block
fork1'  = Chain Block -> AnchoredFragment Block
chainToAnchoredFragment Chain Block
fork1
    fork2' :: AnchoredFragment Block
fork2'  = Chain Block -> AnchoredFragment Block
chainToAnchoredFragment Chain Block
fork2
    forks :: [AnchoredFragment Block]
forks   = [AnchoredFragment Block
fork1', AnchoredFragment Block
fork2']
    -- And just the extensions
    Just (AnchoredFragment Block
_, AnchoredFragment Block
_, AnchoredFragment Block
fork1'', AnchoredFragment Block
fork2'') = AnchoredFragment Block
-> AnchoredFragment Block
-> Maybe
     (AnchoredFragment Block, AnchoredFragment Block,
      AnchoredFragment Block, AnchoredFragment Block)
forall block1 block2.
(HasHeader block1, HasHeader block2,
 HeaderHash block1 ~ HeaderHash block2) =>
AnchoredFragment block1
-> AnchoredFragment block2
-> Maybe
     (AnchoredFragment block1, AnchoredFragment block2,
      AnchoredFragment block1, AnchoredFragment block2)
AnchoredFragment.intersect AnchoredFragment Block
fork1' AnchoredFragment Block
fork2'


-- | In this test we have two candidates chains that are static throughout the
-- run. The two chains share some common prefix (genesis in the degenerate
-- case). The test runs the block fetch logic to download all of both chain
-- candidates.
--
-- In this variant we set up the \"current\" chain as empty (genesis). This
-- means the block has to download the whole of both candidates. This also
-- means that we typically expect there to be overlap in the blocks in the two
-- chains. We rely on there typically being overlap to check properties of that
-- overlap.
--
-- This runs the block fetch and then checks that the trace of the events in
-- that run satisfy the trace properties:
--
-- * 'tracePropertyBlocksRequestedAndRecievedAllPeers'
-- * 'tracePropertyNoDuplicateBlocksBetweenPeers'
-- * 'tracePropertyClientStateSanity'
-- * 'tracePropertyInFlight'
--
-- TODO: 'prop_blockFetchStaticWithOverlap' fails if we introduce delays. issue #2622
--
prop_blockFetchStaticWithOverlap :: TestChainFork -> Property
prop_blockFetchStaticWithOverlap :: TestChainFork -> Property
prop_blockFetchStaticWithOverlap (TestChainFork Chain Block
_common Chain Block
fork1 Chain Block
fork2) =
    let trace :: [Example1TraceEvent]
trace = SimTrace () -> [Example1TraceEvent]
forall a b. Typeable b => SimTrace a -> [b]
selectTraceEventsDynamic ((forall s. IOSim s ()) -> SimTrace ()
forall a. (forall s. IOSim s a) -> SimTrace a
runSimTrace IOSim s ()
forall s. IOSim s ()
simulation)

     in [Char] -> Property -> Property
forall prop. Testable prop => [Char] -> prop -> Property
counterexample ([Char]
"\nTrace:\n" [Char] -> [Char] -> [Char]
forall a. [a] -> [a] -> [a]
++ [[Char]] -> [Char]
unlines ((Example1TraceEvent -> [Char]) -> [Example1TraceEvent] -> [[Char]]
forall a b. (a -> b) -> [a] -> [b]
map Example1TraceEvent -> [Char]
forall a. Show a => a -> [Char]
show [Example1TraceEvent]
trace)) (Property -> Property) -> Property -> Property
forall a b. (a -> b) -> a -> b
$

        -- For fetch reqs added and received, between the two peers we observe
        -- the set of blocks we expect, which is the union of the two chains.
        AnchoredFragment Block
-> AnchoredFragment Block -> [Example1TraceEvent] -> Property
tracePropertyBlocksRequestedAndRecievedAllPeers AnchoredFragment Block
fork1' AnchoredFragment Block
fork2' [Example1TraceEvent]
trace

        -- For fetch reqs added, the set of blocks added for the two peers
        -- should not intersect
   Property -> Property -> Property
forall prop1 prop2.
(Testable prop1, Testable prop2) =>
prop1 -> prop2 -> Property
.&&. AnchoredFragment Block
-> AnchoredFragment Block -> [Example1TraceEvent] -> Property
tracePropertyNoDuplicateBlocksBetweenPeers AnchoredFragment Block
fork1' AnchoredFragment Block
fork2' [Example1TraceEvent]
trace

        -- state sanity check
   Property -> Property -> Property
forall prop1 prop2.
(Testable prop1, Testable prop2) =>
prop1 -> prop2 -> Property
.&&. Bool -> Property
forall prop. Testable prop => prop -> Property
property ([Example1TraceEvent] -> Bool
tracePropertyClientStateSanity [Example1TraceEvent]
trace)

        -- check in-flight requests
   Property -> Property -> Property
forall prop1 prop2.
(Testable prop1, Testable prop2) =>
prop1 -> prop2 -> Property
.&&. [Example1TraceEvent] -> Property
tracePropertyInFlight [Example1TraceEvent]
trace

  where
    simulation :: forall s. IOSim s ()
    simulation :: forall s. IOSim s ()
simulation =
      Tracer
  (IOSim s) [TraceLabelPeer Int (FetchDecision [Point BlockHeader])]
-> Tracer
     (IOSim s) (TraceLabelPeer Int (TraceFetchClientState BlockHeader))
-> Tracer
     (IOSim s)
     (TraceLabelPeer
        Int (TraceSendRecv (BlockFetch Block (Point Block))))
-> Maybe DiffTime
-> Maybe DiffTime
-> ControlMessageSTM (IOSim s)
-> AnchoredFragment Block
-> [AnchoredFragment Block]
-> IOSim s ()
forall (m :: * -> *).
(MonadSTM m, MonadST m, MonadAsync m, MonadDelay m, MonadFork m,
 MonadTime m, MonadTimer m, MonadMask m, MonadThrow (STM m)) =>
Tracer m [TraceLabelPeer Int (FetchDecision [Point BlockHeader])]
-> Tracer
     m (TraceLabelPeer Int (TraceFetchClientState BlockHeader))
-> Tracer
     m
     (TraceLabelPeer
        Int (TraceSendRecv (BlockFetch Block (Point Block))))
-> Maybe DiffTime
-> Maybe DiffTime
-> ControlMessageSTM m
-> AnchoredFragment Block
-> [AnchoredFragment Block]
-> m ()
blockFetchExample1
        (([TraceLabelPeer Int (FetchDecision [Point BlockHeader])]
 -> Example1TraceEvent)
-> Tracer (IOSim s) Example1TraceEvent
-> Tracer
     (IOSim s) [TraceLabelPeer Int (FetchDecision [Point BlockHeader])]
forall a' a. (a' -> a) -> Tracer (IOSim s) a -> Tracer (IOSim s) a'
forall (f :: * -> *) a' a.
Contravariant f =>
(a' -> a) -> f a -> f a'
contramap [TraceLabelPeer Int (FetchDecision [Point BlockHeader])]
-> Example1TraceEvent
TraceFetchDecision       Tracer (IOSim s) Example1TraceEvent
forall a s. Typeable a => Tracer (IOSim s) a
dynamicTracer)
        ((TraceLabelPeer Int (TraceFetchClientState BlockHeader)
 -> Example1TraceEvent)
-> Tracer (IOSim s) Example1TraceEvent
-> Tracer
     (IOSim s) (TraceLabelPeer Int (TraceFetchClientState BlockHeader))
forall a' a. (a' -> a) -> Tracer (IOSim s) a -> Tracer (IOSim s) a'
forall (f :: * -> *) a' a.
Contravariant f =>
(a' -> a) -> f a -> f a'
contramap TraceLabelPeer Int (TraceFetchClientState BlockHeader)
-> Example1TraceEvent
TraceFetchClientState    Tracer (IOSim s) Example1TraceEvent
forall a s. Typeable a => Tracer (IOSim s) a
dynamicTracer)
        ((TraceLabelPeer
   Int (TraceSendRecv (BlockFetch Block (Point Block)))
 -> Example1TraceEvent)
-> Tracer (IOSim s) Example1TraceEvent
-> Tracer
     (IOSim s)
     (TraceLabelPeer
        Int (TraceSendRecv (BlockFetch Block (Point Block))))
forall a' a. (a' -> a) -> Tracer (IOSim s) a -> Tracer (IOSim s) a'
forall (f :: * -> *) a' a.
Contravariant f =>
(a' -> a) -> f a -> f a'
contramap TraceLabelPeer Int (TraceSendRecv (BlockFetch Block (Point Block)))
-> Example1TraceEvent
TraceFetchClientSendRecv Tracer (IOSim s) Example1TraceEvent
forall a s. Typeable a => Tracer (IOSim s) a
dynamicTracer)
        Maybe DiffTime
forall a. Maybe a
Nothing Maybe DiffTime
forall a. Maybe a
Nothing
        (Proxy (IOSim s) -> ControlMessageSTM (IOSim s)
forall (m :: * -> *) (proxy :: (* -> *) -> *).
Applicative (STM m) =>
proxy m -> ControlMessageSTM m
continueForever (Proxy (IOSim s)
forall {k} (t :: k). Proxy t
Proxy :: Proxy (IOSim s)))
        (Anchor Block -> AnchoredFragment Block
forall v a b. Anchorable v a b => a -> AnchoredSeq v a b
AnchoredFragment.Empty Anchor Block
forall block. Anchor block
AnchoredFragment.AnchorGenesis)
        [AnchoredFragment Block]
forks

    -- TODO: consider making a specific generator for anchored fragment forks
    fork1' :: AnchoredFragment Block
fork1'  = Chain Block -> AnchoredFragment Block
chainToAnchoredFragment Chain Block
fork1
    fork2' :: AnchoredFragment Block
fork2'  = Chain Block -> AnchoredFragment Block
chainToAnchoredFragment Chain Block
fork2
    forks :: [AnchoredFragment Block]
forks   = [AnchoredFragment Block
fork1', AnchoredFragment Block
fork2']

chainToAnchoredFragment :: Chain.Chain Block -> AnchoredFragment Block
chainToAnchoredFragment :: Chain Block -> AnchoredFragment Block
chainToAnchoredFragment =
    Anchor Block -> [Block] -> AnchoredFragment Block
forall v a b. Anchorable v a b => a -> [b] -> AnchoredSeq v a b
AnchoredFragment.fromNewestFirst Anchor Block
forall block. Anchor block
AnchoredFragment.AnchorGenesis
  ([Block] -> AnchoredFragment Block)
-> (Chain Block -> [Block])
-> Chain Block
-> AnchoredFragment Block
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Chain Block -> [Block]
forall block. Chain block -> [block]
Chain.chainToList

-- TODO: move elsewhere and generalise
chainPoints :: AnchoredFragment Block -> [Point BlockHeader]
chainPoints :: AnchoredFragment Block -> [Point BlockHeader]
chainPoints = (Block -> Point BlockHeader) -> [Block] -> [Point BlockHeader]
forall a b. (a -> b) -> [a] -> [b]
map (Point Block -> Point BlockHeader
forall {k1} {k2} (b :: k1) (b' :: k2).
Coercible (HeaderHash b) (HeaderHash b') =>
Point b -> Point b'
castPoint (Point Block -> Point BlockHeader)
-> (Block -> Point Block) -> Block -> Point BlockHeader
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Block -> Point Block
forall block. HasHeader block => block -> Point block
blockPoint)
            ([Block] -> [Point BlockHeader])
-> (AnchoredFragment Block -> [Block])
-> AnchoredFragment Block
-> [Point BlockHeader]
forall b c a. (b -> c) -> (a -> b) -> a -> c
. AnchoredFragment Block -> [Block]
forall v a b. AnchoredSeq v a b -> [b]
AnchoredFragment.toOldestFirst

data Example1TraceEvent =
     TraceFetchDecision       [TraceLabelPeer Int
                                (FetchDecision [Point BlockHeader])]
   | TraceFetchClientState    (TraceLabelPeer Int
                                (TraceFetchClientState BlockHeader))
   | TraceFetchClientSendRecv (TraceLabelPeer Int
                                (TraceSendRecv (BlockFetch Block (Point Block))))

instance Show Example1TraceEvent where
  show :: Example1TraceEvent -> [Char]
show (TraceFetchDecision       [TraceLabelPeer Int (FetchDecision [Point BlockHeader])]
x) = [Char]
"TraceFetchDecision " [Char] -> [Char] -> [Char]
forall a. [a] -> [a] -> [a]
++ [TraceLabelPeer Int (FetchDecision [Point BlockHeader])] -> [Char]
forall a. Show a => a -> [Char]
show [TraceLabelPeer Int (FetchDecision [Point BlockHeader])]
x
  show (TraceFetchClientState    TraceLabelPeer Int (TraceFetchClientState BlockHeader)
x) = TraceLabelPeer Int (TraceFetchClientState BlockHeader) -> [Char]
forall a. Show a => a -> [Char]
show TraceLabelPeer Int (TraceFetchClientState BlockHeader)
x
  show (TraceFetchClientSendRecv TraceLabelPeer Int (TraceSendRecv (BlockFetch Block (Point Block)))
x) = TraceLabelPeer Int (TraceSendRecv (BlockFetch Block (Point Block)))
-> [Char]
forall a. Show a => a -> [Char]
show TraceLabelPeer Int (TraceSendRecv (BlockFetch Block (Point Block)))
x


-- | Check the execution trace for a particular property: we observe all the
-- blocks in the 'FetchRequest's added by the decision logic and the blocks
-- received by the fetch clients; check that the ordered sequence of blocks
-- requested and completed by both fetch clients is exactly the sequence
-- expected. The expected sequence is exactly the chain suffixes in order.
--
-- This property is stronger than 'tracePropertyBlocksRequestedAndRecievedAllPeers'
-- since it works with sequences rather than sets and for each chain
-- individually rather than both chains together. It only holds for the
-- situation where the suffixes of the chains that need to be fetched are
-- disjoint, sharing no common prefix.
--
-- It turns out that no duplicates part is not trivial. Earlier versions of the
-- block fetch logic did not satisfy this in all cases.
--
tracePropertyBlocksRequestedAndRecievedPerPeer
  :: AnchoredFragment Block
  -> AnchoredFragment Block
  -> [Example1TraceEvent]
  -> Property
tracePropertyBlocksRequestedAndRecievedPerPeer :: AnchoredFragment Block
-> AnchoredFragment Block -> [Example1TraceEvent] -> Property
tracePropertyBlocksRequestedAndRecievedPerPeer AnchoredFragment Block
fork1 AnchoredFragment Block
fork2 [Example1TraceEvent]
es =
      Map Int [Point BlockHeader]
requestedFetchPoints Map Int [Point BlockHeader]
-> Map Int [Point BlockHeader] -> Property
forall a. (Eq a, Show a) => a -> a -> Property
=== Map Int [Point BlockHeader]
requiredFetchPoints
 Property -> Property -> Property
forall prop1 prop2.
(Testable prop1, Testable prop2) =>
prop1 -> prop2 -> Property
.&&. Map Int [Point BlockHeader]
receivedFetchPoints  Map Int [Point BlockHeader]
-> Map Int [Point BlockHeader] -> Property
forall a. (Eq a, Show a) => a -> a -> Property
=== Map Int [Point BlockHeader]
requiredFetchPoints
  where
    requiredFetchPoints :: Map Int [Point BlockHeader]
requiredFetchPoints =
      ([Point BlockHeader] -> Bool)
-> Map Int [Point BlockHeader] -> Map Int [Point BlockHeader]
forall a k. (a -> Bool) -> Map k a -> Map k a
Map.filter (Bool -> Bool
not (Bool -> Bool)
-> ([Point BlockHeader] -> Bool) -> [Point BlockHeader] -> Bool
forall b c a. (b -> c) -> (a -> b) -> a -> c
. [Point BlockHeader] -> Bool
forall a. [a] -> Bool
forall (t :: * -> *) a. Foldable t => t a -> Bool
Prelude.null) (Map Int [Point BlockHeader] -> Map Int [Point BlockHeader])
-> Map Int [Point BlockHeader] -> Map Int [Point BlockHeader]
forall a b. (a -> b) -> a -> b
$
      [(Int, [Point BlockHeader])] -> Map Int [Point BlockHeader]
forall k a. Ord k => [(k, a)] -> Map k a
Map.fromList ([(Int, [Point BlockHeader])] -> Map Int [Point BlockHeader])
-> [(Int, [Point BlockHeader])] -> Map Int [Point BlockHeader]
forall a b. (a -> b) -> a -> b
$
        [ (Int
1, AnchoredFragment Block -> [Point BlockHeader]
chainPoints AnchoredFragment Block
fork1)
        , (Int
2, AnchoredFragment Block -> [Point BlockHeader]
chainPoints AnchoredFragment Block
fork2)
        ]

    requestedFetchPoints :: Map Int [Point BlockHeader]
    requestedFetchPoints :: Map Int [Point BlockHeader]
requestedFetchPoints =
      ([Point BlockHeader] -> [Point BlockHeader] -> [Point BlockHeader])
-> [(Int, [Point BlockHeader])] -> Map Int [Point BlockHeader]
forall k a. Ord k => (a -> a -> a) -> [(k, a)] -> Map k a
Map.fromListWith (([Point BlockHeader] -> [Point BlockHeader] -> [Point BlockHeader])
-> [Point BlockHeader]
-> [Point BlockHeader]
-> [Point BlockHeader]
forall a b c. (a -> b -> c) -> b -> a -> c
flip [Point BlockHeader] -> [Point BlockHeader] -> [Point BlockHeader]
forall a. [a] -> [a] -> [a]
(++))
        [ (Int
peer, (BlockHeader -> Point BlockHeader)
-> [BlockHeader] -> [Point BlockHeader]
forall a b. (a -> b) -> [a] -> [b]
map BlockHeader -> Point BlockHeader
forall block. HasHeader block => block -> Point block
blockPoint (AnchoredSeq (WithOrigin SlotNo) (Anchor BlockHeader) BlockHeader
-> [BlockHeader]
forall v a b. AnchoredSeq v a b -> [b]
AnchoredFragment.toOldestFirst AnchoredSeq (WithOrigin SlotNo) (Anchor BlockHeader) BlockHeader
fragment))
        | TraceFetchClientState
            (TraceLabelPeer Int
peer
              (AddedFetchRequest
                (FetchRequest [AnchoredSeq (WithOrigin SlotNo) (Anchor BlockHeader) BlockHeader]
fragments) PeerFetchInFlight BlockHeader
_ PeerFetchInFlightLimits
_ PeerFetchStatus BlockHeader
_)) <- [Example1TraceEvent]
es
        , AnchoredSeq (WithOrigin SlotNo) (Anchor BlockHeader) BlockHeader
fragment <- [AnchoredSeq (WithOrigin SlotNo) (Anchor BlockHeader) BlockHeader]
fragments
        ]

    receivedFetchPoints :: Map Int [Point BlockHeader]
    receivedFetchPoints :: Map Int [Point BlockHeader]
receivedFetchPoints =
      ([Point BlockHeader] -> [Point BlockHeader] -> [Point BlockHeader])
-> [(Int, [Point BlockHeader])] -> Map Int [Point BlockHeader]
forall k a. Ord k => (a -> a -> a) -> [(k, a)] -> Map k a
Map.fromListWith (([Point BlockHeader] -> [Point BlockHeader] -> [Point BlockHeader])
-> [Point BlockHeader]
-> [Point BlockHeader]
-> [Point BlockHeader]
forall a b c. (a -> b -> c) -> b -> a -> c
flip [Point BlockHeader] -> [Point BlockHeader] -> [Point BlockHeader]
forall a. [a] -> [a] -> [a]
(++))
        [ (Int
peer, [Point BlockHeader
pt])
        | TraceFetchClientState
            (TraceLabelPeer Int
peer (CompletedBlockFetch Point BlockHeader
pt PeerFetchInFlight BlockHeader
_ PeerFetchInFlightLimits
_ PeerFetchStatus BlockHeader
_ NominalDiffTime
_ SizeInBytes
_)) <- [Example1TraceEvent]
es
        ]


-- | Check the execution trace for a particular property: we observe all the
-- blocks in the 'FetchRequest's added by the decision logic and the blocks
-- received by the fetch clients; check that the set of all blocks requested
-- across the two peers is the set of blocks we expect, and similarly for the
-- set of all blocks received. The expected set of blocks is the union of the
-- blocks on the two candidate chains.
--
-- This property is weaker than 'tracePropertyBlocksRequestedAndRecievedPerPeer'
-- since it does not involve order or frequency, but it holds for the general
-- case of multiple chains with common prefixes.
--
tracePropertyBlocksRequestedAndRecievedAllPeers
  :: AnchoredFragment Block
  -> AnchoredFragment Block
  -> [Example1TraceEvent]
  -> Property
tracePropertyBlocksRequestedAndRecievedAllPeers :: AnchoredFragment Block
-> AnchoredFragment Block -> [Example1TraceEvent] -> Property
tracePropertyBlocksRequestedAndRecievedAllPeers AnchoredFragment Block
fork1 AnchoredFragment Block
fork2 [Example1TraceEvent]
es =
      Set (Point BlockHeader)
requestedFetchPoints Set (Point BlockHeader) -> Set (Point BlockHeader) -> Property
forall a. (Eq a, Show a) => a -> a -> Property
=== Set (Point BlockHeader)
requiredFetchPoints
 Property -> Property -> Property
forall prop1 prop2.
(Testable prop1, Testable prop2) =>
prop1 -> prop2 -> Property
.&&. Set (Point BlockHeader)
receivedFetchPoints  Set (Point BlockHeader) -> Set (Point BlockHeader) -> Property
forall a. (Eq a, Show a) => a -> a -> Property
=== Set (Point BlockHeader)
requiredFetchPoints
  where
    requiredFetchPoints :: Set (Point BlockHeader)
requiredFetchPoints =
      [Point BlockHeader] -> Set (Point BlockHeader)
forall a. Ord a => [a] -> Set a
Set.fromList (AnchoredFragment Block -> [Point BlockHeader]
chainPoints AnchoredFragment Block
fork1 [Point BlockHeader] -> [Point BlockHeader] -> [Point BlockHeader]
forall a. [a] -> [a] -> [a]
++ AnchoredFragment Block -> [Point BlockHeader]
chainPoints AnchoredFragment Block
fork2)

    requestedFetchPoints :: Set (Point BlockHeader)
    requestedFetchPoints :: Set (Point BlockHeader)
requestedFetchPoints =
      [Point BlockHeader] -> Set (Point BlockHeader)
forall a. Ord a => [a] -> Set a
Set.fromList
        [ BlockHeader -> Point BlockHeader
forall block. HasHeader block => block -> Point block
blockPoint BlockHeader
block
        | TraceFetchClientState
            (TraceLabelPeer Int
_
              (AddedFetchRequest
                (FetchRequest [AnchoredSeq (WithOrigin SlotNo) (Anchor BlockHeader) BlockHeader]
fragments) PeerFetchInFlight BlockHeader
_ PeerFetchInFlightLimits
_ PeerFetchStatus BlockHeader
_)) <- [Example1TraceEvent]
es
        , AnchoredSeq (WithOrigin SlotNo) (Anchor BlockHeader) BlockHeader
fragment <- [AnchoredSeq (WithOrigin SlotNo) (Anchor BlockHeader) BlockHeader]
fragments
        , BlockHeader
block    <- AnchoredSeq (WithOrigin SlotNo) (Anchor BlockHeader) BlockHeader
-> [BlockHeader]
forall v a b. AnchoredSeq v a b -> [b]
AnchoredFragment.toOldestFirst AnchoredSeq (WithOrigin SlotNo) (Anchor BlockHeader) BlockHeader
fragment
        ]

    receivedFetchPoints :: Set (Point BlockHeader)
    receivedFetchPoints :: Set (Point BlockHeader)
receivedFetchPoints =
      [Point BlockHeader] -> Set (Point BlockHeader)
forall a. Ord a => [a] -> Set a
Set.fromList
        [ Point BlockHeader
pt
        | TraceFetchClientState
            (TraceLabelPeer Int
_ (CompletedBlockFetch Point BlockHeader
pt PeerFetchInFlight BlockHeader
_ PeerFetchInFlightLimits
_ PeerFetchStatus BlockHeader
_ NominalDiffTime
_ SizeInBytes
_)) <- [Example1TraceEvent]
es
        ]


-- | Check the execution trace for a particular property: we observe all the
-- blocks in the 'FetchRequest's added by the decision logic; check that the
-- set blocks requested for one peer and for the other do not intersect.
--
-- This is a non-trivial property because in the general case, the chain
-- suffixes do intersect by sharing a common prefix. This property therefore
-- demonstrates that in the 'FetchModeBulkSync' the decision logic is properly
-- allocating fetch requests to different peers to avoid asking for duplicate
-- blocks from different peers.
--
tracePropertyNoDuplicateBlocksBetweenPeers
  :: AnchoredFragment Block
  -> AnchoredFragment Block
  -> [Example1TraceEvent]
  -> Property
tracePropertyNoDuplicateBlocksBetweenPeers :: AnchoredFragment Block
-> AnchoredFragment Block -> [Example1TraceEvent] -> Property
tracePropertyNoDuplicateBlocksBetweenPeers AnchoredFragment Block
fork1 AnchoredFragment Block
fork2 [Example1TraceEvent]
es =
    Bool -> [Char] -> Bool -> Property
forall prop. Testable prop => Bool -> [Char] -> prop -> Property
classify ([Point BlockHeader] -> Bool
hasDupes [Point BlockHeader]
requiredFetchPoints)
             [Char]
"Overlapping chains between peers" (Bool -> Property) -> Bool -> Property
forall a b. (a -> b) -> a -> b
$

    Set (Point BlockHeader) -> Bool
forall a. Set a -> Bool
Set.null (Set (Point BlockHeader) -> Bool)
-> Set (Point BlockHeader) -> Bool
forall a b. (a -> b) -> a -> b
$

    Set (Point BlockHeader)
-> Int
-> Map Int (Set (Point BlockHeader))
-> Set (Point BlockHeader)
forall k a. Ord k => a -> k -> Map k a -> a
Map.findWithDefault Set (Point BlockHeader)
forall a. Set a
Set.empty Int
1 Map Int (Set (Point BlockHeader))
requestedFetchPoints
      Set (Point BlockHeader)
-> Set (Point BlockHeader) -> Set (Point BlockHeader)
forall a. Ord a => Set a -> Set a -> Set a
`Set.intersection`
    Set (Point BlockHeader)
-> Int
-> Map Int (Set (Point BlockHeader))
-> Set (Point BlockHeader)
forall k a. Ord k => a -> k -> Map k a -> a
Map.findWithDefault Set (Point BlockHeader)
forall a. Set a
Set.empty Int
2 Map Int (Set (Point BlockHeader))
requestedFetchPoints

  where
    hasDupes :: [Point BlockHeader] -> Bool
hasDupes = Bool -> Bool
not (Bool -> Bool)
-> ([Point BlockHeader] -> Bool) -> [Point BlockHeader] -> Bool
forall b c a. (b -> c) -> (a -> b) -> a -> c
. ([Point BlockHeader] -> Bool) -> [[Point BlockHeader]] -> Bool
forall (t :: * -> *) a. Foldable t => (a -> Bool) -> t a -> Bool
any ((Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
>Int
1) (Int -> Bool)
-> ([Point BlockHeader] -> Int) -> [Point BlockHeader] -> Bool
forall b c a. (b -> c) -> (a -> b) -> a -> c
. [Point BlockHeader] -> Int
forall a. [a] -> Int
forall (t :: * -> *) a. Foldable t => t a -> Int
length)  ([[Point BlockHeader]] -> Bool)
-> ([Point BlockHeader] -> [[Point BlockHeader]])
-> [Point BlockHeader]
-> Bool
forall b c a. (b -> c) -> (a -> b) -> a -> c
. [Point BlockHeader] -> [[Point BlockHeader]]
forall a. Eq a => [a] -> [[a]]
List.group ([Point BlockHeader] -> [[Point BlockHeader]])
-> ([Point BlockHeader] -> [Point BlockHeader])
-> [Point BlockHeader]
-> [[Point BlockHeader]]
forall b c a. (b -> c) -> (a -> b) -> a -> c
. [Point BlockHeader] -> [Point BlockHeader]
forall a. Ord a => [a] -> [a]
List.sort

    requiredFetchPoints :: [Point BlockHeader]
requiredFetchPoints =
      [Point BlockHeader] -> [Point BlockHeader]
forall a. Eq a => [a] -> [a]
List.nub (AnchoredFragment Block -> [Point BlockHeader]
chainPoints AnchoredFragment Block
fork1 [Point BlockHeader] -> [Point BlockHeader] -> [Point BlockHeader]
forall a. [a] -> [a] -> [a]
++ AnchoredFragment Block -> [Point BlockHeader]
chainPoints AnchoredFragment Block
fork2)

    requestedFetchPoints :: Map Int (Set (Point BlockHeader))
    requestedFetchPoints :: Map Int (Set (Point BlockHeader))
requestedFetchPoints =
      (Set (Point BlockHeader)
 -> Set (Point BlockHeader) -> Set (Point BlockHeader))
-> [(Int, Set (Point BlockHeader))]
-> Map Int (Set (Point BlockHeader))
forall k a. Ord k => (a -> a -> a) -> [(k, a)] -> Map k a
Map.fromListWith Set (Point BlockHeader)
-> Set (Point BlockHeader) -> Set (Point BlockHeader)
forall a. Ord a => Set a -> Set a -> Set a
Set.union
        [ (Int
peer, AnchoredSeq (WithOrigin SlotNo) (Anchor BlockHeader) BlockHeader
-> Set (Point BlockHeader)
forall {v} {a}.
AnchoredSeq v a BlockHeader -> Set (Point BlockHeader)
points AnchoredSeq (WithOrigin SlotNo) (Anchor BlockHeader) BlockHeader
fragment)
        | TraceFetchClientState
            (TraceLabelPeer Int
peer
              (AddedFetchRequest
                (FetchRequest [AnchoredSeq (WithOrigin SlotNo) (Anchor BlockHeader) BlockHeader]
fragments) PeerFetchInFlight BlockHeader
_ PeerFetchInFlightLimits
_ PeerFetchStatus BlockHeader
_)) <- [Example1TraceEvent]
es
        , AnchoredSeq (WithOrigin SlotNo) (Anchor BlockHeader) BlockHeader
fragment <- [AnchoredSeq (WithOrigin SlotNo) (Anchor BlockHeader) BlockHeader]
fragments
        , let points :: AnchoredSeq v a BlockHeader -> Set (Point BlockHeader)
points = [Point BlockHeader] -> Set (Point BlockHeader)
forall a. Ord a => [a] -> Set a
Set.fromList ([Point BlockHeader] -> Set (Point BlockHeader))
-> (AnchoredSeq v a BlockHeader -> [Point BlockHeader])
-> AnchoredSeq v a BlockHeader
-> Set (Point BlockHeader)
forall b c a. (b -> c) -> (a -> b) -> a -> c
. (BlockHeader -> Point BlockHeader)
-> [BlockHeader] -> [Point BlockHeader]
forall a b. (a -> b) -> [a] -> [b]
map BlockHeader -> Point BlockHeader
forall block. HasHeader block => block -> Point block
blockPoint
                     ([BlockHeader] -> [Point BlockHeader])
-> (AnchoredSeq v a BlockHeader -> [BlockHeader])
-> AnchoredSeq v a BlockHeader
-> [Point BlockHeader]
forall b c a. (b -> c) -> (a -> b) -> a -> c
. AnchoredSeq v a BlockHeader -> [BlockHeader]
forall v a b. AnchoredSeq v a b -> [b]
AnchoredFragment.toOldestFirst
        ]


-- | This is just a sanity check on the in-flight statistics maintained between
-- the decision logic thread and the block fetch client threads.
--
tracePropertyClientStateSanity :: [Example1TraceEvent] -> Bool
tracePropertyClientStateSanity :: [Example1TraceEvent] -> Bool
tracePropertyClientStateSanity [Example1TraceEvent]
es =
    [Bool] -> Bool
forall (t :: * -> *). Foldable t => t Bool -> Bool
and [ PeerFetchInFlight BlockHeader
-> PeerFetchStatus BlockHeader -> Bool
forall {header} {header}.
PeerFetchInFlight header -> PeerFetchStatus header -> Bool
saneStateValues PeerFetchInFlight BlockHeader
inflight PeerFetchStatus BlockHeader
status
        | TraceFetchClientState
            (TraceLabelPeer Int
_
              (AddedFetchRequest FetchRequest BlockHeader
_ PeerFetchInFlight BlockHeader
inflight PeerFetchInFlightLimits
_ PeerFetchStatus BlockHeader
status)) <- [Example1TraceEvent]
es
        ]
  where
    saneStateValues :: PeerFetchInFlight header -> PeerFetchStatus header -> Bool
saneStateValues PeerFetchInFlight {Word
Set (Point header)
SizeInBytes
MaxSlotNo
peerFetchReqsInFlight :: Word
peerFetchBytesInFlight :: SizeInBytes
peerFetchBlocksInFlight :: Set (Point header)
peerFetchMaxSlotNo :: MaxSlotNo
peerFetchBlocksInFlight :: forall header. PeerFetchInFlight header -> Set (Point header)
peerFetchBytesInFlight :: forall header. PeerFetchInFlight header -> SizeInBytes
peerFetchMaxSlotNo :: forall header. PeerFetchInFlight header -> MaxSlotNo
peerFetchReqsInFlight :: forall header. PeerFetchInFlight header -> Word
..} PeerFetchStatus header
status =
        -- Here we know the fixed dummy block size so we know exactly what
        -- the bytes in flight should be.
           Set (Point header) -> Int
forall a. Set a -> Int
Set.size Set (Point header)
peerFetchBlocksInFlight Int -> Int -> Int
forall a. Num a => a -> a -> a
* Int
2000
        Int -> Int -> Bool
forall a. Eq a => a -> a -> Bool
== SizeInBytes -> Int
forall a b. (Integral a, Num b) => a -> b
fromIntegral SizeInBytes
peerFetchBytesInFlight

     Bool -> Bool -> Bool
&& case PeerFetchStatus header
status of
          PeerFetchStatusReady{} -> Bool
True
          PeerFetchStatus header
PeerFetchStatusBusy    -> Bool
True
          PeerFetchStatus header
_                      -> Bool
False -- not used in this test

     Bool -> Bool -> Bool
&& if Word
peerFetchReqsInFlight Word -> Word -> Bool
forall a. Eq a => a -> a -> Bool
== Word
0
           then Set (Point header) -> Bool
forall a. Set a -> Bool
Set.null Set (Point header)
peerFetchBlocksInFlight
           else Bool
True


-- TODO: the idea of this property was that we check that we're not making too
-- many decisions and waking up unnecessarily. But it's not trivial since
-- towards the end of a download we can have multiple occurences of decisions
-- where it's just the trailing data that's in-flight. Need to think about
-- whether there's any concise and robust way of expressing this.
--
-- tracePropertyDecisions _fork1 _fork2 _es = True

data FetchRequestTrace
    = AddedFetchRequestTrace
        (FetchRequest BlockHeader)
        (PeerFetchInFlight BlockHeader)
    | AcknowledgedFetchRequestTrace
    | CompletedFetchBatchTrace
    | RejectedFetchBatchTrace
  deriving Int -> FetchRequestTrace -> [Char] -> [Char]
[FetchRequestTrace] -> [Char] -> [Char]
FetchRequestTrace -> [Char]
(Int -> FetchRequestTrace -> [Char] -> [Char])
-> (FetchRequestTrace -> [Char])
-> ([FetchRequestTrace] -> [Char] -> [Char])
-> Show FetchRequestTrace
forall a.
(Int -> a -> [Char] -> [Char])
-> (a -> [Char]) -> ([a] -> [Char] -> [Char]) -> Show a
$cshowsPrec :: Int -> FetchRequestTrace -> [Char] -> [Char]
showsPrec :: Int -> FetchRequestTrace -> [Char] -> [Char]
$cshow :: FetchRequestTrace -> [Char]
show :: FetchRequestTrace -> [Char]
$cshowList :: [FetchRequestTrace] -> [Char] -> [Char]
showList :: [FetchRequestTrace] -> [Char] -> [Char]
Show

fetchRequestTrace :: [Example1TraceEvent] -> [TraceLabelPeer Int FetchRequestTrace]
fetchRequestTrace :: [Example1TraceEvent] -> [TraceLabelPeer Int FetchRequestTrace]
fetchRequestTrace = (Example1TraceEvent
 -> Maybe (TraceLabelPeer Int FetchRequestTrace))
-> [Example1TraceEvent] -> [TraceLabelPeer Int FetchRequestTrace]
forall a b. (a -> Maybe b) -> [a] -> [b]
mapMaybe Example1TraceEvent -> Maybe (TraceLabelPeer Int FetchRequestTrace)
f
  where
    f :: Example1TraceEvent -> Maybe (TraceLabelPeer Int FetchRequestTrace)
f (TraceFetchClientState (TraceLabelPeer Int
peerid (AddedFetchRequest FetchRequest BlockHeader
request PeerFetchInFlight BlockHeader
inflight PeerFetchInFlightLimits
_ PeerFetchStatus BlockHeader
_))) =
      TraceLabelPeer Int FetchRequestTrace
-> Maybe (TraceLabelPeer Int FetchRequestTrace)
forall a. a -> Maybe a
Just (Int -> FetchRequestTrace -> TraceLabelPeer Int FetchRequestTrace
forall peerid a. peerid -> a -> TraceLabelPeer peerid a
TraceLabelPeer Int
peerid (FetchRequest BlockHeader
-> PeerFetchInFlight BlockHeader -> FetchRequestTrace
AddedFetchRequestTrace FetchRequest BlockHeader
request PeerFetchInFlight BlockHeader
inflight))
    f (TraceFetchClientState (TraceLabelPeer Int
peerid (AcknowledgedFetchRequest{}))) =
      TraceLabelPeer Int FetchRequestTrace
-> Maybe (TraceLabelPeer Int FetchRequestTrace)
forall a. a -> Maybe a
Just (Int -> FetchRequestTrace -> TraceLabelPeer Int FetchRequestTrace
forall peerid a. peerid -> a -> TraceLabelPeer peerid a
TraceLabelPeer Int
peerid FetchRequestTrace
AcknowledgedFetchRequestTrace)
    f (TraceFetchClientState (TraceLabelPeer Int
peerid CompletedFetchBatch{}))
      = TraceLabelPeer Int FetchRequestTrace
-> Maybe (TraceLabelPeer Int FetchRequestTrace)
forall a. a -> Maybe a
Just (Int -> FetchRequestTrace -> TraceLabelPeer Int FetchRequestTrace
forall peerid a. peerid -> a -> TraceLabelPeer peerid a
TraceLabelPeer Int
peerid FetchRequestTrace
CompletedFetchBatchTrace)
    f (TraceFetchClientState (TraceLabelPeer Int
peerid RejectedFetchBatch{}))
      = TraceLabelPeer Int FetchRequestTrace
-> Maybe (TraceLabelPeer Int FetchRequestTrace)
forall a. a -> Maybe a
Just (Int -> FetchRequestTrace -> TraceLabelPeer Int FetchRequestTrace
forall peerid a. peerid -> a -> TraceLabelPeer peerid a
TraceLabelPeer Int
peerid FetchRequestTrace
RejectedFetchBatchTrace)
    f Example1TraceEvent
_ = Maybe (TraceLabelPeer Int FetchRequestTrace)
forall a. Maybe a
Nothing


-- | This property verifies that the number of in-flight requests is computed
-- according to the following algorithm:
--
-- * when adding requests to 'fetchClientRequestVar' using semigroup instance of
--   'FetchRequest' to calculate the number of requests to add to the number of
--   requests in flight
-- * when finishing receiving a batch, subtract one from the number of requests
--   in flight.
--
-- This tests reconstructs the value of 'fetchClientRequestVar' and
-- 'peerFetchReqsInFlight' from the trace and compares the expected value with
-- the actual value logged in the trace.
--
-- This property also assures that when the client terminates, there are no
-- outstanding in-flight requests.
--
-- Note: the implementation calls in-flight requests the requests that are
-- ordered to be sent (and they may not be sent immediately).  This test tracks
-- requests added to 'fetchClientRequestVar' and the number or requests that
-- were sent (acknowledged) by the client.  The sum of these two values gives
-- in-flight requests.
--
tracePropertyInFlight :: [Example1TraceEvent] -> Property
tracePropertyInFlight :: [Example1TraceEvent] -> Property
tracePropertyInFlight =
      ([FetchRequestTrace] -> Property -> Property)
-> Property -> Map Int [FetchRequestTrace] -> Property
forall a b. (a -> b -> b) -> b -> Map Int a -> b
forall (t :: * -> *) a b.
Foldable t =>
(a -> b -> b) -> b -> t a -> b
foldr (\[FetchRequestTrace]
tr Property
c -> Maybe (FetchRequest BlockHeader)
-> Int -> [FetchRequestTrace] -> Property
checkTrace Maybe (FetchRequest BlockHeader)
forall a. Maybe a
Nothing Int
0 [FetchRequestTrace]
tr Property -> Property -> Property
forall prop1 prop2.
(Testable prop1, Testable prop2) =>
prop1 -> prop2 -> Property
.&&. Property
c) (Bool -> Property
forall prop. Testable prop => prop -> Property
property Bool
True)
    (Map Int [FetchRequestTrace] -> Property)
-> ([Example1TraceEvent] -> Map Int [FetchRequestTrace])
-> [Example1TraceEvent]
-> Property
forall b c a. (b -> c) -> (a -> b) -> a -> c
. ([FetchRequestTrace] -> [FetchRequestTrace] -> [FetchRequestTrace])
-> [(Int, [FetchRequestTrace])] -> Map Int [FetchRequestTrace]
forall k a. Ord k => (a -> a -> a) -> [(k, a)] -> Map k a
Map.fromListWith (([FetchRequestTrace] -> [FetchRequestTrace] -> [FetchRequestTrace])
-> [FetchRequestTrace]
-> [FetchRequestTrace]
-> [FetchRequestTrace]
forall a b c. (a -> b -> c) -> b -> a -> c
flip [FetchRequestTrace] -> [FetchRequestTrace] -> [FetchRequestTrace]
forall a. [a] -> [a] -> [a]
(++))
    ([(Int, [FetchRequestTrace])] -> Map Int [FetchRequestTrace])
-> ([Example1TraceEvent] -> [(Int, [FetchRequestTrace])])
-> [Example1TraceEvent]
-> Map Int [FetchRequestTrace]
forall b c a. (b -> c) -> (a -> b) -> a -> c
. (TraceLabelPeer Int FetchRequestTrace
 -> (Int, [FetchRequestTrace]))
-> [TraceLabelPeer Int FetchRequestTrace]
-> [(Int, [FetchRequestTrace])]
forall a b. (a -> b) -> [a] -> [b]
map (\(TraceLabelPeer Int
peerid FetchRequestTrace
a) -> (Int
peerid, [FetchRequestTrace
a]))
    ([TraceLabelPeer Int FetchRequestTrace]
 -> [(Int, [FetchRequestTrace])])
-> ([Example1TraceEvent] -> [TraceLabelPeer Int FetchRequestTrace])
-> [Example1TraceEvent]
-> [(Int, [FetchRequestTrace])]
forall b c a. (b -> c) -> (a -> b) -> a -> c
. [Example1TraceEvent] -> [TraceLabelPeer Int FetchRequestTrace]
fetchRequestTrace
  where
    checkTrace :: Maybe (FetchRequest BlockHeader)
               --  not yet acknowledged 'FetchRequest', but ones that already
               --  added to 'fetchClientRequestVar';  This value simulates the
               --  content of 'fetchClientRequestVar'
               -> Int
               -- number of requests that were already sent (acknowledged);
               -> [FetchRequestTrace]
               -> Property

    -- 'AddedFetchRequest' when there 'fetchClientRequestVar' is empty
    checkTrace :: Maybe (FetchRequest BlockHeader)
-> Int -> [FetchRequestTrace] -> Property
checkTrace Maybe (FetchRequest BlockHeader)
Nothing Int
reqsInFlight ((AddedFetchRequestTrace FetchRequest BlockHeader
r PeerFetchInFlight {Word
peerFetchReqsInFlight :: forall header. PeerFetchInFlight header -> Word
peerFetchReqsInFlight :: Word
peerFetchReqsInFlight}) : [FetchRequestTrace]
tr)
      | Int
reqsInFlight Int -> Int -> Int
forall a. Num a => a -> a -> a
+ [AnchoredSeq (WithOrigin SlotNo) (Anchor BlockHeader) BlockHeader]
-> Int
forall a. [a] -> Int
forall (t :: * -> *) a. Foldable t => t a -> Int
length (FetchRequest BlockHeader
-> [AnchoredSeq
      (WithOrigin SlotNo) (Anchor BlockHeader) BlockHeader]
forall header. FetchRequest header -> [AnchoredFragment header]
fetchRequestFragments FetchRequest BlockHeader
r) Int -> Int -> Bool
forall a. Eq a => a -> a -> Bool
== Word -> Int
forall a b. (Integral a, Num b) => a -> b
fromIntegral Word
peerFetchReqsInFlight
      = Maybe (FetchRequest BlockHeader)
-> Int -> [FetchRequestTrace] -> Property
checkTrace (FetchRequest BlockHeader -> Maybe (FetchRequest BlockHeader)
forall a. a -> Maybe a
Just FetchRequest BlockHeader
r) Int
reqsInFlight [FetchRequestTrace]
tr
      | Bool
otherwise
      = [Char] -> Bool -> Property
forall prop. Testable prop => [Char] -> prop -> Property
counterexample ([Char]
"tracePropertyInFlight: "
                       [Char] -> [Char] -> [Char]
forall a. [a] -> [a] -> [a]
++ Int -> [Char]
forall a. Show a => a -> [Char]
show Int
reqsInFlight
                       [Char] -> [Char] -> [Char]
forall a. [a] -> [a] -> [a]
++ [Char]
" + "
                       [Char] -> [Char] -> [Char]
forall a. [a] -> [a] -> [a]
++ Int -> [Char]
forall a. Show a => a -> [Char]
show ([AnchoredSeq (WithOrigin SlotNo) (Anchor BlockHeader) BlockHeader]
-> Int
forall a. [a] -> Int
forall (t :: * -> *) a. Foldable t => t a -> Int
length (FetchRequest BlockHeader
-> [AnchoredSeq
      (WithOrigin SlotNo) (Anchor BlockHeader) BlockHeader]
forall header. FetchRequest header -> [AnchoredFragment header]
fetchRequestFragments FetchRequest BlockHeader
r))
                       [Char] -> [Char] -> [Char]
forall a. [a] -> [a] -> [a]
++ [Char]
" /= "
                       [Char] -> [Char] -> [Char]
forall a. [a] -> [a] -> [a]
++ Word -> [Char]
forall a. Show a => a -> [Char]
show Word
peerFetchReqsInFlight
                      ) Bool
False

    -- 'AddedFetchRequest' when there are 'fetchClientRequestVar' is non-empty
    -- in this case we use 'FetchRequest' Semigroup instance to combine new and
    -- old requests.
    checkTrace (Just FetchRequest BlockHeader
r0) Int
reqsInFlight ((AddedFetchRequestTrace FetchRequest BlockHeader
r1 PeerFetchInFlight {Word
peerFetchReqsInFlight :: forall header. PeerFetchInFlight header -> Word
peerFetchReqsInFlight :: Word
peerFetchReqsInFlight}) : [FetchRequestTrace]
tr)
      | Int
reqsInFlight Int -> Int -> Int
forall a. Num a => a -> a -> a
+ [AnchoredSeq (WithOrigin SlotNo) (Anchor BlockHeader) BlockHeader]
-> Int
forall a. [a] -> Int
forall (t :: * -> *) a. Foldable t => t a -> Int
length (FetchRequest BlockHeader
-> [AnchoredSeq
      (WithOrigin SlotNo) (Anchor BlockHeader) BlockHeader]
forall header. FetchRequest header -> [AnchoredFragment header]
fetchRequestFragments (FetchRequest BlockHeader
r0 FetchRequest BlockHeader
-> FetchRequest BlockHeader -> FetchRequest BlockHeader
forall a. Semigroup a => a -> a -> a
<> FetchRequest BlockHeader
r1)) Int -> Int -> Bool
forall a. Eq a => a -> a -> Bool
== Word -> Int
forall a b. (Integral a, Num b) => a -> b
fromIntegral Word
peerFetchReqsInFlight
      = Maybe (FetchRequest BlockHeader)
-> Int -> [FetchRequestTrace] -> Property
checkTrace (FetchRequest BlockHeader -> Maybe (FetchRequest BlockHeader)
forall a. a -> Maybe a
Just (FetchRequest BlockHeader
r0 FetchRequest BlockHeader
-> FetchRequest BlockHeader -> FetchRequest BlockHeader
forall a. Semigroup a => a -> a -> a
<> FetchRequest BlockHeader
r1)) Int
reqsInFlight [FetchRequestTrace]
tr
      | Bool
otherwise
      = [Char] -> Bool -> Property
forall prop. Testable prop => [Char] -> prop -> Property
counterexample ([Char]
"tracePropertyInFlight: "
                       [Char] -> [Char] -> [Char]
forall a. [a] -> [a] -> [a]
++ Int -> [Char]
forall a. Show a => a -> [Char]
show Int
reqsInFlight
                       [Char] -> [Char] -> [Char]
forall a. [a] -> [a] -> [a]
++ [Char]
" + "
                       [Char] -> [Char] -> [Char]
forall a. [a] -> [a] -> [a]
++ Int -> [Char]
forall a. Show a => a -> [Char]
show ([AnchoredSeq (WithOrigin SlotNo) (Anchor BlockHeader) BlockHeader]
-> Int
forall a. [a] -> Int
forall (t :: * -> *) a. Foldable t => t a -> Int
length (FetchRequest BlockHeader
-> [AnchoredSeq
      (WithOrigin SlotNo) (Anchor BlockHeader) BlockHeader]
forall header. FetchRequest header -> [AnchoredFragment header]
fetchRequestFragments (FetchRequest BlockHeader
r0 FetchRequest BlockHeader
-> FetchRequest BlockHeader -> FetchRequest BlockHeader
forall a. Semigroup a => a -> a -> a
<> FetchRequest BlockHeader
r1)))
                       [Char] -> [Char] -> [Char]
forall a. [a] -> [a] -> [a]
++ [Char]
" /= "
                       [Char] -> [Char] -> [Char]
forall a. [a] -> [a] -> [a]
++ Word -> [Char]
forall a. Show a => a -> [Char]
show Word
peerFetchReqsInFlight
                       ) Bool
False

    -- acknowledged fetch requests: update 'reqsInFlight' and continue
    -- traversing the trace
    checkTrace (Just FetchRequest BlockHeader
r) Int
reqsInFlight (FetchRequestTrace
AcknowledgedFetchRequestTrace : [FetchRequestTrace]
tr)
      = Maybe (FetchRequest BlockHeader)
-> Int -> [FetchRequestTrace] -> Property
checkTrace Maybe (FetchRequest BlockHeader)
forall a. Maybe a
Nothing (Int
reqsInFlight Int -> Int -> Int
forall a. Num a => a -> a -> a
+ [AnchoredSeq (WithOrigin SlotNo) (Anchor BlockHeader) BlockHeader]
-> Int
forall a. [a] -> Int
forall (t :: * -> *) a. Foldable t => t a -> Int
length (FetchRequest BlockHeader
-> [AnchoredSeq
      (WithOrigin SlotNo) (Anchor BlockHeader) BlockHeader]
forall header. FetchRequest header -> [AnchoredFragment header]
fetchRequestFragments FetchRequest BlockHeader
r)) [FetchRequestTrace]
tr
    checkTrace Maybe (FetchRequest BlockHeader)
Nothing Int
reqsInFlight (FetchRequestTrace
AcknowledgedFetchRequestTrace : [FetchRequestTrace]
tr)
      = Maybe (FetchRequest BlockHeader)
-> Int -> [FetchRequestTrace] -> Property
checkTrace Maybe (FetchRequest BlockHeader)
forall a. Maybe a
Nothing Int
reqsInFlight [FetchRequestTrace]
tr

    -- batch completed, we subtract `1` from requests in flight
    checkTrace Maybe (FetchRequest BlockHeader)
mr Int
reqsInFlight (FetchRequestTrace
CompletedFetchBatchTrace : [FetchRequestTrace]
tr)
      = Maybe (FetchRequest BlockHeader)
-> Int -> [FetchRequestTrace] -> Property
checkTrace Maybe (FetchRequest BlockHeader)
mr (Int
reqsInFlight Int -> Int -> Int
forall a. Num a => a -> a -> a
- Int
1) [FetchRequestTrace]
tr
    checkTrace Maybe (FetchRequest BlockHeader)
mr Int
reqsInFlight (FetchRequestTrace
RejectedFetchBatchTrace : [FetchRequestTrace]
tr)
      = Maybe (FetchRequest BlockHeader)
-> Int -> [FetchRequestTrace] -> Property
checkTrace Maybe (FetchRequest BlockHeader)
mr (Int
reqsInFlight Int -> Int -> Int
forall a. Num a => a -> a -> a
- Int
1) [FetchRequestTrace]
tr

    -- check that by the end of the trace there are no requests in flight
    checkTrace (Just FetchRequest BlockHeader
_) Int
_ []
      = [Char] -> Bool -> Property
forall prop. Testable prop => [Char] -> prop -> Property
counterexample
          [Char]
"tracePropertyInFlight: fetch requests in flight"
          Bool
False
    checkTrace Maybe (FetchRequest BlockHeader)
Nothing Int
reqsInFlight []
      | Int
reqsInFlight Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
> Int
0
      = [Char] -> Bool -> Property
forall prop. Testable prop => [Char] -> prop -> Property
counterexample
          ([Char]
"traceProeprtyInFlight: reqsInFlight = " [Char] -> [Char] -> [Char]
forall a. [a] -> [a] -> [a]
++ Int -> [Char]
forall a. Show a => a -> [Char]
show Int
reqsInFlight [Char] -> [Char] -> [Char]
forall a. [a] -> [a] -> [a]
++ [Char]
" ≠ 0")
          Bool
False
      | Bool
otherwise
      = Bool -> Property
forall prop. Testable prop => prop -> Property
property Bool
True


--
-- Unit tests
--

unit_bracketSyncWithFetchClient :: (String -> IO ()) -> Assertion
unit_bracketSyncWithFetchClient :: ([Char] -> IO ()) -> IO ()
unit_bracketSyncWithFetchClient [Char] -> IO ()
step = do

    [Char] -> IO ()
step [Char]
"Starting fetch before sync"

    (Either SomeException (), Either SomeException ()) -> IO ()
forall {a} {b} {b}. Show a => (Either a b, b) -> IO ()
checkResultA ((Either SomeException (), Either SomeException ()) -> IO ())
-> IO (Either SomeException (), Either SomeException ()) -> IO ()
forall (m :: * -> *) a b. Monad m => (a -> m b) -> m a -> m b
=<< case (forall s.
 IOSim s (Either SomeException (), Either SomeException ()))
-> Either
     Failure (Either SomeException (), Either SomeException ())
forall a. (forall s. IOSim s a) -> Either Failure a
runSimStrictShutdown (((forall c. IOSim s c -> IOSim s c) -> IOSim s ())
-> ((forall c. IOSim s c -> IOSim s c) -> IOSim s ())
-> ((forall c. IOSim s c -> IOSim s c) -> IOSim s ())
-> IOSim s (Either SomeException (), Either SomeException ())
forall (m :: * -> *) a b d.
(MonadAsync m, MonadDelay m, MonadFork m, MonadMask m,
 MonadThrow m, MonadThrow (STM m), MonadTimer m) =>
((forall c. m c -> m c) -> m a)
-> ((forall c. m c -> m c) -> m b)
-> ((forall c. m c -> m c) -> m d)
-> m (Either SomeException a, Either SomeException b)
testSkeleton
                            (\forall c. IOSim s c -> IOSim s c
action -> DiffTime -> IOSim s ()
forall (m :: * -> *). MonadDelay m => DiffTime -> m ()
threadDelay DiffTime
0.1 IOSim s () -> IOSim s () -> IOSim s ()
forall a b. IOSim s a -> IOSim s b -> IOSim s b
forall (m :: * -> *) a b. Monad m => m a -> m b -> m b
>> IOSim s () -> IOSim s ()
forall c. IOSim s c -> IOSim s c
action (DiffTime -> IOSim s ()
forall (m :: * -> *). MonadDelay m => DiffTime -> m ()
threadDelay DiffTime
0.2))
                            (\forall c. IOSim s c -> IOSim s c
action -> DiffTime -> IOSim s ()
forall (m :: * -> *). MonadDelay m => DiffTime -> m ()
threadDelay DiffTime
0.2 IOSim s () -> IOSim s () -> IOSim s ()
forall a b. IOSim s a -> IOSim s b -> IOSim s b
forall (m :: * -> *) a b. Monad m => m a -> m b -> m b
>> IOSim s () -> IOSim s ()
forall c. IOSim s c -> IOSim s c
action (DiffTime -> IOSim s ()
forall (m :: * -> *). MonadDelay m => DiffTime -> m ()
threadDelay DiffTime
0.2))
                            (\forall c. IOSim s c -> IOSim s c
action -> DiffTime -> IOSim s ()
forall (m :: * -> *). MonadDelay m => DiffTime -> m ()
threadDelay DiffTime
0.1 IOSim s () -> IOSim s () -> IOSim s ()
forall a b. IOSim s a -> IOSim s b -> IOSim s b
forall (m :: * -> *) a b. Monad m => m a -> m b -> m b
>> IOSim s () -> IOSim s ()
forall c. IOSim s c -> IOSim s c
action (DiffTime -> IOSim s ()
forall (m :: * -> *). MonadDelay m => DiffTime -> m ()
threadDelay DiffTime
1.1))) of
                          Left Failure
e  -> [Char] -> IO (Either SomeException (), Either SomeException ())
forall a. HasCallStack => [Char] -> a
error ([Char] -> IO (Either SomeException (), Either SomeException ()))
-> [Char] -> IO (Either SomeException (), Either SomeException ())
forall a b. (a -> b) -> a -> b
$ [Char]
"sim failed with " [Char] -> [Char] -> [Char]
forall a. Semigroup a => a -> a -> a
<> Failure -> [Char]
forall a. Show a => a -> [Char]
show Failure
e
                          Right (Either SomeException (), Either SomeException ())
r -> (Either SomeException (), Either SomeException ())
-> IO (Either SomeException (), Either SomeException ())
forall a. a -> IO a
forall (m :: * -> *) a. Monad m => a -> m a
return (Either SomeException (), Either SomeException ())
r

    [Char] -> IO ()
step [Char]
"Start and kill fetch before sync"
    (Either SomeException (), Either SomeException ()) -> IO ()
forall {a} {b} {b}. (Either a b, b) -> IO ()
checkResultB ((Either SomeException (), Either SomeException ()) -> IO ())
-> IO (Either SomeException (), Either SomeException ()) -> IO ()
forall (m :: * -> *) a b. Monad m => (a -> m b) -> m a -> m b
=<< case (forall s.
 IOSim s (Either SomeException (), Either SomeException ()))
-> Either
     Failure (Either SomeException (), Either SomeException ())
forall a. (forall s. IOSim s a) -> Either Failure a
runSimStrictShutdown (((forall c. IOSim s c -> IOSim s c) -> IOSim s ())
-> ((forall c. IOSim s c -> IOSim s c) -> IOSim s ())
-> ((forall c. IOSim s c -> IOSim s c) -> IOSim s ())
-> IOSim s (Either SomeException (), Either SomeException ())
forall (m :: * -> *) a b d.
(MonadAsync m, MonadDelay m, MonadFork m, MonadMask m,
 MonadThrow m, MonadThrow (STM m), MonadTimer m) =>
((forall c. m c -> m c) -> m a)
-> ((forall c. m c -> m c) -> m b)
-> ((forall c. m c -> m c) -> m d)
-> m (Either SomeException a, Either SomeException b)
testSkeleton
                            (\forall c. IOSim s c -> IOSim s c
action -> IOSim s () -> IOSim s ()
forall c. IOSim s c -> IOSim s c
action (DiffTime -> IOSim s ()
forall (m :: * -> *). MonadDelay m => DiffTime -> m ()
threadDelay DiffTime
3))
                            (\forall c. IOSim s c -> IOSim s c
_action -> DiffTime -> IOSim s ()
forall (m :: * -> *). MonadDelay m => DiffTime -> m ()
threadDelay DiffTime
3)
                            (\forall c. IOSim s c -> IOSim s c
action -> DiffTime -> IOSim s ()
forall (m :: * -> *). MonadDelay m => DiffTime -> m ()
threadDelay DiffTime
0.1 IOSim s () -> IOSim s () -> IOSim s ()
forall a b. IOSim s a -> IOSim s b -> IOSim s b
forall (m :: * -> *) a b. Monad m => m a -> m b -> m b
>> IOSim s () -> IOSim s ()
forall c. IOSim s c -> IOSim s c
action (DiffTime -> IOSim s ()
forall (m :: * -> *). MonadDelay m => DiffTime -> m ()
threadDelay DiffTime
1.1))) of
                          Left Failure
e  -> [Char] -> IO (Either SomeException (), Either SomeException ())
forall a. HasCallStack => [Char] -> a
error ([Char] -> IO (Either SomeException (), Either SomeException ()))
-> [Char] -> IO (Either SomeException (), Either SomeException ())
forall a b. (a -> b) -> a -> b
$ [Char]
"sim failed with " [Char] -> [Char] -> [Char]
forall a. Semigroup a => a -> a -> a
<> Failure -> [Char]
forall a. Show a => a -> [Char]
show Failure
e
                          Right (Either SomeException (), Either SomeException ())
r -> (Either SomeException (), Either SomeException ())
-> IO (Either SomeException (), Either SomeException ())
forall a. a -> IO a
forall (m :: * -> *) a. Monad m => a -> m a
return (Either SomeException (), Either SomeException ())
r

    [Char] -> IO ()
step [Char]
"Starting sync before fetch"
    (Either SomeException (), Either SomeException ()) -> IO ()
forall {a} {b} {b}. Show a => (Either a b, b) -> IO ()
checkResultA ((Either SomeException (), Either SomeException ()) -> IO ())
-> IO (Either SomeException (), Either SomeException ()) -> IO ()
forall (m :: * -> *) a b. Monad m => (a -> m b) -> m a -> m b
=<< case (forall s.
 IOSim s (Either SomeException (), Either SomeException ()))
-> Either
     Failure (Either SomeException (), Either SomeException ())
forall a. (forall s. IOSim s a) -> Either Failure a
runSimStrictShutdown (((forall c. IOSim s c -> IOSim s c) -> IOSim s ())
-> ((forall c. IOSim s c -> IOSim s c) -> IOSim s ())
-> ((forall c. IOSim s c -> IOSim s c) -> IOSim s ())
-> IOSim s (Either SomeException (), Either SomeException ())
forall (m :: * -> *) a b d.
(MonadAsync m, MonadDelay m, MonadFork m, MonadMask m,
 MonadThrow m, MonadThrow (STM m), MonadTimer m) =>
((forall c. m c -> m c) -> m a)
-> ((forall c. m c -> m c) -> m b)
-> ((forall c. m c -> m c) -> m d)
-> m (Either SomeException a, Either SomeException b)
testSkeleton
                            (\forall c. IOSim s c -> IOSim s c
action -> DiffTime -> IOSim s ()
forall (m :: * -> *). MonadDelay m => DiffTime -> m ()
threadDelay DiffTime
0.2 IOSim s () -> IOSim s () -> IOSim s ()
forall a b. IOSim s a -> IOSim s b -> IOSim s b
forall (m :: * -> *) a b. Monad m => m a -> m b -> m b
>> IOSim s () -> IOSim s ()
forall c. IOSim s c -> IOSim s c
action (DiffTime -> IOSim s ()
forall (m :: * -> *). MonadDelay m => DiffTime -> m ()
threadDelay DiffTime
0.2))
                            (\forall c. IOSim s c -> IOSim s c
action -> DiffTime -> IOSim s ()
forall (m :: * -> *). MonadDelay m => DiffTime -> m ()
threadDelay DiffTime
0.1 IOSim s () -> IOSim s () -> IOSim s ()
forall a b. IOSim s a -> IOSim s b -> IOSim s b
forall (m :: * -> *) a b. Monad m => m a -> m b -> m b
>> IOSim s () -> IOSim s ()
forall c. IOSim s c -> IOSim s c
action (DiffTime -> IOSim s ()
forall (m :: * -> *). MonadDelay m => DiffTime -> m ()
threadDelay DiffTime
0.2))
                            (\forall c. IOSim s c -> IOSim s c
action -> DiffTime -> IOSim s ()
forall (m :: * -> *). MonadDelay m => DiffTime -> m ()
threadDelay DiffTime
0.1 IOSim s () -> IOSim s () -> IOSim s ()
forall a b. IOSim s a -> IOSim s b -> IOSim s b
forall (m :: * -> *) a b. Monad m => m a -> m b -> m b
>> IOSim s () -> IOSim s ()
forall c. IOSim s c -> IOSim s c
action (DiffTime -> IOSim s ()
forall (m :: * -> *). MonadDelay m => DiffTime -> m ()
threadDelay DiffTime
0.5))) of
                          Left Failure
e  -> [Char] -> IO (Either SomeException (), Either SomeException ())
forall a. HasCallStack => [Char] -> a
error ([Char] -> IO (Either SomeException (), Either SomeException ()))
-> [Char] -> IO (Either SomeException (), Either SomeException ())
forall a b. (a -> b) -> a -> b
$ [Char]
"sim failed with " [Char] -> [Char] -> [Char]
forall a. Semigroup a => a -> a -> a
<> Failure -> [Char]
forall a. Show a => a -> [Char]
show Failure
e
                          Right (Either SomeException (), Either SomeException ())
r -> (Either SomeException (), Either SomeException ())
-> IO (Either SomeException (), Either SomeException ())
forall a. a -> IO a
forall (m :: * -> *) a. Monad m => a -> m a
return (Either SomeException (), Either SomeException ())
r

    [Char] -> IO ()
step [Char]
"Stopping fetch before sync"
    (Either SomeException (), Either SomeException ()) -> IO ()
forall {a} {a} {b} {b}.
(Show a, Show a) =>
(Either a b, Either a b) -> IO ()
checkResultD ((Either SomeException (), Either SomeException ()) -> IO ())
-> IO (Either SomeException (), Either SomeException ()) -> IO ()
forall (m :: * -> *) a b. Monad m => (a -> m b) -> m a -> m b
=<< case (forall s.
 IOSim s (Either SomeException (), Either SomeException ()))
-> Either
     Failure (Either SomeException (), Either SomeException ())
forall a. (forall s. IOSim s a) -> Either Failure a
runSimStrictShutdown (((forall c. IOSim s c -> IOSim s c) -> IOSim s ())
-> ((forall c. IOSim s c -> IOSim s c) -> IOSim s ())
-> ((forall c. IOSim s c -> IOSim s c) -> IOSim s ())
-> IOSim s (Either SomeException (), Either SomeException ())
forall (m :: * -> *) a b d.
(MonadAsync m, MonadDelay m, MonadFork m, MonadMask m,
 MonadThrow m, MonadThrow (STM m), MonadTimer m) =>
((forall c. m c -> m c) -> m a)
-> ((forall c. m c -> m c) -> m b)
-> ((forall c. m c -> m c) -> m d)
-> m (Either SomeException a, Either SomeException b)
testSkeleton
                            (\forall c. IOSim s c -> IOSim s c
action -> IOSim s () -> IOSim s ()
forall c. IOSim s c -> IOSim s c
action (DiffTime -> IOSim s ()
forall (m :: * -> *). MonadDelay m => DiffTime -> m ()
threadDelay DiffTime
0.1))
                            (\forall c. IOSim s c -> IOSim s c
action -> IOSim s () -> IOSim s ()
forall c. IOSim s c -> IOSim s c
action (DiffTime -> IOSim s ()
forall (m :: * -> *). MonadDelay m => DiffTime -> m ()
threadDelay DiffTime
10))
                            (\forall c. IOSim s c -> IOSim s c
action -> DiffTime -> IOSim s ()
forall (m :: * -> *). MonadDelay m => DiffTime -> m ()
threadDelay DiffTime
0.1 IOSim s () -> IOSim s () -> IOSim s ()
forall a b. IOSim s a -> IOSim s b -> IOSim s b
forall (m :: * -> *) a b. Monad m => m a -> m b -> m b
>> IOSim s () -> IOSim s ()
forall c. IOSim s c -> IOSim s c
action (DiffTime -> IOSim s ()
forall (m :: * -> *). MonadDelay m => DiffTime -> m ()
threadDelay DiffTime
300))) of
                          Left Failure
e  -> [Char] -> IO (Either SomeException (), Either SomeException ())
forall a. HasCallStack => [Char] -> a
error ([Char] -> IO (Either SomeException (), Either SomeException ()))
-> [Char] -> IO (Either SomeException (), Either SomeException ())
forall a b. (a -> b) -> a -> b
$ [Char]
"sim failed with " [Char] -> [Char] -> [Char]
forall a. Semigroup a => a -> a -> a
<> Failure -> [Char]
forall a. Show a => a -> [Char]
show Failure
e
                          Right (Either SomeException (), Either SomeException ())
r -> (Either SomeException (), Either SomeException ())
-> IO (Either SomeException (), Either SomeException ())
forall a. a -> IO a
forall (m :: * -> *) a. Monad m => a -> m a
return (Either SomeException (), Either SomeException ())
r

    [Char] -> IO ()
step [Char]
"Stopping fetch before sync, sync timeout"
    (Either SomeException (), Either SomeException ()) -> IO ()
forall {a} {b} {a} {b}. Show a => (Either a b, Either a b) -> IO ()
checkResultC ((Either SomeException (), Either SomeException ()) -> IO ())
-> IO (Either SomeException (), Either SomeException ()) -> IO ()
forall (m :: * -> *) a b. Monad m => (a -> m b) -> m a -> m b
=<< case (forall s.
 IOSim s (Either SomeException (), Either SomeException ()))
-> Either
     Failure (Either SomeException (), Either SomeException ())
forall a. (forall s. IOSim s a) -> Either Failure a
runSimStrictShutdown (((forall c. IOSim s c -> IOSim s c) -> IOSim s ())
-> ((forall c. IOSim s c -> IOSim s c) -> IOSim s ())
-> ((forall c. IOSim s c -> IOSim s c) -> IOSim s ())
-> IOSim s (Either SomeException (), Either SomeException ())
forall (m :: * -> *) a b d.
(MonadAsync m, MonadDelay m, MonadFork m, MonadMask m,
 MonadThrow m, MonadThrow (STM m), MonadTimer m) =>
((forall c. m c -> m c) -> m a)
-> ((forall c. m c -> m c) -> m b)
-> ((forall c. m c -> m c) -> m d)
-> m (Either SomeException a, Either SomeException b)
testSkeleton
                            (\forall c. IOSim s c -> IOSim s c
action -> IOSim s () -> IOSim s ()
forall c. IOSim s c -> IOSim s c
action (DiffTime -> IOSim s ()
forall (m :: * -> *). MonadDelay m => DiffTime -> m ()
threadDelay DiffTime
0.1))
                            (\forall c. IOSim s c -> IOSim s c
action -> IOSim s () -> IOSim s ()
forall c. IOSim s c -> IOSim s c
action (DiffTime -> IOSim s ()
forall (m :: * -> *). MonadDelay m => DiffTime -> m ()
threadDelay DiffTime
600))
                            (\forall c. IOSim s c -> IOSim s c
action -> DiffTime -> IOSim s ()
forall (m :: * -> *). MonadDelay m => DiffTime -> m ()
threadDelay DiffTime
0.1 IOSim s () -> IOSim s () -> IOSim s ()
forall a b. IOSim s a -> IOSim s b -> IOSim s b
forall (m :: * -> *) a b. Monad m => m a -> m b -> m b
>> IOSim s () -> IOSim s ()
forall c. IOSim s c -> IOSim s c
action (DiffTime -> IOSim s ()
forall (m :: * -> *). MonadDelay m => DiffTime -> m ()
threadDelay DiffTime
400))) of
                          Left Failure
e  -> [Char] -> IO (Either SomeException (), Either SomeException ())
forall a. HasCallStack => [Char] -> a
error ([Char] -> IO (Either SomeException (), Either SomeException ()))
-> [Char] -> IO (Either SomeException (), Either SomeException ())
forall a b. (a -> b) -> a -> b
$ [Char]
"sim failed with " [Char] -> [Char] -> [Char]
forall a. Semigroup a => a -> a -> a
<> Failure -> [Char]
forall a. Show a => a -> [Char]
show Failure
e
                          Right (Either SomeException (), Either SomeException ())
r -> (Either SomeException (), Either SomeException ())
-> IO (Either SomeException (), Either SomeException ())
forall a. a -> IO a
forall (m :: * -> *) a. Monad m => a -> m a
return (Either SomeException (), Either SomeException ())
r

    [Char] -> IO ()
step [Char]
"Stopping fetch before sync, keepalive exits"
    (Either SomeException (), Either SomeException ()) -> IO ()
forall {a} {b} {a} {b}. (Either a b, Either a b) -> IO ()
checkResultE ((Either SomeException (), Either SomeException ()) -> IO ())
-> IO (Either SomeException (), Either SomeException ()) -> IO ()
forall (m :: * -> *) a b. Monad m => (a -> m b) -> m a -> m b
=<< case (forall s.
 IOSim s (Either SomeException (), Either SomeException ()))
-> Either
     Failure (Either SomeException (), Either SomeException ())
forall a. (forall s. IOSim s a) -> Either Failure a
runSimStrictShutdown (((forall c. IOSim s c -> IOSim s c) -> IOSim s ())
-> ((forall c. IOSim s c -> IOSim s c) -> IOSim s ())
-> ((forall c. IOSim s c -> IOSim s c) -> IOSim s ())
-> IOSim s (Either SomeException (), Either SomeException ())
forall (m :: * -> *) a b d.
(MonadAsync m, MonadDelay m, MonadFork m, MonadMask m,
 MonadThrow m, MonadThrow (STM m), MonadTimer m) =>
((forall c. m c -> m c) -> m a)
-> ((forall c. m c -> m c) -> m b)
-> ((forall c. m c -> m c) -> m d)
-> m (Either SomeException a, Either SomeException b)
testSkeleton
                            (\forall c. IOSim s c -> IOSim s c
action -> IOSim s () -> IOSim s ()
forall c. IOSim s c -> IOSim s c
action (DiffTime -> IOSim s ()
forall (m :: * -> *). MonadDelay m => DiffTime -> m ()
threadDelay DiffTime
0.1))
                            (\forall c. IOSim s c -> IOSim s c
action -> IOSim s () -> IOSim s ()
forall c. IOSim s c -> IOSim s c
action (DiffTime -> IOSim s ()
forall (m :: * -> *). MonadDelay m => DiffTime -> m ()
threadDelay DiffTime
600))
                            (\forall c. IOSim s c -> IOSim s c
action -> DiffTime -> IOSim s ()
forall (m :: * -> *). MonadDelay m => DiffTime -> m ()
threadDelay DiffTime
0.1 IOSim s () -> IOSim s () -> IOSim s ()
forall a b. IOSim s a -> IOSim s b -> IOSim s b
forall (m :: * -> *) a b. Monad m => m a -> m b -> m b
>> IOSim s () -> IOSim s ()
forall c. IOSim s c -> IOSim s c
action (DiffTime -> IOSim s ()
forall (m :: * -> *). MonadDelay m => DiffTime -> m ()
threadDelay DiffTime
1))) of
                          Left Failure
e  -> [Char] -> IO (Either SomeException (), Either SomeException ())
forall a. HasCallStack => [Char] -> a
error ([Char] -> IO (Either SomeException (), Either SomeException ()))
-> [Char] -> IO (Either SomeException (), Either SomeException ())
forall a b. (a -> b) -> a -> b
$ [Char]
"sim failed with " [Char] -> [Char] -> [Char]
forall a. Semigroup a => a -> a -> a
<> Failure -> [Char]
forall a. Show a => a -> [Char]
show Failure
e
                          Right (Either SomeException (), Either SomeException ())
r -> (Either SomeException (), Either SomeException ())
-> IO (Either SomeException (), Either SomeException ())
forall a. a -> IO a
forall (m :: * -> *) a. Monad m => a -> m a
return (Either SomeException (), Either SomeException ())
r

    [Char] -> IO ()
step [Char]
"Stopping sync before fetch"
    (Either SomeException (), Either SomeException ()) -> IO ()
forall {a} {a} {b} {b}.
(Show a, Show a) =>
(Either a b, Either a b) -> IO ()
checkResultD ((Either SomeException (), Either SomeException ()) -> IO ())
-> IO (Either SomeException (), Either SomeException ()) -> IO ()
forall (m :: * -> *) a b. Monad m => (a -> m b) -> m a -> m b
=<< case (forall s.
 IOSim s (Either SomeException (), Either SomeException ()))
-> Either
     Failure (Either SomeException (), Either SomeException ())
forall a. (forall s. IOSim s a) -> Either Failure a
runSimStrictShutdown (((forall c. IOSim s c -> IOSim s c) -> IOSim s ())
-> ((forall c. IOSim s c -> IOSim s c) -> IOSim s ())
-> ((forall c. IOSim s c -> IOSim s c) -> IOSim s ())
-> IOSim s (Either SomeException (), Either SomeException ())
forall (m :: * -> *) a b d.
(MonadAsync m, MonadDelay m, MonadFork m, MonadMask m,
 MonadThrow m, MonadThrow (STM m), MonadTimer m) =>
((forall c. m c -> m c) -> m a)
-> ((forall c. m c -> m c) -> m b)
-> ((forall c. m c -> m c) -> m d)
-> m (Either SomeException a, Either SomeException b)
testSkeleton
                            (\forall c. IOSim s c -> IOSim s c
action -> IOSim s () -> IOSim s ()
forall c. IOSim s c -> IOSim s c
action (DiffTime -> IOSim s ()
forall (m :: * -> *). MonadDelay m => DiffTime -> m ()
threadDelay DiffTime
1.0))
                            (\forall c. IOSim s c -> IOSim s c
action -> IOSim s () -> IOSim s ()
forall c. IOSim s c -> IOSim s c
action (DiffTime -> IOSim s ()
forall (m :: * -> *). MonadDelay m => DiffTime -> m ()
threadDelay DiffTime
0.1))
                            (\forall c. IOSim s c -> IOSim s c
action -> DiffTime -> IOSim s ()
forall (m :: * -> *). MonadDelay m => DiffTime -> m ()
threadDelay DiffTime
0 IOSim s () -> IOSim s () -> IOSim s ()
forall a b. IOSim s a -> IOSim s b -> IOSim s b
forall (m :: * -> *) a b. Monad m => m a -> m b -> m b
>> IOSim s () -> IOSim s ()
forall c. IOSim s c -> IOSim s c
action (DiffTime -> IOSim s ()
forall (m :: * -> *). MonadDelay m => DiffTime -> m ()
threadDelay DiffTime
1.5))) of
                          Left Failure
e  -> [Char] -> IO (Either SomeException (), Either SomeException ())
forall a. HasCallStack => [Char] -> a
error ([Char] -> IO (Either SomeException (), Either SomeException ()))
-> [Char] -> IO (Either SomeException (), Either SomeException ())
forall a b. (a -> b) -> a -> b
$ [Char]
"sim failed with " [Char] -> [Char] -> [Char]
forall a. Semigroup a => a -> a -> a
<> Failure -> [Char]
forall a. Show a => a -> [Char]
show Failure
e
                          Right (Either SomeException (), Either SomeException ())
r -> (Either SomeException (), Either SomeException ())
-> IO (Either SomeException (), Either SomeException ())
forall a. a -> IO a
forall (m :: * -> *) a. Monad m => a -> m a
return (Either SomeException (), Either SomeException ())
r

    [Char] -> IO ()
step [Char]
"Exception in fetch"
    (Either SomeException Any, Either SomeException ()) -> IO ()
forall {a} {b} {b}. (Either a b, b) -> IO ()
checkResultB ((Either SomeException Any, Either SomeException ()) -> IO ())
-> IO (Either SomeException Any, Either SomeException ()) -> IO ()
forall (m :: * -> *) a b. Monad m => (a -> m b) -> m a -> m b
=<< case (forall s.
 IOSim s (Either SomeException Any, Either SomeException ()))
-> Either
     Failure (Either SomeException Any, Either SomeException ())
forall a. (forall s. IOSim s a) -> Either Failure a
runSimStrictShutdown (((forall c. IOSim s c -> IOSim s c) -> IOSim s Any)
-> ((forall c. IOSim s c -> IOSim s c) -> IOSim s ())
-> ((forall c. IOSim s c -> IOSim s c) -> IOSim s ())
-> IOSim s (Either SomeException Any, Either SomeException ())
forall (m :: * -> *) a b d.
(MonadAsync m, MonadDelay m, MonadFork m, MonadMask m,
 MonadThrow m, MonadThrow (STM m), MonadTimer m) =>
((forall c. m c -> m c) -> m a)
-> ((forall c. m c -> m c) -> m b)
-> ((forall c. m c -> m c) -> m d)
-> m (Either SomeException a, Either SomeException b)
testSkeleton
                            (\forall c. IOSim s c -> IOSim s c
action -> IOSim s Any -> IOSim s Any
forall c. IOSim s c -> IOSim s c
action (DiffTime -> IOSim s ()
forall (m :: * -> *). MonadDelay m => DiffTime -> m ()
threadDelay DiffTime
0.1 IOSim s () -> IOSim s Any -> IOSim s Any
forall a b. IOSim s a -> IOSim s b -> IOSim s b
forall (m :: * -> *) a b. Monad m => m a -> m b -> m b
>> AsyncCancelled -> IOSim s Any
forall e a. Exception e => e -> IOSim s a
forall (m :: * -> *) e a. (MonadThrow m, Exception e) => e -> m a
throwIO AsyncCancelled
AsyncCancelled))
                            (\forall c. IOSim s c -> IOSim s c
action -> IOSim s () -> IOSim s ()
forall c. IOSim s c -> IOSim s c
action (DiffTime -> IOSim s ()
forall (m :: * -> *). MonadDelay m => DiffTime -> m ()
threadDelay DiffTime
0.2))
                            (\forall c. IOSim s c -> IOSim s c
action -> DiffTime -> IOSim s ()
forall (m :: * -> *). MonadDelay m => DiffTime -> m ()
threadDelay DiffTime
0.1 IOSim s () -> IOSim s () -> IOSim s ()
forall a b. IOSim s a -> IOSim s b -> IOSim s b
forall (m :: * -> *) a b. Monad m => m a -> m b -> m b
>> IOSim s () -> IOSim s ()
forall c. IOSim s c -> IOSim s c
action (DiffTime -> IOSim s ()
forall (m :: * -> *). MonadDelay m => DiffTime -> m ()
threadDelay DiffTime
0.3))) of
                          Left Failure
e  -> [Char] -> IO (Either SomeException Any, Either SomeException ())
forall a. HasCallStack => [Char] -> a
error ([Char] -> IO (Either SomeException Any, Either SomeException ()))
-> [Char] -> IO (Either SomeException Any, Either SomeException ())
forall a b. (a -> b) -> a -> b
$ [Char]
"sim failed with " [Char] -> [Char] -> [Char]
forall a. Semigroup a => a -> a -> a
<> Failure -> [Char]
forall a. Show a => a -> [Char]
show Failure
e
                          Right (Either SomeException Any, Either SomeException ())
r -> (Either SomeException Any, Either SomeException ())
-> IO (Either SomeException Any, Either SomeException ())
forall a. a -> IO a
forall (m :: * -> *) a. Monad m => a -> m a
return (Either SomeException Any, Either SomeException ())
r

    [Char] -> IO ()
step [Char]
"Exception in sync"
    (Either SomeException (), Either SomeException Any) -> IO ()
forall {a} {b} {a} {b}. Show a => (Either a b, Either a b) -> IO ()
checkResultC ((Either SomeException (), Either SomeException Any) -> IO ())
-> IO (Either SomeException (), Either SomeException Any) -> IO ()
forall (m :: * -> *) a b. Monad m => (a -> m b) -> m a -> m b
=<< case (forall s.
 IOSim s (Either SomeException (), Either SomeException Any))
-> Either
     Failure (Either SomeException (), Either SomeException Any)
forall a. (forall s. IOSim s a) -> Either Failure a
runSimStrictShutdown (((forall c. IOSim s c -> IOSim s c) -> IOSim s ())
-> ((forall c. IOSim s c -> IOSim s c) -> IOSim s Any)
-> ((forall c. IOSim s c -> IOSim s c) -> IOSim s ())
-> IOSim s (Either SomeException (), Either SomeException Any)
forall (m :: * -> *) a b d.
(MonadAsync m, MonadDelay m, MonadFork m, MonadMask m,
 MonadThrow m, MonadThrow (STM m), MonadTimer m) =>
((forall c. m c -> m c) -> m a)
-> ((forall c. m c -> m c) -> m b)
-> ((forall c. m c -> m c) -> m d)
-> m (Either SomeException a, Either SomeException b)
testSkeleton
                            (\forall c. IOSim s c -> IOSim s c
action -> IOSim s () -> IOSim s ()
forall c. IOSim s c -> IOSim s c
action (DiffTime -> IOSim s ()
forall (m :: * -> *). MonadDelay m => DiffTime -> m ()
threadDelay DiffTime
0.2))
                            (\forall c. IOSim s c -> IOSim s c
action -> IOSim s Any -> IOSim s Any
forall c. IOSim s c -> IOSim s c
action (DiffTime -> IOSim s ()
forall (m :: * -> *). MonadDelay m => DiffTime -> m ()
threadDelay DiffTime
0.1 IOSim s () -> IOSim s Any -> IOSim s Any
forall a b. IOSim s a -> IOSim s b -> IOSim s b
forall (m :: * -> *) a b. Monad m => m a -> m b -> m b
>> AsyncCancelled -> IOSim s Any
forall e a. Exception e => e -> IOSim s a
forall (m :: * -> *) e a. (MonadThrow m, Exception e) => e -> m a
throwIO AsyncCancelled
AsyncCancelled))
                            (\forall c. IOSim s c -> IOSim s c
action -> DiffTime -> IOSim s ()
forall (m :: * -> *). MonadDelay m => DiffTime -> m ()
threadDelay DiffTime
0.1 IOSim s () -> IOSim s () -> IOSim s ()
forall a b. IOSim s a -> IOSim s b -> IOSim s b
forall (m :: * -> *) a b. Monad m => m a -> m b -> m b
>> IOSim s () -> IOSim s ()
forall c. IOSim s c -> IOSim s c
action (DiffTime -> IOSim s ()
forall (m :: * -> *). MonadDelay m => DiffTime -> m ()
threadDelay DiffTime
0.3))) of
                          Left Failure
e  -> [Char] -> IO (Either SomeException (), Either SomeException Any)
forall a. HasCallStack => [Char] -> a
error ([Char] -> IO (Either SomeException (), Either SomeException Any))
-> [Char] -> IO (Either SomeException (), Either SomeException Any)
forall a b. (a -> b) -> a -> b
$ [Char]
"sim failed with " [Char] -> [Char] -> [Char]
forall a. Semigroup a => a -> a -> a
<> Failure -> [Char]
forall a. Show a => a -> [Char]
show Failure
e
                          Right (Either SomeException (), Either SomeException Any)
r -> (Either SomeException (), Either SomeException Any)
-> IO (Either SomeException (), Either SomeException Any)
forall a. a -> IO a
forall (m :: * -> *) a. Monad m => a -> m a
return (Either SomeException (), Either SomeException Any)
r

    [Char] -> IO ()
step [Char]
"Exception in keepalive"
    (Either SomeException (), Either SomeException ()) -> IO ()
forall {a} {b} {a} {b}. (Either a b, Either a b) -> IO ()
checkResultE ((Either SomeException (), Either SomeException ()) -> IO ())
-> IO (Either SomeException (), Either SomeException ()) -> IO ()
forall (m :: * -> *) a b. Monad m => (a -> m b) -> m a -> m b
=<< case (forall s.
 IOSim s (Either SomeException (), Either SomeException ()))
-> Either
     Failure (Either SomeException (), Either SomeException ())
forall a. (forall s. IOSim s a) -> Either Failure a
runSimStrictShutdown (((forall c. IOSim s c -> IOSim s c) -> IOSim s ())
-> ((forall c. IOSim s c -> IOSim s c) -> IOSim s ())
-> ((forall c. IOSim s c -> IOSim s c) -> IOSim s Any)
-> IOSim s (Either SomeException (), Either SomeException ())
forall (m :: * -> *) a b d.
(MonadAsync m, MonadDelay m, MonadFork m, MonadMask m,
 MonadThrow m, MonadThrow (STM m), MonadTimer m) =>
((forall c. m c -> m c) -> m a)
-> ((forall c. m c -> m c) -> m b)
-> ((forall c. m c -> m c) -> m d)
-> m (Either SomeException a, Either SomeException b)
testSkeleton
                            (\forall c. IOSim s c -> IOSim s c
action -> IOSim s () -> IOSim s ()
forall c. IOSim s c -> IOSim s c
action (DiffTime -> IOSim s ()
forall (m :: * -> *). MonadDelay m => DiffTime -> m ()
threadDelay DiffTime
0.2))
                            (\forall c. IOSim s c -> IOSim s c
action -> IOSim s () -> IOSim s ()
forall c. IOSim s c -> IOSim s c
action (DiffTime -> IOSim s ()
forall (m :: * -> *). MonadDelay m => DiffTime -> m ()
threadDelay DiffTime
0.2))
                            (\forall c. IOSim s c -> IOSim s c
action -> IOSim s Any -> IOSim s Any
forall c. IOSim s c -> IOSim s c
action (DiffTime -> IOSim s ()
forall (m :: * -> *). MonadDelay m => DiffTime -> m ()
threadDelay DiffTime
0.1 IOSim s () -> IOSim s Any -> IOSim s Any
forall a b. IOSim s a -> IOSim s b -> IOSim s b
forall (m :: * -> *) a b. Monad m => m a -> m b -> m b
>> AsyncCancelled -> IOSim s Any
forall e a. Exception e => e -> IOSim s a
forall (m :: * -> *) e a. (MonadThrow m, Exception e) => e -> m a
throwIO AsyncCancelled
AsyncCancelled))) of
                          Left Failure
e  -> [Char] -> IO (Either SomeException (), Either SomeException ())
forall a. HasCallStack => [Char] -> a
error ([Char] -> IO (Either SomeException (), Either SomeException ()))
-> [Char] -> IO (Either SomeException (), Either SomeException ())
forall a b. (a -> b) -> a -> b
$ [Char]
"sim failed with " [Char] -> [Char] -> [Char]
forall a. Semigroup a => a -> a -> a
<> Failure -> [Char]
forall a. Show a => a -> [Char]
show Failure
e
                          Right (Either SomeException (), Either SomeException ())
r -> (Either SomeException (), Either SomeException ())
-> IO (Either SomeException (), Either SomeException ())
forall a. a -> IO a
forall (m :: * -> *) a. Monad m => a -> m a
return (Either SomeException (), Either SomeException ())
r

    [Char] -> IO ()
step [Char]
"Keep alive kills fetch"
    (Either SomeException (), Either SomeException ()) -> IO ()
forall {a} {b} {a} {b}. (Either a b, Either a b) -> IO ()
checkResultE ((Either SomeException (), Either SomeException ()) -> IO ())
-> IO (Either SomeException (), Either SomeException ()) -> IO ()
forall (m :: * -> *) a b. Monad m => (a -> m b) -> m a -> m b
=<< case (forall s.
 IOSim s (Either SomeException (), Either SomeException ()))
-> Either
     Failure (Either SomeException (), Either SomeException ())
forall a. (forall s. IOSim s a) -> Either Failure a
runSimStrictShutdown (((forall c. IOSim s c -> IOSim s c) -> IOSim s ())
-> ((forall c. IOSim s c -> IOSim s c) -> IOSim s ())
-> ((forall c. IOSim s c -> IOSim s c) -> IOSim s ())
-> IOSim s (Either SomeException (), Either SomeException ())
forall (m :: * -> *) a b d.
(MonadAsync m, MonadDelay m, MonadFork m, MonadMask m,
 MonadThrow m, MonadThrow (STM m), MonadTimer m) =>
((forall c. m c -> m c) -> m a)
-> ((forall c. m c -> m c) -> m b)
-> ((forall c. m c -> m c) -> m d)
-> m (Either SomeException a, Either SomeException b)
testSkeleton
                            (\forall c. IOSim s c -> IOSim s c
action -> DiffTime -> IOSim s ()
forall (m :: * -> *). MonadDelay m => DiffTime -> m ()
threadDelay DiffTime
0.1 IOSim s () -> IOSim s () -> IOSim s ()
forall a b. IOSim s a -> IOSim s b -> IOSim s b
forall (m :: * -> *) a b. Monad m => m a -> m b -> m b
>> IOSim s () -> IOSim s ()
forall c. IOSim s c -> IOSim s c
action (DiffTime -> IOSim s ()
forall (m :: * -> *). MonadDelay m => DiffTime -> m ()
threadDelay DiffTime
60))
                            (\forall c. IOSim s c -> IOSim s c
action -> DiffTime -> IOSim s ()
forall (m :: * -> *). MonadDelay m => DiffTime -> m ()
threadDelay DiffTime
0.1 IOSim s () -> IOSim s () -> IOSim s ()
forall a b. IOSim s a -> IOSim s b -> IOSim s b
forall (m :: * -> *) a b. Monad m => m a -> m b -> m b
>> IOSim s () -> IOSim s ()
forall c. IOSim s c -> IOSim s c
action (DiffTime -> IOSim s ()
forall (m :: * -> *). MonadDelay m => DiffTime -> m ()
threadDelay DiffTime
60))
                            (\forall c. IOSim s c -> IOSim s c
action -> DiffTime -> IOSim s ()
forall (m :: * -> *). MonadDelay m => DiffTime -> m ()
threadDelay DiffTime
0.1 IOSim s () -> IOSim s () -> IOSim s ()
forall a b. IOSim s a -> IOSim s b -> IOSim s b
forall (m :: * -> *) a b. Monad m => m a -> m b -> m b
>> IOSim s () -> IOSim s ()
forall c. IOSim s c -> IOSim s c
action (DiffTime -> IOSim s ()
forall (m :: * -> *). MonadDelay m => DiffTime -> m ()
threadDelay DiffTime
1.3))) of
                          Left Failure
e  -> [Char] -> IO (Either SomeException (), Either SomeException ())
forall a. HasCallStack => [Char] -> a
error ([Char] -> IO (Either SomeException (), Either SomeException ()))
-> [Char] -> IO (Either SomeException (), Either SomeException ())
forall a b. (a -> b) -> a -> b
$ [Char]
"sim failed with " [Char] -> [Char] -> [Char]
forall a. Semigroup a => a -> a -> a
<> Failure -> [Char]
forall a. Show a => a -> [Char]
show Failure
e
                          Right (Either SomeException (), Either SomeException ())
r -> (Either SomeException (), Either SomeException ())
-> IO (Either SomeException (), Either SomeException ())
forall a. a -> IO a
forall (m :: * -> *) a. Monad m => a -> m a
return (Either SomeException (), Either SomeException ())
r

    [Char] -> IO ()
step [Char]
"Deadlock without keep alive"
    case (forall s.
 IOSim s (Either SomeException (), Either SomeException ()))
-> Either
     Failure (Either SomeException (), Either SomeException ())
forall a. (forall s. IOSim s a) -> Either Failure a
runSimStrictShutdown (((forall c. IOSim s c -> IOSim s c) -> IOSim s ())
-> ((forall c. IOSim s c -> IOSim s c) -> IOSim s ())
-> ((forall c. IOSim s c -> IOSim s c) -> IOSim s ())
-> IOSim s (Either SomeException (), Either SomeException ())
forall (m :: * -> *) a b d.
(MonadAsync m, MonadDelay m, MonadFork m, MonadMask m,
 MonadThrow m, MonadThrow (STM m), MonadTimer m) =>
((forall c. m c -> m c) -> m a)
-> ((forall c. m c -> m c) -> m b)
-> ((forall c. m c -> m c) -> m d)
-> m (Either SomeException a, Either SomeException b)
testSkeleton
           (\forall c. IOSim s c -> IOSim s c
action -> DiffTime -> IOSim s ()
forall (m :: * -> *). MonadDelay m => DiffTime -> m ()
threadDelay DiffTime
0.1 IOSim s () -> IOSim s () -> IOSim s ()
forall a b. IOSim s a -> IOSim s b -> IOSim s b
forall (m :: * -> *) a b. Monad m => m a -> m b -> m b
>> IOSim s () -> IOSim s ()
forall c. IOSim s c -> IOSim s c
action (DiffTime -> IOSim s ()
forall (m :: * -> *). MonadDelay m => DiffTime -> m ()
threadDelay DiffTime
60))
           (\forall c. IOSim s c -> IOSim s c
action -> DiffTime -> IOSim s ()
forall (m :: * -> *). MonadDelay m => DiffTime -> m ()
threadDelay DiffTime
0.1 IOSim s () -> IOSim s () -> IOSim s ()
forall a b. IOSim s a -> IOSim s b -> IOSim s b
forall (m :: * -> *) a b. Monad m => m a -> m b -> m b
>> IOSim s () -> IOSim s ()
forall c. IOSim s c -> IOSim s c
action (DiffTime -> IOSim s ()
forall (m :: * -> *). MonadDelay m => DiffTime -> m ()
threadDelay DiffTime
60))
           (\forall c. IOSim s c -> IOSim s c
_action -> DiffTime -> IOSim s ()
forall (m :: * -> *). MonadDelay m => DiffTime -> m ()
threadDelay DiffTime
3.1 )) of
         Left (FailureDeadlock [Labelled IOSimThreadId]
_) -> () -> IO ()
forall a. a -> IO a
forall (m :: * -> *) a. Monad m => a -> m a
return ()
         Left Failure
e                   -> [Char] -> IO ()
forall a. HasCallStack => [Char] -> a
error ([Char] -> IO ()) -> [Char] -> IO ()
forall a b. (a -> b) -> a -> b
$ [Char]
"sim failed with " [Char] -> [Char] -> [Char]
forall a. Semigroup a => a -> a -> a
<> Failure -> [Char]
forall a. Show a => a -> [Char]
show Failure
e
         Right (Either SomeException (), Either SomeException ())
_                  -> [Char] -> IO ()
forall a. HasCallStack => [Char] -> a
error [Char]
"unexpected success"

    [Char] -> IO ()
step [Char]
"Deadlock without fetch"
    case (forall s.
 IOSim s (Either SomeException (), Either SomeException ()))
-> Either
     Failure (Either SomeException (), Either SomeException ())
forall a. (forall s. IOSim s a) -> Either Failure a
runSimStrictShutdown (((forall c. IOSim s c -> IOSim s c) -> IOSim s ())
-> ((forall c. IOSim s c -> IOSim s c) -> IOSim s ())
-> ((forall c. IOSim s c -> IOSim s c) -> IOSim s ())
-> IOSim s (Either SomeException (), Either SomeException ())
forall (m :: * -> *) a b d.
(MonadAsync m, MonadDelay m, MonadFork m, MonadMask m,
 MonadThrow m, MonadThrow (STM m), MonadTimer m) =>
((forall c. m c -> m c) -> m a)
-> ((forall c. m c -> m c) -> m b)
-> ((forall c. m c -> m c) -> m d)
-> m (Either SomeException a, Either SomeException b)
testSkeleton
           (\forall c. IOSim s c -> IOSim s c
_action -> DiffTime -> IOSim s ()
forall (m :: * -> *). MonadDelay m => DiffTime -> m ()
threadDelay DiffTime
3)
           (\forall c. IOSim s c -> IOSim s c
action -> DiffTime -> IOSim s ()
forall (m :: * -> *). MonadDelay m => DiffTime -> m ()
threadDelay DiffTime
0.1 IOSim s () -> IOSim s () -> IOSim s ()
forall a b. IOSim s a -> IOSim s b -> IOSim s b
forall (m :: * -> *) a b. Monad m => m a -> m b -> m b
>> IOSim s () -> IOSim s ()
forall c. IOSim s c -> IOSim s c
action (DiffTime -> IOSim s ()
forall (m :: * -> *). MonadDelay m => DiffTime -> m ()
threadDelay DiffTime
60))
           (\forall c. IOSim s c -> IOSim s c
action -> DiffTime -> IOSim s ()
forall (m :: * -> *). MonadDelay m => DiffTime -> m ()
threadDelay DiffTime
0.1 IOSim s () -> IOSim s () -> IOSim s ()
forall a b. IOSim s a -> IOSim s b -> IOSim s b
forall (m :: * -> *) a b. Monad m => m a -> m b -> m b
>> IOSim s () -> IOSim s ()
forall c. IOSim s c -> IOSim s c
action (DiffTime -> IOSim s ()
forall (m :: * -> *). MonadDelay m => DiffTime -> m ()
threadDelay DiffTime
1.3))) of
         Left (FailureDeadlock [Labelled IOSimThreadId]
_) -> () -> IO ()
forall a. a -> IO a
forall (m :: * -> *) a. Monad m => a -> m a
return ()
         Left Failure
e                   -> [Char] -> IO ()
forall a. HasCallStack => [Char] -> a
error ([Char] -> IO ()) -> [Char] -> IO ()
forall a b. (a -> b) -> a -> b
$ [Char]
"sim failed with " [Char] -> [Char] -> [Char]
forall a. Semigroup a => a -> a -> a
<> Failure -> [Char]
forall a. Show a => a -> [Char]
show Failure
e
         Right (Either SomeException (), Either SomeException ())
_                  -> [Char] -> IO ()
forall a. HasCallStack => [Char] -> a
error [Char]
"unexpected success"


    () -> IO ()
forall a. a -> IO a
forall (m :: * -> *) a. Monad m => a -> m a
return ()

  where
    dummyPolicy :: forall b h m. (MonadSTM m) => STM m (FetchClientPolicy h b m)
    dummyPolicy :: forall b h (m :: * -> *).
MonadSTM m =>
STM m (FetchClientPolicy h b m)
dummyPolicy =
      let addFetchedBlock :: p -> p -> m ()
addFetchedBlock p
_ p
_ = () -> m ()
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return ()
          forgeTime :: p -> m a
forgeTime p
_ = a -> m a
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return ([Char] -> a
forall a. Read a => [Char] -> a
read [Char]
"2000-01-01 00:00:00 UTC")
          bfSize :: p -> a
bfSize p
_ = a
1024
          matchesHeader :: p -> p -> Bool
matchesHeader p
_ p
_ = Bool
True in
      FetchClientPolicy h b m -> STM m (FetchClientPolicy h b m)
forall a. a -> STM m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (FetchClientPolicy h b m -> STM m (FetchClientPolicy h b m))
-> FetchClientPolicy h b m -> STM m (FetchClientPolicy h b m)
forall a b. (a -> b) -> a -> b
$ (h -> SizeInBytes)
-> (h -> b -> Bool)
-> (Point b -> b -> m ())
-> (FromConsensus b -> STM m UTCTime)
-> FetchClientPolicy h b m
forall header block (m :: * -> *).
(header -> SizeInBytes)
-> (header -> block -> Bool)
-> (Point block -> block -> m ())
-> (FromConsensus block -> STM m UTCTime)
-> FetchClientPolicy header block m
FetchClientPolicy
          h -> SizeInBytes
forall {a} {p}. Num a => p -> a
bfSize
          h -> b -> Bool
forall {p} {p}. p -> p -> Bool
matchesHeader
          Point b -> b -> m ()
forall {m :: * -> *} {p} {p}. Monad m => p -> p -> m ()
addFetchedBlock
          FromConsensus b -> STM m UTCTime
forall {m :: * -> *} {a} {p}. (Monad m, Read a) => p -> m a
forgeTime


    -- Fetch success
    checkResultA :: (Either a b, b) -> IO ()
checkResultA (Right b
_, b
_) = () -> IO ()
forall a. a -> IO a
forall (m :: * -> *) a. Monad m => a -> m a
return ()
    checkResultA (Left a
e, b
_)  = [Char] -> IO ()
forall a. HasCallStack => [Char] -> IO a
assertFailure ([Char] -> IO ()) -> [Char] -> IO ()
forall a b. (a -> b) -> a -> b
$ [Char]
"unexpected fetch failure " [Char] -> [Char] -> [Char]
forall a. [a] -> [a] -> [a]
++ a -> [Char]
forall a. Show a => a -> [Char]
show a
e

    -- Fetch Failure
    checkResultB :: (Either a b, b) -> IO ()
checkResultB (Right b
_, b
_) = [Char] -> IO ()
forall a. HasCallStack => [Char] -> IO a
assertFailure [Char]
"unexpected fetch success"
    checkResultB (Left a
_, b
_)  = () -> IO ()
forall a. a -> IO a
forall (m :: * -> *) a. Monad m => a -> m a
return ()

    -- Fetch success and Sync failure
    checkResultC :: (Either a b, Either a b) -> IO ()
checkResultC (Right b
_, Left a
_)  = () -> IO ()
forall a. a -> IO a
forall (m :: * -> *) a. Monad m => a -> m a
return ()
    checkResultC (Right b
_, Right b
_) = [Char] -> IO ()
forall a. HasCallStack => [Char] -> IO a
assertFailure [Char]
"unexpected sync success"
    checkResultC (Left a
e, Either a b
_)        = [Char] -> IO ()
forall a. HasCallStack => [Char] -> IO a
assertFailure ([Char] -> IO ()) -> [Char] -> IO ()
forall a b. (a -> b) -> a -> b
$ [Char]
"unexpected fetch failure " [Char] -> [Char] -> [Char]
forall a. [a] -> [a] -> [a]
++ a -> [Char]
forall a. Show a => a -> [Char]
show a
e

    -- Fetch and Sync sucess
    checkResultD :: (Either a b, Either a b) -> IO ()
checkResultD (Right b
_, Right b
_) = () -> IO ()
forall a. a -> IO a
forall (m :: * -> *) a. Monad m => a -> m a
return ()
    checkResultD (Left a
e, Either a b
_)        = [Char] -> IO ()
forall a. HasCallStack => [Char] -> IO a
assertFailure ([Char] -> IO ()) -> [Char] -> IO ()
forall a b. (a -> b) -> a -> b
$ [Char]
"unexpected fetch failure " [Char] -> [Char] -> [Char]
forall a. [a] -> [a] -> [a]
++ a -> [Char]
forall a. Show a => a -> [Char]
show a
e
    checkResultD (Right b
_, Left a
e)  = [Char] -> IO ()
forall a. HasCallStack => [Char] -> IO a
assertFailure ([Char] -> IO ()) -> [Char] -> IO ()
forall a b. (a -> b) -> a -> b
$ [Char]
"unexpected sync failure " [Char] -> [Char] -> [Char]
forall a. [a] -> [a] -> [a]
++ a -> [Char]
forall a. Show a => a -> [Char]
show a
e

    -- Fetch and Sync failure
    checkResultE :: (Either a b, Either a b) -> IO ()
checkResultE (Left a
_, Left a
_) = () -> IO ()
forall a. a -> IO a
forall (m :: * -> *) a. Monad m => a -> m a
return ()
    checkResultE (Either a b, Either a b)
_                = [Char] -> IO ()
forall a. HasCallStack => [Char] -> IO a
assertFailure [Char]
"unexpected success"

    testSkeleton :: forall m a b d.
                    (MonadAsync m, MonadDelay m, MonadFork m, MonadMask m,
                     MonadThrow m, MonadThrow (STM m), MonadTimer m)
                 => ((forall c. m c -> m c) -> m a)
                 -> ((forall c. m c -> m c) -> m b)
                 -> ((forall c. m c -> m c) -> m d)
                 -> m (Either SomeException a, Either SomeException b)
    testSkeleton :: forall (m :: * -> *) a b d.
(MonadAsync m, MonadDelay m, MonadFork m, MonadMask m,
 MonadThrow m, MonadThrow (STM m), MonadTimer m) =>
((forall c. m c -> m c) -> m a)
-> ((forall c. m c -> m c) -> m b)
-> ((forall c. m c -> m c) -> m d)
-> m (Either SomeException a, Either SomeException b)
testSkeleton (forall c. m c -> m c) -> m a
withFetchTestAction (forall c. m c -> m c) -> m b
withSyncTestAction (forall c. m c -> m c) -> m d
withKeepAliveTestAction = do
      registry <- m (FetchClientRegistry [Char] BlockHeader Any m)
forall (m :: * -> *) peer header block.
MonadSTM m =>
m (FetchClientRegistry peer header block m)
newFetchClientRegistry
      setFetchClientContext registry nullTracer (const dummyPolicy)

      fetchStatePeerChainsVar <- newTVarIO Map.empty

      let peer  = [Char]
"thepeer"
          fetch :: m a
          fetch = (forall c. m c -> m c) -> m a
withFetchTestAction ((forall c. m c -> m c) -> m a) -> (forall c. m c -> m c) -> m a
forall a b. (a -> b) -> a -> b
$ \m c
body ->
                    FetchClientRegistry [Char] BlockHeader Any m
-> NodeToNodeVersion
-> (NodeToNodeVersion -> WhetherReceivingTentativeBlocks)
-> [Char]
-> (FetchClientContext BlockHeader Any m -> m c)
-> m c
forall (m :: * -> *) a peer header block version.
(MonadFork m, MonadMask m, MonadTimer m, Ord peer) =>
FetchClientRegistry peer header block m
-> version
-> (version -> WhetherReceivingTentativeBlocks)
-> peer
-> (FetchClientContext header block m -> m a)
-> m a
bracketFetchClient FetchClientRegistry [Char] BlockHeader Any m
registry NodeToNodeVersion
forall a. Bounded a => a
maxBound NodeToNodeVersion -> WhetherReceivingTentativeBlocks
isPipeliningEnabled [Char]
peer ((FetchClientContext BlockHeader Any m -> m c) -> m c)
-> (FetchClientContext BlockHeader Any m -> m c) -> m c
forall a b. (a -> b) -> a -> b
$ \FetchClientContext BlockHeader Any m
_ ->
                      m c
body

          sync :: m b
          sync  = (forall c. m c -> m c) -> m b
withSyncTestAction ((forall c. m c -> m c) -> m b) -> (forall c. m c -> m c) -> m b
forall a b. (a -> b) -> a -> b
$ \m c
body ->
                    FetchClientRegistry [Char] BlockHeader Any m
-> [Char] -> m c -> m c
forall (m :: * -> *) a peer header block.
(MonadSTM m, MonadFork m, MonadCatch m, Ord peer) =>
FetchClientRegistry peer header block m -> peer -> m a -> m a
bracketSyncWithFetchClient FetchClientRegistry [Char] BlockHeader Any m
registry [Char]
peer (m c -> m c) -> m c -> m c
forall a b. (a -> b) -> a -> b
$
                      m () -> m () -> m c -> m c
forall a b c. m a -> m b -> m c -> m c
forall (m :: * -> *) a b c.
MonadThrow m =>
m a -> m b -> m c -> m c
bracket_
                        (STM m () -> m ()
forall a. HasCallStack => STM m a -> m a
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
STM m a -> m a
atomically (StrictTVar m (Map [Char] ())
-> (Map [Char] () -> Map [Char] ()) -> STM m ()
forall (m :: * -> *) a.
MonadSTM m =>
StrictTVar m a -> (a -> a) -> STM m ()
modifyTVar StrictTVar m (Map [Char] ())
fetchStatePeerChainsVar
                                                ([Char] -> () -> Map [Char] () -> Map [Char] ()
forall k a. Ord k => k -> a -> Map k a -> Map k a
Map.insert [Char]
peer ())))
                        (STM m () -> m ()
forall a. HasCallStack => STM m a -> m a
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
STM m a -> m a
atomically (StrictTVar m (Map [Char] ())
-> (Map [Char] () -> Map [Char] ()) -> STM m ()
forall (m :: * -> *) a.
MonadSTM m =>
StrictTVar m a -> (a -> a) -> STM m ()
modifyTVar StrictTVar m (Map [Char] ())
fetchStatePeerChainsVar
                                                ([Char] -> Map [Char] () -> Map [Char] ()
forall k a. Ord k => k -> Map k a -> Map k a
Map.delete [Char]
peer)))
                        m c
body

          keep :: m d
          keep  = (forall c. m c -> m c) -> m d
withKeepAliveTestAction ((forall c. m c -> m c) -> m d) -> (forall c. m c -> m c) -> m d
forall a b. (a -> b) -> a -> b
$ \m c
body ->
                    FetchClientRegistry [Char] BlockHeader Any m
-> [Char] -> (StrictTVar m (Map [Char] PeerGSV) -> m c) -> m c
forall (m :: * -> *) a peer header block.
(MonadSTM m, MonadFork m, MonadMask m, Ord peer) =>
FetchClientRegistry peer header block m
-> peer -> (StrictTVar m (Map peer PeerGSV) -> m a) -> m a
bracketKeepAliveClient FetchClientRegistry [Char] BlockHeader Any m
registry [Char]
peer ((StrictTVar m (Map [Char] PeerGSV) -> m c) -> m c)
-> (StrictTVar m (Map [Char] PeerGSV) -> m c) -> m c
forall a b. (a -> b) -> a -> b
$ m c -> StrictTVar m (Map [Char] PeerGSV) -> m c
forall a b. a -> b -> a
const m c
body

          logic :: (Map String (PeerFetchStatus BlockHeader), Map String ())
                -> m ()
          logic (Map [Char] (PeerFetchStatus BlockHeader), Map [Char] ())
fingerprint = do
            fingerprint' <- STM m (Map [Char] (PeerFetchStatus BlockHeader), Map [Char] ())
-> m (Map [Char] (PeerFetchStatus BlockHeader), Map [Char] ())
forall a. HasCallStack => STM m a -> m a
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
STM m a -> m a
atomically (STM m (Map [Char] (PeerFetchStatus BlockHeader), Map [Char] ())
 -> m (Map [Char] (PeerFetchStatus BlockHeader), Map [Char] ()))
-> STM m (Map [Char] (PeerFetchStatus BlockHeader), Map [Char] ())
-> m (Map [Char] (PeerFetchStatus BlockHeader), Map [Char] ())
forall a b. (a -> b) -> a -> b
$ do
              fetchStatePeerStates <- FetchClientRegistry [Char] BlockHeader Any m
-> STM m (Map [Char] (PeerFetchStatus BlockHeader))
forall (m :: * -> *) peer header block.
MonadSTM m =>
FetchClientRegistry peer header block m
-> STM m (Map peer (PeerFetchStatus header))
readFetchClientsStatus FetchClientRegistry [Char] BlockHeader Any m
registry
              fetchStatePeerChains <- readTVar fetchStatePeerChainsVar
              let fingerprint' = (Map [Char] (PeerFetchStatus BlockHeader)
fetchStatePeerStates, Map [Char] ()
fetchStatePeerChains)
              check (fingerprint' /= fingerprint)
              return fingerprint'

            let (fetchStatePeerStates, fetchStatePeerChains) = fingerprint'
            unless (                 Map.keysSet fetchStatePeerChains
                    `Set.isSubsetOf` Map.keysSet fetchStatePeerStates) $
              throwIO (AssertionFailed "detected state mismatch")

            logic fingerprint'

      withAsync     keep  $ \Async m d
keepAsync  ->
        m a
-> (Async m a
    -> m (Either SomeException a, Either SomeException b))
-> m (Either SomeException a, Either SomeException b)
forall a b. m a -> (Async m a -> m b) -> m b
forall (m :: * -> *) a b.
MonadAsync m =>
m a -> (Async m a -> m b) -> m b
withAsync   m a
fetch ((Async m a -> m (Either SomeException a, Either SomeException b))
 -> m (Either SomeException a, Either SomeException b))
-> (Async m a
    -> m (Either SomeException a, Either SomeException b))
-> m (Either SomeException a, Either SomeException b)
forall a b. (a -> b) -> a -> b
$ \Async m a
fetchAsync ->
          m b
-> (Async m b
    -> m (Either SomeException a, Either SomeException b))
-> m (Either SomeException a, Either SomeException b)
forall a b. m a -> (Async m a -> m b) -> m b
forall (m :: * -> *) a b.
MonadAsync m =>
m a -> (Async m a -> m b) -> m b
withAsync m b
sync  ((Async m b -> m (Either SomeException a, Either SomeException b))
 -> m (Either SomeException a, Either SomeException b))
-> (Async m b
    -> m (Either SomeException a, Either SomeException b))
-> m (Either SomeException a, Either SomeException b)
forall a b. (a -> b) -> a -> b
$ \Async m b
syncAsync  ->
            m ()
-> (Async m ()
    -> m (Either SomeException a, Either SomeException b))
-> m (Either SomeException a, Either SomeException b)
forall a b. m a -> (Async m a -> m b) -> m b
forall (m :: * -> *) a b.
MonadAsync m =>
m a -> (Async m a -> m b) -> m b
withAsync ((Map [Char] (PeerFetchStatus BlockHeader), Map [Char] ()) -> m ()
logic (Map [Char] (PeerFetchStatus BlockHeader)
forall k a. Map k a
Map.empty, Map [Char] ()
forall k a. Map k a
Map.empty)) ((Async m () -> m (Either SomeException a, Either SomeException b))
 -> m (Either SomeException a, Either SomeException b))
-> (Async m ()
    -> m (Either SomeException a, Either SomeException b))
-> m (Either SomeException a, Either SomeException b)
forall a b. (a -> b) -> a -> b
$ \Async m ()
logicAsync -> do
              m (Either (Either SomeException a) (Either SomeException b))
-> m ()
forall (f :: * -> *) a. Functor f => f a -> f ()
void (m (Either (Either SomeException a) (Either SomeException b))
 -> m ())
-> m (Either (Either SomeException a) (Either SomeException b))
-> m ()
forall a b. (a -> b) -> a -> b
$ STM m (Either (Either SomeException a) (Either SomeException b))
-> m (Either (Either SomeException a) (Either SomeException b))
forall a. HasCallStack => STM m a -> m a
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
STM m a -> m a
atomically (STM m (Either (Either SomeException a) (Either SomeException b))
 -> m (Either (Either SomeException a) (Either SomeException b)))
-> STM m (Either (Either SomeException a) (Either SomeException b))
-> m (Either (Either SomeException a) (Either SomeException b))
forall a b. (a -> b) -> a -> b
$ do
                res <- Async m () -> STM m (Maybe (Either SomeException ()))
forall a. Async m a -> STM m (Maybe (Either SomeException a))
forall (m :: * -> *) a.
MonadAsync m =>
Async m a -> STM m (Maybe (Either SomeException a))
pollSTM Async m ()
logicAsync
                case res of
                  Maybe (Either SomeException ())
Nothing         -> Async m a
-> Async m b
-> STM m (Either (Either SomeException a) (Either SomeException b))
forall a b.
Async m a
-> Async m b
-> STM m (Either (Either SomeException a) (Either SomeException b))
forall (m :: * -> *) a b.
MonadAsync m =>
Async m a
-> Async m b
-> STM m (Either (Either SomeException a) (Either SomeException b))
waitEitherCatchSTM Async m a
fetchAsync Async m b
syncAsync
                  Just (Left  SomeException
e)  -> SomeException
-> STM m (Either (Either SomeException a) (Either SomeException b))
forall e a. Exception e => e -> STM m a
forall (m :: * -> *) e a. (MonadThrow m, Exception e) => e -> m a
throwIO SomeException
e
                  Just (Right ()) -> [Char]
-> STM m (Either (Either SomeException a) (Either SomeException b))
forall a. HasCallStack => [Char] -> a
error [Char]
"impossible"

              DiffTime -> m ()
forall (m :: * -> *). MonadDelay m => DiffTime -> m ()
threadDelay DiffTime
0.1
              -- give the logic thread a chance to detect any final problems
              STM m () -> m ()
forall a. HasCallStack => STM m a -> m a
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
STM m a -> m a
atomically (STM m () -> m ()) -> STM m () -> m ()
forall a b. (a -> b) -> a -> b
$ do
                x <- Async m () -> STM m (Maybe (Either SomeException ()))
forall a. Async m a -> STM m (Maybe (Either SomeException a))
forall (m :: * -> *) a.
MonadAsync m =>
Async m a -> STM m (Maybe (Either SomeException a))
pollSTM Async m ()
logicAsync
                case x of
                  Just (Left SomeException
e) -> SomeException -> STM m ()
forall e a. Exception e => e -> STM m a
forall (m :: * -> *) e a. (MonadThrow m, Exception e) => e -> m a
throwIO SomeException
e
                  Maybe (Either SomeException ())
_             -> () -> STM m ()
forall a. a -> STM m a
forall (m :: * -> *) a. Monad m => a -> m a
return ()

              fetchRes <- Async m a -> m (Either SomeException a)
forall a. Async m a -> m (Either SomeException a)
forall (m :: * -> *) a.
MonadAsync m =>
Async m a -> m (Either SomeException a)
waitCatch Async m a
fetchAsync
              syncRes  <- waitCatch syncAsync
              void $ waitCatch keepAsync
              atomically $ do
                fr <- readTVar $ fcrFetchRegistry registry
                sr <- readTVar $ fcrSyncRegistry  registry
                dr <- readTVar $ fcrDqRegistry    registry
                kr <- readTVar $ fcrKeepRegistry  registry
                yr <- readTVar $ fcrDying         registry
                if and [Map.null fr, Map.null sr, Map.null dr, Map.null kr, Set.null yr]
                   then return ()
                   else error "state leak"
              return (fetchRes, syncRes)

-- | Check that the client can terminate using `ControlMessage` mechanism.
--
-- The 'awaitDelay' of @100 * delay@ is a bit arbitrary.  It would be nicer to
-- make a proper calculation what should it be.  At the moment this test shows
-- that the block fetch protocol can exit within some large time limit.
--
prop_terminate :: TestChainFork -> Positive SmallDelay -> Property
prop_terminate :: TestChainFork -> Positive SmallDelay -> Property
prop_terminate (TestChainFork Chain Block
_commonChain Chain Block
forkChain Chain Block
_forkChain) (Positive (SmallDelay DiffTime
delay)) =
    let tr :: SimTrace Bool
tr = (forall s. IOSim s Bool) -> SimTrace Bool
forall a. (forall s. IOSim s a) -> SimTrace a
runSimTrace IOSim s Bool
forall s. IOSim s Bool
simulation
        trace :: [FetchRequestTrace]
        trace :: [FetchRequestTrace]
trace  = SimTrace Bool -> [FetchRequestTrace]
forall a b. Typeable b => SimTrace a -> [b]
selectTraceEventsDynamic SimTrace Bool
tr
    in [Char] -> Property -> Property
forall prop. Testable prop => [Char] -> prop -> Property
counterexample
        ([Char]
"Trace: \n" [Char] -> [Char] -> [Char]
forall a. [a] -> [a] -> [a]
++ [[Char]] -> [Char]
unlines ((FetchRequestTrace -> [Char]) -> [FetchRequestTrace] -> [[Char]]
forall a b. (a -> b) -> [a] -> [b]
map FetchRequestTrace -> [Char]
forall a. Show a => a -> [Char]
show [FetchRequestTrace]
trace))
        (case Bool -> SimTrace Bool -> Either Failure Bool
forall a. Bool -> SimTrace a -> Either Failure a
traceResult Bool
True SimTrace Bool
tr of
           Left Failure
e  -> Failure -> Property
forall a e. (HasCallStack, Exception e) => e -> a
throw Failure
e
           Right Bool
x -> [Char] -> Bool -> Property
forall prop. Testable prop => [Char] -> prop -> Property
counterexample [Char]
"block-fetch was unstoppable" Bool
x)
  where
    simulation :: forall s. IOSim s Bool
    simulation :: forall s. IOSim s Bool
simulation = do
      controlMessageVar <- ControlMessage -> IOSim s (StrictTVar (IOSim s) ControlMessage)
forall (m :: * -> *) a. MonadSTM m => a -> m (StrictTVar m a)
newTVarIO ControlMessage
Continue
      result <-
        race
          (do
            threadId <- myThreadId
            labelThread threadId "control-message"
            let terminateDelay =
                  Int -> DiffTime
forall a b. (Real a, Fractional b) => a -> b
realToFrac (Chain Block -> Int
forall block. Chain block -> Int
Chain.length Chain Block
forkChain) DiffTime -> DiffTime -> DiffTime
forall a. Num a => a -> a -> a
* DiffTime
delay DiffTime -> DiffTime -> DiffTime
forall a. Fractional a => a -> a -> a
/ DiffTime
2
            threadDelay terminateDelay
            atomically (writeTVar controlMessageVar Terminate)
            let awaitDelay = DiffTime
delay DiffTime -> DiffTime -> DiffTime
forall a. Num a => a -> a -> a
* DiffTime
100 DiffTime -> DiffTime -> DiffTime
forall a. Num a => a -> a -> a
+ DiffTime
500
            threadDelay awaitDelay)
          (do
            threadId <- myThreadId
            labelThread threadId "block-fetch"
            blockFetchExample0
              (contramap TraceFetchDecision       dynamicTracer)
              (contramap TraceFetchClientState    dynamicTracer)
              (contramap TraceFetchClientSendRecv dynamicTracer)
              (Just delay) (Just delay)
              (readTVar controlMessageVar)
              (AnchoredFragment.Empty AnchoredFragment.AnchorGenesis)
              fork')
      -- `IOSim` on `Windows` is using `defaultRegisterTimeout`.  It does not
      -- cancel forked threads.   The timeout which leaves running thread comes
      -- from 'runPipelinedPeerWithLimits'.
      -- threadDelay 60
      return $ case result of
        Left ()
_  -> Bool
False
        Right ()
_ -> Bool
True

    fork' :: AnchoredFragment Block
fork'  = Chain Block -> AnchoredFragment Block
chainToAnchoredFragment Chain Block
forkChain

-- TODO: moved to some shared place (cannot be moved to
-- `ouroboros-network-testing` which doesn't depend on `ouroboros-network`)
newtype PeerGSVT = PeerGSVT {
      PeerGSVT -> PeerGSV
unPeerGSVT :: PeerGSV
    } deriving Int -> PeerGSVT -> [Char] -> [Char]
[PeerGSVT] -> [Char] -> [Char]
PeerGSVT -> [Char]
(Int -> PeerGSVT -> [Char] -> [Char])
-> (PeerGSVT -> [Char])
-> ([PeerGSVT] -> [Char] -> [Char])
-> Show PeerGSVT
forall a.
(Int -> a -> [Char] -> [Char])
-> (a -> [Char]) -> ([a] -> [Char] -> [Char]) -> Show a
$cshowsPrec :: Int -> PeerGSVT -> [Char] -> [Char]
showsPrec :: Int -> PeerGSVT -> [Char] -> [Char]
$cshow :: PeerGSVT -> [Char]
show :: PeerGSVT -> [Char]
$cshowList :: [PeerGSVT] -> [Char] -> [Char]
showList :: [PeerGSVT] -> [Char] -> [Char]
Show

instance Arbitrary PeerGSVT where
    arbitrary :: Gen PeerGSVT
arbitrary = do
        Delay gIn <- Int -> Gen Delay -> Gen Delay
forall a. Int -> Gen a -> Gen a
resize Int
1000 Gen Delay
forall a. Arbitrary a => Gen a
arbitrary
        Delay gOut <- resize 1000 arbitrary
        let gsvIn  = DiffTime -> DiffTime -> Distribution DiffTime -> GSV
ballisticGSV DiffTime
gIn  DiffTime
2e-6 (DiffTime -> Distribution DiffTime
forall n. n -> Distribution n
degenerateDistribution DiffTime
0)
            gsvOut = DiffTime -> DiffTime -> Distribution DiffTime -> GSV
ballisticGSV DiffTime
gOut DiffTime
2e-6 (DiffTime -> Distribution DiffTime
forall n. n -> Distribution n
degenerateDistribution DiffTime
0)
        return $ PeerGSVT $ PeerGSV (Time 0) gsvOut gsvIn

    shrink :: PeerGSVT -> [PeerGSVT]
shrink (PeerGSVT (PeerGSV Time
ts (GSV DiffTime
gOut SizeInBytes -> DiffTime
sOut Distribution DiffTime
vOut) (GSV DiffTime
gIn SizeInBytes -> DiffTime
sIn Distribution DiffTime
vIn))) =
        [PeerGSV -> PeerGSVT
PeerGSVT (Time -> GSV -> GSV -> PeerGSV
PeerGSV Time
ts  (DiffTime
-> (SizeInBytes -> DiffTime) -> Distribution DiffTime -> GSV
GSV DiffTime
gOut' SizeInBytes -> DiffTime
sOut Distribution DiffTime
vOut) (DiffTime
-> (SizeInBytes -> DiffTime) -> Distribution DiffTime -> GSV
GSV DiffTime
gIn' SizeInBytes -> DiffTime
sIn Distribution DiffTime
vIn))
         | (Delay DiffTime
gIn', Delay DiffTime
gOut') <- (Delay, Delay) -> [(Delay, Delay)]
forall a. Arbitrary a => a -> [a]
shrink (DiffTime -> Delay
Delay DiffTime
gIn, DiffTime -> Delay
Delay DiffTime
gOut)]


-- | Check that comparePeerGSV satisfies Ord axioms
prop_comparePeerGSV :: Int -> Int -> Int -> PeerGSVT -> PeerGSVT -> Bool -> Bool -> Property
prop_comparePeerGSV :: Int
-> Int -> Int -> PeerGSVT -> PeerGSVT -> Bool -> Bool -> Property
prop_comparePeerGSV Int
salt Int
pa Int
pb (PeerGSVT PeerGSV
a) (PeerGSVT PeerGSV
b) Bool
aActive Bool
bActive =
    let peerSet :: Set Int
peerSet = case (Bool
aActive, Bool
bActive) of
                       (Bool
False, Bool
False) -> Set Int
forall a. Set a
Set.empty
                       (Bool
True, Bool
False)  -> Int -> Set Int
forall a. a -> Set a
Set.singleton Int
pa
                       (Bool
False, Bool
True)  -> Int -> Set Int
forall a. a -> Set a
Set.singleton Int
pb
                       (Bool
True, Bool
True)   -> [Int] -> Set Int
forall a. Ord a => [a] -> Set a
Set.fromList [Int
pa, Int
pb] in
    case Set Int -> Int -> (PeerGSV, Int) -> (PeerGSV, Int) -> Ordering
forall peer.
(Hashable peer, Ord peer) =>
Set peer -> Int -> (PeerGSV, peer) -> (PeerGSV, peer) -> Ordering
comparePeerGSV Set Int
peerSet Int
salt (PeerGSV
a, Int
pa) (PeerGSV
b, Int
pb) of
         Ordering
LT -> Set Int -> Int -> (PeerGSV, Int) -> (PeerGSV, Int) -> Ordering
forall peer.
(Hashable peer, Ord peer) =>
Set peer -> Int -> (PeerGSV, peer) -> (PeerGSV, peer) -> Ordering
comparePeerGSV Set Int
peerSet Int
salt (PeerGSV
b, Int
pb) (PeerGSV
a, Int
pa) Ordering -> Ordering -> Property
forall a. (Eq a, Show a) => a -> a -> Property
=== Ordering
GT
         Ordering
GT -> Set Int -> Int -> (PeerGSV, Int) -> (PeerGSV, Int) -> Ordering
forall peer.
(Hashable peer, Ord peer) =>
Set peer -> Int -> (PeerGSV, peer) -> (PeerGSV, peer) -> Ordering
comparePeerGSV Set Int
peerSet Int
salt (PeerGSV
b, Int
pb) (PeerGSV
a, Int
pa) Ordering -> Ordering -> Property
forall a. (Eq a, Show a) => a -> a -> Property
=== Ordering
LT
         Ordering
EQ -> Set Int -> Int -> (PeerGSV, Int) -> (PeerGSV, Int) -> Ordering
forall peer.
(Hashable peer, Ord peer) =>
Set peer -> Int -> (PeerGSV, peer) -> (PeerGSV, peer) -> Ordering
comparePeerGSV Set Int
peerSet Int
salt (PeerGSV
b, Int
pb) (PeerGSV
a, Int
pa) Ordering -> Ordering -> Property
forall a. (Eq a, Show a) => a -> a -> Property
=== Ordering
EQ

-- | Check that identical peers are equal
prop_comparePeerGSVEq :: Int -> Int -> PeerGSVT -> Bool -> Property
prop_comparePeerGSVEq :: Int -> Int -> PeerGSVT -> Bool -> Property
prop_comparePeerGSVEq Int
salt Int
p (PeerGSVT PeerGSV
a) Bool
aActive =
    let peerSet :: Set Int
peerSet = if Bool
aActive then Int -> Set Int
forall a. a -> Set a
Set.singleton Int
p
                             else Set Int
forall a. Set a
Set.empty in
    Set Int -> Int -> (PeerGSV, Int) -> (PeerGSV, Int) -> Ordering
forall peer.
(Hashable peer, Ord peer) =>
Set peer -> Int -> (PeerGSV, peer) -> (PeerGSV, peer) -> Ordering
comparePeerGSV Set Int
peerSet Int
salt (PeerGSV
a, Int
p) (PeerGSV
a, Int
p) Ordering -> Ordering -> Property
forall a. (Eq a, Show a) => a -> a -> Property
=== Ordering
EQ


--
-- Trace utils
--

dynamicTracer :: Typeable a => Tracer (IOSim s) a
dynamicTracer :: forall a s. Typeable a => Tracer (IOSim s) a
dynamicTracer = (a -> IOSim s ()) -> Tracer (IOSim s) a
forall (m :: * -> *) a. (a -> m ()) -> Tracer m a
Tracer a -> IOSim s ()
forall a s. Typeable a => a -> IOSim s ()
traceM