{-# 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
  , LgrDB.TraceReplayEvent
  , SelectionChangedInfo (..)
  , TraceAddBlockEvent (..)
  , TraceCopyToImmutableDBEvent (..)
  , TraceEvent (..)
  , TraceFollowerEvent (..)
  , TraceGCEvent (..)
  , TraceInitChainSelEvent (..)
  , TraceIteratorEvent (..)
  , TraceOpenEvent (..)
  , TracePipeliningEvent (..)
  , TraceValidationEvent (..)
    -- * Re-exported for convenience
  , Args.RelativeMountPoint (..)
  , ImmutableDB.ImmutableDbSerialiseConstraints
  , LgrDB.LgrDbSerialiseConstraints
  , VolatileDB.VolatileDbSerialiseConstraints
    -- * Internals for testing purposes
  , Internal (..)
  , openDBInternal
  ) where

import           Control.Monad (when)
import           Control.Monad.Trans.Class (lift)
import           Control.ResourceRegistry (WithTempRegistry, allocate,
                     runInnerWithTempRegistry, runWithTempRegistry)
import           Control.Tracer
import           Data.Functor (void, (<&>))
import           Data.Functor.Identity (Identity)
import qualified Data.Map.Strict as Map
import           Data.Maybe.Strict (StrictMaybe (..))
import           GHC.Stack (HasCallStack)
import           Ouroboros.Consensus.Block
import qualified Ouroboros.Consensus.Fragment.Validated as VF
import           Ouroboros.Consensus.HardFork.Abstract
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.LgrDB as LgrDB
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.VolatileDB as VolatileDB
import           Ouroboros.Consensus.Util (newFuse, whenJust, withFuse)
import           Ouroboros.Consensus.Util.IOLike
import           Ouroboros.Consensus.Util.STM (Fingerprint (..),
                     WithFingerprint (..))
import qualified Ouroboros.Network.AnchoredFragment as AF

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

withDB ::
     forall m blk a.
     ( IOLike m
     , LedgerSupportsProtocol blk
     , BlockSupportsDiffusionPipelining blk
     , InspectLedger blk
     , HasHardForkHistory blk
     , ConvertRawHash blk
     , SerialiseDiskConstraints blk
     )
  => ChainDbArgs Identity 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) =>
ChainDbArgs Identity m blk -> (ChainDB m blk -> m a) -> m a
withDB ChainDbArgs Identity 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
<$> ChainDbArgs Identity m blk
-> Bool -> m (ChainDB m blk, Internal m blk)
forall (m :: * -> *) blk.
(IOLike m, LedgerSupportsProtocol blk,
 BlockSupportsDiffusionPipelining blk, InspectLedger blk,
 HasHardForkHistory blk, ConvertRawHash blk,
 SerialiseDiskConstraints blk) =>
ChainDbArgs Identity m blk
-> Bool -> m (ChainDB m blk, Internal m blk)
openDBInternal ChainDbArgs Identity m blk
args Bool
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
     )
  => ChainDbArgs Identity m blk
  -> m (ChainDB m blk)
openDB :: forall (m :: * -> *) blk.
(IOLike m, LedgerSupportsProtocol blk,
 BlockSupportsDiffusionPipelining blk, InspectLedger blk,
 HasHardForkHistory blk, ConvertRawHash blk,
 SerialiseDiskConstraints blk) =>
ChainDbArgs Identity m blk -> m (ChainDB m blk)
openDB ChainDbArgs Identity 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
<$> ChainDbArgs Identity m blk
-> Bool -> m (ChainDB m blk, Internal m blk)
forall (m :: * -> *) blk.
(IOLike m, LedgerSupportsProtocol blk,
 BlockSupportsDiffusionPipelining blk, InspectLedger blk,
 HasHardForkHistory blk, ConvertRawHash blk,
 SerialiseDiskConstraints blk) =>
ChainDbArgs Identity m blk
-> Bool -> m (ChainDB m blk, Internal m blk)
openDBInternal ChainDbArgs Identity m blk
args Bool
True

