{-# LANGUAGE ConstraintKinds #-}
{-# LANGUAGE DataKinds #-}
{-# LANGUAGE DeriveAnyClass #-}
{-# LANGUAGE DeriveGeneric #-}
{-# LANGUAGE DeriveTraversable #-}
{-# LANGUAGE DerivingVia #-}
{-# LANGUAGE DuplicateRecordFields #-}
{-# LANGUAGE FlexibleContexts #-}
{-# LANGUAGE FlexibleInstances #-}
{-# LANGUAGE GeneralizedNewtypeDeriving #-}
{-# LANGUAGE LambdaCase #-}
{-# LANGUAGE NamedFieldPuns #-}
{-# LANGUAGE RecordWildCards #-}
{-# LANGUAGE ScopedTypeVariables #-}
{-# LANGUAGE StandaloneDeriving #-}
{-# LANGUAGE TemplateHaskell #-}
{-# LANGUAGE TupleSections #-}
{-# LANGUAGE TypeApplications #-}
{-# LANGUAGE TypeFamilies #-}
{-# LANGUAGE UndecidableInstances #-}

{-# OPTIONS_GHC -Wno-orphans #-}

-- | Main tests for the chain DB.
--
-- These are the main tests for the chain DB. Commands include
--
-- * Add a block
-- * Add a block with a @SlotNo@ that is ahead of the wall-clock.
-- * Get the current chain and/or ledger state
-- * Create a new iterator and use it to stream blocks
-- * Create a new follower and use it to follow the chain
-- * (Limited) disk corruption (the chain DB relies on the immutable DB and
--   volatile DB for the storage proper and /they/ have extensive disk corruption
--   tests, so we don't need to repeat that here).
--
-- Note that it is important to tests blocks with a @SlotNo@ ahead of the
-- wallclock separately, because the Ouroboros protocol says such blocks should
-- not be adopted, but we do want to allow for some clock skew in upstream nodes;
-- this means that such "blocks from the future" are stored without being added to
-- the chain just yet, to be considered later. Moreover, we have to be very careful
-- in how we do this "from the future" check; for example, if the ledger state is
-- far behind the wallclock, we might not have sufficient knowledge to translate
-- the wallclock to a @SlotNo@, although we /can/ always translate the @SlotNo@
-- at the tip of the chain to a @UTCTime@.
--
module Test.Ouroboros.Storage.ChainDB.StateMachine (
    -- * Commands
    At (..)
  , Cmd (..)
  , FollowerRef
  , IterRef
  , IteratorResult (..)
  , IteratorResultGCed (..)
    -- * Responses
  , Resp (..)
  , Success (..)
    -- * Model
  , MaxClockSkew (..)
  , Model
  , ShouldGarbageCollect (..)
    -- * Running the model
  , runCmdsLockstep
    -- * System under test
  , ChainDBEnv (..)
  , ChainDBState (..)
  , close
  , mkTestCfg
  , open
  , persistBlks
    -- * Specifying block components
  , AllComponents
  , allComponents
    -- * Constraints
  , TestConstraints
    -- * Tracing
  , traceEventName
    -- * Entry point to the tests
  , tests
  ) where

import           Codec.Serialise (Serialise)
import           Control.Monad (replicateM, void)
import           Control.Tracer as CT
import           Data.Bifoldable
import           Data.Bifunctor
import qualified Data.Bifunctor.TH as TH
import           Data.Bitraversable
import           Data.ByteString.Lazy (ByteString)
import           Data.Function (on)
import           Data.Functor (($>))
import           Data.Functor.Classes (Eq1, Show1)
import           Data.Functor.Identity (Identity)
import           Data.List (sortOn)
import qualified Data.List.NonEmpty as NE
import qualified Data.Map.Strict as Map
import           Data.Maybe (fromMaybe)
import           Data.Ord (Down (..))
import           Data.Proxy
import           Data.TreeDiff
import           Data.Typeable
import           Data.Void (Void)
import           Data.Word (Word16, Word64)
import qualified Generics.SOP as SOP
import           GHC.Generics (Generic)
import           NoThunks.Class (AllowThunk (..))
import           Ouroboros.Consensus.Block
import           Ouroboros.Consensus.Config
import qualified Ouroboros.Consensus.Fragment.InFuture as InFuture
import           Ouroboros.Consensus.HardFork.Abstract
import           Ouroboros.Consensus.HeaderValidation
import           Ouroboros.Consensus.Ledger.Abstract
import           Ouroboros.Consensus.Ledger.Extended
import           Ouroboros.Consensus.Ledger.Inspect
import           Ouroboros.Consensus.Ledger.SupportsProtocol
import           Ouroboros.Consensus.Protocol.Abstract
import           Ouroboros.Consensus.Storage.ChainDB hiding
                     (TraceFollowerEvent (..))
import qualified Ouroboros.Consensus.Storage.ChainDB as ChainDB
import qualified Ouroboros.Consensus.Storage.ChainDB.API.Types.InvalidBlockPunishment as InvalidBlockPunishment
import qualified Ouroboros.Consensus.Storage.ChainDB.Impl.Args as ChainDB
import           Ouroboros.Consensus.Storage.Common (SizeInBytes)
import qualified Ouroboros.Consensus.Storage.ImmutableDB as ImmutableDB
import           Ouroboros.Consensus.Storage.ImmutableDB.Chunks.Internal
                     (unsafeChunkNoToEpochNo)
import           Ouroboros.Consensus.Storage.LedgerDB (LedgerDB)
import qualified Ouroboros.Consensus.Storage.LedgerDB as LedgerDB
import qualified Ouroboros.Consensus.Storage.VolatileDB as VolatileDB
import           Ouroboros.Consensus.Util (split)
import           Ouroboros.Consensus.Util.CallStack
import           Ouroboros.Consensus.Util.Condense (condense)
import           Ouroboros.Consensus.Util.Enclose
import           Ouroboros.Consensus.Util.IOLike hiding (invariant)
import           Ouroboros.Consensus.Util.ResourceRegistry
import           Ouroboros.Network.AnchoredFragment (AnchoredFragment)
import qualified Ouroboros.Network.AnchoredFragment as AF
import           Ouroboros.Network.Block (ChainUpdate, MaxSlotNo)
import qualified Ouroboros.Network.Mock.Chain as Chain
import qualified System.FS.Sim.MockFS as Mock
import           System.FS.Sim.MockFS (MockFS)
import qualified Test.Ouroboros.Storage.ChainDB.Model as Model
import           Test.Ouroboros.Storage.ChainDB.Model (FollowerId, IteratorId,
                     ModelSupportsBlock,
                     ShouldGarbageCollect (DoNotGarbageCollect, GarbageCollect))
import           Test.Ouroboros.Storage.Orphans ()
import           Test.Ouroboros.Storage.TestBlock
import           Test.QuickCheck hiding (elements, forAll)
import qualified Test.QuickCheck.Monadic as QC
import           Test.StateMachine
import qualified Test.StateMachine.Labelling as C
import qualified Test.StateMachine.Sequential as QSM
import qualified Test.StateMachine.Types as QSM
import qualified Test.StateMachine.Types.Rank2 as Rank2
import           Test.Tasty (TestTree, testGroup)
import           Test.Tasty.QuickCheck (testProperty)
import           Test.Util.ChainDB
import           Test.Util.ChunkInfo
import           Test.Util.Orphans.Arbitrary ()
import           Test.Util.Orphans.ToExpr ()
import           Test.Util.QuickCheck
import qualified Test.Util.RefEnv as RE
import           Test.Util.RefEnv (RefEnv)
import           Test.Util.SOP
import           Test.Util.TestEnv (adjustQuickCheckTests)
import           Test.Util.ToExpr ()
import           Test.Util.Tracer (recordingTracerIORef)
import           Test.Util.WithEq

{-------------------------------------------------------------------------------
  Abstract model
-------------------------------------------------------------------------------}

-- | Commands
data Cmd blk it flr
  = AddBlock       blk
    -- ^ Advance the current slot to the block's slot (unless smaller than the
    -- current slot), add the block and run chain selection.
  | AddFutureBlock blk SlotNo
    -- ^ Advance the current slot to the given slot, which is guaranteed to be
    -- smaller than the block's slot number (such that the block is from the
    -- future) and larger or equal to the current slot, and add the block.
  | GetCurrentChain
  | GetLedgerDB
  | GetTipBlock
  | GetTipHeader
  | GetTipPoint
  | GetBlockComponent     (RealPoint blk)
  | GetGCedBlockComponent (RealPoint blk)
    -- ^ Only for blocks that may have been garbage collected.
  | GetMaxSlotNo
  | GetIsValid            (RealPoint blk)
  | Stream                (StreamFrom blk) (StreamTo blk)
  | UpdateLoE             (AnchoredFragment blk)
    -- ^ Update the LoE fragment and run chain selection.
  | IteratorNext          it
  | IteratorNextGCed      it
    -- ^ Only for blocks that may have been garbage collected.
  | IteratorClose         it
  | NewFollower           ChainType
  | FollowerInstruction   flr
    -- ^ 'followerInstructionBlocking' is excluded, as it requires multiple
    -- threads. Its code path is pretty much the same as 'followerInstruction'
    -- anyway.
  | FollowerForward       flr [Point blk]
  | FollowerClose         flr
  | Close
  | Reopen

    -- Internal
  | PersistBlks
    -- ^ Copy the blocks older than @k@ from the Volatile DB to the Immutable
    -- DB.
  | PersistBlksThenGC
    -- ^ Copy the blocks older than @k@ from the Volatile DB to the Immutable
    -- DB __and then__ perform garbage colllection.
    --
    -- The garbage collection procedure of the Chain DB (our system under test)
    -- removes the blocks from the volatile DB __without__ caring about whether
    -- the removed blocks were persisted. Therefore, this part of the Chain DB
    -- logic assumes that copy to the immutable DB took place __before__
    -- garbage collection. The model uses this assumption as well. As a result,
    -- we cannot perform garbage collection in isolation, since this will break
    -- the model's 'invariant'.
  | UpdateLedgerSnapshots
    -- ^ Write a new 'LedgerDB' snapshot to disk and remove the oldest ones.

    -- Corruption
  | WipeVolatileDB
  deriving ((forall x. Cmd blk it flr -> Rep (Cmd blk it flr) x)
-> (forall x. Rep (Cmd blk it flr) x -> Cmd blk it flr)
-> Generic (Cmd blk it flr)
forall x. Rep (Cmd blk it flr) x -> Cmd blk it flr
forall x. Cmd blk it flr -> Rep (Cmd blk it flr) x
forall a.
(forall x. a -> Rep a x) -> (forall x. Rep a x -> a) -> Generic a
forall blk it flr x. Rep (Cmd blk it flr) x -> Cmd blk it flr
forall blk it flr x. Cmd blk it flr -> Rep (Cmd blk it flr) x
$cfrom :: forall blk it flr x. Cmd blk it flr -> Rep (Cmd blk it flr) x
from :: forall x. Cmd blk it flr -> Rep (Cmd blk it flr) x
$cto :: forall blk it flr x. Rep (Cmd blk it flr) x -> Cmd blk it flr
to :: forall x. Rep (Cmd blk it flr) x -> Cmd blk it flr
Generic, Int -> Cmd blk it flr -> ShowS
[Cmd blk it flr] -> ShowS
Cmd blk it flr -> String
(Int -> Cmd blk it flr -> ShowS)
-> (Cmd blk it flr -> String)
-> ([Cmd blk it flr] -> ShowS)
-> Show (Cmd blk it flr)
forall a.
(Int -> a -> ShowS) -> (a -> String) -> ([a] -> ShowS) -> Show a
forall blk it flr.
(StandardHash blk, Show blk, Show it, Show flr) =>
Int -> Cmd blk it flr -> ShowS
forall blk it flr.
(StandardHash blk, Show blk, Show it, Show flr) =>
[Cmd blk it flr] -> ShowS
forall blk it flr.
(StandardHash blk, Show blk, Show it, Show flr) =>
Cmd blk it flr -> String
$cshowsPrec :: forall blk it flr.
(StandardHash blk, Show blk, Show it, Show flr) =>
Int -> Cmd blk it flr -> ShowS
showsPrec :: Int -> Cmd blk it flr -> ShowS
$cshow :: forall blk it flr.
(StandardHash blk, Show blk, Show it, Show flr) =>
Cmd blk it flr -> String
show :: Cmd blk it flr -> String
$cshowList :: forall blk it flr.
(StandardHash blk, Show blk, Show it, Show flr) =>
[Cmd blk it flr] -> ShowS
showList :: [Cmd blk it flr] -> ShowS
Show, (forall a b. (a -> b) -> Cmd blk it a -> Cmd blk it b)
-> (forall a b. a -> Cmd blk it b -> Cmd blk it a)
-> Functor (Cmd blk it)
forall a b. a -> Cmd blk it b -> Cmd blk it a
forall a b. (a -> b) -> Cmd blk it a -> Cmd blk it b
forall blk it a b. a -> Cmd blk it b -> Cmd blk it a
forall blk it a b. (a -> b) -> Cmd blk it a -> Cmd blk it b
forall (f :: * -> *).
(forall a b. (a -> b) -> f a -> f b)
-> (forall a b. a -> f b -> f a) -> Functor f
$cfmap :: forall blk it a b. (a -> b) -> Cmd blk it a -> Cmd blk it b
fmap :: forall a b. (a -> b) -> Cmd blk it a -> Cmd blk it b
$c<$ :: forall blk it a b. a -> Cmd blk it b -> Cmd blk it a
<$ :: forall a b. a -> Cmd blk it b -> Cmd blk it a
Functor, (forall m. Monoid m => Cmd blk it m -> m)
-> (forall m a. Monoid m => (a -> m) -> Cmd blk it a -> m)
-> (forall m a. Monoid m => (a -> m) -> Cmd blk it a -> m)
-> (forall a b. (a -> b -> b) -> b -> Cmd blk it a -> b)
-> (forall a b. (a -> b -> b) -> b -> Cmd blk it a -> b)
-> (forall b a. (b -> a -> b) -> b -> Cmd blk it a -> b)
-> (forall b a. (b -> a -> b) -> b -> Cmd blk it a -> b)
-> (forall a. (a -> a -> a) -> Cmd blk it a -> a)
-> (forall a. (a -> a -> a) -> Cmd blk it a -> a)
-> (forall a. Cmd blk it a -> [a])
-> (forall a. Cmd blk it a -> Bool)
-> (forall a. Cmd blk it a -> Int)
-> (forall a. Eq a => a -> Cmd blk it a -> Bool)
-> (forall a. Ord a => Cmd blk it a -> a)
-> (forall a. Ord a => Cmd blk it a -> a)
-> (forall a. Num a => Cmd blk it a -> a)
-> (forall a. Num a => Cmd blk it a -> a)
-> Foldable (Cmd blk it)
forall a. Eq a => a -> Cmd blk it a -> Bool
forall a. Num a => Cmd blk it a -> a
forall a. Ord a => Cmd blk it a -> a
forall m. Monoid m => Cmd blk it m -> m
forall a. Cmd blk it a -> Bool
forall a. Cmd blk it a -> Int
forall a. Cmd blk it a -> [a]
forall a. (a -> a -> a) -> Cmd blk it a -> a
forall m a. Monoid m => (a -> m) -> Cmd blk it a -> m
forall b a. (b -> a -> b) -> b -> Cmd blk it a -> b
forall a b. (a -> b -> b) -> b -> Cmd blk it a -> b
forall blk it a. Eq a => a -> Cmd blk it a -> Bool
forall blk it a. Num a => Cmd blk it a -> a
forall blk it a. Ord a => Cmd blk it a -> a
forall blk it m. Monoid m => Cmd blk it m -> m
forall blk it a. Cmd blk it a -> Bool
forall blk it a. Cmd blk it a -> Int
forall blk it a. Cmd blk it a -> [a]
forall blk it a. (a -> a -> a) -> Cmd blk it a -> a
forall blk it m a. Monoid m => (a -> m) -> Cmd blk it a -> m
forall blk it b a. (b -> a -> b) -> b -> Cmd blk it a -> b
forall blk it a b. (a -> b -> b) -> b -> Cmd blk it a -> b
forall (t :: * -> *).
(forall m. Monoid m => t m -> m)
-> (forall m a. Monoid m => (a -> m) -> t a -> m)
-> (forall m a. Monoid m => (a -> m) -> t a -> m)
-> (forall a b. (a -> b -> b) -> b -> t a -> b)
-> (forall a b. (a -> b -> b) -> b -> t a -> b)
-> (forall b a. (b -> a -> b) -> b -> t a -> b)
-> (forall b a. (b -> a -> b) -> b -> t a -> b)
-> (forall a. (a -> a -> a) -> t a -> a)
-> (forall a. (a -> a -> a) -> t a -> a)
-> (forall a. t a -> [a])
-> (forall a. t a -> Bool)
-> (forall a. t a -> Int)
-> (forall a. Eq a => a -> t a -> Bool)
-> (forall a. Ord a => t a -> a)
-> (forall a. Ord a => t a -> a)
-> (forall a. Num a => t a -> a)
-> (forall a. Num a => t a -> a)
-> Foldable t
$cfold :: forall blk it m. Monoid m => Cmd blk it m -> m
fold :: forall m. Monoid m => Cmd blk it m -> m
$cfoldMap :: forall blk it m a. Monoid m => (a -> m) -> Cmd blk it a -> m
foldMap :: forall m a. Monoid m => (a -> m) -> Cmd blk it a -> m
$cfoldMap' :: forall blk it m a. Monoid m => (a -> m) -> Cmd blk it a -> m
foldMap' :: forall m a. Monoid m => (a -> m) -> Cmd blk it a -> m
$cfoldr :: forall blk it a b. (a -> b -> b) -> b -> Cmd blk it a -> b
foldr :: forall a b. (a -> b -> b) -> b -> Cmd blk it a -> b
$cfoldr' :: forall blk it a b. (a -> b -> b) -> b -> Cmd blk it a -> b
foldr' :: forall a b. (a -> b -> b) -> b -> Cmd blk it a -> b
$cfoldl :: forall blk it b a. (b -> a -> b) -> b -> Cmd blk it a -> b
foldl :: forall b a. (b -> a -> b) -> b -> Cmd blk it a -> b
$cfoldl' :: forall blk it b a. (b -> a -> b) -> b -> Cmd blk it a -> b
foldl' :: forall b a. (b -> a -> b) -> b -> Cmd blk it a -> b
$cfoldr1 :: forall blk it a. (a -> a -> a) -> Cmd blk it a -> a
foldr1 :: forall a. (a -> a -> a) -> Cmd blk it a -> a
$cfoldl1 :: forall blk it a. (a -> a -> a) -> Cmd blk it a -> a
foldl1 :: forall a. (a -> a -> a) -> Cmd blk it a -> a
$ctoList :: forall blk it a. Cmd blk it a -> [a]
toList :: forall a. Cmd blk it a -> [a]
$cnull :: forall blk it a. Cmd blk it a -> Bool
null :: forall a. Cmd blk it a -> Bool
$clength :: forall blk it a. Cmd blk it a -> Int
length :: forall a. Cmd blk it a -> Int
$celem :: forall blk it a. Eq a => a -> Cmd blk it a -> Bool
elem :: forall a. Eq a => a -> Cmd blk it a -> Bool
$cmaximum :: forall blk it a. Ord a => Cmd blk it a -> a
maximum :: forall a. Ord a => Cmd blk it a -> a
$cminimum :: forall blk it a. Ord a => Cmd blk it a -> a
minimum :: forall a. Ord a => Cmd blk it a -> a
$csum :: forall blk it a. Num a => Cmd blk it a -> a
sum :: forall a. Num a => Cmd blk it a -> a
$cproduct :: forall blk it a. Num a => Cmd blk it a -> a
product :: forall a. Num a => Cmd blk it a -> a
Foldable, Functor (Cmd blk it)
Foldable (Cmd blk it)
(Functor (Cmd blk it), Foldable (Cmd blk it)) =>
(forall (f :: * -> *) a b.
 Applicative f =>
 (a -> f b) -> Cmd blk it a -> f (Cmd blk it b))
-> (forall (f :: * -> *) a.
    Applicative f =>
    Cmd blk it (f a) -> f (Cmd blk it a))
-> (forall (m :: * -> *) a b.
    Monad m =>
    (a -> m b) -> Cmd blk it a -> m (Cmd blk it b))
-> (forall (m :: * -> *) a.
    Monad m =>
    Cmd blk it (m a) -> m (Cmd blk it a))
-> Traversable (Cmd blk it)
forall blk it. Functor (Cmd blk it)
forall blk it. Foldable (Cmd blk it)
forall blk it (m :: * -> *) a.
Monad m =>
Cmd blk it (m a) -> m (Cmd blk it a)
forall blk it (f :: * -> *) a.
Applicative f =>
Cmd blk it (f a) -> f (Cmd blk it a)
forall blk it (m :: * -> *) a b.
Monad m =>
(a -> m b) -> Cmd blk it a -> m (Cmd blk it b)
forall blk it (f :: * -> *) a b.
Applicative f =>
(a -> f b) -> Cmd blk it a -> f (Cmd blk it b)
forall (t :: * -> *).
(Functor t, Foldable t) =>
(forall (f :: * -> *) a b.
 Applicative f =>
 (a -> f b) -> t a -> f (t b))
-> (forall (f :: * -> *) a. Applicative f => t (f a) -> f (t a))
-> (forall (m :: * -> *) a b.
    Monad m =>
    (a -> m b) -> t a -> m (t b))
-> (forall (m :: * -> *) a. Monad m => t (m a) -> m (t a))
-> Traversable t
forall (m :: * -> *) a.
Monad m =>
Cmd blk it (m a) -> m (Cmd blk it a)
forall (f :: * -> *) a.
Applicative f =>
Cmd blk it (f a) -> f (Cmd blk it a)
forall (m :: * -> *) a b.
Monad m =>
(a -> m b) -> Cmd blk it a -> m (Cmd blk it b)
forall (f :: * -> *) a b.
Applicative f =>
(a -> f b) -> Cmd blk it a -> f (Cmd blk it b)
$ctraverse :: forall blk it (f :: * -> *) a b.
Applicative f =>
(a -> f b) -> Cmd blk it a -> f (Cmd blk it b)
traverse :: forall (f :: * -> *) a b.
Applicative f =>
(a -> f b) -> Cmd blk it a -> f (Cmd blk it b)
$csequenceA :: forall blk it (f :: * -> *) a.
Applicative f =>
Cmd blk it (f a) -> f (Cmd blk it a)
sequenceA :: forall (f :: * -> *) a.
Applicative f =>
Cmd blk it (f a) -> f (Cmd blk it a)
$cmapM :: forall blk it (m :: * -> *) a b.
Monad m =>
(a -> m b) -> Cmd blk it a -> m (Cmd blk it b)
mapM :: forall (m :: * -> *) a b.
Monad m =>
(a -> m b) -> Cmd blk it a -> m (Cmd blk it b)
$csequence :: forall blk it (m :: * -> *) a.
Monad m =>
Cmd blk it (m a) -> m (Cmd blk it a)
sequence :: forall (m :: * -> *) a.
Monad m =>
Cmd blk it (m a) -> m (Cmd blk it a)
Traversable)

-- = Invalid blocks
--
-- We don't test 'getIsInvalidBlock' because the simple chain selection in the
-- model and the incremental chain selection in the real implementation differ
-- non-trivially.
--
-- In the real chain selection, if a block is invalid, no chains containing
-- that block will be validated again. So if a successor of the block is added
-- afterwards, it will never be validated, as any chain it would be part of
-- would also contain the block we know to be invalid. So if the successor is
-- also invalid, we would never discover and record it (as there is no need
-- from the point of chain selection, as we'll never use it in a candidate
-- chain anyway). In the model implementation of chain selection, all possible
-- chains are (re)validated each time, including previously invalid chains, so
-- new invalid blocks that are successors of known invalid blocks /are/ being
-- validated and recorded as invalid blocks.
--
-- Further complicating this is the fact that the recorded invalid blocks are
-- also garbage-collected. We can work around this, just like for 'getBlock'.
--
-- While it is certainly possible to overcome the issues described above,
-- e.g., we could change the model to closer match the real implementation
-- (but at the cost of more complexity), it is not worth the effort. The whole
-- point of recording invalid blocks is to avoid constructing candidates
-- containing known invalid blocks and needlessly validating them, which is
-- something we are testing in 'prop_trace', see
-- 'invalidBlockNeverValidatedAgain'.

deriving instance SOP.Generic         (Cmd blk it flr)
deriving instance SOP.HasDatatypeInfo (Cmd blk it flr)

-- | Return type for successful database operations.
data Success blk it flr
  = Unit                ()
  | Chain               (AnchoredFragment (Header blk))
  | LedgerDB            (LedgerDB (ExtLedgerState blk))
  | MbBlock             (Maybe blk)
  | MbAllComponents     (Maybe (AllComponents blk))
  | MbGCedAllComponents (MaybeGCedBlock (AllComponents blk))
  | MbHeader            (Maybe (Header blk))
  | Point               (Point blk)
  | IsValid             IsValidResult
  | UnknownRange        (UnknownRange blk)
  | Iter                it
  | IterResult          (IteratorResult blk (AllComponents blk))
  | IterResultGCed      (IteratorResultGCed blk)
  | Flr                 flr
  | MbChainUpdate       (Maybe (ChainUpdate blk (AllComponents blk)))
  | MbPoint             (Maybe (Point blk))
  | MaxSlot             MaxSlotNo
  deriving ((forall a b. (a -> b) -> Success blk it a -> Success blk it b)
-> (forall a b. a -> Success blk it b -> Success blk it a)
-> Functor (Success blk it)
forall a b. a -> Success blk it b -> Success blk it a
forall a b. (a -> b) -> Success blk it a -> Success blk it b
forall blk it a b. a -> Success blk it b -> Success blk it a
forall blk it a b. (a -> b) -> Success blk it a -> Success blk it b
forall (f :: * -> *).
(forall a b. (a -> b) -> f a -> f b)
-> (forall a b. a -> f b -> f a) -> Functor f
$cfmap :: forall blk it a b. (a -> b) -> Success blk it a -> Success blk it b
fmap :: forall a b. (a -> b) -> Success blk it a -> Success blk it b
$c<$ :: forall blk it a b. a -> Success blk it b -> Success blk it a
<$ :: forall a b. a -> Success blk it b -> Success blk it a
Functor, (forall m. Monoid m => Success blk it m -> m)
-> (forall m a. Monoid m => (a -> m) -> Success blk it a -> m)
-> (forall m a. Monoid m => (a -> m) -> Success blk it a -> m)
-> (forall a b. (a -> b -> b) -> b -> Success blk it a -> b)
-> (forall a b. (a -> b -> b) -> b -> Success blk it a -> b)
-> (forall b a. (b -> a -> b) -> b -> Success blk it a -> b)
-> (forall b a. (b -> a -> b) -> b -> Success blk it a -> b)
-> (forall a. (a -> a -> a) -> Success blk it a -> a)
-> (forall a. (a -> a -> a) -> Success blk it a -> a)
-> (forall a. Success blk it a -> [a])
-> (forall a. Success blk it a -> Bool)
-> (forall a. Success blk it a -> Int)
-> (forall a. Eq a => a -> Success blk it a -> Bool)
-> (forall a. Ord a => Success blk it a -> a)
-> (forall a. Ord a => Success blk it a -> a)
-> (forall a. Num a => Success blk it a -> a)
-> (forall a. Num a => Success blk it a -> a)
-> Foldable (Success blk it)
forall a. Eq a => a -> Success blk it a -> Bool
forall a. Num a => Success blk it a -> a
forall a. Ord a => Success blk it a -> a
forall m. Monoid m => Success blk it m -> m
forall a. Success blk it a -> Bool
forall a. Success blk it a -> Int
forall a. Success blk it a -> [a]
forall a. (a -> a -> a) -> Success blk it a -> a
forall m a. Monoid m => (a -> m) -> Success blk it a -> m
forall b a. (b -> a -> b) -> b -> Success blk it a -> b
forall a b. (a -> b -> b) -> b -> Success blk it a -> b
forall blk it a. Eq a => a -> Success blk it a -> Bool
forall blk it a. Num a => Success blk it a -> a
forall blk it a. Ord a => Success blk it a -> a
forall blk it m. Monoid m => Success blk it m -> m
forall blk it a. Success blk it a -> Bool
forall blk it a. Success blk it a -> Int
forall blk it a. Success blk it a -> [a]
forall blk it a. (a -> a -> a) -> Success blk it a -> a
forall blk it m a. Monoid m => (a -> m) -> Success blk it a -> m
forall blk it b a. (b -> a -> b) -> b -> Success blk it a -> b
forall blk it a b. (a -> b -> b) -> b -> Success blk it a -> b
forall (t :: * -> *).
(forall m. Monoid m => t m -> m)
-> (forall m a. Monoid m => (a -> m) -> t a -> m)
-> (forall m a. Monoid m => (a -> m) -> t a -> m)
-> (forall a b. (a -> b -> b) -> b -> t a -> b)
-> (forall a b. (a -> b -> b) -> b -> t a -> b)
-> (forall b a. (b -> a -> b) -> b -> t a -> b)
-> (forall b a. (b -> a -> b) -> b -> t a -> b)
-> (forall a. (a -> a -> a) -> t a -> a)
-> (forall a. (a -> a -> a) -> t a -> a)
-> (forall a. t a -> [a])
-> (forall a. t a -> Bool)
-> (forall a. t a -> Int)
-> (forall a. Eq a => a -> t a -> Bool)
-> (forall a. Ord a => t a -> a)
-> (forall a. Ord a => t a -> a)
-> (forall a. Num a => t a -> a)
-> (forall a. Num a => t a -> a)
-> Foldable t
$cfold :: forall blk it m. Monoid m => Success blk it m -> m
fold :: forall m. Monoid m => Success blk it m -> m
$cfoldMap :: forall blk it m a. Monoid m => (a -> m) -> Success blk it a -> m
foldMap :: forall m a. Monoid m => (a -> m) -> Success blk it a -> m
$cfoldMap' :: forall blk it m a. Monoid m => (a -> m) -> Success blk it a -> m
foldMap' :: forall m a. Monoid m => (a -> m) -> Success blk it a -> m
$cfoldr :: forall blk it a b. (a -> b -> b) -> b -> Success blk it a -> b
foldr :: forall a b. (a -> b -> b) -> b -> Success blk it a -> b
$cfoldr' :: forall blk it a b. (a -> b -> b) -> b -> Success blk it a -> b
foldr' :: forall a b. (a -> b -> b) -> b -> Success blk it a -> b
$cfoldl :: forall blk it b a. (b -> a -> b) -> b -> Success blk it a -> b
foldl :: forall b a. (b -> a -> b) -> b -> Success blk it a -> b
$cfoldl' :: forall blk it b a. (b -> a -> b) -> b -> Success blk it a -> b
foldl' :: forall b a. (b -> a -> b) -> b -> Success blk it a -> b
$cfoldr1 :: forall blk it a. (a -> a -> a) -> Success blk it a -> a
foldr1 :: forall a. (a -> a -> a) -> Success blk it a -> a
$cfoldl1 :: forall blk it a. (a -> a -> a) -> Success blk it a -> a
foldl1 :: forall a. (a -> a -> a) -> Success blk it a -> a
$ctoList :: forall blk it a. Success blk it a -> [a]
toList :: forall a. Success blk it a -> [a]
$cnull :: forall blk it a. Success blk it a -> Bool
null :: forall a. Success blk it a -> Bool
$clength :: forall blk it a. Success blk it a -> Int
length :: forall a. Success blk it a -> Int
$celem :: forall blk it a. Eq a => a -> Success blk it a -> Bool
elem :: forall a. Eq a => a -> Success blk it a -> Bool
$cmaximum :: forall blk it a. Ord a => Success blk it a -> a
maximum :: forall a. Ord a => Success blk it a -> a
$cminimum :: forall blk it a. Ord a => Success blk it a -> a
minimum :: forall a. Ord a => Success blk it a -> a
$csum :: forall blk it a. Num a => Success blk it a -> a
sum :: forall a. Num a => Success blk it a -> a
$cproduct :: forall blk it a. Num a => Success blk it a -> a
product :: forall a. Num a => Success blk it a -> a
Foldable, Functor (Success blk it)
Foldable (Success blk it)
(Functor (Success blk it), Foldable (Success blk it)) =>
(forall (f :: * -> *) a b.
 Applicative f =>
 (a -> f b) -> Success blk it a -> f (Success blk it b))
-> (forall (f :: * -> *) a.
    Applicative f =>
    Success blk it (f a) -> f (Success blk it a))
-> (forall (m :: * -> *) a b.
    Monad m =>
    (a -> m b) -> Success blk it a -> m (Success blk it b))
-> (forall (m :: * -> *) a.
    Monad m =>
    Success blk it (m a) -> m (Success blk it a))
-> Traversable (Success blk it)
forall blk it. Functor (Success blk it)
forall blk it. Foldable (Success blk it)
forall blk it (m :: * -> *) a.
Monad m =>
Success blk it (m a) -> m (Success blk it a)
forall blk it (f :: * -> *) a.
Applicative f =>
Success blk it (f a) -> f (Success blk it a)
forall blk it (m :: * -> *) a b.
Monad m =>
(a -> m b) -> Success blk it a -> m (Success blk it b)
forall blk it (f :: * -> *) a b.
Applicative f =>
(a -> f b) -> Success blk it a -> f (Success blk it b)
forall (t :: * -> *).
(Functor t, Foldable t) =>
(forall (f :: * -> *) a b.
 Applicative f =>
 (a -> f b) -> t a -> f (t b))
-> (forall (f :: * -> *) a. Applicative f => t (f a) -> f (t a))
-> (forall (m :: * -> *) a b.
    Monad m =>
    (a -> m b) -> t a -> m (t b))
-> (forall (m :: * -> *) a. Monad m => t (m a) -> m (t a))
-> Traversable t
forall (m :: * -> *) a.
Monad m =>
Success blk it (m a) -> m (Success blk it a)
forall (f :: * -> *) a.
Applicative f =>
Success blk it (f a) -> f (Success blk it a)
forall (m :: * -> *) a b.
Monad m =>
(a -> m b) -> Success blk it a -> m (Success blk it b)
forall (f :: * -> *) a b.
Applicative f =>
(a -> f b) -> Success blk it a -> f (Success blk it b)
$ctraverse :: forall blk it (f :: * -> *) a b.
Applicative f =>
(a -> f b) -> Success blk it a -> f (Success blk it b)
traverse :: forall (f :: * -> *) a b.
Applicative f =>
(a -> f b) -> Success blk it a -> f (Success blk it b)
$csequenceA :: forall blk it (f :: * -> *) a.
Applicative f =>
Success blk it (f a) -> f (Success blk it a)
sequenceA :: forall (f :: * -> *) a.
Applicative f =>
Success blk it (f a) -> f (Success blk it a)
$cmapM :: forall blk it (m :: * -> *) a b.
Monad m =>
(a -> m b) -> Success blk it a -> m (Success blk it b)
mapM :: forall (m :: * -> *) a b.
Monad m =>
(a -> m b) -> Success blk it a -> m (Success blk it b)
$csequence :: forall blk it (m :: * -> *) a.
Monad m =>
Success blk it (m a) -> m (Success blk it a)
sequence :: forall (m :: * -> *) a.
Monad m =>
Success blk it (m a) -> m (Success blk it a)
Traversable)

-- | Product of all 'BlockComponent's. As this is a GADT, generating random
-- values of it (and combinations!) is not so simple. Therefore, we just
-- always request all block components.
allComponents :: BlockComponent blk (AllComponents blk)
allComponents :: forall blk. BlockComponent blk (AllComponents blk)
allComponents = (,,,,,,,,,,)
    (blk
 -> blk
 -> Header blk
 -> ByteString
 -> ByteString
 -> HeaderHash blk
 -> SlotNo
 -> IsEBB
 -> SizeInBytes
 -> Word16
 -> SomeSecond (NestedCtxt Header) blk
 -> (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
     SlotNo, IsEBB, SizeInBytes, Word16,
     SomeSecond (NestedCtxt Header) blk))
-> BlockComponent blk blk
-> BlockComponent
     blk
     (blk
      -> Header blk
      -> ByteString
      -> ByteString
      -> HeaderHash blk
      -> SlotNo
      -> IsEBB
      -> SizeInBytes
      -> Word16
      -> SomeSecond (NestedCtxt Header) blk
      -> (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
          SlotNo, IsEBB, SizeInBytes, Word16,
          SomeSecond (NestedCtxt Header) blk))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> BlockComponent blk blk
forall blk. BlockComponent blk blk
GetVerifiedBlock
    BlockComponent
  blk
  (blk
   -> Header blk
   -> ByteString
   -> ByteString
   -> HeaderHash blk
   -> SlotNo
   -> IsEBB
   -> SizeInBytes
   -> Word16
   -> SomeSecond (NestedCtxt Header) blk
   -> (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
       SlotNo, IsEBB, SizeInBytes, Word16,
       SomeSecond (NestedCtxt Header) blk))
-> BlockComponent blk blk
-> BlockComponent
     blk
     (Header blk
      -> ByteString
      -> ByteString
      -> HeaderHash blk
      -> SlotNo
      -> IsEBB
      -> SizeInBytes
      -> Word16
      -> SomeSecond (NestedCtxt Header) blk
      -> (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
          SlotNo, IsEBB, SizeInBytes, Word16,
          SomeSecond (NestedCtxt Header) blk))
forall a b.
BlockComponent blk (a -> b)
-> BlockComponent blk a -> BlockComponent blk b
forall (f :: * -> *) a b. Applicative f => f (a -> b) -> f a -> f b
<*> BlockComponent blk blk
forall blk. BlockComponent blk blk
GetBlock
    BlockComponent
  blk
  (Header blk
   -> ByteString
   -> ByteString
   -> HeaderHash blk
   -> SlotNo
   -> IsEBB
   -> SizeInBytes
   -> Word16
   -> SomeSecond (NestedCtxt Header) blk
   -> (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
       SlotNo, IsEBB, SizeInBytes, Word16,
       SomeSecond (NestedCtxt Header) blk))
-> BlockComponent blk (Header blk)
-> BlockComponent
     blk
     (ByteString
      -> ByteString
      -> HeaderHash blk
      -> SlotNo
      -> IsEBB
      -> SizeInBytes
      -> Word16
      -> SomeSecond (NestedCtxt Header) blk
      -> (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
          SlotNo, IsEBB, SizeInBytes, Word16,
          SomeSecond (NestedCtxt Header) blk))
forall a b.
BlockComponent blk (a -> b)
-> BlockComponent blk a -> BlockComponent blk b
forall (f :: * -> *) a b. Applicative f => f (a -> b) -> f a -> f b
<*> BlockComponent blk (Header blk)
forall blk. BlockComponent blk (Header blk)
GetHeader
    BlockComponent
  blk
  (ByteString
   -> ByteString
   -> HeaderHash blk
   -> SlotNo
   -> IsEBB
   -> SizeInBytes
   -> Word16
   -> SomeSecond (NestedCtxt Header) blk
   -> (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
       SlotNo, IsEBB, SizeInBytes, Word16,
       SomeSecond (NestedCtxt Header) blk))
-> BlockComponent blk ByteString
-> BlockComponent
     blk
     (ByteString
      -> HeaderHash blk
      -> SlotNo
      -> IsEBB
      -> SizeInBytes
      -> Word16
      -> SomeSecond (NestedCtxt Header) blk
      -> (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
          SlotNo, IsEBB, SizeInBytes, Word16,
          SomeSecond (NestedCtxt Header) blk))
forall a b.
BlockComponent blk (a -> b)
-> BlockComponent blk a -> BlockComponent blk b
forall (f :: * -> *) a b. Applicative f => f (a -> b) -> f a -> f b
<*> BlockComponent blk ByteString
forall blk. BlockComponent blk ByteString
GetRawBlock
    BlockComponent
  blk
  (ByteString
   -> HeaderHash blk
   -> SlotNo
   -> IsEBB
   -> SizeInBytes
   -> Word16
   -> SomeSecond (NestedCtxt Header) blk
   -> (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
       SlotNo, IsEBB, SizeInBytes, Word16,
       SomeSecond (NestedCtxt Header) blk))
-> BlockComponent blk ByteString
-> BlockComponent
     blk
     (HeaderHash blk
      -> SlotNo
      -> IsEBB
      -> SizeInBytes
      -> Word16
      -> SomeSecond (NestedCtxt Header) blk
      -> (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
          SlotNo, IsEBB, SizeInBytes, Word16,
          SomeSecond (NestedCtxt Header) blk))
forall a b.
BlockComponent blk (a -> b)
-> BlockComponent blk a -> BlockComponent blk b
forall (f :: * -> *) a b. Applicative f => f (a -> b) -> f a -> f b
<*> BlockComponent blk ByteString
forall blk. BlockComponent blk ByteString
GetRawHeader
    BlockComponent
  blk
  (HeaderHash blk
   -> SlotNo
   -> IsEBB
   -> SizeInBytes
   -> Word16
   -> SomeSecond (NestedCtxt Header) blk
   -> (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
       SlotNo, IsEBB, SizeInBytes, Word16,
       SomeSecond (NestedCtxt Header) blk))
-> BlockComponent blk (HeaderHash blk)
-> BlockComponent
     blk
     (SlotNo
      -> IsEBB
      -> SizeInBytes
      -> Word16
      -> SomeSecond (NestedCtxt Header) blk
      -> (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
          SlotNo, IsEBB, SizeInBytes, Word16,
          SomeSecond (NestedCtxt Header) blk))
forall a b.
BlockComponent blk (a -> b)
-> BlockComponent blk a -> BlockComponent blk b
forall (f :: * -> *) a b. Applicative f => f (a -> b) -> f a -> f b
<*> BlockComponent blk (HeaderHash blk)
forall blk. BlockComponent blk (HeaderHash blk)
GetHash
    BlockComponent
  blk
  (SlotNo
   -> IsEBB
   -> SizeInBytes
   -> Word16
   -> SomeSecond (NestedCtxt Header) blk
   -> (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
       SlotNo, IsEBB, SizeInBytes, Word16,
       SomeSecond (NestedCtxt Header) blk))
-> BlockComponent blk SlotNo
-> BlockComponent
     blk
     (IsEBB
      -> SizeInBytes
      -> Word16
      -> SomeSecond (NestedCtxt Header) blk
      -> (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
          SlotNo, IsEBB, SizeInBytes, Word16,
          SomeSecond (NestedCtxt Header) blk))
forall a b.
BlockComponent blk (a -> b)
-> BlockComponent blk a -> BlockComponent blk b
forall (f :: * -> *) a b. Applicative f => f (a -> b) -> f a -> f b
<*> BlockComponent blk SlotNo
forall blk. BlockComponent blk SlotNo
GetSlot
    BlockComponent
  blk
  (IsEBB
   -> SizeInBytes
   -> Word16
   -> SomeSecond (NestedCtxt Header) blk
   -> (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
       SlotNo, IsEBB, SizeInBytes, Word16,
       SomeSecond (NestedCtxt Header) blk))
-> BlockComponent blk IsEBB
-> BlockComponent
     blk
     (SizeInBytes
      -> Word16
      -> SomeSecond (NestedCtxt Header) blk
      -> (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
          SlotNo, IsEBB, SizeInBytes, Word16,
          SomeSecond (NestedCtxt Header) blk))
forall a b.
BlockComponent blk (a -> b)
-> BlockComponent blk a -> BlockComponent blk b
forall (f :: * -> *) a b. Applicative f => f (a -> b) -> f a -> f b
<*> BlockComponent blk IsEBB
forall blk. BlockComponent blk IsEBB
GetIsEBB
    BlockComponent
  blk
  (SizeInBytes
   -> Word16
   -> SomeSecond (NestedCtxt Header) blk
   -> (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
       SlotNo, IsEBB, SizeInBytes, Word16,
       SomeSecond (NestedCtxt Header) blk))
-> BlockComponent blk SizeInBytes
-> BlockComponent
     blk
     (Word16
      -> SomeSecond (NestedCtxt Header) blk
      -> (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
          SlotNo, IsEBB, SizeInBytes, Word16,
          SomeSecond (NestedCtxt Header) blk))
forall a b.
BlockComponent blk (a -> b)
-> BlockComponent blk a -> BlockComponent blk b
forall (f :: * -> *) a b. Applicative f => f (a -> b) -> f a -> f b
<*> BlockComponent blk SizeInBytes
forall blk. BlockComponent blk SizeInBytes
GetBlockSize
    BlockComponent
  blk
  (Word16
   -> SomeSecond (NestedCtxt Header) blk
   -> (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
       SlotNo, IsEBB, SizeInBytes, Word16,
       SomeSecond (NestedCtxt Header) blk))
-> BlockComponent blk Word16
-> BlockComponent
     blk
     (SomeSecond (NestedCtxt Header) blk
      -> (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
          SlotNo, IsEBB, SizeInBytes, Word16,
          SomeSecond (NestedCtxt Header) blk))
forall a b.
BlockComponent blk (a -> b)
-> BlockComponent blk a -> BlockComponent blk b
forall (f :: * -> *) a b. Applicative f => f (a -> b) -> f a -> f b
<*> BlockComponent blk Word16
forall blk. BlockComponent blk Word16
GetHeaderSize
    BlockComponent
  blk
  (SomeSecond (NestedCtxt Header) blk
   -> (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
       SlotNo, IsEBB, SizeInBytes, Word16,
       SomeSecond (NestedCtxt Header) blk))
-> BlockComponent blk (SomeSecond (NestedCtxt Header) blk)
-> BlockComponent
     blk
     (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
      SlotNo, IsEBB, SizeInBytes, Word16,
      SomeSecond (NestedCtxt Header) blk)
forall a b.
BlockComponent blk (a -> b)
-> BlockComponent blk a -> BlockComponent blk b
forall (f :: * -> *) a b. Applicative f => f (a -> b) -> f a -> f b
<*> BlockComponent blk (SomeSecond (NestedCtxt Header) blk)
forall blk. BlockComponent blk (SomeSecond (NestedCtxt Header) blk)
GetNestedCtxt

-- | A list of all the 'BlockComponent' indices (@b@) we are interested in.
type AllComponents blk =
  ( blk
  , blk
  , Header blk
  , ByteString
  , ByteString
  , HeaderHash blk
  , SlotNo
  , IsEBB
  , SizeInBytes
  , Word16
  , SomeSecond (NestedCtxt Header) blk
  )

type TestConstraints blk =
  ( ConsensusProtocol  (BlockProtocol blk)
  , LedgerSupportsProtocol            blk
  , BlockSupportsDiffusionPipelining  blk
  , InspectLedger                     blk
  , Eq (ChainDepState  (BlockProtocol blk))
  , Eq (LedgerState                   blk)
  , Eq                                blk
  , Show                              blk
  , HasHeader                         blk
  , StandardHash                      blk
  , Serialise                         blk
  , ModelSupportsBlock                blk
  , Eq                       (Header  blk)
  , Show                     (Header  blk)
  , ConvertRawHash                    blk
  , HasHardForkHistory                blk
  , SerialiseDiskConstraints          blk
  )

deriving instance (TestConstraints blk, Eq   it, Eq   flr)
               => Eq   (Success blk it flr)
deriving instance (TestConstraints blk, Show it, Show flr)
               => Show (Success blk it flr)

-- | Short-hand
type TestIterator m blk = WithEq (Iterator m blk (AllComponents blk))
-- | Short-hand
type TestFollower m blk = WithEq (Follower m blk (AllComponents blk))

-- | The current ChainDB instance and things related to it.
--
-- When closing and reopening the ChainDB, this record will be replaced in the
-- 'varDB' field of 'ChainDBEnv' with a new one.
data ChainDBState m blk = ChainDBState
    { forall (m :: * -> *) blk. ChainDBState m blk -> ChainDB m blk
chainDB       :: ChainDB m blk
    , forall (m :: * -> *) blk. ChainDBState m blk -> Internal m blk
internal      :: ChainDB.Internal m blk
    , forall (m :: * -> *) blk. ChainDBState m blk -> Async m Void
addBlockAsync :: Async m Void
      -- ^ Background thread that adds blocks to the ChainDB
    }
  deriving Context -> ChainDBState m blk -> IO (Maybe ThunkInfo)
Proxy (ChainDBState m blk) -> String
(Context -> ChainDBState m blk -> IO (Maybe ThunkInfo))
-> (Context -> ChainDBState m blk -> IO (Maybe ThunkInfo))
-> (Proxy (ChainDBState m blk) -> String)
-> NoThunks (ChainDBState m blk)
forall a.
(Context -> a -> IO (Maybe ThunkInfo))
-> (Context -> a -> IO (Maybe ThunkInfo))
-> (Proxy a -> String)
-> NoThunks a
forall (m :: * -> *) blk.
Context -> ChainDBState m blk -> IO (Maybe ThunkInfo)
forall (m :: * -> *) blk. Proxy (ChainDBState m blk) -> String
$cnoThunks :: forall (m :: * -> *) blk.
Context -> ChainDBState m blk -> IO (Maybe ThunkInfo)
noThunks :: Context -> ChainDBState m blk -> IO (Maybe ThunkInfo)
$cwNoThunks :: forall (m :: * -> *) blk.
Context -> ChainDBState m blk -> IO (Maybe ThunkInfo)
wNoThunks :: Context -> ChainDBState m blk -> IO (Maybe ThunkInfo)
$cshowTypeOf :: forall (m :: * -> *) blk. Proxy (ChainDBState m blk) -> String
showTypeOf :: Proxy (ChainDBState m blk) -> String
NoThunks via AllowThunk (ChainDBState m blk)

-- | Environment to run commands against the real ChainDB implementation.
data ChainDBEnv m blk = ChainDBEnv {
    forall (m :: * -> *) blk.
ChainDBEnv m blk -> StrictTVar m (ChainDBState m blk)
varDB           :: StrictTVar m (ChainDBState m blk)
  , forall (m :: * -> *) blk. ChainDBEnv m blk -> ResourceRegistry m
registry        :: ResourceRegistry m
  , forall (m :: * -> *) blk. ChainDBEnv m blk -> StrictTVar m SlotNo
varCurSlot      :: StrictTVar m SlotNo
  , forall (m :: * -> *) blk. ChainDBEnv m blk -> StrictTVar m Id
varNextId       :: StrictTVar m Id
  , forall (m :: * -> *) blk. ChainDBEnv m blk -> StrictTMVar m MockFS
varVolatileDbFs :: StrictTMVar m MockFS
  , forall (m :: * -> *) blk.
ChainDBEnv m blk -> ChainDbArgs Identity m blk
args            :: ChainDbArgs Identity m blk
    -- ^ Needed to reopen a ChainDB, i.e., open a new one.
  , forall (m :: * -> *) blk.
ChainDBEnv m blk -> StrictTVar m (AnchoredFragment (Header blk))
varLoEFragment  :: StrictTVar m (AnchoredFragment (Header blk))
  }

open ::
     (IOLike m, TestConstraints blk)
  => ChainDbArgs Identity m blk -> m (ChainDBState m blk)
open :: forall (m :: * -> *) blk.
(IOLike m, TestConstraints blk) =>
ChainDbArgs Identity m blk -> m (ChainDBState m blk)
open ChainDbArgs Identity m blk
args = do
    (ChainDB m blk
chainDB, Internal m blk
internal) <- ChainDbArgs Identity m blk
-> Bool -> m (ChainDB m blk, Internal m blk)
forall (m :: * -> *) blk.
(IOLike m, LedgerSupportsProtocol blk,
 BlockSupportsDiffusionPipelining blk, InspectLedger blk,
 HasHardForkHistory blk, ConvertRawHash blk,
 SerialiseDiskConstraints blk) =>
ChainDbArgs Identity m blk
-> Bool -> m (ChainDB m blk, Internal m blk)
openDBInternal ChainDbArgs Identity m blk
args Bool
False
    Async m Void
addBlockAsync       <- m Void -> m (Async m Void)
forall a. m a -> m (Async m a)
forall (m :: * -> *) a. MonadAsync m => m a -> m (Async m a)
async (Internal m blk -> m Void
forall (m :: * -> *) blk. Internal m blk -> m Void
intAddBlockRunner Internal m blk
internal)
    Async m Void -> m ()
forall (m :: * -> *) a.
(MonadAsync m, MonadFork m, MonadMask m) =>
Async m a -> m ()
link Async m Void
addBlockAsync
    ChainDBState m blk -> m (ChainDBState m blk)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return ChainDBState { ChainDB m blk
$sel:chainDB:ChainDBState :: ChainDB m blk
chainDB :: ChainDB m blk
chainDB, Internal m blk
$sel:internal:ChainDBState :: Internal m blk
internal :: Internal m blk
internal, Async m Void
$sel:addBlockAsync:ChainDBState :: Async m Void
addBlockAsync :: Async m Void
addBlockAsync }

-- PRECONDITION: the ChainDB is closed
reopen ::
     (IOLike m, TestConstraints blk)
  => ChainDBEnv m blk -> m ()
reopen :: forall (m :: * -> *) blk.
(IOLike m, TestConstraints blk) =>
ChainDBEnv m blk -> m ()
reopen ChainDBEnv { StrictTVar m (ChainDBState m blk)
varDB :: forall (m :: * -> *) blk.
ChainDBEnv m blk -> StrictTVar m (ChainDBState m blk)
varDB :: StrictTVar m (ChainDBState m blk)
varDB, ChainDbArgs Identity m blk
$sel:args:ChainDBEnv :: forall (m :: * -> *) blk.
ChainDBEnv m blk -> ChainDbArgs Identity m blk
args :: ChainDbArgs Identity m blk
args } = do
    ChainDBState m blk
chainDBState <- ChainDbArgs Identity m blk -> m (ChainDBState m blk)
forall (m :: * -> *) blk.
(IOLike m, TestConstraints blk) =>
ChainDbArgs Identity m blk -> m (ChainDBState m blk)
open ChainDbArgs Identity m blk
args
    m (ChainDBState m blk) -> m ()
forall (f :: * -> *) a. Functor f => f a -> f ()
void (m (ChainDBState m blk) -> m ()) -> m (ChainDBState m blk) -> m ()
forall a b. (a -> b) -> a -> b
$ STM m (ChainDBState m blk) -> m (ChainDBState m blk)
forall a. HasCallStack => STM m a -> m a
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
STM m a -> m a
atomically (STM m (ChainDBState m blk) -> m (ChainDBState m blk))
-> STM m (ChainDBState m blk) -> m (ChainDBState m blk)
forall a b. (a -> b) -> a -> b
$ StrictTVar m (ChainDBState m blk)
-> ChainDBState m blk -> STM m (ChainDBState m blk)
forall (m :: * -> *) a.
MonadSTM m =>
StrictTVar m a -> a -> STM m a
swapTVar StrictTVar m (ChainDBState m blk)
varDB ChainDBState m blk
chainDBState

close :: IOLike m => ChainDBState m blk -> m ()
close :: forall (m :: * -> *) blk. IOLike m => ChainDBState m blk -> m ()
close ChainDBState { ChainDB m blk
$sel:chainDB:ChainDBState :: forall (m :: * -> *) blk. ChainDBState m blk -> ChainDB m blk
chainDB :: ChainDB m blk
chainDB, Async m Void
$sel:addBlockAsync:ChainDBState :: forall (m :: * -> *) blk. ChainDBState m blk -> Async m Void
addBlockAsync :: Async m Void
addBlockAsync } = do
    Async m Void -> m ()
forall a. Async m a -> m ()
forall (m :: * -> *) a. MonadAsync m => Async m a -> m ()
cancel Async m Void
addBlockAsync
    ChainDB m blk -> m ()
forall (m :: * -> *) blk. ChainDB m blk -> m ()
closeDB ChainDB m blk
chainDB

run :: forall m blk.
       (IOLike m, TestConstraints blk)
    => ChainDBEnv m blk
    ->    Cmd     blk (TestIterator m blk) (TestFollower m blk)
    -> m (Success blk (TestIterator m blk) (TestFollower m blk))
run :: forall (m :: * -> *) blk.
(IOLike m, TestConstraints blk) =>
ChainDBEnv m blk
-> Cmd blk (TestIterator m blk) (TestFollower m blk)
-> m (Success blk (TestIterator m blk) (TestFollower m blk))
run env :: ChainDBEnv m blk
env@ChainDBEnv { StrictTVar m (ChainDBState m blk)
varDB :: forall (m :: * -> *) blk.
ChainDBEnv m blk -> StrictTVar m (ChainDBState m blk)
varDB :: StrictTVar m (ChainDBState m blk)
varDB, StrictTVar m SlotNo
StrictTVar m (AnchoredFragment (Header blk))
StrictTVar m Id
StrictTMVar m MockFS
ResourceRegistry m
ChainDbArgs Identity m blk
$sel:registry:ChainDBEnv :: forall (m :: * -> *) blk. ChainDBEnv m blk -> ResourceRegistry m
$sel:varCurSlot:ChainDBEnv :: forall (m :: * -> *) blk. ChainDBEnv m blk -> StrictTVar m SlotNo
$sel:varNextId:ChainDBEnv :: forall (m :: * -> *) blk. ChainDBEnv m blk -> StrictTVar m Id
$sel:varVolatileDbFs:ChainDBEnv :: forall (m :: * -> *) blk. ChainDBEnv m blk -> StrictTMVar m MockFS
$sel:args:ChainDBEnv :: forall (m :: * -> *) blk.
ChainDBEnv m blk -> ChainDbArgs Identity m blk
$sel:varLoEFragment:ChainDBEnv :: forall (m :: * -> *) blk.
ChainDBEnv m blk -> StrictTVar m (AnchoredFragment (Header blk))
registry :: ResourceRegistry m
varCurSlot :: StrictTVar m SlotNo
varNextId :: StrictTVar m Id
varVolatileDbFs :: StrictTMVar m MockFS
args :: ChainDbArgs Identity m blk
varLoEFragment :: StrictTVar m (AnchoredFragment (Header blk))
.. } Cmd blk (TestIterator m blk) (TestFollower m blk)
cmd =
    StrictTVar m (ChainDBState m blk) -> m (ChainDBState m blk)
forall (m :: * -> *) a. MonadSTM m => StrictTVar m a -> m a
readTVarIO StrictTVar m (ChainDBState m blk)
varDB m (ChainDBState m blk)
-> (ChainDBState m blk
    -> m (Success blk (TestIterator m blk) (TestFollower m blk)))
-> m (Success blk (TestIterator m blk) (TestFollower m blk))
forall a b. m a -> (a -> m b) -> m b
forall (m :: * -> *) a b. Monad m => m a -> (a -> m b) -> m b
>>= \st :: ChainDBState m blk
st@ChainDBState { $sel:chainDB:ChainDBState :: forall (m :: * -> *) blk. ChainDBState m blk -> ChainDB m blk
chainDB = ChainDB{m (Maybe blk)
m (Maybe (Header blk))
m ()
m (ChainSelectionPromise m)
STM m Bool
STM m (Point blk)
STM m (LedgerDB' blk)
STM
  m
  (WithFingerprint
     (HeaderHash blk -> Maybe (InvalidBlockReason blk)))
STM m (HeaderStateHistory blk)
STM m (AnchoredFragment (Header blk))
STM m MaxSlotNo
STM m (Point blk -> Bool)
STM m (RealPoint blk -> Maybe Bool)
InvalidBlockPunishment m -> blk -> m (AddBlockPromise m blk)
forall b. BlockComponent blk b -> RealPoint blk -> m (Maybe b)
forall b.
ResourceRegistry m
-> BlockComponent blk b
-> StreamFrom blk
-> StreamTo blk
-> m (Either (UnknownRange blk) (Iterator m blk b))
forall b.
ResourceRegistry m
-> ChainType -> BlockComponent blk b -> m (Follower m blk b)
closeDB :: forall (m :: * -> *) blk. ChainDB m blk -> m ()
addBlockAsync :: InvalidBlockPunishment m -> blk -> m (AddBlockPromise m blk)
chainSelAsync :: m (ChainSelectionPromise m)
getCurrentChain :: STM m (AnchoredFragment (Header blk))
getLedgerDB :: STM m (LedgerDB' blk)
getHeaderStateHistory :: STM m (HeaderStateHistory blk)
getTipBlock :: m (Maybe blk)
getTipHeader :: m (Maybe (Header blk))
getTipPoint :: STM m (Point blk)
getBlockComponent :: forall b. BlockComponent blk b -> RealPoint blk -> m (Maybe b)
getIsFetched :: STM m (Point blk -> Bool)
getIsValid :: STM m (RealPoint blk -> Maybe Bool)
getMaxSlotNo :: STM m MaxSlotNo
stream :: forall b.
ResourceRegistry m
-> BlockComponent blk b
-> StreamFrom blk
-> StreamTo blk
-> m (Either (UnknownRange blk) (Iterator m blk b))
newFollower :: forall b.
ResourceRegistry m
-> ChainType -> BlockComponent blk b -> m (Follower m blk b)
getIsInvalidBlock :: STM
  m
  (WithFingerprint
     (HeaderHash blk -> Maybe (InvalidBlockReason blk)))
closeDB :: m ()
isOpen :: STM m Bool
addBlockAsync :: forall (m :: * -> *) blk.
ChainDB m blk
-> InvalidBlockPunishment m -> blk -> m (AddBlockPromise m blk)
chainSelAsync :: forall (m :: * -> *) blk.
ChainDB m blk -> m (ChainSelectionPromise m)
getCurrentChain :: forall (m :: * -> *) blk.
ChainDB m blk -> STM m (AnchoredFragment (Header blk))
getLedgerDB :: forall (m :: * -> *) blk. ChainDB m blk -> STM m (LedgerDB' blk)
getHeaderStateHistory :: forall (m :: * -> *) blk.
ChainDB m blk -> STM m (HeaderStateHistory blk)
getTipBlock :: forall (m :: * -> *) blk. ChainDB m blk -> m (Maybe blk)
getTipHeader :: forall (m :: * -> *) blk. ChainDB m blk -> m (Maybe (Header blk))
getTipPoint :: forall (m :: * -> *) blk. ChainDB m blk -> STM m (Point blk)
getBlockComponent :: forall (m :: * -> *) blk.
ChainDB m blk
-> forall b. BlockComponent blk b -> RealPoint blk -> m (Maybe b)
getIsFetched :: forall (m :: * -> *) blk.
ChainDB m blk -> STM m (Point blk -> Bool)
getIsValid :: forall (m :: * -> *) blk.
ChainDB m blk -> STM m (RealPoint blk -> Maybe Bool)
getMaxSlotNo :: forall (m :: * -> *) blk. ChainDB m blk -> STM m MaxSlotNo
stream :: forall (m :: * -> *) blk.
ChainDB m blk
-> forall b.
   ResourceRegistry m
   -> BlockComponent blk b
   -> StreamFrom blk
   -> StreamTo blk
   -> m (Either (UnknownRange blk) (Iterator m blk b))
newFollower :: forall (m :: * -> *) blk.
ChainDB m blk
-> forall b.
   ResourceRegistry m
   -> ChainType -> BlockComponent blk b -> m (Follower m blk b)
getIsInvalidBlock :: forall (m :: * -> *) blk.
ChainDB m blk
-> STM
     m
     (WithFingerprint
        (HeaderHash blk -> Maybe (InvalidBlockReason blk)))
isOpen :: forall (m :: * -> *) blk. ChainDB m blk -> STM m Bool
..}, Internal m blk
$sel:internal:ChainDBState :: forall (m :: * -> *) blk. ChainDBState m blk -> Internal m blk
internal :: Internal m blk
internal } -> case Cmd blk (TestIterator m blk) (TestFollower m blk)
cmd of
      AddBlock blk
blk             -> Point blk -> Success blk (TestIterator m blk) (TestFollower m blk)
forall blk it flr. Point blk -> Success blk it flr
Point               (Point blk
 -> Success blk (TestIterator m blk) (TestFollower m blk))
-> m (Point blk)
-> m (Success blk (TestIterator m blk) (TestFollower m blk))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> (ChainDBState m blk -> SlotNo -> blk -> m (Point blk)
advanceAndAdd ChainDBState m blk
st (blk -> SlotNo
forall b. HasHeader b => b -> SlotNo
blockSlot blk
blk) blk
blk)
      AddFutureBlock blk
blk SlotNo
s     -> Point blk -> Success blk (TestIterator m blk) (TestFollower m blk)
forall blk it flr. Point blk -> Success blk it flr
Point               (Point blk
 -> Success blk (TestIterator m blk) (TestFollower m blk))
-> m (Point blk)
-> m (Success blk (TestIterator m blk) (TestFollower m blk))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> (ChainDBState m blk -> SlotNo -> blk -> m (Point blk)
advanceAndAdd ChainDBState m blk
st SlotNo
s               blk
blk)
      Cmd blk (TestIterator m blk) (TestFollower m blk)
GetCurrentChain          -> AnchoredFragment (Header blk)
-> Success blk (TestIterator m blk) (TestFollower m blk)
forall blk it flr.
AnchoredFragment (Header blk) -> Success blk it flr
Chain               (AnchoredFragment (Header blk)
 -> Success blk (TestIterator m blk) (TestFollower m blk))
-> m (AnchoredFragment (Header blk))
-> m (Success blk (TestIterator m blk) (TestFollower m blk))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> STM m (AnchoredFragment (Header blk))
-> m (AnchoredFragment (Header blk))
forall a. HasCallStack => STM m a -> m a
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
STM m a -> m a
atomically STM m (AnchoredFragment (Header blk))
getCurrentChain
      Cmd blk (TestIterator m blk) (TestFollower m blk)
GetLedgerDB              -> LedgerDB' blk
-> Success blk (TestIterator m blk) (TestFollower m blk)
forall blk it flr.
LedgerDB (ExtLedgerState blk) -> Success blk it flr
LedgerDB            (LedgerDB' blk
 -> Success blk (TestIterator m blk) (TestFollower m blk))
-> m (LedgerDB' blk)
-> m (Success blk (TestIterator m blk) (TestFollower m blk))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> STM m (LedgerDB' blk) -> m (LedgerDB' blk)
forall a. HasCallStack => STM m a -> m a
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
STM m a -> m a
atomically STM m (LedgerDB' blk)
getLedgerDB
      Cmd blk (TestIterator m blk) (TestFollower m blk)
GetTipBlock              -> Maybe blk -> Success blk (TestIterator m blk) (TestFollower m blk)
forall blk it flr. Maybe blk -> Success blk it flr
MbBlock             (Maybe blk
 -> Success blk (TestIterator m blk) (TestFollower m blk))
-> m (Maybe blk)
-> m (Success blk (TestIterator m blk) (TestFollower m blk))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> m (Maybe blk)
getTipBlock
      Cmd blk (TestIterator m blk) (TestFollower m blk)
GetTipHeader             -> Maybe (Header blk)
-> Success blk (TestIterator m blk) (TestFollower m blk)
forall blk it flr. Maybe (Header blk) -> Success blk it flr
MbHeader            (Maybe (Header blk)
 -> Success blk (TestIterator m blk) (TestFollower m blk))
-> m (Maybe (Header blk))
-> m (Success blk (TestIterator m blk) (TestFollower m blk))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> m (Maybe (Header blk))
getTipHeader
      Cmd blk (TestIterator m blk) (TestFollower m blk)
GetTipPoint              -> Point blk -> Success blk (TestIterator m blk) (TestFollower m blk)
forall blk it flr. Point blk -> Success blk it flr
Point               (Point blk
 -> Success blk (TestIterator m blk) (TestFollower m blk))
-> m (Point blk)
-> m (Success blk (TestIterator m blk) (TestFollower m blk))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> STM m (Point blk) -> m (Point blk)
forall a. HasCallStack => STM m a -> m a
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
STM m a -> m a
atomically STM m (Point blk)
getTipPoint
      GetBlockComponent RealPoint blk
pt     -> Maybe (AllComponents blk)
-> Success blk (TestIterator m blk) (TestFollower m blk)
forall blk it flr. Maybe (AllComponents blk) -> Success blk it flr
MbAllComponents     (Maybe (AllComponents blk)
 -> Success blk (TestIterator m blk) (TestFollower m blk))
-> m (Maybe (AllComponents blk))
-> m (Success blk (TestIterator m blk) (TestFollower m blk))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> BlockComponent blk (AllComponents blk)
-> RealPoint blk -> m (Maybe (AllComponents blk))
forall b. BlockComponent blk b -> RealPoint blk -> m (Maybe b)
getBlockComponent BlockComponent blk (AllComponents blk)
forall blk. BlockComponent blk (AllComponents blk)
allComponents RealPoint blk
pt
      GetGCedBlockComponent RealPoint blk
pt -> Maybe (AllComponents blk)
-> Success blk (TestIterator m blk) (TestFollower m blk)
forall blk it flr. Maybe (AllComponents blk) -> Success blk it flr
mbGCedAllComponents (Maybe (AllComponents blk)
 -> Success blk (TestIterator m blk) (TestFollower m blk))
-> m (Maybe (AllComponents blk))
-> m (Success blk (TestIterator m blk) (TestFollower m blk))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> BlockComponent blk (AllComponents blk)
-> RealPoint blk -> m (Maybe (AllComponents blk))
forall b. BlockComponent blk b -> RealPoint blk -> m (Maybe b)
getBlockComponent BlockComponent blk (AllComponents blk)
forall blk. BlockComponent blk (AllComponents blk)
allComponents RealPoint blk
pt
      GetIsValid RealPoint blk
pt            -> Maybe Bool -> Success blk (TestIterator m blk) (TestFollower m blk)
forall {blk} {it} {flr}. Maybe Bool -> Success blk it flr
isValidResult       (Maybe Bool
 -> Success blk (TestIterator m blk) (TestFollower m blk))
-> ((RealPoint blk -> Maybe Bool) -> Maybe Bool)
-> (RealPoint blk -> Maybe Bool)
-> Success blk (TestIterator m blk) (TestFollower m blk)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> ((RealPoint blk -> Maybe Bool) -> RealPoint blk -> Maybe Bool
forall a b. (a -> b) -> a -> b
$ RealPoint blk
pt) ((RealPoint blk -> Maybe Bool)
 -> Success blk (TestIterator m blk) (TestFollower m blk))
-> m (RealPoint blk -> Maybe Bool)
-> m (Success blk (TestIterator m blk) (TestFollower m blk))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> STM m (RealPoint blk -> Maybe Bool)
-> m (RealPoint blk -> Maybe Bool)
forall a. HasCallStack => STM m a -> m a
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
STM m a -> m a
atomically STM m (RealPoint blk -> Maybe Bool)
getIsValid
      Cmd blk (TestIterator m blk) (TestFollower m blk)
GetMaxSlotNo             -> MaxSlotNo -> Success blk (TestIterator m blk) (TestFollower m blk)
forall blk it flr. MaxSlotNo -> Success blk it flr
MaxSlot             (MaxSlotNo
 -> Success blk (TestIterator m blk) (TestFollower m blk))
-> m MaxSlotNo
-> m (Success blk (TestIterator m blk) (TestFollower m blk))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> STM m MaxSlotNo -> m MaxSlotNo
forall a. HasCallStack => STM m a -> m a
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
STM m a -> m a
atomically STM m MaxSlotNo
getMaxSlotNo
      UpdateLoE AnchoredFragment blk
frag           -> Point blk -> Success blk (TestIterator m blk) (TestFollower m blk)
forall blk it flr. Point blk -> Success blk it flr
Point               (Point blk
 -> Success blk (TestIterator m blk) (TestFollower m blk))
-> m (Point blk)
-> m (Success blk (TestIterator m blk) (TestFollower m blk))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> ChainDBState m blk -> AnchoredFragment blk -> m (Point blk)
updateLoE ChainDBState m blk
st AnchoredFragment blk
frag
      Stream StreamFrom blk
from StreamTo blk
to           -> Either (UnknownRange blk) (Iterator m blk (AllComponents blk))
-> m (Success blk (TestIterator m blk) (TestFollower m blk))
iter                (Either (UnknownRange blk) (Iterator m blk (AllComponents blk))
 -> m (Success blk (TestIterator m blk) (TestFollower m blk)))
-> m (Either
        (UnknownRange blk) (Iterator m blk (AllComponents blk)))
-> m (Success blk (TestIterator m blk) (TestFollower m blk))
forall (m :: * -> *) a b. Monad m => (a -> m b) -> m a -> m b
=<< ResourceRegistry m
-> BlockComponent blk (AllComponents blk)
-> StreamFrom blk
-> StreamTo blk
-> m (Either
        (UnknownRange blk) (Iterator m blk (AllComponents blk)))
forall b.
ResourceRegistry m
-> BlockComponent blk b
-> StreamFrom blk
-> StreamTo blk
-> m (Either (UnknownRange blk) (Iterator m blk b))
stream ResourceRegistry m
registry BlockComponent blk (AllComponents blk)
forall blk. BlockComponent blk (AllComponents blk)
allComponents StreamFrom blk
from StreamTo blk
to
      IteratorNext  TestIterator m blk
it         -> IteratorResult blk (AllComponents blk)
-> Success blk (TestIterator m blk) (TestFollower m blk)
forall blk it flr.
IteratorResult blk (AllComponents blk) -> Success blk it flr
IterResult          (IteratorResult blk (AllComponents blk)
 -> Success blk (TestIterator m blk) (TestFollower m blk))
-> m (IteratorResult blk (AllComponents blk))
-> m (Success blk (TestIterator m blk) (TestFollower m blk))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Iterator m blk (AllComponents blk)
-> m (IteratorResult blk (AllComponents blk))
forall (m :: * -> *) blk b.
Iterator m blk b -> m (IteratorResult blk b)
iteratorNext (TestIterator m blk -> Iterator m blk (AllComponents blk)
forall a. WithEq a -> a
unWithEq TestIterator m blk
it)
      IteratorNextGCed  TestIterator m blk
it     -> IteratorResult blk (AllComponents blk)
-> Success blk (TestIterator m blk) (TestFollower m blk)
forall blk it flr.
IteratorResult blk (AllComponents blk) -> Success blk it flr
iterResultGCed      (IteratorResult blk (AllComponents blk)
 -> Success blk (TestIterator m blk) (TestFollower m blk))
-> m (IteratorResult blk (AllComponents blk))
-> m (Success blk (TestIterator m blk) (TestFollower m blk))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Iterator m blk (AllComponents blk)
-> m (IteratorResult blk (AllComponents blk))
forall (m :: * -> *) blk b.
Iterator m blk b -> m (IteratorResult blk b)
iteratorNext (TestIterator m blk -> Iterator m blk (AllComponents blk)
forall a. WithEq a -> a
unWithEq TestIterator m blk
it)
      IteratorClose TestIterator m blk
it         -> () -> Success blk (TestIterator m blk) (TestFollower m blk)
forall blk it flr. () -> Success blk it flr
Unit                (() -> Success blk (TestIterator m blk) (TestFollower m blk))
-> m ()
-> m (Success blk (TestIterator m blk) (TestFollower m blk))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Iterator m blk (AllComponents blk) -> m ()
forall (m :: * -> *) blk b. Iterator m blk b -> m ()
iteratorClose (TestIterator m blk -> Iterator m blk (AllComponents blk)
forall a. WithEq a -> a
unWithEq TestIterator m blk
it)
      NewFollower ChainType
ct           -> Follower m blk (AllComponents blk)
-> m (Success blk (TestIterator m blk) (TestFollower m blk))
follower            (Follower m blk (AllComponents blk)
 -> m (Success blk (TestIterator m blk) (TestFollower m blk)))
-> m (Follower m blk (AllComponents blk))
-> m (Success blk (TestIterator m blk) (TestFollower m blk))
forall (m :: * -> *) a b. Monad m => (a -> m b) -> m a -> m b
=<< ResourceRegistry m
-> ChainType
-> BlockComponent blk (AllComponents blk)
-> m (Follower m blk (AllComponents blk))
forall b.
ResourceRegistry m
-> ChainType -> BlockComponent blk b -> m (Follower m blk b)
newFollower ResourceRegistry m
registry ChainType
ct BlockComponent blk (AllComponents blk)
forall blk. BlockComponent blk (AllComponents blk)
allComponents
      FollowerInstruction TestFollower m blk
flr  -> Maybe (ChainUpdate blk (AllComponents blk))
-> Success blk (TestIterator m blk) (TestFollower m blk)
forall blk it flr.
Maybe (ChainUpdate blk (AllComponents blk)) -> Success blk it flr
MbChainUpdate       (Maybe (ChainUpdate blk (AllComponents blk))
 -> Success blk (TestIterator m blk) (TestFollower m blk))
-> m (Maybe (ChainUpdate blk (AllComponents blk)))
-> m (Success blk (TestIterator m blk) (TestFollower m blk))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Follower m blk (AllComponents blk)
-> m (Maybe (ChainUpdate blk (AllComponents blk)))
forall (m :: * -> *) blk a.
Follower m blk a -> m (Maybe (ChainUpdate blk a))
followerInstruction (TestFollower m blk -> Follower m blk (AllComponents blk)
forall a. WithEq a -> a
unWithEq TestFollower m blk
flr)
      FollowerForward TestFollower m blk
flr [Point blk]
pts  -> Maybe (Point blk)
-> Success blk (TestIterator m blk) (TestFollower m blk)
forall blk it flr. Maybe (Point blk) -> Success blk it flr
MbPoint             (Maybe (Point blk)
 -> Success blk (TestIterator m blk) (TestFollower m blk))
-> m (Maybe (Point blk))
-> m (Success blk (TestIterator m blk) (TestFollower m blk))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Follower m blk (AllComponents blk)
-> [Point blk] -> m (Maybe (Point blk))
forall (m :: * -> *) blk a.
Follower m blk a -> [Point blk] -> m (Maybe (Point blk))
followerForward (TestFollower m blk -> Follower m blk (AllComponents blk)
forall a. WithEq a -> a
unWithEq TestFollower m blk
flr) [Point blk]
pts
      FollowerClose TestFollower m blk
flr        -> () -> Success blk (TestIterator m blk) (TestFollower m blk)
forall blk it flr. () -> Success blk it flr
Unit                (() -> Success blk (TestIterator m blk) (TestFollower m blk))
-> m ()
-> m (Success blk (TestIterator m blk) (TestFollower m blk))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Follower m blk (AllComponents blk) -> m ()
forall (m :: * -> *) blk a. Follower m blk a -> m ()
followerClose (TestFollower m blk -> Follower m blk (AllComponents blk)
forall a. WithEq a -> a
unWithEq TestFollower m blk
flr)
      Cmd blk (TestIterator m blk) (TestFollower m blk)
Close                    -> () -> Success blk (TestIterator m blk) (TestFollower m blk)
forall blk it flr. () -> Success blk it flr
Unit                (() -> Success blk (TestIterator m blk) (TestFollower m blk))
-> m ()
-> m (Success blk (TestIterator m blk) (TestFollower m blk))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> ChainDBState m blk -> m ()
forall (m :: * -> *) blk. IOLike m => ChainDBState m blk -> m ()
close ChainDBState m blk
st
      Cmd blk (TestIterator m blk) (TestFollower m blk)
Reopen                   -> () -> Success blk (TestIterator m blk) (TestFollower m blk)
forall blk it flr. () -> Success blk it flr
Unit                (() -> Success blk (TestIterator m blk) (TestFollower m blk))
-> m ()
-> m (Success blk (TestIterator m blk) (TestFollower m blk))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> ChainDBEnv m blk -> m ()
forall (m :: * -> *) blk.
(IOLike m, TestConstraints blk) =>
ChainDBEnv m blk -> m ()
reopen ChainDBEnv m blk
env
      Cmd blk (TestIterator m blk) (TestFollower m blk)
PersistBlks              -> () -> Success blk (TestIterator m blk) (TestFollower m blk)
forall {p} {blk} {it} {flr}. p -> Success blk it flr
ignore              (() -> Success blk (TestIterator m blk) (TestFollower m blk))
-> m ()
-> m (Success blk (TestIterator m blk) (TestFollower m blk))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> ShouldGarbageCollect -> Internal m blk -> m ()
forall (m :: * -> *) blk.
IOLike m =>
ShouldGarbageCollect -> Internal m blk -> m ()
persistBlks ShouldGarbageCollect
DoNotGarbageCollect Internal m blk
internal
      Cmd blk (TestIterator m blk) (TestFollower m blk)
PersistBlksThenGC        -> () -> Success blk (TestIterator m blk) (TestFollower m blk)
forall {p} {blk} {it} {flr}. p -> Success blk it flr
ignore              (() -> Success blk (TestIterator m blk) (TestFollower m blk))
-> m ()
-> m (Success blk (TestIterator m blk) (TestFollower m blk))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> ShouldGarbageCollect -> Internal m blk -> m ()
forall (m :: * -> *) blk.
IOLike m =>
ShouldGarbageCollect -> Internal m blk -> m ()
persistBlks ShouldGarbageCollect
GarbageCollect Internal m blk
internal
      Cmd blk (TestIterator m blk) (TestFollower m blk)
UpdateLedgerSnapshots    -> () -> Success blk (TestIterator m blk) (TestFollower m blk)
forall {p} {blk} {it} {flr}. p -> Success blk it flr
ignore              (() -> Success blk (TestIterator m blk) (TestFollower m blk))
-> m ()
-> m (Success blk (TestIterator m blk) (TestFollower m blk))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Internal m blk -> m ()
forall (m :: * -> *) blk. Internal m blk -> m ()
intUpdateLedgerSnapshots Internal m blk
internal
      Cmd blk (TestIterator m blk) (TestFollower m blk)
WipeVolatileDB           -> Point blk -> Success blk (TestIterator m blk) (TestFollower m blk)
forall blk it flr. Point blk -> Success blk it flr
Point               (Point blk
 -> Success blk (TestIterator m blk) (TestFollower m blk))
-> m (Point blk)
-> m (Success blk (TestIterator m blk) (TestFollower m blk))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> ChainDBState m blk -> m (Point blk)
wipeVolatileDB ChainDBState m blk
st
  where
    mbGCedAllComponents :: Maybe
  (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
   SlotNo, IsEBB, SizeInBytes, Word16,
   SomeSecond (NestedCtxt Header) blk)
-> Success blk it flr
mbGCedAllComponents = MaybeGCedBlock
  (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
   SlotNo, IsEBB, SizeInBytes, Word16,
   SomeSecond (NestedCtxt Header) blk)
-> Success blk it flr
forall blk it flr.
MaybeGCedBlock (AllComponents blk) -> Success blk it flr
MbGCedAllComponents (MaybeGCedBlock
   (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
    SlotNo, IsEBB, SizeInBytes, Word16,
    SomeSecond (NestedCtxt Header) blk)
 -> Success blk it flr)
-> (Maybe
      (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
       SlotNo, IsEBB, SizeInBytes, Word16,
       SomeSecond (NestedCtxt Header) blk)
    -> MaybeGCedBlock
         (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
          SlotNo, IsEBB, SizeInBytes, Word16,
          SomeSecond (NestedCtxt Header) blk))
-> Maybe
     (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
      SlotNo, IsEBB, SizeInBytes, Word16,
      SomeSecond (NestedCtxt Header) blk)
-> Success blk it flr
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Bool
-> Maybe
     (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
      SlotNo, IsEBB, SizeInBytes, Word16,
      SomeSecond (NestedCtxt Header) blk)
-> MaybeGCedBlock
     (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
      SlotNo, IsEBB, SizeInBytes, Word16,
      SomeSecond (NestedCtxt Header) blk)
forall blk. Bool -> Maybe blk -> MaybeGCedBlock blk
MaybeGCedBlock Bool
True
    isValidResult :: Maybe Bool -> Success blk it flr
isValidResult = IsValidResult -> Success blk it flr
forall blk it flr. IsValidResult -> Success blk it flr
IsValid (IsValidResult -> Success blk it flr)
-> (Maybe Bool -> IsValidResult)
-> Maybe Bool
-> Success blk it flr
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Bool -> Maybe Bool -> IsValidResult
IsValidResult Bool
True
    iterResultGCed :: IteratorResult
  blk
  (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
   SlotNo, IsEBB, SizeInBytes, Word16,
   SomeSecond (NestedCtxt Header) blk)
-> Success blk it flr
iterResultGCed = IteratorResultGCed blk -> Success blk it flr
forall blk it flr. IteratorResultGCed blk -> Success blk it flr
IterResultGCed (IteratorResultGCed blk -> Success blk it flr)
-> (IteratorResult
      blk
      (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
       SlotNo, IsEBB, SizeInBytes, Word16,
       SomeSecond (NestedCtxt Header) blk)
    -> IteratorResultGCed blk)
-> IteratorResult
     blk
     (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
      SlotNo, IsEBB, SizeInBytes, Word16,
      SomeSecond (NestedCtxt Header) blk)
-> Success blk it flr
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Bool
-> IteratorResult
     blk
     (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
      SlotNo, IsEBB, SizeInBytes, Word16,
      SomeSecond (NestedCtxt Header) blk)
-> IteratorResultGCed blk
forall blk.
Bool
-> IteratorResult blk (AllComponents blk) -> IteratorResultGCed blk
IteratorResultGCed Bool
True
    iter :: Either (UnknownRange blk) (Iterator m blk (AllComponents blk))
-> m (Success blk (TestIterator m blk) (TestFollower m blk))
iter = (UnknownRange blk
 -> m (Success blk (TestIterator m blk) (TestFollower m blk)))
-> (Iterator m blk (AllComponents blk)
    -> m (Success blk (TestIterator m blk) (TestFollower m blk)))
-> Either (UnknownRange blk) (Iterator m blk (AllComponents blk))
-> m (Success blk (TestIterator m blk) (TestFollower m blk))
forall a c b. (a -> c) -> (b -> c) -> Either a b -> c
either (Success blk (TestIterator m blk) (TestFollower m blk)
-> m (Success blk (TestIterator m blk) (TestFollower m blk))
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (Success blk (TestIterator m blk) (TestFollower m blk)
 -> m (Success blk (TestIterator m blk) (TestFollower m blk)))
-> (UnknownRange blk
    -> Success blk (TestIterator m blk) (TestFollower m blk))
-> UnknownRange blk
-> m (Success blk (TestIterator m blk) (TestFollower m blk))
forall b c a. (b -> c) -> (a -> b) -> a -> c
. UnknownRange blk
-> Success blk (TestIterator m blk) (TestFollower m blk)
forall blk it flr. UnknownRange blk -> Success blk it flr
UnknownRange) ((TestIterator m blk
 -> Success blk (TestIterator m blk) (TestFollower m blk))
-> m (TestIterator m blk)
-> m (Success blk (TestIterator m blk) (TestFollower m blk))
forall a b. (a -> b) -> m a -> m b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap TestIterator m blk
-> Success blk (TestIterator m blk) (TestFollower m blk)
forall blk it flr. it -> Success blk it flr
Iter (m (TestIterator m blk)
 -> m (Success blk (TestIterator m blk) (TestFollower m blk)))
-> (Iterator m blk (AllComponents blk) -> m (TestIterator m blk))
-> Iterator m blk (AllComponents blk)
-> m (Success blk (TestIterator m blk) (TestFollower m blk))
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Iterator m blk (AllComponents blk) -> m (TestIterator m blk)
forall a. a -> m (WithEq a)
giveWithEq)
    follower :: Follower m blk (AllComponents blk)
-> m (Success blk (TestIterator m blk) (TestFollower m blk))
follower = (TestFollower m blk
 -> Success blk (TestIterator m blk) (TestFollower m blk))
-> m (TestFollower m blk)
-> m (Success blk (TestIterator m blk) (TestFollower m blk))
forall a b. (a -> b) -> m a -> m b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap TestFollower m blk
-> Success blk (TestIterator m blk) (TestFollower m blk)
forall blk it flr. flr -> Success blk it flr
Flr (m (TestFollower m blk)
 -> m (Success blk (TestIterator m blk) (TestFollower m blk)))
-> (Follower m blk (AllComponents blk) -> m (TestFollower m blk))
-> Follower m blk (AllComponents blk)
-> m (Success blk (TestIterator m blk) (TestFollower m blk))
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Follower m blk (AllComponents blk) -> m (TestFollower m blk)
forall a. a -> m (WithEq a)
giveWithEq
    ignore :: p -> Success blk it flr
ignore p
_ = () -> Success blk it flr
forall blk it flr. () -> Success blk it flr
Unit ()

    advanceAndAdd :: ChainDBState m blk -> SlotNo -> blk -> m (Point blk)
    advanceAndAdd :: ChainDBState m blk -> SlotNo -> blk -> m (Point blk)
advanceAndAdd ChainDBState { ChainDB m blk
$sel:chainDB:ChainDBState :: forall (m :: * -> *) blk. ChainDBState m blk -> ChainDB m blk
chainDB :: ChainDB m blk
chainDB } SlotNo
newCurSlot blk
blk = 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
$ StrictTVar m SlotNo -> (SlotNo -> SlotNo) -> STM m ()
forall (m :: * -> *) a.
MonadSTM m =>
StrictTVar m a -> (a -> a) -> STM m ()
modifyTVar StrictTVar m SlotNo
varCurSlot (SlotNo -> SlotNo -> SlotNo
forall a. Ord a => a -> a -> a
max SlotNo
newCurSlot)
      -- `blockProcessed` always returns 'Just'
      AddBlockResult blk
res <- ChainDB m blk
-> InvalidBlockPunishment m -> blk -> m (AddBlockResult blk)
forall (m :: * -> *) blk.
IOLike m =>
ChainDB m blk
-> InvalidBlockPunishment m -> blk -> m (AddBlockResult blk)
addBlock ChainDB m blk
chainDB InvalidBlockPunishment m
forall (m :: * -> *). Applicative m => InvalidBlockPunishment m
InvalidBlockPunishment.noPunishment blk
blk
      ChainDB m blk -> m ()
forall (m :: * -> *) blk. IOLike m => ChainDB m blk -> m ()
ChainDB.triggerChainSelection ChainDB m blk
chainDB
      Point blk -> m (Point blk)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (Point blk -> m (Point blk)) -> Point blk -> m (Point blk)
forall a b. (a -> b) -> a -> b
$ case AddBlockResult blk
res of
        FailedToAddBlock String
f       -> String -> Point blk
forall a. HasCallStack => String -> a
error (String -> Point blk) -> String -> Point blk
forall a b. (a -> b) -> a -> b
$ String
"advanceAndAdd: block not added - " String -> ShowS
forall a. [a] -> [a] -> [a]
++ String
f
        SuccesfullyAddedBlock Point blk
pt -> Point blk
pt

    updateLoE :: ChainDBState m blk -> AnchoredFragment blk -> m (Point blk)
    updateLoE :: ChainDBState m blk -> AnchoredFragment blk -> m (Point blk)
updateLoE ChainDBState { ChainDB m blk
$sel:chainDB:ChainDBState :: forall (m :: * -> *) blk. ChainDBState m blk -> ChainDB m blk
chainDB :: ChainDB m blk
chainDB } AnchoredFragment blk
frag = do
      let headersFrag :: AnchoredFragment (Header blk)
headersFrag = (blk -> Header blk)
-> AnchoredFragment blk -> AnchoredFragment (Header blk)
forall block2 block1.
(HasHeader block2, HeaderHash block1 ~ HeaderHash block2) =>
(block1 -> block2)
-> AnchoredFragment block1 -> AnchoredFragment block2
AF.mapAnchoredFragment blk -> Header blk
forall blk. GetHeader blk => blk -> Header blk
getHeader AnchoredFragment blk
frag
      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 (AnchoredFragment (Header blk))
-> AnchoredFragment (Header blk) -> STM m ()
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
StrictTVar m a -> a -> STM m ()
writeTVar StrictTVar m (AnchoredFragment (Header blk))
varLoEFragment AnchoredFragment (Header blk)
headersFrag
      ChainDB m blk -> m ()
forall (m :: * -> *) blk. IOLike m => ChainDB m blk -> m ()
ChainDB.triggerChainSelection ChainDB m blk
chainDB
      STM m (Point blk) -> m (Point blk)
forall a. HasCallStack => STM m a -> m a
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
STM m a -> m a
atomically (STM m (Point blk) -> m (Point blk))
-> STM m (Point blk) -> m (Point blk)
forall a b. (a -> b) -> a -> b
$ ChainDB m blk -> STM m (Point blk)
forall (m :: * -> *) blk. ChainDB m blk -> STM m (Point blk)
getTipPoint ChainDB m blk
chainDB

    wipeVolatileDB :: ChainDBState m blk -> m (Point blk)
    wipeVolatileDB :: ChainDBState m blk -> m (Point blk)
wipeVolatileDB ChainDBState m blk
st = do
      ChainDBState m blk -> m ()
forall (m :: * -> *) blk. IOLike m => ChainDBState m blk -> m ()
close ChainDBState m blk
st
      m () -> m ()
forall (f :: * -> *) a. Functor f => f a -> f ()
void (m () -> m ()) -> m () -> m ()
forall a b. (a -> b) -> a -> b
$ 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
$ StrictTMVar m MockFS -> MockFS -> STM m ()
forall (m :: * -> *) a.
MonadSTM m =>
StrictTMVar m a -> a -> STM m ()
writeTMVar StrictTMVar m MockFS
varVolatileDbFs MockFS
Mock.empty
      ChainDBEnv m blk -> m ()
forall (m :: * -> *) blk.
(IOLike m, TestConstraints blk) =>
ChainDBEnv m blk -> m ()
reopen ChainDBEnv m blk
env
      ChainDB { STM m (Point blk)
getTipPoint :: forall (m :: * -> *) blk. ChainDB m blk -> STM m (Point blk)
getTipPoint :: STM m (Point blk)
getTipPoint } <- ChainDBState m blk -> ChainDB m blk
forall (m :: * -> *) blk. ChainDBState m blk -> ChainDB m blk
chainDB (ChainDBState m blk -> ChainDB m blk)
-> m (ChainDBState m blk) -> m (ChainDB m blk)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> StrictTVar m (ChainDBState m blk) -> m (ChainDBState m blk)
forall (m :: * -> *) a. MonadSTM m => StrictTVar m a -> m a
readTVarIO StrictTVar m (ChainDBState m blk)
varDB
      STM m (Point blk) -> m (Point blk)
forall a. HasCallStack => STM m a -> m a
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
STM m a -> m a
atomically STM m (Point blk)
getTipPoint

    giveWithEq :: a -> m (WithEq a)
    giveWithEq :: forall a. a -> m (WithEq a)
giveWithEq a
a =
      (Id -> WithEq a) -> m Id -> m (WithEq a)
forall a b. (a -> b) -> m a -> m b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap (Id -> a -> WithEq a
forall a. Id -> a -> WithEq a
`WithEq` a
a) (m Id -> m (WithEq a)) -> m Id -> m (WithEq a)
forall a b. (a -> b) -> a -> b
$ STM m Id -> m Id
forall a. HasCallStack => STM m a -> m a
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
STM m a -> m a
atomically (STM m Id -> m Id) -> STM m Id -> m Id
forall a b. (a -> b) -> a -> b
$ StrictTVar m Id -> (Id -> (Id, Id)) -> STM m Id
forall (m :: * -> *) s a.
MonadSTM m =>
StrictTVar m s -> (s -> (a, s)) -> STM m a
stateTVar StrictTVar m Id
varNextId ((Id -> (Id, Id)) -> STM m Id) -> (Id -> (Id, Id)) -> STM m Id
forall a b. (a -> b) -> a -> b
$ \Id
i -> (Id
i, Id -> Id
forall a. Enum a => a -> a
succ Id
i)

persistBlks :: IOLike m => ShouldGarbageCollect -> ChainDB.Internal m blk -> m ()
persistBlks :: forall (m :: * -> *) blk.
IOLike m =>
ShouldGarbageCollect -> Internal m blk -> m ()
persistBlks ShouldGarbageCollect
collectGarbage ChainDB.Internal{m Void
m ()
m (WithOrigin SlotNo)
StrictTVar m (m ())
SlotNo -> m ()
intAddBlockRunner :: forall (m :: * -> *) blk. Internal m blk -> m Void
intUpdateLedgerSnapshots :: forall (m :: * -> *) blk. Internal m blk -> m ()
intCopyToImmutableDB :: m (WithOrigin SlotNo)
intGarbageCollect :: SlotNo -> m ()
intUpdateLedgerSnapshots :: m ()
intAddBlockRunner :: m Void
intKillBgThreads :: StrictTVar m (m ())
intCopyToImmutableDB :: forall (m :: * -> *) blk. Internal m blk -> m (WithOrigin SlotNo)
intGarbageCollect :: forall (m :: * -> *) blk. Internal m blk -> SlotNo -> m ()
intKillBgThreads :: forall (m :: * -> *) blk. Internal m blk -> StrictTVar m (m ())
..} = do
    WithOrigin SlotNo
mSlotNo <- m (WithOrigin SlotNo)
intCopyToImmutableDB
    case (ShouldGarbageCollect
collectGarbage, WithOrigin SlotNo
mSlotNo) of
      (ShouldGarbageCollect
DoNotGarbageCollect, WithOrigin SlotNo
_               ) -> () -> m ()
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure ()
      (ShouldGarbageCollect
GarbageCollect     , WithOrigin SlotNo
Origin          ) -> () -> m ()
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure ()
      (ShouldGarbageCollect
GarbageCollect     , NotOrigin SlotNo
slotNo) -> SlotNo -> m ()
intGarbageCollect SlotNo
slotNo

-- | Result type for 'getBlock'. Note that the real implementation of
-- 'getBlock' is non-deterministic: if the requested block is older than @k@
-- and not part of the current chain, it might have been garbage collected.
--
-- The first source of non-determinism is whether or not the background thread
-- that performs garbage collection has been run yet. We disable this thread in
-- the state machine tests and instead generate the 'PersistBlksThenGC'
-- command that triggers the garbage collection explicitly, after persisting
-- the blocks older than @k@ from the current chain's tip. So this source of
-- non-determinism is not a problem in the tests.
--
-- However, there is a second source of non-determinism: if a garbage
-- collection has been performed and the block was eligible for collection, it
-- might still not have been removed because it was part of a file that
-- contained other blocks that cannot be garbage collected yet. So the block
-- is still left in the VolatileDB. We do not try to imitate this behaviour,
-- which would couple the test too tightly to the actual implementation.
-- Instead, we accept this source of non-determinism and are more lenient when
-- comparing the results of 'getBlock' when the block may have been garbage
-- collected.
--
-- Equality of two 'MaybeGCedBlock' is determined as follows:
-- * If both are produced by a model implementation, then the @Maybe blk@s must
--   be equal, as the these results are deterministic.
-- * If at least one of them is produced by a real implementation, then:
--   * If either is 'Nothing', which means the block might have been
--     garbage-collected, then they are equal (even if the other is 'Just',
--     which means it was not yet garbage-collected).
--   * If both are 'Just's, then the blocks must be equal.
--
-- In practice, this equality is used when comparing the result of the real
-- implementation with the result of the model implementation.
data MaybeGCedBlock blk = MaybeGCedBlock
  { forall blk. MaybeGCedBlock blk -> Bool
real    :: Bool
    -- ^ 'True':  result of calling 'getBlock' on the real implementation
    -- ^ 'False': result of calling 'getBlock' on the model implementation
  , forall blk. MaybeGCedBlock blk -> Maybe blk
mbBlock :: Maybe blk
    -- ^ A value of 'Nothing' in this field indicates that the block might have
    -- been garbage collected.
  } deriving (Int -> MaybeGCedBlock blk -> ShowS
[MaybeGCedBlock blk] -> ShowS
MaybeGCedBlock blk -> String
(Int -> MaybeGCedBlock blk -> ShowS)
-> (MaybeGCedBlock blk -> String)
-> ([MaybeGCedBlock blk] -> ShowS)
-> Show (MaybeGCedBlock blk)
forall blk. Show blk => Int -> MaybeGCedBlock blk -> ShowS
forall blk. Show blk => [MaybeGCedBlock blk] -> ShowS
forall blk. Show blk => MaybeGCedBlock blk -> String
forall a.
(Int -> a -> ShowS) -> (a -> String) -> ([a] -> ShowS) -> Show a
$cshowsPrec :: forall blk. Show blk => Int -> MaybeGCedBlock blk -> ShowS
showsPrec :: Int -> MaybeGCedBlock blk -> ShowS
$cshow :: forall blk. Show blk => MaybeGCedBlock blk -> String
show :: MaybeGCedBlock blk -> String
$cshowList :: forall blk. Show blk => [MaybeGCedBlock blk] -> ShowS
showList :: [MaybeGCedBlock blk] -> ShowS
Show)

instance Eq blk => Eq (MaybeGCedBlock blk) where
  MaybeGCedBlock Bool
real1 Maybe blk
mbBlock1 == :: MaybeGCedBlock blk -> MaybeGCedBlock blk -> Bool
== MaybeGCedBlock Bool
real2 Maybe blk
mbBlock2 =
    -- Two @MaybeGCedBlock@s are equal iff either:
    --
    -- - they are both produced by the model and contain the same result, or
    -- - at least one of them was garbage collected, or
    -- - none of them were garbage collected and they contain the same block.
    --
    -- See the comments on 'MaybeGCedBlock' for a justification on why we
    -- implemented this form of lenient equality.
      case (Bool
real1, Bool
real2) of
        (Bool
False, Bool
False) -> Maybe blk
mbBlock1 Maybe blk -> Maybe blk -> Bool
forall a. Eq a => a -> a -> Bool
== Maybe blk
mbBlock2
        (Bool
True,  Bool
_)     -> Bool
eqIfJust
        (Bool
_,     Bool
True)  -> Bool
eqIfJust
    where
      eqIfJust :: Bool
eqIfJust = case (Maybe blk
mbBlock1, Maybe blk
mbBlock2) of
        (Just blk
b1, Just blk
b2) -> blk
b1 blk -> blk -> Bool
forall a. Eq a => a -> a -> Bool
== blk
b2
        (Maybe blk, Maybe blk)
_                  -> Bool
True

-- | Similar to 'MaybeGCedBlock', but for the block returned by
-- 'iteratorNext'. A garbage-collected block could result in
-- 'IteratorBlockGCed' instead of 'IteratorResult'.
data IteratorResultGCed blk = IteratorResultGCed
  { forall blk. IteratorResultGCed blk -> Bool
real       :: Bool
    -- ^ 'True':  result of calling 'getBlock' on the real implementation
    -- ^ 'False': result of calling 'getBlock' on the model implementation
  , forall blk.
IteratorResultGCed blk -> IteratorResult blk (AllComponents blk)
iterResult :: IteratorResult blk (AllComponents blk)
  }

deriving instance ( Show blk
                  , Show (Header blk)
                  , StandardHash blk
                  , HasNestedContent Header blk
                  ) => Show (IteratorResultGCed blk)

instance (Eq blk, Eq (Header blk), StandardHash blk, HasNestedContent Header blk)
      => Eq (IteratorResultGCed blk) where
  IteratorResultGCed Bool
real1 IteratorResult
  blk
  (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
   SlotNo, IsEBB, SizeInBytes, Word16,
   SomeSecond (NestedCtxt Header) blk)
iterResult1 == :: IteratorResultGCed blk -> IteratorResultGCed blk -> Bool
== IteratorResultGCed Bool
real2 IteratorResult
  blk
  (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
   SlotNo, IsEBB, SizeInBytes, Word16,
   SomeSecond (NestedCtxt Header) blk)
iterResult2 =
      case (Bool
real1, Bool
real2) of
        (Bool
False, Bool
False) -> IteratorResult
  blk
  (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
   SlotNo, IsEBB, SizeInBytes, Word16,
   SomeSecond (NestedCtxt Header) blk)
iterResult1 IteratorResult
  blk
  (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
   SlotNo, IsEBB, SizeInBytes, Word16,
   SomeSecond (NestedCtxt Header) blk)
-> IteratorResult
     blk
     (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
      SlotNo, IsEBB, SizeInBytes, Word16,
      SomeSecond (NestedCtxt Header) blk)
-> Bool
forall a. Eq a => a -> a -> Bool
== IteratorResult
  blk
  (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
   SlotNo, IsEBB, SizeInBytes, Word16,
   SomeSecond (NestedCtxt Header) blk)
iterResult2
        (Bool
True,  Bool
_)     -> Bool
eqIfNotGCed
        (Bool
_,     Bool
True)  -> Bool
eqIfNotGCed
    where
      eqIfNotGCed :: Bool
eqIfNotGCed = case (IteratorResult
  blk
  (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
   SlotNo, IsEBB, SizeInBytes, Word16,
   SomeSecond (NestedCtxt Header) blk)
iterResult1, IteratorResult
  blk
  (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
   SlotNo, IsEBB, SizeInBytes, Word16,
   SomeSecond (NestedCtxt Header) blk)
iterResult2) of
        (IteratorBlockGCed {}, IteratorResult
  blk
  (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
   SlotNo, IsEBB, SizeInBytes, Word16,
   SomeSecond (NestedCtxt Header) blk)
_)                    -> Bool
True
        (IteratorResult
  blk
  (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
   SlotNo, IsEBB, SizeInBytes, Word16,
   SomeSecond (NestedCtxt Header) blk)
_,                    IteratorBlockGCed {}) -> Bool
True
        (IteratorResult (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
 SlotNo, IsEBB, SizeInBytes, Word16,
 SomeSecond (NestedCtxt Header) blk)
b1,    IteratorResult (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
 SlotNo, IsEBB, SizeInBytes, Word16,
 SomeSecond (NestedCtxt Header) blk)
b2)    -> (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
 SlotNo, IsEBB, SizeInBytes, Word16,
 SomeSecond (NestedCtxt Header) blk)
b1 (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
 SlotNo, IsEBB, SizeInBytes, Word16,
 SomeSecond (NestedCtxt Header) blk)
-> (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
    SlotNo, IsEBB, SizeInBytes, Word16,
    SomeSecond (NestedCtxt Header) blk)
-> Bool
forall a. Eq a => a -> a -> Bool
== (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
 SlotNo, IsEBB, SizeInBytes, Word16,
 SomeSecond (NestedCtxt Header) blk)
b2
        (IteratorResult
  blk
  (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
   SlotNo, IsEBB, SizeInBytes, Word16,
   SomeSecond (NestedCtxt Header) blk)
IteratorExhausted,    IteratorResult
  blk
  (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
   SlotNo, IsEBB, SizeInBytes, Word16,
   SomeSecond (NestedCtxt Header) blk)
IteratorExhausted)    -> Bool
True
        (IteratorResult
   blk
   (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
    SlotNo, IsEBB, SizeInBytes, Word16,
    SomeSecond (NestedCtxt Header) blk),
 IteratorResult
   blk
   (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
    SlotNo, IsEBB, SizeInBytes, Word16,
    SomeSecond (NestedCtxt Header) blk))
_                                            -> Bool
False

-- | The model knows about all valid blocks whereas the real implementation
-- only knows about blocks that have been validated in the VolatileDB if they
-- were part of a chain selected by chain selection.
--
-- 'Nothing' means the validity of the block is unknown.
--
-- When the real implementation returned 'Nothing', we ignore the result of
-- the model. If the model returned 'Nothing', the real implementation must
-- too. In the 'Just' case, the result of the implementation and the model
-- must match.
data IsValidResult = IsValidResult
  { IsValidResult -> Bool
real    :: Bool
    -- ^ 'True':  result of calling 'getIsValid' on the real implementation
    -- ^ 'False': result of calling 'getIsValid' on the model implementation
  , IsValidResult -> Maybe Bool
isValid :: Maybe Bool
  } deriving (Int -> IsValidResult -> ShowS
[IsValidResult] -> ShowS
IsValidResult -> String
(Int -> IsValidResult -> ShowS)
-> (IsValidResult -> String)
-> ([IsValidResult] -> ShowS)
-> Show IsValidResult
forall a.
(Int -> a -> ShowS) -> (a -> String) -> ([a] -> ShowS) -> Show a
$cshowsPrec :: Int -> IsValidResult -> ShowS
showsPrec :: Int -> IsValidResult -> ShowS
$cshow :: IsValidResult -> String
show :: IsValidResult -> String
$cshowList :: [IsValidResult] -> ShowS
showList :: [IsValidResult] -> ShowS
Show)

instance Eq IsValidResult where
  IsValidResult Bool
real1 Maybe Bool
isValid1 == :: IsValidResult -> IsValidResult -> Bool
== IsValidResult Bool
real2 Maybe Bool
isValid2 =
      case (Bool
real1, Bool
real2) of
        (Bool
False, Bool
False) -> Maybe Bool
isValid1 Maybe Bool -> Maybe Bool -> Bool
forall a. Eq a => a -> a -> Bool
== Maybe Bool
isValid2
        (Bool
True,  Bool
False) -> Maybe Bool -> Maybe Bool -> Bool
forall {a}. Eq a => Maybe a -> Maybe a -> Bool
realMatchesModel Maybe Bool
isValid1 Maybe Bool
isValid2
        (Bool
False, Bool
True)  -> Maybe Bool -> Maybe Bool -> Bool
forall {a}. Eq a => Maybe a -> Maybe a -> Bool
realMatchesModel Maybe Bool
isValid2 Maybe Bool
isValid1
        (Bool
True,  Bool
True)  -> Bool
eqIfJust
    where
      eqIfJust :: Bool
eqIfJust = case (Maybe Bool
isValid1, Maybe Bool
isValid2) of
        (Just Bool
x1, Just Bool
x2) -> Bool
x1 Bool -> Bool -> Bool
forall a. Eq a => a -> a -> Bool
== Bool
x2
        (Maybe Bool, Maybe Bool)
_                  -> Bool
True

      realMatchesModel :: Maybe a -> Maybe a -> Bool
realMatchesModel Maybe a
real Maybe a
model = case (Maybe a
real, Maybe a
model) of
        (Just a
x1, Just a
x2) -> a
x1 a -> a -> Bool
forall a. Eq a => a -> a -> Bool
== a
x2
        (Maybe a
Nothing, Maybe a
Nothing) -> Bool
True
        (Maybe a
Nothing, Just a
_)  -> Bool
True
        (Just a
_,  Maybe a
Nothing) -> Bool
False

{-------------------------------------------------------------------------------
  Max clock skew
-------------------------------------------------------------------------------}

-- | Max clock skew in number of slots
newtype MaxClockSkew = MaxClockSkew Word64
  deriving (MaxClockSkew -> MaxClockSkew -> Bool
(MaxClockSkew -> MaxClockSkew -> Bool)
-> (MaxClockSkew -> MaxClockSkew -> Bool) -> Eq MaxClockSkew
forall a. (a -> a -> Bool) -> (a -> a -> Bool) -> Eq a
$c== :: MaxClockSkew -> MaxClockSkew -> Bool
== :: MaxClockSkew -> MaxClockSkew -> Bool
$c/= :: MaxClockSkew -> MaxClockSkew -> Bool
/= :: MaxClockSkew -> MaxClockSkew -> Bool
Eq, Int -> MaxClockSkew -> ShowS
[MaxClockSkew] -> ShowS
MaxClockSkew -> String
(Int -> MaxClockSkew -> ShowS)
-> (MaxClockSkew -> String)
-> ([MaxClockSkew] -> ShowS)
-> Show MaxClockSkew
forall a.
(Int -> a -> ShowS) -> (a -> String) -> ([a] -> ShowS) -> Show a
$cshowsPrec :: Int -> MaxClockSkew -> ShowS
showsPrec :: Int -> MaxClockSkew -> ShowS
$cshow :: MaxClockSkew -> String
show :: MaxClockSkew -> String
$cshowList :: [MaxClockSkew] -> ShowS
showList :: [MaxClockSkew] -> ShowS
Show)

instance Arbitrary MaxClockSkew where
  -- TODO make sure no blocks from the future exceed the max clock skew:
  -- <https://github.com/IntersectMBO/ouroboros-network/issues/2232>
  arbitrary :: Gen MaxClockSkew
arbitrary = MaxClockSkew -> Gen MaxClockSkew
forall a. a -> Gen a
forall (m :: * -> *) a. Monad m => a -> m a
return (MaxClockSkew -> Gen MaxClockSkew)
-> MaxClockSkew -> Gen MaxClockSkew
forall a b. (a -> b) -> a -> b
$ Word64 -> MaxClockSkew
MaxClockSkew Word64
100000
  -- arbitrary = MaxClockSkew <$> choose (0, 3)
  -- -- We're only interested in 0 or 1
  -- shrink (MaxClockSkew 0) = []
  -- shrink (MaxClockSkew 1) = []
  -- shrink (MaxClockSkew _) = MaxClockSkew <$> [0, 1]

{-------------------------------------------------------------------------------
  Instantiating the semantics
-------------------------------------------------------------------------------}

-- | Responses are either successful termination or an error.
newtype Resp blk it flr = Resp
  { forall blk it flr.
Resp blk it flr -> Either (ChainDbError blk) (Success blk it flr)
getResp :: Either (ChainDbError blk) (Success blk it flr) }
  deriving ((forall a b. (a -> b) -> Resp blk it a -> Resp blk it b)
-> (forall a b. a -> Resp blk it b -> Resp blk it a)
-> Functor (Resp blk it)
forall a b. a -> Resp blk it b -> Resp blk it a
forall a b. (a -> b) -> Resp blk it a -> Resp blk it b
forall blk it a b. a -> Resp blk it b -> Resp blk it a
forall blk it a b. (a -> b) -> Resp blk it a -> Resp blk it b
forall (f :: * -> *).
(forall a b. (a -> b) -> f a -> f b)
-> (forall a b. a -> f b -> f a) -> Functor f
$cfmap :: forall blk it a b. (a -> b) -> Resp blk it a -> Resp blk it b
fmap :: forall a b. (a -> b) -> Resp blk it a -> Resp blk it b
$c<$ :: forall blk it a b. a -> Resp blk it b -> Resp blk it a
<$ :: forall a b. a -> Resp blk it b -> Resp blk it a
Functor, (forall m. Monoid m => Resp blk it m -> m)
-> (forall m a. Monoid m => (a -> m) -> Resp blk it a -> m)
-> (forall m a. Monoid m => (a -> m) -> Resp blk it a -> m)
-> (forall a b. (a -> b -> b) -> b -> Resp blk it a -> b)
-> (forall a b. (a -> b -> b) -> b -> Resp blk it a -> b)
-> (forall b a. (b -> a -> b) -> b -> Resp blk it a -> b)
-> (forall b a. (b -> a -> b) -> b -> Resp blk it a -> b)
-> (forall a. (a -> a -> a) -> Resp blk it a -> a)
-> (forall a. (a -> a -> a) -> Resp blk it a -> a)
-> (forall a. Resp blk it a -> [a])
-> (forall a. Resp blk it a -> Bool)
-> (forall a. Resp blk it a -> Int)
-> (forall a. Eq a => a -> Resp blk it a -> Bool)
-> (forall a. Ord a => Resp blk it a -> a)
-> (forall a. Ord a => Resp blk it a -> a)
-> (forall a. Num a => Resp blk it a -> a)
-> (forall a. Num a => Resp blk it a -> a)
-> Foldable (Resp blk it)
forall a. Eq a => a -> Resp blk it a -> Bool
forall a. Num a => Resp blk it a -> a
forall a. Ord a => Resp blk it a -> a
forall m. Monoid m => Resp blk it m -> m
forall a. Resp blk it a -> Bool
forall a. Resp blk it a -> Int
forall a. Resp blk it a -> [a]
forall a. (a -> a -> a) -> Resp blk it a -> a
forall m a. Monoid m => (a -> m) -> Resp blk it a -> m
forall b a. (b -> a -> b) -> b -> Resp blk it a -> b
forall a b. (a -> b -> b) -> b -> Resp blk it a -> b
forall blk it a. Eq a => a -> Resp blk it a -> Bool
forall blk it a. Num a => Resp blk it a -> a
forall blk it a. Ord a => Resp blk it a -> a
forall blk it m. Monoid m => Resp blk it m -> m
forall blk it a. Resp blk it a -> Bool
forall blk it a. Resp blk it a -> Int
forall blk it a. Resp blk it a -> [a]
forall blk it a. (a -> a -> a) -> Resp blk it a -> a
forall blk it m a. Monoid m => (a -> m) -> Resp blk it a -> m
forall blk it b a. (b -> a -> b) -> b -> Resp blk it a -> b
forall blk it a b. (a -> b -> b) -> b -> Resp blk it a -> b
forall (t :: * -> *).
(forall m. Monoid m => t m -> m)
-> (forall m a. Monoid m => (a -> m) -> t a -> m)
-> (forall m a. Monoid m => (a -> m) -> t a -> m)
-> (forall a b. (a -> b -> b) -> b -> t a -> b)
-> (forall a b. (a -> b -> b) -> b -> t a -> b)
-> (forall b a. (b -> a -> b) -> b -> t a -> b)
-> (forall b a. (b -> a -> b) -> b -> t a -> b)
-> (forall a. (a -> a -> a) -> t a -> a)
-> (forall a. (a -> a -> a) -> t a -> a)
-> (forall a. t a -> [a])
-> (forall a. t a -> Bool)
-> (forall a. t a -> Int)
-> (forall a. Eq a => a -> t a -> Bool)
-> (forall a. Ord a => t a -> a)
-> (forall a. Ord a => t a -> a)
-> (forall a. Num a => t a -> a)
-> (forall a. Num a => t a -> a)
-> Foldable t
$cfold :: forall blk it m. Monoid m => Resp blk it m -> m
fold :: forall m. Monoid m => Resp blk it m -> m
$cfoldMap :: forall blk it m a. Monoid m => (a -> m) -> Resp blk it a -> m
foldMap :: forall m a. Monoid m => (a -> m) -> Resp blk it a -> m
$cfoldMap' :: forall blk it m a. Monoid m => (a -> m) -> Resp blk it a -> m
foldMap' :: forall m a. Monoid m => (a -> m) -> Resp blk it a -> m
$cfoldr :: forall blk it a b. (a -> b -> b) -> b -> Resp blk it a -> b
foldr :: forall a b. (a -> b -> b) -> b -> Resp blk it a -> b
$cfoldr' :: forall blk it a b. (a -> b -> b) -> b -> Resp blk it a -> b
foldr' :: forall a b. (a -> b -> b) -> b -> Resp blk it a -> b
$cfoldl :: forall blk it b a. (b -> a -> b) -> b -> Resp blk it a -> b
foldl :: forall b a. (b -> a -> b) -> b -> Resp blk it a -> b
$cfoldl' :: forall blk it b a. (b -> a -> b) -> b -> Resp blk it a -> b
foldl' :: forall b a. (b -> a -> b) -> b -> Resp blk it a -> b
$cfoldr1 :: forall blk it a. (a -> a -> a) -> Resp blk it a -> a
foldr1 :: forall a. (a -> a -> a) -> Resp blk it a -> a
$cfoldl1 :: forall blk it a. (a -> a -> a) -> Resp blk it a -> a
foldl1 :: forall a. (a -> a -> a) -> Resp blk it a -> a
$ctoList :: forall blk it a. Resp blk it a -> [a]
toList :: forall a. Resp blk it a -> [a]
$cnull :: forall blk it a. Resp blk it a -> Bool
null :: forall a. Resp blk it a -> Bool
$clength :: forall blk it a. Resp blk it a -> Int
length :: forall a. Resp blk it a -> Int
$celem :: forall blk it a. Eq a => a -> Resp blk it a -> Bool
elem :: forall a. Eq a => a -> Resp blk it a -> Bool
$cmaximum :: forall blk it a. Ord a => Resp blk it a -> a
maximum :: forall a. Ord a => Resp blk it a -> a
$cminimum :: forall blk it a. Ord a => Resp blk it a -> a
minimum :: forall a. Ord a => Resp blk it a -> a
$csum :: forall blk it a. Num a => Resp blk it a -> a
sum :: forall a. Num a => Resp blk it a -> a
$cproduct :: forall blk it a. Num a => Resp blk it a -> a
product :: forall a. Num a => Resp blk it a -> a
Foldable, Functor (Resp blk it)
Foldable (Resp blk it)
(Functor (Resp blk it), Foldable (Resp blk it)) =>
(forall (f :: * -> *) a b.
 Applicative f =>
 (a -> f b) -> Resp blk it a -> f (Resp blk it b))
-> (forall (f :: * -> *) a.
    Applicative f =>
    Resp blk it (f a) -> f (Resp blk it a))
-> (forall (m :: * -> *) a b.
    Monad m =>
    (a -> m b) -> Resp blk it a -> m (Resp blk it b))
-> (forall (m :: * -> *) a.
    Monad m =>
    Resp blk it (m a) -> m (Resp blk it a))
-> Traversable (Resp blk it)
forall blk it. Functor (Resp blk it)
forall blk it. Foldable (Resp blk it)
forall blk it (m :: * -> *) a.
Monad m =>
Resp blk it (m a) -> m (Resp blk it a)
forall blk it (f :: * -> *) a.
Applicative f =>
Resp blk it (f a) -> f (Resp blk it a)
forall blk it (m :: * -> *) a b.
Monad m =>
(a -> m b) -> Resp blk it a -> m (Resp blk it b)
forall blk it (f :: * -> *) a b.
Applicative f =>
(a -> f b) -> Resp blk it a -> f (Resp blk it b)
forall (t :: * -> *).
(Functor t, Foldable t) =>
(forall (f :: * -> *) a b.
 Applicative f =>
 (a -> f b) -> t a -> f (t b))
-> (forall (f :: * -> *) a. Applicative f => t (f a) -> f (t a))
-> (forall (m :: * -> *) a b.
    Monad m =>
    (a -> m b) -> t a -> m (t b))
-> (forall (m :: * -> *) a. Monad m => t (m a) -> m (t a))
-> Traversable t
forall (m :: * -> *) a.
Monad m =>
Resp blk it (m a) -> m (Resp blk it a)
forall (f :: * -> *) a.
Applicative f =>
Resp blk it (f a) -> f (Resp blk it a)
forall (m :: * -> *) a b.
Monad m =>
(a -> m b) -> Resp blk it a -> m (Resp blk it b)
forall (f :: * -> *) a b.
Applicative f =>
(a -> f b) -> Resp blk it a -> f (Resp blk it b)
$ctraverse :: forall blk it (f :: * -> *) a b.
Applicative f =>
(a -> f b) -> Resp blk it a -> f (Resp blk it b)
traverse :: forall (f :: * -> *) a b.
Applicative f =>
(a -> f b) -> Resp blk it a -> f (Resp blk it b)
$csequenceA :: forall blk it (f :: * -> *) a.
Applicative f =>
Resp blk it (f a) -> f (Resp blk it a)
sequenceA :: forall (f :: * -> *) a.
Applicative f =>
Resp blk it (f a) -> f (Resp blk it a)
$cmapM :: forall blk it (m :: * -> *) a b.
Monad m =>
(a -> m b) -> Resp blk it a -> m (Resp blk it b)
mapM :: forall (m :: * -> *) a b.
Monad m =>
(a -> m b) -> Resp blk it a -> m (Resp blk it b)
$csequence :: forall blk it (m :: * -> *) a.
Monad m =>
Resp blk it (m a) -> m (Resp blk it a)
sequence :: forall (m :: * -> *) a.
Monad m =>
Resp blk it (m a) -> m (Resp blk it a)
Traversable)

deriving instance (TestConstraints blk, Show it, Show flr)
               => Show (Resp blk it flr)

instance (TestConstraints blk, Eq it, Eq flr) => Eq (Resp blk it flr) where
  Resp (Left  ChainDbError blk
e) == :: Resp blk it flr -> Resp blk it flr -> Bool
== Resp (Left  ChainDbError blk
e') = ChainDbError blk
e ChainDbError blk -> ChainDbError blk -> Bool
forall a. Eq a => a -> a -> Bool
== ChainDbError blk
e'
  Resp (Right Success blk it flr
a) == Resp (Right Success blk it flr
a') = Success blk it flr
a Success blk it flr -> Success blk it flr -> Bool
forall a. Eq a => a -> a -> Bool
== Success blk it flr
a'
  Resp blk it flr
_              == Resp blk it flr
_               = Bool
False


type DBModel blk = Model.Model blk

-- We can't reuse 'run' because the 'ChainDB' API uses 'STM'. Instead, we call
-- the model directly.
runPure :: forall blk.
           TestConstraints blk
        => TopLevelConfig  blk
        -> Cmd             blk IteratorId FollowerId
        -> DBModel         blk
        -> (Resp           blk IteratorId FollowerId, DBModel blk)
runPure :: forall blk.
TestConstraints blk =>
TopLevelConfig blk
-> Cmd blk Int Int
-> DBModel blk
-> (Resp blk Int Int, DBModel blk)
runPure TopLevelConfig blk
cfg = \case
    AddBlock blk
blk             -> (Point blk -> Success blk Int Int)
-> (DBModel blk -> (Point blk, DBModel blk))
-> DBModel blk
-> (Resp blk Int Int, DBModel blk)
forall {a} {blk} {it} {flr} {blk}.
(a -> Success blk it flr)
-> (Model blk -> (a, Model blk))
-> Model blk
-> (Resp blk it flr, Model blk)
ok  Point blk -> Success blk Int Int
forall blk it flr. Point blk -> Success blk it flr
Point               ((DBModel blk -> (Point blk, DBModel blk))
 -> DBModel blk -> (Resp blk Int Int, DBModel blk))
-> (DBModel blk -> (Point blk, DBModel blk))
-> DBModel blk
-> (Resp blk Int Int, DBModel blk)
forall a b. (a -> b) -> a -> b
$ (DBModel blk -> (Point blk, DBModel blk))
-> DBModel blk -> (Point blk, DBModel blk)
forall {t} {t}. (t -> t) -> t -> t
update  (SlotNo -> blk -> DBModel blk -> (Point blk, DBModel blk)
advanceAndAdd (blk -> SlotNo
forall b. HasHeader b => b -> SlotNo
blockSlot blk
blk) blk
blk)
    AddFutureBlock blk
blk SlotNo
s     -> (Point blk -> Success blk Int Int)
-> (DBModel blk -> (Point blk, DBModel blk))
-> DBModel blk
-> (Resp blk Int Int, DBModel blk)
forall {a} {blk} {it} {flr} {blk}.
(a -> Success blk it flr)
-> (Model blk -> (a, Model blk))
-> Model blk
-> (Resp blk it flr, Model blk)
ok  Point blk -> Success blk Int Int
forall blk it flr. Point blk -> Success blk it flr
Point               ((DBModel blk -> (Point blk, DBModel blk))
 -> DBModel blk -> (Resp blk Int Int, DBModel blk))
-> (DBModel blk -> (Point blk, DBModel blk))
-> DBModel blk
-> (Resp blk Int Int, DBModel blk)
forall a b. (a -> b) -> a -> b
$ (DBModel blk -> (Point blk, DBModel blk))
-> DBModel blk -> (Point blk, DBModel blk)
forall {t} {t}. (t -> t) -> t -> t
update  (SlotNo -> blk -> DBModel blk -> (Point blk, DBModel blk)
advanceAndAdd SlotNo
s               blk
blk)
    Cmd blk Int Int
GetCurrentChain          -> (AnchoredFragment (Header blk) -> Success blk Int Int)
-> (DBModel blk -> (AnchoredFragment (Header blk), DBModel blk))
-> DBModel blk
-> (Resp blk Int Int, DBModel blk)
forall {a} {blk} {it} {flr} {blk}.
(a -> Success blk it flr)
-> (Model blk -> (a, Model blk))
-> Model blk
-> (Resp blk it flr, Model blk)
ok  AnchoredFragment (Header blk) -> Success blk Int Int
forall blk it flr.
AnchoredFragment (Header blk) -> Success blk it flr
Chain               ((DBModel blk -> (AnchoredFragment (Header blk), DBModel blk))
 -> DBModel blk -> (Resp blk Int Int, DBModel blk))
-> (DBModel blk -> (AnchoredFragment (Header blk), DBModel blk))
-> DBModel blk
-> (Resp blk Int Int, DBModel blk)
forall a b. (a -> b) -> a -> b
$ (DBModel blk -> AnchoredFragment (Header blk))
-> DBModel blk -> (AnchoredFragment (Header blk), DBModel blk)
forall {b} {a}. (b -> a) -> b -> (a, b)
query   (SecurityParam
-> (blk -> Header blk)
-> DBModel blk
-> AnchoredFragment (Header blk)
forall a blk.
(HasHeader a, HasHeader blk) =>
SecurityParam -> (blk -> a) -> Model blk -> AnchoredFragment a
Model.volatileChain SecurityParam
k blk -> Header blk
forall blk. GetHeader blk => blk -> Header blk
getHeader)
    Cmd blk Int Int
GetLedgerDB              -> (LedgerDB (ExtLedgerState blk) -> Success blk Int Int)
-> (DBModel blk -> (LedgerDB (ExtLedgerState blk), DBModel blk))
-> DBModel blk
-> (Resp blk Int Int, DBModel blk)
forall {a} {blk} {it} {flr} {blk}.
(a -> Success blk it flr)
-> (Model blk -> (a, Model blk))
-> Model blk
-> (Resp blk it flr, Model blk)
ok  LedgerDB (ExtLedgerState blk) -> Success blk Int Int
forall blk it flr.
LedgerDB (ExtLedgerState blk) -> Success blk it flr
LedgerDB            ((DBModel blk -> (LedgerDB (ExtLedgerState blk), DBModel blk))
 -> DBModel blk -> (Resp blk Int Int, DBModel blk))
-> (DBModel blk -> (LedgerDB (ExtLedgerState blk), DBModel blk))
-> DBModel blk
-> (Resp blk Int Int, DBModel blk)
forall a b. (a -> b) -> a -> b
$ (DBModel blk -> LedgerDB (ExtLedgerState blk))
-> DBModel blk -> (LedgerDB (ExtLedgerState blk), DBModel blk)
forall {b} {a}. (b -> a) -> b -> (a, b)
query   (TopLevelConfig blk -> DBModel blk -> LedgerDB (ExtLedgerState blk)
forall blk.
LedgerSupportsProtocol blk =>
TopLevelConfig blk -> Model blk -> LedgerDB (ExtLedgerState blk)
Model.getLedgerDB TopLevelConfig blk
cfg)
    Cmd blk Int Int
GetTipBlock              -> (Maybe blk -> Success blk Int Int)
-> (DBModel blk -> (Maybe blk, DBModel blk))
-> DBModel blk
-> (Resp blk Int Int, DBModel blk)
forall {a} {blk} {it} {flr} {blk}.
(a -> Success blk it flr)
-> (Model blk -> (a, Model blk))
-> Model blk
-> (Resp blk it flr, Model blk)
ok  Maybe blk -> Success blk Int Int
forall blk it flr. Maybe blk -> Success blk it flr
MbBlock             ((DBModel blk -> (Maybe blk, DBModel blk))
 -> DBModel blk -> (Resp blk Int Int, DBModel blk))
-> (DBModel blk -> (Maybe blk, DBModel blk))
-> DBModel blk
-> (Resp blk Int Int, DBModel blk)
forall a b. (a -> b) -> a -> b
$ (DBModel blk -> Maybe blk)
-> DBModel blk -> (Maybe blk, DBModel blk)
forall {b} {a}. (b -> a) -> b -> (a, b)
query    DBModel blk -> Maybe blk
forall blk. Model blk -> Maybe blk
Model.tipBlock
    Cmd blk Int Int
GetTipHeader             -> (Maybe (Header blk) -> Success blk Int Int)
-> (DBModel blk -> (Maybe (Header blk), DBModel blk))
-> DBModel blk
-> (Resp blk Int Int, DBModel blk)
forall {a} {blk} {it} {flr} {blk}.
(a -> Success blk it flr)
-> (Model blk -> (a, Model blk))
-> Model blk
-> (Resp blk it flr, Model blk)
ok  Maybe (Header blk) -> Success blk Int Int
forall blk it flr. Maybe (Header blk) -> Success blk it flr
MbHeader            ((DBModel blk -> (Maybe (Header blk), DBModel blk))
 -> DBModel blk -> (Resp blk Int Int, DBModel blk))
-> (DBModel blk -> (Maybe (Header blk), DBModel blk))
-> DBModel blk
-> (Resp blk Int Int, DBModel blk)
forall a b. (a -> b) -> a -> b
$ (DBModel blk -> Maybe (Header blk))
-> DBModel blk -> (Maybe (Header blk), DBModel blk)
forall {b} {a}. (b -> a) -> b -> (a, b)
query   ((blk -> Header blk) -> Maybe blk -> Maybe (Header blk)
forall a b. (a -> b) -> Maybe a -> Maybe b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap blk -> Header blk
forall blk. GetHeader blk => blk -> Header blk
getHeader (Maybe blk -> Maybe (Header blk))
-> (DBModel blk -> Maybe blk) -> DBModel blk -> Maybe (Header blk)
forall b c a. (b -> c) -> (a -> b) -> a -> c
. DBModel blk -> Maybe blk
forall blk. Model blk -> Maybe blk
Model.tipBlock)
    Cmd blk Int Int
GetTipPoint              -> (Point blk -> Success blk Int Int)
-> (DBModel blk -> (Point blk, DBModel blk))
-> DBModel blk
-> (Resp blk Int Int, DBModel blk)
forall {a} {blk} {it} {flr} {blk}.
(a -> Success blk it flr)
-> (Model blk -> (a, Model blk))
-> Model blk
-> (Resp blk it flr, Model blk)
ok  Point blk -> Success blk Int Int
forall blk it flr. Point blk -> Success blk it flr
Point               ((DBModel blk -> (Point blk, DBModel blk))
 -> DBModel blk -> (Resp blk Int Int, DBModel blk))
-> (DBModel blk -> (Point blk, DBModel blk))
-> DBModel blk
-> (Resp blk Int Int, DBModel blk)
forall a b. (a -> b) -> a -> b
$ (DBModel blk -> Point blk)
-> DBModel blk -> (Point blk, DBModel blk)
forall {b} {a}. (b -> a) -> b -> (a, b)
query    DBModel blk -> Point blk
forall blk. HasHeader blk => Model blk -> Point blk
Model.tipPoint
    GetBlockComponent RealPoint blk
pt     -> (Maybe
   (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
    SlotNo, IsEBB, SizeInBytes, Word16,
    SomeSecond (NestedCtxt Header) blk)
 -> Success blk Int Int)
-> (DBModel blk
    -> (Either
          (ChainDbError blk)
          (Maybe
             (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
              SlotNo, IsEBB, SizeInBytes, Word16,
              SomeSecond (NestedCtxt Header) blk)),
        DBModel blk))
-> DBModel blk
-> (Resp blk Int Int, DBModel blk)
forall {a} {blk} {it} {flr} {blk}.
(a -> Success blk it flr)
-> (Model blk -> (Either (ChainDbError blk) a, Model blk))
-> Model blk
-> (Resp blk it flr, Model blk)
err Maybe
  (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
   SlotNo, IsEBB, SizeInBytes, Word16,
   SomeSecond (NestedCtxt Header) blk)
-> Success blk Int Int
forall blk it flr. Maybe (AllComponents blk) -> Success blk it flr
MbAllComponents     ((DBModel blk
  -> (Either
        (ChainDbError blk)
        (Maybe
           (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
            SlotNo, IsEBB, SizeInBytes, Word16,
            SomeSecond (NestedCtxt Header) blk)),
      DBModel blk))
 -> DBModel blk -> (Resp blk Int Int, DBModel blk))
-> (DBModel blk
    -> (Either
          (ChainDbError blk)
          (Maybe
             (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
              SlotNo, IsEBB, SizeInBytes, Word16,
              SomeSecond (NestedCtxt Header) blk)),
        DBModel blk))
-> DBModel blk
-> (Resp blk Int Int, DBModel blk)
forall a b. (a -> b) -> a -> b
$ (DBModel blk
 -> Either
      (ChainDbError blk)
      (Maybe
         (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
          SlotNo, IsEBB, SizeInBytes, Word16,
          SomeSecond (NestedCtxt Header) blk)))
-> DBModel blk
-> (Either
      (ChainDbError blk)
      (Maybe
         (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
          SlotNo, IsEBB, SizeInBytes, Word16,
          SomeSecond (NestedCtxt Header) blk)),
    DBModel blk)
forall {b} {a}. (b -> a) -> b -> (a, b)
query   (BlockComponent
  blk
  (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
   SlotNo, IsEBB, SizeInBytes, Word16,
   SomeSecond (NestedCtxt Header) blk)
-> RealPoint blk
-> DBModel blk
-> Either
     (ChainDbError blk)
     (Maybe
        (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
         SlotNo, IsEBB, SizeInBytes, Word16,
         SomeSecond (NestedCtxt Header) blk))
forall blk b.
ModelSupportsBlock blk =>
BlockComponent blk b
-> RealPoint blk
-> Model blk
-> Either (ChainDbError blk) (Maybe b)
Model.getBlockComponentByPoint BlockComponent
  blk
  (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
   SlotNo, IsEBB, SizeInBytes, Word16,
   SomeSecond (NestedCtxt Header) blk)
forall blk. BlockComponent blk (AllComponents blk)
allComponents RealPoint blk
pt)
    GetGCedBlockComponent RealPoint blk
pt -> (Maybe
   (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
    SlotNo, IsEBB, SizeInBytes, Word16,
    SomeSecond (NestedCtxt Header) blk)
 -> Success blk Int Int)
-> (DBModel blk
    -> (Either
          (ChainDbError blk)
          (Maybe
             (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
              SlotNo, IsEBB, SizeInBytes, Word16,
              SomeSecond (NestedCtxt Header) blk)),
        DBModel blk))
-> DBModel blk
-> (Resp blk Int Int, DBModel blk)
forall {a} {blk} {it} {flr} {blk}.
(a -> Success blk it flr)
-> (Model blk -> (Either (ChainDbError blk) a, Model blk))
-> Model blk
-> (Resp blk it flr, Model blk)
err Maybe
  (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
   SlotNo, IsEBB, SizeInBytes, Word16,
   SomeSecond (NestedCtxt Header) blk)
-> Success blk Int Int
forall blk it flr. Maybe (AllComponents blk) -> Success blk it flr
mbGCedAllComponents ((DBModel blk
  -> (Either
        (ChainDbError blk)
        (Maybe
           (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
            SlotNo, IsEBB, SizeInBytes, Word16,
            SomeSecond (NestedCtxt Header) blk)),
      DBModel blk))
 -> DBModel blk -> (Resp blk Int Int, DBModel blk))
-> (DBModel blk
    -> (Either
          (ChainDbError blk)
          (Maybe
             (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
              SlotNo, IsEBB, SizeInBytes, Word16,
              SomeSecond (NestedCtxt Header) blk)),
        DBModel blk))
-> DBModel blk
-> (Resp blk Int Int, DBModel blk)
forall a b. (a -> b) -> a -> b
$ (DBModel blk
 -> Either
      (ChainDbError blk)
      (Maybe
         (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
          SlotNo, IsEBB, SizeInBytes, Word16,
          SomeSecond (NestedCtxt Header) blk)))
-> DBModel blk
-> (Either
      (ChainDbError blk)
      (Maybe
         (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
          SlotNo, IsEBB, SizeInBytes, Word16,
          SomeSecond (NestedCtxt Header) blk)),
    DBModel blk)
forall {b} {a}. (b -> a) -> b -> (a, b)
query   (BlockComponent
  blk
  (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
   SlotNo, IsEBB, SizeInBytes, Word16,
   SomeSecond (NestedCtxt Header) blk)
-> RealPoint blk
-> DBModel blk
-> Either
     (ChainDbError blk)
     (Maybe
        (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
         SlotNo, IsEBB, SizeInBytes, Word16,
         SomeSecond (NestedCtxt Header) blk))
forall blk b.
ModelSupportsBlock blk =>
BlockComponent blk b
-> RealPoint blk
-> Model blk
-> Either (ChainDbError blk) (Maybe b)
Model.getBlockComponentByPoint BlockComponent
  blk
  (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
   SlotNo, IsEBB, SizeInBytes, Word16,
   SomeSecond (NestedCtxt Header) blk)
forall blk. BlockComponent blk (AllComponents blk)
allComponents RealPoint blk
pt)
    Cmd blk Int Int
GetMaxSlotNo             -> (MaxSlotNo -> Success blk Int Int)
-> (DBModel blk -> (MaxSlotNo, DBModel blk))
-> DBModel blk
-> (Resp blk Int Int, DBModel blk)
forall {a} {blk} {it} {flr} {blk}.
(a -> Success blk it flr)
-> (Model blk -> (a, Model blk))
-> Model blk
-> (Resp blk it flr, Model blk)
ok  MaxSlotNo -> Success blk Int Int
forall blk it flr. MaxSlotNo -> Success blk it flr
MaxSlot             ((DBModel blk -> (MaxSlotNo, DBModel blk))
 -> DBModel blk -> (Resp blk Int Int, DBModel blk))
-> (DBModel blk -> (MaxSlotNo, DBModel blk))
-> DBModel blk
-> (Resp blk Int Int, DBModel blk)
forall a b. (a -> b) -> a -> b
$ (DBModel blk -> MaxSlotNo)
-> DBModel blk -> (MaxSlotNo, DBModel blk)
forall {b} {a}. (b -> a) -> b -> (a, b)
query    DBModel blk -> MaxSlotNo
forall blk. HasHeader blk => Model blk -> MaxSlotNo
Model.getMaxSlotNo
    GetIsValid RealPoint blk
pt            -> (Maybe Bool -> Success blk Int Int)
-> (DBModel blk -> (Maybe Bool, DBModel blk))
-> DBModel blk
-> (Resp blk Int Int, DBModel blk)
forall {a} {blk} {it} {flr} {blk}.
(a -> Success blk it flr)
-> (Model blk -> (a, Model blk))
-> Model blk
-> (Resp blk it flr, Model blk)
ok  Maybe Bool -> Success blk Int Int
forall {blk} {it} {flr}. Maybe Bool -> Success blk it flr
isValidResult       ((DBModel blk -> (Maybe Bool, DBModel blk))
 -> DBModel blk -> (Resp blk Int Int, DBModel blk))
-> (DBModel blk -> (Maybe Bool, DBModel blk))
-> DBModel blk
-> (Resp blk Int Int, DBModel blk)
forall a b. (a -> b) -> a -> b
$ (DBModel blk -> Maybe Bool)
-> DBModel blk -> (Maybe Bool, DBModel blk)
forall {b} {a}. (b -> a) -> b -> (a, b)
query   (RealPoint blk -> DBModel blk -> Maybe Bool
forall blk.
LedgerSupportsProtocol blk =>
RealPoint blk -> Model blk -> Maybe Bool
Model.isValid RealPoint blk
pt)
    UpdateLoE AnchoredFragment blk
frag           -> (Point blk -> Success blk Int Int)
-> (DBModel blk -> (Point blk, DBModel blk))
-> DBModel blk
-> (Resp blk Int Int, DBModel blk)
forall {a} {blk} {it} {flr} {blk}.
(a -> Success blk it flr)
-> (Model blk -> (a, Model blk))
-> Model blk
-> (Resp blk it flr, Model blk)
ok  Point blk -> Success blk Int Int
forall blk it flr. Point blk -> Success blk it flr
Point               ((DBModel blk -> (Point blk, DBModel blk))
 -> DBModel blk -> (Resp blk Int Int, DBModel blk))
-> (DBModel blk -> (Point blk, DBModel blk))
-> DBModel blk
-> (Resp blk Int Int, DBModel blk)
forall a b. (a -> b) -> a -> b
$ (DBModel blk -> (Point blk, DBModel blk))
-> DBModel blk -> (Point blk, DBModel blk)
forall {t} {t}. (t -> t) -> t -> t
update  (TopLevelConfig blk
-> AnchoredFragment blk -> DBModel blk -> (Point blk, DBModel blk)
forall blk.
LedgerSupportsProtocol blk =>
TopLevelConfig blk
-> AnchoredFragment blk -> Model blk -> (Point blk, Model blk)
Model.updateLoE TopLevelConfig blk
cfg AnchoredFragment blk
frag)
    Stream StreamFrom blk
from StreamTo blk
to           -> (Either (UnknownRange blk) Int -> Success blk Int Int)
-> (DBModel blk
    -> (Either (ChainDbError blk) (Either (UnknownRange blk) Int),
        DBModel blk))
-> DBModel blk
-> (Resp blk Int Int, DBModel blk)
forall {a} {blk} {it} {flr} {blk}.
(a -> Success blk it flr)
-> (Model blk -> (Either (ChainDbError blk) a, Model blk))
-> Model blk
-> (Resp blk it flr, Model blk)
err Either (UnknownRange blk) Int -> Success blk Int Int
forall {blk} {it} {flr}.
Either (UnknownRange blk) it -> Success blk it flr
iter                ((DBModel blk
  -> (Either (ChainDbError blk) (Either (UnknownRange blk) Int),
      DBModel blk))
 -> DBModel blk -> (Resp blk Int Int, DBModel blk))
-> (DBModel blk
    -> (Either (ChainDbError blk) (Either (UnknownRange blk) Int),
        DBModel blk))
-> DBModel blk
-> (Resp blk Int Int, DBModel blk)
forall a b. (a -> b) -> a -> b
$ (DBModel blk
 -> Either
      (ChainDbError blk) (Either (UnknownRange blk) Int, DBModel blk))
-> DBModel blk
-> (Either (ChainDbError blk) (Either (UnknownRange blk) Int),
    DBModel blk)
forall {b} {a} {b}. (b -> Either a (b, b)) -> b -> (Either a b, b)
updateE (SecurityParam
-> StreamFrom blk
-> StreamTo blk
-> DBModel blk
-> Either
     (ChainDbError blk) (Either (UnknownRange blk) Int, DBModel blk)
forall blk.
GetPrevHash blk =>
SecurityParam
-> StreamFrom blk
-> StreamTo blk
-> Model blk
-> Either
     (ChainDbError blk) (Either (UnknownRange blk) Int, Model blk)
Model.stream SecurityParam
k StreamFrom blk
from StreamTo blk
to)
    IteratorNext  Int
it         -> (IteratorResult
   blk
   (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
    SlotNo, IsEBB, SizeInBytes, Word16,
    SomeSecond (NestedCtxt Header) blk)
 -> Success blk Int Int)
-> (DBModel blk
    -> (IteratorResult
          blk
          (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
           SlotNo, IsEBB, SizeInBytes, Word16,
           SomeSecond (NestedCtxt Header) blk),
        DBModel blk))
-> DBModel blk
-> (Resp blk Int Int, DBModel blk)
forall {a} {blk} {it} {flr} {blk}.
(a -> Success blk it flr)
-> (Model blk -> (a, Model blk))
-> Model blk
-> (Resp blk it flr, Model blk)
ok  IteratorResult
  blk
  (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
   SlotNo, IsEBB, SizeInBytes, Word16,
   SomeSecond (NestedCtxt Header) blk)
-> Success blk Int Int
forall blk it flr.
IteratorResult blk (AllComponents blk) -> Success blk it flr
IterResult          ((DBModel blk
  -> (IteratorResult
        blk
        (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
         SlotNo, IsEBB, SizeInBytes, Word16,
         SomeSecond (NestedCtxt Header) blk),
      DBModel blk))
 -> DBModel blk -> (Resp blk Int Int, DBModel blk))
-> (DBModel blk
    -> (IteratorResult
          blk
          (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
           SlotNo, IsEBB, SizeInBytes, Word16,
           SomeSecond (NestedCtxt Header) blk),
        DBModel blk))
-> DBModel blk
-> (Resp blk Int Int, DBModel blk)
forall a b. (a -> b) -> a -> b
$ (DBModel blk
 -> (IteratorResult
       blk
       (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
        SlotNo, IsEBB, SizeInBytes, Word16,
        SomeSecond (NestedCtxt Header) blk),
     DBModel blk))
-> DBModel blk
-> (IteratorResult
      blk
      (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
       SlotNo, IsEBB, SizeInBytes, Word16,
       SomeSecond (NestedCtxt Header) blk),
    DBModel blk)
forall {t} {t}. (t -> t) -> t -> t
update  (Int
-> BlockComponent
     blk
     (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
      SlotNo, IsEBB, SizeInBytes, Word16,
      SomeSecond (NestedCtxt Header) blk)
-> DBModel blk
-> (IteratorResult
      blk
      (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
       SlotNo, IsEBB, SizeInBytes, Word16,
       SomeSecond (NestedCtxt Header) blk),
    DBModel blk)
forall blk b.
ModelSupportsBlock blk =>
Int
-> BlockComponent blk b
-> Model blk
-> (IteratorResult blk b, Model blk)
Model.iteratorNext Int
it BlockComponent
  blk
  (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
   SlotNo, IsEBB, SizeInBytes, Word16,
   SomeSecond (NestedCtxt Header) blk)
forall blk. BlockComponent blk (AllComponents blk)
allComponents)
    IteratorNextGCed Int
it      -> (IteratorResult
   blk
   (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
    SlotNo, IsEBB, SizeInBytes, Word16,
    SomeSecond (NestedCtxt Header) blk)
 -> Success blk Int Int)
-> (DBModel blk
    -> (IteratorResult
          blk
          (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
           SlotNo, IsEBB, SizeInBytes, Word16,
           SomeSecond (NestedCtxt Header) blk),
        DBModel blk))
-> DBModel blk
-> (Resp blk Int Int, DBModel blk)
forall {a} {blk} {it} {flr} {blk}.
(a -> Success blk it flr)
-> (Model blk -> (a, Model blk))
-> Model blk
-> (Resp blk it flr, Model blk)
ok  IteratorResult
  blk
  (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
   SlotNo, IsEBB, SizeInBytes, Word16,
   SomeSecond (NestedCtxt Header) blk)
-> Success blk Int Int
forall blk it flr.
IteratorResult blk (AllComponents blk) -> Success blk it flr
iterResultGCed      ((DBModel blk
  -> (IteratorResult
        blk
        (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
         SlotNo, IsEBB, SizeInBytes, Word16,
         SomeSecond (NestedCtxt Header) blk),
      DBModel blk))
 -> DBModel blk -> (Resp blk Int Int, DBModel blk))
-> (DBModel blk
    -> (IteratorResult
          blk
          (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
           SlotNo, IsEBB, SizeInBytes, Word16,
           SomeSecond (NestedCtxt Header) blk),
        DBModel blk))
-> DBModel blk
-> (Resp blk Int Int, DBModel blk)
forall a b. (a -> b) -> a -> b
$ (DBModel blk
 -> (IteratorResult
       blk
       (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
        SlotNo, IsEBB, SizeInBytes, Word16,
        SomeSecond (NestedCtxt Header) blk),
     DBModel blk))
-> DBModel blk
-> (IteratorResult
      blk
      (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
       SlotNo, IsEBB, SizeInBytes, Word16,
       SomeSecond (NestedCtxt Header) blk),
    DBModel blk)
forall {t} {t}. (t -> t) -> t -> t
update  (Int
-> BlockComponent
     blk
     (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
      SlotNo, IsEBB, SizeInBytes, Word16,
      SomeSecond (NestedCtxt Header) blk)
-> DBModel blk
-> (IteratorResult
      blk
      (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
       SlotNo, IsEBB, SizeInBytes, Word16,
       SomeSecond (NestedCtxt Header) blk),
    DBModel blk)
forall blk b.
ModelSupportsBlock blk =>
Int
-> BlockComponent blk b
-> Model blk
-> (IteratorResult blk b, Model blk)
Model.iteratorNext Int
it BlockComponent
  blk
  (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
   SlotNo, IsEBB, SizeInBytes, Word16,
   SomeSecond (NestedCtxt Header) blk)
forall blk. BlockComponent blk (AllComponents blk)
allComponents)
    IteratorClose Int
it         -> (() -> Success blk Int Int)
-> (DBModel blk -> ((), DBModel blk))
-> DBModel blk
-> (Resp blk Int Int, DBModel blk)
forall {a} {blk} {it} {flr} {blk}.
(a -> Success blk it flr)
-> (Model blk -> (a, Model blk))
-> Model blk
-> (Resp blk it flr, Model blk)
ok  () -> Success blk Int Int
forall blk it flr. () -> Success blk it flr
Unit                ((DBModel blk -> ((), DBModel blk))
 -> DBModel blk -> (Resp blk Int Int, DBModel blk))
-> (DBModel blk -> ((), DBModel blk))
-> DBModel blk
-> (Resp blk Int Int, DBModel blk)
forall a b. (a -> b) -> a -> b
$ (DBModel blk -> DBModel blk) -> DBModel blk -> ((), DBModel blk)
forall {t} {b}. (t -> b) -> t -> ((), b)
update_ (Int -> DBModel blk -> DBModel blk
forall blk. Int -> Model blk -> Model blk
Model.iteratorClose Int
it)
    -- As tentative followers differ from normal followers only during chain
    -- selection, this test can not distinguish between them due to its
    -- sequential nature. Hence, we don't add a pure model for tentative
    -- followers.
    NewFollower ChainType
_            -> (Int -> Success blk Int Int)
-> (DBModel blk -> (Int, DBModel blk))
-> DBModel blk
-> (Resp blk Int Int, DBModel blk)
forall {a} {blk} {it} {flr} {blk}.
(a -> Success blk it flr)
-> (Model blk -> (a, Model blk))
-> Model blk
-> (Resp blk it flr, Model blk)
ok  Int -> Success blk Int Int
forall blk it flr. flr -> Success blk it flr
Flr                 ((DBModel blk -> (Int, DBModel blk))
 -> DBModel blk -> (Resp blk Int Int, DBModel blk))
-> (DBModel blk -> (Int, DBModel blk))
-> DBModel blk
-> (Resp blk Int Int, DBModel blk)
forall a b. (a -> b) -> a -> b
$ (DBModel blk -> (Int, DBModel blk))
-> DBModel blk -> (Int, DBModel blk)
forall {t} {t}. (t -> t) -> t -> t
update   DBModel blk -> (Int, DBModel blk)
forall blk. HasHeader blk => Model blk -> (Int, Model blk)
Model.newFollower
    FollowerInstruction Int
flr  -> (Maybe
   (ChainUpdate
      blk
      (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
       SlotNo, IsEBB, SizeInBytes, Word16,
       SomeSecond (NestedCtxt Header) blk))
 -> Success blk Int Int)
-> (DBModel blk
    -> (Either
          (ChainDbError blk)
          (Maybe
             (ChainUpdate
                blk
                (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
                 SlotNo, IsEBB, SizeInBytes, Word16,
                 SomeSecond (NestedCtxt Header) blk))),
        DBModel blk))
-> DBModel blk
-> (Resp blk Int Int, DBModel blk)
forall {a} {blk} {it} {flr} {blk}.
(a -> Success blk it flr)
-> (Model blk -> (Either (ChainDbError blk) a, Model blk))
-> Model blk
-> (Resp blk it flr, Model blk)
err Maybe
  (ChainUpdate
     blk
     (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
      SlotNo, IsEBB, SizeInBytes, Word16,
      SomeSecond (NestedCtxt Header) blk))
-> Success blk Int Int
forall blk it flr.
Maybe (ChainUpdate blk (AllComponents blk)) -> Success blk it flr
MbChainUpdate       ((DBModel blk
  -> (Either
        (ChainDbError blk)
        (Maybe
           (ChainUpdate
              blk
              (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
               SlotNo, IsEBB, SizeInBytes, Word16,
               SomeSecond (NestedCtxt Header) blk))),
      DBModel blk))
 -> DBModel blk -> (Resp blk Int Int, DBModel blk))
-> (DBModel blk
    -> (Either
          (ChainDbError blk)
          (Maybe
             (ChainUpdate
                blk
                (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
                 SlotNo, IsEBB, SizeInBytes, Word16,
                 SomeSecond (NestedCtxt Header) blk))),
        DBModel blk))
-> DBModel blk
-> (Resp blk Int Int, DBModel blk)
forall a b. (a -> b) -> a -> b
$ (DBModel blk
 -> Either
      (ChainDbError blk)
      (Maybe
         (ChainUpdate
            blk
            (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
             SlotNo, IsEBB, SizeInBytes, Word16,
             SomeSecond (NestedCtxt Header) blk)),
       DBModel blk))
-> DBModel blk
-> (Either
      (ChainDbError blk)
      (Maybe
         (ChainUpdate
            blk
            (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
             SlotNo, IsEBB, SizeInBytes, Word16,
             SomeSecond (NestedCtxt Header) blk))),
    DBModel blk)
forall {b} {a} {b}. (b -> Either a (b, b)) -> b -> (Either a b, b)
updateE (Int
-> BlockComponent
     blk
     (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
      SlotNo, IsEBB, SizeInBytes, Word16,
      SomeSecond (NestedCtxt Header) blk)
-> DBModel blk
-> Either
     (ChainDbError blk)
     (Maybe
        (ChainUpdate
           blk
           (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
            SlotNo, IsEBB, SizeInBytes, Word16,
            SomeSecond (NestedCtxt Header) blk)),
      DBModel blk)
forall blk b.
ModelSupportsBlock blk =>
Int
-> BlockComponent blk b
-> Model blk
-> Either (ChainDbError blk) (Maybe (ChainUpdate blk b), Model blk)
Model.followerInstruction Int
flr BlockComponent
  blk
  (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
   SlotNo, IsEBB, SizeInBytes, Word16,
   SomeSecond (NestedCtxt Header) blk)
forall blk. BlockComponent blk (AllComponents blk)
allComponents)
    FollowerForward Int
flr [Point blk]
pts  -> (Maybe (Point blk) -> Success blk Int Int)
-> (DBModel blk
    -> (Either (ChainDbError blk) (Maybe (Point blk)), DBModel blk))
-> DBModel blk
-> (Resp blk Int Int, DBModel blk)
forall {a} {blk} {it} {flr} {blk}.
(a -> Success blk it flr)
-> (Model blk -> (Either (ChainDbError blk) a, Model blk))
-> Model blk
-> (Resp blk it flr, Model blk)
err Maybe (Point blk) -> Success blk Int Int
forall blk it flr. Maybe (Point blk) -> Success blk it flr
MbPoint             ((DBModel blk
  -> (Either (ChainDbError blk) (Maybe (Point blk)), DBModel blk))
 -> DBModel blk -> (Resp blk Int Int, DBModel blk))
-> (DBModel blk
    -> (Either (ChainDbError blk) (Maybe (Point blk)), DBModel blk))
-> DBModel blk
-> (Resp blk Int Int, DBModel blk)
forall a b. (a -> b) -> a -> b
$ (DBModel blk
 -> Either (ChainDbError blk) (Maybe (Point blk), DBModel blk))
-> DBModel blk
-> (Either (ChainDbError blk) (Maybe (Point blk)), DBModel blk)
forall {b} {a} {b}. (b -> Either a (b, b)) -> b -> (Either a b, b)
updateE (Int
-> [Point blk]
-> DBModel blk
-> Either (ChainDbError blk) (Maybe (Point blk), DBModel blk)
forall blk.
HasHeader blk =>
Int
-> [Point blk]
-> Model blk
-> Either (ChainDbError blk) (Maybe (Point blk), Model blk)
Model.followerForward Int
flr [Point blk]
pts)
    FollowerClose Int
flr        -> (() -> Success blk Int Int)
-> (DBModel blk -> ((), DBModel blk))
-> DBModel blk
-> (Resp blk Int Int, DBModel blk)
forall {a} {blk} {it} {flr} {blk}.
(a -> Success blk it flr)
-> (Model blk -> (a, Model blk))
-> Model blk
-> (Resp blk it flr, Model blk)
ok  () -> Success blk Int Int
forall blk it flr. () -> Success blk it flr
Unit                ((DBModel blk -> ((), DBModel blk))
 -> DBModel blk -> (Resp blk Int Int, DBModel blk))
-> (DBModel blk -> ((), DBModel blk))
-> DBModel blk
-> (Resp blk Int Int, DBModel blk)
forall a b. (a -> b) -> a -> b
$ (DBModel blk -> DBModel blk) -> DBModel blk -> ((), DBModel blk)
forall {t} {b}. (t -> b) -> t -> ((), b)
update_ (Int -> DBModel blk -> DBModel blk
forall blk. Int -> Model blk -> Model blk
Model.followerClose Int
flr)
    Cmd blk Int Int
PersistBlks              -> (() -> Success blk Int Int)
-> (DBModel blk -> ((), DBModel blk))
-> DBModel blk
-> (Resp blk Int Int, DBModel blk)
forall {a} {blk} {it} {flr} {blk}.
(a -> Success blk it flr)
-> (Model blk -> (a, Model blk))
-> Model blk
-> (Resp blk it flr, Model blk)
ok  () -> Success blk Int Int
forall blk it flr. () -> Success blk it flr
Unit                ((DBModel blk -> ((), DBModel blk))
 -> DBModel blk -> (Resp blk Int Int, DBModel blk))
-> (DBModel blk -> ((), DBModel blk))
-> DBModel blk
-> (Resp blk Int Int, DBModel blk)
forall a b. (a -> b) -> a -> b
$ (DBModel blk -> DBModel blk) -> DBModel blk -> ((), DBModel blk)
forall {t} {b}. (t -> b) -> t -> ((), b)
update_ (SecurityParam -> ShouldGarbageCollect -> DBModel blk -> DBModel blk
forall blk.
HasHeader blk =>
SecurityParam -> ShouldGarbageCollect -> Model blk -> Model blk
Model.copyToImmutableDB SecurityParam
k ShouldGarbageCollect
DoNotGarbageCollect)
    Cmd blk Int Int
PersistBlksThenGC        -> (() -> Success blk Int Int)
-> (DBModel blk -> ((), DBModel blk))
-> DBModel blk
-> (Resp blk Int Int, DBModel blk)
forall {a} {blk} {it} {flr} {blk}.
(a -> Success blk it flr)
-> (Model blk -> (a, Model blk))
-> Model blk
-> (Resp blk it flr, Model blk)
ok  () -> Success blk Int Int
forall blk it flr. () -> Success blk it flr
Unit                ((DBModel blk -> ((), DBModel blk))
 -> DBModel blk -> (Resp blk Int Int, DBModel blk))
-> (DBModel blk -> ((), DBModel blk))
-> DBModel blk
-> (Resp blk Int Int, DBModel blk)
forall a b. (a -> b) -> a -> b
$ (DBModel blk -> DBModel blk) -> DBModel blk -> ((), DBModel blk)
forall {t} {b}. (t -> b) -> t -> ((), b)
update_ (SecurityParam -> ShouldGarbageCollect -> DBModel blk -> DBModel blk
forall blk.
HasHeader blk =>
SecurityParam -> ShouldGarbageCollect -> Model blk -> Model blk
Model.copyToImmutableDB SecurityParam
k ShouldGarbageCollect
GarbageCollect)
    -- TODO: The model does not capture the notion of ledger snapshots,
    -- therefore we ignore this command here. This introduces an assymetry in
    -- the way the 'UpdateLedgerSnapshots' command is handled in the model and
    -- in the system under test. It would be better if we modelled the
    -- snapshots so that this aspect of the system would be explicitly
    -- specified. See https://github.com/IntersectMBO/ouroboros-network/issues/3375
    --
    Cmd blk Int Int
UpdateLedgerSnapshots    -> (() -> Success blk Int Int)
-> (DBModel blk -> ((), DBModel blk))
-> DBModel blk
-> (Resp blk Int Int, DBModel blk)
forall {a} {blk} {it} {flr} {blk}.
(a -> Success blk it flr)
-> (Model blk -> (a, Model blk))
-> Model blk
-> (Resp blk it flr, Model blk)
ok  () -> Success blk Int Int
forall blk it flr. () -> Success blk it flr
Unit                ((DBModel blk -> ((), DBModel blk))
 -> DBModel blk -> (Resp blk Int Int, DBModel blk))
-> (DBModel blk -> ((), DBModel blk))
-> DBModel blk
-> (Resp blk Int Int, DBModel blk)
forall a b. (a -> b) -> a -> b
$ ((), )
    Cmd blk Int Int
Close                    -> (DBModel blk -> ((), DBModel blk))
-> DBModel blk -> (Resp blk Int Int, DBModel blk)
forall {p :: * -> * -> *} {a} {c} {blk} {it} {flr}.
Bifunctor p =>
(a -> p () c) -> a -> p (Resp blk it flr) c
openOrClosed            ((DBModel blk -> ((), DBModel blk))
 -> DBModel blk -> (Resp blk Int Int, DBModel blk))
-> (DBModel blk -> ((), DBModel blk))
-> DBModel blk
-> (Resp blk Int Int, DBModel blk)
forall a b. (a -> b) -> a -> b
$ (DBModel blk -> DBModel blk) -> DBModel blk -> ((), DBModel blk)
forall {t} {b}. (t -> b) -> t -> ((), b)
update_  DBModel blk -> DBModel blk
forall blk. Model blk -> Model blk
Model.closeDB
    Cmd blk Int Int
Reopen                   -> (DBModel blk -> ((), DBModel blk))
-> DBModel blk -> (Resp blk Int Int, DBModel blk)
forall {p :: * -> * -> *} {a} {c} {blk} {it} {flr}.
Bifunctor p =>
(a -> p () c) -> a -> p (Resp blk it flr) c
openOrClosed            ((DBModel blk -> ((), DBModel blk))
 -> DBModel blk -> (Resp blk Int Int, DBModel blk))
-> (DBModel blk -> ((), DBModel blk))
-> DBModel blk
-> (Resp blk Int Int, DBModel blk)
forall a b. (a -> b) -> a -> b
$ (DBModel blk -> DBModel blk) -> DBModel blk -> ((), DBModel blk)
forall {t} {b}. (t -> b) -> t -> ((), b)
update_  DBModel blk -> DBModel blk
forall blk. Model blk -> Model blk
Model.reopen
    Cmd blk Int Int
WipeVolatileDB           -> (Point blk -> Success blk Int Int)
-> (DBModel blk -> (Point blk, DBModel blk))
-> DBModel blk
-> (Resp blk Int Int, DBModel blk)
forall {a} {blk} {it} {flr} {blk}.
(a -> Success blk it flr)
-> (Model blk -> (a, Model blk))
-> Model blk
-> (Resp blk it flr, Model blk)
ok  Point blk -> Success blk Int Int
forall blk it flr. Point blk -> Success blk it flr
Point               ((DBModel blk -> (Point blk, DBModel blk))
 -> DBModel blk -> (Resp blk Int Int, DBModel blk))
-> (DBModel blk -> (Point blk, DBModel blk))
-> DBModel blk
-> (Resp blk Int Int, DBModel blk)
forall a b. (a -> b) -> a -> b
$ (DBModel blk -> (Point blk, DBModel blk))
-> DBModel blk -> (Point blk, DBModel blk)
forall {t} {t}. (t -> t) -> t -> t
update  (TopLevelConfig blk -> DBModel blk -> (Point blk, DBModel blk)
forall blk.
LedgerSupportsProtocol blk =>
TopLevelConfig blk -> Model blk -> (Point blk, Model blk)
Model.wipeVolatileDB TopLevelConfig blk
cfg)
  where
    k :: SecurityParam
k = TopLevelConfig blk -> SecurityParam
forall blk.
ConsensusProtocol (BlockProtocol blk) =>
TopLevelConfig blk -> SecurityParam
configSecurityParam TopLevelConfig blk
cfg

    advanceAndAdd :: SlotNo -> blk -> DBModel blk -> (Point blk, DBModel blk)
advanceAndAdd SlotNo
slot blk
blk DBModel blk
m = (DBModel blk -> Point blk
forall blk. HasHeader blk => Model blk -> Point blk
Model.tipPoint DBModel blk
m', DBModel blk
m')
      where
        m' :: DBModel blk
m' = TopLevelConfig blk -> blk -> DBModel blk -> DBModel blk
forall blk.
LedgerSupportsProtocol blk =>
TopLevelConfig blk -> blk -> Model blk -> Model blk
Model.addBlock TopLevelConfig blk
cfg blk
blk (DBModel blk -> DBModel blk) -> DBModel blk -> DBModel blk
forall a b. (a -> b) -> a -> b
$ SlotNo -> DBModel blk -> DBModel blk
forall blk. SlotNo -> Model blk -> Model blk
Model.advanceCurSlot SlotNo
slot DBModel blk
m

    iter :: Either (UnknownRange blk) it -> Success blk it flr
iter = (UnknownRange blk -> Success blk it flr)
-> (it -> Success blk it flr)
-> Either (UnknownRange blk) it
-> Success blk it flr
forall a c b. (a -> c) -> (b -> c) -> Either a b -> c
either UnknownRange blk -> Success blk it flr
forall blk it flr. UnknownRange blk -> Success blk it flr
UnknownRange it -> Success blk it flr
forall blk it flr. it -> Success blk it flr
Iter
    mbGCedAllComponents :: Maybe
  (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
   SlotNo, IsEBB, SizeInBytes, Word16,
   SomeSecond (NestedCtxt Header) blk)
-> Success blk it flr
mbGCedAllComponents = MaybeGCedBlock
  (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
   SlotNo, IsEBB, SizeInBytes, Word16,
   SomeSecond (NestedCtxt Header) blk)
-> Success blk it flr
forall blk it flr.
MaybeGCedBlock (AllComponents blk) -> Success blk it flr
MbGCedAllComponents (MaybeGCedBlock
   (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
    SlotNo, IsEBB, SizeInBytes, Word16,
    SomeSecond (NestedCtxt Header) blk)
 -> Success blk it flr)
-> (Maybe
      (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
       SlotNo, IsEBB, SizeInBytes, Word16,
       SomeSecond (NestedCtxt Header) blk)
    -> MaybeGCedBlock
         (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
          SlotNo, IsEBB, SizeInBytes, Word16,
          SomeSecond (NestedCtxt Header) blk))
-> Maybe
     (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
      SlotNo, IsEBB, SizeInBytes, Word16,
      SomeSecond (NestedCtxt Header) blk)
-> Success blk it flr
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Bool
-> Maybe
     (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
      SlotNo, IsEBB, SizeInBytes, Word16,
      SomeSecond (NestedCtxt Header) blk)
-> MaybeGCedBlock
     (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
      SlotNo, IsEBB, SizeInBytes, Word16,
      SomeSecond (NestedCtxt Header) blk)
forall blk. Bool -> Maybe blk -> MaybeGCedBlock blk
MaybeGCedBlock Bool
False
    iterResultGCed :: IteratorResult
  blk
  (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
   SlotNo, IsEBB, SizeInBytes, Word16,
   SomeSecond (NestedCtxt Header) blk)
-> Success blk it flr
iterResultGCed = IteratorResultGCed blk -> Success blk it flr
forall blk it flr. IteratorResultGCed blk -> Success blk it flr
IterResultGCed (IteratorResultGCed blk -> Success blk it flr)
-> (IteratorResult
      blk
      (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
       SlotNo, IsEBB, SizeInBytes, Word16,
       SomeSecond (NestedCtxt Header) blk)
    -> IteratorResultGCed blk)
-> IteratorResult
     blk
     (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
      SlotNo, IsEBB, SizeInBytes, Word16,
      SomeSecond (NestedCtxt Header) blk)
-> Success blk it flr
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Bool
-> IteratorResult
     blk
     (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
      SlotNo, IsEBB, SizeInBytes, Word16,
      SomeSecond (NestedCtxt Header) blk)
-> IteratorResultGCed blk
forall blk.
Bool
-> IteratorResult blk (AllComponents blk) -> IteratorResultGCed blk
IteratorResultGCed Bool
False
    isValidResult :: Maybe Bool -> Success blk it flr
isValidResult = IsValidResult -> Success blk it flr
forall blk it flr. IsValidResult -> Success blk it flr
IsValid (IsValidResult -> Success blk it flr)
-> (Maybe Bool -> IsValidResult)
-> Maybe Bool
-> Success blk it flr
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Bool -> Maybe Bool -> IsValidResult
IsValidResult Bool
False

    query :: (b -> a) -> b -> (a, b)
query   b -> a
f b
m = (b -> a
f b
m, b
m)

    update :: (t -> t) -> t -> t
update  t -> t
f t
m = t -> t
f t
m
    update_ :: (t -> b) -> t -> ((), b)
update_ t -> b
f t
m = ((), t -> b
f t
m)
    updateE :: (b -> Either a (b, b)) -> b -> (Either a b, b)
updateE b -> Either a (b, b)
f b
m = case b -> Either a (b, b)
f b
m of
      Left  a
e       -> (a -> Either a b
forall a b. a -> Either a b
Left a
e, b
m)
      Right (b
a, b
m') -> (b -> Either a b
forall a b. b -> Either a b
Right b
a, b
m')

    -- Only executed when the ChainDB is open, otherwise a 'ClosedDBError' is
    -- returned.
    ok :: (a -> Success blk it flr)
-> (Model blk -> (a, Model blk))
-> Model blk
-> (Resp blk it flr, Model blk)
ok a -> Success blk it flr
toSuccess Model blk -> (a, Model blk)
f = (a -> Success blk it flr)
-> (Model blk -> (Either (ChainDbError blk) a, Model blk))
-> Model blk
-> (Resp blk it flr, Model blk)
forall {a} {blk} {it} {flr} {blk}.
(a -> Success blk it flr)
-> (Model blk -> (Either (ChainDbError blk) a, Model blk))
-> Model blk
-> (Resp blk it flr, Model blk)
err a -> Success blk it flr
toSuccess ((a -> Either (ChainDbError blk) a)
-> (a, Model blk) -> (Either (ChainDbError blk) a, Model blk)
forall a b c. (a -> b) -> (a, c) -> (b, c)
forall (p :: * -> * -> *) a b c.
Bifunctor p =>
(a -> b) -> p a c -> p b c
first a -> Either (ChainDbError blk) a
forall a b. b -> Either a b
Right ((a, Model blk) -> (Either (ChainDbError blk) a, Model blk))
-> (Model blk -> (a, Model blk))
-> Model blk
-> (Either (ChainDbError blk) a, Model blk)
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Model blk -> (a, Model blk)
f)
    err :: (a -> Success blk it flr)
-> (Model blk -> (Either (ChainDbError blk) a, Model blk))
-> Model blk
-> (Resp blk it flr, Model blk)
err a -> Success blk it flr
toSuccess Model blk -> (Either (ChainDbError blk) a, Model blk)
f Model blk
m
      | Model blk -> Bool
forall blk. Model blk -> Bool
Model.isOpen Model blk
m
      = (Either (ChainDbError blk) a -> Resp blk it flr)
-> (Either (ChainDbError blk) a, Model blk)
-> (Resp blk it flr, Model blk)
forall a b c. (a -> b) -> (a, c) -> (b, c)
forall (p :: * -> * -> *) a b c.
Bifunctor p =>
(a -> b) -> p a c -> p b c
first (Either (ChainDbError blk) (Success blk it flr) -> Resp blk it flr
forall blk it flr.
Either (ChainDbError blk) (Success blk it flr) -> Resp blk it flr
Resp (Either (ChainDbError blk) (Success blk it flr) -> Resp blk it flr)
-> (Either (ChainDbError blk) a
    -> Either (ChainDbError blk) (Success blk it flr))
-> Either (ChainDbError blk) a
-> Resp blk it flr
forall b c a. (b -> c) -> (a -> b) -> a -> c
. (a -> Success blk it flr)
-> Either (ChainDbError blk) a
-> Either (ChainDbError blk) (Success blk it flr)
forall a b.
(a -> b)
-> Either (ChainDbError blk) a -> Either (ChainDbError blk) b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap a -> Success blk it flr
toSuccess) (Model blk -> (Either (ChainDbError blk) a, Model blk)
f Model blk
m)
      | Bool
otherwise
      = (Either (ChainDbError blk) (Success blk it flr) -> Resp blk it flr
forall blk it flr.
Either (ChainDbError blk) (Success blk it flr) -> Resp blk it flr
Resp (ChainDbError blk -> Either (ChainDbError blk) (Success blk it flr)
forall a b. a -> Either a b
Left (PrettyCallStack -> ChainDbError blk
forall blk. PrettyCallStack -> ChainDbError blk
ClosedDBError PrettyCallStack
HasCallStack => PrettyCallStack
prettyCallStack)), Model blk
m)

    -- Executed whether the ChainDB is open or closed.
    openOrClosed :: (a -> p () c) -> a -> p (Resp blk it flr) c
openOrClosed a -> p () c
f = (() -> Resp blk it flr) -> p () c -> p (Resp blk it flr) c
forall a b c. (a -> b) -> p a c -> p b c
forall (p :: * -> * -> *) a b c.
Bifunctor p =>
(a -> b) -> p a c -> p b c
first (Either (ChainDbError blk) (Success blk it flr) -> Resp blk it flr
forall blk it flr.
Either (ChainDbError blk) (Success blk it flr) -> Resp blk it flr
Resp (Either (ChainDbError blk) (Success blk it flr) -> Resp blk it flr)
-> (() -> Either (ChainDbError blk) (Success blk it flr))
-> ()
-> Resp blk it flr
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Success blk it flr
-> Either (ChainDbError blk) (Success blk it flr)
forall a b. b -> Either a b
Right (Success blk it flr
 -> Either (ChainDbError blk) (Success blk it flr))
-> (() -> Success blk it flr)
-> ()
-> Either (ChainDbError blk) (Success blk it flr)
forall b c a. (b -> c) -> (a -> b) -> a -> c
. () -> Success blk it flr
forall blk it flr. () -> Success blk it flr
Unit) (p () c -> p (Resp blk it flr) c)
-> (a -> p () c) -> a -> p (Resp blk it flr) c
forall b c a. (b -> c) -> (a -> b) -> a -> c
. a -> p () c
f

runIO :: TestConstraints blk
      => ChainDBEnv IO blk
      ->     Cmd  blk (TestIterator IO blk) (TestFollower IO blk)
      -> IO (Resp blk (TestIterator IO blk) (TestFollower IO blk))
runIO :: forall blk.
TestConstraints blk =>
ChainDBEnv IO blk
-> Cmd blk (TestIterator IO blk) (TestFollower IO blk)
-> IO (Resp blk (TestIterator IO blk) (TestFollower IO blk))
runIO ChainDBEnv IO blk
env Cmd blk (TestIterator IO blk) (TestFollower IO blk)
cmd = Either
  (ChainDbError blk)
  (Success blk (TestIterator IO blk) (TestFollower IO blk))
-> Resp blk (TestIterator IO blk) (TestFollower IO blk)
forall blk it flr.
Either (ChainDbError blk) (Success blk it flr) -> Resp blk it flr
Resp (Either
   (ChainDbError blk)
   (Success blk (TestIterator IO blk) (TestFollower IO blk))
 -> Resp blk (TestIterator IO blk) (TestFollower IO blk))
-> IO
     (Either
        (ChainDbError blk)
        (Success blk (TestIterator IO blk) (TestFollower IO blk)))
-> IO (Resp blk (TestIterator IO blk) (TestFollower IO blk))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> IO (Success blk (TestIterator IO blk) (TestFollower IO blk))
-> IO
     (Either
        (ChainDbError blk)
        (Success blk (TestIterator IO blk) (TestFollower IO blk)))
forall e a. Exception e => IO a -> IO (Either e a)
forall (m :: * -> *) e a.
(MonadCatch m, Exception e) =>
m a -> m (Either e a)
try (ChainDBEnv IO blk
-> Cmd blk (TestIterator IO blk) (TestFollower IO blk)
-> IO (Success blk (TestIterator IO blk) (TestFollower IO blk))
forall (m :: * -> *) blk.
(IOLike m, TestConstraints blk) =>
ChainDBEnv m blk
-> Cmd blk (TestIterator m blk) (TestFollower m blk)
-> m (Success blk (TestIterator m blk) (TestFollower m blk))
run ChainDBEnv IO blk
env Cmd blk (TestIterator IO blk) (TestFollower IO blk)
cmd)

{-------------------------------------------------------------------------------
  Collect arguments
-------------------------------------------------------------------------------}

-- | Collect all iterators created.
iters :: Bitraversable t => t it flr -> [it]
iters :: forall (t :: * -> * -> *) it flr.
Bitraversable t =>
t it flr -> [it]
iters = (it -> [it]) -> (flr -> [it]) -> t it flr -> [it]
forall m a b. Monoid m => (a -> m) -> (b -> m) -> t a b -> m
forall (p :: * -> * -> *) m a b.
(Bifoldable p, Monoid m) =>
(a -> m) -> (b -> m) -> p a b -> m
bifoldMap (it -> [it] -> [it]
forall a. a -> [a] -> [a]
:[]) ([it] -> flr -> [it]
forall a b. a -> b -> a
const [])

-- | Collect all followers created.
flrs :: Bitraversable t => t it flr -> [flr]
flrs :: forall (t :: * -> * -> *) it flr.
Bitraversable t =>
t it flr -> [flr]
flrs = (it -> [flr]) -> (flr -> [flr]) -> t it flr -> [flr]
forall m a b. Monoid m => (a -> m) -> (b -> m) -> t a b -> m
forall (p :: * -> * -> *) m a b.
(Bifoldable p, Monoid m) =>
(a -> m) -> (b -> m) -> p a b -> m
bifoldMap ([flr] -> it -> [flr]
forall a b. a -> b -> a
const []) (flr -> [flr] -> [flr]
forall a. a -> [a] -> [a]
:[])

{-------------------------------------------------------------------------------
  Bitraversable instances
-------------------------------------------------------------------------------}

TH.deriveBifunctor     ''Cmd
TH.deriveBifoldable    ''Cmd
TH.deriveBitraversable ''Cmd

TH.deriveBifunctor     ''Success
TH.deriveBifoldable    ''Success
TH.deriveBitraversable ''Success

TH.deriveBifunctor     ''Resp
TH.deriveBifoldable    ''Resp
TH.deriveBitraversable ''Resp

{-------------------------------------------------------------------------------
  Model
-------------------------------------------------------------------------------}

-- | Concrete or symbolic references to a real iterator
type IterRef blk m r = Reference (Opaque (TestIterator m blk)) r

-- | Mapping between iterator references and mocked iterators
type KnownIters blk m r = RefEnv (Opaque (TestIterator m blk)) IteratorId r

-- | Concrete or symbolic references to a real follower
type FollowerRef blk m r = Reference (Opaque (TestFollower m blk)) r

-- | Mapping between iterator references and mocked followers
type KnownFollowers blk m r = RefEnv (Opaque (TestFollower m blk)) FollowerId r

-- | Execution model
data Model blk m r = Model
  { forall blk (m :: * -> *) (r :: * -> *).
Model blk m r -> DBModel blk
dbModel        :: DBModel                blk
  , forall blk (m :: * -> *) (r :: * -> *).
Model blk m r -> KnownIters blk m r
knownIters     :: KnownIters             blk m r
  , forall blk (m :: * -> *) (r :: * -> *).
Model blk m r -> KnownFollowers blk m r
knownFollowers :: KnownFollowers         blk m r
  , forall blk (m :: * -> *) (r :: * -> *).
Model blk m r -> Opaque (TopLevelConfig blk)
modelConfig    :: Opaque (TopLevelConfig blk)
  } deriving ((forall x. Model blk m r -> Rep (Model blk m r) x)
-> (forall x. Rep (Model blk m r) x -> Model blk m r)
-> Generic (Model blk m r)
forall x. Rep (Model blk m r) x -> Model blk m r
forall x. Model blk m r -> Rep (Model blk m r) x
forall a.
(forall x. a -> Rep a x) -> (forall x. Rep a x -> a) -> Generic a
forall blk (m :: * -> *) (r :: * -> *) x.
Rep (Model blk m r) x -> Model blk m r
forall blk (m :: * -> *) (r :: * -> *) x.
Model blk m r -> Rep (Model blk m r) x
$cfrom :: forall blk (m :: * -> *) (r :: * -> *) x.
Model blk m r -> Rep (Model blk m r) x
from :: forall x. Model blk m r -> Rep (Model blk m r) x
$cto :: forall blk (m :: * -> *) (r :: * -> *) x.
Rep (Model blk m r) x -> Model blk m r
to :: forall x. Rep (Model blk m r) x -> Model blk m r
Generic)

deriving instance (TestConstraints blk, Show1 r) => Show (Model blk m r)

-- | Initial model
initModel :: HasHeader blk
          => LoE ()
          -> TopLevelConfig blk
          -> ExtLedgerState blk
          -> MaxClockSkew
          -> Model blk m r
initModel :: forall blk (m :: * -> *) (r :: * -> *).
HasHeader blk =>
LoE ()
-> TopLevelConfig blk
-> ExtLedgerState blk
-> MaxClockSkew
-> Model blk m r
initModel LoE ()
loe TopLevelConfig blk
cfg ExtLedgerState blk
initLedger (MaxClockSkew Word64
maxClockSkew) = Model
  { $sel:dbModel:Model :: DBModel blk
dbModel        = LoE () -> ExtLedgerState blk -> Word64 -> DBModel blk
forall blk.
HasHeader blk =>
LoE () -> ExtLedgerState blk -> Word64 -> Model blk
Model.empty LoE ()
loe ExtLedgerState blk
initLedger Word64
maxClockSkew
  , $sel:knownIters:Model :: KnownIters blk m r
knownIters     = KnownIters blk m r
forall k a (r :: * -> *). RefEnv k a r
RE.empty
  , $sel:knownFollowers:Model :: KnownFollowers blk m r
knownFollowers = KnownFollowers blk m r
forall k a (r :: * -> *). RefEnv k a r
RE.empty
  , $sel:modelConfig:Model :: Opaque (TopLevelConfig blk)
modelConfig    = TopLevelConfig blk -> Opaque (TopLevelConfig blk)
forall a. a -> Opaque a
QSM.Opaque TopLevelConfig blk
cfg
  }

-- | Key property of the model is that we can go from real to mock responses
toMock :: (Bifunctor (t blk), Eq1 r)
       => Model blk m r -> At t blk m r -> t blk IteratorId FollowerId
toMock :: forall (t :: * -> * -> * -> *) blk (r :: * -> *) (m :: * -> *).
(Bifunctor (t blk), Eq1 r) =>
Model blk m r -> At t blk m r -> t blk Int Int
toMock Model {KnownFollowers blk m r
KnownIters blk m r
Opaque (TopLevelConfig blk)
DBModel blk
$sel:dbModel:Model :: forall blk (m :: * -> *) (r :: * -> *).
Model blk m r -> DBModel blk
$sel:knownIters:Model :: forall blk (m :: * -> *) (r :: * -> *).
Model blk m r -> KnownIters blk m r
$sel:knownFollowers:Model :: forall blk (m :: * -> *) (r :: * -> *).
Model blk m r -> KnownFollowers blk m r
$sel:modelConfig:Model :: forall blk (m :: * -> *) (r :: * -> *).
Model blk m r -> Opaque (TopLevelConfig blk)
dbModel :: DBModel blk
knownIters :: KnownIters blk m r
knownFollowers :: KnownFollowers blk m r
modelConfig :: Opaque (TopLevelConfig blk)
..} (At t blk (IterRef blk m r) (FollowerRef blk m r)
t) = (IterRef blk m r -> Int)
-> (FollowerRef blk m r -> Int)
-> t blk (IterRef blk m r) (FollowerRef blk m r)
-> t blk Int Int
forall a b c d. (a -> b) -> (c -> d) -> t blk a c -> t blk b d
forall (p :: * -> * -> *) a b c d.
Bifunctor p =>
(a -> b) -> (c -> d) -> p a c -> p b d
bimap (KnownIters blk m r
knownIters KnownIters blk m r -> IterRef blk m r -> Int
forall k (r :: * -> *) a.
(Eq k, Eq1 r) =>
RefEnv k a r -> Reference k r -> a
RE.!) (KnownFollowers blk m r
knownFollowers KnownFollowers blk m r -> FollowerRef blk m r -> Int
forall k (r :: * -> *) a.
(Eq k, Eq1 r) =>
RefEnv k a r -> Reference k r -> a
RE.!) t blk (IterRef blk m r) (FollowerRef blk m r)
t

-- | Step the mock semantics
--
-- We cannot step the whole Model here (see 'event', below)
step ::
     (TestConstraints blk, Eq1 r)
  => Model  blk m r
  -> At Cmd blk m r
  -> (Resp  blk IteratorId FollowerId, DBModel blk)
step :: forall blk (r :: * -> *) (m :: * -> *).
(TestConstraints blk, Eq1 r) =>
Model blk m r -> At Cmd blk m r -> (Resp blk Int Int, DBModel blk)
step model :: Model blk m r
model@Model { DBModel blk
$sel:dbModel:Model :: forall blk (m :: * -> *) (r :: * -> *).
Model blk m r -> DBModel blk
dbModel :: DBModel blk
dbModel, Opaque (TopLevelConfig blk)
$sel:modelConfig:Model :: forall blk (m :: * -> *) (r :: * -> *).
Model blk m r -> Opaque (TopLevelConfig blk)
modelConfig :: Opaque (TopLevelConfig blk)
modelConfig } At Cmd blk m r
cmd =
    TopLevelConfig blk
-> Cmd blk Int Int
-> DBModel blk
-> (Resp blk Int Int, DBModel blk)
forall blk.
TestConstraints blk =>
TopLevelConfig blk
-> Cmd blk Int Int
-> DBModel blk
-> (Resp blk Int Int, DBModel blk)
runPure (Opaque (TopLevelConfig blk) -> TopLevelConfig blk
forall a. Opaque a -> a
QSM.unOpaque Opaque (TopLevelConfig blk)
modelConfig) (Model blk m r -> At Cmd blk m r -> Cmd blk Int Int
forall (t :: * -> * -> * -> *) blk (r :: * -> *) (m :: * -> *).
(Bifunctor (t blk), Eq1 r) =>
Model blk m r -> At t blk m r -> t blk Int Int
toMock Model blk m r
model At Cmd blk m r
cmd) DBModel blk
dbModel

{-------------------------------------------------------------------------------
  Wrapping in quickcheck-state-machine references
-------------------------------------------------------------------------------}

-- | Instantiate functor @t blk@ to
-- @t blk ('IterRef' blk m r) ('FollowerRef' blk m r)@.
--
-- Needed because we need to (partially) apply @'At' t blk flr m@ to @r@.
newtype At t blk m r = At { forall (t :: * -> * -> * -> *) blk (m :: * -> *) (r :: * -> *).
At t blk m r -> t blk (IterRef blk m r) (FollowerRef blk m r)
unAt :: t blk (IterRef blk m r) (FollowerRef blk m r) }
  deriving ((forall x. At t blk m r -> Rep (At t blk m r) x)
-> (forall x. Rep (At t blk m r) x -> At t blk m r)
-> Generic (At t blk m r)
forall x. Rep (At t blk m r) x -> At t blk m r
forall x. At t blk m r -> Rep (At t blk m r) x
forall a.
(forall x. a -> Rep a x) -> (forall x. Rep a x -> a) -> Generic a
forall (t :: * -> * -> * -> *) blk (m :: * -> *) (r :: * -> *) x.
Rep (At t blk m r) x -> At t blk m r
forall (t :: * -> * -> * -> *) blk (m :: * -> *) (r :: * -> *) x.
At t blk m r -> Rep (At t blk m r) x
$cfrom :: forall (t :: * -> * -> * -> *) blk (m :: * -> *) (r :: * -> *) x.
At t blk m r -> Rep (At t blk m r) x
from :: forall x. At t blk m r -> Rep (At t blk m r) x
$cto :: forall (t :: * -> * -> * -> *) blk (m :: * -> *) (r :: * -> *) x.
Rep (At t blk m r) x -> At t blk m r
to :: forall x. Rep (At t blk m r) x -> At t blk m r
Generic)


deriving newtype instance Show (t blk (IterRef blk m r) (FollowerRef blk m r))
               => Show (At t blk m r)

deriving instance (TestConstraints blk, Eq1 r) => Eq (At Resp blk m r)

instance Bifunctor (t blk) => Rank2.Functor (At t blk m) where
  fmap :: forall (p :: * -> *) (q :: * -> *).
(forall x. p x -> q x) -> At t blk m p -> At t blk m q
fmap = \forall x. p x -> q x
f (At t blk (IterRef blk m p) (FollowerRef blk m p)
x) -> t blk (IterRef blk m q) (FollowerRef blk m q) -> At t blk m q
forall (t :: * -> * -> * -> *) blk (m :: * -> *) (r :: * -> *).
t blk (IterRef blk m r) (FollowerRef blk m r) -> At t blk m r
At ((IterRef blk m p -> IterRef blk m q)
-> (FollowerRef blk m p -> FollowerRef blk m q)
-> t blk (IterRef blk m p) (FollowerRef blk m p)
-> t blk (IterRef blk m q) (FollowerRef blk m q)
forall a b c d. (a -> b) -> (c -> d) -> t blk a c -> t blk b d
forall (p :: * -> * -> *) a b c d.
Bifunctor p =>
(a -> b) -> (c -> d) -> p a c -> p b d
bimap ((p (Opaque
      (WithEq
         (Iterator
            m
            blk
            (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
             SlotNo, IsEBB, SizeInBytes, Word16,
             SomeSecond (NestedCtxt Header) blk))))
 -> q (Opaque
         (WithEq
            (Iterator
               m
               blk
               (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
                SlotNo, IsEBB, SizeInBytes, Word16,
                SomeSecond (NestedCtxt Header) blk)))))
-> IterRef blk m p -> IterRef blk m q
forall (r :: * -> *) x (r' :: * -> *).
(r x -> r' x) -> Reference x r -> Reference x r'
app p (Opaque
     (WithEq
        (Iterator
           m
           blk
           (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
            SlotNo, IsEBB, SizeInBytes, Word16,
            SomeSecond (NestedCtxt Header) blk))))
-> q (Opaque
        (WithEq
           (Iterator
              m
              blk
              (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
               SlotNo, IsEBB, SizeInBytes, Word16,
               SomeSecond (NestedCtxt Header) blk))))
forall x. p x -> q x
f) ((p (Opaque
      (WithEq
         (Follower
            m
            blk
            (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
             SlotNo, IsEBB, SizeInBytes, Word16,
             SomeSecond (NestedCtxt Header) blk))))
 -> q (Opaque
         (WithEq
            (Follower
               m
               blk
               (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
                SlotNo, IsEBB, SizeInBytes, Word16,
                SomeSecond (NestedCtxt Header) blk)))))
-> FollowerRef blk m p -> FollowerRef blk m q
forall (r :: * -> *) x (r' :: * -> *).
(r x -> r' x) -> Reference x r -> Reference x r'
app p (Opaque
     (WithEq
        (Follower
           m
           blk
           (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
            SlotNo, IsEBB, SizeInBytes, Word16,
            SomeSecond (NestedCtxt Header) blk))))
-> q (Opaque
        (WithEq
           (Follower
              m
              blk
              (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
               SlotNo, IsEBB, SizeInBytes, Word16,
               SomeSecond (NestedCtxt Header) blk))))
forall x. p x -> q x
f) t blk (IterRef blk m p) (FollowerRef blk m p)
x)
    where
      app :: (r x -> r' x) -> QSM.Reference x r -> QSM.Reference x r'
      app :: forall (r :: * -> *) x (r' :: * -> *).
(r x -> r' x) -> Reference x r -> Reference x r'
app r x -> r' x
f (QSM.Reference r x
x) = r' x -> Reference x r'
forall a (r :: * -> *). r a -> Reference a r
QSM.Reference (r x -> r' x
f r x
x)

instance Bifoldable (t blk) => Rank2.Foldable (At t blk m) where
  foldMap :: forall m (p :: * -> *).
Monoid m =>
(forall x. p x -> m) -> At t blk m p -> m
foldMap = \forall x. p x -> m
f (At t blk (IterRef blk m p) (FollowerRef blk m p)
x) -> (IterRef blk m p -> m)
-> (FollowerRef blk m p -> m)
-> t blk (IterRef blk m p) (FollowerRef blk m p)
-> m
forall m a b. Monoid m => (a -> m) -> (b -> m) -> t blk a b -> m
forall (p :: * -> * -> *) m a b.
(Bifoldable p, Monoid m) =>
(a -> m) -> (b -> m) -> p a b -> m
bifoldMap ((p (Opaque
      (WithEq
         (Iterator
            m
            blk
            (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
             SlotNo, IsEBB, SizeInBytes, Word16,
             SomeSecond (NestedCtxt Header) blk))))
 -> m)
-> IterRef blk m p -> m
forall (r :: * -> *) x n. (r x -> n) -> Reference x r -> n
app p (Opaque
     (WithEq
        (Iterator
           m
           blk
           (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
            SlotNo, IsEBB, SizeInBytes, Word16,
            SomeSecond (NestedCtxt Header) blk))))
-> m
forall x. p x -> m
f) ((p (Opaque
      (WithEq
         (Follower
            m
            blk
            (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
             SlotNo, IsEBB, SizeInBytes, Word16,
             SomeSecond (NestedCtxt Header) blk))))
 -> m)
-> FollowerRef blk m p -> m
forall (r :: * -> *) x n. (r x -> n) -> Reference x r -> n
app p (Opaque
     (WithEq
        (Follower
           m
           blk
           (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
            SlotNo, IsEBB, SizeInBytes, Word16,
            SomeSecond (NestedCtxt Header) blk))))
-> m
forall x. p x -> m
f) t blk (IterRef blk m p) (FollowerRef blk m p)
x
    where
      app :: (r x -> n) -> QSM.Reference x r -> n
      app :: forall (r :: * -> *) x n. (r x -> n) -> Reference x r -> n
app r x -> n
f (QSM.Reference r x
x) = r x -> n
f r x
x

instance Bitraversable (t blk) => Rank2.Traversable (At t blk m) where
  traverse :: forall (f :: * -> *) (p :: * -> *) (q :: * -> *).
Applicative f =>
(forall a. p a -> f (q a)) -> At t blk m p -> f (At t blk m q)
traverse = \forall a. p a -> f (q a)
f (At t blk (IterRef blk m p) (FollowerRef blk m p)
x) -> t blk (IterRef blk m q) (FollowerRef blk m q) -> At t blk m q
forall (t :: * -> * -> * -> *) blk (m :: * -> *) (r :: * -> *).
t blk (IterRef blk m r) (FollowerRef blk m r) -> At t blk m r
At (t blk (IterRef blk m q) (FollowerRef blk m q) -> At t blk m q)
-> f (t blk (IterRef blk m q) (FollowerRef blk m q))
-> f (At t blk m q)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> (IterRef blk m p -> f (IterRef blk m q))
-> (FollowerRef blk m p -> f (FollowerRef blk m q))
-> t blk (IterRef blk m p) (FollowerRef blk m p)
-> f (t blk (IterRef blk m q) (FollowerRef blk m q))
forall (f :: * -> *) a c b d.
Applicative f =>
(a -> f c) -> (b -> f d) -> t blk a b -> f (t blk c d)
forall (t :: * -> * -> *) (f :: * -> *) a c b d.
(Bitraversable t, Applicative f) =>
(a -> f c) -> (b -> f d) -> t a b -> f (t c d)
bitraverse ((p (Opaque
      (WithEq
         (Iterator
            m
            blk
            (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
             SlotNo, IsEBB, SizeInBytes, Word16,
             SomeSecond (NestedCtxt Header) blk))))
 -> f (q (Opaque
            (WithEq
               (Iterator
                  m
                  blk
                  (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
                   SlotNo, IsEBB, SizeInBytes, Word16,
                   SomeSecond (NestedCtxt Header) blk))))))
-> IterRef blk m p -> f (IterRef blk m q)
forall (f :: * -> *) (r :: * -> *) x (r' :: * -> *).
Functor f =>
(r x -> f (r' x)) -> Reference x r -> f (Reference x r')
app p (Opaque
     (WithEq
        (Iterator
           m
           blk
           (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
            SlotNo, IsEBB, SizeInBytes, Word16,
            SomeSecond (NestedCtxt Header) blk))))
-> f (q (Opaque
           (WithEq
              (Iterator
                 m
                 blk
                 (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
                  SlotNo, IsEBB, SizeInBytes, Word16,
                  SomeSecond (NestedCtxt Header) blk)))))
forall a. p a -> f (q a)
f) ((p (Opaque
      (WithEq
         (Follower
            m
            blk
            (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
             SlotNo, IsEBB, SizeInBytes, Word16,
             SomeSecond (NestedCtxt Header) blk))))
 -> f (q (Opaque
            (WithEq
               (Follower
                  m
                  blk
                  (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
                   SlotNo, IsEBB, SizeInBytes, Word16,
                   SomeSecond (NestedCtxt Header) blk))))))
-> FollowerRef blk m p -> f (FollowerRef blk m q)
forall (f :: * -> *) (r :: * -> *) x (r' :: * -> *).
Functor f =>
(r x -> f (r' x)) -> Reference x r -> f (Reference x r')
app p (Opaque
     (WithEq
        (Follower
           m
           blk
           (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
            SlotNo, IsEBB, SizeInBytes, Word16,
            SomeSecond (NestedCtxt Header) blk))))
-> f (q (Opaque
           (WithEq
              (Follower
                 m
                 blk
                 (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
                  SlotNo, IsEBB, SizeInBytes, Word16,
                  SomeSecond (NestedCtxt Header) blk)))))
forall a. p a -> f (q a)
f) t blk (IterRef blk m p) (FollowerRef blk m p)
x
    where
      app :: Functor f
          => (r x -> f (r' x)) -> QSM.Reference x r -> f (QSM.Reference x r')
      app :: forall (f :: * -> *) (r :: * -> *) x (r' :: * -> *).
Functor f =>
(r x -> f (r' x)) -> Reference x r -> f (Reference x r')
app r x -> f (r' x)
f (QSM.Reference r x
x) = r' x -> Reference x r'
forall a (r :: * -> *). r a -> Reference a r
QSM.Reference (r' x -> Reference x r') -> f (r' x) -> f (Reference x r')
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> r x -> f (r' x)
f r x
x

{-------------------------------------------------------------------------------
  Events
-------------------------------------------------------------------------------}

-- | An event records the model before and after a command along with the
-- command itself, and a mocked version of the response.
data Event blk m r = Event
  { forall blk (m :: * -> *) (r :: * -> *).
Event blk m r -> Model blk m r
eventBefore   :: Model  blk m r
  , forall blk (m :: * -> *) (r :: * -> *).
Event blk m r -> At Cmd blk m r
eventCmd      :: At Cmd blk m r
  , forall blk (m :: * -> *) (r :: * -> *).
Event blk m r -> Model blk m r
eventAfter    :: Model  blk m r
  , forall blk (m :: * -> *) (r :: * -> *).
Event blk m r -> Resp blk Int Int
eventMockResp :: Resp   blk     IteratorId FollowerId
  }

deriving instance (TestConstraints blk, Show1 r) => Show (Event blk m r)

-- | Construct an event
lockstep :: (TestConstraints blk, Eq1 r, Show1 r)
         => Model     blk m r
         -> At Cmd    blk m r
         -> At Resp   blk m r
         -> Event     blk m r
lockstep :: forall blk (r :: * -> *) (m :: * -> *).
(TestConstraints blk, Eq1 r, Show1 r) =>
Model blk m r -> At Cmd blk m r -> At Resp blk m r -> Event blk m r
lockstep model :: Model blk m r
model@Model {KnownFollowers blk m r
KnownIters blk m r
Opaque (TopLevelConfig blk)
DBModel blk
$sel:dbModel:Model :: forall blk (m :: * -> *) (r :: * -> *).
Model blk m r -> DBModel blk
$sel:knownIters:Model :: forall blk (m :: * -> *) (r :: * -> *).
Model blk m r -> KnownIters blk m r
$sel:knownFollowers:Model :: forall blk (m :: * -> *) (r :: * -> *).
Model blk m r -> KnownFollowers blk m r
$sel:modelConfig:Model :: forall blk (m :: * -> *) (r :: * -> *).
Model blk m r -> Opaque (TopLevelConfig blk)
dbModel :: DBModel blk
knownIters :: KnownIters blk m r
knownFollowers :: KnownFollowers blk m r
modelConfig :: Opaque (TopLevelConfig blk)
..} At Cmd blk m r
cmd (At Resp blk (IterRef blk m r) (FollowerRef blk m r)
resp) = Event
    { $sel:eventBefore:Event :: Model blk m r
eventBefore   = Model blk m r
model
    , $sel:eventCmd:Event :: At Cmd blk m r
eventCmd      = At Cmd blk m r
cmd
    , $sel:eventAfter:Event :: Model blk m r
eventAfter    = Model blk m r
model'
    , $sel:eventMockResp:Event :: Resp blk Int Int
eventMockResp = Resp blk Int Int
mockResp
    }
  where
    (Resp blk Int Int
mockResp, DBModel blk
dbModel') = Model blk m r -> At Cmd blk m r -> (Resp blk Int Int, DBModel blk)
forall blk (r :: * -> *) (m :: * -> *).
(TestConstraints blk, Eq1 r) =>
Model blk m r -> At Cmd blk m r -> (Resp blk Int Int, DBModel blk)
step Model blk m r
model At Cmd blk m r
cmd
    newIters :: KnownIters blk m r
newIters   = [(IterRef blk m r, Int)] -> KnownIters blk m r
forall k a (r :: * -> *).
(Eq k, Show k, Eq a, Show a, Eq1 r, Show1 r, HasCallStack) =>
[(Reference k r, a)] -> RefEnv k a r
RE.fromList ([(IterRef blk m r, Int)] -> KnownIters blk m r)
-> [(IterRef blk m r, Int)] -> KnownIters blk m r
forall a b. (a -> b) -> a -> b
$ [IterRef blk m r] -> [Int] -> [(IterRef blk m r, Int)]
forall a b. [a] -> [b] -> [(a, b)]
zip (Resp blk (IterRef blk m r) (FollowerRef blk m r)
-> [IterRef blk m r]
forall (t :: * -> * -> *) it flr.
Bitraversable t =>
t it flr -> [it]
iters Resp blk (IterRef blk m r) (FollowerRef blk m r)
resp) (Resp blk Int Int -> [Int]
forall (t :: * -> * -> *) it flr.
Bitraversable t =>
t it flr -> [it]
iters Resp blk Int Int
mockResp)
    newFollowers :: KnownFollowers blk m r
newFollowers = [(FollowerRef blk m r, Int)] -> KnownFollowers blk m r
forall k a (r :: * -> *).
(Eq k, Show k, Eq a, Show a, Eq1 r, Show1 r, HasCallStack) =>
[(Reference k r, a)] -> RefEnv k a r
RE.fromList ([(FollowerRef blk m r, Int)] -> KnownFollowers blk m r)
-> [(FollowerRef blk m r, Int)] -> KnownFollowers blk m r
forall a b. (a -> b) -> a -> b
$ [FollowerRef blk m r] -> [Int] -> [(FollowerRef blk m r, Int)]
forall a b. [a] -> [b] -> [(a, b)]
zip (Resp blk (IterRef blk m r) (FollowerRef blk m r)
-> [FollowerRef blk m r]
forall (t :: * -> * -> *) it flr.
Bitraversable t =>
t it flr -> [flr]
flrs  Resp blk (IterRef blk m r) (FollowerRef blk m r)
resp) (Resp blk Int Int -> [Int]
forall (t :: * -> * -> *) it flr.
Bitraversable t =>
t it flr -> [flr]
flrs  Resp blk Int Int
mockResp)
    model' :: Model blk m r
model' = case At Cmd blk m r -> Cmd blk (IterRef blk m r) (FollowerRef blk m r)
forall (t :: * -> * -> * -> *) blk (m :: * -> *) (r :: * -> *).
At t blk m r -> t blk (IterRef blk m r) (FollowerRef blk m r)
unAt At Cmd blk m r
cmd of
      -- When closing the database, all open iterators and followers are closed
      -- too, so forget them.
      Cmd blk (IterRef blk m r) (FollowerRef blk m r)
Close -> Model blk m r
model
        { dbModel        = dbModel'
        , knownIters     = RE.empty
        , knownFollowers = RE.empty
        }
      Cmd blk (IterRef blk m r) (FollowerRef blk m r)
WipeVolatileDB -> Model blk m r
model
        { dbModel        = dbModel'
        , knownIters     = RE.empty
        , knownFollowers = RE.empty
        }
      Cmd blk (IterRef blk m r) (FollowerRef blk m r)
_ -> Model blk m r
model
        { dbModel        = dbModel'
        , knownIters     = knownIters `RE.union` newIters
        , knownFollowers = knownFollowers `RE.union` newFollowers
        }


{-------------------------------------------------------------------------------
  Generator
-------------------------------------------------------------------------------}

type BlockGen blk m = Model blk m Symbolic -> Gen blk

-- | Generate a 'Cmd'
generator ::
     forall blk m. TestConstraints blk
  => LoE ()
  -> BlockGen     blk m
  -> Model        blk m Symbolic
  -> Gen (At Cmd  blk m Symbolic)
generator :: forall blk (m :: * -> *).
TestConstraints blk =>
LoE ()
-> BlockGen blk m
-> Model blk m Symbolic
-> Gen (At Cmd blk m Symbolic)
generator LoE ()
loe BlockGen blk m
genBlock m :: Model blk m Symbolic
m@Model {KnownFollowers blk m Symbolic
KnownIters blk m Symbolic
Opaque (TopLevelConfig blk)
DBModel blk
$sel:dbModel:Model :: forall blk (m :: * -> *) (r :: * -> *).
Model blk m r -> DBModel blk
$sel:knownIters:Model :: forall blk (m :: * -> *) (r :: * -> *).
Model blk m r -> KnownIters blk m r
$sel:knownFollowers:Model :: forall blk (m :: * -> *) (r :: * -> *).
Model blk m r -> KnownFollowers blk m r
$sel:modelConfig:Model :: forall blk (m :: * -> *) (r :: * -> *).
Model blk m r -> Opaque (TopLevelConfig blk)
dbModel :: DBModel blk
knownIters :: KnownIters blk m Symbolic
knownFollowers :: KnownFollowers blk m Symbolic
modelConfig :: Opaque (TopLevelConfig blk)
..} = Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic)
-> At Cmd blk m Symbolic
forall (t :: * -> * -> * -> *) blk (m :: * -> *) (r :: * -> *).
t blk (IterRef blk m r) (FollowerRef blk m r) -> At t blk m r
At (Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic)
 -> At Cmd blk m Symbolic)
-> Gen
     (Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic))
-> Gen (At Cmd blk m Symbolic)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> [(Int,
  Gen
    (Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic)))]
-> Gen
     (Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic))
forall a. [(Int, Gen a)] -> Gen a
frequency
    [ (Int
30, Gen (Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic))
genAddBlock)
    , (if Bool
empty then Int
1 else Int
10, Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic)
-> Gen
     (Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic))
forall a. a -> Gen a
forall (m :: * -> *) a. Monad m => a -> m a
return Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic)
forall blk it flr. Cmd blk it flr
GetCurrentChain)
    , (if Bool
empty then Int
1 else Int
10, Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic)
-> Gen
     (Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic))
forall a. a -> Gen a
forall (m :: * -> *) a. Monad m => a -> m a
return Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic)
forall blk it flr. Cmd blk it flr
GetLedgerDB)
    , (if Bool
empty then Int
1 else Int
10, Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic)
-> Gen
     (Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic))
forall a. a -> Gen a
forall (m :: * -> *) a. Monad m => a -> m a
return Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic)
forall blk it flr. Cmd blk it flr
GetTipBlock)
      -- To check that we're on the right chain
    , (if Bool
empty then Int
1 else Int
10, Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic)
-> Gen
     (Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic))
forall a. a -> Gen a
forall (m :: * -> *) a. Monad m => a -> m a
return Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic)
forall blk it flr. Cmd blk it flr
GetTipPoint)
    , (Int
10, Gen (Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic))
forall it flr. Gen (Cmd blk it flr)
genGetBlockComponent)
    , (if Bool
empty then Int
1 else Int
10, Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic)
-> Gen
     (Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic))
forall a. a -> Gen a
forall (m :: * -> *) a. Monad m => a -> m a
return Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic)
forall blk it flr. Cmd blk it flr
GetMaxSlotNo)
    , (if Bool
empty then Int
1 else Int
10, Gen (Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic))
forall it flr. Gen (Cmd blk it flr)
genGetIsValid)

    , let freq :: Int
freq = case LoE ()
loe of
            LoE ()
LoEDisabled   -> Int
0
            LoEEnabled () -> if Bool
empty then Int
1 else Int
10
      in (Int
freq, AnchoredFragment blk
-> Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic)
forall blk it flr. AnchoredFragment blk -> Cmd blk it flr
UpdateLoE (AnchoredFragment blk
 -> Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic))
-> Gen (AnchoredFragment blk)
-> Gen
     (Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Gen (AnchoredFragment blk)
genLoEFragment)

    -- Iterators
    , (if Bool
empty then Int
1 else Int
10, (StreamFrom blk
 -> StreamTo blk
 -> Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic))
-> (StreamFrom blk, StreamTo blk)
-> Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic)
forall a b c. (a -> b -> c) -> (a, b) -> c
uncurry StreamFrom blk
-> StreamTo blk
-> Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic)
forall blk it flr. StreamFrom blk -> StreamTo blk -> Cmd blk it flr
Stream ((StreamFrom blk, StreamTo blk)
 -> Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic))
-> Gen (StreamFrom blk, StreamTo blk)
-> Gen
     (Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Gen (StreamFrom blk, StreamTo blk)
genBounds)
    , (if [IterRef blk m Symbolic] -> Bool
forall a. [a] -> Bool
forall (t :: * -> *) a. Foldable t => t a -> Bool
null [IterRef blk m Symbolic]
iterators then Int
0 else Int
20, Gen (Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic))
genIteratorNext)
      -- Use a lower frequency for closing, so that the chance increases that
      -- we can stream multiple blocks from an iterator.
    , (if [IterRef blk m Symbolic] -> Bool
forall a. [a] -> Bool
forall (t :: * -> *) a. Foldable t => t a -> Bool
null [IterRef blk m Symbolic]
iterators then Int
0 else Int
2, Gen (Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic))
genIteratorClose)

    -- Followers
    , (Int
10, Gen (Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic))
forall {blk} {it} {flr}. Gen (Cmd blk it flr)
genNewFollower)
    , (if [FollowerRef blk m Symbolic] -> Bool
forall a. [a] -> Bool
forall (t :: * -> *) a. Foldable t => t a -> Bool
null [FollowerRef blk m Symbolic]
followers then Int
0 else Int
10, Gen (Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic))
genFollowerInstruction)
    , (if [FollowerRef blk m Symbolic] -> Bool
forall a. [a] -> Bool
forall (t :: * -> *) a. Foldable t => t a -> Bool
null [FollowerRef blk m Symbolic]
followers then Int
0 else Int
10, Gen (Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic))
genFollowerForward)
      -- Use a lower frequency for closing, so that the chance increases that
      -- we can read multiple blocks from a follower
    , (if [FollowerRef blk m Symbolic] -> Bool
forall a. [a] -> Bool
forall (t :: * -> *) a. Foldable t => t a -> Bool
null [FollowerRef blk m Symbolic]
followers then Int
0 else Int
2, Gen (Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic))
genFollowerClose)

    , (if Bool
empty then Int
1 else Int
10, Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic)
-> Gen
     (Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic))
forall a. a -> Gen a
forall (m :: * -> *) a. Monad m => a -> m a
return Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic)
forall blk it flr. Cmd blk it flr
Close)
    , (if DBModel blk -> Bool
forall blk. Model blk -> Bool
Model.isOpen DBModel blk
dbModel then
         (if Bool
empty then Int
1 else Int
10)
       else Int
0, Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic)
-> Gen
     (Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic))
forall a. a -> Gen a
forall (m :: * -> *) a. Monad m => a -> m a
return Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic)
forall blk it flr. Cmd blk it flr
Reopen)

      -- Internal
    , (if Bool
empty then Int
1 else Int
10, Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic)
-> Gen
     (Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic))
forall a. a -> Gen a
forall (m :: * -> *) a. Monad m => a -> m a
return Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic)
forall blk it flr. Cmd blk it flr
PersistBlks)
    , (if Bool
empty then Int
1 else Int
10, Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic)
-> Gen
     (Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic))
forall a. a -> Gen a
forall (m :: * -> *) a. Monad m => a -> m a
return Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic)
forall blk it flr. Cmd blk it flr
PersistBlksThenGC)
    , (if Bool
empty then Int
1 else Int
10, Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic)
-> Gen
     (Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic))
forall a. a -> Gen a
forall (m :: * -> *) a. Monad m => a -> m a
return Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic)
forall blk it flr. Cmd blk it flr
UpdateLedgerSnapshots)
    , (if Bool
empty then Int
1 else Int
10, Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic)
-> Gen
     (Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic))
