{-# LANGUAGE BangPatterns #-}
{-# LANGUAGE BlockArguments #-}
{-# LANGUAGE DataKinds #-}
{-# LANGUAGE DeriveAnyClass #-}
{-# LANGUAGE DeriveGeneric #-}
{-# LANGUAGE DerivingVia #-}
{-# LANGUAGE FlexibleContexts #-}
{-# LANGUAGE GADTs #-}
{-# LANGUAGE GeneralizedNewtypeDeriving #-}
{-# LANGUAGE LambdaCase #-}
{-# LANGUAGE NamedFieldPuns #-}
{-# LANGUAGE RankNTypes #-}
{-# LANGUAGE ScopedTypeVariables #-}
{-# LANGUAGE StandaloneDeriving #-}
{-# LANGUAGE TypeApplications #-}
{-# LANGUAGE TypeFamilies #-}
{-# LANGUAGE TypeOperators #-}
{-# LANGUAGE UndecidableInstances #-}

-- | Types used throughout the implementation: handle, state, environment,
-- types, trace types, etc.
module Ouroboros.Consensus.Storage.ChainDB.Impl.Types
  ( ChainDbEnv (..)
  , ChainDbHandle (..)
  , ChainDbState (..)
  , ChainSelectionPromise (..)
  , SerialiseDiskConstraints
  , getEnv
  , getEnv1
  , getEnv2
  , getEnvSTM
  , getEnvSTM1

    -- * Exposed internals for testing purposes
  , Internal (..)
  , InternalChain (..)
  , checkInternalChain

    -- * Iterator-related
  , IteratorKey (..)

    -- * Follower-related
  , FollowerHandle (..)
  , FollowerKey (..)
  , FollowerRollState (..)
  , FollowerState (..)
  , followerRollStatePoint

    -- * Invalid blocks
  , InvalidBlockInfo (..)
  , InvalidBlocks

    -- * Blocks to add
  , BlockToAdd (..)
  , ChainSelMessage (..)
  , ChainSelQueue -- opaque
  , addBlockToAdd
  , addReprocessLoEBlocks
  , closeChainSelQueue
  , getChainSelMessage
  , getMaxSlotNoChainSelQueue
  , memberChainSelQueue
  , newChainSelQueue
  , processedChainSelMessage

    -- * Trace types
  , SelectionChangedInfo (..)
  , TraceAddBlockEvent (..)
  , TraceChainSelStarvationEvent (..)
  , TraceCopyToImmutableDBEvent (..)
  , TraceEvent (..)
  , TraceFollowerEvent (..)
  , TraceGCEvent (..)
  , TraceInitChainSelEvent (..)
  , TraceIteratorEvent (..)
  , TraceOpenEvent (..)
  , TracePipeliningEvent (..)
  , TraceValidationEvent (..)
  ) where

import Control.Monad (when)
import Control.ResourceRegistry
import Control.Tracer
import Data.Foldable (traverse_)
import Data.Map.Strict (Map)
import Data.Maybe (mapMaybe)
import Data.Maybe.Strict (StrictMaybe (..))
import Data.MultiSet (MultiSet)
import qualified Data.MultiSet as MultiSet
import Data.Typeable
import Data.Void (Void)
import Data.Word (Word64)
import GHC.Generics (Generic)
import NoThunks.Class (OnlyCheckWhnfNamed (..))
import Ouroboros.Consensus.Block
import Ouroboros.Consensus.Config
import Ouroboros.Consensus.Fragment.Diff (ChainDiff)
import Ouroboros.Consensus.HeaderValidation (HeaderWithTime (..))
import Ouroboros.Consensus.Ledger.Extended (ExtValidationError)
import Ouroboros.Consensus.Ledger.Inspect
import Ouroboros.Consensus.Ledger.SupportsProtocol
import Ouroboros.Consensus.Protocol.Abstract
import Ouroboros.Consensus.Storage.ChainDB.API
  ( AddBlockPromise (..)
  , AddBlockResult (..)
  , ChainDbError (..)
  , ChainSelectionPromise (..)
  , ChainType
  , LoE
  , StreamFrom
  , StreamTo
  , UnknownRange
  )
import Ouroboros.Consensus.Storage.ChainDB.API.Types.InvalidBlockPunishment
  ( InvalidBlockPunishment
  )
import Ouroboros.Consensus.Storage.ImmutableDB
  ( ImmutableDB
  , ImmutableDbSerialiseConstraints
  )
import qualified Ouroboros.Consensus.Storage.ImmutableDB as ImmutableDB
import Ouroboros.Consensus.Storage.LedgerDB
  ( LedgerDB'
  , LedgerDbSerialiseConstraints
  )
import qualified Ouroboros.Consensus.Storage.LedgerDB as LedgerDB
import Ouroboros.Consensus.Storage.Serialisation
import Ouroboros.Consensus.Storage.VolatileDB
  ( VolatileDB
  , VolatileDbSerialiseConstraints
  )
import qualified Ouroboros.Consensus.Storage.VolatileDB as VolatileDB
import Ouroboros.Consensus.Util (Fuse)
import Ouroboros.Consensus.Util.CallStack
import Ouroboros.Consensus.Util.Enclose (Enclosing, Enclosing' (..))
import Ouroboros.Consensus.Util.IOLike
import Ouroboros.Consensus.Util.Orphans ()
import Ouroboros.Consensus.Util.STM (WithFingerprint)
import Ouroboros.Network.AnchoredFragment (AnchoredFragment)
import qualified Ouroboros.Network.AnchoredFragment as AF
import Ouroboros.Network.Block (MaxSlotNo (..))
import Ouroboros.Network.BlockFetch.ConsensusInterface
  ( ChainSelStarvation (..)
  )

-- | All the serialisation related constraints needed by the ChainDB.
class
  ( ImmutableDbSerialiseConstraints blk
  , LedgerDbSerialiseConstraints blk
  , VolatileDbSerialiseConstraints blk
  , -- Needed for Follower
    EncodeDiskDep (NestedCtxt Header) blk
  ) =>
  SerialiseDiskConstraints blk

-- | A handle to the internal ChainDB state
newtype ChainDbHandle m blk = CDBHandle (StrictTVar m (ChainDbState m blk))

-- | Check if the ChainDB is open, if so, executing the given function on the
-- 'ChainDbEnv', otherwise, throw a 'CloseDBError'.
getEnv ::
  forall m blk r.
  (IOLike m, HasCallStack, HasHeader blk) =>
  ChainDbHandle m blk ->
  (ChainDbEnv m blk -> m r) ->
  m r
getEnv :: forall (m :: * -> *) blk r.
(IOLike m, HasCallStack, HasHeader blk) =>
ChainDbHandle m blk -> (ChainDbEnv m blk -> m r) -> m r
getEnv (CDBHandle StrictTVar m (ChainDbState m blk)
varState) ChainDbEnv m blk -> m r
f =
  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 (StrictTVar m (ChainDbState m blk) -> STM m (ChainDbState m blk)
forall (m :: * -> *) a. MonadSTM m => StrictTVar m a -> STM m a
readTVar StrictTVar m (ChainDbState m blk)
varState) m (ChainDbState m blk) -> (ChainDbState m blk -> m r) -> m r
forall a b. m a -> (a -> m b) -> m b
forall (m :: * -> *) a b. Monad m => m a -> (a -> m b) -> m b
>>= \case
    ChainDbOpen ChainDbEnv m blk
env -> ChainDbEnv m blk -> m r
f ChainDbEnv m blk
env
    ChainDbState m blk
ChainDbClosed -> ChainDbError blk -> m r
forall e a. Exception e => e -> m a
forall (m :: * -> *) e a. (MonadThrow m, Exception e) => e -> m a
throwIO (ChainDbError blk -> m r) -> ChainDbError blk -> m r
forall a b. (a -> b) -> a -> b
$ forall blk. PrettyCallStack -> ChainDbError blk
ClosedDBError @blk PrettyCallStack
HasCallStack => PrettyCallStack
prettyCallStack

-- | Variant 'of 'getEnv' for functions taking one argument.
getEnv1 ::
  (IOLike m, HasCallStack, HasHeader blk) =>
  ChainDbHandle m blk ->
  (ChainDbEnv m blk -> a -> m r) ->
  a ->
  m r
getEnv1 :: forall (m :: * -> *) blk a r.
(IOLike m, HasCallStack, HasHeader blk) =>
ChainDbHandle m blk -> (ChainDbEnv m blk -> a -> m r) -> a -> m r
getEnv1 ChainDbHandle m blk
h ChainDbEnv m blk -> a -> m r
f a
a = ChainDbHandle m blk -> (ChainDbEnv m blk -> m r) -> m r
forall (m :: * -> *) blk r.
(IOLike m, HasCallStack, HasHeader blk) =>
ChainDbHandle m blk -> (ChainDbEnv m blk -> m r) -> m r
getEnv ChainDbHandle m blk
h (\ChainDbEnv m blk
env -> ChainDbEnv m blk -> a -> m r
f ChainDbEnv m blk
env a
a)

-- | Variant 'of 'getEnv' for functions taking two arguments.
getEnv2 ::
  (IOLike m, HasCallStack, HasHeader blk) =>
  ChainDbHandle m blk ->
  (ChainDbEnv m blk -> a -> b -> m r) ->
  a ->
  b ->
  m r
getEnv2 :: forall (m :: * -> *) blk a b r.
(IOLike m, HasCallStack, HasHeader blk) =>
ChainDbHandle m blk
-> (ChainDbEnv m blk -> a -> b -> m r) -> a -> b -> m r
getEnv2 ChainDbHandle m blk
h ChainDbEnv m blk -> a -> b -> m r
f a
a b
b = ChainDbHandle m blk -> (ChainDbEnv m blk -> m r) -> m r
forall (m :: * -> *) blk r.
(IOLike m, HasCallStack, HasHeader blk) =>
ChainDbHandle m blk -> (ChainDbEnv m blk -> m r) -> m r
getEnv ChainDbHandle m blk
h (\ChainDbEnv m blk
env -> ChainDbEnv m blk -> a -> b -> m r
f ChainDbEnv m blk
env a
a b
b)

-- | Variant of 'getEnv' that works in 'STM'.
getEnvSTM ::
  forall m blk r.
  (IOLike m, HasCallStack, HasHeader blk) =>
  ChainDbHandle m blk ->
  (ChainDbEnv m blk -> STM m r) ->
  STM m r
getEnvSTM :: forall (m :: * -> *) blk r.
(IOLike m, HasCallStack, HasHeader blk) =>
ChainDbHandle m blk -> (ChainDbEnv m blk -> STM m r) -> STM m r
getEnvSTM (CDBHandle StrictTVar m (ChainDbState m blk)
varState) ChainDbEnv m blk -> STM m r
f =
  StrictTVar m (ChainDbState m blk) -> STM m (ChainDbState m blk)
forall (m :: * -> *) a. MonadSTM m => StrictTVar m a -> STM m a
readTVar StrictTVar m (ChainDbState m blk)
varState STM m (ChainDbState m blk)
-> (ChainDbState m blk -> STM m r) -> STM m r
forall a b. STM m a -> (a -> STM m b) -> STM m b
forall (m :: * -> *) a b. Monad m => m a -> (a -> m b) -> m b
>>= \case
    ChainDbOpen ChainDbEnv m blk
env -> ChainDbEnv m blk -> STM m r
f ChainDbEnv m blk
env
    ChainDbState m blk
ChainDbClosed -> ChainDbError blk -> STM m r
forall (m :: * -> *) e a.
(MonadSTM m, MonadThrow (STM m), Exception e) =>
e -> STM m a
throwSTM (ChainDbError blk -> STM m r) -> ChainDbError blk -> STM m r
forall a b. (a -> b) -> a -> b
$ forall blk. PrettyCallStack -> ChainDbError blk
ClosedDBError @blk PrettyCallStack
HasCallStack => PrettyCallStack
prettyCallStack

-- | Variant of 'getEnv1' that works in 'STM'.
getEnvSTM1 ::
  forall m blk a r.
  (IOLike m, HasCallStack, HasHeader blk) =>
  ChainDbHandle m blk ->
  (ChainDbEnv m blk -> a -> STM m r) ->
  a ->
  STM m r
getEnvSTM1 :: forall (m :: * -> *) blk a r.
(IOLike m, HasCallStack, HasHeader blk) =>
ChainDbHandle m blk
-> (ChainDbEnv m blk -> a -> STM m r) -> a -> STM m r
getEnvSTM1 (CDBHandle StrictTVar m (ChainDbState m blk)
varState) ChainDbEnv m blk -> a -> STM m r
f a
a =
  StrictTVar m (ChainDbState m blk) -> STM m (ChainDbState m blk)
forall (m :: * -> *) a. MonadSTM m => StrictTVar m a -> STM m a
readTVar StrictTVar m (ChainDbState m blk)
varState STM m (ChainDbState m blk)
-> (ChainDbState m blk -> STM m r) -> STM m r
forall a b. STM m a -> (a -> STM m b) -> STM m b
forall (m :: * -> *) a b. Monad m => m a -> (a -> m b) -> m b
>>= \case
    ChainDbOpen ChainDbEnv m blk
env -> ChainDbEnv m blk -> a -> STM m r
f ChainDbEnv m blk
env a
a
    ChainDbState m blk
ChainDbClosed -> ChainDbError blk -> STM m r
forall (m :: * -> *) e a.
(MonadSTM m, MonadThrow (STM m), Exception e) =>
e -> STM m a
throwSTM (ChainDbError blk -> STM m r) -> ChainDbError blk -> STM m r
forall a b. (a -> b) -> a -> b
$ forall blk. PrettyCallStack -> ChainDbError blk
ClosedDBError @blk PrettyCallStack
HasCallStack => PrettyCallStack
prettyCallStack

data ChainDbState m blk
  = ChainDbOpen !(ChainDbEnv m blk)
  | ChainDbClosed
  deriving ((forall x. ChainDbState m blk -> Rep (ChainDbState m blk) x)
-> (forall x. Rep (ChainDbState m blk) x -> ChainDbState m blk)
-> Generic (ChainDbState m blk)
forall x. Rep (ChainDbState m blk) x -> ChainDbState m blk
forall x. ChainDbState m blk -> Rep (ChainDbState m blk) x
forall a.
(forall x. a -> Rep a x) -> (forall x. Rep a x -> a) -> Generic a
forall (m :: * -> *) blk x.
Rep (ChainDbState m blk) x -> ChainDbState m blk
forall (m :: * -> *) blk x.
ChainDbState m blk -> Rep (ChainDbState m blk) x
$cfrom :: forall (m :: * -> *) blk x.
ChainDbState m blk -> Rep (ChainDbState m blk) x
from :: forall x. ChainDbState m blk -> Rep (ChainDbState m blk) x
$cto :: forall (m :: * -> *) blk x.
Rep (ChainDbState m blk) x -> ChainDbState m blk
to :: forall x. Rep (ChainDbState m blk) x -> ChainDbState m blk
Generic, 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.
(IOLike m, LedgerSupportsProtocol blk,
 BlockSupportsDiffusionPipelining blk) =>
Context -> ChainDbState m blk -> IO (Maybe ThunkInfo)
forall (m :: * -> *) blk.
(IOLike m, LedgerSupportsProtocol blk,
 BlockSupportsDiffusionPipelining blk) =>
Proxy (ChainDbState m blk) -> String
$cnoThunks :: forall (m :: * -> *) blk.
(IOLike m, LedgerSupportsProtocol blk,
 BlockSupportsDiffusionPipelining blk) =>
Context -> ChainDbState m blk -> IO (Maybe ThunkInfo)
noThunks :: Context -> ChainDbState m blk -> IO (Maybe ThunkInfo)
$cwNoThunks :: forall (m :: * -> *) blk.
(IOLike m, LedgerSupportsProtocol blk,
 BlockSupportsDiffusionPipelining blk) =>
Context -> ChainDbState m blk -> IO (Maybe ThunkInfo)
wNoThunks :: Context -> ChainDbState m blk -> IO (Maybe ThunkInfo)
$cshowTypeOf :: forall (m :: * -> *) blk.
(IOLike m, LedgerSupportsProtocol blk,
 BlockSupportsDiffusionPipelining blk) =>
Proxy (ChainDbState m blk) -> String
showTypeOf :: Proxy (ChainDbState m blk) -> String
NoThunks)

-- | The current chain, both without and with slot times
--
-- INVARIANT @'AF.mapAnchoredFragment' 'hwtHeader' . 'icWithTime' = 'icWithoutTime'@
--
-- The fragment with times is maintained separately --- but exactly in parallel
-- --- for performance reasons and modularity reasons, trading a few thousand
-- pointers to avoid extra allocation per use, more granular interfaces
-- (notably
-- 'Ouroboros.Network.BlockFetch.ConsensusInterface.BlockFetchConsensusInterface'),
-- etc.
data InternalChain blk = InternalChain
  { forall blk. InternalChain blk -> AnchoredFragment (Header blk)
icWithoutTime :: !(AnchoredFragment (Header blk))
  , forall blk.
InternalChain blk -> AnchoredFragment (HeaderWithTime blk)
icWithTime :: !(AnchoredFragment (HeaderWithTime blk))
  }
  deriving (forall x. InternalChain blk -> Rep (InternalChain blk) x)
-> (forall x. Rep (InternalChain blk) x -> InternalChain blk)
-> Generic (InternalChain blk)
forall x. Rep (InternalChain blk) x -> InternalChain blk
forall x. InternalChain blk -> Rep (InternalChain blk) x
forall a.
(forall x. a -> Rep a x) -> (forall x. Rep a x -> a) -> Generic a
forall blk x. Rep (InternalChain blk) x -> InternalChain blk
forall blk x. InternalChain blk -> Rep (InternalChain blk) x
$cfrom :: forall blk x. InternalChain blk -> Rep (InternalChain blk) x
from :: forall x. InternalChain blk -> Rep (InternalChain blk) x
$cto :: forall blk x. Rep (InternalChain blk) x -> InternalChain blk
to :: forall x. Rep (InternalChain blk) x -> InternalChain blk
Generic

deriving instance (HasHeader blk, NoThunks (Header blk)) => NoThunks (InternalChain blk)

checkInternalChain ::
  forall blk.
  (HasHeader blk, HasHeader (Header blk)) =>
  InternalChain blk ->
  Maybe String
checkInternalChain :: forall blk.
(HasHeader blk, HasHeader (Header blk)) =>
InternalChain blk -> Maybe String
checkInternalChain (InternalChain AnchoredFragment (Header blk)
cur AnchoredFragment (HeaderWithTime blk)
curWithTime) =
  if (Header blk -> Header blk)
-> AnchoredFragment (Header blk) -> (Point blk, [Point blk])
forall h.
(HeaderHash h ~ HeaderHash blk) =>
(h -> Header blk) -> AnchoredFragment h -> (Point blk, [Point blk])
cnv Header blk -> Header blk
forall a. a -> a
id AnchoredFragment (Header blk)
cur (Point blk, [Point blk]) -> (Point blk, [Point blk]) -> Bool
forall a. Eq a => a -> a -> Bool
== (HeaderWithTime blk -> Header blk)
-> AnchoredFragment (HeaderWithTime blk)
-> (Point blk, [Point blk])
forall h.
(HeaderHash h ~ HeaderHash blk) =>
(h -> Header blk) -> AnchoredFragment h -> (Point blk, [Point blk])
cnv HeaderWithTime blk -> Header blk
forall blk. HeaderWithTime blk -> Header blk
hwtHeader AnchoredFragment (HeaderWithTime blk)
curWithTime
    then Maybe String
forall a. Maybe a
Nothing
    else
      String -> Maybe String
forall a. a -> Maybe a
Just (String -> Maybe String) -> String -> Maybe String
forall a b. (a -> b) -> a -> b
$
        Context -> String
unlines
          [ String
"cdbChain and cdbChainWithTime were out of sync:"
          , (Point blk, [Point blk]) -> String
forall a. Show a => a -> String
show ((Header blk -> Header blk)
-> AnchoredFragment (Header blk) -> (Point blk, [Point blk])
forall h.
(HeaderHash h ~ HeaderHash blk) =>
(h -> Header blk) -> AnchoredFragment h -> (Point blk, [Point blk])
cnv Header blk -> Header blk
forall a. a -> a
id AnchoredFragment (Header blk)
cur)
          , (Point blk, [Point blk]) -> String
forall a. Show a => a -> String
show ((HeaderWithTime blk -> Header blk)
-> AnchoredFragment (HeaderWithTime blk)
-> (Point blk, [Point blk])
forall h.
(HeaderHash h ~ HeaderHash blk) =>
(h -> Header blk) -> AnchoredFragment h -> (Point blk, [Point blk])
cnv HeaderWithTime blk -> Header blk
forall blk. HeaderWithTime blk -> Header blk
hwtHeader AnchoredFragment (HeaderWithTime blk)
curWithTime)
          ]
 where
  cnv ::
    HeaderHash h ~ HeaderHash blk =>
    (h -> Header blk) -> AnchoredFragment h -> (Point blk, [Point blk])
  cnv :: forall h.
(HeaderHash h ~ HeaderHash blk) =>
(h -> Header blk) -> AnchoredFragment h -> (Point blk, [Point blk])
cnv h -> Header blk
f AnchoredFragment h
af =
    ( Point h -> Point blk
forall {k1} {k2} (b :: k1) (b' :: k2).
Coercible (HeaderHash b) (HeaderHash b') =>
Point b -> Point b'
castPoint (Point h -> Point blk) -> Point h -> Point blk
forall a b. (a -> b) -> a -> b
$ AnchoredFragment h -> Point h
forall block. AnchoredFragment block -> Point block
AF.anchorPoint AnchoredFragment h
af
    , (Header blk -> Point blk
forall blk. HasHeader (Header blk) => Header blk -> Point blk
headerPoint (Header blk -> Point blk) -> (h -> Header blk) -> h -> Point blk
forall b c a. (b -> c) -> (a -> b) -> a -> c
. h -> Header blk
f) (h -> Point blk) -> [h] -> [Point blk]
forall a b. (a -> b) -> [a] -> [b]
`map` AnchoredFragment h -> [h]
forall v a b. AnchoredSeq v a b -> [b]
AF.toNewestFirst AnchoredFragment h
af
    )

data ChainDbEnv m blk = CDB
  { forall (m :: * -> *) blk. ChainDbEnv m blk -> ImmutableDB m blk
cdbImmutableDB :: !(ImmutableDB m blk)
  , forall (m :: * -> *) blk. ChainDbEnv m blk -> VolatileDB m blk
cdbVolatileDB :: !(VolatileDB m blk)
  , forall (m :: * -> *) blk. ChainDbEnv m blk -> LedgerDB' m blk
cdbLedgerDB :: !(LedgerDB' m blk)
  , forall (m :: * -> *) blk.
ChainDbEnv m blk -> StrictTVar m (InternalChain blk)
cdbChain :: !(StrictTVar m (InternalChain blk))
  -- ^ Contains the current chain fragment.
  --
  -- INVARIANT: the anchor point of this fragment is the tip of the
  -- ImmutableDB. This implies that this fragment never contains any blocks
  -- that are stored in the immutable DB.
  --
  -- Note that this fragment might be shorter than @k@ headers when the
  -- whole chain is shorter than @k@ or in case of corruption of the
  -- VolatileDB.
  --
  -- Note that this fragment might also be /longer/ than @k@ headers,
  -- because the oldest blocks from the fragment might not yet have been
  -- copied from the VolatileDB to the ImmutableDB.
  --
  -- The anchor point of this chain should be the most recent \"immutable\"
  -- block according to the protocol, i.e., a block that cannot be rolled
  -- back.
  --
  -- Note that the \"immutable\" block isn't necessarily at the tip of the
  -- ImmutableDB, but could temporarily still be on the in-memory chain
  -- fragment. When the background thread that copies blocks to the
  -- ImmutableDB has caught up, the \"immutable\" block will be at the tip
  -- of the ImmutableDB again.
  --
  -- Note that the \"immutable\" block might be less than @k@ blocks from
  -- our tip in case the whole chain is shorter than @k@ or in case of
  -- corruption of the VolatileDB.
  --
  -- Note that the \"immutable\" block will /never/ be /more/ than @k@
  -- blocks back, as opposed to the anchor point of 'cdbChain'.
  , forall (m :: * -> *) blk.
ChainDbEnv m blk -> StrictTVar m (TentativeHeaderState blk)
cdbTentativeState :: !(StrictTVar m (TentativeHeaderState blk))
  , forall (m :: * -> *) blk.
ChainDbEnv m blk -> StrictTVar m (StrictMaybe (Header blk))
cdbTentativeHeader :: !(StrictTVar m (StrictMaybe (Header blk)))
  -- ^ The tentative header, for diffusion pipelining.
  --
  -- INVARIANT: It fits on top of the current chain, and its body is not known
  -- to be invalid, but might turn out to be.
  , forall (m :: * -> *) blk.
ChainDbEnv m blk -> StrictTVar m (Map IteratorKey (m ()))
cdbIterators :: !(StrictTVar m (Map IteratorKey (m ())))
  -- ^ The iterators.
  --
  -- This maps the 'IteratorKey's of each open 'Iterator' to a function
  -- that, when called, closes the iterator. This is used when closing the
  -- ChainDB: the open file handles used by iterators can be closed, and the
  -- iterators themselves are closed so that it is impossible to use an
  -- iterator after closing the ChainDB itself.
  , forall (m :: * -> *) blk.
ChainDbEnv m blk
-> StrictTVar m (Map FollowerKey (FollowerHandle m blk))
cdbFollowers :: !(StrictTVar m (Map FollowerKey (FollowerHandle m blk)))
  -- ^ The followers.
  --
  -- A follower is open iff its 'FollowerKey' is this 'Map'.
  --
  -- INVARIANT: the 'followerPoint' of each follower is 'withinFragmentBounds'
  -- of the current chain fragment (retrieved 'cdbGetCurrentChain', not by
  -- reading 'cdbChain' directly).
  , forall (m :: * -> *) blk. ChainDbEnv m blk -> TopLevelConfig blk
cdbTopLevelConfig :: !(TopLevelConfig blk)
  , forall (m :: * -> *) blk.
ChainDbEnv m blk
-> StrictTVar m (WithFingerprint (InvalidBlocks blk))
cdbInvalid :: !(StrictTVar m (WithFingerprint (InvalidBlocks blk)))
  -- ^ See the docstring of 'InvalidBlocks'.
  --
  -- The 'Fingerprint' changes every time a hash is added to the map, but
  -- not when hashes are garbage-collected from the map.
  , forall (m :: * -> *) blk.
ChainDbEnv m blk -> StrictTVar m IteratorKey
cdbNextIteratorKey :: !(StrictTVar m IteratorKey)
  , forall (m :: * -> *) blk.
ChainDbEnv m blk -> StrictTVar m FollowerKey
cdbNextFollowerKey :: !(StrictTVar m FollowerKey)
  , forall (m :: * -> *) blk. ChainDbEnv m blk -> Fuse m
cdbCopyFuse :: !(Fuse m)
  , forall (m :: * -> *) blk. ChainDbEnv m blk -> Fuse m
cdbChainSelFuse :: !(Fuse m)
  , forall (m :: * -> *) blk.
ChainDbEnv m blk -> Tracer m (TraceEvent blk)
cdbTracer :: !(Tracer m (TraceEvent blk))
  , forall (m :: * -> *) blk. ChainDbEnv m blk -> ResourceRegistry m
cdbRegistry :: !(ResourceRegistry m)
  , forall (m :: * -> *) blk. ChainDbEnv m blk -> DiffTime
cdbGcDelay :: !DiffTime
  -- ^ How long to wait between copying a block from the VolatileDB to
  -- ImmutableDB and garbage collecting it from the VolatileDB
  , forall (m :: * -> *) blk. ChainDbEnv m blk -> DiffTime
cdbGcInterval :: !DiffTime
  -- ^ Minimum time between two garbage collections. Is used to batch
  -- garbage collections.
  , forall (m :: * -> *) blk. ChainDbEnv m blk -> StrictTVar m (m ())
cdbKillBgThreads :: !(StrictTVar m (m ()))
  -- ^ A handle to kill the background threads.
  , forall (m :: * -> *) blk. ChainDbEnv m blk -> ChainSelQueue m blk
cdbChainSelQueue :: !(ChainSelQueue m blk)
  -- ^ Queue of blocks that still have to be added.
  , forall (m :: * -> *) blk.
ChainDbEnv m blk -> m (LoE (AnchoredFragment (HeaderWithTime blk)))
cdbLoE :: !(m (LoE (AnchoredFragment (HeaderWithTime blk))))
  -- ^ Configure the Limit on Eagerness. If this is 'LoEEnabled', it contains
  -- an action that returns the LoE fragment, which indicates the latest rollback
  -- point, i.e. we are not allowed to select a chain from which we could not
  -- switch back to a chain containing it. The fragment is usually anchored at
  -- a recent immutable tip; if it does not, it will conservatively be treated
  -- as the empty fragment anchored in the current immutable tip.
  , forall (m :: * -> *) blk.
ChainDbEnv m blk -> StrictTVar m ChainSelStarvation
cdbChainSelStarvation :: !(StrictTVar m ChainSelStarvation)
  -- ^ Information on the last starvation of ChainSel, whether ongoing or
  -- ended recently.
  }
  deriving (forall x. ChainDbEnv m blk -> Rep (ChainDbEnv m blk) x)
-> (forall x. Rep (ChainDbEnv m blk) x -> ChainDbEnv m blk)
-> Generic (ChainDbEnv m blk)
forall x. Rep (ChainDbEnv m blk) x -> ChainDbEnv m blk
forall x. ChainDbEnv m blk -> Rep (ChainDbEnv m blk) x
forall a.
(forall x. a -> Rep a x) -> (forall x. Rep a x -> a) -> Generic a
forall (m :: * -> *) blk x.
Rep (ChainDbEnv m blk) x -> ChainDbEnv m blk
forall (m :: * -> *) blk x.
ChainDbEnv m blk -> Rep (ChainDbEnv m blk) x
$cfrom :: forall (m :: * -> *) blk x.
ChainDbEnv m blk -> Rep (ChainDbEnv m blk) x
from :: forall x. ChainDbEnv m blk -> Rep (ChainDbEnv m blk) x
$cto :: forall (m :: * -> *) blk x.
Rep (ChainDbEnv m blk) x -> ChainDbEnv m blk
to :: forall x. Rep (ChainDbEnv m blk) x -> ChainDbEnv m blk
Generic

-- | We include @blk@ in 'showTypeOf' because it helps resolving type families
-- (but avoid including @m@ because we cannot impose @Typeable m@ as a
-- constraint and still have it work with the simulator)
instance
  (IOLike m, LedgerSupportsProtocol blk, BlockSupportsDiffusionPipelining blk) =>
  NoThunks (ChainDbEnv m blk)
  where
  showTypeOf :: Proxy (ChainDbEnv m blk) -> String
showTypeOf Proxy (ChainDbEnv m blk)
_ = String
"ChainDbEnv m " String -> String -> String
forall a. [a] -> [a] -> [a]
++ TypeRep -> String
forall a. Show a => a -> String
show (Proxy blk -> TypeRep
forall {k} (proxy :: k -> *) (a :: k).
Typeable a =>
proxy a -> TypeRep
typeRep (forall t. Proxy t
forall {k} (t :: k). Proxy t
Proxy @blk))

{-------------------------------------------------------------------------------
  Exposed internals for testing purposes
-------------------------------------------------------------------------------}

data Internal m blk = Internal
  { forall (m :: * -> *) blk. Internal m blk -> m (WithOrigin SlotNo)
intCopyToImmutableDB :: m (WithOrigin SlotNo)
  -- ^ Copy the blocks older than @k@ from to the VolatileDB to the
  -- ImmutableDB and update the in-memory chain fragment correspondingly.
  --
  -- The 'SlotNo' of the tip of the ImmutableDB after copying the blocks is
  -- returned. This can be used for a garbage collection on the VolatileDB.
  , forall (m :: * -> *) blk. Internal m blk -> SlotNo -> m ()
intGarbageCollect :: SlotNo -> m ()
  -- ^ Perform garbage collection for blocks <= the given 'SlotNo'.
  , forall (m :: * -> *) blk. Internal m blk -> m ()
intTryTakeSnapshot :: m ()
  -- ^ Write a new LedgerDB snapshot to disk and remove the oldest one(s).
  , forall (m :: * -> *) blk. Internal m blk -> m Void
intAddBlockRunner :: m Void
  -- ^ Start the loop that adds blocks to the ChainDB retrieved from the
  -- queue populated by 'ChainDB.addBlock'. Execute this loop in a separate
  -- thread.
  , forall (m :: * -> *) blk. Internal m blk -> StrictTVar m (m ())
intKillBgThreads :: StrictTVar m (m ())
  -- ^ A handle to kill the background threads.
  }

{-------------------------------------------------------------------------------
  Iterator-related
-------------------------------------------------------------------------------}

-- | We use this internally to track iterators in a map ('cdbIterators') in
-- the ChainDB state so that we can remove them from the map when the iterator
-- is closed.
--
-- We store them in the map so that the ChainDB can close all open iterators
-- when it is closed itself.
newtype IteratorKey = IteratorKey Word
  deriving stock Int -> IteratorKey -> String -> String
[IteratorKey] -> String -> String
IteratorKey -> String
(Int -> IteratorKey -> String -> String)
-> (IteratorKey -> String)
-> ([IteratorKey] -> String -> String)
-> Show IteratorKey
forall a.
(Int -> a -> String -> String)
-> (a -> String) -> ([a] -> String -> String) -> Show a
$cshowsPrec :: Int -> IteratorKey -> String -> String
showsPrec :: Int -> IteratorKey -> String -> String
$cshow :: IteratorKey -> String
show :: IteratorKey -> String
$cshowList :: [IteratorKey] -> String -> String
showList :: [IteratorKey] -> String -> String
Show
  deriving newtype (IteratorKey -> IteratorKey -> Bool
(IteratorKey -> IteratorKey -> Bool)
-> (IteratorKey -> IteratorKey -> Bool) -> Eq IteratorKey
forall a. (a -> a -> Bool) -> (a -> a -> Bool) -> Eq a
$c== :: IteratorKey -> IteratorKey -> Bool
== :: IteratorKey -> IteratorKey -> Bool
$c/= :: IteratorKey -> IteratorKey -> Bool
/= :: IteratorKey -> IteratorKey -> Bool
Eq, Eq IteratorKey
Eq IteratorKey =>
(IteratorKey -> IteratorKey -> Ordering)
-> (IteratorKey -> IteratorKey -> Bool)
-> (IteratorKey -> IteratorKey -> Bool)
-> (IteratorKey -> IteratorKey -> Bool)
-> (IteratorKey -> IteratorKey -> Bool)
-> (IteratorKey -> IteratorKey -> IteratorKey)
-> (IteratorKey -> IteratorKey -> IteratorKey)
-> Ord IteratorKey
IteratorKey -> IteratorKey -> Bool
IteratorKey -> IteratorKey -> Ordering
IteratorKey -> IteratorKey -> IteratorKey
forall a.
Eq a =>
(a -> a -> Ordering)
-> (a -> a -> Bool)
-> (a -> a -> Bool)
-> (a -> a -> Bool)
-> (a -> a -> Bool)
-> (a -> a -> a)
-> (a -> a -> a)
-> Ord a
$ccompare :: IteratorKey -> IteratorKey -> Ordering
compare :: IteratorKey -> IteratorKey -> Ordering
$c< :: IteratorKey -> IteratorKey -> Bool
< :: IteratorKey -> IteratorKey -> Bool
$c<= :: IteratorKey -> IteratorKey -> Bool
<= :: IteratorKey -> IteratorKey -> Bool
$c> :: IteratorKey -> IteratorKey -> Bool
> :: IteratorKey -> IteratorKey -> Bool
$c>= :: IteratorKey -> IteratorKey -> Bool
>= :: IteratorKey -> IteratorKey -> Bool
$cmax :: IteratorKey -> IteratorKey -> IteratorKey
max :: IteratorKey -> IteratorKey -> IteratorKey
$cmin :: IteratorKey -> IteratorKey -> IteratorKey
min :: IteratorKey -> IteratorKey -> IteratorKey
Ord, Int -> IteratorKey
IteratorKey -> Int
IteratorKey -> [IteratorKey]
IteratorKey -> IteratorKey
IteratorKey -> IteratorKey -> [IteratorKey]
IteratorKey -> IteratorKey -> IteratorKey -> [IteratorKey]
(IteratorKey -> IteratorKey)
-> (IteratorKey -> IteratorKey)
-> (Int -> IteratorKey)
-> (IteratorKey -> Int)
-> (IteratorKey -> [IteratorKey])
-> (IteratorKey -> IteratorKey -> [IteratorKey])
-> (IteratorKey -> IteratorKey -> [IteratorKey])
-> (IteratorKey -> IteratorKey -> IteratorKey -> [IteratorKey])
-> Enum IteratorKey
forall a.
(a -> a)
-> (a -> a)
-> (Int -> a)
-> (a -> Int)
-> (a -> [a])
-> (a -> a -> [a])
-> (a -> a -> [a])
-> (a -> a -> a -> [a])
-> Enum a
$csucc :: IteratorKey -> IteratorKey
succ :: IteratorKey -> IteratorKey
$cpred :: IteratorKey -> IteratorKey
pred :: IteratorKey -> IteratorKey
$ctoEnum :: Int -> IteratorKey
toEnum :: Int -> IteratorKey
$cfromEnum :: IteratorKey -> Int
fromEnum :: IteratorKey -> Int
$cenumFrom :: IteratorKey -> [IteratorKey]
enumFrom :: IteratorKey -> [IteratorKey]
$cenumFromThen :: IteratorKey -> IteratorKey -> [IteratorKey]
enumFromThen :: IteratorKey -> IteratorKey -> [IteratorKey]
$cenumFromTo :: IteratorKey -> IteratorKey -> [IteratorKey]
enumFromTo :: IteratorKey -> IteratorKey -> [IteratorKey]
$cenumFromThenTo :: IteratorKey -> IteratorKey -> IteratorKey -> [IteratorKey]
enumFromThenTo :: IteratorKey -> IteratorKey -> IteratorKey -> [IteratorKey]
Enum, Context -> IteratorKey -> IO (Maybe ThunkInfo)
Proxy IteratorKey -> String
(Context -> IteratorKey -> IO (Maybe ThunkInfo))
-> (Context -> IteratorKey -> IO (Maybe ThunkInfo))
-> (Proxy IteratorKey -> String)
-> NoThunks IteratorKey
forall a.
(Context -> a -> IO (Maybe ThunkInfo))
-> (Context -> a -> IO (Maybe ThunkInfo))
-> (Proxy a -> String)
-> NoThunks a
$cnoThunks :: Context -> IteratorKey -> IO (Maybe ThunkInfo)
noThunks :: Context -> IteratorKey -> IO (Maybe ThunkInfo)
$cwNoThunks :: Context -> IteratorKey -> IO (Maybe ThunkInfo)
wNoThunks :: Context -> IteratorKey -> IO (Maybe ThunkInfo)
$cshowTypeOf :: Proxy IteratorKey -> String
showTypeOf :: Proxy IteratorKey -> String
NoThunks)

{-------------------------------------------------------------------------------
  Follower-related
-------------------------------------------------------------------------------}

-- Note: these things are not in the Follower module, because 'TraceEvent'
-- depends on them, 'ChainDbEnv.cdbTracer' depends on 'TraceEvent', and most
-- modules depend on 'ChainDbEnv'. Also, 'ChainDbEnv.cdbFollowers' depends on
-- 'FollowerState'.

-- | We use this internally to track follower in a map ('cdbFollowers') in the
-- ChainDB state so that we can remove them from the map when the follower is
-- closed.
--
-- We store them in the map so that the ChainDB can close all open followers
-- when it is closed itself and to update the followers in case we switch to a
-- different chain.
newtype FollowerKey = FollowerKey Word
  deriving stock Int -> FollowerKey -> String -> String
[FollowerKey] -> String -> String
FollowerKey -> String
(Int -> FollowerKey -> String -> String)
-> (FollowerKey -> String)
-> ([FollowerKey] -> String -> String)
-> Show FollowerKey
forall a.
(Int -> a -> String -> String)
-> (a -> String) -> ([a] -> String -> String) -> Show a
$cshowsPrec :: Int -> FollowerKey -> String -> String
showsPrec :: Int -> FollowerKey -> String -> String
$cshow :: FollowerKey -> String
show :: FollowerKey -> String
$cshowList :: [FollowerKey] -> String -> String
showList :: [FollowerKey] -> String -> String
Show
  deriving newtype (FollowerKey -> FollowerKey -> Bool
(FollowerKey -> FollowerKey -> Bool)
-> (FollowerKey -> FollowerKey -> Bool) -> Eq FollowerKey
forall a. (a -> a -> Bool) -> (a -> a -> Bool) -> Eq a
$c== :: FollowerKey -> FollowerKey -> Bool
== :: FollowerKey -> FollowerKey -> Bool
$c/= :: FollowerKey -> FollowerKey -> Bool
/= :: FollowerKey -> FollowerKey -> Bool
Eq, Eq FollowerKey
Eq FollowerKey =>
(FollowerKey -> FollowerKey -> Ordering)
-> (FollowerKey -> FollowerKey -> Bool)
-> (FollowerKey -> FollowerKey -> Bool)
-> (FollowerKey -> FollowerKey -> Bool)
-> (FollowerKey -> FollowerKey -> Bool)
-> (FollowerKey -> FollowerKey -> FollowerKey)
-> (FollowerKey -> FollowerKey -> FollowerKey)
-> Ord FollowerKey
FollowerKey -> FollowerKey -> Bool
FollowerKey -> FollowerKey -> Ordering
FollowerKey -> FollowerKey -> FollowerKey
forall a.
Eq a =>
(a -> a -> Ordering)
-> (a -> a -> Bool)
-> (a -> a -> Bool)
-> (a -> a -> Bool)
-> (a -> a -> Bool)
-> (a -> a -> a)
-> (a -> a -> a)
-> Ord a
$ccompare :: FollowerKey -> FollowerKey -> Ordering
compare :: FollowerKey -> FollowerKey -> Ordering
$c< :: FollowerKey -> FollowerKey -> Bool
< :: FollowerKey -> FollowerKey -> Bool
$c<= :: FollowerKey -> FollowerKey -> Bool
<= :: FollowerKey -> FollowerKey -> Bool
$c> :: FollowerKey -> FollowerKey -> Bool
> :: FollowerKey -> FollowerKey -> Bool
$c>= :: FollowerKey -> FollowerKey -> Bool
>= :: FollowerKey -> FollowerKey -> Bool
$cmax :: FollowerKey -> FollowerKey -> FollowerKey
max :: FollowerKey -> FollowerKey -> FollowerKey
$cmin :: FollowerKey -> FollowerKey -> FollowerKey
min :: FollowerKey -> FollowerKey -> FollowerKey
Ord, Int -> FollowerKey
FollowerKey -> Int
FollowerKey -> [FollowerKey]
FollowerKey -> FollowerKey
FollowerKey -> FollowerKey -> [FollowerKey]
FollowerKey -> FollowerKey -> FollowerKey -> [FollowerKey]
(FollowerKey -> FollowerKey)
-> (FollowerKey -> FollowerKey)
-> (Int -> FollowerKey)
-> (FollowerKey -> Int)
-> (FollowerKey -> [FollowerKey])
-> (FollowerKey -> FollowerKey -> [FollowerKey])
-> (FollowerKey -> FollowerKey -> [FollowerKey])
-> (FollowerKey -> FollowerKey -> FollowerKey -> [FollowerKey])
-> Enum FollowerKey
forall a.
(a -> a)
-> (a -> a)
-> (Int -> a)
-> (a -> Int)
-> (a -> [a])
-> (a -> a -> [a])
-> (a -> a -> [a])
-> (a -> a -> a -> [a])
-> Enum a
$csucc :: FollowerKey -> FollowerKey
succ :: FollowerKey -> FollowerKey
$cpred :: FollowerKey -> FollowerKey
pred :: FollowerKey -> FollowerKey
$ctoEnum :: Int -> FollowerKey
toEnum :: Int -> FollowerKey
$cfromEnum :: FollowerKey -> Int
fromEnum :: FollowerKey -> Int
$cenumFrom :: FollowerKey -> [FollowerKey]
enumFrom :: FollowerKey -> [FollowerKey]
$cenumFromThen :: FollowerKey -> FollowerKey -> [FollowerKey]
enumFromThen :: FollowerKey -> FollowerKey -> [FollowerKey]
$cenumFromTo :: FollowerKey -> FollowerKey -> [FollowerKey]
enumFromTo :: FollowerKey -> FollowerKey -> [FollowerKey]
$cenumFromThenTo :: FollowerKey -> FollowerKey -> FollowerKey -> [FollowerKey]
enumFromThenTo :: FollowerKey -> FollowerKey -> FollowerKey -> [FollowerKey]
Enum, Context -> FollowerKey -> IO (Maybe ThunkInfo)
Proxy FollowerKey -> String
(Context -> FollowerKey -> IO (Maybe ThunkInfo))
-> (Context -> FollowerKey -> IO (Maybe ThunkInfo))
-> (Proxy FollowerKey -> String)
-> NoThunks FollowerKey
forall a.
(Context -> a -> IO (Maybe ThunkInfo))
-> (Context -> a -> IO (Maybe ThunkInfo))
-> (Proxy a -> String)
-> NoThunks a
$cnoThunks :: Context -> FollowerKey -> IO (Maybe ThunkInfo)
noThunks :: Context -> FollowerKey -> IO (Maybe ThunkInfo)
$cwNoThunks :: Context -> FollowerKey -> IO (Maybe ThunkInfo)
wNoThunks :: Context -> FollowerKey -> IO (Maybe ThunkInfo)
$cshowTypeOf :: Proxy FollowerKey -> String
showTypeOf :: Proxy FollowerKey -> String
NoThunks)

-- | Internal handle to a 'Follower' without an explicit @b@ (@blk@, @'Header'
-- blk@, etc.) parameter so 'Follower's with different' @b@s can be stored
-- together in 'cdbFollowers'.
data FollowerHandle m blk = FollowerHandle
  { forall (m :: * -> *) blk. FollowerHandle m blk -> ChainType
fhChainType :: ChainType
  -- ^ Whether we follow the tentative chain.
  , forall (m :: * -> *) blk.
FollowerHandle m blk -> AnchoredFragment (Header blk) -> STM m ()
fhSwitchFork :: AnchoredFragment (Header blk) -> STM m ()
  -- ^ When we have switched to a fork, all open 'Follower's must be notified.
  --
  -- Receives the suffix of the old chain anchored at the intersection with the
  -- new chain.
  , forall (m :: * -> *) blk. FollowerHandle m blk -> m ()
fhClose :: m ()
  -- ^ When closing the ChainDB, we must also close all open 'Follower's, as
  -- they might be holding on to resources.
  --
  -- Call 'fhClose' will release the resources used by the 'Follower'.
  --
  -- NOTE the 'Follower' is not removed from 'cdbFollowers'. (That is done by
  -- 'closeAllFollowers').
  }
  deriving Context -> FollowerHandle m blk -> IO (Maybe ThunkInfo)
Proxy (FollowerHandle m blk) -> String
(Context -> FollowerHandle m blk -> IO (Maybe ThunkInfo))
-> (Context -> FollowerHandle m blk -> IO (Maybe ThunkInfo))
-> (Proxy (FollowerHandle m blk) -> String)
-> NoThunks (FollowerHandle m blk)
forall a.
(Context -> a -> IO (Maybe ThunkInfo))
-> (Context -> a -> IO (Maybe ThunkInfo))
-> (Proxy a -> String)
-> NoThunks a
forall (m :: * -> *) blk.
Context -> FollowerHandle m blk -> IO (Maybe ThunkInfo)
forall (m :: * -> *) blk. Proxy (FollowerHandle m blk) -> String
$cnoThunks :: forall (m :: * -> *) blk.
Context -> FollowerHandle m blk -> IO (Maybe ThunkInfo)
noThunks :: Context -> FollowerHandle m blk -> IO (Maybe ThunkInfo)
$cwNoThunks :: forall (m :: * -> *) blk.
Context -> FollowerHandle m blk -> IO (Maybe ThunkInfo)
wNoThunks :: Context -> FollowerHandle m blk -> IO (Maybe ThunkInfo)
$cshowTypeOf :: forall (m :: * -> *) blk. Proxy (FollowerHandle m blk) -> String
showTypeOf :: Proxy (FollowerHandle m blk) -> String
NoThunks via OnlyCheckWhnfNamed "FollowerHandle" (FollowerHandle m blk)

-- | @b@ corresponds to the 'BlockComponent' that is being read.
data FollowerState m blk b
  = -- | The 'Follower' is in its initial state. Its 'FollowerRollState' is
    -- @'RollBackTo' 'genesisPoint'@.
    --
    -- This is equivalent to having a 'FollowerInImmutableDB' with the same
    -- 'FollowerRollState' and an iterator streaming after genesis. Opening such
    -- an iterator has a cost (index files will have to be read). However, in
    -- most cases, right after opening a Follower, the user of the Follower will try
    -- to move it forward, moving it from genesis to a more recent point on the
    -- chain. So we incur the cost of opening the iterator while not even using
    -- it.
    --
    -- Therefore, we have this extra initial state, that avoids this cost.
    -- When the user doesn't move the Follower forward, an iterator is opened.
    FollowerInit
  | -- | The 'Follower' is reading from the ImmutableDB.
    --
    -- Note that the iterator includes 'Point blk' in addition to @b@, as it
    -- is needed to keep track of where the iterator is.
    --
    -- INVARIANT: for all @FollowerInImmutableDB rollState immIt@: the predecessor
    -- of the next block streamed by @immIt@ must be the block identified by
    -- @followerRollStatePoint rollState@. In other words: the iterator is
    -- positioned /on/ @followerRollStatePoint rollState@.
    FollowerInImmutableDB
      !(FollowerRollState blk)
      !(ImmutableDB.Iterator m blk (Point blk, b))
  | -- | The 'Follower' is reading from the in-memory current chain fragment.
    FollowerInMem !(FollowerRollState blk)
  deriving ((forall x. FollowerState m blk b -> Rep (FollowerState m blk b) x)
-> (forall x.
    Rep (FollowerState m blk b) x -> FollowerState m blk b)
-> Generic (FollowerState m blk b)
forall x. Rep (FollowerState m blk b) x -> FollowerState m blk b
forall x. FollowerState m blk b -> Rep (FollowerState m blk b) x
forall a.
(forall x. a -> Rep a x) -> (forall x. Rep a x -> a) -> Generic a
forall (m :: * -> *) blk b x.
Rep (FollowerState m blk b) x -> FollowerState m blk b
forall (m :: * -> *) blk b x.
FollowerState m blk b -> Rep (FollowerState m blk b) x
$cfrom :: forall (m :: * -> *) blk b x.
FollowerState m blk b -> Rep (FollowerState m blk b) x
from :: forall x. FollowerState m blk b -> Rep (FollowerState m blk b) x
$cto :: forall (m :: * -> *) blk b x.
Rep (FollowerState m blk b) x -> FollowerState m blk b
to :: forall x. Rep (FollowerState m blk b) x -> FollowerState m blk b
Generic, Context -> FollowerState m blk b -> IO (Maybe ThunkInfo)
Proxy (FollowerState m blk b) -> String
(Context -> FollowerState m blk b -> IO (Maybe ThunkInfo))
-> (Context -> FollowerState m blk b -> IO (Maybe ThunkInfo))
-> (Proxy (FollowerState m blk b) -> String)
-> NoThunks (FollowerState m blk b)
forall a.
(Context -> a -> IO (Maybe ThunkInfo))
-> (Context -> a -> IO (Maybe ThunkInfo))
-> (Proxy a -> String)
-> NoThunks a
forall (m :: * -> *) blk b.
StandardHash blk =>
Context -> FollowerState m blk b -> IO (Maybe ThunkInfo)
forall (m :: * -> *) blk b.
StandardHash blk =>
Proxy (FollowerState m blk b) -> String
$cnoThunks :: forall (m :: * -> *) blk b.
StandardHash blk =>
Context -> FollowerState m blk b -> IO (Maybe ThunkInfo)
noThunks :: Context -> FollowerState m blk b -> IO (Maybe ThunkInfo)
$cwNoThunks :: forall (m :: * -> *) blk b.
StandardHash blk =>
Context -> FollowerState m blk b -> IO (Maybe ThunkInfo)
wNoThunks :: Context -> FollowerState m blk b -> IO (Maybe ThunkInfo)
$cshowTypeOf :: forall (m :: * -> *) blk b.
StandardHash blk =>
Proxy (FollowerState m blk b) -> String
showTypeOf :: Proxy (FollowerState m blk b) -> String
NoThunks)

-- | Similar to 'Ouroboros.Network.Mock.ProducerState.FollowerState'.
data FollowerRollState blk
  = -- | We don't know at which point the user is, but the next message we'll
    -- send is to roll back to this point.
    RollBackTo !(Point blk)
  | -- | We know that the follower is at this point and the next message we'll
    -- send is to roll forward to the point /after/ this point on our chain.
    RollForwardFrom !(Point blk)
  deriving (FollowerRollState blk -> FollowerRollState blk -> Bool
(FollowerRollState blk -> FollowerRollState blk -> Bool)
-> (FollowerRollState blk -> FollowerRollState blk -> Bool)
-> Eq (FollowerRollState blk)
forall blk.
StandardHash blk =>
FollowerRollState blk -> FollowerRollState blk -> Bool
forall a. (a -> a -> Bool) -> (a -> a -> Bool) -> Eq a
$c== :: forall blk.
StandardHash blk =>
FollowerRollState blk -> FollowerRollState blk -> Bool
== :: FollowerRollState blk -> FollowerRollState blk -> Bool
$c/= :: forall blk.
StandardHash blk =>
FollowerRollState blk -> FollowerRollState blk -> Bool
/= :: FollowerRollState blk -> FollowerRollState blk -> Bool
Eq, Int -> FollowerRollState blk -> String -> String
[FollowerRollState blk] -> String -> String
FollowerRollState blk -> String
(Int -> FollowerRollState blk -> String -> String)
-> (FollowerRollState blk -> String)
-> ([FollowerRollState blk] -> String -> String)
-> Show (FollowerRollState blk)
forall blk.
StandardHash blk =>
Int -> FollowerRollState blk -> String -> String
forall blk.
StandardHash blk =>
[FollowerRollState blk] -> String -> String
forall blk. StandardHash blk => FollowerRollState blk -> String
forall a.
(Int -> a -> String -> String)
-> (a -> String) -> ([a] -> String -> String) -> Show a
$cshowsPrec :: forall blk.
StandardHash blk =>
Int -> FollowerRollState blk -> String -> String
showsPrec :: Int -> FollowerRollState blk -> String -> String
$cshow :: forall blk. StandardHash blk => FollowerRollState blk -> String
show :: FollowerRollState blk -> String
$cshowList :: forall blk.
StandardHash blk =>
[FollowerRollState blk] -> String -> String
showList :: [FollowerRollState blk] -> String -> String
Show, (forall x. FollowerRollState blk -> Rep (FollowerRollState blk) x)
-> (forall x.
    Rep (FollowerRollState blk) x -> FollowerRollState blk)
-> Generic (FollowerRollState blk)
forall x. Rep (FollowerRollState blk) x -> FollowerRollState blk
forall x. FollowerRollState blk -> Rep (FollowerRollState blk) x
forall a.
(forall x. a -> Rep a x) -> (forall x. Rep a x -> a) -> Generic a
forall blk x.
Rep (FollowerRollState blk) x -> FollowerRollState blk
forall blk x.
FollowerRollState blk -> Rep (FollowerRollState blk) x
$cfrom :: forall blk x.
FollowerRollState blk -> Rep (FollowerRollState blk) x
from :: forall x. FollowerRollState blk -> Rep (FollowerRollState blk) x
$cto :: forall blk x.
Rep (FollowerRollState blk) x -> FollowerRollState blk
to :: forall x. Rep (FollowerRollState blk) x -> FollowerRollState blk
Generic, Context -> FollowerRollState blk -> IO (Maybe ThunkInfo)
Proxy (FollowerRollState blk) -> String
(Context -> FollowerRollState blk -> IO (Maybe ThunkInfo))
-> (Context -> FollowerRollState blk -> IO (Maybe ThunkInfo))
-> (Proxy (FollowerRollState blk) -> String)
-> NoThunks (FollowerRollState blk)
forall blk.
StandardHash blk =>
Context -> FollowerRollState blk -> IO (Maybe ThunkInfo)
forall blk.
StandardHash blk =>
Proxy (FollowerRollState blk) -> String
forall a.
(Context -> a -> IO (Maybe ThunkInfo))
-> (Context -> a -> IO (Maybe ThunkInfo))
-> (Proxy a -> String)
-> NoThunks a
$cnoThunks :: forall blk.
StandardHash blk =>
Context -> FollowerRollState blk -> IO (Maybe ThunkInfo)
noThunks :: Context -> FollowerRollState blk -> IO (Maybe ThunkInfo)
$cwNoThunks :: forall blk.
StandardHash blk =>
Context -> FollowerRollState blk -> IO (Maybe ThunkInfo)
wNoThunks :: Context -> FollowerRollState blk -> IO (Maybe ThunkInfo)
$cshowTypeOf :: forall blk.
StandardHash blk =>
Proxy (FollowerRollState blk) -> String
showTypeOf :: Proxy (FollowerRollState blk) -> String
NoThunks)

-- | Get the point the 'FollowerRollState' should roll back to or roll forward
-- from.
followerRollStatePoint :: FollowerRollState blk -> Point blk
followerRollStatePoint :: forall blk. FollowerRollState blk -> Point blk
followerRollStatePoint (RollBackTo Point blk
pt) = Point blk
pt
followerRollStatePoint (RollForwardFrom Point blk
pt) = Point blk
pt

{-------------------------------------------------------------------------------
  Invalid blocks
-------------------------------------------------------------------------------}

-- | Hashes corresponding to invalid blocks. This is used to ignore these
-- blocks during chain selection.
type InvalidBlocks blk = Map (HeaderHash blk) (InvalidBlockInfo blk)

-- | In addition to the reason why a block is invalid, the slot number of the
-- block is stored, so that whenever a garbage collection is performed on the
-- VolatileDB for some slot @s@, the hashes older or equal to @s@ can be
-- removed from this map.
data InvalidBlockInfo blk = InvalidBlockInfo
  { forall blk. InvalidBlockInfo blk -> ExtValidationError blk
invalidBlockReason :: !(ExtValidationError blk)
  , forall blk. InvalidBlockInfo blk -> SlotNo
invalidBlockSlotNo :: !SlotNo
  }
  deriving (InvalidBlockInfo blk -> InvalidBlockInfo blk -> Bool
(InvalidBlockInfo blk -> InvalidBlockInfo blk -> Bool)
-> (InvalidBlockInfo blk -> InvalidBlockInfo blk -> Bool)
-> Eq (InvalidBlockInfo blk)
forall blk.
LedgerSupportsProtocol blk =>
InvalidBlockInfo blk -> InvalidBlockInfo blk -> Bool
forall a. (a -> a -> Bool) -> (a -> a -> Bool) -> Eq a
$c== :: forall blk.
LedgerSupportsProtocol blk =>
InvalidBlockInfo blk -> InvalidBlockInfo blk -> Bool
== :: InvalidBlockInfo blk -> InvalidBlockInfo blk -> Bool
$c/= :: forall blk.
LedgerSupportsProtocol blk =>
InvalidBlockInfo blk -> InvalidBlockInfo blk -> Bool
/= :: InvalidBlockInfo blk -> InvalidBlockInfo blk -> Bool
Eq, Int -> InvalidBlockInfo blk -> String -> String
[InvalidBlockInfo blk] -> String -> String
InvalidBlockInfo blk -> String
(Int -> InvalidBlockInfo blk -> String -> String)
-> (InvalidBlockInfo blk -> String)
-> ([InvalidBlockInfo blk] -> String -> String)
-> Show (InvalidBlockInfo blk)
forall blk.
LedgerSupportsProtocol blk =>
Int -> InvalidBlockInfo blk -> String -> String
forall blk.
LedgerSupportsProtocol blk =>
[InvalidBlockInfo blk] -> String -> String
forall blk.
LedgerSupportsProtocol blk =>
InvalidBlockInfo blk -> String
forall a.
(Int -> a -> String -> String)
-> (a -> String) -> ([a] -> String -> String) -> Show a
$cshowsPrec :: forall blk.
LedgerSupportsProtocol blk =>
Int -> InvalidBlockInfo blk -> String -> String
showsPrec :: Int -> InvalidBlockInfo blk -> String -> String
$cshow :: forall blk.
LedgerSupportsProtocol blk =>
InvalidBlockInfo blk -> String
show :: InvalidBlockInfo blk -> String
$cshowList :: forall blk.
LedgerSupportsProtocol blk =>
[InvalidBlockInfo blk] -> String -> String
showList :: [InvalidBlockInfo blk] -> String -> String
Show, (forall x. InvalidBlockInfo blk -> Rep (InvalidBlockInfo blk) x)
-> (forall x. Rep (InvalidBlockInfo blk) x -> InvalidBlockInfo blk)
-> Generic (InvalidBlockInfo blk)
forall x. Rep (InvalidBlockInfo blk) x -> InvalidBlockInfo blk
forall x. InvalidBlockInfo blk -> Rep (InvalidBlockInfo blk) x
forall a.
(forall x. a -> Rep a x) -> (forall x. Rep a x -> a) -> Generic a
forall blk x. Rep (InvalidBlockInfo blk) x -> InvalidBlockInfo blk
forall blk x. InvalidBlockInfo blk -> Rep (InvalidBlockInfo blk) x
$cfrom :: forall blk x. InvalidBlockInfo blk -> Rep (InvalidBlockInfo blk) x
from :: forall x. InvalidBlockInfo blk -> Rep (InvalidBlockInfo blk) x
$cto :: forall blk x. Rep (InvalidBlockInfo blk) x -> InvalidBlockInfo blk
to :: forall x. Rep (InvalidBlockInfo blk) x -> InvalidBlockInfo blk
Generic, Context -> InvalidBlockInfo blk -> IO (Maybe ThunkInfo)
Proxy (InvalidBlockInfo blk) -> String
(Context -> InvalidBlockInfo blk -> IO (Maybe ThunkInfo))
-> (Context -> InvalidBlockInfo blk -> IO (Maybe ThunkInfo))
-> (Proxy (InvalidBlockInfo blk) -> String)
-> NoThunks (InvalidBlockInfo blk)
forall blk.
LedgerSupportsProtocol blk =>
Context -> InvalidBlockInfo blk -> IO (Maybe ThunkInfo)
forall blk.
LedgerSupportsProtocol blk =>
Proxy (InvalidBlockInfo blk) -> String
forall a.
(Context -> a -> IO (Maybe ThunkInfo))
-> (Context -> a -> IO (Maybe ThunkInfo))
-> (Proxy a -> String)
-> NoThunks a
$cnoThunks :: forall blk.
LedgerSupportsProtocol blk =>
Context -> InvalidBlockInfo blk -> IO (Maybe ThunkInfo)
noThunks :: Context -> InvalidBlockInfo blk -> IO (Maybe ThunkInfo)
$cwNoThunks :: forall blk.
LedgerSupportsProtocol blk =>
Context -> InvalidBlockInfo blk -> IO (Maybe ThunkInfo)
wNoThunks :: Context -> InvalidBlockInfo blk -> IO (Maybe ThunkInfo)
$cshowTypeOf :: forall blk.
LedgerSupportsProtocol blk =>
Proxy (InvalidBlockInfo blk) -> String
showTypeOf :: Proxy (InvalidBlockInfo blk) -> String
NoThunks)

{-------------------------------------------------------------------------------
  Blocks to add
-------------------------------------------------------------------------------}

-- | FIFO queue used to add blocks asynchronously to the ChainDB. Blocks are
-- read from this queue by a background thread, which processes the blocks
-- synchronously.
--
-- We also maintain a multiset of the points of all of the blocks in the queue,
-- plus potentially the one block for which chain selection is currently in
-- progress. It is used to account for queued blocks in eg 'getIsFetched' and
-- 'getMaxSlotNo'.
--
-- INVARIANT: Counted with multiplicity, @varChainSelPoints@ contains exactly
-- the same hashes or at most one additional hash compared to the hashes of
-- blocks in @varChainSelQueue@.
data ChainSelQueue m blk = ChainSelQueue
  { forall (m :: * -> *) blk.
ChainSelQueue m blk -> TBQueue m (ChainSelMessage m blk)
varChainSelQueue :: TBQueue m (ChainSelMessage m blk)
  , forall (m :: * -> *) blk.
ChainSelQueue m blk -> StrictTVar m (MultiSet (RealPoint blk))
varChainSelPoints :: StrictTVar m (MultiSet (RealPoint blk))
  }
  deriving Context -> ChainSelQueue m blk -> IO (Maybe ThunkInfo)
Proxy (ChainSelQueue m blk) -> String
(Context -> ChainSelQueue m blk -> IO (Maybe ThunkInfo))
-> (Context -> ChainSelQueue m blk -> IO (Maybe ThunkInfo))
-> (Proxy (ChainSelQueue m blk) -> String)
-> NoThunks (ChainSelQueue m blk)
forall a.
(Context -> a -> IO (Maybe ThunkInfo))
-> (Context -> a -> IO (Maybe ThunkInfo))
-> (Proxy a -> String)
-> NoThunks a
forall (m :: * -> *) blk.
Context -> ChainSelQueue m blk -> IO (Maybe ThunkInfo)
forall (m :: * -> *) blk. Proxy (ChainSelQueue m blk) -> String
$cnoThunks :: forall (m :: * -> *) blk.
Context -> ChainSelQueue m blk -> IO (Maybe ThunkInfo)
noThunks :: Context -> ChainSelQueue m blk -> IO (Maybe ThunkInfo)
$cwNoThunks :: forall (m :: * -> *) blk.
Context -> ChainSelQueue m blk -> IO (Maybe ThunkInfo)
wNoThunks :: Context -> ChainSelQueue m blk -> IO (Maybe ThunkInfo)
$cshowTypeOf :: forall (m :: * -> *) blk. Proxy (ChainSelQueue m blk) -> String
showTypeOf :: Proxy (ChainSelQueue m blk) -> String
NoThunks via OnlyCheckWhnfNamed "ChainSelQueue" (ChainSelQueue m blk)

-- | Entry in the 'ChainSelQueue' queue: a block together with the 'TMVar's used
-- to implement 'AddBlockPromise'.
data BlockToAdd m blk = BlockToAdd
  { forall (m :: * -> *) blk.
BlockToAdd m blk -> InvalidBlockPunishment m
blockPunish :: !(InvalidBlockPunishment m)
  -- ^ Executed immediately upon determining this block or one from its prefix
  -- is invalid.
  , forall (m :: * -> *) blk. BlockToAdd m blk -> blk
blockToAdd :: !blk
  , forall (m :: * -> *) blk. BlockToAdd m blk -> StrictTMVar m Bool
varBlockWrittenToDisk :: !(StrictTMVar m Bool)
  -- ^ Used for the 'blockWrittenToDisk' field of 'AddBlockPromise'.
  , forall (m :: * -> *) blk.
BlockToAdd m blk -> StrictTMVar m (AddBlockResult blk)
varBlockProcessed :: !(StrictTMVar m (AddBlockResult blk))
  -- ^ Used for the 'blockProcessed' field of 'AddBlockPromise'.
  }

-- | Different async tasks for triggering ChainSel
data ChainSelMessage m blk
  = -- | Add a new block
    ChainSelAddBlock !(BlockToAdd m blk)
  | -- | Reprocess blocks that have been postponed by the LoE.
    ChainSelReprocessLoEBlocks
      -- | Used for 'ChainSelectionPromise'.
      !(StrictTMVar m ())

-- | Create a new 'ChainSelQueue' with the given size.
newChainSelQueue :: (IOLike m, StandardHash blk, Typeable blk) => Word -> m (ChainSelQueue m blk)
newChainSelQueue :: forall (m :: * -> *) blk.
(IOLike m, StandardHash blk, Typeable blk) =>
Word -> m (ChainSelQueue m blk)
newChainSelQueue Word
chainSelQueueCapacity = do
  varChainSelQueue <- Natural -> m (TBQueue m (ChainSelMessage m blk))
forall a. Natural -> m (TBQueue m a)
forall (m :: * -> *) a. MonadSTM m => Natural -> m (TBQueue m a)
newTBQueueIO (Word -> Natural
forall a b. (Integral a, Num b) => a -> b
fromIntegral Word
chainSelQueueCapacity)
  varChainSelPoints <- newTVarIO MultiSet.empty
  pure
    ChainSelQueue
      { varChainSelQueue
      , varChainSelPoints
      }

-- | Add a block to the 'ChainSelQueue' queue. Can block when the queue is full.
addBlockToAdd ::
  (IOLike m, HasHeader blk) =>
  Tracer m (TraceAddBlockEvent blk) ->
  ChainSelQueue m blk ->
  InvalidBlockPunishment m ->
  blk ->
  m (AddBlockPromise m blk)
addBlockToAdd :: forall (m :: * -> *) blk.
(IOLike m, HasHeader blk) =>
Tracer m (TraceAddBlockEvent blk)
-> ChainSelQueue m blk
-> InvalidBlockPunishment m
-> blk
-> m (AddBlockPromise m blk)
addBlockToAdd Tracer m (TraceAddBlockEvent blk)
tracer (ChainSelQueue{TBQueue m (ChainSelMessage m blk)
varChainSelQueue :: forall (m :: * -> *) blk.
ChainSelQueue m blk -> TBQueue m (ChainSelMessage m blk)
varChainSelQueue :: TBQueue m (ChainSelMessage m blk)
varChainSelQueue, StrictTVar m (MultiSet (RealPoint blk))
varChainSelPoints :: forall (m :: * -> *) blk.
ChainSelQueue m blk -> StrictTVar m (MultiSet (RealPoint blk))
varChainSelPoints :: StrictTVar m (MultiSet (RealPoint blk))
varChainSelPoints}) InvalidBlockPunishment m
punish blk
blk = do
  varBlockWrittenToDisk <- m (StrictTMVar m Bool)
forall (m :: * -> *) a. MonadSTM m => m (StrictTMVar m a)
newEmptyTMVarIO
  varBlockProcessed <- newEmptyTMVarIO
  let !toAdd =
        BlockToAdd
          { blockPunish :: InvalidBlockPunishment m
blockPunish = InvalidBlockPunishment m
punish
          , blockToAdd :: blk
blockToAdd = blk
blk
          , StrictTMVar m Bool
varBlockWrittenToDisk :: StrictTMVar m Bool
varBlockWrittenToDisk :: StrictTMVar m Bool
varBlockWrittenToDisk
          , StrictTMVar m (AddBlockResult blk)
varBlockProcessed :: StrictTMVar m (AddBlockResult blk)
varBlockProcessed :: StrictTMVar m (AddBlockResult blk)
varBlockProcessed
          }
      pt = blk -> RealPoint blk
forall blk. HasHeader blk => blk -> RealPoint blk
blockRealPoint blk
blk
  traceWith tracer $ AddedBlockToQueue pt RisingEdge
  queueSize <- atomically $ do
    writeTBQueue varChainSelQueue (ChainSelAddBlock toAdd)
    modifyTVar varChainSelPoints $ MultiSet.insert pt
    lengthTBQueue varChainSelQueue
  traceWith tracer $
    AddedBlockToQueue (blockRealPoint blk) (FallingEdgeWith (fromIntegral queueSize))
  return
    AddBlockPromise
      { blockWrittenToDisk = readTMVar varBlockWrittenToDisk
      , blockProcessed = readTMVar varBlockProcessed
      }

-- | Try to add blocks again that were postponed due to the LoE.
addReprocessLoEBlocks ::
  IOLike m =>
  Tracer m (TraceAddBlockEvent blk) ->
  ChainSelQueue m blk ->
  m (ChainSelectionPromise m)
addReprocessLoEBlocks :: forall (m :: * -> *) blk.
IOLike m =>
Tracer m (TraceAddBlockEvent blk)
-> ChainSelQueue m blk -> m (ChainSelectionPromise m)
addReprocessLoEBlocks Tracer m (TraceAddBlockEvent blk)
tracer ChainSelQueue{TBQueue m (ChainSelMessage m blk)
varChainSelQueue :: forall (m :: * -> *) blk.
ChainSelQueue m blk -> TBQueue m (ChainSelMessage m blk)
varChainSelQueue :: TBQueue m (ChainSelMessage m blk)
varChainSelQueue} = do
  varProcessed <- m (StrictTMVar m ())
forall (m :: * -> *) a. MonadSTM m => m (StrictTMVar m a)
newEmptyTMVarIO
  let waitUntilRan = 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 () -> STM m ()
forall (m :: * -> *) a. MonadSTM m => StrictTMVar m a -> STM m a
readTMVar StrictTMVar m ()
varProcessed
  traceWith tracer $ AddedReprocessLoEBlocksToQueue
  atomically $
    writeTBQueue varChainSelQueue $
      ChainSelReprocessLoEBlocks varProcessed
  return $ ChainSelectionPromise waitUntilRan

-- | Get the oldest message from the 'ChainSelQueue' queue. Can block when the
-- queue is empty; in that case, reports the starvation (and its end) via the
-- given tracer.
getChainSelMessage ::
  forall m blk.
  (HasHeader blk, IOLike m) =>
  Tracer m (TraceChainSelStarvationEvent blk) ->
  StrictTVar m ChainSelStarvation ->
  ChainSelQueue m blk ->
  m (ChainSelMessage m blk)
getChainSelMessage :: forall (m :: * -> *) blk.
(HasHeader blk, IOLike m) =>
Tracer m (TraceChainSelStarvationEvent blk)
-> StrictTVar m ChainSelStarvation
-> ChainSelQueue m blk
-> m (ChainSelMessage m blk)
getChainSelMessage Tracer m (TraceChainSelStarvationEvent blk)
starvationTracer StrictTVar m ChainSelStarvation
starvationVar ChainSelQueue m blk
chainSelQueue =
  STM m (Maybe (ChainSelMessage m blk))
-> m (Maybe (ChainSelMessage m blk))
forall a. HasCallStack => STM m a -> m a
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
STM m a -> m a
atomically (TBQueue m (ChainSelMessage m blk)
-> STM m (Maybe (ChainSelMessage m blk))
forall (m :: * -> *) a.
MonadSTM m =>
TBQueue m a -> STM m (Maybe a)
tryReadTBQueue' TBQueue m (ChainSelMessage m blk)
queue) m (Maybe (ChainSelMessage m blk))
-> (Maybe (ChainSelMessage m blk) -> m (ChainSelMessage m blk))
-> m (ChainSelMessage 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
>>= \case
    Just ChainSelMessage m blk
msg -> ChainSelMessage m blk -> m (ChainSelMessage m blk)
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure ChainSelMessage m blk
msg
    Maybe (ChainSelMessage m blk)
Nothing -> do
      m ()
startStarvationMeasure
      msg <- STM m (ChainSelMessage m blk) -> m (ChainSelMessage m blk)
forall a. HasCallStack => STM m a -> m a
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
STM m a -> m a
atomically (STM m (ChainSelMessage m blk) -> m (ChainSelMessage m blk))
-> STM m (ChainSelMessage m blk) -> m (ChainSelMessage m blk)
forall a b. (a -> b) -> a -> b
$ TBQueue m (ChainSelMessage m blk) -> STM m (ChainSelMessage m blk)
forall a. TBQueue m a -> STM m a
forall (m :: * -> *) a. MonadSTM m => TBQueue m a -> STM m a
readTBQueue TBQueue m (ChainSelMessage m blk)
queue
      terminateStarvationMeasure msg
      pure msg
 where
  ChainSelQueue
    { varChainSelQueue :: forall (m :: * -> *) blk.
ChainSelQueue m blk -> TBQueue m (ChainSelMessage m blk)
varChainSelQueue = TBQueue m (ChainSelMessage m blk)
queue
    } = ChainSelQueue m blk
chainSelQueue

  startStarvationMeasure :: m ()
  startStarvationMeasure :: m ()
startStarvationMeasure = do
    prevStarvation <- STM m ChainSelStarvation -> m ChainSelStarvation
forall a. HasCallStack => STM m a -> m a
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
STM m a -> m a
atomically (STM m ChainSelStarvation -> m ChainSelStarvation)
-> STM m ChainSelStarvation -> m ChainSelStarvation
forall a b. (a -> b) -> a -> b
$ StrictTVar m ChainSelStarvation
-> ChainSelStarvation -> STM m ChainSelStarvation
forall (m :: * -> *) a.
MonadSTM m =>
StrictTVar m a -> a -> STM m a
swapTVar StrictTVar m ChainSelStarvation
starvationVar ChainSelStarvation
ChainSelStarvationOngoing
    when (prevStarvation /= ChainSelStarvationOngoing) $
      traceWith starvationTracer $
        ChainSelStarvation RisingEdge

  terminateStarvationMeasure :: ChainSelMessage m blk -> m ()
  terminateStarvationMeasure :: ChainSelMessage m blk -> m ()
terminateStarvationMeasure = \case
    ChainSelAddBlock BlockToAdd{blockToAdd :: forall (m :: * -> *) blk. BlockToAdd m blk -> blk
blockToAdd = blk
block} -> do
      let pt :: RealPoint blk
pt = blk -> RealPoint blk
forall blk. HasHeader blk => blk -> RealPoint blk
blockRealPoint blk
block
      Tracer m (TraceChainSelStarvationEvent blk)
-> TraceChainSelStarvationEvent blk -> m ()
forall (m :: * -> *) a. Tracer m a -> a -> m ()
traceWith Tracer m (TraceChainSelStarvationEvent blk)
starvationTracer (TraceChainSelStarvationEvent blk -> m ())
-> TraceChainSelStarvationEvent blk -> m ()
forall a b. (a -> b) -> a -> b
$ Enclosing' (RealPoint blk) -> TraceChainSelStarvationEvent blk
forall blk.
Enclosing' (RealPoint blk) -> TraceChainSelStarvationEvent blk
ChainSelStarvation (RealPoint blk -> Enclosing' (RealPoint blk)
forall a. a -> Enclosing' a
FallingEdgeWith RealPoint blk
pt)
      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 ()) -> (Time -> STM m ()) -> Time -> m ()
forall b c a. (b -> c) -> (a -> b) -> a -> c
. StrictTVar m ChainSelStarvation -> ChainSelStarvation -> STM m ()
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
StrictTVar m a -> a -> STM m ()
writeTVar StrictTVar m ChainSelStarvation
starvationVar (ChainSelStarvation -> STM m ())
-> (Time -> ChainSelStarvation) -> Time -> STM m ()
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Time -> ChainSelStarvation
ChainSelStarvationEndedAt (Time -> m ()) -> m Time -> m ()
forall (m :: * -> *) a b. Monad m => (a -> m b) -> m a -> m b
=<< m Time
forall (m :: * -> *). MonadMonotonicTime m => m Time
getMonotonicTime
    ChainSelReprocessLoEBlocks{} -> () -> m ()
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure ()

-- TODO Can't use tryReadTBQueue from io-classes because it is broken for IOSim
-- (but not for IO). https://github.com/input-output-hk/io-sim/issues/195
tryReadTBQueue' :: MonadSTM m => TBQueue m a -> STM m (Maybe a)
tryReadTBQueue' :: forall (m :: * -> *) a.
MonadSTM m =>
TBQueue m a -> STM m (Maybe a)
tryReadTBQueue' TBQueue m a
q = (a -> Maybe a
forall a. a -> Maybe a
Just (a -> Maybe a) -> STM m a -> STM m (Maybe a)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> TBQueue m a -> STM m a
forall a. TBQueue m a -> STM m a
forall (m :: * -> *) a. MonadSTM m => TBQueue m a -> STM m a
readTBQueue TBQueue m a
q) STM m (Maybe a) -> STM m (Maybe a) -> STM m (Maybe a)
forall a. STM m a -> STM m a -> STM m a
forall (m :: * -> *) a. MonadSTM m => STM m a -> STM m a -> STM m a
`orElse` Maybe a -> STM m (Maybe a)
forall a. a -> STM m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure Maybe a
forall a. Maybe a
Nothing

-- | Flush the 'ChainSelQueue' queue and notify the waiting threads.
closeChainSelQueue :: IOLike m => ChainSelQueue m blk -> STM m ()
closeChainSelQueue :: forall (m :: * -> *) blk.
IOLike m =>
ChainSelQueue m blk -> STM m ()
closeChainSelQueue ChainSelQueue{varChainSelQueue :: forall (m :: * -> *) blk.
ChainSelQueue m blk -> TBQueue m (ChainSelMessage m blk)
varChainSelQueue = TBQueue m (ChainSelMessage m blk)
queue} = do
  as <- (ChainSelMessage m blk -> Maybe (BlockToAdd m blk))
-> [ChainSelMessage m blk] -> [BlockToAdd m blk]
forall a b. (a -> Maybe b) -> [a] -> [b]
mapMaybe ChainSelMessage m blk -> Maybe (BlockToAdd m blk)
forall {m :: * -> *} {blk}.
ChainSelMessage m blk -> Maybe (BlockToAdd m blk)
blockAdd ([ChainSelMessage m blk] -> [BlockToAdd m blk])
-> STM m [ChainSelMessage m blk] -> STM m [BlockToAdd m blk]
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> TBQueue m (ChainSelMessage m blk) -> STM m [ChainSelMessage m blk]
forall a. TBQueue m a -> STM m [a]
forall (m :: * -> *) a. MonadSTM m => TBQueue m a -> STM m [a]
flushTBQueue TBQueue m (ChainSelMessage m blk)
queue
  traverse_
    ( \BlockToAdd m blk
a ->
        StrictTMVar m (AddBlockResult blk)
-> AddBlockResult blk -> STM m Bool
forall (m :: * -> *) a.
MonadSTM m =>
StrictTMVar m a -> a -> STM m Bool
tryPutTMVar
          (BlockToAdd m blk -> StrictTMVar m (AddBlockResult blk)
forall (m :: * -> *) blk.
BlockToAdd m blk -> StrictTMVar m (AddBlockResult blk)
varBlockProcessed BlockToAdd m blk
a)
          (String -> AddBlockResult blk
forall blk. String -> AddBlockResult blk
FailedToAddBlock String
"Queue flushed")
    )
    as
 where
  blockAdd :: ChainSelMessage m blk -> Maybe (BlockToAdd m blk)
blockAdd = \case
    ChainSelAddBlock BlockToAdd m blk
ab -> BlockToAdd m blk -> Maybe (BlockToAdd m blk)
forall a. a -> Maybe a
Just BlockToAdd m blk
ab
    ChainSelReprocessLoEBlocks StrictTMVar m ()
_ -> Maybe (BlockToAdd m blk)
forall a. Maybe a
Nothing

-- | To invoke when the given 'ChainSelMessage' has been processed by ChainSel.
-- This is used to remove the respective point from the multiset of points in
-- the 'ChainSelQueue' (as the block has now been written to disk by ChainSel).
processedChainSelMessage ::
  (IOLike m, HasHeader blk) =>
  ChainSelQueue m blk ->
  ChainSelMessage m blk ->
  STM m ()
processedChainSelMessage :: forall (m :: * -> *) blk.
(IOLike m, HasHeader blk) =>
ChainSelQueue m blk -> ChainSelMessage m blk -> STM m ()
processedChainSelMessage ChainSelQueue{StrictTVar m (MultiSet (RealPoint blk))
varChainSelPoints :: forall (m :: * -> *) blk.
ChainSelQueue m blk -> StrictTVar m (MultiSet (RealPoint blk))
varChainSelPoints :: StrictTVar m (MultiSet (RealPoint blk))
varChainSelPoints} = \case
  ChainSelAddBlock BlockToAdd{blockToAdd :: forall (m :: * -> *) blk. BlockToAdd m blk -> blk
blockToAdd = blk
blk} ->
    StrictTVar m (MultiSet (RealPoint blk))
-> (MultiSet (RealPoint blk) -> MultiSet (RealPoint blk))
-> STM m ()
forall (m :: * -> *) a.
MonadSTM m =>
StrictTVar m a -> (a -> a) -> STM m ()
modifyTVar StrictTVar m (MultiSet (RealPoint blk))
varChainSelPoints ((MultiSet (RealPoint blk) -> MultiSet (RealPoint blk))
 -> STM m ())
-> (MultiSet (RealPoint blk) -> MultiSet (RealPoint blk))
-> STM m ()
forall a b. (a -> b) -> a -> b
$ RealPoint blk
-> MultiSet (RealPoint blk) -> MultiSet (RealPoint blk)
forall a. Ord a => a -> MultiSet a -> MultiSet a
MultiSet.delete (blk -> RealPoint blk
forall blk. HasHeader blk => blk -> RealPoint blk
blockRealPoint blk
blk)
  ChainSelReprocessLoEBlocks{} ->
    () -> STM m ()
forall a. a -> STM m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure ()

-- | Return a function to test the membership
memberChainSelQueue ::
  (IOLike m, HasHeader blk) =>
  ChainSelQueue m blk ->
  STM m (RealPoint blk -> Bool)
memberChainSelQueue :: forall (m :: * -> *) blk.
(IOLike m, HasHeader blk) =>
ChainSelQueue m blk -> STM m (RealPoint blk -> Bool)
memberChainSelQueue ChainSelQueue{StrictTVar m (MultiSet (RealPoint blk))
varChainSelPoints :: forall (m :: * -> *) blk.
ChainSelQueue m blk -> StrictTVar m (MultiSet (RealPoint blk))
varChainSelPoints :: StrictTVar m (MultiSet (RealPoint blk))
varChainSelPoints} =
  (RealPoint blk -> MultiSet (RealPoint blk) -> Bool)
-> MultiSet (RealPoint blk) -> RealPoint blk -> Bool
forall a b c. (a -> b -> c) -> b -> a -> c
flip RealPoint blk -> MultiSet (RealPoint blk) -> Bool
forall a. Ord a => a -> MultiSet a -> Bool
MultiSet.member (MultiSet (RealPoint blk) -> RealPoint blk -> Bool)
-> STM m (MultiSet (RealPoint blk))
-> STM m (RealPoint blk -> Bool)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> StrictTVar m (MultiSet (RealPoint blk))
-> STM m (MultiSet (RealPoint blk))
forall (m :: * -> *) a. MonadSTM m => StrictTVar m a -> STM m a
readTVar StrictTVar m (MultiSet (RealPoint blk))
varChainSelPoints

getMaxSlotNoChainSelQueue ::
  IOLike m =>
  ChainSelQueue m blk ->
  STM m MaxSlotNo
getMaxSlotNoChainSelQueue :: forall (m :: * -> *) blk.
IOLike m =>
ChainSelQueue m blk -> STM m MaxSlotNo
getMaxSlotNoChainSelQueue ChainSelQueue{StrictTVar m (MultiSet (RealPoint blk))
varChainSelPoints :: forall (m :: * -> *) blk.
ChainSelQueue m blk -> StrictTVar m (MultiSet (RealPoint blk))
varChainSelPoints :: StrictTVar m (MultiSet (RealPoint blk))
varChainSelPoints} =
  MultiSet (RealPoint blk) -> MaxSlotNo
forall blk. MultiSet (RealPoint blk) -> MaxSlotNo
aux (MultiSet (RealPoint blk) -> MaxSlotNo)
-> STM m (MultiSet (RealPoint blk)) -> STM m MaxSlotNo
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> StrictTVar m (MultiSet (RealPoint blk))
-> STM m (MultiSet (RealPoint blk))
forall (m :: * -> *) a. MonadSTM m => StrictTVar m a -> STM m a
readTVar StrictTVar m (MultiSet (RealPoint blk))
varChainSelPoints
 where
  -- \| The 'Ord' instance of 'RealPoint' orders by 'SlotNo' first, so the
  -- maximal key of the map has the greatest 'SlotNo'.
  aux :: MultiSet (RealPoint blk) -> MaxSlotNo
  aux :: forall blk. MultiSet (RealPoint blk) -> MaxSlotNo
aux MultiSet (RealPoint blk)
pts = case MultiSet (RealPoint blk)
-> Maybe (RealPoint blk, MultiSet (RealPoint blk))
forall a. MultiSet a -> Maybe (a, MultiSet a)
MultiSet.maxView MultiSet (RealPoint blk)
pts of
    Maybe (RealPoint blk, MultiSet (RealPoint blk))
Nothing -> MaxSlotNo
NoMaxSlotNo
    Just (RealPoint SlotNo
s HeaderHash blk
_, MultiSet (RealPoint blk)
_) -> SlotNo -> MaxSlotNo
MaxSlotNo SlotNo
s

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

-- | Trace type for the various events of the ChainDB.
data TraceEvent blk
  = TraceAddBlockEvent (TraceAddBlockEvent blk)
  | TraceFollowerEvent (TraceFollowerEvent blk)
  | TraceCopyToImmutableDBEvent (TraceCopyToImmutableDBEvent blk)
  | TraceGCEvent (TraceGCEvent blk)
  | TraceInitChainSelEvent (TraceInitChainSelEvent blk)
  | TraceOpenEvent (TraceOpenEvent blk)
  | TraceIteratorEvent (TraceIteratorEvent blk)
  | TraceLedgerDBEvent (LedgerDB.TraceEvent blk)
  | TraceImmutableDBEvent (ImmutableDB.TraceEvent blk)
  | TraceVolatileDBEvent (VolatileDB.TraceEvent blk)
  | TraceLastShutdownUnclean
  | TraceChainSelStarvationEvent (TraceChainSelStarvationEvent blk)
  deriving (forall x. TraceEvent blk -> Rep (TraceEvent blk) x)
-> (forall x. Rep (TraceEvent blk) x -> TraceEvent blk)
-> Generic (TraceEvent blk)
forall x. Rep (TraceEvent blk) x -> TraceEvent blk
forall x. TraceEvent blk -> Rep (TraceEvent blk) x
forall a.
(forall x. a -> Rep a x) -> (forall x. Rep a x -> a) -> Generic a
forall blk x. Rep (TraceEvent blk) x -> TraceEvent blk
forall blk x. TraceEvent blk -> Rep (TraceEvent blk) x
$cfrom :: forall blk x. TraceEvent blk -> Rep (TraceEvent blk) x
from :: forall x. TraceEvent blk -> Rep (TraceEvent blk) x
$cto :: forall blk x. Rep (TraceEvent blk) x -> TraceEvent blk
to :: forall x. Rep (TraceEvent blk) x -> TraceEvent blk
Generic

deriving instance
  ( Eq (Header blk)
  , LedgerSupportsProtocol blk
  , InspectLedger blk
  ) =>
  Eq (TraceEvent blk)
deriving instance
  ( Show (Header blk)
  , LedgerSupportsProtocol blk
  , InspectLedger blk
  ) =>
  Show (TraceEvent blk)

data TraceOpenEvent blk
  = -- | The ChainDB started the process of opening.
    StartedOpeningDB
  | -- | The ChainDB was opened.
    OpenedDB
      -- | Immutable tip
      (Point blk)
      -- | Tip of the current chain
      (Point blk)
  | -- | The ChainDB was closed.
    ClosedDB
      -- | Immutable tip
      (Point blk)
      -- | Tip of the current chain
      (Point blk)
  | -- | The ImmutableDB started the process of opening.
    StartedOpeningImmutableDB
  | -- | The ImmutableDB was opened.
    OpenedImmutableDB
      -- | Immutable tip
      (Point blk)
      -- | Chunk number of the immutable tip
      ImmutableDB.ChunkNo
  | -- | The VolatileDB started opening.
    StartedOpeningVolatileDB
  | -- | The VolatileDB was opened, with the highest seen slot number for any
    -- block currently in the DB.
    OpenedVolatileDB MaxSlotNo
  | -- | The LedgerDB started opening.
    StartedOpeningLgrDB
  | -- | The LedgerDB was opened.
    OpenedLgrDB
  deriving ((forall x. TraceOpenEvent blk -> Rep (TraceOpenEvent blk) x)
-> (forall x. Rep (TraceOpenEvent blk) x -> TraceOpenEvent blk)
-> Generic (TraceOpenEvent blk)
forall x. Rep (TraceOpenEvent blk) x -> TraceOpenEvent blk
forall x. TraceOpenEvent blk -> Rep (TraceOpenEvent blk) x
forall a.
(forall x. a -> Rep a x) -> (forall x. Rep a x -> a) -> Generic a
forall blk x. Rep (TraceOpenEvent blk) x -> TraceOpenEvent blk
forall blk x. TraceOpenEvent blk -> Rep (TraceOpenEvent blk) x
$cfrom :: forall blk x. TraceOpenEvent blk -> Rep (TraceOpenEvent blk) x
from :: forall x. TraceOpenEvent blk -> Rep (TraceOpenEvent blk) x
$cto :: forall blk x. Rep (TraceOpenEvent blk) x -> TraceOpenEvent blk
to :: forall x. Rep (TraceOpenEvent blk) x -> TraceOpenEvent blk
Generic, TraceOpenEvent blk -> TraceOpenEvent blk -> Bool
(TraceOpenEvent blk -> TraceOpenEvent blk -> Bool)
-> (TraceOpenEvent blk -> TraceOpenEvent blk -> Bool)
-> Eq (TraceOpenEvent blk)
forall blk.
StandardHash blk =>
TraceOpenEvent blk -> TraceOpenEvent blk -> Bool
forall a. (a -> a -> Bool) -> (a -> a -> Bool) -> Eq a
$c== :: forall blk.
StandardHash blk =>
TraceOpenEvent blk -> TraceOpenEvent blk -> Bool
== :: TraceOpenEvent blk -> TraceOpenEvent blk -> Bool
$c/= :: forall blk.
StandardHash blk =>
TraceOpenEvent blk -> TraceOpenEvent blk -> Bool
/= :: TraceOpenEvent blk -> TraceOpenEvent blk -> Bool
Eq, Int -> TraceOpenEvent blk -> String -> String
[TraceOpenEvent blk] -> String -> String
TraceOpenEvent blk -> String
(Int -> TraceOpenEvent blk -> String -> String)
-> (TraceOpenEvent blk -> String)
-> ([TraceOpenEvent blk] -> String -> String)
-> Show (TraceOpenEvent blk)
forall blk.
StandardHash blk =>
Int -> TraceOpenEvent blk -> String -> String
forall blk.
StandardHash blk =>
[TraceOpenEvent blk] -> String -> String
forall blk. StandardHash blk => TraceOpenEvent blk -> String
forall a.
(Int -> a -> String -> String)
-> (a -> String) -> ([a] -> String -> String) -> Show a
$cshowsPrec :: forall blk.
StandardHash blk =>
Int -> TraceOpenEvent blk -> String -> String
showsPrec :: Int -> TraceOpenEvent blk -> String -> String
$cshow :: forall blk. StandardHash blk => TraceOpenEvent blk -> String
show :: TraceOpenEvent blk -> String
$cshowList :: forall blk.
StandardHash blk =>
[TraceOpenEvent blk] -> String -> String
showList :: [TraceOpenEvent blk] -> String -> String
Show)

-- | Information on having changed our selection to a chain with a (necessarily)
-- new tip.
--
-- NOTE: the fields of this record are intentionally lazy to prevent the
-- forcing of this information in case it doesn't have to be traced. However,
-- this means that the tracer processing this message /must not/ hold on to
-- it, otherwise it leaks memory.
data SelectionChangedInfo blk = SelectionChangedInfo
  { forall blk. SelectionChangedInfo blk -> RealPoint blk
newTipPoint :: RealPoint blk
  -- ^ The new tip of the current chain.
  , forall blk. SelectionChangedInfo blk -> EpochNo
newTipEpoch :: EpochNo
  -- ^ The epoch of the new tip.
  , forall blk. SelectionChangedInfo blk -> Word64
newTipSlotInEpoch :: Word64
  -- ^ The slot in the epoch, i.e., the relative slot number, of the new
  -- tip.
  , forall blk. SelectionChangedInfo blk -> RealPoint blk
newTipTrigger :: RealPoint blk
  -- ^ The new tip of the current chain ('newTipPoint') is the result of
  -- performing chain selection for a /trigger/ block ('newTipTrigger').
  -- In most cases, we add a new block to the tip of the current chain, in
  -- which case the new tip /is/ the trigger block.
  --
  -- However, this is not always the case. For example, with our current
  -- chain being A and having a disconnected C lying around, adding B will
  -- result in A -> B -> C as the new chain. The trigger B /= the new tip
  -- C.
  , forall blk.
SelectionChangedInfo blk -> SelectView (BlockProtocol blk)
newTipSelectView :: SelectView (BlockProtocol blk)
  -- ^ The 'SelectView' of the new tip. It is guaranteed that
  --
  -- > Just newTipSelectView > oldTipSelectView
  -- True
  , forall blk.
SelectionChangedInfo blk -> Maybe (SelectView (BlockProtocol blk))
oldTipSelectView :: Maybe (SelectView (BlockProtocol blk))
  -- ^ The 'SelectView' of the old, previous tip. This can be 'Nothing' when
  -- the previous chain/tip was Genesis.
  }
  deriving (forall x.
 SelectionChangedInfo blk -> Rep (SelectionChangedInfo blk) x)
-> (forall x.
    Rep (SelectionChangedInfo blk) x -> SelectionChangedInfo blk)
-> Generic (SelectionChangedInfo blk)
forall x.
Rep (SelectionChangedInfo blk) x -> SelectionChangedInfo blk
forall x.
SelectionChangedInfo blk -> Rep (SelectionChangedInfo blk) x
forall a.
(forall x. a -> Rep a x) -> (forall x. Rep a x -> a) -> Generic a
forall blk x.
Rep (SelectionChangedInfo blk) x -> SelectionChangedInfo blk
forall blk x.
SelectionChangedInfo blk -> Rep (SelectionChangedInfo blk) x
$cfrom :: forall blk x.
SelectionChangedInfo blk -> Rep (SelectionChangedInfo blk) x
from :: forall x.
SelectionChangedInfo blk -> Rep (SelectionChangedInfo blk) x
$cto :: forall blk x.
Rep (SelectionChangedInfo blk) x -> SelectionChangedInfo blk
to :: forall x.
Rep (SelectionChangedInfo blk) x -> SelectionChangedInfo blk
Generic

deriving stock instance
  (Show (SelectView (BlockProtocol blk)), StandardHash blk) => Show (SelectionChangedInfo blk)
deriving stock instance
  (Eq (SelectView (BlockProtocol blk)), StandardHash blk) => Eq (SelectionChangedInfo blk)

-- | Trace type for the various events that occur when adding a block.
data TraceAddBlockEvent blk
  = -- | A block with a 'BlockNo' more than @k@ back than the current tip was
    -- ignored.
    IgnoreBlockOlderThanK (RealPoint blk)
  | -- | A block that is already in the Volatile DB was ignored.
    IgnoreBlockAlreadyInVolatileDB (RealPoint blk)
  | -- | A block that is know to be invalid was ignored.
    IgnoreInvalidBlock (RealPoint blk) (ExtValidationError blk)
  | -- | The block was added to the queue and will be added to the ChainDB by
    -- the background thread. The size of the queue is included.
    AddedBlockToQueue (RealPoint blk) (Enclosing' Word)
  | -- | The block popped from the queue and will imminently be added to the
    -- ChainDB.
    PoppedBlockFromQueue (Enclosing' (RealPoint blk))
  | -- | A message was added to the queue that requests that ChainSel reprocess
    -- blocks that were postponed by the LoE.
    AddedReprocessLoEBlocksToQueue
  | -- | ChainSel will reprocess blocks that were postponed by the LoE.
    PoppedReprocessLoEBlocksFromQueue
  | -- | A block was added to the Volatile DB
    AddedBlockToVolatileDB (RealPoint blk) BlockNo IsEBB Enclosing
  | -- | The block fits onto the current chain, we'll try to use it to extend
    -- our chain.
    TryAddToCurrentChain (RealPoint blk)
  | -- | The block fits onto some fork, we'll try to switch to that fork (if
    -- it is preferable to our chain).
    TrySwitchToAFork (RealPoint blk) (ChainDiff (HeaderFields blk))
  | -- | The block doesn't fit onto any other block, so we store it and ignore
    -- it.
    StoreButDontChange (RealPoint blk)
  | -- | Debugging information about chain selection and LoE
    ChainSelectionLoEDebug (AnchoredFragment (Header blk)) (LoE (AnchoredFragment (Header blk)))
  | -- | The new block fits onto the current chain (first
    -- fragment) and we have successfully used it to extend our (new) current
    -- chain (second fragment).
    AddedToCurrentChain
      [LedgerEvent blk]
      (SelectionChangedInfo blk)
      (AnchoredFragment (Header blk))
      (AnchoredFragment (Header blk))
  | -- | The new block fits onto some fork and we have switched to that fork
    -- (second fragment), as it is preferable to our (previous) current chain
    -- (first fragment).
    SwitchedToAFork
      [LedgerEvent blk]
      (SelectionChangedInfo blk)
      (AnchoredFragment (Header blk))
      (AnchoredFragment (Header blk))
  | -- | An event traced during validating performed while adding a block.
    AddBlockValidation (TraceValidationEvent blk)
  | -- | The tentative header (in the context of diffusion pipelining) has been
    -- updated.
    PipeliningEvent (TracePipeliningEvent blk)
  | -- | Herald of 'AddedToCurrentChain' or 'SwitchedToAFork'. Lists the tip of
    -- the new chain.
    ChangingSelection (Point blk)
  deriving (forall x.
 TraceAddBlockEvent blk -> Rep (TraceAddBlockEvent blk) x)
-> (forall x.
    Rep (TraceAddBlockEvent blk) x -> TraceAddBlockEvent blk)
-> Generic (TraceAddBlockEvent blk)
forall x. Rep (TraceAddBlockEvent blk) x -> TraceAddBlockEvent blk
forall x. TraceAddBlockEvent blk -> Rep (TraceAddBlockEvent blk) x
forall a.
(forall x. a -> Rep a x) -> (forall x. Rep a x -> a) -> Generic a
forall blk x.
Rep (TraceAddBlockEvent blk) x -> TraceAddBlockEvent blk
forall blk x.
TraceAddBlockEvent blk -> Rep (TraceAddBlockEvent blk) x
$cfrom :: forall blk x.
TraceAddBlockEvent blk -> Rep (TraceAddBlockEvent blk) x
from :: forall x. TraceAddBlockEvent blk -> Rep (TraceAddBlockEvent blk) x
$cto :: forall blk x.
Rep (TraceAddBlockEvent blk) x -> TraceAddBlockEvent blk
to :: forall x. Rep (TraceAddBlockEvent blk) x -> TraceAddBlockEvent blk
Generic

deriving instance
  ( Eq (Header blk)
  , LedgerSupportsProtocol blk
  , InspectLedger blk
  ) =>
  Eq (TraceAddBlockEvent blk)
deriving instance
  ( Show (Header blk)
  , LedgerSupportsProtocol blk
  , InspectLedger blk
  ) =>
  Show (TraceAddBlockEvent blk)

data TraceValidationEvent blk
  = -- | A point was found to be invalid.
    InvalidBlock
      (ExtValidationError blk)
      (RealPoint blk)
  | -- | A candidate chain was valid.
    ValidCandidate (AnchoredFragment (Header blk))
  | UpdateLedgerDbTraceEvent (LedgerDB.TraceValidateEvent blk)
  deriving (forall x.
 TraceValidationEvent blk -> Rep (TraceValidationEvent blk) x)
-> (forall x.
    Rep (TraceValidationEvent blk) x -> TraceValidationEvent blk)
-> Generic (TraceValidationEvent blk)
forall x.
Rep (TraceValidationEvent blk) x -> TraceValidationEvent blk
forall x.
TraceValidationEvent blk -> Rep (TraceValidationEvent blk) x
forall a.
(forall x. a -> Rep a x) -> (forall x. Rep a x -> a) -> Generic a
forall blk x.
Rep (TraceValidationEvent blk) x -> TraceValidationEvent blk
forall blk x.
TraceValidationEvent blk -> Rep (TraceValidationEvent blk) x
$cfrom :: forall blk x.
TraceValidationEvent blk -> Rep (TraceValidationEvent blk) x
from :: forall x.
TraceValidationEvent blk -> Rep (TraceValidationEvent blk) x
$cto :: forall blk x.
Rep (TraceValidationEvent blk) x -> TraceValidationEvent blk
to :: forall x.
Rep (TraceValidationEvent blk) x -> TraceValidationEvent blk
Generic

deriving instance
  ( Eq (Header blk)
  , LedgerSupportsProtocol blk
  ) =>
  Eq (TraceValidationEvent blk)
deriving instance
  ( Show (Header blk)
  , LedgerSupportsProtocol blk
  ) =>
  Show (TraceValidationEvent blk)

data TracePipeliningEvent blk
  = -- | A new tentative header got set.
    SetTentativeHeader (Header blk) Enclosing
  | -- | The body of tentative block turned out to be invalid.
    TrapTentativeHeader (Header blk)
  | -- | We selected a new (better) chain, which cleared the previous tentative
    -- header.
    OutdatedTentativeHeader (Header blk)

deriving stock instance Eq (Header blk) => Eq (TracePipeliningEvent blk)
deriving stock instance Show (Header blk) => Show (TracePipeliningEvent blk)

data TraceInitChainSelEvent blk
  = -- | An event traced when inital chain selection has started during the
    -- initialization of ChainDB
    StartedInitChainSelection
  | -- | An event traced when inital chain has been selected
    InitialChainSelected
  | -- | An event traced during validation performed while performing initial
    -- chain selection.
    InitChainSelValidation (TraceValidationEvent blk)
  deriving (forall x.
 TraceInitChainSelEvent blk -> Rep (TraceInitChainSelEvent blk) x)
-> (forall x.
    Rep (TraceInitChainSelEvent blk) x -> TraceInitChainSelEvent blk)
-> Generic (TraceInitChainSelEvent blk)
forall x.
Rep (TraceInitChainSelEvent blk) x -> TraceInitChainSelEvent blk
forall x.
TraceInitChainSelEvent blk -> Rep (TraceInitChainSelEvent blk) x
forall a.
(forall x. a -> Rep a x) -> (forall x. Rep a x -> a) -> Generic a
forall blk x.
Rep (TraceInitChainSelEvent blk) x -> TraceInitChainSelEvent blk
forall blk x.
TraceInitChainSelEvent blk -> Rep (TraceInitChainSelEvent blk) x
$cfrom :: forall blk x.
TraceInitChainSelEvent blk -> Rep (TraceInitChainSelEvent blk) x
from :: forall x.
TraceInitChainSelEvent blk -> Rep (TraceInitChainSelEvent blk) x
$cto :: forall blk x.
Rep (TraceInitChainSelEvent blk) x -> TraceInitChainSelEvent blk
to :: forall x.
Rep (TraceInitChainSelEvent blk) x -> TraceInitChainSelEvent blk
Generic

deriving instance
  ( Eq (Header blk)
  , LedgerSupportsProtocol blk
  ) =>
  Eq (TraceInitChainSelEvent blk)
deriving instance
  ( Show (Header blk)
  , LedgerSupportsProtocol blk
  ) =>
  Show (TraceInitChainSelEvent blk)

data TraceFollowerEvent blk
  = -- | A new follower was created.
    NewFollower
  | -- | The follower was in the 'FollowerInMem' state but its point is no longer on
    -- the in-memory chain fragment, so it has to switch to the
    -- 'FollowerInImmutableDB' state.
    FollowerNoLongerInMem (FollowerRollState blk)
  | -- | The follower was in the 'FollowerInImmutableDB' state and is switched to
    -- the 'FollowerInMem' state.
    FollowerSwitchToMem
      -- | Point at which the follower is
      (Point blk)
      -- | Slot number at the tip of the ImmutableDB
      (WithOrigin SlotNo)
  | -- | The follower is in the 'FollowerInImmutableDB' state but the iterator is
    -- exhausted while the ImmutableDB has grown, so we open a new iterator to
    -- stream these blocks too.
    FollowerNewImmIterator
      -- | Point at which the follower is
      (Point blk)
      -- | Slot number at the tip of the ImmutableDB
      (WithOrigin SlotNo)
  deriving ((forall x.
 TraceFollowerEvent blk -> Rep (TraceFollowerEvent blk) x)
-> (forall x.
    Rep (TraceFollowerEvent blk) x -> TraceFollowerEvent blk)
-> Generic (TraceFollowerEvent blk)
forall x. Rep (TraceFollowerEvent blk) x -> TraceFollowerEvent blk
forall x. TraceFollowerEvent blk -> Rep (TraceFollowerEvent blk) x
forall a.
(forall x. a -> Rep a x) -> (forall x. Rep a x -> a) -> Generic a
forall blk x.
Rep (TraceFollowerEvent blk) x -> TraceFollowerEvent blk
forall blk x.
TraceFollowerEvent blk -> Rep (TraceFollowerEvent blk) x
$cfrom :: forall blk x.
TraceFollowerEvent blk -> Rep (TraceFollowerEvent blk) x
from :: forall x. TraceFollowerEvent blk -> Rep (TraceFollowerEvent blk) x
$cto :: forall blk x.
Rep (TraceFollowerEvent blk) x -> TraceFollowerEvent blk
to :: forall x. Rep (TraceFollowerEvent blk) x -> TraceFollowerEvent blk
Generic, TraceFollowerEvent blk -> TraceFollowerEvent blk -> Bool
(TraceFollowerEvent blk -> TraceFollowerEvent blk -> Bool)
-> (TraceFollowerEvent blk -> TraceFollowerEvent blk -> Bool)
-> Eq (TraceFollowerEvent blk)
forall blk.
StandardHash blk =>
TraceFollowerEvent blk -> TraceFollowerEvent blk -> Bool
forall a. (a -> a -> Bool) -> (a -> a -> Bool) -> Eq a
$c== :: forall blk.
StandardHash blk =>
TraceFollowerEvent blk -> TraceFollowerEvent blk -> Bool
== :: TraceFollowerEvent blk -> TraceFollowerEvent blk -> Bool
$c/= :: forall blk.
StandardHash blk =>
TraceFollowerEvent blk -> TraceFollowerEvent blk -> Bool
/= :: TraceFollowerEvent blk -> TraceFollowerEvent blk -> Bool
Eq, Int -> TraceFollowerEvent blk -> String -> String
[TraceFollowerEvent blk] -> String -> String
TraceFollowerEvent blk -> String
(Int -> TraceFollowerEvent blk -> String -> String)
-> (TraceFollowerEvent blk -> String)
-> ([TraceFollowerEvent blk] -> String -> String)
-> Show (TraceFollowerEvent blk)
forall blk.
StandardHash blk =>
Int -> TraceFollowerEvent blk -> String -> String
forall blk.
StandardHash blk =>
[TraceFollowerEvent blk] -> String -> String
forall blk. StandardHash blk => TraceFollowerEvent blk -> String
forall a.
(Int -> a -> String -> String)
-> (a -> String) -> ([a] -> String -> String) -> Show a
$cshowsPrec :: forall blk.
StandardHash blk =>
Int -> TraceFollowerEvent blk -> String -> String
showsPrec :: Int -> TraceFollowerEvent blk -> String -> String
$cshow :: forall blk. StandardHash blk => TraceFollowerEvent blk -> String
show :: TraceFollowerEvent blk -> String
$cshowList :: forall blk.
StandardHash blk =>
[TraceFollowerEvent blk] -> String -> String
showList :: [TraceFollowerEvent blk] -> String -> String
Show)

data TraceCopyToImmutableDBEvent blk
  = -- | A block was successfully copied to the ImmutableDB.
    CopiedBlockToImmutableDB (Point blk)
  | -- | There are no block to copy to the ImmutableDB.
    NoBlocksToCopyToImmutableDB
  deriving ((forall x.
 TraceCopyToImmutableDBEvent blk
 -> Rep (TraceCopyToImmutableDBEvent blk) x)
-> (forall x.
    Rep (TraceCopyToImmutableDBEvent blk) x
    -> TraceCopyToImmutableDBEvent blk)
-> Generic (TraceCopyToImmutableDBEvent blk)
forall x.
Rep (TraceCopyToImmutableDBEvent blk) x
-> TraceCopyToImmutableDBEvent blk
forall x.
TraceCopyToImmutableDBEvent blk
-> Rep (TraceCopyToImmutableDBEvent blk) x
forall a.
(forall x. a -> Rep a x) -> (forall x. Rep a x -> a) -> Generic a
forall blk x.
Rep (TraceCopyToImmutableDBEvent blk) x
-> TraceCopyToImmutableDBEvent blk
forall blk x.
TraceCopyToImmutableDBEvent blk
-> Rep (TraceCopyToImmutableDBEvent blk) x
$cfrom :: forall blk x.
TraceCopyToImmutableDBEvent blk
-> Rep (TraceCopyToImmutableDBEvent blk) x
from :: forall x.
TraceCopyToImmutableDBEvent blk
-> Rep (TraceCopyToImmutableDBEvent blk) x
$cto :: forall blk x.
Rep (TraceCopyToImmutableDBEvent blk) x
-> TraceCopyToImmutableDBEvent blk
to :: forall x.
Rep (TraceCopyToImmutableDBEvent blk) x
-> TraceCopyToImmutableDBEvent blk
Generic, TraceCopyToImmutableDBEvent blk
-> TraceCopyToImmutableDBEvent blk -> Bool
(TraceCopyToImmutableDBEvent blk
 -> TraceCopyToImmutableDBEvent blk -> Bool)
-> (TraceCopyToImmutableDBEvent blk
    -> TraceCopyToImmutableDBEvent blk -> Bool)
-> Eq (TraceCopyToImmutableDBEvent blk)
forall blk.
StandardHash blk =>
TraceCopyToImmutableDBEvent blk
-> TraceCopyToImmutableDBEvent blk -> Bool
forall a. (a -> a -> Bool) -> (a -> a -> Bool) -> Eq a
$c== :: forall blk.
StandardHash blk =>
TraceCopyToImmutableDBEvent blk
-> TraceCopyToImmutableDBEvent blk -> Bool
== :: TraceCopyToImmutableDBEvent blk
-> TraceCopyToImmutableDBEvent blk -> Bool
$c/= :: forall blk.
StandardHash blk =>
TraceCopyToImmutableDBEvent blk
-> TraceCopyToImmutableDBEvent blk -> Bool
/= :: TraceCopyToImmutableDBEvent blk
-> TraceCopyToImmutableDBEvent blk -> Bool
Eq, Int -> TraceCopyToImmutableDBEvent blk -> String -> String
[TraceCopyToImmutableDBEvent blk] -> String -> String
TraceCopyToImmutableDBEvent blk -> String
(Int -> TraceCopyToImmutableDBEvent blk -> String -> String)
-> (TraceCopyToImmutableDBEvent blk -> String)
-> ([TraceCopyToImmutableDBEvent blk] -> String -> String)
-> Show (TraceCopyToImmutableDBEvent blk)
forall blk.
StandardHash blk =>
Int -> TraceCopyToImmutableDBEvent blk -> String -> String
forall blk.
StandardHash blk =>
[TraceCopyToImmutableDBEvent blk] -> String -> String
forall blk.
StandardHash blk =>
TraceCopyToImmutableDBEvent blk -> String
forall a.
(Int -> a -> String -> String)
-> (a -> String) -> ([a] -> String -> String) -> Show a
$cshowsPrec :: forall blk.
StandardHash blk =>
Int -> TraceCopyToImmutableDBEvent blk -> String -> String
showsPrec :: Int -> TraceCopyToImmutableDBEvent blk -> String -> String
$cshow :: forall blk.
StandardHash blk =>
TraceCopyToImmutableDBEvent blk -> String
show :: TraceCopyToImmutableDBEvent blk -> String
$cshowList :: forall blk.
StandardHash blk =>
[TraceCopyToImmutableDBEvent blk] -> String -> String
showList :: [TraceCopyToImmutableDBEvent blk] -> String -> String
Show)

data TraceGCEvent blk
  = -- | A garbage collection for the given 'SlotNo' was scheduled to happen
    -- at the given time.
    ScheduledGC SlotNo Time
  | -- | A garbage collection for the given 'SlotNo' was performed.
    PerformedGC SlotNo
  deriving ((forall x. TraceGCEvent blk -> Rep (TraceGCEvent blk) x)
-> (forall x. Rep (TraceGCEvent blk) x -> TraceGCEvent blk)
-> Generic (TraceGCEvent blk)
forall x. Rep (TraceGCEvent blk) x -> TraceGCEvent blk
forall x. TraceGCEvent blk -> Rep (TraceGCEvent blk) x
forall a.
(forall x. a -> Rep a x) -> (forall x. Rep a x -> a) -> Generic a
forall blk x. Rep (TraceGCEvent blk) x -> TraceGCEvent blk
forall blk x. TraceGCEvent blk -> Rep (TraceGCEvent blk) x
$cfrom :: forall blk x. TraceGCEvent blk -> Rep (TraceGCEvent blk) x
from :: forall x. TraceGCEvent blk -> Rep (TraceGCEvent blk) x
$cto :: forall blk x. Rep (TraceGCEvent blk) x -> TraceGCEvent blk
to :: forall x. Rep (TraceGCEvent blk) x -> TraceGCEvent blk
Generic, TraceGCEvent blk -> TraceGCEvent blk -> Bool
(TraceGCEvent blk -> TraceGCEvent blk -> Bool)
-> (TraceGCEvent blk -> TraceGCEvent blk -> Bool)
-> Eq (TraceGCEvent blk)
forall blk. TraceGCEvent blk -> TraceGCEvent blk -> Bool
forall a. (a -> a -> Bool) -> (a -> a -> Bool) -> Eq a
$c== :: forall blk. TraceGCEvent blk -> TraceGCEvent blk -> Bool
== :: TraceGCEvent blk -> TraceGCEvent blk -> Bool
$c/= :: forall blk. TraceGCEvent blk -> TraceGCEvent blk -> Bool
/= :: TraceGCEvent blk -> TraceGCEvent blk -> Bool
Eq, Int -> TraceGCEvent blk -> String -> String
[TraceGCEvent blk] -> String -> String
TraceGCEvent blk -> String
(Int -> TraceGCEvent blk -> String -> String)
-> (TraceGCEvent blk -> String)
-> ([TraceGCEvent blk] -> String -> String)
-> Show (TraceGCEvent blk)
forall blk. Int -> TraceGCEvent blk -> String -> String
forall blk. [TraceGCEvent blk] -> String -> String
forall blk. TraceGCEvent blk -> String
forall a.
(Int -> a -> String -> String)
-> (a -> String) -> ([a] -> String -> String) -> Show a
$cshowsPrec :: forall blk. Int -> TraceGCEvent blk -> String -> String
showsPrec :: Int -> TraceGCEvent blk -> String -> String
$cshow :: forall blk. TraceGCEvent blk -> String
show :: TraceGCEvent blk -> String
$cshowList :: forall blk. [TraceGCEvent blk] -> String -> String
showList :: [TraceGCEvent blk] -> String -> String
Show)

data TraceIteratorEvent blk
  = -- | An unknown range was requested, see 'UnknownRange'.
    UnknownRangeRequested (UnknownRange blk)
  | -- | Stream only from the VolatileDB.
    StreamFromVolatileDB
      (StreamFrom blk)
      (StreamTo blk)
      [RealPoint blk]
  | -- | Stream only from the ImmutableDB.
    StreamFromImmutableDB
      (StreamFrom blk)
      (StreamTo blk)
  | -- | Stream from both the VolatileDB and the ImmutableDB.
    StreamFromBoth
      (StreamFrom blk)
      (StreamTo blk)
      [RealPoint blk]
  | -- | A block is no longer in the VolatileDB because it has been garbage
    -- collected. It might now be in the ImmutableDB if it was part of the
    -- current chain.
    BlockMissingFromVolatileDB (RealPoint blk)
  | -- | A block that has been garbage collected from the VolatileDB is now
    -- found and streamed from the ImmutableDB.
    BlockWasCopiedToImmutableDB (RealPoint blk)
  | -- | A block is no longer in the VolatileDB and isn't in the ImmutableDB
    -- either; it wasn't part of the current chain.
    BlockGCedFromVolatileDB (RealPoint blk)
  | -- | We have streamed one or more blocks from the ImmutableDB that were part
    -- of the VolatileDB when initialising the iterator. Now, we have to look
    -- back in the VolatileDB again because the ImmutableDB doesn't have the
    -- next block we're looking for.
    SwitchBackToVolatileDB
  deriving ((forall x.
 TraceIteratorEvent blk -> Rep (TraceIteratorEvent blk) x)
-> (forall x.
    Rep (TraceIteratorEvent blk) x -> TraceIteratorEvent blk)
-> Generic (TraceIteratorEvent blk)
forall x. Rep (TraceIteratorEvent blk) x -> TraceIteratorEvent blk
forall x. TraceIteratorEvent blk -> Rep (TraceIteratorEvent blk) x
forall a.
(forall x. a -> Rep a x) -> (forall x. Rep a x -> a) -> Generic a
forall blk x.
Rep (TraceIteratorEvent blk) x -> TraceIteratorEvent blk
forall blk x.
TraceIteratorEvent blk -> Rep (TraceIteratorEvent blk) x
$cfrom :: forall blk x.
TraceIteratorEvent blk -> Rep (TraceIteratorEvent blk) x
from :: forall x. TraceIteratorEvent blk -> Rep (TraceIteratorEvent blk) x
$cto :: forall blk x.
Rep (TraceIteratorEvent blk) x -> TraceIteratorEvent blk
to :: forall x. Rep (TraceIteratorEvent blk) x -> TraceIteratorEvent blk
Generic, TraceIteratorEvent blk -> TraceIteratorEvent blk -> Bool
(TraceIteratorEvent blk -> TraceIteratorEvent blk -> Bool)
-> (TraceIteratorEvent blk -> TraceIteratorEvent blk -> Bool)
-> Eq (TraceIteratorEvent blk)
forall blk.
StandardHash blk =>
TraceIteratorEvent blk -> TraceIteratorEvent blk -> Bool
forall a. (a -> a -> Bool) -> (a -> a -> Bool) -> Eq a
$c== :: forall blk.
StandardHash blk =>
TraceIteratorEvent blk -> TraceIteratorEvent blk -> Bool
== :: TraceIteratorEvent blk -> TraceIteratorEvent blk -> Bool
$c/= :: forall blk.
StandardHash blk =>
TraceIteratorEvent blk -> TraceIteratorEvent blk -> Bool
/= :: TraceIteratorEvent blk -> TraceIteratorEvent blk -> Bool
Eq, Int -> TraceIteratorEvent blk -> String -> String
[TraceIteratorEvent blk] -> String -> String
TraceIteratorEvent blk -> String
(Int -> TraceIteratorEvent blk -> String -> String)
-> (TraceIteratorEvent blk -> String)
-> ([TraceIteratorEvent blk] -> String -> String)
-> Show (TraceIteratorEvent blk)
forall blk.
StandardHash blk =>
Int -> TraceIteratorEvent blk -> String -> String
forall blk.
StandardHash blk =>
[TraceIteratorEvent blk] -> String -> String
forall blk. StandardHash blk => TraceIteratorEvent blk -> String
forall a.
(Int -> a -> String -> String)
-> (a -> String) -> ([a] -> String -> String) -> Show a
$cshowsPrec :: forall blk.
StandardHash blk =>
Int -> TraceIteratorEvent blk -> String -> String
showsPrec :: Int -> TraceIteratorEvent blk -> String -> String
$cshow :: forall blk. StandardHash blk => TraceIteratorEvent blk -> String
show :: TraceIteratorEvent blk -> String
$cshowList :: forall blk.
StandardHash blk =>
[TraceIteratorEvent blk] -> String -> String
showList :: [TraceIteratorEvent blk] -> String -> String
Show)

-- | Chain selection is /starved/ when the background thread runs out of work.
-- This is the usual case and innocent while caught-up; but while syncing, it
-- means that we are downloading blocks at a smaller rate than we can validate
-- them, even though we generally expect to be CPU-bound.
--
-- TODO: Investigate why it happens regularly during syncing for very short
-- times.
--
-- The point in the trace is the block that finished the starvation.
newtype TraceChainSelStarvationEvent blk
  = ChainSelStarvation (Enclosing' (RealPoint blk))
  deriving ((forall x.
 TraceChainSelStarvationEvent blk
 -> Rep (TraceChainSelStarvationEvent blk) x)
-> (forall x.
    Rep (TraceChainSelStarvationEvent blk) x
    -> TraceChainSelStarvationEvent blk)
-> Generic (TraceChainSelStarvationEvent blk)
forall x.
Rep (TraceChainSelStarvationEvent blk) x
-> TraceChainSelStarvationEvent blk
forall x.
TraceChainSelStarvationEvent blk
-> Rep (TraceChainSelStarvationEvent blk) x
forall a.
(forall x. a -> Rep a x) -> (forall x. Rep a x -> a) -> Generic a
forall blk x.
Rep (TraceChainSelStarvationEvent blk) x
-> TraceChainSelStarvationEvent blk
forall blk x.
TraceChainSelStarvationEvent blk
-> Rep (TraceChainSelStarvationEvent blk) x
$cfrom :: forall blk x.
TraceChainSelStarvationEvent blk
-> Rep (TraceChainSelStarvationEvent blk) x
from :: forall x.
TraceChainSelStarvationEvent blk
-> Rep (TraceChainSelStarvationEvent blk) x
$cto :: forall blk x.
Rep (TraceChainSelStarvationEvent blk) x
-> TraceChainSelStarvationEvent blk
to :: forall x.
Rep (TraceChainSelStarvationEvent blk) x
-> TraceChainSelStarvationEvent blk
Generic, TraceChainSelStarvationEvent blk
-> TraceChainSelStarvationEvent blk -> Bool
(TraceChainSelStarvationEvent blk
 -> TraceChainSelStarvationEvent blk -> Bool)
-> (TraceChainSelStarvationEvent blk
    -> TraceChainSelStarvationEvent blk -> Bool)
-> Eq (TraceChainSelStarvationEvent blk)
forall blk.
StandardHash blk =>
TraceChainSelStarvationEvent blk
-> TraceChainSelStarvationEvent blk -> Bool
forall a. (a -> a -> Bool) -> (a -> a -> Bool) -> Eq a
$c== :: forall blk.
StandardHash blk =>
TraceChainSelStarvationEvent blk
-> TraceChainSelStarvationEvent blk -> Bool
== :: TraceChainSelStarvationEvent blk
-> TraceChainSelStarvationEvent blk -> Bool
$c/= :: forall blk.
StandardHash blk =>
TraceChainSelStarvationEvent blk
-> TraceChainSelStarvationEvent blk -> Bool
/= :: TraceChainSelStarvationEvent blk
-> TraceChainSelStarvationEvent blk -> Bool
Eq, Int -> TraceChainSelStarvationEvent blk -> String -> String
[TraceChainSelStarvationEvent blk] -> String -> String
TraceChainSelStarvationEvent blk -> String
(Int -> TraceChainSelStarvationEvent blk -> String -> String)
-> (TraceChainSelStarvationEvent blk -> String)
-> ([TraceChainSelStarvationEvent blk] -> String -> String)
-> Show (TraceChainSelStarvationEvent blk)
forall blk.
StandardHash blk =>
Int -> TraceChainSelStarvationEvent blk -> String -> String
forall blk.
StandardHash blk =>
[TraceChainSelStarvationEvent blk] -> String -> String
forall blk.
StandardHash blk =>
TraceChainSelStarvationEvent blk -> String
forall a.
(Int -> a -> String -> String)
-> (a -> String) -> ([a] -> String -> String) -> Show a
$cshowsPrec :: forall blk.
StandardHash blk =>
Int -> TraceChainSelStarvationEvent blk -> String -> String
showsPrec :: Int -> TraceChainSelStarvationEvent blk -> String -> String
$cshow :: forall blk.
StandardHash blk =>
TraceChainSelStarvationEvent blk -> String
show :: TraceChainSelStarvationEvent blk -> String
$cshowList :: forall blk.
StandardHash blk =>
[TraceChainSelStarvationEvent blk] -> String -> String
showList :: [TraceChainSelStarvationEvent blk] -> String -> String
Show)