openDBInternal ::
     forall m blk.
     ( IOLike m
     , LedgerSupportsProtocol blk
     , BlockSupportsDiffusionPipelining blk
     , InspectLedger blk
     , HasHardForkHistory blk
     , ConvertRawHash blk
     , SerialiseDiskConstraints blk
     )
  => ChainDbArgs Identity 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) =>
ChainDbArgs Identity m blk
-> Bool -> m (ChainDB m blk, Internal m blk)
openDBInternal ChainDbArgs Identity 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 m blk
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
    Point blk
immutableDbTipPoint <- m (Point blk) -> WithTempRegistry (ChainDbEnv m blk) m (Point blk)
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 (Point blk)
 -> WithTempRegistry (ChainDbEnv m blk) m (Point blk))
-> m (Point blk)
-> WithTempRegistry (ChainDbEnv m blk) m (Point blk)
forall a b. (a -> b) -> a -> b
$ STM m (Point blk) -> m (Point blk)
forall a. HasCallStack => STM m a -> m a
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
STM m a -> m a
atomically (STM m (Point blk) -> m (Point blk))
-> STM m (Point blk) -> m (Point blk)
forall a b. (a -> b) -> a -> b
$ ImmutableDB m blk -> STM m (Point blk)
forall (m :: * -> *) blk.
(MonadSTM m, HasCallStack) =>
ImmutableDB m blk -> STM m (Point blk)
ImmutableDB.getTipPoint ImmutableDB m blk
immutableDB
    let immutableDbTipChunk :: ChunkNo
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
    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 -> TraceEvent blk)
-> TraceOpenEvent blk -> TraceEvent blk
forall a b. (a -> b) -> a -> b
$
        Point blk -> ChunkNo -> TraceOpenEvent blk
forall blk. Point blk -> ChunkNo -> TraceOpenEvent blk
OpenedImmutableDB Point blk
immutableDbTipPoint ChunkNo
immutableDbTipChunk

    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
StartedOpeningVolatileDB
    VolatileDB m blk
volatileDB <- Complete VolatileDbArgs m blk
-> (forall {st}.
    WithTempRegistry st m (VolatileDB m blk, st)
    -> WithTempRegistry (ChainDbEnv m blk) m (VolatileDB m blk))
-> WithTempRegistry (ChainDbEnv m blk) m (VolatileDB m blk)
forall (m :: * -> *) blk ans.
(HasCallStack, IOLike m, GetPrevHash blk,
 VolatileDbSerialiseConstraints blk) =>
Complete VolatileDbArgs m blk
-> (forall st. WithTempRegistry st m (VolatileDB m blk, st) -> ans)
-> ans
VolatileDB.openDB Complete VolatileDbArgs m blk
argsVolatileDb ((forall {st}.
  WithTempRegistry st m (VolatileDB m blk, st)
  -> WithTempRegistry (ChainDbEnv m blk) m (VolatileDB m blk))
 -> WithTempRegistry (ChainDbEnv m blk) m (VolatileDB m blk))
-> (forall {st}.
    WithTempRegistry st m (VolatileDB m blk, st)
    -> WithTempRegistry (ChainDbEnv m blk) m (VolatileDB m blk))
-> WithTempRegistry (ChainDbEnv m blk) m (VolatileDB m blk)
forall a b. (a -> b) -> a -> b
$ (VolatileDB m blk -> m ())
-> WithTempRegistry st m (VolatileDB m blk, st)
-> WithTempRegistry (ChainDbEnv m blk) m (VolatileDB m blk)
forall (m :: * -> *) innerDB st blk.
IOLike m =>
(innerDB -> m ())
-> WithTempRegistry st m (innerDB, st)
-> WithTempRegistry (ChainDbEnv m blk) m innerDB
innerOpenCont VolatileDB m blk -> m ()
VolatileDB m blk -> HasCallStack => m ()
forall (m :: * -> *) blk. VolatileDB m blk -> HasCallStack => m ()
VolatileDB.closeDB
    MaxSlotNo