forall a. a -> Gen a
forall (m :: * -> *) a. Monad m => a -> m a
return Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic)
forall blk it flr. Cmd blk it flr
WipeVolatileDB)
    ]
    -- TODO adjust the frequencies after labelling
  where
    cfg :: TopLevelConfig blk
    cfg :: TopLevelConfig blk
cfg = Opaque (TopLevelConfig blk) -> TopLevelConfig blk
forall a. Opaque a -> a
unOpaque Opaque (TopLevelConfig blk)
modelConfig

    secParam :: SecurityParam
    secParam :: SecurityParam
secParam = TopLevelConfig blk -> SecurityParam
forall blk.
ConsensusProtocol (BlockProtocol blk) =>
TopLevelConfig blk -> SecurityParam
configSecurityParam TopLevelConfig blk
cfg

    iterators :: [Reference (Opaque (TestIterator m blk)) Symbolic]
    iterators :: [IterRef blk m Symbolic]
iterators = KnownIters blk m Symbolic -> [IterRef blk m Symbolic]
forall k a (r :: * -> *). RefEnv k a r -> [Reference k r]
RE.keys KnownIters blk m Symbolic
knownIters

    followers :: [Reference (Opaque (TestFollower m blk)) Symbolic]
    followers :: [FollowerRef blk m Symbolic]
