{-# LANGUAGE DerivingStrategies #-}
{-# LANGUAGE GeneralizedNewtypeDeriving #-}
{-# LANGUAGE NamedFieldPuns #-}
{-# LANGUAGE RecordWildCards #-}

-- | Tests for the garbage collection schedule.
--
-- The real implementation of course is stateful, running some code at various
-- intervals. We have a model of the implementation along with some @QuickCheck@
-- properties checking various invariants of the model such as
--
-- * The length of the queue is bounded
-- * The overlap between the volatile DB and immutable DB (due to blocks that
--   could have been moved but haven't yet) is bounded.
--
-- We then test that the real implementation behaves exactly as the model
-- predicts.
--
module Test.Ouroboros.Storage.ChainDB.GcSchedule (
    example
  , tests
  ) where

import           Control.Monad (forM)
import           Control.Monad.IOSim (runSimOrThrow)
import           Control.Tracer (nullTracer)
import           Data.Fixed (div')
import           Data.List as List (foldl', partition, sort)
import           Data.Time.Clock
import           Data.Void (Void)
import           Ouroboros.Consensus.Block
import           Ouroboros.Consensus.Storage.ChainDB.Impl.Background
                     (GcParams (..), ScheduledGc (..))
import qualified Ouroboros.Consensus.Storage.ChainDB.Impl.Background as Impl
import           Ouroboros.Consensus.Util (lastMaybe, safeMaximum)
import           Ouroboros.Consensus.Util.Condense
import           Ouroboros.Consensus.Util.IOLike
import           Test.QuickCheck
import           Test.Tasty
import           Test.Tasty.QuickCheck
import           Test.Util.Orphans.IOLike ()
import           Test.Util.QuickCheck

{-------------------------------------------------------------------------------
  Top-level tests
-------------------------------------------------------------------------------}

tests :: TestTree
tests :: TestTree
tests = TestName -> [TestTree] -> TestTree
testGroup TestName
"GcSchedule"
    [ TestName -> (TestSetup -> Property) -> TestTree
forall a. Testable a => TestName -> a -> TestTree
testProperty TestName
"queueLength"        TestSetup -> Property
prop_queueLength
    , TestName -> (TestSetup -> Property) -> TestTree
forall a. Testable a => TestName -> a -> TestTree
testProperty TestName
"overlap"            TestSetup -> Property
prop_overlap
    , TestName -> (TestSetup -> Property) -> TestTree
forall a. Testable a => TestName -> a -> TestTree
testProperty TestName
"unnecessaryOverlap" TestSetup -> Property
prop_unnecessaryOverlap
    , TestName -> (TestSetup -> Property) -> TestTree
forall a. Testable a => TestName -> a -> TestTree
testProperty TestName
"model vs impl"      TestSetup -> Property
prop_model_vs_impl
    ]

{-------------------------------------------------------------------------------
  Properties
-------------------------------------------------------------------------------}

-- | Property 1
--
-- 'queueLength' <= 'gcDelay' `div` 'gcInterval' + @slack@
--
-- Where:
-- * @slack = 1@ when 'gcInterval' divides 'gcDelay'. In this case, the delay
--   divides the interval nicely in different buckets. However, if we're not
--   at the start of a bucket and part of it is in the past, we'll need one
--   extra bucket to compensate, hence 1.
-- * @slack = 2@ in the other cases: in addition to the 1 of the previous
--   case, we must also account for one extra bucket because 'gcInterval'
--   doesn't nicely divide 'gcDelay' into buckets. In other words: we need to
--   round up.
prop_queueLength :: TestSetup -> Property
prop_queueLength :: TestSetup -> Property
prop_queueLength TestSetup{Int
Integer
Trace GcStateSummary
GcParams
GcGarbageCollections
testNumBlocks :: Int
testDelay :: Integer
testInterval :: Integer
testGcParams :: GcParams
testTrace :: Trace GcStateSummary
testGcGarbageCollections :: GcGarbageCollections
testNumBlocks :: TestSetup -> Int
testDelay :: TestSetup -> Integer
testInterval :: TestSetup -> Integer
testGcParams :: TestSetup -> GcParams
testTrace :: TestSetup -> Trace GcStateSummary
testGcGarbageCollections :: TestSetup -> GcGarbageCollections
..} =
    Integer
testDelay Integer -> Integer -> Bool
forall a. Ord a => a -> a -> Bool
>= Integer
testInterval Bool -> Property -> Property
forall prop. Testable prop => Bool -> prop -> Property
==>
      [Property] -> Property
forall prop. Testable prop => [prop] -> Property
conjoin
        [ Int
gcSummaryQueueLength Int -> Int -> Property
forall a. (Ord a, Show a) => a -> a -> Property
`le` (DiffTime
gcDelay DiffTime -> DiffTime -> Int
forall a b. (Real a, Integral b) => a -> a -> b
`div'` DiffTime
gcInterval) Int -> Int -> Int
forall a. Num a => a -> a -> a
+ Int
slack
        | GcStateSummary { Int
gcSummaryQueueLength :: Int
gcSummaryQueueLength :: GcStateSummary -> Int
gcSummaryQueueLength } <- Trace GcStateSummary
testTrace
        ]
  where
    GcParams{DiffTime
gcDelay :: GcParams -> DiffTime
gcInterval :: GcParams -> DiffTime
gcDelay :: DiffTime
gcInterval :: DiffTime
..} = GcParams
testGcParams
    slack :: Int
slack
      | Integer
testDelay Integer -> Integer -> Integer
forall a. Integral a => a -> a -> a
`mod` Integer
testInterval Integer -> Integer -> Bool
forall a. Eq a => a -> a -> Bool
== Integer
0
      = Int
1
      | Bool
otherwise
      = Int
2

-- | Property 2:
--
-- 'overlap' < the number of blocks that could arrive 'gcDelay' +
-- 'gcInterval'.
prop_overlap :: TestSetup -> Property
prop_overlap :: TestSetup -> Property
prop_overlap TestSetup{Int
Integer
Trace GcStateSummary
GcParams
GcGarbageCollections
testNumBlocks :: TestSetup -> Int
testDelay :: TestSetup -> Integer
testInterval :: TestSetup -> Integer
testGcParams :: TestSetup -> GcParams
testTrace :: TestSetup -> Trace GcStateSummary
testGcGarbageCollections :: TestSetup -> GcGarbageCollections
testNumBlocks :: Int
testDelay :: Integer
testInterval :: Integer
testGcParams :: GcParams
testTrace :: Trace GcStateSummary
testGcGarbageCollections :: GcGarbageCollections
..} =
    [Property] -> Property
forall prop. Testable prop => [prop] -> Property
conjoin
      [ Int
gcSummaryOverlap Int -> Int -> Property
forall a. (Ord a, Show a) => a -> a -> Property
`lt` DiffTime -> Int
blocksInInterval (DiffTime
gcDelay DiffTime -> DiffTime -> DiffTime
forall a. Num a => a -> a -> a
+ DiffTime
gcInterval)
      | GcStateSummary { Int
gcSummaryOverlap :: Int
gcSummaryOverlap :: GcStateSummary -> Int
gcSummaryOverlap } <- Trace GcStateSummary
testTrace
      ]
  where
    GcParams{DiffTime
gcDelay :: GcParams -> DiffTime
gcInterval :: GcParams -> DiffTime
gcDelay :: DiffTime
gcInterval :: DiffTime
..} = GcParams
testGcParams

-- | Property 3:
--
-- 'unnecessaryOverlap' < the number of blocks that could arrive in
-- 'gcInterval'.
prop_unnecessaryOverlap :: TestSetup -> Property
prop_unnecessaryOverlap :: TestSetup -> Property
prop_unnecessaryOverlap TestSetup{Int
Integer
Trace GcStateSummary
GcParams
GcGarbageCollections
testNumBlocks :: TestSetup -> Int
testDelay :: TestSetup -> Integer
testInterval :: TestSetup -> Integer
testGcParams :: TestSetup -> GcParams
testTrace :: TestSetup -> Trace GcStateSummary
testGcGarbageCollections :: TestSetup -> GcGarbageCollections
testNumBlocks :: Int
testDelay :: Integer
testInterval :: Integer
testGcParams :: GcParams
testTrace :: Trace GcStateSummary
testGcGarbageCollections :: GcGarbageCollections
..} =
    [Property] -> Property
forall prop. Testable prop => [prop] -> Property
conjoin
      [ Int
gcSummaryUnnecessary Int -> Int -> Property
forall a. (Ord a, Show a) => a -> a -> Property
`lt` DiffTime -> Int
blocksInInterval DiffTime
gcInterval
      | GcStateSummary { Int
gcSummaryUnnecessary :: Int
gcSummaryUnnecessary :: GcStateSummary -> Int
gcSummaryUnnecessary } <- Trace GcStateSummary
testTrace
      ]
  where
    GcParams{DiffTime
gcDelay :: GcParams -> DiffTime
gcInterval :: GcParams -> DiffTime
gcInterval :: DiffTime
gcDelay :: DiffTime
..} = GcParams
testGcParams

-- TODO the unnecessaryOverlap should at some point go back to 0 after it has
-- increased: test this property

blocksInInterval :: DiffTime -> Int
blocksInInterval :: DiffTime -> Int
blocksInInterval DiffTime
interval = Double -> Int
forall b. Integral b => Double -> b
forall a b. (RealFrac a, Integral b) => a -> b
round (DiffTime -> Double
forall a b. (Real a, Fractional b) => a -> b
realToFrac DiffTime
interval :: Double)

-- | Verify that the queue of the real implementation matches the model queue
-- at each point in the trace.
--
-- Moreover, verify that the real implementation will have performed all its
-- garbage collections at the same times as the model implementation.
prop_model_vs_impl :: TestSetup -> Property
prop_model_vs_impl :: TestSetup -> Property
prop_model_vs_impl TestSetup {Int
Integer
Trace GcStateSummary
GcParams
GcGarbageCollections
testNumBlocks :: TestSetup -> Int
testDelay :: TestSetup -> Integer
testInterval :: TestSetup -> Integer
testGcParams :: TestSetup -> GcParams
testTrace :: TestSetup -> Trace GcStateSummary
testGcGarbageCollections :: TestSetup -> GcGarbageCollections
testNumBlocks :: Int
testDelay :: Integer
testInterval :: Integer
testGcParams :: GcParams
testTrace :: Trace GcStateSummary
testGcGarbageCollections :: GcGarbageCollections
..} = [Property] -> Property
forall prop. Testable prop => [prop] -> Property
conjoin
    [ TestName -> Property -> Property
forall prop. Testable prop => TestName -> prop -> Property
counterexample TestName
"Expected queue evolution /= actual" (Property -> Property) -> Property -> Property
forall a b. (a -> b) -> a -> b
$
        (GcStateSummary -> GcQueue) -> Trace GcStateSummary -> [GcQueue]
forall a b. (a -> b) -> [a] -> [b]
map (GcQueue -> GcQueue
reverseQueue (GcQueue -> GcQueue)
-> (GcStateSummary -> GcQueue) -> GcStateSummary -> GcQueue
forall b c a. (b -> c) -> (a -> b) -> a -> c
. GcStateSummary -> GcQueue
gcSummaryQueue) Trace GcStateSummary
testTrace [GcQueue] -> [GcQueue] -> Property
forall a. (Eq a, Show a) => a -> a -> Property
=== [GcQueue]
gcQueueTrace
    , TestName -> Property -> Property
forall prop. Testable prop => TestName -> prop -> Property
counterexample TestName
"Expected final garbage collections /= actual" (Property -> Property) -> Property -> Property
forall a b. (a -> b) -> a -> b
$
        GcGarbageCollections
testGcGarbageCollections GcGarbageCollections -> GcGarbageCollections -> Property
forall a. (Eq a, Show a) => a -> a -> Property
=== GcGarbageCollections
gcs
    ]
  where
    ([GcQueue]
gcQueueTrace, GcGarbageCollections
gcs) = GcParams -> [Block] -> ([GcQueue], GcGarbageCollections)
runGcSchedule GcParams
testGcParams (Int -> [Block]
genBlocks Int
testNumBlocks)

    -- In the model we store the queue in reverse order, so we have to reverse
    -- it to match the order of the implementation's queue.
    reverseQueue :: GcQueue -> GcQueue
    reverseQueue :: GcQueue -> GcQueue
reverseQueue (GcQueue [ScheduledGc]
q) = [ScheduledGc] -> GcQueue
GcQueue ([ScheduledGc] -> [ScheduledGc]
forall a. [a] -> [a]
reverse [ScheduledGc]
q)

{-------------------------------------------------------------------------------
  Block
-------------------------------------------------------------------------------}

newtype Block = Block Int
  deriving stock   (Int -> Block -> ShowS
[Block] -> ShowS
Block -> TestName
(Int -> Block -> ShowS)
-> (Block -> TestName) -> ([Block] -> ShowS) -> Show Block
forall a.
(Int -> a -> ShowS) -> (a -> TestName) -> ([a] -> ShowS) -> Show a
$cshowsPrec :: Int -> Block -> ShowS
showsPrec :: Int -> Block -> ShowS
$cshow :: Block -> TestName
show :: Block -> TestName
$cshowList :: [Block] -> ShowS
showList :: [Block] -> ShowS
Show)
  deriving newtype (Block -> TestName
(Block -> TestName) -> Condense Block
forall a. (a -> TestName) -> Condense a
$ccondense :: Block -> TestName
condense :: Block -> TestName
Condense)

blockArrivalTime :: Block -> Time
blockArrivalTime :: Block -> Time
blockArrivalTime (Block Int
n) = DiffTime -> Time
Time (Integer -> DiffTime
secondsToDiffTime (Int -> Integer
forall a b. (Integral a, Num b) => a -> b
fromIntegral Int
n))

blockSlotNo :: Block -> SlotNo
blockSlotNo :: Block -> SlotNo
blockSlotNo (Block Int
n) = Word64 -> SlotNo
SlotNo (Int -> Word64
forall a b. (Integral a, Num b) => a -> b
fromIntegral Int
n)

{-------------------------------------------------------------------------------
  GcQueue, GcBlocks, GcGarbageCollected, GcState
-------------------------------------------------------------------------------}

-- | Queue of scheduled GCs, in reverse order
newtype GcQueue = GcQueue { GcQueue -> [ScheduledGc]
unGcQueue :: [ScheduledGc] }
  deriving newtype (GcQueue -> GcQueue -> Bool
(GcQueue -> GcQueue -> Bool)
-> (GcQueue -> GcQueue -> Bool) -> Eq GcQueue
forall a. (a -> a -> Bool) -> (a -> a -> Bool) -> Eq a
$c== :: GcQueue -> GcQueue -> Bool
== :: GcQueue -> GcQueue -> Bool
$c/= :: GcQueue -> GcQueue -> Bool
/= :: GcQueue -> GcQueue -> Bool
Eq, GcQueue -> TestName
(GcQueue -> TestName) -> Condense GcQueue
forall a. (a -> TestName) -> Condense a
$ccondense :: GcQueue -> TestName
condense :: GcQueue -> TestName
Condense)

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

-- | Blocks still to GC, together with the earliest time at which the block
-- could have been GC'ed.
--
-- In no particular order.
--
-- NOTE: in the real implementation, a GC for slot @s@ means removing all
-- blocks with a slot number < @s@ (because of EBBs, which share the slot with
-- the regular block after it). In this test, we ignore this and use <=, so a
-- GC for the slot of the block will remove the block.
newtype GcBlocks = GcBlocks { GcBlocks -> [(Block, Time)]
unGcBlocks :: [(Block, Time)] }
  deriving newtype (GcBlocks -> TestName
(GcBlocks -> TestName) -> Condense GcBlocks
forall a. (a -> TestName) -> Condense a
$ccondense :: GcBlocks -> TestName
condense :: GcBlocks -> TestName
Condense)

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

-- | Garbage collections that have happened. A garbage collection is triggered
-- for a slot number. We remember at which time it happened.
--
-- In no particular order.
--
-- The NOTE of 'GcBlocks' also applies here.
newtype GcGarbageCollections = GcGarbageCollections [(SlotNo, Time)]
  deriving newtype (GcGarbageCollections -> GcGarbageCollections -> Bool
(GcGarbageCollections -> GcGarbageCollections -> Bool)
-> (GcGarbageCollections -> GcGarbageCollections -> Bool)
-> Eq GcGarbageCollections
forall a. (a -> a -> Bool) -> (a -> a -> Bool) -> Eq a
$c== :: GcGarbageCollections -> GcGarbageCollections -> Bool
== :: GcGarbageCollections -> GcGarbageCollections -> Bool
$c/= :: GcGarbageCollections -> GcGarbageCollections -> Bool
/= :: GcGarbageCollections -> GcGarbageCollections -> Bool
Eq, GcGarbageCollections -> TestName
(GcGarbageCollections -> TestName) -> Condense GcGarbageCollections
forall a. (a -> TestName) -> Condense a
$ccondense :: GcGarbageCollections -> TestName
condense :: GcGarbageCollections -> TestName
Condense, Context -> GcGarbageCollections -> IO (Maybe ThunkInfo)
Proxy GcGarbageCollections -> TestName
(Context -> GcGarbageCollections -> IO (Maybe ThunkInfo))
-> (Context -> GcGarbageCollections -> IO (Maybe ThunkInfo))
-> (Proxy GcGarbageCollections -> TestName)
-> NoThunks GcGarbageCollections
forall a.
(Context -> a -> IO (Maybe ThunkInfo))
-> (Context -> a -> IO (Maybe ThunkInfo))
-> (Proxy a -> TestName)
-> NoThunks a
$cnoThunks :: Context -> GcGarbageCollections -> IO (Maybe ThunkInfo)
noThunks :: Context -> GcGarbageCollections -> IO (Maybe ThunkInfo)
$cwNoThunks :: Context -> GcGarbageCollections -> IO (Maybe ThunkInfo)
wNoThunks :: Context -> GcGarbageCollections -> IO (Maybe ThunkInfo)
$cshowTypeOf :: Proxy GcGarbageCollections -> TestName
showTypeOf :: Proxy GcGarbageCollections -> TestName
NoThunks)

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

data GcState = GcState {
      GcState -> GcQueue
gcQueue              :: GcQueue
    , GcState -> GcBlocks
gcBlocks             :: GcBlocks
    , GcState -> GcGarbageCollections
gcGarbageCollections :: GcGarbageCollections
    }
  deriving (Int -> GcState -> ShowS
[GcState] -> ShowS
GcState -> TestName
(Int -> GcState -> ShowS)
-> (GcState -> TestName) -> ([GcState] -> ShowS) -> Show GcState
forall a.
(Int -> a -> ShowS) -> (a -> TestName) -> ([a] -> ShowS) -> Show a
$cshowsPrec :: Int -> GcState -> ShowS
showsPrec :: Int -> GcState -> ShowS
$cshow :: GcState -> TestName
show :: GcState -> TestName
$cshowList :: [GcState] -> ShowS
showList :: [GcState] -> ShowS
Show)

emptyGcState :: GcState
emptyGcState :: GcState
emptyGcState =
    GcQueue -> GcBlocks -> GcGarbageCollections -> GcState
GcState
      ([ScheduledGc] -> GcQueue
GcQueue [])
      ([(Block, Time)] -> GcBlocks
GcBlocks [])
      ([(SlotNo, Time)] -> GcGarbageCollections
GcGarbageCollections [])

-- | The length of the queue
queueLength :: GcState -> Int
queueLength :: GcState -> Int
queueLength = [ScheduledGc] -> Int
forall a. [a] -> Int
forall (t :: * -> *) a. Foldable t => t a -> Int
length ([ScheduledGc] -> Int)
-> (GcState -> [ScheduledGc]) -> GcState -> Int
forall b c a. (b -> c) -> (a -> b) -> a -> c
. GcQueue -> [ScheduledGc]
unGcQueue (GcQueue -> [ScheduledGc])
-> (GcState -> GcQueue) -> GcState -> [ScheduledGc]
forall b c a. (b -> c) -> (a -> b) -> a -> c
. GcState -> GcQueue
gcQueue

-- | The overlap (number of blocks) between ImmutableDB and VolatileDB
overlap :: GcState -> Int
overlap :: GcState -> Int
overlap = [(Block, Time)] -> Int
forall a. [a] -> Int
forall (t :: * -> *) a. Foldable t => t a -> Int
length ([(Block, Time)] -> Int)
-> (GcState -> [(Block, Time)]) -> GcState -> Int
forall b c a. (b -> c) -> (a -> b) -> a -> c
. GcBlocks -> [(Block, Time)]
unGcBlocks (GcBlocks -> [(Block, Time)])
-> (GcState -> GcBlocks) -> GcState -> [(Block, Time)]
forall b c a. (b -> c) -> (a -> b) -> a -> c
. GcState -> GcBlocks
gcBlocks

-- | Number of blocks that could be GC'ed but haven't been
unnecessaryOverlap ::
     Time  -- ^ The current time
  -> GcState
  -> Int
unnecessaryOverlap :: Time -> GcState -> Int
unnecessaryOverlap Time
now =
    [(Block, Time)] -> Int
forall a. [a] -> Int
forall (t :: * -> *) a. Foldable t => t a -> Int
length ([(Block, Time)] -> Int)
-> (GcState -> [(Block, Time)]) -> GcState -> Int
forall b c a. (b -> c) -> (a -> b) -> a -> c
. ((Block, Time) -> Bool) -> [(Block, Time)] -> [(Block, Time)]
forall a. (a -> Bool) -> [a] -> [a]
filter ((Time -> Time -> Bool
forall a. Ord a => a -> a -> Bool
<= Time
now) (Time -> Bool) -> ((Block, Time) -> Time) -> (Block, Time) -> Bool
forall b c a. (b -> c) -> (a -> b) -> a -> c
. (Block, Time) -> Time
forall a b. (a, b) -> b
snd) ([(Block, Time)] -> [(Block, Time)])
-> (GcState -> [(Block, Time)]) -> GcState -> [(Block, Time)]
forall b c a. (b -> c) -> (a -> b) -> a -> c
. GcBlocks -> [(Block, Time)]
unGcBlocks (GcBlocks -> [(Block, Time)])
-> (GcState -> GcBlocks) -> GcState -> [(Block, Time)]
forall b c a. (b -> c) -> (a -> b) -> a -> c
. GcState -> GcBlocks
gcBlocks

-- | Run all garbage collections schedule before or at the given time.
runGc :: Time -> GcState -> GcState
runGc :: Time -> GcState -> GcState
runGc Time
now GcState
gcState = GcState {
      gcQueue :: GcQueue
gcQueue              = [ScheduledGc] -> GcQueue
GcQueue [ScheduledGc]
gcQueueLater
    , gcBlocks :: GcBlocks
gcBlocks             = case Maybe SlotNo
mbHighestGCedSlot of
        Maybe SlotNo
Nothing              -> GcState -> GcBlocks
gcBlocks GcState
gcState
        Just SlotNo
highestGCedSlot -> [(Block, Time)] -> GcBlocks
GcBlocks ([(Block, Time)] -> GcBlocks) -> [(Block, Time)] -> GcBlocks
forall a b. (a -> b) -> a -> b
$
          ((Block, Time) -> Bool) -> [(Block, Time)] -> [(Block, Time)]
forall a. (a -> Bool) -> [a] -> [a]
filter
            ((SlotNo -> SlotNo -> Bool
forall a. Ord a => a -> a -> Bool
> SlotNo
highestGCedSlot) (SlotNo -> Bool)
-> ((Block, Time) -> SlotNo) -> (Block, Time) -> Bool
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Block -> SlotNo
blockSlotNo (Block -> SlotNo)
-> ((Block, Time) -> Block) -> (Block, Time) -> SlotNo
forall b c a. (b -> c) -> (a -> b) -> a -> c
. (Block, Time) -> Block
forall a b. (a, b) -> a
fst)
            (GcBlocks -> [(Block, Time)]
unGcBlocks (GcState -> GcBlocks
gcBlocks GcState
gcState))
    , gcGarbageCollections :: GcGarbageCollections
gcGarbageCollections = [(SlotNo, Time)] -> GcGarbageCollections
GcGarbageCollections ([(SlotNo, Time)] -> GcGarbageCollections)
-> [(SlotNo, Time)] -> GcGarbageCollections
forall a b. (a -> b) -> a -> b
$
        (ScheduledGc -> (SlotNo, Time))
-> [ScheduledGc] -> [(SlotNo, Time)]
forall a b. (a -> b) -> [a] -> [b]
map ScheduledGc -> (SlotNo, Time)
toGarbageCollection [ScheduledGc]
gcQueueNow [(SlotNo, Time)] -> [(SlotNo, Time)] -> [(SlotNo, Time)]
forall a. Semigroup a => a -> a -> a
<> [(SlotNo, Time)]
pastGarbageCollections
    }
  where
    ([ScheduledGc]
gcQueueLater, [ScheduledGc]
gcQueueNow) =
      (ScheduledGc -> Bool)
-> [ScheduledGc] -> ([ScheduledGc], [ScheduledGc])
forall a. (a -> Bool) -> [a] -> ([a], [a])
partition ((Time -> Time -> Bool
forall a. Ord a => a -> a -> Bool
> Time
now) (Time -> Bool) -> (ScheduledGc -> Time) -> ScheduledGc -> Bool
forall b c a. (b -> c) -> (a -> b) -> a -> c
. ScheduledGc -> Time
scheduledGcTime) (GcQueue -> [ScheduledGc]
unGcQueue (GcState -> GcQueue
gcQueue GcState
gcState))
    mbHighestGCedSlot :: Maybe SlotNo
mbHighestGCedSlot = [SlotNo] -> Maybe SlotNo
forall a. Ord a => [a] -> Maybe a
safeMaximum ([SlotNo] -> Maybe SlotNo) -> [SlotNo] -> Maybe SlotNo
forall a b. (a -> b) -> a -> b
$ (ScheduledGc -> SlotNo) -> [ScheduledGc] -> [SlotNo]
forall a b. (a -> b) -> [a] -> [b]
map ScheduledGc -> SlotNo
scheduledGcSlot [ScheduledGc]
gcQueueNow
    GcGarbageCollections [(SlotNo, Time)]
pastGarbageCollections =
      GcState -> GcGarbageCollections
gcGarbageCollections GcState
gcState

    toGarbageCollection :: ScheduledGc -> (SlotNo, Time)
    toGarbageCollection :: ScheduledGc -> (SlotNo, Time)
toGarbageCollection (ScheduledGc Time
time SlotNo
slot) = (SlotNo
slot, Time
time)

step ::
     GcParams
  -> Block
  -> GcState
  -> GcState
step :: GcParams -> Block -> GcState -> GcState
step GcParams
gcParams Block
block =
    -- Note the two calls to 'runGc': we simulate the behaviour of two threads
    -- (schedule GCs, execute schedule) from this (single-threaded) function.
    --
    -- The first (innermost) 'runGc' is needed to run any outstanding GCs at
    -- @now@. In other words, we run the "execute schedule" thread. Otherwise,
    -- we will see GCs scheduled in the past in the queue when we schedule a
    -- new one.
    --
    -- The second (outermost) 'runGc' is needed to immediately run the
    -- scheduled GCs in case we have a 'gcDelay' of 0.
      Time -> GcState -> GcState
runGc Time
now
    (GcState -> GcState) -> (GcState -> GcState) -> GcState -> GcState
forall b c a. (b -> c) -> (a -> b) -> a -> c
. GcState -> GcState
schedule
    (GcState -> GcState) -> (GcState -> GcState) -> GcState -> GcState
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Time -> GcState -> GcState
runGc Time
now
  where
    slot :: SlotNo
slot = Block -> SlotNo
blockSlotNo Block
block
    now :: Time
now  = Block -> Time
blockArrivalTime Block
block

    schedule :: GcState -> GcState
    schedule :: GcState -> GcState
schedule GcState
gcState = GcState {
          gcQueue :: GcQueue
gcQueue              = [ScheduledGc] -> GcQueue
GcQueue [ScheduledGc]
gcQueue'
        , gcBlocks :: GcBlocks
gcBlocks             = [(Block, Time)] -> GcBlocks
GcBlocks ([(Block, Time)] -> GcBlocks) -> [(Block, Time)] -> GcBlocks
forall a b. (a -> b) -> a -> b
$
              (Block
block, GcParams -> DiffTime
gcDelay GcParams
gcParams DiffTime -> Time -> Time
`addTime` Time
now)
            (Block, Time) -> [(Block, Time)] -> [(Block, Time)]
forall a. a -> [a] -> [a]
: GcBlocks -> [(Block, Time)]
unGcBlocks (GcState -> GcBlocks
gcBlocks GcState
gcState)
        , gcGarbageCollections :: GcGarbageCollections
gcGarbageCollections = GcState -> GcGarbageCollections
gcGarbageCollections GcState
gcState
        }
      where
        scheduledTime :: Time
scheduledTime = GcParams -> Time -> Time
Impl.computeTimeForGC GcParams
gcParams Time
now
        gcQueue' :: [ScheduledGc]
gcQueue' = case GcQueue -> [ScheduledGc]
unGcQueue (GcState -> GcQueue
gcQueue GcState
gcState) of
          ScheduledGc Time
prevScheduledTime SlotNo
_prevSlot:[ScheduledGc]
queue'
            | Time
scheduledTime Time -> Time -> Bool
forall a. Eq a => a -> a -> Bool
== Time
prevScheduledTime
            -> Time -> SlotNo -> ScheduledGc
ScheduledGc Time
scheduledTime SlotNo
slotScheduledGc -> [ScheduledGc] -> [ScheduledGc]
forall a. a -> [a] -> [a]
:[ScheduledGc]
queue'
          [ScheduledGc]
queue
            -> Time -> SlotNo -> ScheduledGc
ScheduledGc Time
scheduledTime SlotNo
slotScheduledGc -> [ScheduledGc] -> [ScheduledGc]
forall a. a -> [a] -> [a]
:[ScheduledGc]
queue

{-------------------------------------------------------------------------------
  GcStateSummary
-------------------------------------------------------------------------------}

data GcStateSummary = GcStateSummary {
      GcStateSummary -> Time
gcSummaryNow         :: Time
    , GcStateSummary -> GcQueue
gcSummaryQueue       :: GcQueue
    , GcStateSummary -> Int
gcSummaryQueueLength :: Int
    , GcStateSummary -> Int
gcSummaryOverlap     :: Int
    , GcStateSummary -> Int
gcSummaryUnnecessary :: Int
    }
  deriving (Int -> GcStateSummary -> ShowS
Trace GcStateSummary -> ShowS
GcStateSummary -> TestName
(Int -> GcStateSummary -> ShowS)
-> (GcStateSummary -> TestName)
-> (Trace GcStateSummary -> ShowS)
-> Show GcStateSummary
forall a.
(Int -> a -> ShowS) -> (a -> TestName) -> ([a] -> ShowS) -> Show a
$cshowsPrec :: Int -> GcStateSummary -> ShowS
showsPrec :: Int -> GcStateSummary -> ShowS
$cshow :: GcStateSummary -> TestName
show :: GcStateSummary -> TestName
$cshowList :: Trace GcStateSummary -> ShowS
showList :: Trace GcStateSummary -> ShowS
Show)

computeGcStateSummary :: Time -> GcState -> GcStateSummary
computeGcStateSummary :: Time -> GcState -> GcStateSummary
computeGcStateSummary Time
now GcState
gcState = GcStateSummary {
      gcSummaryNow :: Time
gcSummaryNow         = Time
now
    , gcSummaryQueue :: GcQueue
gcSummaryQueue       = GcState -> GcQueue
gcQueue                GcState
gcState
    , gcSummaryQueueLength :: Int
gcSummaryQueueLength = GcState -> Int
queueLength            GcState
gcState
    , gcSummaryOverlap :: Int
gcSummaryOverlap     = GcState -> Int
overlap                GcState
gcState
    , gcSummaryUnnecessary :: Int
gcSummaryUnnecessary = Time -> GcState -> Int
unnecessaryOverlap Time
now GcState
gcState
    }

{-------------------------------------------------------------------------------
  Trace
-------------------------------------------------------------------------------}

type Trace a = [a]

computeTrace :: GcParams -> [Block] -> Trace (Time, GcState)
computeTrace :: GcParams -> [Block] -> Trace (Time, GcState)
computeTrace GcParams
gcParams [Block]
blocks =
    [Time] -> [GcState] -> Trace (Time, GcState)
forall a b. [a] -> [b] -> [(a, b)]
zip
      ((Block -> Time) -> [Block] -> [Time]
forall a b. (a -> b) -> [a] -> [b]
map Block -> Time
blockArrivalTime [Block]
blocks)
      -- Remember:
      -- scanl f z [x1, x2, ...] == [z, z `f` x1, (z `f` x1) `f` x2, ...]
      (Int -> [GcState] -> [GcState]
forall a. Int -> [a] -> [a]
drop Int
1 ((GcState -> Block -> GcState) -> GcState -> [Block] -> [GcState]
forall b a. (b -> a -> b) -> b -> [a] -> [b]
scanl ((Block -> GcState -> GcState) -> GcState -> Block -> GcState
forall a b c. (a -> b -> c) -> b -> a -> c
flip (GcParams -> Block -> GcState -> GcState
step GcParams
gcParams)) GcState
emptyGcState [Block]
blocks))

summarise :: GcParams -> Int -> Trace GcStateSummary
summarise :: GcParams -> Int -> Trace GcStateSummary
summarise GcParams
gcParams Int
numBlocks =
   ((Time, GcState) -> GcStateSummary)
-> Trace (Time, GcState) -> Trace GcStateSummary
forall a b. (a -> b) -> [a] -> [b]
map ((Time -> GcState -> GcStateSummary)
-> (Time, GcState) -> GcStateSummary
forall a b c. (a -> b -> c) -> (a, b) -> c
uncurry Time -> GcState -> GcStateSummary
computeGcStateSummary) (Trace (Time, GcState) -> Trace GcStateSummary)
-> Trace (Time, GcState) -> Trace GcStateSummary
forall a b. (a -> b) -> a -> b
$
     GcParams -> [Block] -> Trace (Time, GcState)
computeTrace GcParams
gcParams (Int -> [Block]
genBlocks Int
numBlocks)

example :: GcParams -> Trace GcStateSummary
example :: GcParams -> Trace GcStateSummary
example GcParams
gcParams = GcParams -> Int -> Trace GcStateSummary
summarise GcParams
gcParams Int
1000

-- | Process the remaining scheduled garbage collections in the queue. The
-- already performed garbage collections ('gcGarbageCollections') are included
-- in the final 'GcGarbageCollections'.
processQueueToEnd :: GcState -> GcGarbageCollections
processQueueToEnd :: GcState -> GcGarbageCollections
processQueueToEnd gcState :: GcState
gcState@GcState { gcQueue :: GcState -> GcQueue
gcQueue = GcQueue [ScheduledGc]
queue } =
    GcState -> GcGarbageCollections
gcGarbageCollections ((GcState -> Time -> GcState) -> GcState -> [Time] -> GcState
forall b a. (b -> a -> b) -> b -> [a] -> b
forall (t :: * -> *) b a.
Foldable t =>
(b -> a -> b) -> b -> t a -> b
List.foldl' ((Time -> GcState -> GcState) -> GcState -> Time -> GcState
forall a b c. (a -> b -> c) -> b -> a -> c
flip Time -> GcState -> GcState
runGc) GcState
gcState [Time]
timesToGcAt)
  where
    timesToGcAt :: [Time]
timesToGcAt = [Time] -> [Time]
forall a. Ord a => [a] -> [a]
sort ((ScheduledGc -> Time) -> [ScheduledGc] -> [Time]
forall a b. (a -> b) -> [a] -> [b]
map ScheduledGc -> Time
scheduledGcTime [ScheduledGc]
queue)

{-------------------------------------------------------------------------------
  Run the real GcSchedule
-------------------------------------------------------------------------------}

runGcSchedule :: GcParams -> [Block] -> (Trace GcQueue, GcGarbageCollections)
runGcSchedule :: GcParams -> [Block] -> ([GcQueue], GcGarbageCollections)
runGcSchedule GcParams
gcParams [Block]
blocks = (forall s. IOSim s ([GcQueue], GcGarbageCollections))
-> ([GcQueue], GcGarbageCollections)
forall a. (forall s. IOSim s a) -> a
runSimOrThrow IOSim s ([GcQueue], GcGarbageCollections)
forall s. IOSim s ([GcQueue], GcGarbageCollections)
forall (m :: * -> *).
IOLike m =>
m ([GcQueue], GcGarbageCollections)
test
  where
    test :: IOLike m => m (Trace GcQueue, GcGarbageCollections)
    test :: forall (m :: * -> *).
IOLike m =>
m ([GcQueue], GcGarbageCollections)
test = do
      StrictTVar m GcGarbageCollections
varGCs <- GcGarbageCollections -> m (StrictTVar m GcGarbageCollections)
forall (m :: * -> *) a. MonadSTM m => a -> m (StrictTVar m a)
uncheckedNewTVarM ([(SlotNo, Time)] -> GcGarbageCollections
GcGarbageCollections [])
      GcSchedule m
gcSchedule <- m (GcSchedule m)
forall (m :: * -> *). IOLike m => m (GcSchedule m)
Impl.newGcSchedule
      m Void
-> (Async m Void -> m ([GcQueue], GcGarbageCollections))
-> m ([GcQueue], GcGarbageCollections)
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 (StrictTVar m GcGarbageCollections -> GcSchedule m -> m Void
forall (m :: * -> *).
IOLike m =>
StrictTVar m GcGarbageCollections -> GcSchedule m -> m Void
gcThread StrictTVar m GcGarbageCollections
varGCs GcSchedule m
gcSchedule) ((Async m Void -> m ([GcQueue], GcGarbageCollections))
 -> m ([GcQueue], GcGarbageCollections))
-> (Async m Void -> m ([GcQueue], GcGarbageCollections))
-> m ([GcQueue], GcGarbageCollections)
forall a b. (a -> b) -> a -> b
$ \Async m Void
asyncGcThread -> do
        Async m Void -> m ()
forall (m :: * -> *) a.
(MonadAsync m, MonadFork m, MonadMask m) =>
Async m a -> m ()
link Async m Void
asyncGcThread

        [GcQueue]
gcQueueTrace <- [Block] -> (Block -> m GcQueue) -> m [GcQueue]
forall (t :: * -> *) (m :: * -> *) a b.
(Traversable t, Monad m) =>
t a -> (a -> m b) -> m (t b)
forM [Block]
blocks ((Block -> m GcQueue) -> m [GcQueue])
-> (Block -> m GcQueue) -> m [GcQueue]
forall a b. (a -> b) -> a -> b
$ \Block
block -> do
          Time -> m ()
forall (m :: * -> *). IOLike m => Time -> m ()
waitUntil (Block -> Time
blockArrivalTime Block
block)
          Tracer m (TraceGCEvent Any)
-> SlotNo -> GcParams -> GcSchedule m -> m ()
forall (m :: * -> *) blk.
IOLike m =>
Tracer m (TraceGCEvent blk)
-> SlotNo -> GcParams -> GcSchedule m -> m ()
Impl.scheduleGC Tracer m (TraceGCEvent Any)
forall (m :: * -> *) a. Applicative m => Tracer m a
nullTracer (Block -> SlotNo
blockSlotNo Block
block) GcParams
gcParams GcSchedule m
gcSchedule
          -- Just the minimal number of time so that the background thread
          -- gets its chance to run. Since this is the IO simulator, it will
          -- run instantly.
          DiffTime -> m ()
forall (m :: * -> *). MonadDelay m => DiffTime -> m ()
threadDelay (Integer -> DiffTime
picosecondsToDiffTime Integer
1)
          [ScheduledGc] -> GcQueue
GcQueue ([ScheduledGc] -> GcQueue) -> m [ScheduledGc] -> m GcQueue
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> STM m [ScheduledGc] -> m [ScheduledGc]
forall a. HasCallStack => STM m a -> m a
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
STM m a -> m a
atomically (GcSchedule m -> STM m [ScheduledGc]
forall (m :: * -> *).
IOLike m =>
GcSchedule m -> STM m [ScheduledGc]
Impl.dumpGcSchedule GcSchedule m
gcSchedule)

        -- Wait until the implementation's queue is empty
        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
          [ScheduledGc]
queue <- GcSchedule m -> STM m [ScheduledGc]
forall (m :: * -> *).
IOLike m =>
GcSchedule m -> STM m [ScheduledGc]
Impl.dumpGcSchedule GcSchedule m
gcSchedule
          Bool -> STM m ()
forall (m :: * -> *). MonadSTM m => Bool -> STM m ()
check ([ScheduledGc] -> Bool
forall a. [a] -> Bool
forall (t :: * -> *) a. Foldable t => t a -> Bool
null [ScheduledGc]
queue)

        Async m Void -> m ()
forall a. Async m a -> m ()
forall (m :: * -> *) a. MonadAsync m => Async m a -> m ()
cancel Async m Void
asyncGcThread
        GcGarbageCollections
gcs <- STM m GcGarbageCollections -> m GcGarbageCollections
forall a. HasCallStack => STM m a -> m a
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
STM m a -> m a
atomically (STM m GcGarbageCollections -> m GcGarbageCollections)
-> STM m GcGarbageCollections -> m GcGarbageCollections
forall a b. (a -> b) -> a -> b
$ StrictTVar m GcGarbageCollections -> STM m GcGarbageCollections
forall (m :: * -> *) a. MonadSTM m => StrictTVar m a -> STM m a
readTVar StrictTVar m GcGarbageCollections
varGCs
        ([GcQueue], GcGarbageCollections)
-> m ([GcQueue], GcGarbageCollections)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return ([GcQueue]
gcQueueTrace, GcGarbageCollections
gcs)

    gcThread
      :: IOLike m
      => StrictTVar m GcGarbageCollections
      -> Impl.GcSchedule m
      -> m Void
    gcThread :: forall (m :: * -> *).
IOLike m =>
StrictTVar m GcGarbageCollections -> GcSchedule m -> m Void
gcThread StrictTVar m GcGarbageCollections
varGCs GcSchedule m
gcSchedule =
      GcSchedule m -> (SlotNo -> m ()) -> m Void
forall (m :: * -> *).
IOLike m =>
GcSchedule m -> (SlotNo -> m ()) -> m Void
Impl.gcScheduleRunner GcSchedule m
gcSchedule ((SlotNo -> m ()) -> m Void) -> (SlotNo -> m ()) -> m Void
forall a b. (a -> b) -> a -> b
$ \SlotNo
slotNo -> do
        -- Record the time at which a GC for @slotNo@ was triggered in a TVar
        Time
now <- m Time
forall (m :: * -> *). MonadMonotonicTime m => m Time
getMonotonicTime
        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
$ StrictTVar m GcGarbageCollections
-> (GcGarbageCollections -> GcGarbageCollections) -> STM m ()
forall (m :: * -> *) a.
MonadSTM m =>
StrictTVar m a -> (a -> a) -> STM m ()
modifyTVar StrictTVar m GcGarbageCollections
varGCs ((GcGarbageCollections -> GcGarbageCollections) -> STM m ())
-> (GcGarbageCollections -> GcGarbageCollections) -> STM m ()
forall a b. (a -> b) -> a -> b
$ \(GcGarbageCollections [(SlotNo, Time)]
gcs) ->
          [(SlotNo, Time)] -> GcGarbageCollections
GcGarbageCollections ([(SlotNo, Time)] -> GcGarbageCollections)
-> [(SlotNo, Time)] -> GcGarbageCollections
forall a b. (a -> b) -> a -> b
$ (SlotNo
slotNo, Time
now) (SlotNo, Time) -> [(SlotNo, Time)] -> [(SlotNo, Time)]
forall a. a -> [a] -> [a]
: [(SlotNo, Time)]
gcs

    waitUntil :: IOLike m => Time -> m ()
    waitUntil :: forall (m :: * -> *). IOLike m => Time -> m ()
waitUntil Time
t = do
      Time
now <- m Time
forall (m :: * -> *). MonadMonotonicTime m => m Time
getMonotonicTime
      let toWait :: DiffTime
toWait = DiffTime -> DiffTime -> DiffTime
forall a. Ord a => a -> a -> a
max DiffTime
0 (Time
t Time -> Time -> DiffTime
`diffTime` Time
now)
      DiffTime -> m ()
forall (m :: * -> *). MonadDelay m => DiffTime -> m ()
threadDelay DiffTime
toWait

{-------------------------------------------------------------------------------
  TestSetup
-------------------------------------------------------------------------------}

data TestSetup = TestSetup {
    -- | Number of blocks
    --
    -- This determines the length of the trace. Shrinking this value means
    -- we find the smallest trace that yields the error
    TestSetup -> Int
testNumBlocks            :: Int

    -- | GC delay in seconds
    --
    -- We keep this as a separate value /in seconds/ so that (1) it is easily
    -- shrinkable and (2) we can meaningfully use 'blocksInInterval'
  , TestSetup -> Integer
testDelay                :: Integer

    -- | GC interval in seconds
    --
    -- See 'testDelay'
  , TestSetup -> Integer
testInterval             :: Integer

    -- Derived
  , TestSetup -> GcParams
testGcParams             :: GcParams
  , TestSetup -> Trace GcStateSummary
testTrace                :: Trace GcStateSummary
    -- | The garbage collections that will have been performed after
    -- processing the whole queue.
  , TestSetup -> GcGarbageCollections
testGcGarbageCollections :: GcGarbageCollections
  }
  deriving (Int -> TestSetup -> ShowS
[TestSetup] -> ShowS
TestSetup -> TestName
(Int -> TestSetup -> ShowS)
-> (TestSetup -> TestName)
-> ([TestSetup] -> ShowS)
-> Show TestSetup
forall a.
(Int -> a -> ShowS) -> (a -> TestName) -> ([a] -> ShowS) -> Show a
$cshowsPrec :: Int -> TestSetup -> ShowS
showsPrec :: Int -> TestSetup -> ShowS
$cshow :: TestSetup -> TestName
show :: TestSetup -> TestName
$cshowList :: [TestSetup] -> ShowS
showList :: [TestSetup] -> ShowS
Show)

genBlocks :: Int -> [Block]
genBlocks :: Int -> [Block]
genBlocks Int
numBlocks = (Int -> Block) -> [Int] -> [Block]
forall a b. (a -> b) -> [a] -> [b]
map Int -> Block
Block [Int
1..Int
numBlocks]

mkTestSetup :: Int -> Integer -> Integer -> TestSetup
mkTestSetup :: Int -> Integer -> Integer -> TestSetup
mkTestSetup Int
numBlocks Integer
delay Integer
interval = TestSetup {
      testNumBlocks :: Int
testNumBlocks            = Int
numBlocks
    , testDelay :: Integer
testDelay                = Integer
delay
    , testInterval :: Integer
testInterval             = Integer
interval
      -- Derived values
    , testGcParams :: GcParams
testGcParams             = GcParams
gcParams
    , testTrace :: Trace GcStateSummary
testTrace                = ((Time, GcState) -> GcStateSummary)
-> Trace (Time, GcState) -> Trace GcStateSummary
forall a b. (a -> b) -> [a] -> [b]
map ((Time -> GcState -> GcStateSummary)
-> (Time, GcState) -> GcStateSummary
forall a b c. (a -> b -> c) -> (a, b) -> c
uncurry Time -> GcState -> GcStateSummary
computeGcStateSummary) Trace (Time, GcState)
trace
    , testGcGarbageCollections :: GcGarbageCollections
testGcGarbageCollections = GcState -> GcGarbageCollections
processQueueToEnd GcState
finalState
    }
  where
    trace :: Trace (Time, GcState)
trace = GcParams -> [Block] -> Trace (Time, GcState)
computeTrace GcParams
gcParams (Int -> [Block]
genBlocks Int
numBlocks)

    finalState :: GcState
finalState = GcState
-> ((Time, GcState) -> GcState) -> Maybe (Time, GcState) -> GcState
forall b a. b -> (a -> b) -> Maybe a -> b
maybe GcState
emptyGcState (Time, GcState) -> GcState
forall a b. (a, b) -> b
snd (Trace (Time, GcState) -> Maybe (Time, GcState)
forall a. [a] -> Maybe a
lastMaybe Trace (Time, GcState)
trace)

    gcParams :: GcParams
    gcParams :: GcParams
gcParams = GcParams {
          gcDelay :: DiffTime
gcDelay    = Integer -> DiffTime
secondsToDiffTime Integer
delay
        , gcInterval :: DiffTime
gcInterval = Integer -> DiffTime
secondsToDiffTime Integer
interval
        }


instance Arbitrary TestSetup where
  arbitrary :: Gen TestSetup
arbitrary =
      Int -> Integer -> Integer -> TestSetup
mkTestSetup
        (Int -> Integer -> Integer -> TestSetup)
-> Gen Int -> Gen (Integer -> Integer -> TestSetup)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> ((Int -> Int -> Int
forall a. Num a => a -> a -> a
* Int
10) (Int -> Int) -> Gen Int -> Gen Int
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Gen Int
getSize) -- Number of blocks
        Gen (Integer -> Integer -> TestSetup)
-> Gen Integer -> Gen (Integer -> TestSetup)
forall a b. Gen (a -> b) -> Gen a -> Gen b
forall (f :: * -> *) a b. Applicative f => f (a -> b) -> f a -> f b
<*> (Integer, Integer) -> Gen Integer
forall a. Random a => (a, a) -> Gen a
choose (Integer
0, Integer
100)      -- Delay
        Gen (Integer -> TestSetup) -> Gen Integer -> Gen TestSetup
forall a b. Gen (a -> b) -> Gen a -> Gen b
forall (f :: * -> *) a b. Applicative f => f (a -> b) -> f a -> f b
<*> (Integer, Integer) -> Gen Integer
forall a. Random a => (a, a) -> Gen a
choose (Integer
1, Integer
120)      -- Interval

  shrink :: TestSetup -> [TestSetup]
shrink TestSetup{Int
Integer
Trace GcStateSummary
GcParams
GcGarbageCollections
testNumBlocks :: TestSetup -> Int
testDelay :: TestSetup -> Integer
testInterval :: TestSetup -> Integer
testGcParams :: TestSetup -> GcParams
testTrace :: TestSetup -> Trace GcStateSummary
testGcGarbageCollections :: TestSetup -> GcGarbageCollections
testNumBlocks :: Int
testDelay :: Integer
testInterval :: Integer
testGcParams :: GcParams
testTrace :: Trace GcStateSummary
testGcGarbageCollections :: GcGarbageCollections
..} = [[TestSetup]] -> [TestSetup]
forall (t :: * -> *) a. Foldable t => t [a] -> [a]
concat [
        [ Int -> Integer -> Integer -> TestSetup
mkTestSetup Int
testNumBlocks' Integer
testDelay Integer
testInterval
        | Int
testNumBlocks' <- Int -> [Int]
forall a. Arbitrary a => a -> [a]
shrink Int
testNumBlocks
        ]

      , [ Int -> Integer -> Integer -> TestSetup
mkTestSetup Int
testNumBlocks Integer
testDelay' Integer
testInterval
        | Integer
testDelay' <- Integer -> [Integer]
forall a. Arbitrary a => a -> [a]
shrink Integer
testDelay
        ]

      , [ Int -> Integer -> Integer -> TestSetup
mkTestSetup Int
testNumBlocks Integer
testDelay Integer
testInterval'
        | Integer
testInterval' <- Integer -> [Integer]
forall a. Arbitrary a => a -> [a]
shrink Integer
testInterval
        , Integer
testInterval' Integer -> Integer -> Bool
forall a. Ord a => a -> a -> Bool
> Integer
0
        ]

        -- Shrink two values shrink /together/
        -- Note: we don't compute all possible combinations, we shrink both
      , [ Int -> Integer -> Integer -> TestSetup
mkTestSetup Int
testNumBlocks Integer
testDelay' Integer
testInterval'
        | Integer
testDelay    Integer -> Integer -> Bool
forall a. Ord a => a -> a -> Bool
> Integer
0
        , Integer
testInterval Integer -> Integer -> Bool
forall a. Ord a => a -> a -> Bool
> Integer
1
        , let testDelay' :: Integer
testDelay'    = Integer
testDelay    Integer -> Integer -> Integer
forall a. Num a => a -> a -> a
- Integer
1
        , let testInterval' :: Integer
testInterval' = Integer
testInterval Integer -> Integer -> Integer
forall a. Num a => a -> a -> a
- Integer
1
        ]
      ]