maxSlot <- m MaxSlotNo -> WithTempRegistry (ChainDbEnv m blk) m MaxSlotNo
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 MaxSlotNo -> WithTempRegistry (ChainDbEnv m blk) m MaxSlotNo)
-> m MaxSlotNo -> WithTempRegistry (ChainDbEnv m blk) m MaxSlotNo
forall a b. (a -> b) -> a -> b
$ STM m MaxSlotNo -> m MaxSlotNo
forall a. HasCallStack => STM m a -> m a
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
STM m a -> m a
atomically (STM m MaxSlotNo -> m MaxSlotNo) -> STM m MaxSlotNo -> m MaxSlotNo
forall a b. (a -> b) -> a -> b
$ VolatileDB m blk -> HasCallStack => STM m MaxSlotNo
forall (m :: * -> *) blk.
VolatileDB m blk -> HasCallStack => STM m MaxSlotNo
VolatileDB.getMaxSlotNo VolatileDB m blk
volatileDB
    (ChainDB m blk
chainDB, Internal m blk
testing, ChainDbEnv m blk
env) <- m (ChainDB m blk, Internal m blk, ChainDbEnv m blk)
-> WithTempRegistry
     (ChainDbEnv m blk)
     m
     (ChainDB m blk, Internal m blk, ChainDbEnv m blk)
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 (ChainDB m blk, Internal m blk, ChainDbEnv m blk)
 -> WithTempRegistry
      (ChainDbEnv m blk)
      m
      (ChainDB m blk, Internal m blk, ChainDbEnv m blk))
-> m (ChainDB m blk, Internal m blk, ChainDbEnv m blk)
-> WithTempRegistry
     (ChainDbEnv m blk)
     m
     (ChainDB m blk, Internal m blk, ChainDbEnv m blk)
forall a b. (a -> b) -> a -> b
$ do
      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 (MaxSlotNo -> TraceOpenEvent blk
forall blk. MaxSlotNo -> TraceOpenEvent blk
OpenedVolatileDB MaxSlotNo
maxSlot)
      let lgrReplayTracer :: Tracer m (ReplayGoal blk -> TraceReplayEvent blk)
lgrReplayTracer =
            Point blk
-> Tracer m (TraceReplayEvent blk)
-> Tracer m (ReplayGoal blk -> TraceReplayEvent blk)
forall blk (m :: * -> *).
Point blk
-> Tracer m (TraceReplayEvent blk)
-> Tracer m (ReplayGoal blk -> TraceReplayEvent blk)
LgrDB.decorateReplayTracerWithGoal
              Point blk
immutableDbTipPoint
              ((TraceReplayEvent blk -> TraceEvent blk)
-> Tracer m (TraceEvent blk) -> Tracer m (TraceReplayEvent blk)
forall a' a. (a' -> a) -> Tracer m a -> Tracer m a'
forall (f :: * -> *) a' a.
Contravariant f =>
(a' -> a) -> f a -> f a'
contramap TraceReplayEvent blk -> TraceEvent blk
forall blk. TraceReplayEvent blk -> TraceEvent blk
TraceLedgerReplayEvent Tracer m (TraceEvent blk)
tracer)
      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
StartedOpeningLgrDB
      (LgrDB m blk
lgrDB, Word64
replayed) <- Complete LgrDbArgs m blk
-> Tracer m (ReplayGoal blk -> TraceReplayEvent blk)
-> ImmutableDB m blk
-> (RealPoint blk -> m blk)
-> m (LgrDB m blk, Word64)
forall (m :: * -> *) blk.
(IOLike m, LedgerSupportsProtocol blk,
 LgrDbSerialiseConstraints blk, InspectLedger blk, HasCallStack) =>
Complete LgrDbArgs m blk
-> Tracer m (ReplayGoal blk -> TraceReplayEvent blk)
-> ImmutableDB m blk
-> (RealPoint blk -> m blk)
-> m (LgrDB m blk, Word64)
LgrDB.openDB Complete LgrDbArgs m blk
argsLgrDb
                            Tracer m (ReplayGoal blk -> TraceReplayEvent blk)
lgrReplayTracer
                            ImmutableDB m blk
