{-# LANGUAGE FlexibleContexts #-}
{-# LANGUAGE LambdaCase #-}
{-# LANGUAGE OverloadedStrings #-}
{-# LANGUAGE RecordWildCards #-}
{-# LANGUAGE ScopedTypeVariables #-}
{-# LANGUAGE TypeApplications #-}

module Ouroboros.Consensus.Storage.ChainDB.Impl (
    -- * Initialization
    ChainDbArgs (..)
  , SerialiseDiskConstraints
  , defaultArgs
  , openDB
  , withDB
    -- * Trace types
  , SelectionChangedInfo (..)
  , TraceAddBlockEvent (..)
  , TraceChainSelStarvationEvent (..)
  , TraceCopyToImmutableDBEvent (..)
  , TraceEvent (..)
  , TraceFollowerEvent (..)
  , TraceGCEvent (..)
  , TraceInitChainSelEvent (..)
  , TraceIteratorEvent (..)
  , TraceOpenEvent (..)
  , TracePipeliningEvent (..)
  , TraceValidationEvent (..)
    -- * Re-exported for convenience
  , Args.RelativeMountPoint (..)
  , ImmutableDB.ImmutableDbSerialiseConstraints
  , LedgerDB.LedgerDbSerialiseConstraints
  , VolatileDB.VolatileDbSerialiseConstraints
    -- * Internals for testing purposes
  , Internal (..)
  , openDBInternal
  ) where

import           Control.Monad (void, when)
import           Control.Monad.Trans.Class (lift)
import           Control.ResourceRegistry (WithTempRegistry, allocate,
                     runInnerWithTempRegistry, runWithTempRegistry,
                     withRegistry)
import           Control.Tracer
import           Data.Functor ((<&>))
import           Data.Functor.Contravariant ((>$<))
import qualified Data.Map.Strict as Map
import           Data.Maybe.Strict (StrictMaybe (..))
import           GHC.Stack (HasCallStack)
import           Ouroboros.Consensus.Block
import           Ouroboros.Consensus.Config
import qualified Ouroboros.Consensus.Fragment.Validated as VF
import           Ouroboros.Consensus.HardFork.Abstract
import           Ouroboros.Consensus.HeaderValidation (mkHeaderWithTime)
import           Ouroboros.Consensus.Ledger.Extended (ledgerState)
import           Ouroboros.Consensus.Ledger.Inspect
import           Ouroboros.Consensus.Ledger.SupportsProtocol
import           Ouroboros.Consensus.Storage.ChainDB.API (ChainDB)
import qualified Ouroboros.Consensus.Storage.ChainDB.API as API
import           Ouroboros.Consensus.Storage.ChainDB.Impl.Args (ChainDbArgs,
                     defaultArgs)
import qualified Ouroboros.Consensus.Storage.ChainDB.Impl.Args as Args
import qualified Ouroboros.Consensus.Storage.ChainDB.Impl.Background as Background
import qualified Ouroboros.Consensus.Storage.ChainDB.Impl.ChainSel as ChainSel
import qualified Ouroboros.Consensus.Storage.ChainDB.Impl.Follower as Follower
import qualified Ouroboros.Consensus.Storage.ChainDB.Impl.Iterator as Iterator
import qualified Ouroboros.Consensus.Storage.ChainDB.Impl.Query as Query
import           Ouroboros.Consensus.Storage.ChainDB.Impl.Types
import qualified Ouroboros.Consensus.Storage.ImmutableDB as ImmutableDB
import qualified Ouroboros.Consensus.Storage.ImmutableDB.Stream as ImmutableDB
import           Ouroboros.Consensus.Storage.LedgerDB (LedgerSupportsLedgerDB)
import qualified Ouroboros.Consensus.Storage.LedgerDB as LedgerDB
import qualified Ouroboros.Consensus.Storage.VolatileDB as VolatileDB
import           Ouroboros.Consensus.Util (newFuse, whenJust, withFuse)
import           Ouroboros.Consensus.Util.Args
import           Ouroboros.Consensus.Util.IOLike
import           Ouroboros.Consensus.Util.STM (Fingerprint (..),
                     WithFingerprint (..))
import qualified Ouroboros.Network.AnchoredFragment as AF
import           Ouroboros.Network.BlockFetch.ConsensusInterface
                     (ChainSelStarvation (..))

{-------------------------------------------------------------------------------
  Initialization
-------------------------------------------------------------------------------}

withDB ::
     forall m blk a.
     ( IOLike m
     , LedgerSupportsProtocol blk
     , BlockSupportsDiffusionPipelining blk
     , InspectLedger blk
     , HasHardForkHistory blk
     , ConvertRawHash blk
     , SerialiseDiskConstraints blk
     , LedgerSupportsLedgerDB blk
     )
  => Complete Args.ChainDbArgs m blk
  -> (ChainDB m blk -> m a)
  -> m a
withDB :: forall (m :: * -> *) blk a.
(IOLike m, LedgerSupportsProtocol blk,
 BlockSupportsDiffusionPipelining blk, InspectLedger blk,
 HasHardForkHistory blk, ConvertRawHash blk,
 SerialiseDiskConstraints blk, LedgerSupportsLedgerDB blk) =>
Complete ChainDbArgs m blk -> (ChainDB m blk -> m a) -> m a
withDB Complete ChainDbArgs m blk
args = m (ChainDB m blk)
-> (ChainDB m blk -> m ()) -> (ChainDB m blk -> m a) -> m a
forall a b c. m a -> (a -> m b) -> (a -> m c) -> m c
forall (m :: * -> *) a b c.
MonadThrow m =>
m a -> (a -> m b) -> (a -> m c) -> m c
bracket ((ChainDB m blk, Internal m blk) -> ChainDB m blk
forall a b. (a, b) -> a
fst ((ChainDB m blk, Internal m blk) -> ChainDB m blk)
-> m (ChainDB m blk, Internal m blk) -> m (ChainDB m blk)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Complete ChainDbArgs m blk
-> Bool -> m (ChainDB m blk, Internal m blk)
forall (m :: * -> *) blk.
(IOLike m, LedgerSupportsProtocol blk,
 BlockSupportsDiffusionPipelining blk, InspectLedger blk,
 HasHardForkHistory blk, ConvertRawHash blk,
 SerialiseDiskConstraints blk, HasCallStack,
 LedgerSupportsLedgerDB blk) =>
Complete ChainDbArgs m blk
-> Bool -> m (ChainDB m blk, Internal m blk)
openDBInternal Complete ChainDbArgs m blk
args Bool
True) ChainDB m blk -> m ()
forall (m :: * -> *) blk. ChainDB m blk -> m ()
API.closeDB

openDB ::
     forall m blk.
     ( IOLike m
     , LedgerSupportsProtocol blk
     , BlockSupportsDiffusionPipelining blk
     , InspectLedger blk
     , HasHardForkHistory blk
     , ConvertRawHash blk
     , SerialiseDiskConstraints blk
     , LedgerSupportsLedgerDB blk
     )
  => Complete Args.ChainDbArgs m blk
  -> m (ChainDB m blk)
openDB :: forall (m :: * -> *) blk.
(IOLike m, LedgerSupportsProtocol blk,
 BlockSupportsDiffusionPipelining blk, InspectLedger blk,
 HasHardForkHistory blk, ConvertRawHash blk,
 SerialiseDiskConstraints blk, LedgerSupportsLedgerDB blk) =>
Complete ChainDbArgs m blk -> m (ChainDB m blk)
openDB Complete ChainDbArgs m blk
args = (ChainDB m blk, Internal m blk) -> ChainDB m blk
forall a b. (a, b) -> a
fst ((ChainDB m blk, Internal m blk) -> ChainDB m blk)
-> m (ChainDB m blk, Internal m blk) -> m (ChainDB m blk)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Complete ChainDbArgs m blk
-> Bool -> m (ChainDB m blk, Internal m blk)
forall (m :: * -> *) blk.
(IOLike m, LedgerSupportsProtocol blk,
 BlockSupportsDiffusionPipelining blk, InspectLedger blk,
 HasHardForkHistory blk, ConvertRawHash blk,
 SerialiseDiskConstraints blk, HasCallStack,
 LedgerSupportsLedgerDB blk) =>
Complete ChainDbArgs m blk
-> Bool -> m (ChainDB m blk, Internal m blk)
openDBInternal Complete ChainDbArgs m blk
args Bool
True

openDBInternal ::
     forall m blk.
     ( IOLike m
     , LedgerSupportsProtocol blk
     , BlockSupportsDiffusionPipelining blk
     , InspectLedger blk
     , HasHardForkHistory blk
     , ConvertRawHash blk
     , SerialiseDiskConstraints blk
     , HasCallStack
     , LedgerSupportsLedgerDB blk
     )
  => Complete Args.ChainDbArgs m blk
  -> Bool -- ^ 'True' = Launch background tasks
  -> m (ChainDB m blk, Internal m blk)
openDBInternal :: forall (m :: * -> *) blk.
(IOLike m, LedgerSupportsProtocol blk,
 BlockSupportsDiffusionPipelining blk, InspectLedger blk,
 HasHardForkHistory blk, ConvertRawHash blk,
 SerialiseDiskConstraints blk, HasCallStack,
 LedgerSupportsLedgerDB blk) =>
Complete ChainDbArgs m blk
-> Bool -> m (ChainDB m blk, Internal m blk)
openDBInternal Complete ChainDbArgs m blk
args Bool
launchBgTasks = WithTempRegistry
  (ChainDbEnv m blk)
  m
  ((ChainDB m blk, Internal m blk), ChainDbEnv m blk)
-> m (ChainDB m blk, Internal m blk)
forall (m :: * -> *) st a.
(MonadSTM m, MonadMask m, MonadThread m, HasCallStack) =>
WithTempRegistry st m (a, st) -> m a
runWithTempRegistry (WithTempRegistry
   (ChainDbEnv m blk)
   m
   ((ChainDB m blk, Internal m blk), ChainDbEnv m blk)
 -> m (ChainDB m blk, Internal m blk))
-> WithTempRegistry
     (ChainDbEnv m blk)
     m
     ((ChainDB m blk, Internal m blk), ChainDbEnv m blk)
-> m (ChainDB m blk, Internal m blk)
forall a b. (a -> b) -> a -> b
$ do
    m () -> WithTempRegistry (ChainDbEnv m blk) m ()
forall (m :: * -> *) a.
Monad m =>
m a -> WithTempRegistry (ChainDbEnv m blk) m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (m () -> WithTempRegistry (ChainDbEnv m blk) m ())
-> m () -> WithTempRegistry (ChainDbEnv m blk) m ()
forall a b. (a -> b) -> a -> b
$ Tracer m (TraceEvent blk) -> TraceEvent blk -> m ()
forall (m :: * -> *) a. Tracer m a -> a -> m ()
traceWith Tracer m (TraceEvent blk)
tracer (TraceEvent blk -> m ()) -> TraceEvent blk -> m ()
forall a b. (a -> b) -> a -> b
$ TraceOpenEvent blk -> TraceEvent blk
forall blk. TraceOpenEvent blk -> TraceEvent blk
TraceOpenEvent TraceOpenEvent blk
forall blk. TraceOpenEvent blk
StartedOpeningDB
    m () -> WithTempRegistry (ChainDbEnv m blk) m ()
forall (m :: * -> *) a.
Monad m =>
m a -> WithTempRegistry (ChainDbEnv m blk) m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (m () -> WithTempRegistry (ChainDbEnv m blk) m ())
-> m () -> WithTempRegistry (ChainDbEnv m blk) m ()
forall a b. (a -> b) -> a -> b
$ Tracer m (TraceEvent blk) -> TraceEvent blk -> m ()
forall (m :: * -> *) a. Tracer m a -> a -> m ()
traceWith Tracer m (TraceEvent blk)
tracer (TraceEvent blk -> m ()) -> TraceEvent blk -> m ()
forall a b. (a -> b) -> a -> b
$ TraceOpenEvent blk -> TraceEvent blk
forall blk. TraceOpenEvent blk -> TraceEvent blk
TraceOpenEvent TraceOpenEvent blk
forall blk. TraceOpenEvent blk
StartedOpeningImmutableDB
    immutableDB <- Complete ImmutableDbArgs m blk
-> (forall st.
    WithTempRegistry st m (ImmutableDB m blk, st)
    -> WithTempRegistry (ChainDbEnv m blk) m (ImmutableDB m blk))
-> WithTempRegistry (ChainDbEnv m blk) m (ImmutableDB m blk)
forall (m :: * -> *) blk ans.
(IOLike m, GetPrevHash blk, ConvertRawHash blk,
 ImmutableDbSerialiseConstraints blk, HasCallStack) =>
Complete ImmutableDbArgs m blk
-> (forall st.
    WithTempRegistry st m (ImmutableDB m blk, st) -> ans)
-> ans
ImmutableDB.openDB Complete ImmutableDbArgs m blk
argsImmutableDb ((forall st.
  WithTempRegistry st m (ImmutableDB m blk, st)
  -> WithTempRegistry (ChainDbEnv m blk) m (ImmutableDB m blk))
 -> WithTempRegistry (ChainDbEnv m blk) m (ImmutableDB m blk))
-> (forall st.
    WithTempRegistry st m (ImmutableDB m blk, st)
    -> WithTempRegistry (ChainDbEnv m blk) m (ImmutableDB m blk))
-> WithTempRegistry (ChainDbEnv m blk) m (ImmutableDB m blk)
forall a b. (a -> b) -> a -> b
$ (ImmutableDB m blk -> m ())
-> WithTempRegistry st m (ImmutableDB m blk, st)
-> WithTempRegistry (ChainDbEnv m blk) m (ImmutableDB m blk)
forall (m :: * -> *) innerDB st blk.
IOLike m =>
(innerDB -> m ())
-> WithTempRegistry st m (innerDB, st)
-> WithTempRegistry (ChainDbEnv m blk) m innerDB
innerOpenCont ImmutableDB m blk -> m ()
forall (m :: * -> *) blk. HasCallStack => ImmutableDB m blk -> m ()
ImmutableDB.closeDB
    immutableDbTipPoint <- lift $ atomically $ ImmutableDB.getTipPoint immutableDB
    let immutableDbTipChunk =
          ChunkInfo -> Point blk -> ChunkNo
forall blk. ChunkInfo -> Point blk -> ChunkNo
chunkIndexOfPoint (Complete ImmutableDbArgs m blk -> HKD Identity ChunkInfo
forall (f :: * -> *) (m :: * -> *) blk.
ImmutableDbArgs f m blk -> HKD f ChunkInfo
ImmutableDB.immChunkInfo Complete ImmutableDbArgs m blk
argsImmutableDb) Point blk
immutableDbTipPoint
    lift $ traceWith tracer $
      TraceOpenEvent $
        OpenedImmutableDB immutableDbTipPoint immutableDbTipChunk

    lift $ traceWith tracer $ TraceOpenEvent StartedOpeningVolatileDB
    volatileDB <- VolatileDB.openDB argsVolatileDb $ innerOpenCont VolatileDB.closeDB
    maxSlot <- lift $ atomically $ VolatileDB.getMaxSlotNo volatileDB
    (chainDB, testing, env) <- lift $ do
      traceWith tracer $ TraceOpenEvent (OpenedVolatileDB maxSlot)
      traceWith tracer $ TraceOpenEvent StartedOpeningLgrDB
      (lgrDB, replayed) <- LedgerDB.openDB
                            argsLgrDb
                            (ImmutableDB.streamAPI immutableDB)
                            immutableDbTipPoint
                            (Query.getAnyKnownBlock immutableDB volatileDB)
      traceWith tracer $ TraceOpenEvent OpenedLgrDB

      varInvalid      <- newTVarIO (WithFingerprint Map.empty (Fingerprint 0))

      let initChainSelTracer = TraceInitChainSelEvent blk -> TraceEvent blk
forall blk. TraceInitChainSelEvent blk -> TraceEvent blk
TraceInitChainSelEvent (TraceInitChainSelEvent blk -> TraceEvent blk)
-> Tracer m (TraceEvent blk)
-> Tracer m (TraceInitChainSelEvent blk)
forall (f :: * -> *) a b. Contravariant f => (a -> b) -> f b -> f a
>$< Tracer m (TraceEvent blk)
tracer

      traceWith initChainSelTracer StartedInitChainSelection
      initialLoE     <- Args.cdbsLoE cdbSpecificArgs
      chain <- withRegistry $ \ResourceRegistry m
rr -> do
        chainAndLedger <- ImmutableDB m blk
-> VolatileDB m blk
-> LedgerDB' m blk
-> ResourceRegistry m
-> Tracer m (TraceInitChainSelEvent blk)
-> TopLevelConfig blk
-> StrictTVar
     m (WithFingerprint (Map (HeaderHash blk) (InvalidBlockInfo blk)))
-> LoE ()
-> m (ChainAndLedger m blk)
forall (m :: * -> *) blk.
(IOLike m, LedgerSupportsProtocol blk,
 BlockSupportsDiffusionPipelining blk) =>
ImmutableDB m blk
-> VolatileDB m blk
-> LedgerDB' m blk
-> ResourceRegistry m
-> Tracer m (TraceInitChainSelEvent blk)
-> TopLevelConfig blk
-> StrictTVar m (WithFingerprint (InvalidBlocks blk))
-> LoE ()
-> m (ChainAndLedger m blk)
ChainSel.initialChainSelection
                          ImmutableDB m blk
immutableDB
                          VolatileDB m blk
volatileDB
                          LedgerDB' m blk
lgrDB
                          ResourceRegistry m
rr
                          Tracer m (TraceInitChainSelEvent blk)
initChainSelTracer
                          (ChainDbSpecificArgs Identity m blk
-> HKD Identity (TopLevelConfig blk)
forall (f :: * -> *) (m :: * -> *) blk.
ChainDbSpecificArgs f m blk -> HKD f (TopLevelConfig blk)
Args.cdbsTopLevelConfig ChainDbSpecificArgs Identity m blk
cdbSpecificArgs)
                          StrictTVar
  m (WithFingerprint (Map (HeaderHash blk) (InvalidBlockInfo blk)))
varInvalid
                          (LoE (AnchoredFragment (HeaderWithTime blk)) -> LoE ()
forall (f :: * -> *) a. Functor f => f a -> f ()
void LoE (AnchoredFragment (HeaderWithTime blk))
initialLoE)
        traceWith initChainSelTracer InitialChainSelected

        let chain  = ChainAndLedger m blk -> AnchoredFragment (Header blk)
forall b l. ValidatedFragment b l -> AnchoredFragment b
VF.validatedFragment ChainAndLedger m blk
chainAndLedger
            ledger = ChainAndLedger m blk -> Forker' m blk
forall b l. ValidatedFragment b l -> l
VF.validatedLedger   ChainAndLedger m blk
chainAndLedger

        atomically $ LedgerDB.forkerCommit ledger
        LedgerDB.forkerClose ledger
        pure chain
      LedgerDB.tryFlush lgrDB

      curLedger <- atomically $ LedgerDB.getVolatileTip lgrDB
      let lcfg = TopLevelConfig blk -> LedgerCfg (LedgerState blk)
forall blk. TopLevelConfig blk -> LedgerConfig blk
configLedger (ChainDbSpecificArgs Identity m blk
-> HKD Identity (TopLevelConfig blk)
forall (f :: * -> *) (m :: * -> *) blk.
ChainDbSpecificArgs f m blk -> HKD f (TopLevelConfig blk)
Args.cdbsTopLevelConfig ChainDbSpecificArgs Identity m blk
cdbSpecificArgs)

          -- the volatile tip ledger state can translate the slots of the volatile
          -- headers
          chainWithTime =
            (Header blk -> HeaderWithTime blk)
-> AnchoredFragment (Header blk)
-> AnchoredFragment (HeaderWithTime blk)
forall block2 block1.
(HasHeader block2, HeaderHash block1 ~ HeaderHash block2) =>
(block1 -> block2)
-> AnchoredFragment block1 -> AnchoredFragment block2
AF.mapAnchoredFragment
              (LedgerCfg (LedgerState blk)
-> LedgerState blk EmptyMK -> Header blk -> HeaderWithTime blk
forall blk (mk :: MapKind).
(HasHardForkHistory blk, HasHeader (Header blk)) =>
LedgerConfig blk
-> LedgerState blk mk -> Header blk -> HeaderWithTime blk
mkHeaderWithTime
                 LedgerCfg (LedgerState blk)
lcfg
                 (ExtLedgerState blk EmptyMK -> LedgerState blk EmptyMK
forall blk (mk :: MapKind).
ExtLedgerState blk mk -> LedgerState blk mk
ledgerState ExtLedgerState blk EmptyMK
curLedger)
              )
              AnchoredFragment (Header blk)
chain

      varChain           <- newTVarWithInvariantIO checkInternalChain $ InternalChain chain chainWithTime
      varTentativeState  <- newTVarIO $ initialTentativeHeaderState (Proxy @blk)
      varTentativeHeader <- newTVarIO SNothing
      varIterators       <- newTVarIO Map.empty
      varFollowers       <- newTVarIO Map.empty
      varNextIteratorKey <- newTVarIO (IteratorKey 0)
      varNextFollowerKey <- newTVarIO (FollowerKey   0)
      varKillBgThreads   <- newTVarIO $ return ()
      copyFuse           <- newFuse "copy to immutable db"
      chainSelFuse       <- newFuse "chain selection"
      chainSelQueue      <- newChainSelQueue (Args.cdbsBlocksToAddSize cdbSpecificArgs)
      varChainSelStarvation <- newTVarIO ChainSelStarvationOngoing

      let env = CDB { cdbImmutableDB :: ImmutableDB m blk
cdbImmutableDB     = ImmutableDB m blk
immutableDB
                    , cdbVolatileDB :: VolatileDB m blk
cdbVolatileDB      = VolatileDB m blk
volatileDB
                    , cdbLedgerDB :: LedgerDB' m blk
cdbLedgerDB        = LedgerDB' m blk
lgrDB
                    , cdbChain :: StrictTVar m (InternalChain blk)
cdbChain           = StrictTVar m (InternalChain blk)
varChain
                    , cdbTentativeState :: StrictTVar m (TentativeHeaderState blk)
cdbTentativeState  = StrictTVar m (TentativeHeaderState blk)
varTentativeState
                    , cdbTentativeHeader :: StrictTVar m (StrictMaybe (Header blk))
cdbTentativeHeader = StrictTVar m (StrictMaybe (Header blk))
varTentativeHeader
                    , cdbIterators :: StrictTVar m (Map IteratorKey (m ()))
cdbIterators       = StrictTVar m (Map IteratorKey (m ()))
varIterators
                    , cdbFollowers :: StrictTVar m (Map FollowerKey (FollowerHandle m blk))
cdbFollowers       = StrictTVar m (Map FollowerKey (FollowerHandle m blk))
varFollowers
                    , cdbTopLevelConfig :: TopLevelConfig blk
cdbTopLevelConfig  = ChainDbSpecificArgs Identity m blk
-> HKD Identity (TopLevelConfig blk)
forall (f :: * -> *) (m :: * -> *) blk.
ChainDbSpecificArgs f m blk -> HKD f (TopLevelConfig blk)
Args.cdbsTopLevelConfig ChainDbSpecificArgs Identity m blk
cdbSpecificArgs
                    , cdbInvalid :: StrictTVar
  m (WithFingerprint (Map (HeaderHash blk) (InvalidBlockInfo blk)))
cdbInvalid         = StrictTVar
  m (WithFingerprint (Map (HeaderHash blk) (InvalidBlockInfo blk)))
varInvalid
                    , cdbNextIteratorKey :: StrictTVar m IteratorKey
cdbNextIteratorKey = StrictTVar m IteratorKey
varNextIteratorKey
                    , cdbNextFollowerKey :: StrictTVar m FollowerKey
cdbNextFollowerKey = StrictTVar m FollowerKey
varNextFollowerKey
                    , cdbCopyFuse :: Fuse m
cdbCopyFuse        = Fuse m
copyFuse
                    , cdbChainSelFuse :: Fuse m
cdbChainSelFuse    = Fuse m
chainSelFuse
                    , cdbTracer :: Tracer m (TraceEvent blk)
cdbTracer          = Tracer m (TraceEvent blk)
tracer
                    , cdbRegistry :: ResourceRegistry m
cdbRegistry        = ChainDbSpecificArgs Identity m blk
-> HKD Identity (ResourceRegistry m)
forall (f :: * -> *) (m :: * -> *) blk.
ChainDbSpecificArgs f m blk -> HKD f (ResourceRegistry m)
Args.cdbsRegistry ChainDbSpecificArgs Identity m blk
cdbSpecificArgs
                    , cdbGcDelay :: DiffTime
cdbGcDelay         = ChainDbSpecificArgs Identity m blk -> DiffTime
forall (f :: * -> *) (m :: * -> *) blk.
ChainDbSpecificArgs f m blk -> DiffTime
Args.cdbsGcDelay ChainDbSpecificArgs Identity m blk
cdbSpecificArgs
                    , cdbGcInterval :: DiffTime
cdbGcInterval      = ChainDbSpecificArgs Identity m blk -> DiffTime
forall (f :: * -> *) (m :: * -> *) blk.
ChainDbSpecificArgs f m blk -> DiffTime
Args.cdbsGcInterval ChainDbSpecificArgs Identity m blk
cdbSpecificArgs
                    , cdbKillBgThreads :: StrictTVar m (m ())
cdbKillBgThreads   = StrictTVar m (m ())
varKillBgThreads
                    , cdbChainSelQueue :: ChainSelQueue m blk
cdbChainSelQueue   = ChainSelQueue m blk
chainSelQueue
                    , cdbLoE :: GetLoEFragment m blk
cdbLoE             = ChainDbSpecificArgs Identity m blk -> GetLoEFragment m blk
forall (f :: * -> *) (m :: * -> *) blk.
ChainDbSpecificArgs f m blk -> GetLoEFragment m blk
Args.cdbsLoE ChainDbSpecificArgs Identity m blk
cdbSpecificArgs
                    , cdbChainSelStarvation :: StrictTVar m ChainSelStarvation
cdbChainSelStarvation = StrictTVar m ChainSelStarvation
varChainSelStarvation
                    }
      h <- fmap CDBHandle $ newTVarIO $ ChainDbOpen env
      let chainDB = API.ChainDB
            { addBlockAsync :: InvalidBlockPunishment m -> blk -> m (AddBlockPromise m blk)
addBlockAsync            = ChainDbHandle m blk
-> (ChainDbEnv m blk
    -> InvalidBlockPunishment m -> blk -> m (AddBlockPromise m blk))
-> InvalidBlockPunishment m
-> blk
-> m (AddBlockPromise m blk)
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
-> InvalidBlockPunishment m -> blk -> m (AddBlockPromise m blk)
forall (m :: * -> *) blk.
(IOLike m, HasHeader blk) =>
ChainDbEnv m blk
-> InvalidBlockPunishment m -> blk -> m (AddBlockPromise m blk)
ChainSel.addBlockAsync
            , chainSelAsync :: m (ChainSelectionPromise m)
chainSelAsync         = ChainDbHandle m blk
-> (ChainDbEnv m blk -> m (ChainSelectionPromise m))
-> m (ChainSelectionPromise m)
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 -> m (ChainSelectionPromise m)
forall (m :: * -> *) blk.
IOLike m =>
ChainDbEnv m blk -> m (ChainSelectionPromise m)
ChainSel.triggerChainSelectionAsync
            , getCurrentChain :: STM m (AnchoredFragment (Header blk))
getCurrentChain          = ChainDbHandle m blk
-> (ChainDbEnv m blk -> STM m (AnchoredFragment (Header blk)))
-> STM m (AnchoredFragment (Header blk))
forall (m :: * -> *) blk r.
(IOLike m, HasCallStack, HasHeader blk) =>
ChainDbHandle m blk -> (ChainDbEnv m blk -> STM m r) -> STM m r
getEnvSTM  ChainDbHandle m blk
h ChainDbEnv m blk -> STM m (AnchoredFragment (Header blk))
forall (m :: * -> *) blk.
(IOLike m, HasHeader (Header blk),
 ConsensusProtocol (BlockProtocol blk)) =>
ChainDbEnv m blk -> STM m (AnchoredFragment (Header blk))
Query.getCurrentChain
            , getCurrentChainWithTime :: STM m (AnchoredFragment (HeaderWithTime blk))
getCurrentChainWithTime  = ChainDbHandle m blk
-> (ChainDbEnv m blk
    -> STM m (AnchoredFragment (HeaderWithTime blk)))
-> STM m (AnchoredFragment (HeaderWithTime blk))
forall (m :: * -> *) blk r.
(IOLike m, HasCallStack, HasHeader blk) =>
ChainDbHandle m blk -> (ChainDbEnv m blk -> STM m r) -> STM m r
getEnvSTM  ChainDbHandle m blk
h ChainDbEnv m blk -> STM m (AnchoredFragment (HeaderWithTime blk))
forall (m :: * -> *) blk.
(IOLike m, HasHeader (HeaderWithTime blk),
 ConsensusProtocol (BlockProtocol blk)) =>
ChainDbEnv m blk -> STM m (AnchoredFragment (HeaderWithTime blk))
Query.getCurrentChainWithTime
            , getTipBlock :: m (Maybe blk)
getTipBlock              = ChainDbHandle m blk
-> (ChainDbEnv m blk -> m (Maybe blk)) -> m (Maybe blk)
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 -> m (Maybe blk)
forall (m :: * -> *) blk.
(IOLike m, HasHeader blk, HasHeader (Header blk)) =>
ChainDbEnv m blk -> m (Maybe blk)
Query.getTipBlock
            , getTipHeader :: m (Maybe (Header blk))
getTipHeader             = ChainDbHandle m blk
-> (ChainDbEnv m blk -> m (Maybe (Header blk)))
-> m (Maybe (Header blk))
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 -> m (Maybe (Header blk))
forall (m :: * -> *) blk.
(IOLike m, HasHeader blk, HasHeader (Header blk)) =>
ChainDbEnv m blk -> m (Maybe (Header blk))
Query.getTipHeader
            , getTipPoint :: STM m (Point blk)
getTipPoint              = ChainDbHandle m blk
-> (ChainDbEnv m blk -> STM m (Point blk)) -> STM m (Point blk)
forall (m :: * -> *) blk r.
(IOLike m, HasCallStack, HasHeader blk) =>
ChainDbHandle m blk -> (ChainDbEnv m blk -> STM m r) -> STM m r
getEnvSTM  ChainDbHandle m blk
h ChainDbEnv m blk -> STM m (Point blk)
forall (m :: * -> *) blk.
(IOLike m, HasHeader (Header blk)) =>
ChainDbEnv m blk -> STM m (Point blk)
Query.getTipPoint
            , getBlockComponent :: forall b. BlockComponent blk b -> RealPoint blk -> m (Maybe b)
getBlockComponent        = ChainDbHandle m blk
-> (ChainDbEnv m blk
    -> BlockComponent blk b -> RealPoint blk -> m (Maybe b))
-> BlockComponent blk b
-> RealPoint blk
-> m (Maybe b)
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
-> BlockComponent blk b -> RealPoint blk -> m (Maybe b)
forall (m :: * -> *) blk b.
IOLike m =>
ChainDbEnv m blk
-> BlockComponent blk b -> RealPoint blk -> m (Maybe b)
Query.getBlockComponent
            , getIsFetched :: STM m (Point blk -> Bool)
getIsFetched             = ChainDbHandle m blk
-> (ChainDbEnv m blk -> STM m (Point blk -> Bool))
-> STM m (Point blk -> Bool)
forall (m :: * -> *) blk r.
(IOLike m, HasCallStack, HasHeader blk) =>
ChainDbHandle m blk -> (ChainDbEnv m blk -> STM m r) -> STM m r
getEnvSTM  ChainDbHandle m blk
h ChainDbEnv m blk -> STM m (Point blk -> Bool)
forall (m :: * -> *) blk.
(IOLike m, HasHeader blk) =>
ChainDbEnv m blk -> STM m (Point blk -> Bool)
Query.getIsFetched
            , getIsValid :: STM m (RealPoint blk -> Maybe Bool)
getIsValid               = ChainDbHandle m blk
-> (ChainDbEnv m blk -> STM m (RealPoint blk -> Maybe Bool))
-> STM m (RealPoint blk -> Maybe Bool)
forall (m :: * -> *) blk r.
(IOLike m, HasCallStack, HasHeader blk) =>
ChainDbHandle m blk -> (ChainDbEnv m blk -> STM m r) -> STM m r
getEnvSTM  ChainDbHandle m blk
h ChainDbEnv m blk -> STM m (RealPoint blk -> Maybe Bool)
forall (m :: * -> *) blk.
(IOLike m, HasHeader blk) =>
ChainDbEnv m blk -> STM m (RealPoint blk -> Maybe Bool)
Query.getIsValid
            , getMaxSlotNo :: STM m MaxSlotNo
getMaxSlotNo             = ChainDbHandle m blk
-> (ChainDbEnv m blk -> STM m MaxSlotNo) -> STM m MaxSlotNo
forall (m :: * -> *) blk r.
(IOLike m, HasCallStack, HasHeader blk) =>
ChainDbHandle m blk -> (ChainDbEnv m blk -> STM m r) -> STM m r
getEnvSTM  ChainDbHandle m blk
h ChainDbEnv m blk -> STM m MaxSlotNo
forall (m :: * -> *) blk.
(IOLike m, HasHeader (Header blk)) =>
ChainDbEnv m blk -> STM m MaxSlotNo
Query.getMaxSlotNo
            , stream :: forall b.
ResourceRegistry m
-> BlockComponent blk b
-> StreamFrom blk
-> StreamTo blk
-> m (Either (UnknownRange blk) (Iterator m blk b))
stream                   = ChainDbHandle m blk
-> ResourceRegistry m
-> BlockComponent blk b
-> StreamFrom blk
-> StreamTo blk
-> m (Either (UnknownRange blk) (Iterator m blk b))
forall (m :: * -> *) blk b.
(IOLike m, HasHeader blk, HasCallStack) =>
ChainDbHandle m blk
-> ResourceRegistry m
-> BlockComponent blk b
-> StreamFrom blk
-> StreamTo blk
-> m (Either (UnknownRange blk) (Iterator m blk b))
Iterator.stream  ChainDbHandle m blk
h
            , newFollower :: forall b.
ResourceRegistry m
-> ChainType -> BlockComponent blk b -> m (Follower m blk b)
newFollower              = ChainDbHandle m blk
-> ResourceRegistry m
-> ChainType
-> BlockComponent blk b
-> m (Follower m blk b)
forall (m :: * -> *) blk b.
(IOLike m, HasHeader blk, GetHeader blk,
 HasNestedContent Header blk,
 EncodeDiskDep (NestedCtxt Header) blk) =>
ChainDbHandle m blk
-> ResourceRegistry m
-> ChainType
-> BlockComponent blk b
-> m (Follower m blk b)
Follower.newFollower ChainDbHandle m blk
h
            , getIsInvalidBlock :: STM
  m
  (WithFingerprint
     (HeaderHash blk -> Maybe (ExtValidationError blk)))
getIsInvalidBlock        = ChainDbHandle m blk
-> (ChainDbEnv m blk
    -> STM
         m
         (WithFingerprint
            (HeaderHash blk -> Maybe (ExtValidationError blk))))
-> STM
     m
     (WithFingerprint
        (HeaderHash blk -> Maybe (ExtValidationError blk)))
forall (m :: * -> *) blk r.
(IOLike m, HasCallStack, HasHeader blk) =>
ChainDbHandle m blk -> (ChainDbEnv m blk -> STM m r) -> STM m r
getEnvSTM  ChainDbHandle m blk
h ChainDbEnv m blk
-> STM
     m
     (WithFingerprint
        (HeaderHash blk -> Maybe (ExtValidationError blk)))
forall (m :: * -> *) blk.
(IOLike m, HasHeader blk) =>
ChainDbEnv m blk
-> STM
     m
     (WithFingerprint
        (HeaderHash blk -> Maybe (ExtValidationError blk)))
Query.getIsInvalidBlock
            , getChainSelStarvation :: STM m ChainSelStarvation
getChainSelStarvation    = ChainDbHandle m blk
-> (ChainDbEnv m blk -> STM m ChainSelStarvation)
-> STM m ChainSelStarvation
forall (m :: * -> *) blk r.
(IOLike m, HasCallStack, HasHeader blk) =>
ChainDbHandle m blk -> (ChainDbEnv m blk -> STM m r) -> STM m r
getEnvSTM  ChainDbHandle m blk
h ChainDbEnv m blk -> STM m ChainSelStarvation
forall (m :: * -> *) blk.
IOLike m =>
ChainDbEnv m blk -> STM m ChainSelStarvation
Query.getChainSelStarvation
            , closeDB :: m ()
closeDB                  = ChainDbHandle m blk -> m ()
forall (m :: * -> *) blk.
(IOLike m, HasHeader (Header blk), HasCallStack) =>
ChainDbHandle m blk -> m ()
closeDB ChainDbHandle m blk
h
            , isOpen :: STM m Bool
isOpen                   = ChainDbHandle m blk -> STM m Bool
forall (m :: * -> *) blk.
IOLike m =>
ChainDbHandle m blk -> STM m Bool
isOpen  ChainDbHandle m blk
h
            , getCurrentLedger :: STM m (ExtLedgerState blk EmptyMK)
getCurrentLedger         = ChainDbHandle m blk
-> (ChainDbEnv m blk -> STM m (ExtLedgerState blk EmptyMK))
-> STM m (ExtLedgerState blk EmptyMK)
forall (m :: * -> *) blk r.
(IOLike m, HasCallStack, HasHeader blk) =>
ChainDbHandle m blk -> (ChainDbEnv m blk -> STM m r) -> STM m r
getEnvSTM  ChainDbHandle m blk
h ChainDbEnv m blk -> STM m (ExtLedgerState blk EmptyMK)
forall (m :: * -> *) blk.
ChainDbEnv m blk -> STM m (ExtLedgerState blk EmptyMK)
Query.getCurrentLedger
            , getImmutableLedger :: STM m (ExtLedgerState blk EmptyMK)
getImmutableLedger       = ChainDbHandle m blk
-> (ChainDbEnv m blk -> STM m (ExtLedgerState blk EmptyMK))
-> STM m (ExtLedgerState blk EmptyMK)
forall (m :: * -> *) blk r.
(IOLike m, HasCallStack, HasHeader blk) =>
ChainDbHandle m blk -> (ChainDbEnv m blk -> STM m r) -> STM m r
getEnvSTM  ChainDbHandle m blk
h ChainDbEnv m blk -> STM m (ExtLedgerState blk EmptyMK)
forall (m :: * -> *) blk.
ChainDbEnv m blk -> STM m (ExtLedgerState blk EmptyMK)
Query.getImmutableLedger
            , getPastLedger :: Point blk -> STM m (Maybe (ExtLedgerState blk EmptyMK))
getPastLedger            = ChainDbHandle m blk
-> (ChainDbEnv m blk
    -> Point blk -> STM m (Maybe (ExtLedgerState blk EmptyMK)))
-> Point blk
-> STM m (Maybe (ExtLedgerState blk EmptyMK))
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 ChainDbHandle m blk
h ChainDbEnv m blk
-> Point blk -> STM m (Maybe (ExtLedgerState blk EmptyMK))
forall (m :: * -> *) blk.
ChainDbEnv m blk
-> Point blk -> STM m (Maybe (ExtLedgerState blk EmptyMK))
Query.getPastLedger
            , getHeaderStateHistory :: STM m (HeaderStateHistory blk)
getHeaderStateHistory    = ChainDbHandle m blk
-> (ChainDbEnv m blk -> STM m (HeaderStateHistory blk))
-> STM m (HeaderStateHistory blk)
forall (m :: * -> *) blk r.
(IOLike m, HasCallStack, HasHeader blk) =>
ChainDbHandle m blk -> (ChainDbEnv m blk -> STM m r) -> STM m r
getEnvSTM  ChainDbHandle m blk
h ChainDbEnv m blk -> STM m (HeaderStateHistory blk)
forall (m :: * -> *) blk.
ChainDbEnv m blk -> STM m (HeaderStateHistory blk)
Query.getHeaderStateHistory
            , getReadOnlyForkerAtPoint :: ResourceRegistry m
-> Target (Point blk)
-> m (Either GetForkerError (ReadOnlyForker' m blk))
getReadOnlyForkerAtPoint = ChainDbHandle m blk
-> (ChainDbEnv m blk
    -> ResourceRegistry m
    -> Target (Point blk)
    -> m (Either GetForkerError (ReadOnlyForker' m blk)))
-> ResourceRegistry m
-> Target (Point blk)
-> m (Either GetForkerError (ReadOnlyForker' m blk))
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
-> ResourceRegistry m
-> Target (Point blk)
-> m (Either GetForkerError (ReadOnlyForker' m blk))
forall (m :: * -> *) blk.
IOLike m =>
ChainDbEnv m blk
-> ResourceRegistry m
-> Target (Point blk)
-> m (Either GetForkerError (ReadOnlyForker' m blk))
Query.getReadOnlyForkerAtPoint
            , getLedgerTablesAtFor :: Point blk
-> LedgerTables (ExtLedgerState blk) KeysMK
-> m (Maybe (LedgerTables (ExtLedgerState blk) ValuesMK))
getLedgerTablesAtFor     = ChainDbHandle m blk
-> (ChainDbEnv m blk
    -> Point blk
    -> LedgerTables (ExtLedgerState blk) KeysMK
    -> m (Maybe (LedgerTables (ExtLedgerState blk) ValuesMK)))
-> Point blk
-> LedgerTables (ExtLedgerState blk) KeysMK
-> m (Maybe (LedgerTables (ExtLedgerState blk) ValuesMK))
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
-> Point blk
-> LedgerTables (ExtLedgerState blk) KeysMK
-> m (Maybe (LedgerTables (ExtLedgerState blk) ValuesMK))
forall (m :: * -> *) blk.
IOLike m =>
ChainDbEnv m blk
-> Point blk
-> LedgerTables (ExtLedgerState blk) KeysMK
-> m (Maybe (LedgerTables (ExtLedgerState blk) ValuesMK))
Query.getLedgerTablesAtFor
            , getStatistics :: m (Maybe Statistics)
getStatistics            = ChainDbHandle m blk
-> (ChainDbEnv m blk -> m (Maybe Statistics))
-> m (Maybe Statistics)
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 -> m (Maybe Statistics)
forall (m :: * -> *) blk.
IOLike m =>
ChainDbEnv m blk -> m (Maybe Statistics)
Query.getStatistics
            }
      addBlockTestFuse <- newFuse "test chain selection"
      copyTestFuse <- newFuse "test copy to immutable db"
      let testing = Internal
            { intCopyToImmutableDB :: m (WithOrigin SlotNo)
intCopyToImmutableDB = ChainDbHandle m blk
-> (ChainDbEnv m blk -> m (WithOrigin SlotNo))
-> m (WithOrigin SlotNo)
forall (m :: * -> *) blk r.
(IOLike m, HasCallStack, HasHeader blk) =>
ChainDbHandle m blk -> (ChainDbEnv m blk -> m r) -> m r
getEnv  ChainDbHandle m blk
h (Fuse m -> Electric m (WithOrigin SlotNo) -> m (WithOrigin SlotNo)
forall (m :: * -> *) a.
(MonadThrow m, MonadMVar m) =>
Fuse m -> Electric m a -> m a
withFuse Fuse m
copyTestFuse (Electric m (WithOrigin SlotNo) -> m (WithOrigin SlotNo))
-> (ChainDbEnv m blk -> Electric m (WithOrigin SlotNo))
-> ChainDbEnv m blk
-> m (WithOrigin SlotNo)
forall b c a. (b -> c) -> (a -> b) -> a -> c
. ChainDbEnv m blk -> Electric m (WithOrigin SlotNo)
forall (m :: * -> *) blk.
(IOLike m, ConsensusProtocol (BlockProtocol blk), HasHeader blk,
 GetHeader blk, HasCallStack) =>
ChainDbEnv m blk -> Electric m (WithOrigin SlotNo)
Background.copyToImmutableDB)
            , intGarbageCollect :: SlotNo -> m ()
intGarbageCollect    = ChainDbHandle m blk
-> (ChainDbEnv m blk -> SlotNo -> m ()) -> SlotNo -> m ()
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 -> SlotNo -> m ()
forall (m :: * -> *) blk.
IOLike m =>
ChainDbEnv m blk -> SlotNo -> m ()
Background.garbageCollect
            , intTryTakeSnapshot :: m ()
intTryTakeSnapshot   = ChainDbHandle m blk -> (ChainDbEnv m blk -> m ()) -> m ()
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 -> m ()) -> m ())
-> (ChainDbEnv m blk -> m ()) -> m ()
forall a b. (a -> b) -> a -> b
$ \ChainDbEnv m blk
env' ->
                m SnapCounters -> m ()
forall (f :: * -> *) a. Functor f => f a -> f ()
void (m SnapCounters -> m ()) -> m SnapCounters -> m ()
forall a b. (a -> b) -> a -> b
$ LedgerDB' m blk
-> (ExtLedgerState blk ~ ExtLedgerState blk) =>
   Maybe (Time, Time) -> Word64 -> m SnapCounters
forall (m :: * -> *) (l :: LedgerStateKind) blk.
LedgerDB m l blk
-> (l ~ ExtLedgerState blk) =>
   Maybe (Time, Time) -> Word64 -> m SnapCounters
LedgerDB.tryTakeSnapshot (ChainDbEnv m blk -> LedgerDB' m blk
forall (m :: * -> *) blk. ChainDbEnv m blk -> LedgerDB' m blk
cdbLedgerDB ChainDbEnv m blk
env') Maybe (Time, Time)
forall a. Maybe a
Nothing Word64
forall a. Bounded a => a
maxBound
            , intAddBlockRunner :: m Void
intAddBlockRunner    = ChainDbHandle m blk -> (ChainDbEnv m blk -> m Void) -> m Void
forall (m :: * -> *) blk r.
(IOLike m, HasCallStack, HasHeader blk) =>
ChainDbHandle m blk -> (ChainDbEnv m blk -> m r) -> m r
getEnv  ChainDbHandle m blk
h (Fuse m -> ChainDbEnv m blk -> m Void
forall (m :: * -> *) blk.
(IOLike m, LedgerSupportsProtocol blk,
 BlockSupportsDiffusionPipelining blk, InspectLedger blk,
 HasHardForkHistory blk, HasCallStack) =>
Fuse m -> ChainDbEnv m blk -> m Void
Background.addBlockRunner Fuse m
addBlockTestFuse)
            , intKillBgThreads :: StrictTVar m (m ())
intKillBgThreads     = StrictTVar m (m ())
varKillBgThreads
            }

      traceWith tracer $ TraceOpenEvent $ OpenedDB
        (castPoint $ AF.anchorPoint chain)
        (castPoint $ AF.headPoint   chain)

      when launchBgTasks $ Background.launchBgTasks env replayed

      return (chainDB, testing, env)

    _ <- lift $ allocate (Args.cdbsRegistry cdbSpecificArgs) (\ResourceId
_ -> ChainDB m blk -> m (ChainDB m blk)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return ChainDB m blk
chainDB) API.closeDB

    return ((chainDB, testing), env)
  where
    tracer :: Tracer m (TraceEvent blk)
tracer = ChainDbSpecificArgs Identity m blk -> Tracer m (TraceEvent blk)
forall (f :: * -> *) (m :: * -> *) blk.
ChainDbSpecificArgs f m blk -> Tracer m (TraceEvent blk)
Args.cdbsTracer ChainDbSpecificArgs Identity m blk
cdbSpecificArgs
    Args.ChainDbArgs Complete ImmutableDbArgs m blk
argsImmutableDb Complete VolatileDbArgs m blk
argsVolatileDb Complete LedgerDbArgs m blk
argsLgrDb ChainDbSpecificArgs Identity m blk
cdbSpecificArgs = Complete ChainDbArgs m blk
args

-- | We use 'runInnerWithTempRegistry' for the component databases.
innerOpenCont ::
     IOLike m
  => (innerDB -> m ())
  -> WithTempRegistry st m (innerDB, st)
  -> WithTempRegistry (ChainDbEnv m blk) m innerDB
innerOpenCont :: forall (m :: * -> *) innerDB st blk.
IOLike m =>
(innerDB -> m ())
-> WithTempRegistry st m (innerDB, st)
-> WithTempRegistry (ChainDbEnv m blk) m innerDB
innerOpenCont innerDB -> m ()
closer WithTempRegistry st m (innerDB, st)
m =
  WithTempRegistry st m (innerDB, st, innerDB)
-> (innerDB -> m Bool)
-> (ChainDbEnv m blk -> innerDB -> Bool)
-> WithTempRegistry (ChainDbEnv m blk) m innerDB
forall innerSt st (m :: * -> *) res a.
(MonadSTM m, MonadMask m, MonadThread m) =>
WithTempRegistry innerSt m (a, innerSt, res)
-> (res -> m Bool)
-> (st -> res -> Bool)
-> WithTempRegistry st m a
runInnerWithTempRegistry
    (((innerDB, st) -> (innerDB, st, innerDB))
-> WithTempRegistry st m (innerDB, st)
-> WithTempRegistry st m (innerDB, st, innerDB)
forall a b.
(a -> b) -> WithTempRegistry st m a -> WithTempRegistry st m b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap (\(innerDB
innerDB, st
st) -> (innerDB
innerDB, st
st, innerDB
innerDB)) WithTempRegistry st m (innerDB, st)
m)
    ((Bool
True Bool -> m () -> m Bool
forall a b. a -> m b -> m a
forall (f :: * -> *) a b. Functor f => a -> f b -> f a
<$) (m () -> m Bool) -> (innerDB -> m ()) -> innerDB -> m Bool
forall b c a. (b -> c) -> (a -> b) -> a -> c
. innerDB -> m ()
closer)
    (\ChainDbEnv m blk
_env innerDB
_innerDB -> Bool
True)
      -- This check is degenerate because handles in @_env@ and the
      -- @_innerDB@ handle do not support an equality check; all of the
      -- identifying data is only in the handle's closure, not
      -- accessible because of our intentional encapsulation choices.

isOpen :: IOLike m => ChainDbHandle m blk -> STM m Bool
isOpen :: forall (m :: * -> *) blk.
IOLike m =>
ChainDbHandle m blk -> STM m Bool
isOpen (CDBHandle StrictTVar m (ChainDbState m blk)
varState) = 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 -> Bool) -> STM m Bool
forall (f :: * -> *) a b. Functor f => f a -> (a -> b) -> f b
<&> \case
    ChainDbState m blk
ChainDbClosed    -> Bool
False
    ChainDbOpen ChainDbEnv m blk
_env -> Bool
True

closeDB ::
     forall m blk.
     ( IOLike m
     , HasHeader (Header blk)
     , HasCallStack
     )
  => ChainDbHandle m blk -> m ()
closeDB :: forall (m :: * -> *) blk.
(IOLike m, HasHeader (Header blk), HasCallStack) =>
ChainDbHandle m blk -> m ()
closeDB (CDBHandle StrictTVar m (ChainDbState m blk)
varState) = do
    mbOpenEnv <- STM m (Maybe (ChainDbEnv m blk)) -> m (Maybe (ChainDbEnv m blk))
forall a. HasCallStack => STM m a -> m a
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
STM m a -> m a
atomically (STM m (Maybe (ChainDbEnv m blk)) -> m (Maybe (ChainDbEnv m blk)))
-> STM m (Maybe (ChainDbEnv m blk)) -> m (Maybe (ChainDbEnv m blk))
forall a b. (a -> b) -> a -> b
$ 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 (Maybe (ChainDbEnv m blk)))
-> STM m (Maybe (ChainDbEnv m blk))
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
      -- Idempotent
      ChainDbState m blk
ChainDbClosed   -> Maybe (ChainDbEnv m blk) -> STM m (Maybe (ChainDbEnv m blk))
forall a. a -> STM m a
forall (m :: * -> *) a. Monad m => a -> m a
return Maybe (ChainDbEnv m blk)
forall a. Maybe a
Nothing
      ChainDbOpen ChainDbEnv m blk
env -> do
        StrictTVar m (ChainDbState m blk) -> ChainDbState m blk -> STM m ()
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
StrictTVar m a -> a -> STM m ()
writeTVar StrictTVar m (ChainDbState m blk)
varState ChainDbState m blk
forall (m :: * -> *) blk. ChainDbState m blk
ChainDbClosed
        Maybe (ChainDbEnv m blk) -> STM m (Maybe (ChainDbEnv m blk))
forall a. a -> STM m a
forall (m :: * -> *) a. Monad m => a -> m a
return (Maybe (ChainDbEnv m blk) -> STM m (Maybe (ChainDbEnv m blk)))
-> Maybe (ChainDbEnv m blk) -> STM m (Maybe (ChainDbEnv m blk))
forall a b. (a -> b) -> a -> b
$ ChainDbEnv m blk -> Maybe (ChainDbEnv m blk)
forall a. a -> Maybe a
Just ChainDbEnv m blk
env

    -- Only when the ChainDB was open
    whenJust mbOpenEnv $ \cdb :: ChainDbEnv m blk
cdb@CDB{m (LoE (AnchoredFragment (HeaderWithTime blk)))
Tracer m (TraceEvent blk)
DiffTime
ResourceRegistry m
StrictTVar m (m ())
StrictTVar m (Map FollowerKey (FollowerHandle m blk))
StrictTVar m (Map IteratorKey (m ()))
StrictTVar m (StrictMaybe (Header blk))
StrictTVar m ChainSelStarvation
StrictTVar m (TentativeHeaderState blk)
StrictTVar m (WithFingerprint (InvalidBlocks blk))
StrictTVar m FollowerKey
StrictTVar m IteratorKey
StrictTVar m (InternalChain blk)
Fuse m
TopLevelConfig blk
VolatileDB m blk
ImmutableDB m blk
LedgerDB' m blk
ChainSelQueue m blk
cdbImmutableDB :: forall (m :: * -> *) blk. ChainDbEnv m blk -> ImmutableDB m blk
cdbVolatileDB :: forall (m :: * -> *) blk. ChainDbEnv m blk -> VolatileDB m blk
cdbLedgerDB :: forall (m :: * -> *) blk. ChainDbEnv m blk -> LedgerDB' m blk
cdbChain :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> StrictTVar m (InternalChain blk)
cdbTentativeState :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> StrictTVar m (TentativeHeaderState blk)
cdbTentativeHeader :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> StrictTVar m (StrictMaybe (Header blk))
cdbIterators :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> StrictTVar m (Map IteratorKey (m ()))
cdbFollowers :: forall (m :: * -> *) blk.
ChainDbEnv m blk
-> StrictTVar m (Map FollowerKey (FollowerHandle m blk))
cdbTopLevelConfig :: forall (m :: * -> *) blk. ChainDbEnv m blk -> TopLevelConfig blk
cdbInvalid :: forall (m :: * -> *) blk.
ChainDbEnv m blk
-> StrictTVar m (WithFingerprint (InvalidBlocks blk))
cdbNextIteratorKey :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> StrictTVar m IteratorKey
cdbNextFollowerKey :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> StrictTVar m FollowerKey
cdbCopyFuse :: forall (m :: * -> *) blk. ChainDbEnv m blk -> Fuse m
cdbChainSelFuse :: forall (m :: * -> *) blk. ChainDbEnv m blk -> Fuse m
cdbTracer :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> Tracer m (TraceEvent blk)
cdbRegistry :: forall (m :: * -> *) blk. ChainDbEnv m blk -> ResourceRegistry m
cdbGcDelay :: forall (m :: * -> *) blk. ChainDbEnv m blk -> DiffTime
cdbGcInterval :: forall (m :: * -> *) blk. ChainDbEnv m blk -> DiffTime
cdbKillBgThreads :: forall (m :: * -> *) blk. ChainDbEnv m blk -> StrictTVar m (m ())
cdbChainSelQueue :: forall (m :: * -> *) blk. ChainDbEnv m blk -> ChainSelQueue m blk
cdbLoE :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> m (LoE (AnchoredFragment (HeaderWithTime blk)))
cdbChainSelStarvation :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> StrictTVar m ChainSelStarvation
cdbImmutableDB :: ImmutableDB m blk
cdbVolatileDB :: VolatileDB m blk
cdbLedgerDB :: LedgerDB' m blk
cdbChain :: StrictTVar m (InternalChain blk)
cdbTentativeState :: StrictTVar m (TentativeHeaderState blk)
cdbTentativeHeader :: StrictTVar m (StrictMaybe (Header blk))
cdbIterators :: StrictTVar m (Map IteratorKey (m ()))
cdbFollowers :: StrictTVar m (Map FollowerKey (FollowerHandle m blk))
cdbTopLevelConfig :: TopLevelConfig blk
cdbInvalid :: StrictTVar m (WithFingerprint (InvalidBlocks blk))
cdbNextIteratorKey :: StrictTVar m IteratorKey
cdbNextFollowerKey :: StrictTVar m FollowerKey
cdbCopyFuse :: Fuse m
cdbChainSelFuse :: Fuse m
cdbTracer :: Tracer m (TraceEvent blk)
cdbRegistry :: ResourceRegistry m
cdbGcDelay :: DiffTime
cdbGcInterval :: DiffTime
cdbKillBgThreads :: StrictTVar m (m ())
cdbChainSelQueue :: ChainSelQueue m blk
cdbLoE :: m (LoE (AnchoredFragment (HeaderWithTime blk)))
cdbChainSelStarvation :: StrictTVar m ChainSelStarvation
..} -> do

      ChainDbEnv m blk -> m ()
forall (m :: * -> *) blk. IOLike m => ChainDbEnv m blk -> m ()
Follower.closeAllFollowers ChainDbEnv m blk
cdb
      ChainDbEnv m blk -> m ()
forall (m :: * -> *) blk. IOLike m => ChainDbEnv m blk -> m ()
Iterator.closeAllIterators ChainDbEnv m blk
cdb

      killBgThreads <- STM m (m ()) -> 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 ()) -> m (m ())) -> STM m (m ()) -> m (m ())
forall a b. (a -> b) -> a -> b
$ StrictTVar m (m ()) -> STM m (m ())
forall (m :: * -> *) a. MonadSTM m => StrictTVar m a -> STM m a
readTVar StrictTVar m (m ())
cdbKillBgThreads
      killBgThreads

      ImmutableDB.closeDB cdbImmutableDB
      VolatileDB.closeDB cdbVolatileDB
      LedgerDB.closeDB cdbLedgerDB

      chain <- atomically $ icWithoutTime <$> readTVar cdbChain

      traceWith cdbTracer $ TraceOpenEvent $ ClosedDB
        (castPoint $ AF.anchorPoint chain)
        (castPoint $ AF.headPoint chain)

{-------------------------------------------------------------------------------
  Auxiliary
-------------------------------------------------------------------------------}

-- | Lift 'chunkIndexOfSlot' to 'Point'
--
-- Returns 'firstChunkNo' in case of 'GenesisPoint'.
chunkIndexOfPoint :: ImmutableDB.ChunkInfo -> Point blk -> ImmutableDB.ChunkNo
chunkIndexOfPoint :: forall blk. ChunkInfo -> Point blk -> ChunkNo
chunkIndexOfPoint ChunkInfo
chunkInfo = \case
    Point blk
GenesisPoint      -> ChunkNo
ImmutableDB.firstChunkNo
    BlockPoint SlotNo
slot HeaderHash blk
_ -> ChunkInfo -> SlotNo -> ChunkNo
ImmutableDB.chunkIndexOfSlot ChunkInfo
chunkInfo SlotNo
slot