followers = KnownFollowers blk m Symbolic -> [FollowerRef blk m Symbolic]
forall k a (r :: * -> *). RefEnv k a r -> [Reference k r]
RE.keys KnownFollowers blk m Symbolic
knownFollowers

    genRandomPoint :: Gen (RealPoint blk)
    genRandomPoint :: Gen (RealPoint blk)
genRandomPoint = blk -> RealPoint blk
forall blk. HasHeader blk => blk -> RealPoint blk
blockRealPoint (blk -> RealPoint blk) -> Gen blk -> Gen (RealPoint blk)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> BlockGen blk m
genBlock Model blk m Symbolic
m

    blocksInDB :: Map.Map (HeaderHash blk) blk
    blocksInDB :: Map (HeaderHash blk) blk
blocksInDB = DBModel blk -> Map (HeaderHash blk) blk
forall blk. HasHeader blk => Model blk -> Map (HeaderHash blk) blk
Model.blocks DBModel blk
dbModel

    pointsInDB :: [RealPoint blk]
    pointsInDB :: [RealPoint blk]
pointsInDB = blk -> RealPoint blk
forall blk. HasHeader blk => blk -> RealPoint blk
blockRealPoint (blk -> RealPoint blk) -> [blk] -> [RealPoint blk]
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Map (HeaderHash blk) blk -> [blk]
forall k a. Map k a -> [a]
Map.elems Map (HeaderHash blk) blk
blocksInDB

    genLoEFragment :: Gen (AnchoredFragment blk)
    genLoEFragment :: Gen (AnchoredFragment blk)