immutableDB
                            (ImmutableDB m blk -> VolatileDB m blk -> RealPoint blk -> m blk
forall (m :: * -> *) blk.
(IOLike m, HasHeader blk) =>
ImmutableDB m blk -> VolatileDB m blk -> RealPoint blk -> m blk
Query.getAnyKnownBlock ImmutableDB m blk
immutableDB VolatileDB m blk
volatileDB)
      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
OpenedLgrDB

      StrictTVar
  m (WithFingerprint (Map (HeaderHash blk) (InvalidBlockInfo blk)))
varInvalid      <- WithFingerprint (Map (HeaderHash blk) (InvalidBlockInfo blk))
-> m (StrictTVar
        m (WithFingerprint (Map (HeaderHash blk) (InvalidBlockInfo blk))))
forall (m :: * -> *) a.
(HasCallStack, MonadSTM m, NoThunks a) =>
a -> m (StrictTVar m a)
newTVarIO (Map (HeaderHash blk) (InvalidBlockInfo blk)
-> Fingerprint
-> WithFingerprint (Map (HeaderHash blk) (InvalidBlockInfo blk))
forall a. a -> Fingerprint -> WithFingerprint a
WithFingerprint Map (HeaderHash blk) (InvalidBlockInfo blk)
forall k a. Map k a
Map.empty (Word64 -> Fingerprint
Fingerprint Word64
0))

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

      Tracer m (TraceInitChainSelEvent blk)
-> TraceInitChainSelEvent blk -> m ()
forall (m :: * -> *) a. Tracer m a -> a -> m ()
traceWith Tracer m (TraceInitChainSelEvent blk)
initChainSelTracer TraceInitChainSelEvent blk
forall blk. TraceInitChainSelEvent blk
StartedInitChainSelection
      LoE (AnchoredFragment (Header blk))
initialLoE     <- 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
      ChainAndLedger blk
chainAndLedger <- ImmutableDB m blk
-> VolatileDB m blk
-> LgrDB m blk
-> Tracer m (TraceInitChainSelEvent blk)
-> TopLevelConfig blk
-> StrictTVar
     m (WithFingerprint (Map (HeaderHash blk) (InvalidBlockInfo blk)))
-> LoE ()
-> m (ChainAndLedger blk)
forall (m :: * -> *) blk.
(IOLike m, LedgerSupportsProtocol blk,
 BlockSupportsDiffusionPipelining blk) =>
ImmutableDB m blk
-> VolatileDB m blk
-> LgrDB m blk
-> Tracer m (TraceInitChainSelEvent blk)
-> TopLevelConfig blk
-> StrictTVar m (WithFingerprint (InvalidBlocks blk))
-> LoE ()
-> m (ChainAndLedger blk)
ChainSel.initialChainSelection
                          ImmutableDB m blk
immutableDB
                          VolatileDB m blk
volatileDB
                          LgrDB m blk
lgrDB
                          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 (Header blk)) -> LoE ()
forall (f :: * -> *) a. Functor f => f a -> f ()
void LoE (AnchoredFragment (Header blk))
initialLoE)
      Tracer m (TraceInitChainSelEvent blk)
-> TraceInitChainSelEvent blk -> m ()
forall (m :: * -> *) a. Tracer m a -> a -> m ()
traceWith Tracer m (TraceInitChainSelEvent blk)
initChainSelTracer TraceInitChainSelEvent blk
forall blk. TraceInitChainSelEvent blk
InitialChainSelected

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

      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
$ LgrDB m blk -> LedgerDB' blk -> STM m ()
forall (m :: * -> *) blk.
IOLike m =>
LgrDB m blk -> LedgerDB' blk -> STM m ()
LgrDB.setCurrent LgrDB m blk
lgrDB LedgerDB' blk
ledger
      StrictTVar m (AnchoredFragment (Header blk))
varChain           <- AnchoredFragment (Header blk)
-> m (StrictTVar m (AnchoredFragment (Header blk)))
forall (m :: * -> *) a.
(HasCallStack, MonadSTM m, NoThunks a) =>
a -> m (StrictTVar m a)
newTVarIO AnchoredFragment (Header blk)
chain
      StrictTVar m (TentativeHeaderState blk)
