{-# LANGUAGE CPP                        #-}
{-# LANGUAGE DerivingStrategies         #-}
{-# LANGUAGE GeneralisedNewtypeDeriving #-}
{-# LANGUAGE LambdaCase                 #-}
{-# LANGUAGE NamedFieldPuns             #-}
{-# LANGUAGE NumericUnderscores         #-}
{-# LANGUAGE ScopedTypeVariables        #-}

#if __GLASGOW_HASKELL__ >= 908
{-# OPTIONS_GHC -Wno-x-partial #-}
#endif
module Test.Ouroboros.Network.PeerSelection.PeerMetric where


import Control.Concurrent.Class.MonadSTM qualified as LazySTM
import Control.Concurrent.Class.MonadSTM.Strict
import Control.DeepSeq (NFData (..))
import Control.Monad (when)
import Control.Monad.Class.MonadTime.SI
import Control.Monad.Class.MonadTimer.SI
import Control.Tracer (Tracer (..), traceWith)

import Data.Foldable as Foldable (foldl', foldr')
import Data.List (sortOn)
import Data.List.NonEmpty qualified as NonEmpty
import Data.Map.Merge.Strict qualified as Map
import Data.Map.Strict (Map)
import Data.Map.Strict qualified as Map
import Data.Set qualified as Set

import Network.Mux.Trace (TraceLabelPeer (..))

import Ouroboros.Network.ConnectionId
import Ouroboros.Network.PeerSelection.PeerMetric (PeerMetrics,
           PeerMetricsConfiguration (..), ReportPeerMetrics (..),
           fetchynessBlocks, fetchynessBytes, joinedPeerMetricAt, newPeerMetric,
           reportMetric, upstreamyness)
import Ouroboros.Network.SizeInBytes

import Cardano.Slotting.Slot (SlotNo (..))

import Control.Monad.IOSim

import NoThunks.Class

import Ouroboros.Network.Testing.Data.Script

import Test.QuickCheck
import Test.QuickCheck.Monoids
import Test.Tasty (TestTree, testGroup)
import Test.Tasty.QuickCheck (testProperty)


tests :: TestTree
tests :: TestTree
tests = String -> [TestTree] -> TestTree
testGroup String
"Ouroboros.Network.PeerSelection.PeerMetric"
  [ String -> (FixedScript -> Property) -> TestTree
forall a. Testable a => String -> a -> TestTree
testProperty String
"insert peer invariant"       FixedScript -> Property
prop_insert_peer
  , String -> (FixedScript -> Property) -> TestTree
forall a. Testable a => String -> a -> TestTree
testProperty String
"metrics results are bounded" FixedScript -> Property
prop_metrics_are_bounded
  , String -> (Positive Int -> FixedScript -> Property) -> TestTree
forall a. Testable a => String -> a -> TestTree
testProperty String
"size property"               Positive Int -> FixedScript -> Property
prop_bounded_size
  ]



newtype TestAddress = TestAddress Int
  deriving stock   (Int -> TestAddress -> ShowS
[TestAddress] -> ShowS
TestAddress -> String
(Int -> TestAddress -> ShowS)
-> (TestAddress -> String)
-> ([TestAddress] -> ShowS)
-> Show TestAddress
forall a.
(Int -> a -> ShowS) -> (a -> String) -> ([a] -> ShowS) -> Show a
$cshowsPrec :: Int -> TestAddress -> ShowS
showsPrec :: Int -> TestAddress -> ShowS
$cshow :: TestAddress -> String
show :: TestAddress -> String
$cshowList :: [TestAddress] -> ShowS
showList :: [TestAddress] -> ShowS
Show, TestAddress -> TestAddress -> Bool
(TestAddress -> TestAddress -> Bool)
-> (TestAddress -> TestAddress -> Bool) -> Eq TestAddress
forall a. (a -> a -> Bool) -> (a -> a -> Bool) -> Eq a
$c== :: TestAddress -> TestAddress -> Bool
== :: TestAddress -> TestAddress -> Bool
$c/= :: TestAddress -> TestAddress -> Bool
/= :: TestAddress -> TestAddress -> Bool
Eq, Eq TestAddress
Eq TestAddress =>
(TestAddress -> TestAddress -> Ordering)
-> (TestAddress -> TestAddress -> Bool)
-> (TestAddress -> TestAddress -> Bool)
-> (TestAddress -> TestAddress -> Bool)
-> (TestAddress -> TestAddress -> Bool)
-> (TestAddress -> TestAddress -> TestAddress)
-> (TestAddress -> TestAddress -> TestAddress)
-> Ord TestAddress
TestAddress -> TestAddress -> Bool
TestAddress -> TestAddress -> Ordering
TestAddress -> TestAddress -> TestAddress
forall a.
Eq a =>
(a -> a -> Ordering)
-> (a -> a -> Bool)
-> (a -> a -> Bool)
-> (a -> a -> Bool)
-> (a -> a -> Bool)
-> (a -> a -> a)
-> (a -> a -> a)
-> Ord a
$ccompare :: TestAddress -> TestAddress -> Ordering
compare :: TestAddress -> TestAddress -> Ordering
$c< :: TestAddress -> TestAddress -> Bool
< :: TestAddress -> TestAddress -> Bool
$c<= :: TestAddress -> TestAddress -> Bool
<= :: TestAddress -> TestAddress -> Bool
$c> :: TestAddress -> TestAddress -> Bool
> :: TestAddress -> TestAddress -> Bool
$c>= :: TestAddress -> TestAddress -> Bool
>= :: TestAddress -> TestAddress -> Bool
$cmax :: TestAddress -> TestAddress -> TestAddress
max :: TestAddress -> TestAddress -> TestAddress
$cmin :: TestAddress -> TestAddress -> TestAddress
min :: TestAddress -> TestAddress -> TestAddress
Ord)
  deriving newtype (Context -> TestAddress -> IO (Maybe ThunkInfo)
Proxy TestAddress -> String
(Context -> TestAddress -> IO (Maybe ThunkInfo))
-> (Context -> TestAddress -> IO (Maybe ThunkInfo))
-> (Proxy TestAddress -> String)
-> NoThunks TestAddress
forall a.
(Context -> a -> IO (Maybe ThunkInfo))
-> (Context -> a -> IO (Maybe ThunkInfo))
-> (Proxy a -> String)
-> NoThunks a
$cnoThunks :: Context -> TestAddress -> IO (Maybe ThunkInfo)
noThunks :: Context -> TestAddress -> IO (Maybe ThunkInfo)
$cwNoThunks :: Context -> TestAddress -> IO (Maybe ThunkInfo)
wNoThunks :: Context -> TestAddress -> IO (Maybe ThunkInfo)
$cshowTypeOf :: Proxy TestAddress -> String
showTypeOf :: Proxy TestAddress -> String
NoThunks, TestAddress -> ()
(TestAddress -> ()) -> NFData TestAddress
forall a. (a -> ()) -> NFData a
$crnf :: TestAddress -> ()
rnf :: TestAddress -> ()
NFData)

instance Arbitrary TestAddress where
    arbitrary :: Gen TestAddress
arbitrary = do
      size <- (Int, Int) -> Gen Int
forall a. Random a => (a, a) -> Gen a
choose (Int
0, Int
20)
      TestAddress . getPositive <$> resize size arbitrary
    shrink :: TestAddress -> [TestAddress]
shrink (TestAddress Int
addr) =
      Int -> TestAddress
TestAddress (Int -> TestAddress)
-> (Positive Int -> Int) -> Positive Int -> TestAddress
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Positive Int -> Int
forall a. Positive a -> a
getPositive (Positive Int -> TestAddress) -> [Positive Int] -> [TestAddress]
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Positive Int -> [Positive Int]
forall a. Arbitrary a => a -> [a]
shrink (Int -> Positive Int
forall a. a -> Positive a
Positive Int
addr)

data Event =
    FetchedHeader TestAddress SlotNo
  | FetchedBlock  TestAddress SlotNo SizeInBytes
  deriving Int -> Event -> ShowS
[Event] -> ShowS
Event -> String
(Int -> Event -> ShowS)
-> (Event -> String) -> ([Event] -> ShowS) -> Show Event
forall a.
(Int -> a -> ShowS) -> (a -> String) -> ([a] -> ShowS) -> Show a
$cshowsPrec :: Int -> Event -> ShowS
showsPrec :: Int -> Event -> ShowS
$cshow :: Event -> String
show :: Event -> String
$cshowList :: [Event] -> ShowS
showList :: [Event] -> ShowS
Show

eventPeer :: Event -> TestAddress
eventPeer :: Event -> TestAddress
eventPeer (FetchedHeader TestAddress
peer SlotNo
_)   = TestAddress
peer
eventPeer (FetchedBlock  TestAddress
peer SlotNo
_ SizeInBytes
_) = TestAddress
peer

eventSlot :: Event -> SlotNo
eventSlot :: Event -> SlotNo
eventSlot (FetchedHeader TestAddress
_ SlotNo
slotNo)   = SlotNo
slotNo
eventSlot (FetchedBlock  TestAddress
_ SlotNo
slotNo SizeInBytes
_) = SlotNo
slotNo

instance Arbitrary Event where
    arbitrary :: Gen Event
arbitrary = [Gen Event] -> Gen Event
forall a. [Gen a] -> Gen a
oneof [ TestAddress -> SlotNo -> Event
FetchedHeader (TestAddress -> SlotNo -> Event)
-> Gen TestAddress -> Gen (SlotNo -> Event)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Gen TestAddress
forall a. Arbitrary a => Gen a
arbitrary
                                      Gen (SlotNo -> Event) -> Gen SlotNo -> Gen Event
forall a b. Gen (a -> b) -> Gen a -> Gen b
forall (f :: * -> *) a b. Applicative f => f (a -> b) -> f a -> f b
<*> (Word64 -> SlotNo
SlotNo (Word64 -> SlotNo)
-> (Positive (Small Word64) -> Word64)
-> Positive (Small Word64)
-> SlotNo
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Small Word64 -> Word64
forall a. Small a -> a
getSmall (Small Word64 -> Word64)
-> (Positive (Small Word64) -> Small Word64)
-> Positive (Small Word64)
-> Word64
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Positive (Small Word64) -> Small Word64
forall a. Positive a -> a
getPositive (Positive (Small Word64) -> SlotNo)
-> Gen (Positive (Small Word64)) -> Gen SlotNo
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Gen (Positive (Small Word64))
forall a. Arbitrary a => Gen a
arbitrary)
                      , TestAddress -> SlotNo -> SizeInBytes -> Event
FetchedBlock  (TestAddress -> SlotNo -> SizeInBytes -> Event)
-> Gen TestAddress -> Gen (SlotNo -> SizeInBytes -> Event)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Gen TestAddress
forall a. Arbitrary a => Gen a
arbitrary
                                      Gen (SlotNo -> SizeInBytes -> Event)
-> Gen SlotNo -> Gen (SizeInBytes -> Event)
forall a b. Gen (a -> b) -> Gen a -> Gen b
forall (f :: * -> *) a b. Applicative f => f (a -> b) -> f a -> f b
<*> (Word64 -> SlotNo
SlotNo (Word64 -> SlotNo)
-> (Positive (Small Word64) -> Word64)
-> Positive (Small Word64)
-> SlotNo
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Small Word64 -> Word64
forall a. Small a -> a
getSmall (Small Word64 -> Word64)
-> (Positive (Small Word64) -> Small Word64)
-> Positive (Small Word64)
-> Word64
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Positive (Small Word64) -> Small Word64
forall a. Positive a -> a
getPositive (Positive (Small Word64) -> SlotNo)
-> Gen (Positive (Small Word64)) -> Gen SlotNo
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Gen (Positive (Small Word64))
forall a. Arbitrary a => Gen a
arbitrary)
                                      Gen (SizeInBytes -> Event) -> Gen SizeInBytes -> Gen Event
forall a b. Gen (a -> b) -> Gen a -> Gen b
forall (f :: * -> *) a b. Applicative f => f (a -> b) -> f a -> f b
<*> (Word32 -> SizeInBytes
SizeInBytes (Word32 -> SizeInBytes) -> Gen Word32 -> Gen SizeInBytes
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> (Gen Word32
forall a. Arbitrary a => Gen a
arbitrary Gen Word32 -> (Word32 -> Bool) -> Gen Word32
forall a. Gen a -> (a -> Bool) -> Gen a
`suchThat` \Word32
sizeInBytes -> Word32
0 Word32 -> Word32 -> Bool
forall a. Ord a => a -> a -> Bool
< Word32
sizeInBytes Bool -> Bool -> Bool
&& Word32
sizeInBytes Word32 -> Word32 -> Bool
forall a. Ord a => a -> a -> Bool
<= Word32
2_000_000))
                      ]
    shrink :: Event -> [Event]
shrink  FetchedHeader {} = []
    shrink (FetchedBlock TestAddress
peer SlotNo
slotNo SizeInBytes
size) =
      [ TestAddress -> SlotNo -> SizeInBytes -> Event
FetchedBlock TestAddress
peer SlotNo
slotNo (Word32 -> SizeInBytes
SizeInBytes Word32
size')
      | Word32
size' <- Word32 -> [Word32]
forall a. Arbitrary a => a -> [a]
shrink (SizeInBytes -> Word32
getSizeInBytes SizeInBytes
size)
      , Word32
size' Word32 -> Word32 -> Bool
forall a. Ord a => a -> a -> Bool
> Word32
0
      ]


newtype FixedScript = FixedScript { FixedScript -> Script Event
getFixedScript :: Script Event }
  deriving Int -> FixedScript -> ShowS
[FixedScript] -> ShowS
FixedScript -> String
(Int -> FixedScript -> ShowS)
-> (FixedScript -> String)
-> ([FixedScript] -> ShowS)
-> Show FixedScript
forall a.
(Int -> a -> ShowS) -> (a -> String) -> ([a] -> ShowS) -> Show a
$cshowsPrec :: Int -> FixedScript -> ShowS
showsPrec :: Int -> FixedScript -> ShowS
$cshow :: FixedScript -> String
show :: FixedScript -> String
$cshowList :: [FixedScript] -> ShowS
showList :: [FixedScript] -> ShowS
Show

-- | Order events by 'SlotNo'
--
-- TODO: 'SizeInBytes' should be a function of 'SlotNo'
--
mkFixedScript :: Script Event -> FixedScript
mkFixedScript :: Script Event -> FixedScript
mkFixedScript (Script NonEmpty Event
events) = Script Event -> FixedScript
FixedScript
                              (Script Event -> FixedScript)
-> (NonEmpty Event -> Script Event)
-> NonEmpty Event
-> FixedScript
forall b c a. (b -> c) -> (a -> b) -> a -> c
. NonEmpty Event -> Script Event
forall a. NonEmpty a -> Script a
Script
                              (NonEmpty Event -> FixedScript) -> NonEmpty Event -> FixedScript
forall a b. (a -> b) -> a -> b
$ (Event -> SlotNo) -> NonEmpty Event -> NonEmpty Event
forall o a. Ord o => (a -> o) -> NonEmpty a -> NonEmpty a
NonEmpty.sortWith
                                  Event -> SlotNo
eventSlot
                                  NonEmpty Event
events

instance Arbitrary FixedScript where
    -- Generated scripts must be long enough. We ignore first 100 results, to
    -- avoid effects when the peer metrics has not enough data  and thus it is
    -- ignoring averages: 'Ouroboros.Network.PeerSelection.PeerMetric.adjustAvg'.
    arbitrary :: Gen FixedScript
arbitrary = Script Event -> FixedScript
mkFixedScript
            (Script Event -> FixedScript)
-> Gen (Script Event) -> Gen FixedScript
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Int -> Gen (Script Event) -> Gen (Script Event)
forall a. Int -> Gen a -> Gen a
resize Int
360 Gen (Script Event)
forall a. Arbitrary a => Gen a
arbitrary
                Gen (Script Event) -> (Script Event -> Bool) -> Gen (Script Event)
forall a. Gen a -> (a -> Bool) -> Gen a
`suchThat` \(Script NonEmpty Event
as) -> NonEmpty Event -> Int
forall a. NonEmpty a -> Int
NonEmpty.length NonEmpty Event
as Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
> Int
100
    shrink :: FixedScript -> [FixedScript]
shrink (FixedScript Script Event
script) = Script Event -> FixedScript
mkFixedScript (Script Event -> FixedScript) -> [Script Event] -> [FixedScript]
forall a b. (a -> b) -> [a] -> [b]
`map` Script Event -> [Script Event]
forall a. Arbitrary a => a -> [a]
shrink Script Event
script


mkTimedScript :: FixedScript -> TimedScript Event
mkTimedScript :: FixedScript -> TimedScript Event
mkTimedScript = Script (Event, SlotNo) -> TimedScript Event
go (Script (Event, SlotNo) -> TimedScript Event)
-> (FixedScript -> Script (Event, SlotNo))
-> FixedScript
-> TimedScript Event
forall b c a. (b -> c) -> (a -> b) -> a -> c
. (Event -> (Event, SlotNo))
-> Script Event -> Script (Event, SlotNo)
forall a b. (a -> b) -> Script a -> Script b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap (\Event
a -> (Event
a, Event -> SlotNo
eventSlot Event
a)) (Script Event -> Script (Event, SlotNo))
-> (FixedScript -> Script Event)
-> FixedScript
-> Script (Event, SlotNo)
forall b c a. (b -> c) -> (a -> b) -> a -> c
. FixedScript -> Script Event
getFixedScript
  where
    go :: Script (Event, SlotNo) -> TimedScript Event
    go :: Script (Event, SlotNo) -> TimedScript Event
go (Script NonEmpty (Event, SlotNo)
script) = NonEmpty (Event, ScriptDelay) -> TimedScript Event
forall a. NonEmpty a -> Script a
Script
                       (NonEmpty (Event, ScriptDelay) -> TimedScript Event)
-> ([((Event, SlotNo), Maybe SlotNo)]
    -> NonEmpty (Event, ScriptDelay))
-> [((Event, SlotNo), Maybe SlotNo)]
-> TimedScript Event
forall b c a. (b -> c) -> (a -> b) -> a -> c
. [(Event, ScriptDelay)] -> NonEmpty (Event, ScriptDelay)
forall a. HasCallStack => [a] -> NonEmpty a
NonEmpty.fromList
                       ([(Event, ScriptDelay)] -> NonEmpty (Event, ScriptDelay))
-> ([((Event, SlotNo), Maybe SlotNo)] -> [(Event, ScriptDelay)])
-> [((Event, SlotNo), Maybe SlotNo)]
-> NonEmpty (Event, ScriptDelay)
forall b c a. (b -> c) -> (a -> b) -> a -> c
. (((Event, SlotNo), Maybe SlotNo)
 -> [(Event, ScriptDelay)] -> [(Event, ScriptDelay)])
-> [(Event, ScriptDelay)]
-> [((Event, SlotNo), Maybe SlotNo)]
-> [(Event, ScriptDelay)]
forall a b. (a -> b -> b) -> b -> [a] -> b
forall (t :: * -> *) a b.
Foldable t =>
(a -> b -> b) -> b -> t a -> b
foldr' ((Event, SlotNo), Maybe SlotNo)
-> [(Event, ScriptDelay)] -> [(Event, ScriptDelay)]
f []
                       ([((Event, SlotNo), Maybe SlotNo)] -> TimedScript Event)
-> [((Event, SlotNo), Maybe SlotNo)] -> TimedScript Event
forall a b. (a -> b) -> a -> b
$ [(Event, SlotNo)]
-> [Maybe SlotNo] -> [((Event, SlotNo), Maybe SlotNo)]
forall a b. [a] -> [b] -> [(a, b)]
zip [(Event, SlotNo)]
events ((SlotNo -> Maybe SlotNo
forall a. a -> Maybe a
Just (SlotNo -> Maybe SlotNo)
-> ((Event, SlotNo) -> SlotNo) -> (Event, SlotNo) -> Maybe SlotNo
forall b c a. (b -> c) -> (a -> b) -> a -> c
. (Event, SlotNo) -> SlotNo
forall a b. (a, b) -> b
snd) ((Event, SlotNo) -> Maybe SlotNo)
-> [(Event, SlotNo)] -> [Maybe SlotNo]
forall a b. (a -> b) -> [a] -> [b]
`map` [(Event, SlotNo)] -> [(Event, SlotNo)]
forall a. HasCallStack => [a] -> [a]
tail [(Event, SlotNo)]
events [Maybe SlotNo] -> [Maybe SlotNo] -> [Maybe SlotNo]
forall a. [a] -> [a] -> [a]
++ [Maybe SlotNo
forall a. Maybe a
Nothing])
      where
        events :: [(Event, SlotNo)]
events = NonEmpty (Event, SlotNo) -> [(Event, SlotNo)]
forall a. NonEmpty a -> [a]
NonEmpty.toList NonEmpty (Event, SlotNo)
script

    f :: ((Event, SlotNo), Maybe SlotNo)
      -> [(Event, ScriptDelay)]
      -> [(Event, ScriptDelay)]
    f :: ((Event, SlotNo), Maybe SlotNo)
-> [(Event, ScriptDelay)] -> [(Event, ScriptDelay)]
f ((Event
event, SlotNo
slotNo), Maybe SlotNo
nextSlotNo) [(Event, ScriptDelay)]
as =
      (Event
event, DiffTime -> ScriptDelay
Delay (DiffTime -> ScriptDelay) -> DiffTime -> ScriptDelay
forall a b. (a -> b) -> a -> b
$ SlotNo -> Maybe SlotNo -> DiffTime
slotDiffTime SlotNo
slotNo Maybe SlotNo
nextSlotNo) (Event, ScriptDelay)
-> [(Event, ScriptDelay)] -> [(Event, ScriptDelay)]
forall a. a -> [a] -> [a]
: [(Event, ScriptDelay)]
as

    slotToTime :: SlotNo -> Time
    slotToTime :: SlotNo -> Time
slotToTime (SlotNo Word64
slotNo) = DiffTime -> Time
Time (DiffTime -> Time) -> DiffTime -> Time
forall a b. (a -> b) -> a -> b
$ Word64 -> DiffTime
forall a b. (Real a, Fractional b) => a -> b
realToFrac Word64
slotNo -- each slot takes 1s

    slotDiffTime :: SlotNo -> Maybe SlotNo -> DiffTime
    slotDiffTime :: SlotNo -> Maybe SlotNo -> DiffTime
slotDiffTime SlotNo
_slotNo Maybe SlotNo
Nothing           = DiffTime
0
    slotDiffTime  SlotNo
slotNo (Just SlotNo
nextSlotNo) = SlotNo -> Time
slotToTime SlotNo
nextSlotNo
                                  Time -> Time -> DiffTime
`diffTime` SlotNo -> Time
slotToTime SlotNo
slotNo


data PeerMetricsTrace = PeerMetricsTrace {
      PeerMetricsTrace -> TestAddress
pmtPeer             :: TestAddress,
      PeerMetricsTrace -> SlotNo
pmtSlot             :: SlotNo,
      PeerMetricsTrace -> Map TestAddress Int
pmtUpstreamyness    :: Map TestAddress Int,
      PeerMetricsTrace -> Map TestAddress Int
pmtFetchynessBytes  :: Map TestAddress Int,
      PeerMetricsTrace -> Map TestAddress Int
pmtFetchynessBlocks :: Map TestAddress Int,
      PeerMetricsTrace -> Map TestAddress SlotNo
pmtJoinedAt         :: Map TestAddress SlotNo
    }
  deriving Int -> PeerMetricsTrace -> ShowS
[PeerMetricsTrace] -> ShowS
PeerMetricsTrace -> String
(Int -> PeerMetricsTrace -> ShowS)
-> (PeerMetricsTrace -> String)
-> ([PeerMetricsTrace] -> ShowS)
-> Show PeerMetricsTrace
forall a.
(Int -> a -> ShowS) -> (a -> String) -> ([a] -> ShowS) -> Show a
$cshowsPrec :: Int -> PeerMetricsTrace -> ShowS
showsPrec :: Int -> PeerMetricsTrace -> ShowS
$cshow :: PeerMetricsTrace -> String
show :: PeerMetricsTrace -> String
$cshowList :: [PeerMetricsTrace] -> ShowS
showList :: [PeerMetricsTrace] -> ShowS
Show

simulatePeerMetricScript
  :: forall m.
     ( MonadDelay m
     , MonadTimer m
     , MonadMonotonicTime m
     , MonadLabelledSTM m
     , MonadTraceSTM m
     )
  => Tracer m PeerMetricsTrace
  -> PeerMetricsConfiguration
  -> FixedScript
  -> m ()
simulatePeerMetricScript :: forall (m :: * -> *).
(MonadDelay m, MonadTimer m, MonadMonotonicTime m,
 MonadLabelledSTM m, MonadTraceSTM m) =>
Tracer m PeerMetricsTrace
-> PeerMetricsConfiguration -> FixedScript -> m ()
simulatePeerMetricScript Tracer m PeerMetricsTrace
tracer PeerMetricsConfiguration
config FixedScript
script = do
      peerMetrics <- PeerMetricsConfiguration -> m (PeerMetrics m TestAddress)
forall (m :: * -> *) p.
(MonadLabelledSTM m, NoThunks p, NFData p) =>
PeerMetricsConfiguration -> m (PeerMetrics m p)
newPeerMetric PeerMetricsConfiguration
config
      let reporter :: ReportPeerMetrics m (ConnectionId TestAddress)
          reporter = PeerMetricsConfiguration
-> PeerMetrics m TestAddress
-> ReportPeerMetrics m (ConnectionId TestAddress)
forall (m :: * -> *) p.
(MonadSTM m, Ord p) =>
PeerMetricsConfiguration
-> PeerMetrics m p -> ReportPeerMetrics m (ConnectionId p)
reportMetric PeerMetricsConfiguration
config PeerMetrics m TestAddress
peerMetrics
      v <- initScript timedScript
      go v peerMetrics reporter
    where
      timedScript ::  TimedScript Event
      timedScript :: TimedScript Event
timedScript = FixedScript -> TimedScript Event
mkTimedScript FixedScript
script

      go :: LazySTM.TVar m (TimedScript Event)
         -> PeerMetrics m TestAddress
         -> ReportPeerMetrics m (ConnectionId TestAddress)
         -> m ()
      go :: TVar m (TimedScript Event)
-> PeerMetrics m TestAddress
-> ReportPeerMetrics m (ConnectionId TestAddress)
-> m ()
go TVar m (TimedScript Event)
v PeerMetrics m TestAddress
peerMetrics reporter :: ReportPeerMetrics m (ConnectionId TestAddress)
reporter@ReportPeerMetrics { Tracer
  (STM m) (TraceLabelPeer (ConnectionId TestAddress) (SlotNo, Time))
reportHeader :: Tracer
  (STM m) (TraceLabelPeer (ConnectionId TestAddress) (SlotNo, Time))
reportHeader :: forall (m :: * -> *) peerAddr.
ReportPeerMetrics m peerAddr
-> Tracer (STM m) (TraceLabelPeer peerAddr (SlotNo, Time))
reportHeader, Tracer
  (STM m)
  (TraceLabelPeer
     (ConnectionId TestAddress) (SizeInBytes, SlotNo, Time))
reportFetch :: Tracer
  (STM m)
  (TraceLabelPeer
     (ConnectionId TestAddress) (SizeInBytes, SlotNo, Time))
reportFetch :: forall (m :: * -> *) peerAddr.
ReportPeerMetrics m peerAddr
-> Tracer
     (STM m) (TraceLabelPeer peerAddr (SizeInBytes, SlotNo, Time))
reportFetch } = do
        (continue, (ev, delay)) <- (\case Left  (Event, ScriptDelay)
a -> (Bool
False, (Event, ScriptDelay)
a)
                                          Right (Event, ScriptDelay)
a -> (Bool
True,  (Event, ScriptDelay)
a))
                               (Either (Event, ScriptDelay) (Event, ScriptDelay)
 -> (Bool, (Event, ScriptDelay)))
-> m (Either (Event, ScriptDelay) (Event, ScriptDelay))
-> m (Bool, (Event, ScriptDelay))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> TVar m (TimedScript Event)
-> m (Either (Event, ScriptDelay) (Event, ScriptDelay))
forall (m :: * -> *) a.
MonadSTM m =>
TVar m (Script a) -> m (Either a a)
stepScriptOrFinish TVar m (TimedScript Event)
v
        time <- getMonotonicTime
        peer <- case ev of
          FetchedHeader TestAddress
peer SlotNo
slotNo -> do
            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
$ Tracer
  (STM m) (TraceLabelPeer (ConnectionId TestAddress) (SlotNo, Time))
-> TraceLabelPeer (ConnectionId TestAddress) (SlotNo, Time)
-> STM m ()
forall (m :: * -> *) a. Tracer m a -> a -> m ()
traceWith Tracer
  (STM m) (TraceLabelPeer (ConnectionId TestAddress) (SlotNo, Time))
reportHeader
                       (TraceLabelPeer (ConnectionId TestAddress) (SlotNo, Time)
 -> STM m ())
-> TraceLabelPeer (ConnectionId TestAddress) (SlotNo, Time)
-> STM m ()
forall a b. (a -> b) -> a -> b
$ ConnectionId TestAddress
-> (SlotNo, Time)
-> TraceLabelPeer (ConnectionId TestAddress) (SlotNo, Time)
forall peerid a. peerid -> a -> TraceLabelPeer peerid a
TraceLabelPeer ConnectionId {
                                            localAddress :: TestAddress
localAddress  = Int -> TestAddress
TestAddress Int
0,
                                            remoteAddress :: TestAddress
remoteAddress = TestAddress
peer
                                          }
                                        (SlotNo
slotNo, Time
time)
            TestAddress -> m TestAddress
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return TestAddress
peer

          FetchedBlock TestAddress
peer SlotNo
slotNo SizeInBytes
size -> do
            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
$ Tracer
  (STM m)
  (TraceLabelPeer
     (ConnectionId TestAddress) (SizeInBytes, SlotNo, Time))
-> TraceLabelPeer
     (ConnectionId TestAddress) (SizeInBytes, SlotNo, Time)
-> STM m ()
forall (m :: * -> *) a. Tracer m a -> a -> m ()
traceWith Tracer
  (STM m)
  (TraceLabelPeer
     (ConnectionId TestAddress) (SizeInBytes, SlotNo, Time))
reportFetch
                       (TraceLabelPeer
   (ConnectionId TestAddress) (SizeInBytes, SlotNo, Time)
 -> STM m ())
-> TraceLabelPeer
     (ConnectionId TestAddress) (SizeInBytes, SlotNo, Time)
-> STM m ()
forall a b. (a -> b) -> a -> b
$ ConnectionId TestAddress
-> (SizeInBytes, SlotNo, Time)
-> TraceLabelPeer
     (ConnectionId TestAddress) (SizeInBytes, SlotNo, Time)
forall peerid a. peerid -> a -> TraceLabelPeer peerid a
TraceLabelPeer ConnectionId {
                                            localAddress :: TestAddress
localAddress  = Int -> TestAddress
TestAddress Int
0,
                                            remoteAddress :: TestAddress
remoteAddress = TestAddress
peer
                                          }
                                        (SizeInBytes
size, SlotNo
slotNo, Time
time)
            TestAddress -> m TestAddress
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return TestAddress
peer

        trace <- atomically $
           PeerMetricsTrace peer (eventSlot ev)
                 <$> upstreamyness      peerMetrics
                 <*> fetchynessBytes    peerMetrics
                 <*> fetchynessBlocks   peerMetrics
                 <*> joinedPeerMetricAt peerMetrics
        traceWith tracer trace

        when continue $ do
          threadDelay (interpretScriptDelay delay)
          go v peerMetrics reporter

interpretScriptDelay :: ScriptDelay -> DiffTime
interpretScriptDelay :: ScriptDelay -> DiffTime
interpretScriptDelay ScriptDelay
NoDelay       = DiffTime
0
interpretScriptDelay ScriptDelay
ShortDelay    = DiffTime
1
interpretScriptDelay ScriptDelay
LongDelay     = DiffTime
3600
interpretScriptDelay (Delay DiffTime
delay) = DiffTime
delay


-- | Check that newly added peer is never in the 20% worst performing peers (if
-- there are at least 5 results).
--
prop_insert_peer :: FixedScript -> Property
prop_insert_peer :: FixedScript -> Property
prop_insert_peer FixedScript
script =
    String -> Property -> Property
forall prop. Testable prop => String -> prop -> Property
label (String
"length: "
           String -> ShowS
forall a. [a] -> [a] -> [a]
++ (Int, Int) -> String
forall a. Show a => a -> String
show (  Int
len Int -> Int -> Int
forall a. Integral a => a -> a -> a
`div` Int
band      Int -> Int -> Int
forall a. Num a => a -> a -> a
* Int
band
                   , (Int
len Int -> Int -> Int
forall a. Integral a => a -> a -> a
`div` Int
band Int -> Int -> Int
forall a. Num a => a -> a -> a
+ Int
1) Int -> Int -> Int
forall a. Num a => a -> a -> a
* Int
band  Int -> Int -> Int
forall a. Num a => a -> a -> a
- Int
1
                   )) (Property -> Property) -> Property -> Property
forall a b. (a -> b) -> a -> b
$
    String -> All -> Property
forall prop. Testable prop => String -> prop -> Property
label (case [PeerMetricsTrace]
trace of
            [] -> String
"empty"
            [PeerMetricsTrace]
_  -> String
"non-empty") (All -> Property) -> All -> Property
forall a b. (a -> b) -> a -> b
$
               ((Maybe PeerMetricsTrace, PeerMetricsTrace) -> All)
-> [(Maybe PeerMetricsTrace, PeerMetricsTrace)] -> All
forall m a. Monoid m => (a -> m) -> [a] -> m
forall (t :: * -> *) m a.
(Foldable t, Monoid m) =>
(a -> m) -> t a -> m
foldMap (Maybe PeerMetricsTrace, PeerMetricsTrace) -> All
go
             ([(Maybe PeerMetricsTrace, PeerMetricsTrace)] -> All)
-> [(Maybe PeerMetricsTrace, PeerMetricsTrace)] -> All
forall a b. (a -> b) -> a -> b
$ [Maybe PeerMetricsTrace]
-> [PeerMetricsTrace]
-> [(Maybe PeerMetricsTrace, PeerMetricsTrace)]
forall a b. [a] -> [b] -> [(a, b)]
zip (Maybe PeerMetricsTrace
forall a. Maybe a
Nothing Maybe PeerMetricsTrace
-> [Maybe PeerMetricsTrace] -> [Maybe PeerMetricsTrace]
forall a. a -> [a] -> [a]
: PeerMetricsTrace -> Maybe PeerMetricsTrace
forall a. a -> Maybe a
Just (PeerMetricsTrace -> Maybe PeerMetricsTrace)
-> [PeerMetricsTrace] -> [Maybe PeerMetricsTrace]
forall a b. (a -> b) -> [a] -> [b]
`map` [PeerMetricsTrace]
trace) [PeerMetricsTrace]
trace
  where
    band :: Int
band = Int
50
    len :: Int
len = case FixedScript -> Script Event
getFixedScript FixedScript
script of Script NonEmpty Event
as -> NonEmpty Event -> Int
forall a. NonEmpty a -> Int
NonEmpty.length NonEmpty Event
as

    config :: PeerMetricsConfiguration
    config :: PeerMetricsConfiguration
config = PeerMetricsConfiguration { maxEntriesToTrack :: Int
maxEntriesToTrack = Int
180 }

    sim :: IOSim s ()
    sim :: forall s. IOSim s ()
sim = Tracer (IOSim s) PeerMetricsTrace
-> PeerMetricsConfiguration -> FixedScript -> IOSim s ()
forall (m :: * -> *).
(MonadDelay m, MonadTimer m, MonadMonotonicTime m,
 MonadLabelledSTM m, MonadTraceSTM m) =>
Tracer m PeerMetricsTrace
-> PeerMetricsConfiguration -> FixedScript -> m ()
simulatePeerMetricScript ((PeerMetricsTrace -> IOSim s ())
-> Tracer (IOSim s) PeerMetricsTrace
forall (m :: * -> *) a. (a -> m ()) -> Tracer m a
Tracer PeerMetricsTrace -> IOSim s ()
forall a s. Typeable a => a -> IOSim s ()
traceM) PeerMetricsConfiguration
config FixedScript
script

    -- drop first 90 slots
    trace :: [PeerMetricsTrace]
    trace :: [PeerMetricsTrace]
trace = (PeerMetricsTrace -> Bool)
-> [PeerMetricsTrace] -> [PeerMetricsTrace]
forall a. (a -> Bool) -> [a] -> [a]
dropWhile (\PeerMetricsTrace
a -> PeerMetricsTrace -> SlotNo
pmtSlot PeerMetricsTrace
a SlotNo -> SlotNo -> Bool
forall a. Ord a => a -> a -> Bool
<= SlotNo
firstSlot SlotNo -> SlotNo -> SlotNo
forall a. Num a => a -> a -> a
+ SlotNo
90)
          ([PeerMetricsTrace] -> [PeerMetricsTrace])
-> [PeerMetricsTrace] -> [PeerMetricsTrace]
forall a b. (a -> b) -> a -> b
$ SimTrace () -> [PeerMetricsTrace]
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 ()
sim)
      where
        firstSlot :: SlotNo
firstSlot = case FixedScript
script of
            FixedScript (Script (Event
a :| [Event]
_)) -> Event -> SlotNo
eventSlot Event
a

    go :: (Maybe PeerMetricsTrace, PeerMetricsTrace)
       -> All
    go :: (Maybe PeerMetricsTrace, PeerMetricsTrace) -> All
go (Maybe PeerMetricsTrace
Nothing, PeerMetricsTrace
_) = Bool -> All
forall p. Testable p => p -> All
All Bool
True
    go (Just PeerMetricsTrace
prev, res :: PeerMetricsTrace
res@PeerMetricsTrace { pmtPeer :: PeerMetricsTrace -> TestAddress
pmtPeer             = TestAddress
peer,
                                          pmtUpstreamyness :: PeerMetricsTrace -> Map TestAddress Int
pmtUpstreamyness    = Map TestAddress Int
upstreamynessResults,
                                          pmtFetchynessBytes :: PeerMetricsTrace -> Map TestAddress Int
pmtFetchynessBytes  = Map TestAddress Int
fetchynessBytesResults,
                                          pmtFetchynessBlocks :: PeerMetricsTrace -> Map TestAddress Int
pmtFetchynessBlocks = Map TestAddress Int
fetchynessBlocksResults,
                                          pmtJoinedAt :: PeerMetricsTrace -> Map TestAddress SlotNo
pmtJoinedAt         = Map TestAddress SlotNo
joinedAtResults
                                        }) =
      if TestAddress
peer TestAddress -> Map TestAddress Int -> Bool
forall k a. Ord k => k -> Map k a -> Bool
`Map.member` PeerMetricsTrace -> Map TestAddress Int
pmtUpstreamyness PeerMetricsTrace
prev
      Bool -> Bool -> Bool
|| TestAddress
peer TestAddress -> Map TestAddress Int -> Bool
forall k a. Ord k => k -> Map k a -> Bool
`Map.member` PeerMetricsTrace -> Map TestAddress Int
pmtFetchynessBytes PeerMetricsTrace
prev
      Bool -> Bool -> Bool
|| TestAddress
peer TestAddress -> Map TestAddress Int -> Bool
forall k a. Ord k => k -> Map k a -> Bool
`Map.member` PeerMetricsTrace -> Map TestAddress Int
pmtFetchynessBlocks PeerMetricsTrace
prev
      then Bool -> All
forall p. Testable p => p -> All
All Bool
True
      else Property -> All
forall p. Testable p => p -> All
All ( String -> Property -> Property
forall prop. Testable prop => String -> prop -> Property
counterexample ((PeerMetricsTrace, PeerMetricsTrace) -> String
forall a. Show a => a -> String
show (PeerMetricsTrace
res, PeerMetricsTrace
prev))
               (Property -> Property) -> Property -> Property
forall a b. (a -> b) -> a -> b
$ String
-> TestAddress
-> Map TestAddress SlotNo
-> Map TestAddress Int
-> Property
checkResult String
"upstreamyness"    TestAddress
peer Map TestAddress SlotNo
joinedAtResults Map TestAddress Int
upstreamynessResults)
        All -> All -> All
forall a. Semigroup a => a -> a -> a
<> Property -> All
forall p. Testable p => p -> All
All ( String -> Property -> Property
forall prop. Testable prop => String -> prop -> Property
counterexample ((PeerMetricsTrace, PeerMetricsTrace) -> String
forall a. Show a => a -> String
show (PeerMetricsTrace
res ,PeerMetricsTrace
prev))
               (Property -> Property) -> Property -> Property
forall a b. (a -> b) -> a -> b
$ String
-> TestAddress
-> Map TestAddress SlotNo
-> Map TestAddress Int
-> Property
checkResult String
"fetchynessBytes"  TestAddress
peer Map TestAddress SlotNo
joinedAtResults Map TestAddress Int
fetchynessBytesResults)
        All -> All -> All
forall a. Semigroup a => a -> a -> a
<> Property -> All
forall p. Testable p => p -> All
All ( String -> Property -> Property
forall prop. Testable prop => String -> prop -> Property
counterexample ((PeerMetricsTrace, PeerMetricsTrace) -> String
forall a. Show a => a -> String
show (PeerMetricsTrace
res, PeerMetricsTrace
prev))
               (Property -> Property) -> Property -> Property
forall a b. (a -> b) -> a -> b
$ String
-> TestAddress
-> Map TestAddress SlotNo
-> Map TestAddress Int
-> Property
checkResult String
"fetchynessBlocks" TestAddress
peer Map TestAddress SlotNo
joinedAtResults Map TestAddress Int
fetchynessBlocksResults)

    -- check that the peer is not in 20% worst peers, but only if there are more
    -- than 5 results.
    checkResult :: String
                -> TestAddress
                -> Map TestAddress SlotNo
                -> Map TestAddress Int
                -> Property
    checkResult :: String
-> TestAddress
-> Map TestAddress SlotNo
-> Map TestAddress Int
-> Property
checkResult String
name TestAddress
peer Map TestAddress SlotNo
joinedAt Map TestAddress Int
m =
          (\Set TestAddress
peers -> String -> Bool -> Property
forall prop. Testable prop => String -> prop -> Property
counterexample (String
name String -> ShowS
forall a. [a] -> [a] -> [a]
++ String
": peer (" String -> ShowS
forall a. [a] -> [a] -> [a]
++ TestAddress -> String
forall a. Show a => a -> String
show TestAddress
peer String -> ShowS
forall a. [a] -> [a] -> [a]
++ String
") member of "
                                          String -> ShowS
forall a. [a] -> [a] -> [a]
++ (Set TestAddress, Map TestAddress (Int, Maybe SlotNo)) -> String
forall a. Show a => a -> String
show (Set TestAddress
peers, Map TestAddress (Int, Maybe SlotNo)
m'))
                                    (Set TestAddress -> Int
forall a. Set a -> Int
Set.size Set TestAddress
peers Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
< Int
5 Bool -> Bool -> Bool
|| TestAddress -> Set TestAddress -> Bool
forall a. Ord a => a -> Set a -> Bool
Set.notMember TestAddress
peer Set TestAddress
peers))
        (Set TestAddress -> Property)
-> (Map TestAddress (Int, Maybe SlotNo) -> Set TestAddress)
-> Map TestAddress (Int, Maybe SlotNo)
-> Property
forall b c a. (b -> c) -> (a -> b) -> a -> c
. [TestAddress] -> Set TestAddress
forall a. Ord a => [a] -> Set a
Set.fromList
        ([TestAddress] -> Set TestAddress)
-> (Map TestAddress (Int, Maybe SlotNo) -> [TestAddress])
-> Map TestAddress (Int, Maybe SlotNo)
-> Set TestAddress
forall b c a. (b -> c) -> (a -> b) -> a -> c
. ((TestAddress, (Int, Maybe SlotNo)) -> TestAddress)
-> [(TestAddress, (Int, Maybe SlotNo))] -> [TestAddress]
forall a b. (a -> b) -> [a] -> [b]
map (TestAddress, (Int, Maybe SlotNo)) -> TestAddress
forall a b. (a, b) -> a
fst
        ([(TestAddress, (Int, Maybe SlotNo))] -> [TestAddress])
-> (Map TestAddress (Int, Maybe SlotNo)
    -> [(TestAddress, (Int, Maybe SlotNo))])
-> Map TestAddress (Int, Maybe SlotNo)
-> [TestAddress]
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Int
-> [(TestAddress, (Int, Maybe SlotNo))]
-> [(TestAddress, (Int, Maybe SlotNo))]
forall a. Int -> [a] -> [a]
take (Int
size Int -> Int -> Int
forall a. Integral a => a -> a -> a
`div` Int
5)
        ([(TestAddress, (Int, Maybe SlotNo))]
 -> [(TestAddress, (Int, Maybe SlotNo))])
-> (Map TestAddress (Int, Maybe SlotNo)
    -> [(TestAddress, (Int, Maybe SlotNo))])
-> Map TestAddress (Int, Maybe SlotNo)
-> [(TestAddress, (Int, Maybe SlotNo))]
forall b c a. (b -> c) -> (a -> b) -> a -> c
. ((TestAddress, (Int, Maybe SlotNo)) -> (Int, Maybe SlotNo))
-> [(TestAddress, (Int, Maybe SlotNo))]
-> [(TestAddress, (Int, Maybe SlotNo))]
forall b a. Ord b => (a -> b) -> [a] -> [a]
sortOn ((a, (Int, Maybe SlotNo)) -> (Int, Maybe SlotNo)
forall {a}. (a, (Int, Maybe SlotNo)) -> (Int, Maybe SlotNo)
forall a b. (a, b) -> b
snd :: (a, (Int, Maybe SlotNo)) -> (Int, Maybe SlotNo))
        ([(TestAddress, (Int, Maybe SlotNo))]
 -> [(TestAddress, (Int, Maybe SlotNo))])
-> (Map TestAddress (Int, Maybe SlotNo)
    -> [(TestAddress, (Int, Maybe SlotNo))])
-> Map TestAddress (Int, Maybe SlotNo)
-> [(TestAddress, (Int, Maybe SlotNo))]
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Map TestAddress (Int, Maybe SlotNo)
-> [(TestAddress, (Int, Maybe SlotNo))]
forall k a. Map k a -> [(k, a)]
Map.toList
        (Map TestAddress (Int, Maybe SlotNo) -> Property)
-> Map TestAddress (Int, Maybe SlotNo) -> Property
forall a b. (a -> b) -> a -> b
$ Map TestAddress (Int, Maybe SlotNo)
m'
      where
        m' :: Map TestAddress (Int, Maybe SlotNo)
m' = SimpleWhenMissing TestAddress Int (Int, Maybe SlotNo)
-> SimpleWhenMissing TestAddress SlotNo (Int, Maybe SlotNo)
-> SimpleWhenMatched TestAddress Int SlotNo (Int, Maybe SlotNo)
-> Map TestAddress Int
-> Map TestAddress SlotNo
-> Map TestAddress (Int, Maybe SlotNo)
forall k a c b.
Ord k =>
SimpleWhenMissing k a c
-> SimpleWhenMissing k b c
-> SimpleWhenMatched k a b c
-> Map k a
-> Map k b
-> Map k c
Map.merge ((TestAddress -> Int -> (Int, Maybe SlotNo))
-> SimpleWhenMissing TestAddress Int (Int, Maybe SlotNo)
forall (f :: * -> *) k x y.
Applicative f =>
(k -> x -> y) -> WhenMissing f k x y
Map.mapMissing (\TestAddress
_ Int
a -> (Int
a, Maybe SlotNo
forall a. Maybe a
Nothing)))
                        SimpleWhenMissing TestAddress SlotNo (Int, Maybe SlotNo)
forall (f :: * -> *) k x y. Applicative f => WhenMissing f k x y
Map.dropMissing
                       ((TestAddress -> Int -> SlotNo -> (Int, Maybe SlotNo))
-> SimpleWhenMatched TestAddress Int SlotNo (Int, Maybe SlotNo)
forall (f :: * -> *) k x y z.
Applicative f =>
(k -> x -> y -> z) -> WhenMatched f k x y z
Map.zipWithMatched (\TestAddress
_ Int
a SlotNo
b -> (Int
a, SlotNo -> Maybe SlotNo
forall a. a -> Maybe a
Just SlotNo
b)))
                       Map TestAddress Int
m
                       Map TestAddress SlotNo
joinedAt
        size :: Int
size = Map TestAddress Int -> Int
forall k a. Map k a -> Int
Map.size Map TestAddress Int
m

-- | Check that the results are always positive.
--
prop_metrics_are_bounded :: FixedScript -> Property
prop_metrics_are_bounded :: FixedScript -> Property
prop_metrics_are_bounded FixedScript
script =
    All -> Property
forall prop. Testable prop => prop -> Property
property (All -> Property) -> All -> Property
forall a b. (a -> b) -> a -> b
$ (PeerMetricsTrace -> All) -> [PeerMetricsTrace] -> All
forall m a. Monoid m => (a -> m) -> [a] -> m
forall (t :: * -> *) m a.
(Foldable t, Monoid m) =>
(a -> m) -> t a -> m
foldMap PeerMetricsTrace -> All
go [PeerMetricsTrace]
trace
  where
    config :: PeerMetricsConfiguration
    config :: PeerMetricsConfiguration
config = PeerMetricsConfiguration { maxEntriesToTrack :: Int
maxEntriesToTrack = Int
180 }

    sim :: IOSim s ()
    sim :: forall s. IOSim s ()
sim = Tracer (IOSim s) PeerMetricsTrace
-> PeerMetricsConfiguration -> FixedScript -> IOSim s ()
forall (m :: * -> *).
(MonadDelay m, MonadTimer m, MonadMonotonicTime m,
 MonadLabelledSTM m, MonadTraceSTM m) =>
Tracer m PeerMetricsTrace
-> PeerMetricsConfiguration -> FixedScript -> m ()
simulatePeerMetricScript ((PeerMetricsTrace -> IOSim s ())
-> Tracer (IOSim s) PeerMetricsTrace
forall (m :: * -> *) a. (a -> m ()) -> Tracer m a
Tracer PeerMetricsTrace -> IOSim s ()
forall a s. Typeable a => a -> IOSim s ()
traceM) PeerMetricsConfiguration
config FixedScript
script

    trace :: [PeerMetricsTrace]
    trace :: [PeerMetricsTrace]
trace = SimTrace () -> [PeerMetricsTrace]
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 ()
sim)

    safeMaximum :: Map a Int -> Int
    safeMaximum :: forall a. Map a Int -> Int
safeMaximum Map a Int
m | Map a Int -> Bool
forall k a. Map k a -> Bool
Map.null Map a Int
m = Int
0
    safeMaximum Map a Int
m = Map a Int -> Int
forall a. Ord a => Map a a -> a
forall (t :: * -> *) a. (Foldable t, Ord a) => t a -> a
maximum Map a Int
m

    -- We bound each result by twice the maximum value, that's very
    -- conservative. Less conservative would be maximal value plus average of
    -- last `maxEntriesToTrack` results or so.
    bound :: Int
    bound :: Int
bound =
        (Int
2 Int -> Int -> Int
forall a. Num a => a -> a -> a
*)
      (Int -> Int) -> (NonEmpty Event -> Int) -> NonEmpty Event -> Int
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Map TestAddress Int -> Int
forall a. Map a Int -> Int
safeMaximum
      (Map TestAddress Int -> Int)
-> (NonEmpty Event -> Map TestAddress Int) -> NonEmpty Event -> Int
forall b c a. (b -> c) -> (a -> b) -> a -> c
. (Int -> Int -> Int) -> [(TestAddress, Int)] -> Map TestAddress Int
forall k a. Ord k => (a -> a -> a) -> [(k, a)] -> Map k a
Map.fromListWith Int -> Int -> Int
forall a. Num a => a -> a -> a
(+)
      ([(TestAddress, Int)] -> Map TestAddress Int)
-> (NonEmpty Event -> [(TestAddress, Int)])
-> NonEmpty Event
-> Map TestAddress Int
forall b c a. (b -> c) -> (a -> b) -> a -> c
. (Event -> [(TestAddress, Int)] -> [(TestAddress, Int)])
-> [(TestAddress, Int)] -> NonEmpty Event -> [(TestAddress, Int)]
forall a b. (a -> b -> b) -> b -> NonEmpty a -> b
forall (t :: * -> *) a b.
Foldable t =>
(a -> b -> b) -> b -> t a -> b
foldr (\Event
a [(TestAddress, Int)]
as -> case Event
a of
                  FetchedHeader TestAddress
peer SlotNo
_   -> (TestAddress
peer, Int
1) (TestAddress, Int) -> [(TestAddress, Int)] -> [(TestAddress, Int)]
forall a. a -> [a] -> [a]
: [(TestAddress, Int)]
as
                  FetchedBlock  TestAddress
peer SlotNo
_ SizeInBytes
_ -> (TestAddress
peer, Int
1) (TestAddress, Int) -> [(TestAddress, Int)] -> [(TestAddress, Int)]
forall a. a -> [a] -> [a]
: [(TestAddress, Int)]
as)
              []
      (NonEmpty Event -> Int) -> NonEmpty Event -> Int
forall a b. (a -> b) -> a -> b
$ case FixedScript -> Script Event
getFixedScript FixedScript
script of
          Script NonEmpty Event
as -> NonEmpty Event
as

    fetchyness_bytes_bound :: Int
    fetchyness_bytes_bound :: Int
fetchyness_bytes_bound =
        (Int
2 Int -> Int -> Int
forall a. Num a => a -> a -> a
*)
      (Int -> Int) -> (NonEmpty Event -> Int) -> NonEmpty Event -> Int
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Map TestAddress Int -> Int
forall a. Map a Int -> Int
safeMaximum
      (Map TestAddress Int -> Int)
-> (NonEmpty Event -> Map TestAddress Int) -> NonEmpty Event -> Int
forall b c a. (b -> c) -> (a -> b) -> a -> c
. (SizeInBytes -> Int)
-> Map TestAddress SizeInBytes -> Map TestAddress Int
forall a b. (a -> b) -> Map TestAddress a -> Map TestAddress b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap SizeInBytes -> Int
forall a b. (Integral a, Num b) => a -> b
fromIntegral
      (Map TestAddress SizeInBytes -> Map TestAddress Int)
-> (NonEmpty Event -> Map TestAddress SizeInBytes)
-> NonEmpty Event
-> Map TestAddress Int
forall b c a. (b -> c) -> (a -> b) -> a -> c
. (SizeInBytes -> SizeInBytes -> SizeInBytes)
-> [(TestAddress, SizeInBytes)] -> Map TestAddress SizeInBytes
forall k a. Ord k => (a -> a -> a) -> [(k, a)] -> Map k a
Map.fromListWith SizeInBytes -> SizeInBytes -> SizeInBytes
forall a. Num a => a -> a -> a
(+)
      ([(TestAddress, SizeInBytes)] -> Map TestAddress SizeInBytes)
-> (NonEmpty Event -> [(TestAddress, SizeInBytes)])
-> NonEmpty Event
-> Map TestAddress SizeInBytes
forall b c a. (b -> c) -> (a -> b) -> a -> c
. (Event
 -> [(TestAddress, SizeInBytes)] -> [(TestAddress, SizeInBytes)])
-> [(TestAddress, SizeInBytes)]
-> NonEmpty Event
-> [(TestAddress, SizeInBytes)]
forall a b. (a -> b -> b) -> b -> NonEmpty a -> b
forall (t :: * -> *) a b.
Foldable t =>
(a -> b -> b) -> b -> t a -> b
foldr (\Event
a [(TestAddress, SizeInBytes)]
as -> case Event
a of
                  FetchedHeader {}          -> [(TestAddress, SizeInBytes)]
as
                  FetchedBlock TestAddress
peer SlotNo
_ SizeInBytes
bytes -> (TestAddress
peer, SizeInBytes
bytes) (TestAddress, SizeInBytes)
-> [(TestAddress, SizeInBytes)] -> [(TestAddress, SizeInBytes)]
forall a. a -> [a] -> [a]
: [(TestAddress, SizeInBytes)]
as)
              []
      (NonEmpty Event -> Int) -> NonEmpty Event -> Int
forall a b. (a -> b) -> a -> b
$ case FixedScript -> Script Event
getFixedScript FixedScript
script of
          Script NonEmpty Event
as -> NonEmpty Event
as


    go :: PeerMetricsTrace -> All
    go :: PeerMetricsTrace -> All
go PeerMetricsTrace { Map TestAddress Int
pmtUpstreamyness :: PeerMetricsTrace -> Map TestAddress Int
pmtUpstreamyness :: Map TestAddress Int
pmtUpstreamyness,
                          Map TestAddress Int
pmtFetchynessBytes :: PeerMetricsTrace -> Map TestAddress Int
pmtFetchynessBytes :: Map TestAddress Int
pmtFetchynessBytes,
                          Map TestAddress Int
pmtFetchynessBlocks :: PeerMetricsTrace -> Map TestAddress Int
pmtFetchynessBlocks :: Map TestAddress Int
pmtFetchynessBlocks
                        } =
         (Int -> All) -> Map TestAddress Int -> All
forall m a. Monoid m => (a -> m) -> Map TestAddress a -> m
forall (t :: * -> *) m a.
(Foldable t, Monoid m) =>
(a -> m) -> t a -> m
foldMap (\Int
a -> Property -> All
forall p. Testable p => p -> All
All
                      (Property -> All) -> Property -> All
forall a b. (a -> b) -> a -> b
$ String -> Bool -> Property
forall prop. Testable prop => String -> prop -> Property
counterexample
                          ((String, Int, Int, Map TestAddress Int) -> String
forall a. Show a => a -> String
show (String
"upstreameness", Int
a, Int
bound, Map TestAddress Int
pmtUpstreamyness))
                          (Int
a Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
>= Int
0))
                 Map TestAddress Int
pmtUpstreamyness
      All -> All -> All
forall a. Semigroup a => a -> a -> a
<> (Int -> All) -> Map TestAddress Int -> All
forall m a. Monoid m => (a -> m) -> Map TestAddress a -> m
forall (t :: * -> *) m a.
(Foldable t, Monoid m) =>
(a -> m) -> t a -> m
foldMap (\Int
a -> Property -> All
forall p. Testable p => p -> All
All
                      (Property -> All) -> Property -> All
forall a b. (a -> b) -> a -> b
$ String -> Bool -> Property
forall prop. Testable prop => String -> prop -> Property
counterexample
                          ((String, Int, Int, Map TestAddress Int) -> String
forall a. Show a => a -> String
show (String
"fetchynessBytes", Int
a, Int
fetchyness_bytes_bound, Map TestAddress Int
pmtFetchynessBytes))
                          (Int
a Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
>= Int
0 Bool -> Bool -> Bool
&& Int
a Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
<= Int
fetchyness_bytes_bound))
                 Map TestAddress Int
pmtFetchynessBytes
      All -> All -> All
forall a. Semigroup a => a -> a -> a
<> (Int -> All) -> Map TestAddress Int -> All
forall m a. Monoid m => (a -> m) -> Map TestAddress a -> m
forall (t :: * -> *) m a.
(Foldable t, Monoid m) =>
(a -> m) -> t a -> m
foldMap (\Int
a -> Property -> All
forall p. Testable p => p -> All
All
                      (Property -> All) -> Property -> All
forall a b. (a -> b) -> a -> b
$ String -> Bool -> Property
forall prop. Testable prop => String -> prop -> Property
counterexample
                          ((String, Int, Int) -> String
forall a. Show a => a -> String
show (String
"fetchynessBlocks", Int
a, Int
bound))
                          (Int
a Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
>= Int
0))
                 Map TestAddress Int
pmtFetchynessBlocks


-- | Check that the result are bounded.
--
-- The bound is 'maxEntriesToTrack' times number of peers in the simulation.
-- This could be lowered by computing number of peers in each
-- 'maxEntriesToTrack' slots window.
--
prop_bounded_size :: Positive Int -> FixedScript -> Property
prop_bounded_size :: Positive Int -> FixedScript -> Property
prop_bounded_size (Positive Int
maxEntriesToTrack) FixedScript
script =
    All -> Property
forall prop. Testable prop => prop -> Property
property (All -> Property) -> All -> Property
forall a b. (a -> b) -> a -> b
$ (PeerMetricsTrace -> All) -> [PeerMetricsTrace] -> All
forall m a. Monoid m => (a -> m) -> [a] -> m
forall (t :: * -> *) m a.
(Foldable t, Monoid m) =>
(a -> m) -> t a -> m
foldMap PeerMetricsTrace -> All
go [PeerMetricsTrace]
trace
  where
    config :: PeerMetricsConfiguration
    config :: PeerMetricsConfiguration
config = PeerMetricsConfiguration { Int
maxEntriesToTrack :: Int
maxEntriesToTrack :: Int
maxEntriesToTrack }

    sim :: IOSim s ()
    sim :: forall s. IOSim s ()
sim = Tracer (IOSim s) PeerMetricsTrace
-> PeerMetricsConfiguration -> FixedScript -> IOSim s ()
forall (m :: * -> *).
(MonadDelay m, MonadTimer m, MonadMonotonicTime m,
 MonadLabelledSTM m, MonadTraceSTM m) =>
Tracer m PeerMetricsTrace
-> PeerMetricsConfiguration -> FixedScript -> m ()
simulatePeerMetricScript ((PeerMetricsTrace -> IOSim s ())
-> Tracer (IOSim s) PeerMetricsTrace
forall (m :: * -> *) a. (a -> m ()) -> Tracer m a
Tracer PeerMetricsTrace -> IOSim s ()
forall a s. Typeable a => a -> IOSim s ()
traceM) PeerMetricsConfiguration
config FixedScript
script

    trace :: [PeerMetricsTrace]
    trace :: [PeerMetricsTrace]
trace = SimTrace () -> [PeerMetricsTrace]
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 ()
sim)

    number_of_peers :: Int
    number_of_peers :: Int
number_of_peers = Set TestAddress -> Int
forall a. Set a -> Int
Set.size
                    (Set TestAddress -> Int)
-> (NonEmpty Event -> Set TestAddress) -> NonEmpty Event -> Int
forall b c a. (b -> c) -> (a -> b) -> a -> c
. [TestAddress] -> Set TestAddress
forall a. Ord a => [a] -> Set a
Set.fromList
                    ([TestAddress] -> Set TestAddress)
-> (NonEmpty Event -> [TestAddress])
-> NonEmpty Event
-> Set TestAddress
forall b c a. (b -> c) -> (a -> b) -> a -> c
. ([TestAddress] -> Event -> [TestAddress])
-> [TestAddress] -> NonEmpty Event -> [TestAddress]
forall b a. (b -> a -> b) -> b -> NonEmpty a -> b
forall (t :: * -> *) b a.
Foldable t =>
(b -> a -> b) -> b -> t a -> b
Foldable.foldl' (\[TestAddress]
as Event
a -> Event -> TestAddress
eventPeer Event
a TestAddress -> [TestAddress] -> [TestAddress]
forall a. a -> [a] -> [a]
: [TestAddress]
as) []
                    (NonEmpty Event -> Int) -> NonEmpty Event -> Int
forall a b. (a -> b) -> a -> b
$ case FixedScript -> Script Event
getFixedScript FixedScript
script of
                        Script NonEmpty Event
as -> NonEmpty Event
as

    bound :: Int
    bound :: Int
bound = Int
maxEntriesToTrack Int -> Int -> Int
forall a. Num a => a -> a -> a
* Int
number_of_peers

    go :: PeerMetricsTrace -> All
    go :: PeerMetricsTrace -> All
go PeerMetricsTrace {
           Map TestAddress Int
pmtUpstreamyness :: PeerMetricsTrace -> Map TestAddress Int
pmtUpstreamyness :: Map TestAddress Int
pmtUpstreamyness,
           Map TestAddress Int
pmtFetchynessBytes :: PeerMetricsTrace -> Map TestAddress Int
pmtFetchynessBytes :: Map TestAddress Int
pmtFetchynessBytes,
           Map TestAddress Int
pmtFetchynessBlocks :: PeerMetricsTrace -> Map TestAddress Int
pmtFetchynessBlocks :: Map TestAddress Int
pmtFetchynessBlocks
         } = Property -> All
forall p. Testable p => p -> All
All ( String -> Bool -> Property
forall prop. Testable prop => String -> prop -> Property
counterexample
                     (    String
"upstreamyness: "
                       String -> ShowS
forall a. [a] -> [a] -> [a]
++ Int -> String
forall a. Show a => a -> String
show (Map TestAddress Int -> Int
forall k a. Map k a -> Int
Map.size Map TestAddress Int
pmtUpstreamyness)
                       String -> ShowS
forall a. [a] -> [a] -> [a]
++ String
" ≰ "
                       String -> ShowS
forall a. [a] -> [a] -> [a]
++ Int -> String
forall a. Show a => a -> String
show Int
maxEntriesToTrack )
                     ( Map TestAddress Int -> Int
forall k a. Map k a -> Int
Map.size Map TestAddress Int
pmtUpstreamyness Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
<= Int
bound )
                 )
          All -> All -> All
forall a. Semigroup a => a -> a -> a
<> Property -> All
forall p. Testable p => p -> All
All ( String -> Bool -> Property
forall prop. Testable prop => String -> prop -> Property
counterexample
                     (    String
"fetchynessBytes: "
                       String -> ShowS
forall a. [a] -> [a] -> [a]
++ Int -> String
forall a. Show a => a -> String
show (Map TestAddress Int -> Int
forall k a. Map k a -> Int
Map.size Map TestAddress Int
pmtFetchynessBytes)
                       String -> ShowS
forall a. [a] -> [a] -> [a]
++ String
" ≰ "
                       String -> ShowS
forall a. [a] -> [a] -> [a]
++ Int -> String
forall a. Show a => a -> String
show Int
maxEntriesToTrack)
                     ( Map TestAddress Int -> Int
forall k a. Map k a -> Int
Map.size Map TestAddress Int
pmtFetchynessBytes Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
<= Int
bound )
                 )
          All -> All -> All
forall a. Semigroup a => a -> a -> a
<> Property -> All
forall p. Testable p => p -> All
All ( String -> Bool -> Property
forall prop. Testable prop => String -> prop -> Property
counterexample
                     (    String
"fetchynessBlocks: "
                       String -> ShowS
forall a. [a] -> [a] -> [a]
++ Int -> String
forall a. Show a => a -> String
show (Map TestAddress Int -> Int
forall k a. Map k a -> Int
Map.size Map TestAddress Int
pmtFetchynessBlocks)
                       String -> ShowS
forall a. [a] -> [a] -> [a]
++ String
" ≰ "
                       String -> ShowS
forall a. [a] -> [a] -> [a]
++ Int -> String
forall a. Show a => a -> String
show Int
maxEntriesToTrack)
                     ( Map TestAddress Int -> Int
forall k a. Map k a -> Int
Map.size Map TestAddress Int
pmtFetchynessBlocks Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
<= Int
bound )
                 )

--
-- The following are focused on creating micro-benchmarks
-- (rather than property testing):
--

-- | microbenchmark1 n - one test of simple property on a FixedScript of length n:
--   one input, one property, one test.
--
-- We split into generating input and running/checking it so we can
-- more accurately measure the latter.

microbenchmark1GenerateInput :: Bool -> Int -> IO FixedScript
microbenchmark1GenerateInput :: Bool -> Int -> IO FixedScript
microbenchmark1GenerateInput Bool
verbose' Int
n = do
  es <- Gen [Event] -> IO [Event]
forall a. Gen a -> IO a
generate (Int -> Gen [Event]
forall a. Arbitrary a => Int -> Gen [a]
vector Int
n)
  let fixedScript = Script Event -> FixedScript
mkFixedScript (NonEmpty Event -> Script Event
forall a. NonEmpty a -> Script a
Script ([Event] -> NonEmpty Event
forall a. HasCallStack => [a] -> NonEmpty a
NonEmpty.fromList [Event]
es))
  when verbose' $
    mapM_ print (let FixedScript s = fixedScript in s)
  return fixedScript

microbenchmark1ProcessInput :: FixedScript -> IO ()
microbenchmark1ProcessInput :: FixedScript -> IO ()
microbenchmark1ProcessInput =
  Args -> Property -> IO ()
forall prop. Testable prop => Args -> prop -> IO ()
quickCheckWith (Args
stdArgs{maxSuccess=1}) (Property -> IO ())
-> (FixedScript -> Property) -> FixedScript -> IO ()
forall b c a. (b -> c) -> (a -> b) -> a -> c
. FixedScript -> Property
prop_simScript

microbenchmark1 :: Bool -> Int -> IO ()
microbenchmark1 :: Bool -> Int -> IO ()
microbenchmark1 Bool
verbose' Int
n =
  Bool -> Int -> IO FixedScript
microbenchmark1GenerateInput Bool
verbose' Int
n IO FixedScript -> (FixedScript -> IO ()) -> IO ()
forall a b. IO a -> (a -> IO b) -> IO b
forall (m :: * -> *) a b. Monad m => m a -> (a -> m b) -> m b
>>= FixedScript -> IO ()
microbenchmark1ProcessInput


-- | one simple property (pmtUpstreamyness >= 0) checked on the trace of a script:
prop_simScript :: FixedScript -> Property
prop_simScript :: FixedScript -> Property
prop_simScript FixedScript
script =
    All -> Property
forall prop. Testable prop => prop -> Property
property (All -> Property) -> All -> Property
forall a b. (a -> b) -> a -> b
$ PeerMetricsTrace -> All
go (PeerMetricsTrace -> All) -> PeerMetricsTrace -> All
forall a b. (a -> b) -> a -> b
$ [PeerMetricsTrace] -> PeerMetricsTrace
forall a. HasCallStack => [a] -> a
last [PeerMetricsTrace]
trace
  where
    config :: PeerMetricsConfiguration
    config :: PeerMetricsConfiguration
config = PeerMetricsConfiguration { maxEntriesToTrack :: Int
maxEntriesToTrack = Int
500 }

    sim :: IOSim s ()
    sim :: forall s. IOSim s ()
sim = Tracer (IOSim s) PeerMetricsTrace
-> PeerMetricsConfiguration -> FixedScript -> IOSim s ()
forall (m :: * -> *).
MonadLabelledSTM m =>
Tracer m PeerMetricsTrace
-> PeerMetricsConfiguration -> FixedScript -> m ()
simulatePeerMetricScriptWithoutDelays ((PeerMetricsTrace -> IOSim s ())
-> Tracer (IOSim s) PeerMetricsTrace
forall (m :: * -> *) a. (a -> m ()) -> Tracer m a
Tracer PeerMetricsTrace -> IOSim s ()
forall a s. Typeable a => a -> IOSim s ()
traceM) PeerMetricsConfiguration
config FixedScript
script

    trace :: [PeerMetricsTrace]
    trace :: [PeerMetricsTrace]
trace = SimTrace () -> [PeerMetricsTrace]
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 ()
sim)

    go :: PeerMetricsTrace -> All
    go :: PeerMetricsTrace -> All
go PeerMetricsTrace { Map TestAddress Int
pmtUpstreamyness :: PeerMetricsTrace -> Map TestAddress Int
pmtUpstreamyness :: Map TestAddress Int
pmtUpstreamyness,
                          pmtFetchynessBytes :: PeerMetricsTrace -> Map TestAddress Int
pmtFetchynessBytes=Map TestAddress Int
_,
                          pmtFetchynessBlocks :: PeerMetricsTrace -> Map TestAddress Int
pmtFetchynessBlocks=Map TestAddress Int
_
                        } =
         (Int -> All) -> Map TestAddress Int -> All
forall m a. Monoid m => (a -> m) -> Map TestAddress a -> m
forall (t :: * -> *) m a.
(Foldable t, Monoid m) =>
(a -> m) -> t a -> m
foldMap (\Int
a -> Property -> All
forall p. Testable p => p -> All
All
                      (Property -> All) -> Property -> All
forall a b. (a -> b) -> a -> b
$ String -> Bool -> Property
forall prop. Testable prop => String -> prop -> Property
counterexample
                          ((String, Int, Map TestAddress Int) -> String
forall a. Show a => a -> String
show (String
"upstreamyness", Int
a, Map TestAddress Int
pmtUpstreamyness))
                          (Int
a Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
>= Int
0))
                 Map TestAddress Int
pmtUpstreamyness

-- | similar to 'simulatePeerMetricScript': but we don't do in
-- "real/simulated" time: not calling getMonotonicTime and
-- threadDelay.
simulatePeerMetricScriptWithoutDelays
  :: forall m.
     ( MonadLabelledSTM m )
  => Tracer m PeerMetricsTrace
  -> PeerMetricsConfiguration
  -> FixedScript
  -> m ()
simulatePeerMetricScriptWithoutDelays :: forall (m :: * -> *).
MonadLabelledSTM m =>
Tracer m PeerMetricsTrace
-> PeerMetricsConfiguration -> FixedScript -> m ()
simulatePeerMetricScriptWithoutDelays Tracer m PeerMetricsTrace
tracer PeerMetricsConfiguration
config FixedScript
script = do
      peerMetrics <- PeerMetricsConfiguration -> m (PeerMetrics m TestAddress)
forall (m :: * -> *) p.
(MonadLabelledSTM m, NoThunks p, NFData p) =>
PeerMetricsConfiguration -> m (PeerMetrics m p)
newPeerMetric PeerMetricsConfiguration
config
      let reporter :: ReportPeerMetrics m (ConnectionId TestAddress)
          reporter = PeerMetricsConfiguration
-> PeerMetrics m TestAddress
-> ReportPeerMetrics m (ConnectionId TestAddress)
forall (m :: * -> *) p.
(MonadSTM m, Ord p) =>
PeerMetricsConfiguration
-> PeerMetrics m p -> ReportPeerMetrics m (ConnectionId p)
reportMetric PeerMetricsConfiguration
config PeerMetrics m TestAddress
peerMetrics
      v <- initScript timedScript
      go v peerMetrics reporter (Time 0)
    where
      timedScript ::  TimedScript Event
      timedScript :: TimedScript Event
timedScript = FixedScript -> TimedScript Event
mkTimedScript FixedScript
script

      go :: LazySTM.TVar m (TimedScript Event)
         -> PeerMetrics m TestAddress
         -> ReportPeerMetrics m (ConnectionId TestAddress)
         -> Time
         -> m ()
      go :: TVar m (TimedScript Event)
-> PeerMetrics m TestAddress
-> ReportPeerMetrics m (ConnectionId TestAddress)
-> Time
-> m ()
go TVar m (TimedScript Event)
v PeerMetrics m TestAddress
peerMetrics reporter :: ReportPeerMetrics m (ConnectionId TestAddress)
reporter@ReportPeerMetrics { Tracer
  (STM m) (TraceLabelPeer (ConnectionId TestAddress) (SlotNo, Time))
reportHeader :: forall (m :: * -> *) peerAddr.
ReportPeerMetrics m peerAddr
-> Tracer (STM m) (TraceLabelPeer peerAddr (SlotNo, Time))
reportHeader :: Tracer
  (STM m) (TraceLabelPeer (ConnectionId TestAddress) (SlotNo, Time))
reportHeader, Tracer
  (STM m)
  (TraceLabelPeer
     (ConnectionId TestAddress) (SizeInBytes, SlotNo, Time))
reportFetch :: forall (m :: * -> *) peerAddr.
ReportPeerMetrics m peerAddr
-> Tracer
     (STM m) (TraceLabelPeer peerAddr (SizeInBytes, SlotNo, Time))
reportFetch :: Tracer
  (STM m)
  (TraceLabelPeer
     (ConnectionId TestAddress) (SizeInBytes, SlotNo, Time))
reportFetch } Time
time = do
        (continue, (ev, delay)) <- (\case Left  (Event, ScriptDelay)
a -> (Bool
False, (Event, ScriptDelay)
a)
                                          Right (Event, ScriptDelay)
a -> (Bool
True,  (Event, ScriptDelay)
a))
                               (Either (Event, ScriptDelay) (Event, ScriptDelay)
 -> (Bool, (Event, ScriptDelay)))
-> m (Either (Event, ScriptDelay) (Event, ScriptDelay))
-> m (Bool, (Event, ScriptDelay))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> TVar m (TimedScript Event)
-> m (Either (Event, ScriptDelay) (Event, ScriptDelay))
forall (m :: * -> *) a.
MonadSTM m =>
TVar m (Script a) -> m (Either a a)
stepScriptOrFinish TVar m (TimedScript Event)
v
        peer <- case ev of
          FetchedHeader TestAddress
peer SlotNo
slotNo -> do
            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
$ Tracer
  (STM m) (TraceLabelPeer (ConnectionId TestAddress) (SlotNo, Time))
-> TraceLabelPeer (ConnectionId TestAddress) (SlotNo, Time)
-> STM m ()
forall (m :: * -> *) a. Tracer m a -> a -> m ()
traceWith Tracer
  (STM m) (TraceLabelPeer (ConnectionId TestAddress) (SlotNo, Time))
reportHeader
                       (TraceLabelPeer (ConnectionId TestAddress) (SlotNo, Time)
 -> STM m ())
-> TraceLabelPeer (ConnectionId TestAddress) (SlotNo, Time)
-> STM m ()
forall a b. (a -> b) -> a -> b
$ ConnectionId TestAddress
-> (SlotNo, Time)
-> TraceLabelPeer (ConnectionId TestAddress) (SlotNo, Time)
forall peerid a. peerid -> a -> TraceLabelPeer peerid a
TraceLabelPeer ConnectionId {
                                            localAddress :: TestAddress
localAddress  = Int -> TestAddress
TestAddress Int
0,
                                            remoteAddress :: TestAddress
remoteAddress = TestAddress
peer
                                          }
                                        (SlotNo
slotNo, Time
time)
            TestAddress -> m TestAddress
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return TestAddress
peer

          FetchedBlock TestAddress
peer SlotNo
slotNo SizeInBytes
size -> do
            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
$ Tracer
  (STM m)
  (TraceLabelPeer
     (ConnectionId TestAddress) (SizeInBytes, SlotNo, Time))
-> TraceLabelPeer
     (ConnectionId TestAddress) (SizeInBytes, SlotNo, Time)
-> STM m ()
forall (m :: * -> *) a. Tracer m a -> a -> m ()
traceWith Tracer
  (STM m)
  (TraceLabelPeer
     (ConnectionId TestAddress) (SizeInBytes, SlotNo, Time))
reportFetch
                       (TraceLabelPeer
   (ConnectionId TestAddress) (SizeInBytes, SlotNo, Time)
 -> STM m ())
-> TraceLabelPeer
     (ConnectionId TestAddress) (SizeInBytes, SlotNo, Time)
-> STM m ()
forall a b. (a -> b) -> a -> b
$ ConnectionId TestAddress
-> (SizeInBytes, SlotNo, Time)
-> TraceLabelPeer
     (ConnectionId TestAddress) (SizeInBytes, SlotNo, Time)
forall peerid a. peerid -> a -> TraceLabelPeer peerid a
TraceLabelPeer ConnectionId {
                                            localAddress :: TestAddress
localAddress  = Int -> TestAddress
TestAddress Int
0,
                                            remoteAddress :: TestAddress
remoteAddress = TestAddress
peer
                                          }
                                        (SizeInBytes
size, SlotNo
slotNo, Time
time)
            TestAddress -> m TestAddress
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return TestAddress
peer

        trace <- atomically $
           PeerMetricsTrace peer (eventSlot ev)
                 <$> upstreamyness      peerMetrics
                 <*> fetchynessBytes    peerMetrics
                 <*> fetchynessBlocks   peerMetrics
                 <*> joinedPeerMetricAt peerMetrics
        traceWith tracer trace

        when continue $
          go v peerMetrics reporter (interpretScriptDelay delay `addTime` time)