genLoEFragment = [(Int, Gen (AnchoredFragment blk))] -> Gen (AnchoredFragment blk)
forall a. [(Int, Gen a)] -> Gen a
frequency
        [ (Int
1, AnchoredFragment blk -> Gen (AnchoredFragment blk)
forall a. a -> Gen a
forall (m :: * -> *) a. Monad m => a -> m a
return (AnchoredFragment blk -> Gen (AnchoredFragment blk))
-> AnchoredFragment blk -> Gen (AnchoredFragment blk)
forall a b. (a -> b) -> a -> b
$ Anchor blk -> AnchoredFragment blk
forall v a b. Anchorable v a b => a -> AnchoredSeq v a b
AF.Empty Anchor blk
forall block. Anchor block
AF.AnchorGenesis)
        , (Int
20, (Gen (Maybe (AnchoredFragment blk))
 -> (Maybe (AnchoredFragment blk) -> Maybe (AnchoredFragment blk))
 -> Gen (AnchoredFragment blk))
-> (Maybe (AnchoredFragment blk) -> Maybe (AnchoredFragment blk))
-> Gen (Maybe (AnchoredFragment blk))
-> Gen (AnchoredFragment blk)
forall a b c. (a -> b -> c) -> b -> a -> c
flip Gen (Maybe (AnchoredFragment blk))
-> (Maybe (AnchoredFragment blk) -> Maybe (AnchoredFragment blk))
-> Gen (AnchoredFragment blk)
forall a b. Gen a -> (a -> Maybe b) -> Gen b
suchThatMap Maybe (AnchoredFragment blk) -> Maybe (AnchoredFragment blk)
forall a. a -> a
id (Gen (Maybe (AnchoredFragment blk)) -> Gen (AnchoredFragment blk))
-> Gen (Maybe (AnchoredFragment blk)) -> Gen (AnchoredFragment blk)
forall a b. (a -> b) -> a -> b
$ do
            -- Generate a fragment between an anchor in the ImmutableDB and a
            -- tip corresponding to either the immutable tip, a volatile block
            -- or a block not yet in the ChainDB.
            Anchor blk
anchor <- [Anchor blk] -> Gen (Anchor blk)
forall a. HasCallStack => [a] -> Gen a
elements ([Anchor blk] -> Gen (Anchor blk))
-> [Anchor blk] -> Gen (Anchor blk)
forall a b. (a -> b) -> a -> b
$
              Anchor blk
forall block. Anchor block
AF.AnchorGenesis Anchor blk -> [Anchor blk] -> [Anchor blk]
forall a. a -> [a] -> [a]
: (blk -> Anchor blk) -> [blk] -> [Anchor blk]
forall a b. (a -> b) -> [a] -> [b]
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap blk -> Anchor blk
forall block. HasHeader block => block -> Anchor block
AF.anchorFromBlock [blk]
immutableBlocks
            blk
blk <- BlockGen blk m
genBlock Model blk m Symbolic
m
            ChainHash blk
tip    <- [(Int, Gen (ChainHash blk))] -> Gen (ChainHash blk)
forall a. [(Int, Gen a)] -> Gen a
frequency
              [ (Int
1, ChainHash blk -> Gen (ChainHash blk)
forall a. a -> Gen a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (ChainHash blk -> Gen (ChainHash blk))
-> ChainHash blk -> Gen (ChainHash blk)
forall a b. (a -> b) -> a -> b
$ Chain blk -> ChainHash blk
forall block. HasHeader block => Chain block -> ChainHash block
Chain.headHash Chain blk
immutableChain)
              , (Int
5, ChainHash blk -> Gen (ChainHash blk)
forall a. a -> Gen a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (ChainHash blk -> Gen (ChainHash blk))
-> ChainHash blk -> Gen (ChainHash blk)
forall a b. (a -> b) -> a -> b
$ HeaderHash blk -> ChainHash blk
forall {k} (b :: k). HeaderHash b -> ChainHash b
BlockHash (blk -> HeaderHash blk
forall b. HasHeader b => b -> HeaderHash b
blockHash blk
blk))
              , ( if [blk] -> Bool
forall a. [a] -> Bool
forall (t :: * -> *) a. Foldable t => t a -> Bool
null [blk]
volatileBlocks then Int
0 else Int
5
                , [ChainHash blk] -> Gen (ChainHash blk)
forall a. HasCallStack => [a] -> Gen a
elements ([ChainHash blk] -> Gen (ChainHash blk))
-> [ChainHash blk] -> Gen (ChainHash blk)
forall a b. (a -> b) -> a -> b
$ HeaderHash blk -> ChainHash blk
forall {k} (b :: k). HeaderHash b -> ChainHash b
BlockHash (HeaderHash blk -> ChainHash blk)
-> (blk -> HeaderHash blk) -> blk -> ChainHash blk
forall b c a. (b -> c) -> (a -> b) -> a -> c
. blk -> HeaderHash blk
forall b. HasHeader b => b -> HeaderHash b
blockHash (blk -> ChainHash blk) -> [blk] -> [ChainHash blk]
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> [blk]
volatileBlocks
                )
              ]
            let blks :: Map (HeaderHash blk) blk
blks = HeaderHash blk
-> blk -> Map (HeaderHash blk) blk -> Map (HeaderHash blk) blk
forall k a. Ord k => k -> a -> Map k a -> Map k a
Map.insert (blk -> HeaderHash blk
forall b. HasHeader b => b -> HeaderHash b
blockHash blk
blk) blk
blk Map (HeaderHash blk) blk
blocksInDB
            Maybe (AnchoredFragment blk) -> Gen (Maybe (AnchoredFragment blk))
forall a. a -> Gen a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (Maybe (AnchoredFragment blk)
 -> Gen (Maybe (AnchoredFragment blk)))
-> Maybe (AnchoredFragment blk)
-> Gen (Maybe (AnchoredFragment blk))
forall a b. (a -> b) -> a -> b
$ Map (HeaderHash blk) blk
-> Anchor blk -> ChainHash blk -> Maybe (AnchoredFragment blk)
forall blk.
GetPrevHash blk =>
Map (HeaderHash blk) blk
-> Anchor blk -> ChainHash blk -> Maybe (AnchoredFragment blk)
Model.getFragmentBetween Map (HeaderHash blk) blk
blks Anchor blk
anchor ChainHash blk
tip)
        ]
      where
        immutableChain :: Chain blk