varTentativeState  <- TentativeHeaderState blk
-> m (StrictTVar m (TentativeHeaderState blk))
forall (m :: * -> *) a.
(HasCallStack, MonadSTM m, NoThunks a) =>
a -> m (StrictTVar m a)
newTVarIO (TentativeHeaderState blk
 -> m (StrictTVar m (TentativeHeaderState blk)))
-> TentativeHeaderState blk
-> m (StrictTVar m (TentativeHeaderState blk))
forall a b. (a -> b) -> a -> b
$ Proxy blk -> TentativeHeaderState blk
forall blk.
BlockSupportsDiffusionPipelining blk =>
Proxy blk -> TentativeHeaderState blk
initialTentativeHeaderState (forall t. Proxy t
forall {k} (t :: k). Proxy t
Proxy @blk)
      StrictTVar m (StrictMaybe (Header blk))
varTentativeHeader <- StrictMaybe (Header blk)
-> m (StrictTVar m (StrictMaybe (Header blk)))
forall (m :: * -> *) a.
(HasCallStack, MonadSTM m, NoThunks a) =>
a -> m (StrictTVar m a)
newTVarIO StrictMaybe (Header blk)
forall a. StrictMaybe a
SNothing
      StrictTVar m (Map IteratorKey (m ()))
varIterators       <- Map IteratorKey (m ()) -> m (StrictTVar m (Map IteratorKey (m ())))
forall (m :: * -> *) a.
(HasCallStack, MonadSTM m, NoThunks a) =>
a -> m (StrictTVar m a)
newTVarIO Map IteratorKey (m ())
forall k a. Map k a
Map.empty
      StrictTVar m (Map FollowerKey (FollowerHandle m blk))
varFollowers       <- Map FollowerKey (FollowerHandle m blk)
-> m (StrictTVar m (Map FollowerKey (FollowerHandle m blk)))
forall (m :: * -> *) a.
(HasCallStack, MonadSTM m, NoThunks a) =>
a -> m (StrictTVar m a)
newTVarIO Map FollowerKey (FollowerHandle m blk)
forall k a. Map k a
Map.empty
      StrictTVar m IteratorKey
varNextIteratorKey <- IteratorKey -> m (StrictTVar m IteratorKey)
forall (m :: * -> *) a.
(HasCallStack, MonadSTM m, NoThunks a) =>
a -> m (StrictTVar m a)
newTVarIO (Word -> IteratorKey
IteratorKey Word
0)
      StrictTVar m FollowerKey
varNextFollowerKey <- FollowerKey -> m (StrictTVar m FollowerKey)
forall (m :: * -> *) a.
(HasCallStack, MonadSTM m, NoThunks a) =>
a -> m (StrictTVar m a)
newTVarIO (Word -> FollowerKey
FollowerKey   Word
0)
      StrictTVar m (m ())
varKillBgThreads   <- m () -> m (StrictTVar m (m ()))
forall (m :: * -> *) a.
(HasCallStack, MonadSTM m, NoThunks a) =>
a -> m (StrictTVar m a)
newTVarIO (m () -> m (StrictTVar m (m ())))
-> m () -> m (StrictTVar m (m ()))
forall a b. (a -> b) -> a -> b
$ () -> m ()
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return ()
      Fuse m
copyFuse           <- Text -> m (Fuse m)
forall (m :: * -> *). MonadMVar m => Text -> m (Fuse m)
newFuse Text
"copy to immutable db"
      Fuse m
chainSelFuse       <- Text -> m (Fuse m)
forall (m :: * -> *). MonadMVar m => Text -> m (Fuse m)
newFuse Text
"chain selection"
      ChainSelQueue m blk
chainSelQueue      <- Word -> m (ChainSelQueue m blk)
forall (m :: * -> *) blk.
IOLike m =>
Word -> m (ChainSelQueue m blk)
newChainSelQueue (ChainDbSpecificArgs Identity m blk -> Word
forall (f :: * -> *) (m :: * -> *) blk.
ChainDbSpecificArgs f m blk -> Word
Args.cdbsBlocksToAddSize ChainDbSpecificArgs Identity m blk
cdbSpecificArgs)

      let env :: ChainDbEnv m blk