immutableChain  = SecurityParam -> DBModel blk -> Chain blk
forall blk. SecurityParam -> Model blk -> Chain blk
Model.immutableChain SecurityParam
secParam DBModel blk
dbModel
        immutableBlocks :: [blk]
immutableBlocks = Chain blk -> [blk]
forall block. Chain block -> [block]
Chain.toNewestFirst Chain blk
immutableChain
        volatileBlocks :: [blk]
volatileBlocks  = Map (HeaderHash blk) blk -> [blk]
forall k a. Map k a -> [a]
Map.elems (Map (HeaderHash blk) blk -> [blk])
-> Map (HeaderHash blk) blk -> [blk]
forall a b. (a -> b) -> a -> b
$ DBModel blk -> Map (HeaderHash blk) blk
forall blk. Model blk -> Map (HeaderHash blk) blk
Model.volatileDbBlocks DBModel blk
dbModel

    empty :: Bool
    empty :: Bool
empty = [RealPoint blk] -> Bool
forall a. [a] -> Bool
forall (t :: * -> *) a. Foldable t => t a -> Bool
null [RealPoint blk]
pointsInDB

    genRealPoint :: Gen (RealPoint blk)
    genRealPoint :: Gen (RealPoint blk)
genRealPoint = [(Int, Gen (RealPoint blk))] -> Gen (RealPoint blk)
forall a. [(Int, Gen a)] -> Gen a
frequency
      [ (Int
1, Gen (RealPoint blk)
genRandomPoint)
      , (if Bool
empty then Int
0 else Int
7, [RealPoint blk] -> Gen (RealPoint blk)
forall a. HasCallStack => [a] -> Gen a
elements [RealPoint blk]
pointsInDB)
      ]

    genPoint :: Gen (Point blk)
    genPoint :: Gen (Point blk)
genPoint = [(Int, Gen (Point blk))] -> Gen (Point blk)
forall a. [(Int, Gen a)] -> Gen a
frequency
      [ (Int
1, Point blk -> Gen (Point blk)
forall a. a -> Gen a
forall (m :: * -> *) a. Monad m => a -> m a
return Point blk
forall {k} (block :: k). Point block
GenesisPoint)
      , (Int
9, RealPoint blk -> Point blk
forall blk. RealPoint blk -> Point blk
realPointToPoint (RealPoint blk -> Point blk)
-> Gen (RealPoint blk) -> Gen (Point blk)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Gen (RealPoint blk)
genRealPoint)
      ]

    genGetIsValid :: Gen (Cmd blk it flr)
    genGetIsValid :: forall it flr. Gen (Cmd blk it flr)
genGetIsValid =
      RealPoint blk -> Cmd blk it flr
forall blk it flr. RealPoint blk -> Cmd blk it flr
GetIsValid (RealPoint blk -> Cmd blk it flr)
-> Gen (RealPoint blk) -> Gen (Cmd blk it flr)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Gen (RealPoint blk)
genRealPoint Gen (RealPoint blk)
-> (RealPoint blk -> Bool) -> Gen (RealPoint blk)
forall a. Gen a -> (a -> Bool) -> Gen a
`suchThat` \(RealPoint SlotNo
_ HeaderHash blk
hash) ->
        -- Ignore blocks from the future, since the real implementation might
        -- have validated them before detecting they're from the future,
        -- whereas the model won't include them in the output of
        -- 'Model.getIsValid' (which uses 'Model.validChains').
        HeaderHash blk -> Map (HeaderHash blk) blk -> Bool