env = CDB { cdbImmutableDB :: ImmutableDB m blk
cdbImmutableDB     = ImmutableDB m blk
immutableDB
                    , cdbVolatileDB :: VolatileDB m blk
cdbVolatileDB      = VolatileDB m blk
volatileDB
                    , cdbLgrDB :: LgrDB m blk
cdbLgrDB           = LgrDB m blk
lgrDB
                    , cdbChain :: StrictTVar m (AnchoredFragment (Header blk))
cdbChain           = StrictTVar m (AnchoredFragment (Header 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
                    }
      ChainDbHandle m blk
h <- (StrictTVar m (ChainDbState m blk) -> ChainDbHandle m blk)
-> m (StrictTVar m (ChainDbState m blk)) -> m (ChainDbHandle m blk)
forall a b. (a -> b) -> m a -> m b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap StrictTVar m (ChainDbState m blk) -> ChainDbHandle m blk
forall (m :: * -> *) blk.
StrictTVar m (ChainDbState m blk) -> ChainDbHandle m blk
CDBHandle (m (StrictTVar m (ChainDbState m blk)) -> m (ChainDbHandle m blk))
-> m (StrictTVar m (ChainDbState m blk)) -> m (ChainDbHandle m blk)
forall a b. (a -> b) -> a -> b
$ ChainDbState m blk -> m (StrictTVar m (ChainDbState m blk))
forall (m :: * -> *) a.
(HasCallStack, MonadSTM m, NoThunks a) =>
a -> m (StrictTVar m a)
newTVarIO (ChainDbState m blk -> m (StrictTVar m (ChainDbState m blk)))
-> ChainDbState m blk -> m (StrictTVar m (ChainDbState m blk))
forall a b. (a -> b) -> a -> b
$ ChainDbEnv m blk -> ChainDbState m blk
forall (m :: * -> *) blk. ChainDbEnv m blk -> ChainDbState m blk
ChainDbOpen ChainDbEnv m blk
env
      let chainDB :: ChainDB m blk
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
            , getLedgerDB :: STM m (LedgerDB' blk)
getLedgerDB           = ChainDbHandle m blk
-> (ChainDbEnv m blk -> STM m (LedgerDB' blk))
-> STM m (LedgerDB' 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 (LedgerDB' blk)
forall (m :: * -> *) blk.
IOLike m =>
ChainDbEnv m blk -> STM m (LedgerDB' blk)
Query.getLedgerDB
            , 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.
(IOLike m, HasHardForkHistory blk, HasAnnTip blk,
 IsLedger (LedgerState blk)) =>
ChainDbEnv m blk -> STM m (HeaderStateHistory blk)
Query.getHeaderStateHistory
            , 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 =>
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
            , 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
            }
      Fuse m
addBlockTestFuse <- Text -> m (Fuse m)
forall (m :: * -> *). MonadMVar m => Text -> m (Fuse m)
newFuse Text
"test chain selection"
      Fuse m
copyTestFuse <- Text -> m (Fuse m)
forall (m :: * -> *). MonadMVar m => Text -> m (Fuse m)
newFuse Text
"test copy to immutable db"
      let testing :: Internal m blk
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
            , intUpdateLedgerSnapshots :: m ()
intUpdateLedgerSnapshots   = 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 ()
forall (m :: * -> *) blk.
(IOLike m, LgrDbSerialiseConstraints blk, HasHeader blk,
 IsLedger (LedgerState blk)) =>
ChainDbEnv m blk -> m ()
Background.updateLedgerSnapshots
            , 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
            }

      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 -> TraceEvent blk)
-> TraceOpenEvent blk -> TraceEvent blk
forall a b. (a -> b) -> a -> b
$ Point blk -> Point blk -> TraceOpenEvent blk
forall blk. Point blk -> Point blk -> TraceOpenEvent blk
OpenedDB
        (Point (Header blk) -> Point blk
forall {k1} {k2} (b :: k1) (b' :: k2).
Coercible (HeaderHash b) (HeaderHash b') =>
Point b -> Point b'
castPoint (Point (Header blk) -> Point blk)
-> Point (Header blk) -> Point blk
forall a b. (a -> b) -> a -> b
$ AnchoredFragment (Header blk) -> Point (Header blk)
forall block. AnchoredFragment block -> Point block
AF.anchorPoint AnchoredFragment (Header blk)
chain)
        (Point (Header blk) -> Point blk
forall {k1} {k2} (b :: k1) (b' :: k2).
Coercible (HeaderHash b) (HeaderHash b') =>
Point b -> Point b'
castPoint (Point (Header blk) -> Point blk)
-> Point (Header blk) -> Point blk
forall a b. (a -> b) -> a -> b
$ AnchoredFragment (Header blk) -> Point (Header blk)
forall block.
HasHeader block =>
AnchoredFragment block -> Point block
AF.headPoint   AnchoredFragment (Header blk)
chain)

      Bool -> m () -> m ()
forall (f :: * -> *). Applicative f => Bool -> f () -> f ()
when Bool
launchBgTasks (m () -> m ()) -> m () -> m ()
forall a b. (a -> b) -> a -> b
$ ChainDbEnv m blk -> Word64 -> m ()
forall (m :: * -> *) blk.
(IOLike m, LedgerSupportsProtocol blk,
 BlockSupportsDiffusionPipelining blk, InspectLedger blk,
 HasHardForkHistory blk, LgrDbSerialiseConstraints blk) =>
ChainDbEnv m blk -> Word64 -> m ()
Background.launchBgTasks ChainDbEnv m blk
env Word64
replayed

      (ChainDB m blk, Internal m blk, ChainDbEnv m blk)
-> m (ChainDB m blk, Internal m blk, ChainDbEnv m blk)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (ChainDB m blk
chainDB, Internal m blk
forall {blk}. Internal m blk
testing, ChainDbEnv m blk
env)

    (ResourceKey m, ChainDB m blk)
_ <- m (ResourceKey m, ChainDB m blk)
-> WithTempRegistry
     (ChainDbEnv m blk) m (ResourceKey m, ChainDB m blk)
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 (ResourceKey m, ChainDB m blk)
 -> WithTempRegistry
      (ChainDbEnv m blk) m (ResourceKey m, ChainDB m blk))
-> m (ResourceKey m, ChainDB m blk)
-> WithTempRegistry
     (ChainDbEnv m blk) m (ResourceKey m, ChainDB m blk)
forall a b. (a -> b) -> a -> b
$ ResourceRegistry m
-> (ResourceId -> m (ChainDB m blk))
-> (ChainDB m blk -> m ())
-> m (ResourceKey m, ChainDB m blk)
forall (m :: * -> *) a.
(MonadSTM m, MonadMask m, MonadThread m, HasCallStack) =>
ResourceRegistry m
-> (ResourceId -> m a) -> (a -> m ()) -> m (ResourceKey m, a)
allocate (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) (\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 -> m (ChainDB m blk))
-> ChainDB m blk -> m (ChainDB m blk)
forall a b. (a -> b) -> a -> b
$ ChainDB m blk
chainDB) ChainDB m blk -> m ()
forall (m :: * -> *) blk. ChainDB m blk -> m ()
API.closeDB

    ((ChainDB m blk, Internal m blk), ChainDbEnv m blk)
-> WithTempRegistry
     (ChainDbEnv m blk)
     m
     ((ChainDB m blk, Internal m blk), ChainDbEnv m blk)
forall a. a -> WithTempRegistry (ChainDbEnv m blk) m a
forall (m :: * -> *) a. Monad m => a -> m a
return ((ChainDB m blk
chainDB, Internal m blk
testing), ChainDbEnv m blk
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 LgrDbArgs m blk
argsLgrDb ChainDbSpecificArgs Identity m blk
cdbSpecificArgs = ChainDbArgs Identity 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
    Maybe (ChainDbEnv m blk)
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
    Maybe (ChainDbEnv m blk) -> (ChainDbEnv m blk -> m ()) -> m ()
forall (f :: * -> *) a.
Applicative f =>
Maybe a -> (a -> f ()) -> f ()
whenJust Maybe (ChainDbEnv m blk)
mbOpenEnv ((ChainDbEnv m blk -> m ()) -> m ())
-> (ChainDbEnv m blk -> m ()) -> m ()
forall a b. (a -> b) -> a -> b
$ \cdb :: ChainDbEnv m blk
cdb@CDB{m (LoE (AnchoredFragment (Header 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 (AnchoredFragment (Header blk))
StrictTVar m (TentativeHeaderState blk)
StrictTVar m (WithFingerprint (InvalidBlocks blk))
StrictTVar m FollowerKey
StrictTVar m IteratorKey
Fuse m
TopLevelConfig blk
VolatileDB m blk
ImmutableDB m blk
LgrDB 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
cdbLgrDB :: forall (m :: * -> *) blk. ChainDbEnv m blk -> LgrDB m blk
cdbChain :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> StrictTVar m (AnchoredFragment (Header 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 (Header blk)))
cdbImmutableDB :: ImmutableDB m blk
cdbVolatileDB :: VolatileDB m blk
cdbLgrDB :: LgrDB m blk
cdbChain :: StrictTVar m (AnchoredFragment (Header 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 (Header blk)))
..} -> 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

      m ()
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
      m ()
killBgThreads

      ImmutableDB m blk -> m ()
forall (m :: * -> *) blk. HasCallStack => ImmutableDB m blk -> m ()
ImmutableDB.closeDB ImmutableDB m blk
cdbImmutableDB
      VolatileDB m blk -> HasCallStack => m ()
forall (m :: * -> *) blk. VolatileDB m blk -> HasCallStack => m ()
VolatileDB.closeDB VolatileDB m blk
cdbVolatileDB

      AnchoredFragment (Header blk)
chain <- STM m (AnchoredFragment (Header blk))
-> m (AnchoredFragment (Header blk))
forall a. HasCallStack => STM m a -> m a
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
STM m a -> m a
atomically (STM m (AnchoredFragment (Header blk))
 -> m (AnchoredFragment (Header blk)))
-> STM m (AnchoredFragment (Header blk))
-> m (AnchoredFragment (Header blk))
forall a b. (a -> b) -> a -> b
$ StrictTVar m (AnchoredFragment (Header blk))
-> STM m (AnchoredFragment (Header blk))
forall (m :: * -> *) a. MonadSTM m => StrictTVar m a -> STM m a
readTVar StrictTVar m (AnchoredFragment (Header blk))
cdbChain

      Tracer m (TraceEvent blk) -> TraceEvent blk -> m ()
forall (m :: * -> *) a. Tracer m a -> a -> m ()
traceWith Tracer m (TraceEvent blk)
cdbTracer (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 -> TraceEvent blk)
-> TraceOpenEvent blk -> TraceEvent blk
forall a b. (a -> b) -> a -> b
$ Point blk -> Point blk -> TraceOpenEvent blk
forall blk. Point blk -> Point blk -> TraceOpenEvent blk
ClosedDB
        (Point (Header blk) -> Point blk
forall {k1} {k2} (b :: k1) (b' :: k2).
Coercible (HeaderHash b) (HeaderHash b') =>
Point b -> Point b'
castPoint (Point (Header blk) -> Point blk)
-> Point (Header blk) -> Point blk
forall a b. (a -> b) -> a -> b
$ AnchoredFragment (Header blk) -> Point (Header blk)
forall block. AnchoredFragment block -> Point block
AF.anchorPoint AnchoredFragment (Header blk)
chain)
        (Point (Header blk) -> Point blk
forall {k1} {k2} (b :: k1) (b' :: k2).
Coercible (HeaderHash b) (HeaderHash b') =>
Point b -> Point b'
castPoint (Point (Header blk) -> Point blk)
-> Point (Header blk) -> Point blk
forall a b. (a -> b) -> a -> b
$ AnchoredFragment (Header blk) -> Point (Header blk)
forall block.
HasHeader block =>
AnchoredFragment block -> Point block
AF.headPoint AnchoredFragment (Header blk)
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