forall k a. Ord k => k -> Map k a -> Bool
Map.notMember HeaderHash blk
hash (DBModel blk -> Map (HeaderHash blk) blk
forall blk. HasHeader blk => Model blk -> Map (HeaderHash blk) blk
Model.futureBlocks DBModel blk
dbModel)

    genGetBlockComponent :: Gen (Cmd blk it flr)
    genGetBlockComponent :: forall it flr. Gen (Cmd blk it flr)
genGetBlockComponent = do
      RealPoint blk
pt <- Gen (RealPoint blk)
genRealPoint
      Cmd blk it flr -> Gen (Cmd blk it flr)
forall a. a -> Gen a
forall (m :: * -> *) a. Monad m => a -> m a
return (Cmd blk it flr -> Gen (Cmd blk it flr))
-> Cmd blk it flr -> Gen (Cmd blk it flr)
forall a b. (a -> b) -> a -> b
$ if SecurityParam -> DBModel blk -> RealPoint blk -> Bool
forall blk.
HasHeader blk =>
SecurityParam -> Model blk -> RealPoint blk -> Bool
Model.garbageCollectablePoint SecurityParam
secParam DBModel blk
dbModel RealPoint blk
pt
        then RealPoint blk -> Cmd blk it flr
forall blk it flr. RealPoint blk -> Cmd blk it flr
GetGCedBlockComponent RealPoint blk
pt
        else RealPoint blk -> Cmd blk it flr
forall blk it flr. RealPoint blk -> Cmd blk it flr
GetBlockComponent     RealPoint blk
pt

    genAddBlock :: Gen (Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic))
genAddBlock = do
      let curSlot :: SlotNo
curSlot = DBModel blk -> SlotNo
forall blk. Model blk -> SlotNo
Model.currentSlot DBModel blk
dbModel
      blk
blk <- BlockGen blk m
genBlock Model blk m Symbolic
m
      if blk -> SlotNo
forall b. HasHeader b => b -> SlotNo
blockSlot blk
blk SlotNo -> SlotNo -> Bool
forall a. Ord a => a -> a -> Bool
> DBModel blk -> SlotNo
forall blk. Model blk -> SlotNo
Model.currentSlot DBModel blk
dbModel
        -- When the slot of the block is in the future, we can either advance
        -- the current time ('AddBlock') or choose to add a block from the
        -- future ('AddFutureBlock')
        then [(Int,
  Gen
    (Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic)))]
-> Gen
     (Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic))
forall a. [(Int, Gen a)] -> Gen a
frequency
          [ (Int
1, Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic)
-> Gen
     (Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic))
forall a. a -> Gen a
forall (m :: * -> *) a. Monad m => a -> m a
return (Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic)
 -> Gen
      (Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic)))
-> Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic)
-> Gen
     (Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic))
forall a b. (a -> b) -> a -> b
$ blk
-> Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic)
forall blk it flr. blk -> Cmd blk it flr
AddBlock blk
blk)
          , (Int
1, blk
-> SlotNo
-> Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic)
forall blk it flr. blk -> SlotNo -> Cmd blk it flr
AddFutureBlock blk
blk (SlotNo
 -> Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic))
-> Gen SlotNo
-> Gen
     (Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> SlotNo -> SlotNo -> Gen SlotNo
chooseSlot SlotNo
curSlot (blk -> SlotNo
forall b. HasHeader b => b -> SlotNo
blockSlot blk
blk SlotNo -> SlotNo -> SlotNo
forall a. Num a => a -> a -> a
- SlotNo
1))
          ]
        else Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic)
-> Gen
     (Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic))
forall a. a -> Gen a
forall (m :: * -> *) a. Monad m => a -> m a
return (Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic)
 -> Gen
      (Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic)))
-> Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic)
-> Gen
     (Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic))
forall a b. (a -> b) -> a -> b
$ blk
-> Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic)
forall blk it flr. blk -> Cmd blk it flr
AddBlock blk
blk

    genBounds :: Gen (StreamFrom blk, StreamTo blk)
    genBounds :: Gen (StreamFrom blk, StreamTo blk)
genBounds = [(Int, Gen (StreamFrom blk, StreamTo blk))]
-> Gen (StreamFrom blk, StreamTo blk)
forall a. [(Int, Gen a)] -> Gen a
frequency
      [ (Int
1, Gen (StreamFrom blk, StreamTo blk)
genRandomBounds)
      , (if Bool
empty then Int
0 else Int
3, Gen (StreamFrom blk, StreamTo blk)
genExistingBounds)
      ]

    genRandomBounds :: Gen (StreamFrom blk, StreamTo blk)
    genRandomBounds :: Gen (StreamFrom blk, StreamTo blk)
genRandomBounds = (,)
      (StreamFrom blk -> StreamTo blk -> (StreamFrom blk, StreamTo blk))
-> Gen (StreamFrom blk)
-> Gen (StreamTo blk -> (StreamFrom blk, StreamTo blk))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> (do Either
  (RealPoint blk -> StreamFrom blk) (Point blk -> StreamFrom blk)
inEx <- Gen
  (Either
     (RealPoint blk -> StreamFrom blk) (Point blk -> StreamFrom blk))
genFromInEx
              case Either
  (RealPoint blk -> StreamFrom blk) (Point blk -> StreamFrom blk)
inEx of
                Left  RealPoint blk -> StreamFrom blk
inc -> RealPoint blk -> StreamFrom blk
inc (RealPoint blk -> StreamFrom blk)
-> Gen (RealPoint blk) -> Gen (StreamFrom blk)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Gen (RealPoint blk)
genRealPoint
                Right Point blk -> StreamFrom blk
exc -> Point blk -> StreamFrom blk
exc (Point blk -> StreamFrom blk)
-> Gen (Point blk) -> Gen (StreamFrom blk)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Gen (Point blk)
genPoint)
      Gen (StreamTo blk -> (StreamFrom blk, StreamTo blk))
-> Gen (StreamTo blk) -> Gen (StreamFrom blk, StreamTo blk)
forall a b. Gen (a -> b) -> Gen a -> Gen b
forall (f :: * -> *) a b. Applicative f => f (a -> b) -> f a -> f b
<*> (RealPoint blk -> StreamTo blk
forall blk. RealPoint blk -> StreamTo blk
StreamToInclusive (RealPoint blk -> StreamTo blk)
-> Gen (RealPoint blk) -> Gen (StreamTo blk)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Gen (RealPoint blk)
genRealPoint)

    genFromInEx :: Gen (Either (RealPoint blk -> StreamFrom blk)
                               (Point     blk -> StreamFrom blk))
    genFromInEx :: Gen
  (Either
     (RealPoint blk -> StreamFrom blk) (Point blk -> StreamFrom blk))
genFromInEx = [Either
   (RealPoint blk -> StreamFrom blk) (Point blk -> StreamFrom blk)]
-> Gen
     (Either
        (RealPoint blk -> StreamFrom blk) (Point blk -> StreamFrom blk))
forall a. HasCallStack => [a] -> Gen a
elements [(RealPoint blk -> StreamFrom blk)
-> Either
     (RealPoint blk -> StreamFrom blk) (Point blk -> StreamFrom blk)
forall a b. a -> Either a b
Left RealPoint blk -> StreamFrom blk
forall blk. RealPoint blk -> StreamFrom blk
StreamFromInclusive, (Point blk -> StreamFrom blk)
-> Either
     (RealPoint blk -> StreamFrom blk) (Point blk -> StreamFrom blk)
forall a b. b -> Either a b
Right Point blk -> StreamFrom blk
forall blk. Point blk -> StreamFrom blk
StreamFromExclusive]

    genFromInEx' :: Gen (RealPoint blk -> StreamFrom blk)
    genFromInEx' :: Gen (RealPoint blk -> StreamFrom blk)
genFromInEx' = ((RealPoint blk -> StreamFrom blk)
 -> RealPoint blk -> StreamFrom blk)
-> ((Point blk -> StreamFrom blk)
    -> RealPoint blk -> StreamFrom blk)
-> Either
     (RealPoint blk -> StreamFrom blk) (Point blk -> StreamFrom blk)
-> RealPoint blk
-> StreamFrom blk
forall a c b. (a -> c) -> (b -> c) -> Either a b -> c
either (RealPoint blk -> StreamFrom blk)
-> RealPoint blk -> StreamFrom blk
forall a. a -> a
id ((Point blk -> StreamFrom blk)
-> (RealPoint blk -> Point blk) -> RealPoint blk -> StreamFrom blk
forall b c a. (b -> c) -> (a -> b) -> a -> c
. RealPoint blk -> Point blk
forall blk. RealPoint blk -> Point blk
realPointToPoint) (Either
   (RealPoint blk -> StreamFrom blk) (Point blk -> StreamFrom blk)
 -> RealPoint blk -> StreamFrom blk)
-> Gen
     (Either
        (RealPoint blk -> StreamFrom blk) (Point blk -> StreamFrom blk))
-> Gen (RealPoint blk -> StreamFrom blk)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Gen
  (Either
     (RealPoint blk -> StreamFrom blk) (Point blk -> StreamFrom blk))
genFromInEx

    -- Generate bounds that correspond to existing blocks in the DB. Make sure
    -- that the start bound is older than the end bound.
    -- NOTE: this does not mean that these bounds are on the same chain.
    genExistingBounds :: Gen (StreamFrom blk, StreamTo blk)
    genExistingBounds :: Gen (StreamFrom blk, StreamTo blk)
genExistingBounds = do
      RealPoint blk
start <- [RealPoint blk] -> Gen (RealPoint blk)
forall a. HasCallStack => [a] -> Gen a
elements [RealPoint blk]
pointsInDB
      RealPoint blk
end   <- [RealPoint blk] -> Gen (RealPoint blk)
forall a. HasCallStack => [a] -> Gen a
elements [RealPoint blk]
pointsInDB Gen (RealPoint blk)
-> (RealPoint blk -> Bool) -> Gen (RealPoint blk)
forall a. Gen a -> (a -> Bool) -> Gen a
`suchThat` ((SlotNo -> SlotNo -> Bool
forall a. Ord a => a -> a -> Bool
>= RealPoint blk -> SlotNo
forall blk. RealPoint blk -> SlotNo
realPointSlot RealPoint blk
start) (SlotNo -> Bool)
-> (RealPoint blk -> SlotNo) -> RealPoint blk -> Bool
forall b c a. (b -> c) -> (a -> b) -> a -> c
.
                                               RealPoint blk -> SlotNo
forall blk. RealPoint blk -> SlotNo
realPointSlot)
      (,) (StreamFrom blk -> StreamTo blk -> (StreamFrom blk, StreamTo blk))
-> Gen (StreamFrom blk)
-> Gen (StreamTo blk -> (StreamFrom blk, StreamTo blk))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> (Gen (RealPoint blk -> StreamFrom blk)
genFromInEx' Gen (RealPoint blk -> StreamFrom blk)
-> Gen (RealPoint blk) -> Gen (StreamFrom blk)
forall a b. Gen (a -> b) -> Gen a -> Gen b
forall (f :: * -> *) a b. Applicative f => f (a -> b) -> f a -> f b
<*> RealPoint blk -> Gen (RealPoint blk)
forall a. a -> Gen a
forall (m :: * -> *) a. Monad m => a -> m a
return RealPoint blk
start)
          Gen (StreamTo blk -> (StreamFrom blk, StreamTo blk))
-> Gen (StreamTo blk) -> Gen (StreamFrom blk, StreamTo blk)
forall a b. Gen (a -> b) -> Gen a -> Gen b
forall (f :: * -> *) a b. Applicative f => f (a -> b) -> f a -> f b
<*> (StreamTo blk -> Gen (StreamTo blk)
forall a. a -> Gen a
forall (m :: * -> *) a. Monad m => a -> m a
return (StreamTo blk -> Gen (StreamTo blk))
-> StreamTo blk -> Gen (StreamTo blk)
forall a b. (a -> b) -> a -> b
$ RealPoint blk -> StreamTo blk
forall blk. RealPoint blk -> StreamTo blk
StreamToInclusive RealPoint blk
end)

    genIteratorClose :: Gen (Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic))
genIteratorClose = IterRef blk m Symbolic
-> Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic)
forall blk it flr. it -> Cmd blk it flr
IteratorClose (IterRef blk m Symbolic
 -> Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic))
-> Gen (IterRef blk m Symbolic)
-> Gen
     (Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> [IterRef blk m Symbolic] -> Gen (IterRef blk m Symbolic)
forall a. HasCallStack => [a] -> Gen a
elements [IterRef blk m Symbolic]
iterators
    genIteratorNext :: Gen (Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic))
genIteratorNext  = do
      IterRef blk m Symbolic
it <- [IterRef blk m Symbolic] -> Gen (IterRef blk m Symbolic)
forall a. HasCallStack => [a] -> Gen a
elements [IterRef blk m Symbolic]
iterators
      let blockCanBeGCed :: Bool
blockCanBeGCed = SecurityParam -> DBModel blk -> Int -> Bool
forall blk.
ModelSupportsBlock blk =>
SecurityParam -> Model blk -> Int -> Bool
Model.garbageCollectableIteratorNext
            SecurityParam
secParam DBModel blk
dbModel (KnownIters blk m Symbolic
knownIters KnownIters blk m Symbolic -> IterRef blk m Symbolic -> Int
forall k (r :: * -> *) a.
(Eq k, Eq1 r) =>
RefEnv k a r -> Reference k r -> a
RE.! IterRef blk m Symbolic
it)
      Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic)
-> Gen
     (Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic))
forall a. a -> Gen a
forall (m :: * -> *) a. Monad m => a -> m a
return (Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic)
 -> Gen
      (Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic)))
-> Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic)
-> Gen
     (Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic))
forall a b. (a -> b) -> a -> b
$ if Bool
blockCanBeGCed
        then IterRef blk m Symbolic
-> Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic)
forall blk it flr. it -> Cmd blk it flr
IteratorNextGCed IterRef blk m Symbolic
it
        else IterRef blk m Symbolic
-> Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic)
forall blk it flr. it -> Cmd blk it flr
IteratorNext     IterRef blk m Symbolic
it

    genNewFollower :: Gen (Cmd blk it flr)
genNewFollower = ChainType -> Cmd blk it flr
forall blk it flr. ChainType -> Cmd blk it flr
NewFollower (ChainType -> Cmd blk it flr)
-> Gen ChainType -> Gen (Cmd blk it flr)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> [ChainType] -> Gen ChainType
forall a. HasCallStack => [a] -> Gen a
elements [ChainType
SelectedChain, ChainType
TentativeChain]

    genFollowerInstruction :: Gen (Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic))
genFollowerInstruction = FollowerRef blk m Symbolic
-> Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic)
forall blk it flr. flr -> Cmd blk it flr
FollowerInstruction (FollowerRef blk m Symbolic
 -> Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic))
-> Gen (FollowerRef blk m Symbolic)
-> Gen
     (Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> [FollowerRef blk m Symbolic] -> Gen (FollowerRef blk m Symbolic)
forall a. HasCallStack => [a] -> Gen a
elements [FollowerRef blk m Symbolic]
followers
    genFollowerForward :: Gen (Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic))
genFollowerForward     = FollowerRef blk m Symbolic
-> [Point blk]
-> Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic)
forall blk it flr. flr -> [Point blk] -> Cmd blk it flr
FollowerForward     (FollowerRef blk m Symbolic
 -> [Point blk]
 -> Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic))
-> Gen (FollowerRef blk m Symbolic)
-> Gen
     ([Point blk]
      -> Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> [FollowerRef blk m Symbolic] -> Gen (FollowerRef blk m Symbolic)
forall a. HasCallStack => [a] -> Gen a
elements [FollowerRef blk m Symbolic]
followers
                                                 Gen
  ([Point blk]
   -> Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic))
-> Gen [Point blk]
-> Gen
     (Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic))
forall a b. Gen (a -> b) -> Gen a -> Gen b
forall (f :: * -> *) a b. Applicative f => f (a -> b) -> f a -> f b
<*> Gen [Point blk]
genFollowerForwardPoints

    genFollowerForwardPoints :: Gen [Point blk]
    genFollowerForwardPoints :: Gen [Point blk]
genFollowerForwardPoints = (Int, Int) -> Gen Int
forall a. Random a => (a, a) -> Gen a
choose (Int
1, Int
3) Gen Int -> (Int -> Gen [Point blk]) -> Gen [Point blk]
forall a b. Gen a -> (a -> Gen b) -> Gen b
forall (m :: * -> *) a b. Monad m => m a -> (a -> m b) -> m b
>>= \Int
n ->
      (Point blk -> Down (WithOrigin SlotNo))
-> [Point blk] -> [Point blk]
forall b a. Ord b => (a -> b) -> [a] -> [a]
sortOn (WithOrigin SlotNo -> Down (WithOrigin SlotNo)
forall a. a -> Down a
Down (WithOrigin SlotNo -> Down (WithOrigin SlotNo))
-> (Point blk -> WithOrigin SlotNo)
-> Point blk
-> Down (WithOrigin SlotNo)
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Point blk -> WithOrigin SlotNo
forall {k} (block :: k). Point block -> WithOrigin SlotNo
pointSlot) ([Point blk] -> [Point blk]) -> Gen [Point blk] -> Gen [Point blk]
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Int -> Gen (Point blk) -> Gen [Point blk]
forall (m :: * -> *) a. Applicative m => Int -> m a -> m [a]
replicateM Int
n Gen (Point blk)
genFollowerForwardPoint

    genFollowerForwardPoint :: Gen (Point blk)
    genFollowerForwardPoint :: Gen (Point blk)
genFollowerForwardPoint = Gen (Point blk)
genPoint

    genFollowerClose :: Gen (Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic))
genFollowerClose = FollowerRef blk m Symbolic
-> Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic)
forall blk it flr. flr -> Cmd blk it flr
FollowerClose (FollowerRef blk m Symbolic
 -> Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic))
-> Gen (FollowerRef blk m Symbolic)
-> Gen
     (Cmd blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> [FollowerRef blk m Symbolic] -> Gen (FollowerRef blk m Symbolic)
forall a. HasCallStack => [a] -> Gen a
elements [FollowerRef blk m Symbolic]
followers

chooseSlot :: SlotNo -> SlotNo -> Gen SlotNo
chooseSlot :: SlotNo -> SlotNo -> Gen SlotNo
chooseSlot (SlotNo Word64
start) (SlotNo Word64
end) = Word64 -> SlotNo
SlotNo (Word64 -> SlotNo) -> Gen Word64 -> Gen SlotNo
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> (Word64, Word64) -> Gen Word64
forall a. Random a => (a, a) -> Gen a
choose (Word64
start, Word64
end)

{-------------------------------------------------------------------------------
  Shrinking
-------------------------------------------------------------------------------}

-- | Shrinker
shrinker :: Model   blk m Symbolic
         ->  At Cmd blk m Symbolic
         -> [At Cmd blk m Symbolic]
shrinker :: forall blk (m :: * -> *).
Model blk m Symbolic
-> At Cmd blk m Symbolic -> [At Cmd blk m Symbolic]
shrinker Model blk m Symbolic
_ = [At Cmd blk m Symbolic]
-> At Cmd blk m Symbolic -> [At Cmd blk m Symbolic]
forall a b. a -> b -> a
const [] -- TODO: implement the shrinker. Command
                      -- 'PersistBlksThenGC' should be shrunk to
                      -- ['PersistBlks']

{-------------------------------------------------------------------------------
  The final state machine
-------------------------------------------------------------------------------}

-- | Mock a response
--
-- We do this by running the pure semantics and then generating mock
-- references for any new handles.
mock :: (TestConstraints blk, Typeable m)
     => Model            blk m Symbolic
     ->         At Cmd   blk m Symbolic
     -> GenSym (At Resp  blk m Symbolic)
mock :: forall blk (m :: * -> *).
(TestConstraints blk, Typeable m) =>
Model blk m Symbolic
-> At Cmd blk m Symbolic -> GenSym (At Resp blk m Symbolic)
mock Model blk m Symbolic
model At Cmd blk m Symbolic
cmd = Resp blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic)
-> At Resp blk m Symbolic
forall (t :: * -> * -> * -> *) blk (m :: * -> *) (r :: * -> *).
t blk (IterRef blk m r) (FollowerRef blk m r) -> At t blk m r
At (Resp blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic)
 -> At Resp blk m Symbolic)
-> GenSym
     (Resp blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic))
-> GenSym (At Resp blk m Symbolic)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> (Int -> GenSym (IterRef blk m Symbolic))
-> (Int -> GenSym (FollowerRef blk m Symbolic))
-> Resp blk Int Int
-> GenSym
     (Resp blk (IterRef blk m Symbolic) (FollowerRef blk m Symbolic))
forall (f :: * -> *) a c b d.
Applicative f =>
(a -> f c) -> (b -> f d) -> Resp blk a b -> f (Resp blk c d)
forall (t :: * -> * -> *) (f :: * -> *) a c b d.
(Bitraversable t, Applicative f) =>
(a -> f c) -> (b -> f d) -> t a b -> f (t c d)
bitraverse (GenSym (IterRef blk m Symbolic)
-> Int -> GenSym (IterRef blk m Symbolic)
forall a b. a -> b -> a
const GenSym (IterRef blk m Symbolic)
forall a. Typeable a => GenSym (Reference a Symbolic)
genSym) (GenSym (FollowerRef blk m Symbolic)
-> Int -> GenSym (FollowerRef blk m Symbolic)
forall a b. a -> b -> a
const GenSym (FollowerRef blk m Symbolic)
forall a. Typeable a => GenSym (Reference a Symbolic)
genSym) Resp blk Int Int
resp
  where
    (Resp blk Int Int
resp, DBModel blk
_dbm) = Model blk m Symbolic
-> At Cmd blk m Symbolic -> (Resp blk Int Int, DBModel blk)
forall blk (r :: * -> *) (m :: * -> *).
(TestConstraints blk, Eq1 r) =>
Model blk m r -> At Cmd blk m r -> (Resp blk Int Int, DBModel blk)
step Model blk m Symbolic
model At Cmd blk m Symbolic
cmd

precondition :: forall m blk. TestConstraints blk
             => Model blk m Symbolic -> At Cmd blk m Symbolic -> Logic
precondition :: forall (m :: * -> *) blk.
TestConstraints blk =>
Model blk m Symbolic -> At Cmd blk m Symbolic -> Logic
precondition Model {KnownFollowers blk m Symbolic
KnownIters blk m Symbolic
Opaque (TopLevelConfig blk)
DBModel blk
$sel:dbModel:Model :: forall blk (m :: * -> *) (r :: * -> *).
Model blk m r -> DBModel blk
$sel:knownIters:Model :: forall blk (m :: * -> *) (r :: * -> *).
Model blk m r -> KnownIters blk m r
$sel:knownFollowers:Model :: forall blk (m :: * -> *) (r :: * -> *).
Model blk m r -> KnownFollowers blk m r
$sel:modelConfig:Model :: forall blk (m :: * -> *) (r :: * -> *).
Model blk m r -> Opaque (TopLevelConfig blk)
dbModel :: DBModel blk
knownIters :: KnownIters blk m Symbolic
knownFollowers :: KnownFollowers blk m Symbolic
modelConfig :: Opaque (TopLevelConfig blk)
..} (At Cmd
  blk
  (Reference
     (Opaque
        (WithEq
           (Iterator
              m
              blk
              (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
               SlotNo, IsEBB, SizeInBytes, Word16,
               SomeSecond (NestedCtxt Header) blk))))
     Symbolic)
  (Reference
     (Opaque
        (WithEq
           (Follower
              m
              blk
              (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
               SlotNo, IsEBB, SizeInBytes, Word16,
               SomeSecond (NestedCtxt Header) blk))))
     Symbolic)
cmd) =
   [Reference
   (Opaque
      (WithEq
         (Iterator
            m
            blk
            (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
             SlotNo, IsEBB, SizeInBytes, Word16,
             SomeSecond (NestedCtxt Header) blk))))
   Symbolic]
-> (Reference
      (Opaque
         (WithEq
            (Iterator
               m
               blk
               (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
                SlotNo, IsEBB, SizeInBytes, Word16,
                SomeSecond (NestedCtxt Header) blk))))
      Symbolic
    -> Logic)
-> Logic
forall a. Show a => [a] -> (a -> Logic) -> Logic
forAll (Cmd
  blk
  (Reference
     (Opaque
        (WithEq
           (Iterator
              m
              blk
              (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
               SlotNo, IsEBB, SizeInBytes, Word16,
               SomeSecond (NestedCtxt Header) blk))))
     Symbolic)
  (Reference
     (Opaque
        (WithEq
           (Follower
              m
              blk
              (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
               SlotNo, IsEBB, SizeInBytes, Word16,
               SomeSecond (NestedCtxt Header) blk))))
     Symbolic)
-> [Reference
      (Opaque
         (WithEq
            (Iterator
               m
               blk
               (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
                SlotNo, IsEBB, SizeInBytes, Word16,
                SomeSecond (NestedCtxt Header) blk))))
      Symbolic]
forall (t :: * -> * -> *) it flr.
Bitraversable t =>
t it flr -> [it]
iters Cmd
  blk
  (Reference
     (Opaque
        (WithEq
           (Iterator
              m
              blk
              (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
               SlotNo, IsEBB, SizeInBytes, Word16,
               SomeSecond (NestedCtxt Header) blk))))
     Symbolic)
  (Reference
     (Opaque
        (WithEq
           (Follower
              m
              blk
              (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
               SlotNo, IsEBB, SizeInBytes, Word16,
               SomeSecond (NestedCtxt Header) blk))))
     Symbolic)
cmd) (Reference
  (Opaque
     (WithEq
        (Iterator
           m
           blk
           (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
            SlotNo, IsEBB, SizeInBytes, Word16,
            SomeSecond (NestedCtxt Header) blk))))
  Symbolic
-> [Reference
      (Opaque
         (WithEq
            (Iterator
               m
               blk
               (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
                SlotNo, IsEBB, SizeInBytes, Word16,
                SomeSecond (NestedCtxt Header) blk))))
      Symbolic]
-> Logic
forall (t :: * -> *) a.
(Foldable t, Eq a, Show a, Show (t a)) =>
a -> t a -> Logic
`member` KnownIters blk m Symbolic
-> [Reference
      (Opaque
         (WithEq
            (Iterator
               m
               blk
               (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
                SlotNo, IsEBB, SizeInBytes, Word16,
                SomeSecond (NestedCtxt Header) blk))))
      Symbolic]
forall k a (r :: * -> *). RefEnv k a r -> [Reference k r]
RE.keys KnownIters blk m Symbolic
knownIters)   Logic -> Logic -> Logic
.&&
   [Reference
   (Opaque
      (WithEq
         (Follower
            m
            blk
            (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
             SlotNo, IsEBB, SizeInBytes, Word16,
             SomeSecond (NestedCtxt Header) blk))))
   Symbolic]
-> (Reference
      (Opaque
         (WithEq
            (Follower
               m
               blk
               (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
                SlotNo, IsEBB, SizeInBytes, Word16,
                SomeSecond (NestedCtxt Header) blk))))
      Symbolic
    -> Logic)
-> Logic
forall a. Show a => [a] -> (a -> Logic) -> Logic
forAll (Cmd
  blk
  (Reference
     (Opaque
        (WithEq
           (Iterator
              m
              blk
              (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
               SlotNo, IsEBB, SizeInBytes, Word16,
               SomeSecond (NestedCtxt Header) blk))))
     Symbolic)
  (Reference
     (Opaque
        (WithEq
           (Follower
              m
              blk
              (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
               SlotNo, IsEBB, SizeInBytes, Word16,
               SomeSecond (NestedCtxt Header) blk))))
     Symbolic)
-> [Reference
      (Opaque
         (WithEq
            (Follower
               m
               blk
               (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
                SlotNo, IsEBB, SizeInBytes, Word16,
                SomeSecond (NestedCtxt Header) blk))))
      Symbolic]
forall (t :: * -> * -> *) it flr.
Bitraversable t =>
t it flr -> [flr]
flrs  Cmd
  blk
  (Reference
     (Opaque
        (WithEq
           (Iterator
              m
              blk
              (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
               SlotNo, IsEBB, SizeInBytes, Word16,
               SomeSecond (NestedCtxt Header) blk))))
     Symbolic)
  (Reference
     (Opaque
        (WithEq
           (Follower
              m
              blk
              (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
               SlotNo, IsEBB, SizeInBytes, Word16,
               SomeSecond (NestedCtxt Header) blk))))
     Symbolic)
cmd) (Reference
  (Opaque
     (WithEq
        (Follower
           m
           blk
           (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
            SlotNo, IsEBB, SizeInBytes, Word16,
            SomeSecond (NestedCtxt Header) blk))))
  Symbolic
-> [Reference
      (Opaque
         (WithEq
            (Follower
               m
               blk
               (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
                SlotNo, IsEBB, SizeInBytes, Word16,
                SomeSecond (NestedCtxt Header) blk))))
      Symbolic]
-> Logic
forall (t :: * -> *) a.
(Foldable t, Eq a, Show a, Show (t a)) =>
a -> t a -> Logic
`member` KnownFollowers blk m Symbolic
-> [Reference
      (Opaque
         (WithEq
            (Follower
               m
               blk
               (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
                SlotNo, IsEBB, SizeInBytes, Word16,
                SomeSecond (NestedCtxt Header) blk))))
      Symbolic]
forall k a (r :: * -> *). RefEnv k a r -> [Reference k r]
RE.keys KnownFollowers blk m Symbolic
knownFollowers) Logic -> Logic -> Logic
.&&
   case Cmd
  blk
  (Reference
     (Opaque
        (WithEq
           (Iterator
              m
              blk
              (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
               SlotNo, IsEBB, SizeInBytes, Word16,
               SomeSecond (NestedCtxt Header) blk))))
     Symbolic)
  (Reference
     (Opaque
        (WithEq
           (Follower
              m
              blk
              (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
               SlotNo, IsEBB, SizeInBytes, Word16,
               SomeSecond (NestedCtxt Header) blk))))
     Symbolic)
cmd of
     -- Even though we ensure this in the generator, shrinking might change
     -- it.
     GetBlockComponent     RealPoint blk
pt -> Logic -> Logic
Not (Logic -> Logic) -> Logic -> Logic
forall a b. (a -> b) -> a -> b
$ RealPoint blk -> Logic
garbageCollectable RealPoint blk
pt
     GetGCedBlockComponent RealPoint blk
pt -> RealPoint blk -> Logic
garbageCollectable RealPoint blk
pt
     IteratorNext     Reference
  (Opaque
     (WithEq
        (Iterator
           m
           blk
           (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
            SlotNo, IsEBB, SizeInBytes, Word16,
            SomeSecond (NestedCtxt Header) blk))))
  Symbolic
it      -> Logic -> Logic
Not (Logic -> Logic) -> Logic -> Logic
forall a b. (a -> b) -> a -> b
$ Reference
  (Opaque
     (WithEq
        (Iterator
           m
           blk
           (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
            SlotNo, IsEBB, SizeInBytes, Word16,
            SomeSecond (NestedCtxt Header) blk))))
  Symbolic
-> Logic
garbageCollectableIteratorNext Reference
  (Opaque
     (WithEq
        (Iterator
           m
           blk
           (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
            SlotNo, IsEBB, SizeInBytes, Word16,
            SomeSecond (NestedCtxt Header) blk))))
  Symbolic
it
     IteratorNextGCed Reference
  (Opaque
     (WithEq
        (Iterator
           m
           blk
           (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
            SlotNo, IsEBB, SizeInBytes, Word16,
            SomeSecond (NestedCtxt Header) blk))))
  Symbolic
it      -> Reference
  (Opaque
     (WithEq
        (Iterator
           m
           blk
           (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
            SlotNo, IsEBB, SizeInBytes, Word16,
            SomeSecond (NestedCtxt Header) blk))))
  Symbolic
-> Logic
garbageCollectableIteratorNext Reference
  (Opaque
     (WithEq
        (Iterator
           m
           blk
           (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
            SlotNo, IsEBB, SizeInBytes, Word16,
            SomeSecond (NestedCtxt Header) blk))))
  Symbolic
it

     -- TODO The real implementation allows streaming blocks from the
     -- VolatileDB that have no path to the current chain. The model
     -- implementation disallows this, as it only allows streaming from one of
     -- the possible forks, each starting at genesis. Temporarily only test
     -- with iterators that the model allows. So we only test a subset of the
     -- functionality, which does not include error paths.
     Stream StreamFrom blk
from StreamTo blk
to           -> StreamFrom blk -> StreamTo blk -> Logic
isValidIterator StreamFrom blk
from StreamTo blk
to
     Cmd
  blk
  (Reference
     (Opaque
        (WithEq
           (Iterator
              m
              blk
              (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
               SlotNo, IsEBB, SizeInBytes, Word16,
               SomeSecond (NestedCtxt Header) blk))))
     Symbolic)
  (Reference
     (Opaque
        (WithEq
           (Follower
              m
              blk
              (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
               SlotNo, IsEBB, SizeInBytes, Word16,
               SomeSecond (NestedCtxt Header) blk))))
     Symbolic)
Reopen                   -> Logic -> Logic
Not (Logic -> Logic) -> Logic -> Logic
forall a b. (a -> b) -> a -> b
$ Bool -> Logic
Boolean (DBModel blk -> Bool
forall blk. Model blk -> Bool
Model.isOpen DBModel blk
dbModel)
     -- To be in the future, @blockSlot blk@ must be greater than @slot@.
     --
     -- We do not allow multiple future blocks with the same block number, as
     -- the real implementation might have to switch between forks when they
     -- are no longer in the future, whereas the model will pick the right
     -- chain directly. This causes followers to go out of sync.
     -- https://github.com/IntersectMBO/ouroboros-network/issues/2234
     AddFutureBlock blk
blk SlotNo
s     -> SlotNo
s SlotNo -> SlotNo -> Logic
forall a. (Ord a, Show a) => a -> a -> Logic
.>= DBModel blk -> SlotNo
forall blk. Model blk -> SlotNo
Model.currentSlot DBModel blk
dbModel Logic -> Logic -> Logic
.&&
                                 blk -> SlotNo
forall b. HasHeader b => b -> SlotNo
blockSlot blk
blk SlotNo -> SlotNo -> Logic
forall a. (Ord a, Show a) => a -> a -> Logic
.> SlotNo
s Logic -> Logic -> Logic
.&&
                                 Logic -> Logic
Not (BlockNo -> Logic
futureBlockWithSameBlockNo (blk -> BlockNo
forall b. HasHeader b => b -> BlockNo
blockNo blk
blk))
     Cmd
  blk
  (Reference
     (Opaque
        (WithEq
           (Iterator
              m
              blk
              (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
               SlotNo, IsEBB, SizeInBytes, Word16,
               SomeSecond (NestedCtxt Header) blk))))
     Symbolic)
  (Reference
     (Opaque
        (WithEq
           (Follower
              m
              blk
              (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
               SlotNo, IsEBB, SizeInBytes, Word16,
               SomeSecond (NestedCtxt Header) blk))))
     Symbolic)
WipeVolatileDB           -> Bool -> Logic
Boolean (Bool -> Logic) -> Bool -> Logic
forall a b. (a -> b) -> a -> b
$ DBModel blk -> Bool
forall blk. Model blk -> Bool
Model.isOpen DBModel blk
dbModel
     -- We don't allow 'GetIsValid' for blocks from the future, since the real
     -- implementation might have validated them before detecting they're from
     -- the future, whereas the model won't include them in the output of
     -- 'Model.getIsValid' (which uses 'Model.validChains').
     GetIsValid RealPoint blk
pt            -> Bool -> Logic
Boolean (Bool -> Logic) -> Bool -> Logic
forall a b. (a -> b) -> a -> b
$
                                   HeaderHash blk -> Map (HeaderHash blk) blk -> Bool
forall k a. Ord k => k -> Map k a -> Bool
Map.notMember (RealPoint blk -> HeaderHash blk
forall blk. RealPoint blk -> HeaderHash blk
realPointHash RealPoint blk
pt)
                                                 (DBModel blk -> Map (HeaderHash blk) blk
forall blk. HasHeader blk => Model blk -> Map (HeaderHash blk) blk
Model.futureBlocks DBModel blk
dbModel)
     Cmd
  blk
  (Reference
     (Opaque
        (WithEq
           (Iterator
              m
              blk
              (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
               SlotNo, IsEBB, SizeInBytes, Word16,
               SomeSecond (NestedCtxt Header) blk))))
     Symbolic)
  (Reference
     (Opaque
        (WithEq
           (Follower
              m
              blk
              (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
               SlotNo, IsEBB, SizeInBytes, Word16,
               SomeSecond (NestedCtxt Header) blk))))
     Symbolic)
_                        -> Logic
Top
  where
    garbageCollectable :: RealPoint blk -> Logic
    garbageCollectable :: RealPoint blk -> Logic
garbageCollectable =
      Bool -> Logic
Boolean (Bool -> Logic)
-> (RealPoint blk -> Bool) -> RealPoint blk -> Logic
forall b c a. (b -> c) -> (a -> b) -> a -> c
. SecurityParam -> DBModel blk -> RealPoint blk -> Bool
forall blk.
HasHeader blk =>
SecurityParam -> Model blk -> RealPoint blk -> Bool
Model.garbageCollectablePoint SecurityParam
secParam DBModel blk
dbModel

    garbageCollectableIteratorNext :: IterRef blk m Symbolic -> Logic
    garbageCollectableIteratorNext :: Reference
  (Opaque
     (WithEq
        (Iterator
           m
           blk
           (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
            SlotNo, IsEBB, SizeInBytes, Word16,
            SomeSecond (NestedCtxt Header) blk))))
  Symbolic
-> Logic
garbageCollectableIteratorNext Reference
  (Opaque
     (WithEq
        (Iterator
           m
           blk
           (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
            SlotNo, IsEBB, SizeInBytes, Word16,
            SomeSecond (NestedCtxt Header) blk))))
  Symbolic
it = Bool -> Logic
Boolean (Bool -> Logic) -> Bool -> Logic
forall a b. (a -> b) -> a -> b
$
      SecurityParam -> DBModel blk -> Int -> Bool
forall blk.
ModelSupportsBlock blk =>
SecurityParam -> Model blk -> Int -> Bool
Model.garbageCollectableIteratorNext SecurityParam
secParam DBModel blk
dbModel (KnownIters blk m Symbolic
knownIters KnownIters blk m Symbolic
-> Reference
     (Opaque
        (WithEq
           (Iterator
              m
              blk
              (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
               SlotNo, IsEBB, SizeInBytes, Word16,
               SomeSecond (NestedCtxt Header) blk))))
     Symbolic
-> Int
forall k (r :: * -> *) a.
(Eq k, Eq1 r) =>
RefEnv k a r -> Reference k r -> a
RE.! Reference
  (Opaque
     (WithEq
        (Iterator
           m
           blk
           (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
            SlotNo, IsEBB, SizeInBytes, Word16,
            SomeSecond (NestedCtxt Header) blk))))
  Symbolic
it)

    futureBlockWithSameBlockNo :: BlockNo -> Logic
    futureBlockWithSameBlockNo :: BlockNo -> Logic
futureBlockWithSameBlockNo BlockNo
no =
        Logic -> Logic
Not (Logic -> Logic) -> Logic -> Logic
forall a b. (a -> b) -> a -> b
$ [blk] -> (blk -> Logic) -> Logic
forall a. Show a => [a] -> (a -> Logic) -> Logic
exists (Map (HeaderHash blk) blk -> [blk]
forall k a. Map k a -> [a]
Map.elems (DBModel blk -> Map (HeaderHash blk) blk
forall blk. HasHeader blk => Model blk -> Map (HeaderHash blk) blk
Model.futureBlocks DBModel blk
dbModel)) ((blk -> Logic) -> Logic) -> (blk -> Logic) -> Logic
forall a b. (a -> b) -> a -> b
$ \blk
futureBlock ->
          blk -> BlockNo
forall b. HasHeader b => b -> BlockNo
blockNo blk
futureBlock BlockNo -> BlockNo -> Logic
forall a. (Eq a, Show a) => a -> a -> Logic
.== BlockNo
no

    cfg :: TopLevelConfig blk
    cfg :: TopLevelConfig blk
cfg = Opaque (TopLevelConfig blk) -> TopLevelConfig blk
forall a. Opaque a -> a
unOpaque Opaque (TopLevelConfig blk)
modelConfig

    secParam :: SecurityParam
    secParam :: SecurityParam
secParam = TopLevelConfig blk -> SecurityParam
forall blk.
ConsensusProtocol (BlockProtocol blk) =>
TopLevelConfig blk -> SecurityParam
configSecurityParam TopLevelConfig blk
cfg

    -- TODO #871
    isValidIterator :: StreamFrom blk -> StreamTo blk -> Logic
    isValidIterator :: StreamFrom blk -> StreamTo blk -> Logic
isValidIterator StreamFrom blk
from StreamTo blk
to =
        case SecurityParam
-> StreamFrom blk
-> StreamTo blk
-> DBModel blk
-> Either (UnknownRange blk) [blk]
forall blk.
GetPrevHash blk =>
SecurityParam
-> StreamFrom blk
-> StreamTo blk
-> Model blk
-> Either (UnknownRange blk) [blk]
Model.between SecurityParam
secParam StreamFrom blk
from StreamTo blk
to DBModel blk
dbModel of
          Left  UnknownRange blk
_    -> Logic
Bot
          -- All blocks must be valid
          Right [blk]
blks -> [blk] -> (blk -> Logic) -> Logic
forall a. Show a => [a] -> (a -> Logic) -> Logic
forAll [blk]
blks ((blk -> Logic) -> Logic) -> (blk -> Logic) -> Logic
forall a b. (a -> b) -> a -> b
$ \blk
blk -> Bool -> Logic
Boolean (Bool -> Logic) -> Bool -> Logic
forall a b. (a -> b) -> a -> b
$
            HeaderHash blk
-> Map (HeaderHash blk) (InvalidBlockReason blk, SlotNo) -> Bool
forall k a. Ord k => k -> Map k a -> Bool
Map.notMember (blk -> HeaderHash blk
forall b. HasHeader b => b -> HeaderHash b
blockHash blk
blk) (Map (HeaderHash blk) (InvalidBlockReason blk, SlotNo) -> Bool)
-> Map (HeaderHash blk) (InvalidBlockReason blk, SlotNo) -> Bool
forall a b. (a -> b) -> a -> b
$ DBModel blk
-> Map (HeaderHash blk) (InvalidBlockReason blk, SlotNo)
forall blk. Model blk -> InvalidBlocks blk
Model.invalid DBModel blk
dbModel

transition :: (TestConstraints blk, Show1 r, Eq1 r)
           => Model   blk m r
           -> At Cmd  blk m r
           -> At Resp blk m r
           -> Model   blk m r
transition :: forall blk (r :: * -> *) (m :: * -> *).
(TestConstraints blk, Show1 r, Eq1 r) =>
Model blk m r -> At Cmd blk m r -> At Resp blk m r -> Model blk m r
transition Model blk m r
model At Cmd blk m r
cmd = Event blk m r -> Model blk m r
forall blk (m :: * -> *) (r :: * -> *).
Event blk m r -> Model blk m r
eventAfter (Event blk m r -> Model blk m r)
-> (At Resp blk m r -> Event blk m r)
-> At Resp blk m r
-> Model blk m r
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Model blk m r -> At Cmd blk m r -> At Resp blk m r -> Event blk m r
forall blk (r :: * -> *) (m :: * -> *).
(TestConstraints blk, Eq1 r, Show1 r) =>
Model blk m r -> At Cmd blk m r -> At Resp blk m r -> Event blk m r
lockstep Model blk m r
model At Cmd blk m r
cmd

invariant ::
     forall m blk. TestConstraints blk
  => TopLevelConfig blk
  -> Model blk m Concrete
  -> Logic
invariant :: forall (m :: * -> *) blk.
TestConstraints blk =>
TopLevelConfig blk -> Model blk m Concrete -> Logic
invariant TopLevelConfig blk
cfg Model {KnownFollowers blk m Concrete
KnownIters blk m Concrete
Opaque (TopLevelConfig blk)
DBModel blk
$sel:dbModel:Model :: forall blk (m :: * -> *) (r :: * -> *).
Model blk m r -> DBModel blk
$sel:knownIters:Model :: forall blk (m :: * -> *) (r :: * -> *).
Model blk m r -> KnownIters blk m r
$sel:knownFollowers:Model :: forall blk (m :: * -> *) (r :: * -> *).
Model blk m r -> KnownFollowers blk m r
$sel:modelConfig:Model :: forall blk (m :: * -> *) (r :: * -> *).
Model blk m r -> Opaque (TopLevelConfig blk)
dbModel :: DBModel blk
knownIters :: KnownIters blk m Concrete
knownFollowers :: KnownFollowers blk m Concrete
modelConfig :: Opaque (TopLevelConfig blk)
..} =
    [RealPoint blk] -> (RealPoint blk -> Logic) -> Logic
forall a. Show a => [a] -> (a -> Logic) -> Logic
forAll [RealPoint blk]
ptsOnCurChain (Bool -> Logic
Boolean (Bool -> Logic)
-> (RealPoint blk -> Bool) -> RealPoint blk -> Logic
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Bool -> Maybe Bool -> Bool
forall a. a -> Maybe a -> a
fromMaybe Bool
False (Maybe Bool -> Bool)
-> (RealPoint blk -> Maybe Bool) -> RealPoint blk -> Bool
forall b c a. (b -> c) -> (a -> b) -> a -> c
. DBModel blk -> RealPoint blk -> Maybe Bool
forall blk.
LedgerSupportsProtocol blk =>
Model blk -> RealPoint blk -> Maybe Bool
Model.getIsValid DBModel blk
dbModel)
  where
    -- | The blocks occurring on the current volatile chain fragment
    ptsOnCurChain :: [RealPoint blk]
    ptsOnCurChain :: [RealPoint blk]
ptsOnCurChain =
          (blk -> RealPoint blk) -> [blk] -> [RealPoint blk]
forall a b. (a -> b) -> [a] -> [b]
map blk -> RealPoint blk
forall blk. HasHeader blk => blk -> RealPoint blk
blockRealPoint
        ([blk] -> [RealPoint blk])
-> (DBModel blk -> [blk]) -> DBModel blk -> [RealPoint blk]
forall b c a. (b -> c) -> (a -> b) -> a -> c
. AnchoredSeq (WithOrigin SlotNo) (Anchor blk) blk -> [blk]
forall v a b. AnchoredSeq v a b -> [b]
AF.toOldestFirst
        (AnchoredSeq (WithOrigin SlotNo) (Anchor blk) blk -> [blk])
-> (DBModel blk
    -> AnchoredSeq (WithOrigin SlotNo) (Anchor blk) blk)
-> DBModel blk
-> [blk]
forall b c a. (b -> c) -> (a -> b) -> a -> c
. SecurityParam
-> (blk -> blk)
-> DBModel blk
-> AnchoredSeq (WithOrigin SlotNo) (Anchor blk) blk
forall a blk.
(HasHeader a, HasHeader blk) =>
SecurityParam -> (blk -> a) -> Model blk -> AnchoredFragment a
Model.volatileChain (TopLevelConfig blk -> SecurityParam
forall blk.
ConsensusProtocol (BlockProtocol blk) =>
TopLevelConfig blk -> SecurityParam
configSecurityParam TopLevelConfig blk
cfg) blk -> blk
forall a. a -> a
id
        (DBModel blk -> [RealPoint blk]) -> DBModel blk -> [RealPoint blk]
forall a b. (a -> b) -> a -> b
$ DBModel blk
dbModel

postcondition :: TestConstraints blk
              => Model   blk m Concrete
              -> At Cmd  blk m Concrete
              -> At Resp blk m Concrete
              -> Logic
postcondition :: forall blk (m :: * -> *).
TestConstraints blk =>
Model blk m Concrete
-> At Cmd blk m Concrete -> At Resp blk m Concrete -> Logic
postcondition Model blk m Concrete
model At Cmd blk m Concrete
cmd At Resp blk m Concrete
resp =
    (Model blk m Concrete -> At Resp blk m Concrete -> Resp blk Int Int
forall (t :: * -> * -> * -> *) blk (r :: * -> *) (m :: * -> *).
(Bifunctor (t blk), Eq1 r) =>
Model blk m r -> At t blk m r -> t blk Int Int
toMock (Event blk m Concrete -> Model blk m Concrete
forall blk (m :: * -> *) (r :: * -> *).
Event blk m r -> Model blk m r
eventAfter Event blk m Concrete
ev) At Resp blk m Concrete
resp Resp blk Int Int -> Resp blk Int Int -> Logic
forall a. (Eq a, Show a) => a -> a -> Logic
.== Event blk m Concrete -> Resp blk Int Int
forall blk (m :: * -> *) (r :: * -> *).
Event blk m r -> Resp blk Int Int
eventMockResp Event blk m Concrete
ev)
    Logic -> String -> Logic
.// String
"real response didn't match model response"
  where
    ev :: Event blk m Concrete
ev = Model blk m Concrete
-> At Cmd blk m Concrete
-> At Resp blk m Concrete
-> Event blk m Concrete
forall blk (r :: * -> *) (m :: * -> *).
(TestConstraints blk, Eq1 r, Show1 r) =>
Model blk m r -> At Cmd blk m r -> At Resp blk m r -> Event blk m r
lockstep Model blk m Concrete
model At Cmd blk m Concrete
cmd At Resp blk m Concrete
resp

semantics :: forall blk. TestConstraints blk
          => ChainDBEnv IO blk
          -> At Cmd blk IO Concrete
          -> IO (At Resp blk IO Concrete)
semantics :: forall blk.
TestConstraints blk =>
ChainDBEnv IO blk
-> At Cmd blk IO Concrete -> IO (At Resp blk IO Concrete)
semantics ChainDBEnv IO blk
env (At Cmd blk (IterRef blk IO Concrete) (FollowerRef blk IO Concrete)
cmd) =
    Resp blk (IterRef blk IO Concrete) (FollowerRef blk IO Concrete)
-> At Resp blk IO Concrete
forall (t :: * -> * -> * -> *) blk (m :: * -> *) (r :: * -> *).
t blk (IterRef blk m r) (FollowerRef blk m r) -> At t blk m r
At (Resp blk (IterRef blk IO Concrete) (FollowerRef blk IO Concrete)
 -> At Resp blk IO Concrete)
-> (Resp
      blk
      (WithEq
         (Iterator
            IO
            blk
            (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
             SlotNo, IsEBB, SizeInBytes, Word16,
             SomeSecond (NestedCtxt Header) blk)))
      (WithEq
         (Follower
            IO
            blk
            (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
             SlotNo, IsEBB, SizeInBytes, Word16,
             SomeSecond (NestedCtxt Header) blk)))
    -> Resp
         blk (IterRef blk IO Concrete) (FollowerRef blk IO Concrete))
-> Resp
     blk
     (WithEq
        (Iterator
           IO
           blk
           (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
            SlotNo, IsEBB, SizeInBytes, Word16,
            SomeSecond (NestedCtxt Header) blk)))
     (WithEq
        (Follower
           IO
           blk
           (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
            SlotNo, IsEBB, SizeInBytes, Word16,
            SomeSecond (NestedCtxt Header) blk)))
-> At Resp blk IO Concrete
forall b c a. (b -> c) -> (a -> b) -> a -> c
. ((WithEq
   (Iterator
      IO
      blk
      (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
       SlotNo, IsEBB, SizeInBytes, Word16,
       SomeSecond (NestedCtxt Header) blk))
 -> IterRef blk IO Concrete)
-> (WithEq
      (Follower
         IO
         blk
         (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
          SlotNo, IsEBB, SizeInBytes, Word16,
          SomeSecond (NestedCtxt Header) blk))
    -> FollowerRef blk IO Concrete)
-> Resp
     blk
     (WithEq
        (Iterator
           IO
           blk
           (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
            SlotNo, IsEBB, SizeInBytes, Word16,
            SomeSecond (NestedCtxt Header) blk)))
     (WithEq
        (Follower
           IO
           blk
           (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
            SlotNo, IsEBB, SizeInBytes, Word16,
            SomeSecond (NestedCtxt Header) blk)))
-> Resp blk (IterRef blk IO Concrete) (FollowerRef blk IO Concrete)
forall a b c d.
(a -> b) -> (c -> d) -> Resp blk a c -> Resp blk b d
forall (p :: * -> * -> *) a b c d.
Bifunctor p =>
(a -> b) -> (c -> d) -> p a c -> p b d
bimap (Opaque
  (WithEq
     (Iterator
        IO
        blk
        (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
         SlotNo, IsEBB, SizeInBytes, Word16,
         SomeSecond (NestedCtxt Header) blk)))
-> IterRef blk IO Concrete
forall a. Typeable a => a -> Reference a Concrete
QSM.reference (Opaque
   (WithEq
      (Iterator
         IO
         blk
         (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
          SlotNo, IsEBB, SizeInBytes, Word16,
          SomeSecond (NestedCtxt Header) blk)))
 -> IterRef blk IO Concrete)
-> (WithEq
      (Iterator
         IO
         blk
         (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
          SlotNo, IsEBB, SizeInBytes, Word16,
          SomeSecond (NestedCtxt Header) blk))
    -> Opaque
         (WithEq
            (Iterator
               IO
               blk
               (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
                SlotNo, IsEBB, SizeInBytes, Word16,
                SomeSecond (NestedCtxt Header) blk))))
-> WithEq
     (Iterator
        IO
        blk
        (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
         SlotNo, IsEBB, SizeInBytes, Word16,
         SomeSecond (NestedCtxt Header) blk))
-> IterRef blk IO Concrete
forall b c a. (b -> c) -> (a -> b) -> a -> c
. WithEq
  (Iterator
     IO
     blk
     (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
      SlotNo, IsEBB, SizeInBytes, Word16,
      SomeSecond (NestedCtxt Header) blk))
-> Opaque
     (WithEq
        (Iterator
           IO
           blk
           (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
            SlotNo, IsEBB, SizeInBytes, Word16,
            SomeSecond (NestedCtxt Header) blk)))
forall a. a -> Opaque a
QSM.Opaque) (Opaque
  (WithEq
     (Follower
        IO
        blk
        (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
         SlotNo, IsEBB, SizeInBytes, Word16,
         SomeSecond (NestedCtxt Header) blk)))
-> FollowerRef blk IO Concrete
forall a. Typeable a => a -> Reference a Concrete
QSM.reference (Opaque
   (WithEq
      (Follower
         IO
         blk
         (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
          SlotNo, IsEBB, SizeInBytes, Word16,
          SomeSecond (NestedCtxt Header) blk)))
 -> FollowerRef blk IO Concrete)
-> (WithEq
      (Follower
         IO
         blk
         (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
          SlotNo, IsEBB, SizeInBytes, Word16,
          SomeSecond (NestedCtxt Header) blk))
    -> Opaque
         (WithEq
            (Follower
               IO
               blk
               (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
                SlotNo, IsEBB, SizeInBytes, Word16,
                SomeSecond (NestedCtxt Header) blk))))
-> WithEq
     (Follower
        IO
        blk
        (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
         SlotNo, IsEBB, SizeInBytes, Word16,
         SomeSecond (NestedCtxt Header) blk))
-> FollowerRef blk IO Concrete
forall b c a. (b -> c) -> (a -> b) -> a -> c
. WithEq
  (Follower
     IO
     blk
     (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
      SlotNo, IsEBB, SizeInBytes, Word16,
      SomeSecond (NestedCtxt Header) blk))
-> Opaque
     (WithEq
        (Follower
           IO
           blk
           (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
            SlotNo, IsEBB, SizeInBytes, Word16,
            SomeSecond (NestedCtxt Header) blk)))
forall a. a -> Opaque a
QSM.Opaque)) (Resp
   blk
   (WithEq
      (Iterator
         IO
         blk
         (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
          SlotNo, IsEBB, SizeInBytes, Word16,
          SomeSecond (NestedCtxt Header) blk)))
   (WithEq
      (Follower
         IO
         blk
         (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
          SlotNo, IsEBB, SizeInBytes, Word16,
          SomeSecond (NestedCtxt Header) blk)))
 -> At Resp blk IO Concrete)
-> IO
     (Resp
        blk
        (WithEq
           (Iterator
              IO
              blk
              (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
               SlotNo, IsEBB, SizeInBytes, Word16,
               SomeSecond (NestedCtxt Header) blk)))
        (WithEq
           (Follower
              IO
              blk
              (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
               SlotNo, IsEBB, SizeInBytes, Word16,
               SomeSecond (NestedCtxt Header) blk))))
-> IO (At Resp blk IO Concrete)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$>
    ChainDBEnv IO blk
-> Cmd
     blk
     (WithEq
        (Iterator
           IO
           blk
           (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
            SlotNo, IsEBB, SizeInBytes, Word16,
            SomeSecond (NestedCtxt Header) blk)))
     (WithEq
        (Follower
           IO
           blk
           (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
            SlotNo, IsEBB, SizeInBytes, Word16,
            SomeSecond (NestedCtxt Header) blk)))
-> IO
     (Resp
        blk
        (WithEq
           (Iterator
              IO
              blk
              (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
               SlotNo, IsEBB, SizeInBytes, Word16,
               SomeSecond (NestedCtxt Header) blk)))
        (WithEq
           (Follower
              IO
              blk
              (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
               SlotNo, IsEBB, SizeInBytes, Word16,
               SomeSecond (NestedCtxt Header) blk))))
forall blk.
TestConstraints blk =>
ChainDBEnv IO blk
-> Cmd blk (TestIterator IO blk) (TestFollower IO blk)
-> IO (Resp blk (TestIterator IO blk) (TestFollower IO blk))
runIO ChainDBEnv IO blk
env ((IterRef blk IO Concrete
 -> WithEq
      (Iterator
         IO
         blk
         (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
          SlotNo, IsEBB, SizeInBytes, Word16,
          SomeSecond (NestedCtxt Header) blk)))
-> (FollowerRef blk IO Concrete
    -> WithEq
         (Follower
            IO
            blk
            (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
             SlotNo, IsEBB, SizeInBytes, Word16,
             SomeSecond (NestedCtxt Header) blk)))
-> Cmd blk (IterRef blk IO Concrete) (FollowerRef blk IO Concrete)
-> Cmd
     blk
     (WithEq
        (Iterator
           IO
           blk
           (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
            SlotNo, IsEBB, SizeInBytes, Word16,
            SomeSecond (NestedCtxt Header) blk)))
     (WithEq
        (Follower
           IO
           blk
           (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
            SlotNo, IsEBB, SizeInBytes, Word16,
            SomeSecond (NestedCtxt Header) blk)))
forall a b c d. (a -> b) -> (c -> d) -> Cmd blk a c -> Cmd blk b d
forall (p :: * -> * -> *) a b c d.
Bifunctor p =>
(a -> b) -> (c -> d) -> p a c -> p b d
bimap IterRef blk IO Concrete
-> WithEq
     (Iterator
        IO
        blk
        (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
         SlotNo, IsEBB, SizeInBytes, Word16,
         SomeSecond (NestedCtxt Header) blk))
forall a. Reference (Opaque a) Concrete -> a
QSM.opaque FollowerRef blk IO Concrete
-> WithEq
     (Follower
        IO
        blk
        (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
         SlotNo, IsEBB, SizeInBytes, Word16,
         SomeSecond (NestedCtxt Header) blk))
forall a. Reference (Opaque a) Concrete -> a
QSM.opaque Cmd blk (IterRef blk IO Concrete) (FollowerRef blk IO Concrete)
cmd)

-- | The state machine proper
sm :: TestConstraints blk
   => LoE ()
   -> ChainDBEnv IO blk
   -> BlockGen                  blk IO
   -> TopLevelConfig            blk
   -> ExtLedgerState            blk
   -> MaxClockSkew
   -> StateMachine (Model       blk IO)
                   (At Cmd      blk IO)
                                    IO
                   (At Resp     blk IO)
sm :: forall blk.
TestConstraints blk =>
LoE ()
-> ChainDBEnv IO blk
-> BlockGen blk IO
-> TopLevelConfig blk
-> ExtLedgerState blk
-> MaxClockSkew
-> StateMachine (Model blk IO) (At Cmd blk IO) IO (At Resp blk IO)
sm LoE ()
loe ChainDBEnv IO blk
env BlockGen blk IO
genBlock TopLevelConfig blk
cfg ExtLedgerState blk
initLedger MaxClockSkew
maxClockSkew = StateMachine
  { initModel :: forall (r :: * -> *). Model blk IO r
initModel     = LoE ()
-> TopLevelConfig blk
-> ExtLedgerState blk
-> MaxClockSkew
-> Model blk IO r
forall blk (m :: * -> *) (r :: * -> *).
HasHeader blk =>
LoE ()
-> TopLevelConfig blk
-> ExtLedgerState blk
-> MaxClockSkew
-> Model blk m r
initModel LoE ()
loe TopLevelConfig blk
cfg ExtLedgerState blk
initLedger MaxClockSkew
maxClockSkew
  , transition :: forall (r :: * -> *).
(Show1 r, Ord1 r) =>
Model blk IO r
-> At Cmd blk IO r -> At Resp blk IO r -> Model blk IO r
transition    = Model blk IO r
-> At Cmd blk IO r -> At Resp blk IO r -> Model blk IO r
forall blk (r :: * -> *) (m :: * -> *).
(TestConstraints blk, Show1 r, Eq1 r) =>
Model blk m r -> At Cmd blk m r -> At Resp blk m r -> Model blk m r
forall (r :: * -> *).
(Show1 r, Ord1 r) =>
Model blk IO r
-> At Cmd blk IO r -> At Resp blk IO r -> Model blk IO r
transition
  , precondition :: Model blk IO Symbolic -> At Cmd blk IO Symbolic -> Logic
precondition  = Model blk IO Symbolic -> At Cmd blk IO Symbolic -> Logic
forall (m :: * -> *) blk.
TestConstraints blk =>
Model blk m Symbolic -> At Cmd blk m Symbolic -> Logic
precondition
  , postcondition :: Model blk IO Concrete
-> At Cmd blk IO Concrete -> At Resp blk IO Concrete -> Logic
postcondition = Model blk IO Concrete
-> At Cmd blk IO Concrete -> At Resp blk IO Concrete -> Logic
forall blk (m :: * -> *).
TestConstraints blk =>
Model blk m Concrete
-> At Cmd blk m Concrete -> At Resp blk m Concrete -> Logic
postcondition
  , generator :: Model blk IO Symbolic -> Maybe (Gen (At Cmd blk IO Symbolic))
generator     = Gen (At Cmd blk IO Symbolic)
-> Maybe (Gen (At Cmd blk IO Symbolic))
forall a. a -> Maybe a
Just (Gen (At Cmd blk IO Symbolic)
 -> Maybe (Gen (At Cmd blk IO Symbolic)))
-> (Model blk IO Symbolic -> Gen (At Cmd blk IO Symbolic))
-> Model blk IO Symbolic
-> Maybe (Gen (At Cmd blk IO Symbolic))
forall b c a. (b -> c) -> (a -> b) -> a -> c
. LoE ()
-> BlockGen blk IO
-> Model blk IO Symbolic
-> Gen (At Cmd blk IO Symbolic)
forall blk (m :: * -> *).
TestConstraints blk =>
LoE ()
-> BlockGen blk m
-> Model blk m Symbolic
-> Gen (At Cmd blk m Symbolic)
generator LoE ()
loe BlockGen blk IO
genBlock
  , shrinker :: Model blk IO Symbolic
-> At Cmd blk IO Symbolic -> [At Cmd blk IO Symbolic]
shrinker      = Model blk IO Symbolic
-> At Cmd blk IO Symbolic -> [At Cmd blk IO Symbolic]
forall blk (m :: * -> *).
Model blk m Symbolic
-> At Cmd blk m Symbolic -> [At Cmd blk m Symbolic]
shrinker
  , semantics :: At Cmd blk IO Concrete -> IO (At Resp blk IO Concrete)
semantics     = ChainDBEnv IO blk
-> At Cmd blk IO Concrete -> IO (At Resp blk IO Concrete)
forall blk.
TestConstraints blk =>
ChainDBEnv IO blk
-> At Cmd blk IO Concrete -> IO (At Resp blk IO Concrete)
semantics ChainDBEnv IO blk
env
  , mock :: Model blk IO Symbolic
-> At Cmd blk IO Symbolic -> GenSym (At Resp blk IO Symbolic)
mock          = Model blk IO Symbolic
-> At Cmd blk IO Symbolic -> GenSym (At Resp blk IO Symbolic)
forall blk (m :: * -> *).
(TestConstraints blk, Typeable m) =>
Model blk m Symbolic
-> At Cmd blk m Symbolic -> GenSym (At Resp blk m Symbolic)
mock
  , invariant :: Maybe (Model blk IO Concrete -> Logic)
invariant     = (Model blk IO Concrete -> Logic)
-> Maybe (Model blk IO Concrete -> Logic)
forall a. a -> Maybe a
Just ((Model blk IO Concrete -> Logic)
 -> Maybe (Model blk IO Concrete -> Logic))
-> (Model blk IO Concrete -> Logic)
-> Maybe (Model blk IO Concrete -> Logic)
forall a b. (a -> b) -> a -> b
$ TopLevelConfig blk -> Model blk IO Concrete -> Logic
forall (m :: * -> *) blk.
TestConstraints blk =>
TopLevelConfig blk -> Model blk m Concrete -> Logic
invariant TopLevelConfig blk
cfg
  , cleanup :: Model blk IO Concrete -> IO ()
cleanup       = Model blk IO Concrete -> IO ()
forall (m :: * -> *) (model :: (* -> *) -> *).
Monad m =>
model Concrete -> m ()
noCleanup
  }

{-------------------------------------------------------------------------------
  Required instances

  The 'ToExpr' constraints come from "Data.TreeDiff".
-------------------------------------------------------------------------------}

instance CommandNames (At Cmd blk m) where
  cmdName :: forall (r :: * -> *). At Cmd blk m r -> String
cmdName (At Cmd
  blk
  (Reference
     (Opaque
        (WithEq
           (Iterator
              m
              blk
              (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
               SlotNo, IsEBB, SizeInBytes, Word16,
               SomeSecond (NestedCtxt Header) blk))))
     r)
  (Reference
     (Opaque
        (WithEq
           (Follower
              m
              blk
              (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
               SlotNo, IsEBB, SizeInBytes, Word16,
               SomeSecond (NestedCtxt Header) blk))))
     r)
cmd) = Cmd
  blk
  (Reference
     (Opaque
        (WithEq
           (Iterator
              m
              blk
              (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
               SlotNo, IsEBB, SizeInBytes, Word16,
               SomeSecond (NestedCtxt Header) blk))))
     r)
  (Reference
     (Opaque
        (WithEq
           (Follower
              m
              blk
              (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
               SlotNo, IsEBB, SizeInBytes, Word16,
               SomeSecond (NestedCtxt Header) blk))))
     r)
-> String
forall a. HasDatatypeInfo a => a -> String
constrName Cmd
  blk
  (Reference
     (Opaque
        (WithEq
           (Iterator
              m
              blk
              (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
               SlotNo, IsEBB, SizeInBytes, Word16,
               SomeSecond (NestedCtxt Header) blk))))
     r)
  (Reference
     (Opaque
        (WithEq
           (Follower
              m
              blk
              (blk, blk, Header blk, ByteString, ByteString, HeaderHash blk,
               SlotNo, IsEBB, SizeInBytes, Word16,
               SomeSecond (NestedCtxt Header) blk))))
     r)
cmd
  cmdNames :: forall (r :: * -> *). Proxy (At Cmd blk m r) -> Context
cmdNames (Proxy (At Cmd blk m r)
_ :: Proxy (At Cmd blk m r)) =
    Proxy (Cmd blk () ()) -> Context
forall a (proxy :: * -> *). HasDatatypeInfo a => proxy a -> Context
constrNames (forall t. Proxy t
forall {k} (t :: k). Proxy t
Proxy @(Cmd blk () ()))

deriving instance ( ToExpr blk
                  , ToExpr (HeaderHash  blk)
                  , ToExpr (ChainDepState (BlockProtocol blk))
                  , ToExpr (TipInfo blk)
                  , ToExpr (LedgerState blk)
                  , ToExpr (ExtValidationError blk)
                  )
                 => ToExpr (Model blk IO Concrete)
{-------------------------------------------------------------------------------
  Labelling
-------------------------------------------------------------------------------}

deriving instance SOP.Generic         (TraceEvent blk)
deriving instance SOP.HasDatatypeInfo (TraceEvent blk)
deriving instance SOP.Generic         (TraceAddBlockEvent blk)
deriving instance SOP.HasDatatypeInfo (TraceAddBlockEvent blk)
deriving instance SOP.Generic         (ChainDB.TraceFollowerEvent blk)
deriving instance SOP.HasDatatypeInfo (ChainDB.TraceFollowerEvent blk)
deriving instance SOP.Generic         (TraceCopyToImmutableDBEvent blk)
deriving instance SOP.HasDatatypeInfo (TraceCopyToImmutableDBEvent blk)
deriving instance SOP.Generic         (TraceValidationEvent blk)
deriving instance SOP.HasDatatypeInfo (TraceValidationEvent blk)
deriving instance SOP.Generic         (TraceInitChainSelEvent blk)
deriving instance SOP.HasDatatypeInfo (TraceInitChainSelEvent blk)
deriving instance SOP.Generic         (TraceOpenEvent blk)
deriving instance SOP.HasDatatypeInfo (TraceOpenEvent blk)
deriving instance SOP.Generic         (TraceGCEvent blk)
deriving instance SOP.HasDatatypeInfo (TraceGCEvent blk)
deriving instance SOP.Generic         (TraceIteratorEvent blk)
deriving instance SOP.HasDatatypeInfo (TraceIteratorEvent blk)
deriving instance SOP.Generic         (LedgerDB.TraceSnapshotEvent blk)
deriving instance SOP.HasDatatypeInfo (LedgerDB.TraceSnapshotEvent blk)
deriving instance SOP.Generic         (LedgerDB.TraceReplayEvent blk)
deriving instance SOP.HasDatatypeInfo (LedgerDB.TraceReplayEvent blk)
deriving instance SOP.Generic         (ImmutableDB.TraceEvent blk)
deriving instance SOP.HasDatatypeInfo (ImmutableDB.TraceEvent blk)
deriving instance SOP.Generic         (VolatileDB.TraceEvent blk)
deriving instance SOP.HasDatatypeInfo (VolatileDB.TraceEvent blk)

data Tag =
    TagGetIsValidJust
  | TagGetIsValidNothing
  | TagChainSelReprocessChangedSelection
  | TagChainSelReprocessKeptSelection
  deriving (Int -> Tag -> ShowS
[Tag] -> ShowS
Tag -> String
(Int -> Tag -> ShowS)
-> (Tag -> String) -> ([Tag] -> ShowS) -> Show Tag
forall a.
(Int -> a -> ShowS) -> (a -> String) -> ([a] -> ShowS) -> Show a
$cshowsPrec :: Int -> Tag -> ShowS
showsPrec :: Int -> Tag -> ShowS
$cshow :: Tag -> String
show :: Tag -> String
$cshowList :: [Tag] -> ShowS
showList :: [Tag] -> ShowS
Show, Tag -> Tag -> Bool
(Tag -> Tag -> Bool) -> (Tag -> Tag -> Bool) -> Eq Tag
forall a. (a -> a -> Bool) -> (a -> a -> Bool) -> Eq a
$c== :: Tag -> Tag -> Bool
== :: Tag -> Tag -> Bool
$c/= :: Tag -> Tag -> Bool
/= :: Tag -> Tag -> Bool
Eq)

-- | Predicate on events
type EventPred m = C.Predicate (Event Blk m Symbolic) Tag

-- | Convenience combinator for creating classifiers for successful commands
successful ::
     (    Event Blk m Symbolic
       -> Success Blk IteratorId FollowerId
       -> Either Tag (EventPred m)
     )
  -> EventPred m
successful :: forall (m :: * -> *).
(Event Blk m Symbolic
 -> Success Blk Int Int -> Either Tag (EventPred m))
-> EventPred m
successful Event Blk m Symbolic
-> Success Blk Int Int -> Either Tag (EventPred m)
f = (Event Blk m Symbolic -> Either Tag (EventPred m)) -> EventPred m
forall a b. (a -> Either b (Predicate a b)) -> Predicate a b
C.predicate ((Event Blk m Symbolic -> Either Tag (EventPred m)) -> EventPred m)
-> (Event Blk m Symbolic -> Either Tag (EventPred m))
-> EventPred m
forall a b. (a -> b) -> a -> b
$ \Event Blk m Symbolic
ev -> case Event Blk m Symbolic -> Resp Blk Int Int
forall blk (m :: * -> *) (r :: * -> *).
Event blk m r -> Resp blk Int Int
eventMockResp Event Blk m Symbolic
ev of
    Resp (Left  ChainDbError Blk
_ ) -> EventPred m -> Either Tag (EventPred m)
forall a b. b -> Either a b
Right (EventPred m -> Either Tag (EventPred m))
-> EventPred m -> Either Tag (EventPred m)
forall a b. (a -> b) -> a -> b
$ (Event Blk m Symbolic
 -> Success Blk Int Int -> Either Tag (EventPred m))
-> EventPred m
forall (m :: * -> *).
(Event Blk m Symbolic
 -> Success Blk Int Int -> Either Tag (EventPred m))
-> EventPred m
successful Event Blk m Symbolic
-> Success Blk Int Int -> Either Tag (EventPred m)
f
    Resp (Right Success Blk Int Int
ok) -> Event Blk m Symbolic
-> Success Blk Int Int -> Either Tag (EventPred m)
f Event Blk m Symbolic
ev Success Blk Int Int
ok

-- | Tag commands
--
-- Tagging works on symbolic events, so that we can tag without doing real IO.
tag :: forall m. [Event Blk m Symbolic] -> [Tag]
tag :: forall (m :: * -> *). [Event Blk m Symbolic] -> [Tag]
tag = [Predicate (Event Blk m Symbolic) Tag]
-> [Event Blk m Symbolic] -> [Tag]
forall a b. [Predicate a b] -> [a] -> [b]
C.classify [
      Predicate (Event Blk m Symbolic) Tag
tagGetIsValidJust
    , Predicate (Event Blk m Symbolic) Tag
tagGetIsValidNothing
    ,