{-# LANGUAGE DataKinds #-}
{-# LANGUAGE DeriveAnyClass #-}
{-# LANGUAGE DeriveGeneric #-}
{-# LANGUAGE DerivingVia #-}
{-# LANGUAGE FlexibleContexts #-}
{-# LANGUAGE GADTs #-}
{-# LANGUAGE LambdaCase #-}
{-# LANGUAGE NamedFieldPuns #-}
{-# LANGUAGE OverloadedStrings #-}
{-# LANGUAGE PatternSynonyms #-}
{-# LANGUAGE ScopedTypeVariables #-}
{-# LANGUAGE TypeApplications #-}
{-# LANGUAGE TypeOperators #-}

-- | A 'BackingStore' implementation based on [LMDB](http://www.lmdb.tech/doc/).
module Ouroboros.Consensus.Storage.LedgerDB.V1.BackingStore.Impl.LMDB
  ( -- * Opening a database
    LMDBLimits (LMDBLimits, lmdbMapSize, lmdbMaxDatabases, lmdbMaxReaders)
  , newLMDBBackingStore

    -- * Errors
  , LMDBErr (..)

    -- * Internals exposed for @snapshot-converter@
  , DbSeqNo (..)
  , LMDBMK (..)
  , getDb
  , initLMDBTable
  , withDbSeqNoRWMaybeNull
  ) where

import Cardano.Slotting.Slot (SlotNo, WithOrigin (At))
import qualified Codec.Serialise as S (Serialise (..))
import qualified Control.Concurrent.Class.MonadSTM.TVar as IOLike
import Control.Monad (forM_, unless, void, when)
import qualified Control.Monad.Class.MonadSTM as IOLike
import Control.Monad.IO.Class (MonadIO (liftIO))
import qualified Control.Tracer as Trace
import Data.Functor (($>), (<&>))
import Data.Functor.Contravariant ((>$<))
import Data.Map (Map)
import qualified Data.Map.Strict as Map
import Data.MemPack
import Data.Proxy
import qualified Data.Set as Set
import qualified Data.Text as Strict
import qualified Database.LMDB.Simple as LMDB
import qualified Database.LMDB.Simple.Cursor as LMDB.Cursor
import qualified Database.LMDB.Simple.Extra as LMDB
import qualified Database.LMDB.Simple.Internal as LMDB.Internal
import qualified Database.LMDB.Simple.TransactionHandle as TrH
import GHC.Generics (Generic)
import GHC.Stack (HasCallStack)
import Ouroboros.Consensus.Ledger.Tables
import qualified Ouroboros.Consensus.Ledger.Tables.Diff as Diff
import Ouroboros.Consensus.Storage.LedgerDB.Snapshots
  ( SnapshotBackend (..)
  )
import qualified Ouroboros.Consensus.Storage.LedgerDB.V1.BackingStore.API as API
import qualified Ouroboros.Consensus.Storage.LedgerDB.V1.BackingStore.Impl.LMDB.Bridge as Bridge
import Ouroboros.Consensus.Storage.LedgerDB.V1.BackingStore.Impl.LMDB.Status
  ( Status (..)
  , StatusLock
  )
import qualified Ouroboros.Consensus.Storage.LedgerDB.V1.BackingStore.Impl.LMDB.Status as Status
import Ouroboros.Consensus.Util (foldlM')
import Ouroboros.Consensus.Util.IOLike
  ( Exception (..)
  , IOLike
  , MonadCatch (..)
  , MonadThrow (..)
  , bracket
  )
import Ouroboros.Consensus.Util.IndexedMemPack
import qualified System.FS.API as FS

{-------------------------------------------------------------------------------
  Database definition
-------------------------------------------------------------------------------}

-- | The LMDB database that underlies the backing store.
data Db m l = Db
  { forall (m :: * -> *) (l :: LedgerStateKind).
Db m l -> Environment 'ReadWrite
dbEnv :: !(LMDB.Environment LMDB.ReadWrite)
  -- ^ The LMDB environment is a pointer to the directory that contains the
  -- @`Db`@.
  , forall (m :: * -> *) (l :: LedgerStateKind).
Db m l -> Database () DbSeqNo
dbState :: !(LMDB.Database () DbSeqNo)
  -- ^ The on-disk state of the @`Db`@.
  --
  -- The state is kept in an LDMB table with only one key and one value:
  -- The current sequence number of the @`Db`@.
  , forall (m :: * -> *) (l :: LedgerStateKind).
Db m l -> LedgerTables l LMDBMK
dbBackingTables :: !(LedgerTables l LMDBMK)
  -- ^ The LMDB tables with the key-value stores.
  , forall (m :: * -> *) (l :: LedgerStateKind). Db m l -> String
dbFilePath :: !FilePath
  , forall (m :: * -> *) (l :: LedgerStateKind).
Db m l -> Tracer m BackingStoreTrace
dbTracer :: !(Trace.Tracer m API.BackingStoreTrace)
  , forall (m :: * -> *) (l :: LedgerStateKind). Db m l -> StatusLock m
dbStatusLock :: !(StatusLock m)
  -- ^ Status of the LMDB backing store. When 'Closed', all backing store
  -- (value handle) operations will fail.
  , forall (m :: * -> *) (l :: LedgerStateKind).
Db m l -> TVar m (Map Int (Cleanup m))
dbOpenHandles :: !(IOLike.TVar m (Map Int (Cleanup m)))
  -- ^ Map of open value handles to cleanup actions. When closing the backing
  -- store, these cleanup actions are used to ensure all value handles cleaned
  -- up.
  --
  -- Note: why not use 'bsvhClose' here? We would get nested lock acquisition
  -- on 'dbStatusLock', which causes a deadlock:
  --
  -- * 'bsClose' acquires a write lock
  --
  -- * 'bsvhClose' is called on a value handle
  --
  -- * 'bsvhClose' tries to acquire a read lock, but it has to wait for
  --   'bsClose' to give up its write lock
  , forall (m :: * -> *) (l :: LedgerStateKind). Db m l -> TVar m Int
dbNextId :: !(IOLike.TVar m Int)
  }

newtype LMDBLimits = MkLMDBLimits {LMDBLimits -> Limits
unLMDBLimits :: LMDB.Limits}
  deriving (Int -> LMDBLimits -> ShowS
[LMDBLimits] -> ShowS
LMDBLimits -> String
(Int -> LMDBLimits -> ShowS)
-> (LMDBLimits -> String)
-> ([LMDBLimits] -> ShowS)
-> Show LMDBLimits
forall a.
(Int -> a -> ShowS) -> (a -> String) -> ([a] -> ShowS) -> Show a
$cshowsPrec :: Int -> LMDBLimits -> ShowS
showsPrec :: Int -> LMDBLimits -> ShowS
$cshow :: LMDBLimits -> String
show :: LMDBLimits -> String
$cshowList :: [LMDBLimits] -> ShowS
showList :: [LMDBLimits] -> ShowS
Show, LMDBLimits -> LMDBLimits -> Bool
(LMDBLimits -> LMDBLimits -> Bool)
-> (LMDBLimits -> LMDBLimits -> Bool) -> Eq LMDBLimits
forall a. (a -> a -> Bool) -> (a -> a -> Bool) -> Eq a
$c== :: LMDBLimits -> LMDBLimits -> Bool
== :: LMDBLimits -> LMDBLimits -> Bool
$c/= :: LMDBLimits -> LMDBLimits -> Bool
/= :: LMDBLimits -> LMDBLimits -> Bool
Eq)

{-# COMPLETE LMDBLimits #-}

-- | Configuration to use for LMDB backing store initialisation.
--
-- Keep the following in mind:
--
-- * @'lmdbMapSize'@ should be a multiple of the OS page size.
--
-- * @'lmdbMaxDatabases'@ should be set to at least 2, since the backing store
--    has 2 internal LMDB databases by default: 1 for the actual tables, and
--    1 for the database state @'DbSeqNo'@.
pattern LMDBLimits :: Int -> Int -> Int -> LMDBLimits
pattern $mLMDBLimits :: forall {r}.
LMDBLimits -> (Int -> Int -> Int -> r) -> ((# #) -> r) -> r
$bLMDBLimits :: Int -> Int -> Int -> LMDBLimits
LMDBLimits{LMDBLimits -> Int
lmdbMapSize, LMDBLimits -> Int
lmdbMaxDatabases, LMDBLimits -> Int
lmdbMaxReaders} =
  MkLMDBLimits
    LMDB.Limits
      { LMDB.mapSize = lmdbMapSize
      , LMDB.maxDatabases = lmdbMaxDatabases
      , LMDB.maxReaders = lmdbMaxReaders
      }

-- | The database state consists of only the database sequence number @dbsSeq@.
-- @dbsSeq@ represents the slot up to which we have flushed changes to disk.
-- Note that we only flush changes to disk if they have become immutable.
newtype DbSeqNo = DbSeqNo
  { DbSeqNo -> WithOrigin SlotNo
dbsSeq :: WithOrigin SlotNo
  }
  deriving stock (Int -> DbSeqNo -> ShowS
[DbSeqNo] -> ShowS
DbSeqNo -> String
(Int -> DbSeqNo -> ShowS)
-> (DbSeqNo -> String) -> ([DbSeqNo] -> ShowS) -> Show DbSeqNo
forall a.
(Int -> a -> ShowS) -> (a -> String) -> ([a] -> ShowS) -> Show a
$cshowsPrec :: Int -> DbSeqNo -> ShowS
showsPrec :: Int -> DbSeqNo -> ShowS
$cshow :: DbSeqNo -> String
show :: DbSeqNo -> String
$cshowList :: [DbSeqNo] -> ShowS
showList :: [DbSeqNo] -> ShowS
Show, (forall x. DbSeqNo -> Rep DbSeqNo x)
-> (forall x. Rep DbSeqNo x -> DbSeqNo) -> Generic DbSeqNo
forall x. Rep DbSeqNo x -> DbSeqNo
forall x. DbSeqNo -> Rep DbSeqNo x
forall a.
(forall x. a -> Rep a x) -> (forall x. Rep a x -> a) -> Generic a
$cfrom :: forall x. DbSeqNo -> Rep DbSeqNo x
from :: forall x. DbSeqNo -> Rep DbSeqNo x
$cto :: forall x. Rep DbSeqNo x -> DbSeqNo
to :: forall x. Rep DbSeqNo x -> DbSeqNo
Generic)
  deriving anyclass [DbSeqNo] -> Encoding
DbSeqNo -> Encoding
(DbSeqNo -> Encoding)
-> (forall s. Decoder s DbSeqNo)
-> ([DbSeqNo] -> Encoding)
-> (forall s. Decoder s [DbSeqNo])
-> Serialise DbSeqNo
forall s. Decoder s [DbSeqNo]
forall s. Decoder s DbSeqNo
forall a.
(a -> Encoding)
-> (forall s. Decoder s a)
-> ([a] -> Encoding)
-> (forall s. Decoder s [a])
-> Serialise a
$cencode :: DbSeqNo -> Encoding
encode :: DbSeqNo -> Encoding
$cdecode :: forall s. Decoder s DbSeqNo
decode :: forall s. Decoder s DbSeqNo
$cencodeList :: [DbSeqNo] -> Encoding
encodeList :: [DbSeqNo] -> Encoding
$cdecodeList :: forall s. Decoder s [DbSeqNo]
decodeList :: forall s. Decoder s [DbSeqNo]
S.Serialise

-- | A 'MapKind' that represents an LMDB database handle
data LMDBMK k v = LMDBMK !String !(LMDB.Database k v)

{-------------------------------------------------------------------------------
  Low-level API
-------------------------------------------------------------------------------}

getDb ::
  LMDB.Internal.IsMode mode =>
  K2 String k v ->
  LMDB.Transaction mode (LMDBMK k v)
getDb :: forall (mode :: Mode) k v.
IsMode mode =>
K2 String k v -> Transaction mode (LMDBMK k v)
getDb (K2 String
name) = String -> Database k v -> LMDBMK k v
forall k v. String -> Database k v -> LMDBMK k v
LMDBMK String
name (Database k v -> LMDBMK k v)
-> Transaction mode (Database k v) -> Transaction mode (LMDBMK k v)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Maybe String -> Transaction mode (Database k v)
forall (mode :: Mode) k v.
IsMode mode =>
Maybe String -> Transaction mode (Database k v)
LMDB.getDatabase (String -> Maybe String
forall a. a -> Maybe a
Just String
name)

readAll ::
  (Ord (TxIn l), MemPack (TxIn l), IndexedMemPack idx (TxOut l)) =>
  Proxy l ->
  idx ->
  LMDBMK (TxIn l) (TxOut l) ->
  LMDB.Transaction mode (ValuesMK (TxIn l) (TxOut l))
readAll :: forall (l :: LedgerStateKind) idx (mode :: Mode).
(Ord (TxIn l), MemPack (TxIn l), IndexedMemPack idx (TxOut l)) =>
Proxy l
-> idx
-> LMDBMK (TxIn l) (TxOut l)
-> Transaction mode (ValuesMK (TxIn l) (TxOut l))
readAll Proxy l
_ idx
st (LMDBMK String
_ Database (TxIn l) (TxOut l)
dbMK) =
  Map (TxIn l) (TxOut l) -> ValuesMK (TxIn l) (TxOut l)
forall k v. Map k v -> ValuesMK k v
ValuesMK
    (Map (TxIn l) (TxOut l) -> ValuesMK (TxIn l) (TxOut l))
-> Transaction mode (Map (TxIn l) (TxOut l))
-> Transaction mode (ValuesMK (TxIn l) (TxOut l))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> idx
-> CursorM (TxIn l) (TxOut l) mode (Map (TxIn l) (TxOut l))
-> Database (TxIn l) (TxOut l)
-> Transaction mode (Map (TxIn l) (TxOut l))
forall k idx v (mode :: Mode) a.
(MemPack k, IndexedMemPack idx v) =>
idx -> CursorM k v mode a -> Database k v -> Transaction mode a
Bridge.runCursorAsTransaction'
      idx
st
      CursorM (TxIn l) (TxOut l) mode (Map (TxIn l) (TxOut l))
forall (m :: Mode -> * -> *) k v (mode :: Mode).
(CursorConstraints m k v mode, Ord k) =>
m mode (Map k v)
LMDB.Cursor.cgetAll
      Database (TxIn l) (TxOut l)
dbMK

-- | @'rangeRead' rq dbMK@ performs a range read of @rqCount rq@
-- values from database @dbMK@, starting from some key depending on @rqPrev rq@.
--
-- The @codec@ argument defines how to serialise/deserialise keys and values.
--
-- A range read can return less than @count@ values if there are not enough
-- values to read.
--
-- What the "first" key in the database is, and more generally in which order
-- keys are read, depends on the lexographical ordering of the /serialised/
-- keys. Care should be taken such that the @'Ord'@ instance for @k@ matches the
-- lexicographical ordering of the serialised keys, or the result of this
-- function will be unexpected.
rangeRead ::
  forall mode l idx.
  (Ord (TxIn l), MemPack (TxIn l), IndexedMemPack idx (TxOut l)) =>
  API.RangeQuery (LedgerTables l KeysMK) ->
  idx ->
  LMDBMK (TxIn l) (TxOut l) ->
  LMDB.Transaction mode (ValuesMK (TxIn l) (TxOut l))
rangeRead :: forall (mode :: Mode) (l :: LedgerStateKind) idx.
(Ord (TxIn l), MemPack (TxIn l), IndexedMemPack idx (TxOut l)) =>
RangeQuery (LedgerTables l KeysMK)
-> idx
-> LMDBMK (TxIn l) (TxOut l)
-> Transaction mode (ValuesMK (TxIn l) (TxOut l))
rangeRead RangeQuery (LedgerTables l KeysMK)
rq idx
st LMDBMK (TxIn l) (TxOut l)
dbMK =
  Map (TxIn l) (TxOut l) -> ValuesMK (TxIn l) (TxOut l)
forall k v. Map k v -> ValuesMK k v
ValuesMK (Map (TxIn l) (TxOut l) -> ValuesMK (TxIn l) (TxOut l))
-> Transaction mode (Map (TxIn l) (TxOut l))
-> Transaction mode (ValuesMK (TxIn l) (TxOut l))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> case Maybe (LedgerTables l KeysMK)
ksMK of
    Maybe (LedgerTables l KeysMK)
Nothing -> Maybe (TxIn l, Bound) -> Transaction mode (Map (TxIn l) (TxOut l))
runCursorHelper Maybe (TxIn l, Bound)
forall a. Maybe a
Nothing
    Just (LedgerTables (KeysMK Set (TxIn l)
ks)) -> case Set (TxIn l) -> Maybe (TxIn l)
forall a. Set a -> Maybe a
Set.lookupMax Set (TxIn l)
ks of
      Maybe (TxIn l)
Nothing -> Map (TxIn l) (TxOut l) -> Transaction mode (Map (TxIn l) (TxOut l))
forall a. a -> Transaction mode a
forall (f :: * -> *) a. Applicative f => a -> f a
pure Map (TxIn l) (TxOut l)
forall a. Monoid a => a
mempty
      Just TxIn l
lastExcludedKey ->
        Maybe (TxIn l, Bound) -> Transaction mode (Map (TxIn l) (TxOut l))
runCursorHelper (Maybe (TxIn l, Bound)
 -> Transaction mode (Map (TxIn l) (TxOut l)))
-> Maybe (TxIn l, Bound)
-> Transaction mode (Map (TxIn l) (TxOut l))
forall a b. (a -> b) -> a -> b
$ (TxIn l, Bound) -> Maybe (TxIn l, Bound)
forall a. a -> Maybe a
Just (TxIn l
lastExcludedKey, Bound
LMDB.Cursor.Exclusive)
 where
  LMDBMK String
_ Database (TxIn l) (TxOut l)
db = LMDBMK (TxIn l) (TxOut l)
dbMK

  API.RangeQuery Maybe (LedgerTables l KeysMK)
ksMK Int
count = RangeQuery (LedgerTables l KeysMK)
rq

  runCursorHelper ::
    Maybe (TxIn l, LMDB.Cursor.Bound) ->
    -- \^ Lower bound on read range
    LMDB.Transaction mode (Map (TxIn l) (TxOut l))
  runCursorHelper :: Maybe (TxIn l, Bound) -> Transaction mode (Map (TxIn l) (TxOut l))
runCursorHelper Maybe (TxIn l, Bound)
lb =
    idx
-> CursorM (TxIn l) (TxOut l) mode (Map (TxIn l) (TxOut l))
-> Database (TxIn l) (TxOut l)
-> Transaction mode (Map (TxIn l) (TxOut l))
forall k idx v (mode :: Mode) a.
(MemPack k, IndexedMemPack idx v) =>
idx -> CursorM k v mode a -> Database k v -> Transaction mode a
Bridge.runCursorAsTransaction'
      idx
st
      (Maybe (TxIn l, Bound)
-> Int -> CursorM (TxIn l) (TxOut l) mode (Map (TxIn l) (TxOut l))
forall (m :: Mode -> * -> *) k v (mode :: Mode).
(CursorConstraints m k v mode, Ord k) =>
Maybe (k, Bound) -> Int -> m mode (Map k v)
LMDB.Cursor.cgetMany Maybe (TxIn l, Bound)
lb Int
count)
      Database (TxIn l) (TxOut l)
db

initLMDBTable ::
  (IndexedMemPack idx v, MemPack k) =>
  idx ->
  LMDBMK k v ->
  ValuesMK k v ->
  LMDB.Transaction LMDB.ReadWrite (EmptyMK k v)
initLMDBTable :: forall idx v k.
(IndexedMemPack idx v, MemPack k) =>
idx
-> LMDBMK k v
-> ValuesMK k v
-> Transaction 'ReadWrite (EmptyMK k v)
initLMDBTable idx
st (LMDBMK String
tblName Database k v
db) (ValuesMK Map k v
utxoVals) =
  EmptyMK k v
forall k v. EmptyMK k v
EmptyMK EmptyMK k v
-> Transaction 'ReadWrite ()
-> Transaction 'ReadWrite (EmptyMK k v)
forall a b.
a -> Transaction 'ReadWrite b -> Transaction 'ReadWrite a
forall (f :: * -> *) a b. Functor f => a -> f b -> f a
<$ Transaction 'ReadWrite ()
lmdbInitTable
 where
  lmdbInitTable :: Transaction 'ReadWrite ()
lmdbInitTable = do
    isEmpty <- Database k v -> Transaction 'ReadWrite Bool
forall k v (mode :: Mode). Database k v -> Transaction mode Bool
LMDB.null Database k v
db
    unless isEmpty $ liftIO . throwIO $ LMDBErrInitialisingNonEmpty tblName
    void $
      Map.traverseWithKey
        (Bridge.indexedPut st db)
        utxoVals

readLMDBTable ::
  (IndexedMemPack idx v, MemPack k) =>
  Ord k =>
  idx ->
  LMDBMK k v ->
  KeysMK k v ->
  LMDB.Transaction mode (ValuesMK k v)
readLMDBTable :: forall idx v k (mode :: Mode).
(IndexedMemPack idx v, MemPack k, Ord k) =>
idx -> LMDBMK k v -> KeysMK k v -> Transaction mode (ValuesMK k v)
readLMDBTable idx
st (LMDBMK String
_ Database k v
db) (KeysMK Set k
keys) =
  Map k v -> ValuesMK k v
forall k v. Map k v -> ValuesMK k v
ValuesMK (Map k v -> ValuesMK k v)
-> Transaction mode (Map k v) -> Transaction mode (ValuesMK k v)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Transaction mode (Map k v)
lmdbReadTable
 where
  lmdbReadTable :: Transaction mode (Map k v)
lmdbReadTable = (Map k v -> k -> Transaction mode (Map k v))
-> Map k v -> [k] -> Transaction mode (Map k v)
forall (m :: * -> *) a b.
Monad m =>
(b -> a -> m b) -> b -> [a] -> m b
foldlM' Map k v -> k -> Transaction mode (Map k v)
go Map k v
forall k a. Map k a
Map.empty (Set k -> [k]
forall a. Set a -> [a]
Set.toList Set k
keys)
   where
    go :: Map k v -> k -> Transaction mode (Map k v)
go Map k v
m k
k =
      idx -> Database k v -> k -> Transaction mode (Maybe v)
forall idx v k (mode :: Mode).
(IndexedMemPack idx v, MemPack k) =>
idx -> Database k v -> k -> Transaction mode (Maybe v)
Bridge.indexedGet idx
st Database k v
db k
k Transaction mode (Maybe v)
-> (Maybe v -> Map k v) -> Transaction mode (Map k v)
forall (f :: * -> *) a b. Functor f => f a -> (a -> b) -> f b
<&> \case
        Maybe v
Nothing -> Map k v
m
        Just v
v -> k -> v -> Map k v -> Map k v
forall k a. Ord k => k -> a -> Map k a -> Map k a
Map.insert k
k v
v Map k v
m

writeLMDBTable ::
  (IndexedMemPack idx v, MemPack k) =>
  idx ->
  LMDBMK k v ->
  DiffMK k v ->
  LMDB.Transaction LMDB.ReadWrite (EmptyMK k v)
writeLMDBTable :: forall idx v k.
(IndexedMemPack idx v, MemPack k) =>
idx
-> LMDBMK k v -> DiffMK k v -> Transaction 'ReadWrite (EmptyMK k v)
writeLMDBTable idx
st (LMDBMK String
_ Database k v
db) (DiffMK Diff k v
d) =
  EmptyMK k v
forall k v. EmptyMK k v
EmptyMK EmptyMK k v
-> Transaction 'ReadWrite ()
-> Transaction 'ReadWrite (EmptyMK k v)
forall a b.
a -> Transaction 'ReadWrite b -> Transaction 'ReadWrite a
forall (f :: * -> *) a b. Functor f => a -> f b -> f a
<$ Transaction 'ReadWrite ()
lmdbWriteTable
 where
  lmdbWriteTable :: Transaction 'ReadWrite ()
lmdbWriteTable = Transaction 'ReadWrite () -> Transaction 'ReadWrite ()
forall (f :: * -> *) a. Functor f => f a -> f ()
void (Transaction 'ReadWrite () -> Transaction 'ReadWrite ())
-> Transaction 'ReadWrite () -> Transaction 'ReadWrite ()
forall a b. (a -> b) -> a -> b
$ (k -> Delta v -> Transaction 'ReadWrite ())
-> Diff k v -> Transaction 'ReadWrite ()
forall (t :: * -> *) k v a.
Applicative t =>
(k -> Delta v -> t a) -> Diff k v -> t ()
Diff.traverseDeltaWithKey_ k -> Delta v -> Transaction 'ReadWrite ()
go Diff k v
d
   where
    go :: k -> Delta v -> Transaction 'ReadWrite ()
go k
k Delta v
de = case Delta v
de of
      Delta v
Diff.Delete -> Transaction 'ReadWrite Bool -> Transaction 'ReadWrite ()
forall (f :: * -> *) a. Functor f => f a -> f ()
void (Transaction 'ReadWrite Bool -> Transaction 'ReadWrite ())
-> Transaction 'ReadWrite Bool -> Transaction 'ReadWrite ()
forall a b. (a -> b) -> a -> b
$ Database k v -> k -> Transaction 'ReadWrite Bool
forall k v.
MemPack k =>
Database k v -> k -> Transaction 'ReadWrite Bool
Bridge.delete Database k v
db k
k
      Diff.Insert v
v -> idx -> Database k v -> k -> v -> Transaction 'ReadWrite ()
forall idx v k.
(IndexedMemPack idx v, MemPack k) =>
idx -> Database k v -> k -> v -> Transaction 'ReadWrite ()
Bridge.indexedPut idx
st Database k v
db k
k v
v

{-------------------------------------------------------------------------------
 Db state
-------------------------------------------------------------------------------}

readDbSeqNoMaybeNull ::
  LMDB.Database () DbSeqNo ->
  LMDB.Transaction mode (Maybe DbSeqNo)
readDbSeqNoMaybeNull :: forall (mode :: Mode).
Database () DbSeqNo -> Transaction mode (Maybe DbSeqNo)
readDbSeqNoMaybeNull Database () DbSeqNo
db = Database () DbSeqNo -> () -> Transaction mode (Maybe DbSeqNo)
forall k v (mode :: Mode).
(Serialise k, Serialise v) =>
Database k v -> k -> Transaction mode (Maybe v)
LMDB.get Database () DbSeqNo
db ()

readDbSeqNo ::
  LMDB.Database () DbSeqNo ->
  LMDB.Transaction mode DbSeqNo
readDbSeqNo :: forall (mode :: Mode).
Database () DbSeqNo -> Transaction mode DbSeqNo
readDbSeqNo Database () DbSeqNo
db = Database () DbSeqNo -> Transaction mode (Maybe DbSeqNo)
forall (mode :: Mode).
Database () DbSeqNo -> Transaction mode (Maybe DbSeqNo)
readDbSeqNoMaybeNull Database () DbSeqNo
db Transaction mode (Maybe DbSeqNo)
-> (Maybe DbSeqNo -> Transaction mode DbSeqNo)
-> Transaction mode DbSeqNo
forall a b.
Transaction mode a
-> (a -> Transaction mode b) -> Transaction mode b
forall (m :: * -> *) a b. Monad m => m a -> (a -> m b) -> m b
>>= Transaction mode DbSeqNo
-> (DbSeqNo -> Transaction mode DbSeqNo)
-> Maybe DbSeqNo
-> Transaction mode DbSeqNo
forall b a. b -> (a -> b) -> Maybe a -> b
maybe (IO DbSeqNo -> Transaction mode DbSeqNo
forall a. IO a -> Transaction mode a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO (IO DbSeqNo -> Transaction mode DbSeqNo)
-> (LMDBErr -> IO DbSeqNo) -> LMDBErr -> Transaction mode DbSeqNo
forall b c a. (b -> c) -> (a -> b) -> a -> c
. LMDBErr -> IO DbSeqNo
forall e a. Exception e => e -> IO a
forall (m :: * -> *) e a. (MonadThrow m, Exception e) => e -> m a
throwIO (LMDBErr -> Transaction mode DbSeqNo)
-> LMDBErr -> Transaction mode DbSeqNo
forall a b. (a -> b) -> a -> b
$ LMDBErr
LMDBErrNoDbSeqNo) DbSeqNo -> Transaction mode DbSeqNo
forall a. a -> Transaction mode a
forall (f :: * -> *) a. Applicative f => a -> f a
pure

withDbSeqNoRW ::
  LMDB.Database () DbSeqNo ->
  (DbSeqNo -> LMDB.Transaction LMDB.ReadWrite (a, DbSeqNo)) ->
  LMDB.Transaction LMDB.ReadWrite a
withDbSeqNoRW :: forall a.
Database () DbSeqNo
-> (DbSeqNo -> Transaction 'ReadWrite (a, DbSeqNo))
-> Transaction 'ReadWrite a
withDbSeqNoRW Database () DbSeqNo
db DbSeqNo -> Transaction 'ReadWrite (a, DbSeqNo)
f = Database () DbSeqNo
-> (Maybe DbSeqNo -> Transaction 'ReadWrite (a, DbSeqNo))
-> Transaction 'ReadWrite a
forall a.
Database () DbSeqNo
-> (Maybe DbSeqNo -> Transaction 'ReadWrite (a, DbSeqNo))
-> Transaction 'ReadWrite a
withDbSeqNoRWMaybeNull Database () DbSeqNo
db ((Maybe DbSeqNo -> Transaction 'ReadWrite (a, DbSeqNo))
 -> Transaction 'ReadWrite a)
-> (Maybe DbSeqNo -> Transaction 'ReadWrite (a, DbSeqNo))
-> Transaction 'ReadWrite a
forall a b. (a -> b) -> a -> b
$ Transaction 'ReadWrite (a, DbSeqNo)
-> (DbSeqNo -> Transaction 'ReadWrite (a, DbSeqNo))
-> Maybe DbSeqNo
-> Transaction 'ReadWrite (a, DbSeqNo)
forall b a. b -> (a -> b) -> Maybe a -> b
maybe (IO (a, DbSeqNo) -> Transaction 'ReadWrite (a, DbSeqNo)
forall a. IO a -> Transaction 'ReadWrite a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO (IO (a, DbSeqNo) -> Transaction 'ReadWrite (a, DbSeqNo))
-> (LMDBErr -> IO (a, DbSeqNo))
-> LMDBErr
-> Transaction 'ReadWrite (a, DbSeqNo)
forall b c a. (b -> c) -> (a -> b) -> a -> c
. LMDBErr -> IO (a, DbSeqNo)
forall e a. Exception e => e -> IO a
forall (m :: * -> *) e a. (MonadThrow m, Exception e) => e -> m a
throwIO (LMDBErr -> Transaction 'ReadWrite (a, DbSeqNo))
-> LMDBErr -> Transaction 'ReadWrite (a, DbSeqNo)
forall a b. (a -> b) -> a -> b
$ LMDBErr
LMDBErrNoDbSeqNo) DbSeqNo -> Transaction 'ReadWrite (a, DbSeqNo)
f

withDbSeqNoRWMaybeNull ::
  LMDB.Database () DbSeqNo ->
  (Maybe DbSeqNo -> LMDB.Transaction LMDB.ReadWrite (a, DbSeqNo)) ->
  LMDB.Transaction LMDB.ReadWrite a
withDbSeqNoRWMaybeNull :: forall a.
Database () DbSeqNo
-> (Maybe DbSeqNo -> Transaction 'ReadWrite (a, DbSeqNo))
-> Transaction 'ReadWrite a
withDbSeqNoRWMaybeNull Database () DbSeqNo
db Maybe DbSeqNo -> Transaction 'ReadWrite (a, DbSeqNo)
f =
  Database () DbSeqNo -> Transaction 'ReadWrite (Maybe DbSeqNo)
forall (mode :: Mode).
Database () DbSeqNo -> Transaction mode (Maybe DbSeqNo)
readDbSeqNoMaybeNull Database () DbSeqNo
db Transaction 'ReadWrite (Maybe DbSeqNo)
-> (Maybe DbSeqNo -> Transaction 'ReadWrite (a, DbSeqNo))
-> Transaction 'ReadWrite (a, DbSeqNo)
forall a b.
Transaction 'ReadWrite a
-> (a -> Transaction 'ReadWrite b) -> Transaction 'ReadWrite b
forall (m :: * -> *) a b. Monad m => m a -> (a -> m b) -> m b
>>= Maybe DbSeqNo -> Transaction 'ReadWrite (a, DbSeqNo)
f Transaction 'ReadWrite (a, DbSeqNo)
-> ((a, DbSeqNo) -> Transaction 'ReadWrite a)
-> Transaction 'ReadWrite a
forall a b.
Transaction 'ReadWrite a
-> (a -> Transaction 'ReadWrite b) -> Transaction 'ReadWrite b
forall (m :: * -> *) a b. Monad m => m a -> (a -> m b) -> m b
>>= \(a
r, DbSeqNo
sNew) -> Database () DbSeqNo
-> () -> Maybe DbSeqNo -> Transaction 'ReadWrite ()
forall k v.
(Serialise k, Serialise v) =>
Database k v -> k -> Maybe v -> Transaction 'ReadWrite ()
LMDB.put Database () DbSeqNo
db () (DbSeqNo -> Maybe DbSeqNo
forall a. a -> Maybe a
Just DbSeqNo
sNew) Transaction 'ReadWrite () -> a -> Transaction 'ReadWrite a
forall (f :: * -> *) a b. Functor f => f a -> b -> f b
$> a
r

{-------------------------------------------------------------------------------
 Guards
-------------------------------------------------------------------------------}

data GuardDbDir = DirMustExist | DirMustNotExist

-- | Guard for the existence/non-existence of a database directory,
-- and create it if missing.
checkAndOpenDbDir ::
  (MonadIO m, IOLike m) =>
  GuardDbDir ->
  FS.SomeHasFS m ->
  FS.FsPath ->
  m FilePath
checkAndOpenDbDir :: forall (m :: * -> *).
(MonadIO m, IOLike m) =>
GuardDbDir -> SomeHasFS m -> FsPath -> m String
checkAndOpenDbDir GuardDbDir
mustExistDir (FS.SomeHasFS HasFS m h
fs) FsPath
path = do
  fileEx <- HasFS m h -> HasCallStack => FsPath -> m Bool
forall (m :: * -> *) h.
HasFS m h -> HasCallStack => FsPath -> m Bool
FS.doesFileExist HasFS m h
fs FsPath
path
  when fileEx $
    throwIO $
      LMDBErrNotADir path
  dirEx <- FS.doesDirectoryExist fs path
  lmdbFileExists <-
    FS.doesFileExist fs path{FS.fsPathToList = FS.fsPathToList path ++ [Strict.pack "data.mdb"]}
  filepath <- FS.unsafeToFilePath fs path
  case dirEx of
    Bool
True
      | GuardDbDir
DirMustNotExist <- GuardDbDir
mustExistDir -> LMDBErr -> m ()
forall e a. Exception e => e -> m a
forall (m :: * -> *) e a. (MonadThrow m, Exception e) => e -> m a
throwIO (LMDBErr -> m ()) -> LMDBErr -> m ()
forall a b. (a -> b) -> a -> b
$ String -> LMDBErr
LMDBErrDirExists String
filepath
      | Bool -> Bool
not Bool
lmdbFileExists -> LMDBErr -> m ()
forall e a. Exception e => e -> m a
forall (m :: * -> *) e a. (MonadThrow m, Exception e) => e -> m a
throwIO (LMDBErr -> m ()) -> LMDBErr -> m ()
forall a b. (a -> b) -> a -> b
$ String -> LMDBErr
LMDBErrDirIsNotLMDB String
filepath
      | Bool
otherwise -> () -> m ()
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure ()
    Bool
False
      | GuardDbDir
DirMustExist <- GuardDbDir
mustExistDir -> LMDBErr -> m ()
forall e a. Exception e => e -> m a
forall (m :: * -> *) e a. (MonadThrow m, Exception e) => e -> m a
throwIO (LMDBErr -> m ()) -> LMDBErr -> m ()
forall a b. (a -> b) -> a -> b
$ String -> LMDBErr
LMDBErrDirDoesntExist String
filepath
      | Bool
otherwise -> () -> m ()
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure ()
  FS.createDirectoryIfMissing fs True path
  pure filepath

-- | Same as @`checkAndOpenDbDir`@, but retries the guard if we can make meaningful
-- changes to the filesystem before we perform the retry.
--
-- Note: We only retry if a database directory exists while it shoudn't. In
-- this case, we remove the directory recursively before retrying the guard.
-- This is necessary for initialisation of the LMDB backing store, since the
-- (non-snapshot) tables will probably still be on-disk. These tables are not
-- removed when stopping the node, so they should be "overwritten".
checkAndOpenDbDirWithRetry ::
  (MonadIO m, IOLike m) =>
  GuardDbDir ->
  FS.SomeHasFS m ->
  FS.FsPath ->
  m FilePath
checkAndOpenDbDirWithRetry :: forall (m :: * -> *).
(MonadIO m, IOLike m) =>
GuardDbDir -> SomeHasFS m -> FsPath -> m String
checkAndOpenDbDirWithRetry GuardDbDir
gdd shfs :: SomeHasFS m
shfs@(FS.SomeHasFS HasFS m h
fs) FsPath
path =
  (LMDBErr -> m String) -> m String -> m String
forall e a. Exception e => (e -> m a) -> m a -> m a
forall (m :: * -> *) e a.
(MonadCatch m, Exception e) =>
(e -> m a) -> m a -> m a
handle LMDBErr -> m String
retryHandler (GuardDbDir -> SomeHasFS m -> FsPath -> m String
forall (m :: * -> *).
(MonadIO m, IOLike m) =>
GuardDbDir -> SomeHasFS m -> FsPath -> m String
checkAndOpenDbDir GuardDbDir
gdd SomeHasFS m
shfs FsPath
path)
 where
  retryHandler :: LMDBErr -> m String
retryHandler LMDBErr
e = case (GuardDbDir
gdd, LMDBErr
e) of
    (GuardDbDir
DirMustNotExist, LMDBErrDirExists String
_path) -> do
      HasFS m h -> HasCallStack => FsPath -> m ()
forall (m :: * -> *) h. HasFS m h -> HasCallStack => FsPath -> m ()
FS.removeDirectoryRecursive HasFS m h
fs FsPath
path
      GuardDbDir -> SomeHasFS m -> FsPath -> m String
forall (m :: * -> *).
(MonadIO m, IOLike m) =>
GuardDbDir -> SomeHasFS m -> FsPath -> m String
checkAndOpenDbDir GuardDbDir
DirMustNotExist SomeHasFS m
shfs FsPath
path
    (GuardDbDir, LMDBErr)
_ -> LMDBErr -> m String
forall e a. Exception e => e -> m a
forall (m :: * -> *) e a. (MonadThrow m, Exception e) => e -> m a
throwIO LMDBErr
e

{-------------------------------------------------------------------------------
 Initialize an LMDB
-------------------------------------------------------------------------------}

-- | Initialise an LMDB database from these provided values.
initFromVals ::
  forall l m.
  (HasLedgerTables l, MonadIO m, MemPackIdx l EmptyMK ~ l EmptyMK) =>
  Trace.Tracer m API.BackingStoreTrace ->
  -- | The slot number up to which the ledger tables contain values.
  WithOrigin SlotNo ->
  -- | The ledger tables to initialise the LMDB database tables with.
  LedgerTables l ValuesMK ->
  -- | The LMDB environment.
  LMDB.Environment LMDB.Internal.ReadWrite ->
  LMDB.Database () DbSeqNo ->
  l EmptyMK ->
  LedgerTables l LMDBMK ->
  m ()
initFromVals :: forall (l :: LedgerStateKind) (m :: * -> *).
(HasLedgerTables l, MonadIO m, MemPackIdx l EmptyMK ~ l EmptyMK) =>
Tracer m BackingStoreTrace
-> WithOrigin SlotNo
-> LedgerTables l ValuesMK
-> Environment 'ReadWrite
-> Database () DbSeqNo
-> l EmptyMK
-> LedgerTables l LMDBMK
-> m ()
initFromVals Tracer m BackingStoreTrace
tracer WithOrigin SlotNo
dbsSeq LedgerTables l ValuesMK
vals Environment 'ReadWrite
env Database () DbSeqNo
st l EmptyMK
lst LedgerTables l LMDBMK
backingTables = do
  Tracer m BackingStoreTrace -> BackingStoreTrace -> m ()
forall (m :: * -> *) a. Tracer m a -> a -> m ()
Trace.traceWith Tracer m BackingStoreTrace
tracer (BackingStoreTrace -> m ()) -> BackingStoreTrace -> m ()
forall a b. (a -> b) -> a -> b
$ WithOrigin SlotNo -> BackingStoreTrace
API.BSInitialisingFromValues WithOrigin SlotNo
dbsSeq
  IO () -> m ()
forall a. IO a -> m a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO (IO () -> m ()) -> IO () -> m ()
forall a b. (a -> b) -> a -> b
$
    Environment 'ReadWrite -> Transaction 'ReadWrite () -> IO ()
forall a.
Environment 'ReadWrite -> Transaction 'ReadWrite a -> IO a
LMDB.readWriteTransaction Environment 'ReadWrite
env (Transaction 'ReadWrite () -> IO ())
-> Transaction 'ReadWrite () -> IO ()
forall a b. (a -> b) -> a -> b
$
      Database () DbSeqNo
-> (Maybe DbSeqNo -> Transaction 'ReadWrite ((), DbSeqNo))
-> Transaction 'ReadWrite ()
forall a.
Database () DbSeqNo
-> (Maybe DbSeqNo -> Transaction 'ReadWrite (a, DbSeqNo))
-> Transaction 'ReadWrite a
withDbSeqNoRWMaybeNull Database () DbSeqNo
st ((Maybe DbSeqNo -> Transaction 'ReadWrite ((), DbSeqNo))
 -> Transaction 'ReadWrite ())
-> (Maybe DbSeqNo -> Transaction 'ReadWrite ((), DbSeqNo))
-> Transaction 'ReadWrite ()
forall a b. (a -> b) -> a -> b
$ \case
        Maybe DbSeqNo
Nothing ->
          (forall k v.
 LedgerTableConstraints' l k v =>
 LMDBMK k v -> ValuesMK k v -> Transaction 'ReadWrite (EmptyMK k v))
-> LedgerTables l LMDBMK
-> LedgerTables l ValuesMK
-> Transaction 'ReadWrite (LedgerTables l EmptyMK)
forall (f :: * -> *) (l :: LedgerStateKind) (mk1 :: MapKind)
       (mk2 :: MapKind) (mk3 :: MapKind).
(Applicative f, LedgerTableConstraints l) =>
(forall k v.
 LedgerTableConstraints' l k v =>
 mk1 k v -> mk2 k v -> f (mk3 k v))
-> LedgerTables l mk1
-> LedgerTables l mk2
-> f (LedgerTables l mk3)
ltzipWith2A (l EmptyMK
-> LMDBMK k v
-> ValuesMK k v
-> Transaction 'ReadWrite (EmptyMK k v)
forall idx v k.
(IndexedMemPack idx v, MemPack k) =>
idx
-> LMDBMK k v
-> ValuesMK k v
-> Transaction 'ReadWrite (EmptyMK k v)
initLMDBTable l EmptyMK
lst) LedgerTables l LMDBMK
backingTables LedgerTables l ValuesMK
vals
            Transaction 'ReadWrite (LedgerTables l EmptyMK)
-> ((), DbSeqNo) -> Transaction 'ReadWrite ((), DbSeqNo)
forall (f :: * -> *) a b. Functor f => f a -> b -> f b
$> ((), DbSeqNo{WithOrigin SlotNo
dbsSeq :: WithOrigin SlotNo
dbsSeq :: WithOrigin SlotNo
dbsSeq})
        Just DbSeqNo
_ -> IO ((), DbSeqNo) -> Transaction 'ReadWrite ((), DbSeqNo)
forall a. IO a -> Transaction 'ReadWrite a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO (IO ((), DbSeqNo) -> Transaction 'ReadWrite ((), DbSeqNo))
-> (LMDBErr -> IO ((), DbSeqNo))
-> LMDBErr
-> Transaction 'ReadWrite ((), DbSeqNo)
forall b c a. (b -> c) -> (a -> b) -> a -> c
. LMDBErr -> IO ((), DbSeqNo)
forall e a. Exception e => e -> IO a
forall (m :: * -> *) e a. (MonadThrow m, Exception e) => e -> m a
throwIO (LMDBErr -> Transaction 'ReadWrite ((), DbSeqNo))
-> LMDBErr -> Transaction 'ReadWrite ((), DbSeqNo)
forall a b. (a -> b) -> a -> b
$ LMDBErr
LMDBErrInitialisingAlreadyHasState
  Tracer m BackingStoreTrace -> BackingStoreTrace -> m ()
forall (m :: * -> *) a. Tracer m a -> a -> m ()
Trace.traceWith Tracer m BackingStoreTrace
tracer (BackingStoreTrace -> m ()) -> BackingStoreTrace -> m ()
forall a b. (a -> b) -> a -> b
$ WithOrigin SlotNo -> BackingStoreTrace
API.BSInitialisedFromValues WithOrigin SlotNo
dbsSeq

-- | Initialise an LMDB database from an existing LMDB database.
initFromLMDBs ::
  (MonadIO m, IOLike m) =>
  Trace.Tracer m API.BackingStoreTrace ->
  -- | Configuration for the LMDB database that we initialise from.
  LMDBLimits ->
  -- | Abstraction over the filesystem.
  API.SnapshotsFS m ->
  -- | The path that contains the LMDB database that we want to initialise from.
  FS.FsPath ->
  -- | Abstraction over the filesystem.
  API.LiveLMDBFS m ->
  -- | The path where the new LMDB database should be initialised.
  FS.FsPath ->
  m ()
initFromLMDBs :: forall (m :: * -> *).
(MonadIO m, IOLike m) =>
Tracer m BackingStoreTrace
-> LMDBLimits
-> SnapshotsFS m
-> FsPath
-> LiveLMDBFS m
-> FsPath
-> m ()
initFromLMDBs Tracer m BackingStoreTrace
tracer LMDBLimits
limits (API.SnapshotsFS shfsFrom :: SomeHasFS m
shfsFrom@(FS.SomeHasFS HasFS m h
fsFrom)) FsPath
from0 (API.LiveLMDBFS SomeHasFS m
shfsTo) FsPath
to0 = do
  Tracer m BackingStoreTrace -> BackingStoreTrace -> m ()
forall (m :: * -> *) a. Tracer m a -> a -> m ()
Trace.traceWith Tracer m BackingStoreTrace
tracer (BackingStoreTrace -> m ()) -> BackingStoreTrace -> m ()
forall a b. (a -> b) -> a -> b
$ FsPath -> BackingStoreTrace
API.BSInitialisingFromCopy FsPath
from0
  from <- GuardDbDir -> SomeHasFS m -> FsPath -> m String
forall (m :: * -> *).
(MonadIO m, IOLike m) =>
GuardDbDir -> SomeHasFS m -> FsPath -> m String
checkAndOpenDbDir GuardDbDir
DirMustExist SomeHasFS m
shfsFrom FsPath
from0
  -- On Windows, if we don't choose the mapsize carefully it will make the
  -- snapshot grow. Therefore we are using the current filesize as mapsize
  -- when opening the snapshot to avoid this.
  stat <-
    FS.withFile
      fsFrom
      (from0{FS.fsPathToList = FS.fsPathToList from0 ++ [Strict.pack "data.mdb"]})
      FS.ReadMode
      (FS.hGetSize fsFrom)
  to <- checkAndOpenDbDirWithRetry DirMustNotExist shfsTo to0
  bracket
    (liftIO $ LMDB.openEnvironment from ((unLMDBLimits limits){LMDB.mapSize = fromIntegral stat}))
    (liftIO . LMDB.closeEnvironment)
    (flip (lmdbCopy from0 tracer) to)
  Trace.traceWith tracer $ API.BSInitialisedFromCopy from0

-- | Copy an existing LMDB database to a given directory.
lmdbCopy ::
  MonadIO m =>
  FS.FsPath ->
  Trace.Tracer m API.BackingStoreTrace ->
  -- | The environment in which the LMDB database lives.
  LMDB.Environment LMDB.ReadWrite ->
  -- | The path where the copy should reside.
  FilePath ->
  m ()
lmdbCopy :: forall (m :: * -> *).
MonadIO m =>
FsPath
-> Tracer m BackingStoreTrace
-> Environment 'ReadWrite
-> String
-> m ()
lmdbCopy FsPath
from0 Tracer m BackingStoreTrace
tracer Environment 'ReadWrite
e String
to = do
  Tracer m BackingStoreTrace -> BackingStoreTrace -> m ()
forall (m :: * -> *) a. Tracer m a -> a -> m ()
Trace.traceWith Tracer m BackingStoreTrace
tracer (BackingStoreTrace -> m ()) -> BackingStoreTrace -> m ()
forall a b. (a -> b) -> a -> b
$ FsPath -> BackingStoreTrace
API.BSCopying FsPath
from0
  IO () -> m ()
forall a. IO a -> m a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO (IO () -> m ()) -> IO () -> m ()
forall a b. (a -> b) -> a -> b
$ Environment 'ReadWrite -> String -> IO ()
forall (mode :: Mode). Environment mode -> String -> IO ()
LMDB.copyEnvironment Environment 'ReadWrite
e String
to
  Tracer m BackingStoreTrace -> BackingStoreTrace -> m ()
forall (m :: * -> *) a. Tracer m a -> a -> m ()
Trace.traceWith Tracer m BackingStoreTrace
tracer (BackingStoreTrace -> m ()) -> BackingStoreTrace -> m ()
forall a b. (a -> b) -> a -> b
$ FsPath -> BackingStoreTrace
API.BSCopied FsPath
from0

-- | Initialise a backing store.
newLMDBBackingStore ::
  forall m l.
  ( HasCallStack
  , HasLedgerTables l
  , MonadIO m
  , IOLike m
  , MemPackIdx l EmptyMK ~ l EmptyMK
  ) =>
  Trace.Tracer m API.BackingStoreTrace ->
  -- | Configuration parameters for the LMDB database that we
  -- initialise. In case we initialise the LMDB database from
  -- an existing LMDB database, we use these same configuration parameters
  -- to open the existing LMDB database.
  LMDBLimits ->
  -- | The FS for the LMDB live database
  API.LiveLMDBFS m ->
  API.SnapshotsFS m ->
  API.InitFrom (LedgerTables l ValuesMK) ->
  m (API.LedgerBackingStore m l)
newLMDBBackingStore :: forall (m :: * -> *) (l :: LedgerStateKind).
(HasCallStack, HasLedgerTables l, MonadIO m, IOLike m,
 MemPackIdx l EmptyMK ~ l EmptyMK) =>
Tracer m BackingStoreTrace
-> LMDBLimits
-> LiveLMDBFS m
-> SnapshotsFS m
-> InitFrom (LedgerTables l ValuesMK)
-> m (LedgerBackingStore m l)
newLMDBBackingStore Tracer m BackingStoreTrace
dbTracer LMDBLimits
limits liveFS :: LiveLMDBFS m
liveFS@(API.LiveLMDBFS SomeHasFS m
liveFS') snapFS :: SnapshotsFS m
snapFS@(API.SnapshotsFS SomeHasFS m
snapFS') InitFrom (LedgerTables l ValuesMK)
initFrom = do
  Tracer m BackingStoreTrace -> BackingStoreTrace -> m ()
forall (m :: * -> *) a. Tracer m a -> a -> m ()
Trace.traceWith Tracer m BackingStoreTrace
dbTracer BackingStoreTrace
API.BSOpening

  db@Db
    { dbEnv
    , dbState
    , dbBackingTables
    } <-
    m (Db m l)
createOrGetDB

  maybePopulate dbEnv dbState dbBackingTables

  Trace.traceWith dbTracer $ API.BSOpened $ Just path

  pure $ mkBackingStore db
 where
  path :: FsPath
path = [String] -> FsPath
FS.mkFsPath [String
"tables"]

  st :: InitHint (LedgerTables l ValuesMK)
st = case InitFrom (LedgerTables l ValuesMK)
initFrom of
    API.InitFromCopy InitHint (LedgerTables l ValuesMK)
st' FsPath
_ -> InitHint (LedgerTables l ValuesMK)
st'
    API.InitFromValues WithOrigin SlotNo
_ InitHint (LedgerTables l ValuesMK)
st' LedgerTables l ValuesMK
_ -> InitHint (LedgerTables l ValuesMK)
st'

  createOrGetDB :: m (Db m l)
  createOrGetDB :: m (Db m l)
createOrGetDB = do
    dbOpenHandles <- Map Int (Cleanup m) -> m (TVar m (Map Int (Cleanup m)))
forall a. a -> m (TVar m a)
forall (m :: * -> *) a. MonadSTM m => a -> m (TVar m a)
IOLike.newTVarIO Map Int (Cleanup m)
forall k a. Map k a
Map.empty
    dbStatusLock <- Status.new Open

    -- get the filepath for this db creates the directory if appropriate
    dbFilePath <- checkAndOpenDbDirWithRetry DirMustNotExist liveFS' path

    -- copy from another lmdb path if appropriate
    case initFrom of
      API.InitFromCopy InitHint (LedgerTables l ValuesMK)
_ FsPath
fp -> Tracer m BackingStoreTrace
-> LMDBLimits
-> SnapshotsFS m
-> FsPath
-> LiveLMDBFS m
-> FsPath
-> m ()
forall (m :: * -> *).
(MonadIO m, IOLike m) =>
Tracer m BackingStoreTrace
-> LMDBLimits
-> SnapshotsFS m
-> FsPath
-> LiveLMDBFS m
-> FsPath
-> m ()
initFromLMDBs Tracer m BackingStoreTrace
dbTracer LMDBLimits
limits SnapshotsFS m
snapFS FsPath
fp LiveLMDBFS m
liveFS FsPath
path
      API.InitFromValues{} -> () -> m ()
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure ()

    -- open this database
    dbEnv <- liftIO $ LMDB.openEnvironment dbFilePath (unLMDBLimits limits)

    -- The LMDB.Database that holds the @`DbSeqNo`@ (i.e. sequence number)
    -- This transaction must be read-write because on initialisation it creates the database
    dbState <- liftIO $ LMDB.readWriteTransaction dbEnv $ LMDB.getDatabase (Just "_dbstate")

    -- Here we get the LMDB.Databases for the tables of the ledger state
    -- Must be read-write transaction because tables may need to be created
    dbBackingTables <-
      liftIO $
        LMDB.readWriteTransaction dbEnv $
          lttraverse getDb (ltpure $ K2 "utxo")

    dbNextId <- IOLike.newTVarIO 0

    pure $
      Db
        { dbEnv
        , dbState
        , dbBackingTables
        , dbFilePath
        , dbTracer
        , dbStatusLock
        , dbOpenHandles
        , dbNextId
        }

  maybePopulate ::
    LMDB.Internal.Environment LMDB.Internal.ReadWrite ->
    LMDB.Internal.Database () DbSeqNo ->
    LedgerTables l LMDBMK ->
    m ()
  maybePopulate :: Environment 'ReadWrite
-> Database () DbSeqNo -> LedgerTables l LMDBMK -> m ()
maybePopulate Environment 'ReadWrite
dbEnv Database () DbSeqNo
dbState LedgerTables l LMDBMK
dbBackingTables = do
    -- now initialise those tables if appropriate
    case InitFrom (LedgerTables l ValuesMK)
initFrom of
      API.InitFromValues WithOrigin SlotNo
slot InitHint (LedgerTables l ValuesMK)
_ LedgerTables l ValuesMK
vals -> Tracer m BackingStoreTrace
-> WithOrigin SlotNo
-> LedgerTables l ValuesMK
-> Environment 'ReadWrite
-> Database () DbSeqNo
-> l EmptyMK
-> LedgerTables l LMDBMK
-> m ()
forall (l :: LedgerStateKind) (m :: * -> *).
(HasLedgerTables l, MonadIO m, MemPackIdx l EmptyMK ~ l EmptyMK) =>
Tracer m BackingStoreTrace
-> WithOrigin SlotNo
-> LedgerTables l ValuesMK
-> Environment 'ReadWrite
-> Database () DbSeqNo
-> l EmptyMK
-> LedgerTables l LMDBMK
-> m ()
initFromVals Tracer m BackingStoreTrace
dbTracer WithOrigin SlotNo
slot LedgerTables l ValuesMK
vals Environment 'ReadWrite
dbEnv Database () DbSeqNo
dbState l EmptyMK
InitHint (LedgerTables l ValuesMK)
st LedgerTables l LMDBMK
dbBackingTables
      API.InitFromCopy{} -> () -> m ()
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure ()

  mkBackingStore :: HasCallStack => Db m l -> API.LedgerBackingStore m l
  mkBackingStore :: HasCallStack => Db m l -> LedgerBackingStore m l
mkBackingStore Db m l
db =
    let bsClose :: m ()
        bsClose :: m ()
bsClose = StatusLock m -> m () -> m ((), Status) -> m ()
forall (m :: * -> *) a.
IOLike m =>
StatusLock m -> m a -> m (a, Status) -> m a
Status.withWriteAccess StatusLock m
dbStatusLock m ()
traceAlreadyClosed (m ((), Status) -> m ()) -> m ((), Status) -> m ()
forall a b. (a -> b) -> a -> b
$ do
          Tracer m BackingStoreTrace -> BackingStoreTrace -> m ()
forall (m :: * -> *) a. Tracer m a -> a -> m ()
Trace.traceWith Tracer m BackingStoreTrace
dbTracer BackingStoreTrace
API.BSClosing
          openHandles <- TVar m (Map Int (Cleanup m)) -> m (Map Int (Cleanup m))
forall a. TVar m a -> m a
forall (m :: * -> *) a. MonadSTM m => TVar m a -> m a
IOLike.readTVarIO TVar m (Map Int (Cleanup m))
dbOpenHandles
          forM_ openHandles runCleanup
          IOLike.atomically $ IOLike.writeTVar dbOpenHandles mempty
          liftIO $ LMDB.closeEnvironment dbEnv
          Trace.traceWith dbTracer API.BSClosed
          pure ((), Closed)
         where
          traceAlreadyClosed :: m ()
traceAlreadyClosed = Tracer m BackingStoreTrace -> BackingStoreTrace -> m ()
forall (m :: * -> *) a. Tracer m a -> a -> m ()
Trace.traceWith Tracer m BackingStoreTrace
dbTracer BackingStoreTrace
API.BSAlreadyClosed

        bsCopy :: FsPath -> m ()
bsCopy FsPath
bsp = StatusLock m -> m () -> m () -> m ()
forall (m :: * -> *) a.
IOLike m =>
StatusLock m -> m a -> m a -> m a
Status.withReadAccess StatusLock m
dbStatusLock (LMDBErr -> m ()
forall e a. Exception e => e -> m a
forall (m :: * -> *) e a. (MonadThrow m, Exception e) => e -> m a
throwIO LMDBErr
LMDBErrClosed) (m () -> m ()) -> m () -> m ()
forall a b. (a -> b) -> a -> b
$ do
          to <- GuardDbDir -> SomeHasFS m -> FsPath -> m String
forall (m :: * -> *).
(MonadIO m, IOLike m) =>
GuardDbDir -> SomeHasFS m -> FsPath -> m String
checkAndOpenDbDir GuardDbDir
DirMustNotExist SomeHasFS m
snapFS' FsPath
bsp
          lmdbCopy path dbTracer dbEnv to

        bsValueHandle :: m (BackingStoreValueHandle
     m (LedgerTables l KeysMK) (LedgerTables l ValuesMK))
bsValueHandle = StatusLock m
-> m (BackingStoreValueHandle
        m (LedgerTables l KeysMK) (LedgerTables l ValuesMK))
-> m (BackingStoreValueHandle
        m (LedgerTables l KeysMK) (LedgerTables l ValuesMK))
-> m (BackingStoreValueHandle
        m (LedgerTables l KeysMK) (LedgerTables l ValuesMK))
forall (m :: * -> *) a.
IOLike m =>
StatusLock m -> m a -> m a -> m a
Status.withReadAccess StatusLock m
dbStatusLock (LMDBErr
-> m (BackingStoreValueHandle
        m (LedgerTables l KeysMK) (LedgerTables l ValuesMK))
forall e a. Exception e => e -> m a
forall (m :: * -> *) e a. (MonadThrow m, Exception e) => e -> m a
throwIO LMDBErr
LMDBErrClosed) (m (BackingStoreValueHandle
      m (LedgerTables l KeysMK) (LedgerTables l ValuesMK))
 -> m (BackingStoreValueHandle
         m (LedgerTables l KeysMK) (LedgerTables l ValuesMK)))
-> m (BackingStoreValueHandle
        m (LedgerTables l KeysMK) (LedgerTables l ValuesMK))
-> m (BackingStoreValueHandle
        m (LedgerTables l KeysMK) (LedgerTables l ValuesMK))
forall a b. (a -> b) -> a -> b
$ do
          Db m l
-> m (BackingStoreValueHandle
        m (LedgerTables l KeysMK) (LedgerTables l ValuesMK))
forall (l :: LedgerStateKind) (m :: * -> *).
(HasLedgerTables l, MonadIO m, IOLike m, HasCallStack,
 MemPackIdx l EmptyMK ~ l EmptyMK) =>
Db m l -> m (LedgerBackingStoreValueHandle m l)
mkLMDBBackingStoreValueHandle Db m l
db

        bsWrite :: SlotNo -> (l EmptyMK, l EmptyMK) -> LedgerTables l DiffMK -> m ()
        bsWrite :: SlotNo -> (l EmptyMK, l EmptyMK) -> LedgerTables l DiffMK -> m ()
bsWrite SlotNo
slot (l EmptyMK
_st, l EmptyMK
st') LedgerTables l DiffMK
diffs = do
          Tracer m BackingStoreTrace -> BackingStoreTrace -> m ()
forall (m :: * -> *) a. Tracer m a -> a -> m ()
Trace.traceWith Tracer m BackingStoreTrace
dbTracer (BackingStoreTrace -> m ()) -> BackingStoreTrace -> m ()
forall a b. (a -> b) -> a -> b
$ SlotNo -> BackingStoreTrace
API.BSWriting SlotNo
slot
          StatusLock m -> m () -> m () -> m ()
forall (m :: * -> *) a.
IOLike m =>
StatusLock m -> m a -> m a -> m a
Status.withReadAccess StatusLock m
dbStatusLock (LMDBErr -> m ()
forall e a. Exception e => e -> m a
forall (m :: * -> *) e a. (MonadThrow m, Exception e) => e -> m a
throwIO LMDBErr
LMDBErrClosed) (m () -> m ()) -> m () -> m ()
forall a b. (a -> b) -> a -> b
$ do
            oldSlot <- IO (WithOrigin SlotNo) -> m (WithOrigin SlotNo)
forall a. IO a -> m a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO (IO (WithOrigin SlotNo) -> m (WithOrigin SlotNo))
-> IO (WithOrigin SlotNo) -> m (WithOrigin SlotNo)
forall a b. (a -> b) -> a -> b
$ Environment 'ReadWrite
-> Transaction 'ReadWrite (WithOrigin SlotNo)
-> IO (WithOrigin SlotNo)
forall a.
Environment 'ReadWrite -> Transaction 'ReadWrite a -> IO a
LMDB.readWriteTransaction Environment 'ReadWrite
dbEnv (Transaction 'ReadWrite (WithOrigin SlotNo)
 -> IO (WithOrigin SlotNo))
-> Transaction 'ReadWrite (WithOrigin SlotNo)
-> IO (WithOrigin SlotNo)
forall a b. (a -> b) -> a -> b
$ Database () DbSeqNo
-> (DbSeqNo -> Transaction 'ReadWrite (WithOrigin SlotNo, DbSeqNo))
-> Transaction 'ReadWrite (WithOrigin SlotNo)
forall a.
Database () DbSeqNo
-> (DbSeqNo -> Transaction 'ReadWrite (a, DbSeqNo))
-> Transaction 'ReadWrite a
withDbSeqNoRW Database () DbSeqNo
dbState ((DbSeqNo -> Transaction 'ReadWrite (WithOrigin SlotNo, DbSeqNo))
 -> Transaction 'ReadWrite (WithOrigin SlotNo))
-> (DbSeqNo -> Transaction 'ReadWrite (WithOrigin SlotNo, DbSeqNo))
-> Transaction 'ReadWrite (WithOrigin SlotNo)
forall a b. (a -> b) -> a -> b
$ \s :: DbSeqNo
s@DbSeqNo{WithOrigin SlotNo
dbsSeq :: DbSeqNo -> WithOrigin SlotNo
dbsSeq :: WithOrigin SlotNo
dbsSeq} -> do
              Bool -> Transaction 'ReadWrite () -> Transaction 'ReadWrite ()
forall (f :: * -> *). Applicative f => Bool -> f () -> f ()
unless (WithOrigin SlotNo
dbsSeq WithOrigin SlotNo -> WithOrigin SlotNo -> Bool
forall a. Ord a => a -> a -> Bool
<= SlotNo -> WithOrigin SlotNo
forall t. t -> WithOrigin t
At SlotNo
slot) (Transaction 'ReadWrite () -> Transaction 'ReadWrite ())
-> Transaction 'ReadWrite () -> Transaction 'ReadWrite ()
forall a b. (a -> b) -> a -> b
$
                -- This inequality is non-strict because of EBBs having the
                -- same slot as its predecessor.
                IO () -> Transaction 'ReadWrite ()
forall a. IO a -> Transaction 'ReadWrite a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO (IO () -> Transaction 'ReadWrite ())
-> (LMDBErr -> IO ()) -> LMDBErr -> Transaction 'ReadWrite ()
forall b c a. (b -> c) -> (a -> b) -> a -> c
. LMDBErr -> IO ()
forall e a. Exception e => e -> IO a
forall (m :: * -> *) e a. (MonadThrow m, Exception e) => e -> m a
throwIO (LMDBErr -> Transaction 'ReadWrite ())
-> LMDBErr -> Transaction 'ReadWrite ()
forall a b. (a -> b) -> a -> b
$
                  WithOrigin SlotNo -> WithOrigin SlotNo -> LMDBErr
LMDBErrNonMonotonicSeq (SlotNo -> WithOrigin SlotNo
forall t. t -> WithOrigin t
At SlotNo
slot) WithOrigin SlotNo
dbsSeq
              Transaction 'ReadWrite (LedgerTables l EmptyMK)
-> Transaction 'ReadWrite ()
forall (f :: * -> *) a. Functor f => f a -> f ()
void (Transaction 'ReadWrite (LedgerTables l EmptyMK)
 -> Transaction 'ReadWrite ())
-> Transaction 'ReadWrite (LedgerTables l EmptyMK)
-> Transaction 'ReadWrite ()
forall a b. (a -> b) -> a -> b
$ (forall k v.
 LedgerTableConstraints' l k v =>
 LMDBMK k v -> DiffMK k v -> Transaction 'ReadWrite (EmptyMK k v))
-> LedgerTables l LMDBMK
-> LedgerTables l DiffMK
-> Transaction 'ReadWrite (LedgerTables l EmptyMK)
forall (f :: * -> *) (l :: LedgerStateKind) (mk1 :: MapKind)
       (mk2 :: MapKind) (mk3 :: MapKind).
(Applicative f, LedgerTableConstraints l) =>
(forall k v.
 LedgerTableConstraints' l k v =>
 mk1 k v -> mk2 k v -> f (mk3 k v))
-> LedgerTables l mk1
-> LedgerTables l mk2
-> f (LedgerTables l mk3)
ltzipWith2A (l EmptyMK
-> LMDBMK k v -> DiffMK k v -> Transaction 'ReadWrite (EmptyMK k v)
forall idx v k.
(IndexedMemPack idx v, MemPack k) =>
idx
-> LMDBMK k v -> DiffMK k v -> Transaction 'ReadWrite (EmptyMK k v)
writeLMDBTable l EmptyMK
st') LedgerTables l LMDBMK
dbBackingTables LedgerTables l DiffMK
diffs
              (WithOrigin SlotNo, DbSeqNo)
-> Transaction 'ReadWrite (WithOrigin SlotNo, DbSeqNo)
forall a. a -> Transaction 'ReadWrite a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (WithOrigin SlotNo
dbsSeq, DbSeqNo
s{dbsSeq = At slot})
            Trace.traceWith dbTracer $ API.BSWritten oldSlot slot
     in API.BackingStore
          { bsClose :: m ()
API.bsClose = m ()
bsClose
          , bsCopy :: SerializeTablesHint (LedgerTables l ValuesMK) -> FsPath -> m ()
API.bsCopy = (FsPath -> m ()) -> l EmptyMK -> FsPath -> m ()
forall a b. a -> b -> a
const FsPath -> m ()
bsCopy
          , bsValueHandle :: m (BackingStoreValueHandle
     m (LedgerTables l KeysMK) (LedgerTables l ValuesMK))
API.bsValueHandle = m (BackingStoreValueHandle
     m (LedgerTables l KeysMK) (LedgerTables l ValuesMK))
bsValueHandle
          , bsWrite :: SlotNo
-> WriteHint (LedgerTables l DiffMK)
-> LedgerTables l DiffMK
-> m ()
API.bsWrite = SlotNo -> (l EmptyMK, l EmptyMK) -> LedgerTables l DiffMK -> m ()
SlotNo
-> WriteHint (LedgerTables l DiffMK)
-> LedgerTables l DiffMK
-> m ()
bsWrite
          , bsSnapshotBackend :: SnapshotBackend
API.bsSnapshotBackend = SnapshotBackend
UTxOHDLMDBSnapshot
          }
   where
    Db
      { Environment 'ReadWrite
dbEnv :: forall (m :: * -> *) (l :: LedgerStateKind).
Db m l -> Environment 'ReadWrite
dbEnv :: Environment 'ReadWrite
dbEnv
      , Database () DbSeqNo
dbState :: forall (m :: * -> *) (l :: LedgerStateKind).
Db m l -> Database () DbSeqNo
dbState :: Database () DbSeqNo
dbState
      , LedgerTables l LMDBMK
dbBackingTables :: forall (m :: * -> *) (l :: LedgerStateKind).
Db m l -> LedgerTables l LMDBMK
dbBackingTables :: LedgerTables l LMDBMK
dbBackingTables
      , StatusLock m
dbStatusLock :: forall (m :: * -> *) (l :: LedgerStateKind). Db m l -> StatusLock m
dbStatusLock :: StatusLock m
dbStatusLock
      , TVar m (Map Int (Cleanup m))
dbOpenHandles :: forall (m :: * -> *) (l :: LedgerStateKind).
Db m l -> TVar m (Map Int (Cleanup m))
dbOpenHandles :: TVar m (Map Int (Cleanup m))
dbOpenHandles
      } = Db m l
db

-- | Create a backing store value handle that has a consistent view of the
-- current database state.
mkLMDBBackingStoreValueHandle ::
  forall l m.
  (HasLedgerTables l, MonadIO m, IOLike m, HasCallStack, MemPackIdx l EmptyMK ~ l EmptyMK) =>
  -- | The LMDB database for which the backing store value handle is
  -- created.
  Db m l ->
  m (API.LedgerBackingStoreValueHandle m l)
mkLMDBBackingStoreValueHandle :: forall (l :: LedgerStateKind) (m :: * -> *).
(HasLedgerTables l, MonadIO m, IOLike m, HasCallStack,
 MemPackIdx l EmptyMK ~ l EmptyMK) =>
Db m l -> m (LedgerBackingStoreValueHandle m l)
mkLMDBBackingStoreValueHandle Db m l
db = do
  vhId <- STM m Int -> m Int
forall a. HasCallStack => STM m a -> m a
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
STM m a -> m a
IOLike.atomically (STM m Int -> m Int) -> STM m Int -> m Int
forall a b. (a -> b) -> a -> b
$ do
    vhId <- TVar m Int -> STM m Int
forall a. TVar m a -> STM m a
forall (m :: * -> *) a. MonadSTM m => TVar m a -> STM m a
IOLike.readTVar TVar m Int
dbNextId
    IOLike.modifyTVar' dbNextId (+ 1)
    pure vhId

  let
    dbEnvRo = Environment 'ReadWrite -> Environment 'ReadOnly
LMDB.readOnlyEnvironment Environment 'ReadWrite
dbEnv
    tracer = Maybe Int -> BackingStoreValueHandleTrace -> BackingStoreTrace
API.BSValueHandleTrace (Int -> Maybe Int
forall a. a -> Maybe a
Just Int
vhId) (BackingStoreValueHandleTrace -> BackingStoreTrace)
-> Tracer m BackingStoreTrace
-> Tracer m BackingStoreValueHandleTrace
forall (f :: * -> *) a b. Contravariant f => (a -> b) -> f b -> f a
>$< Tracer m BackingStoreTrace
dbTracer

  Trace.traceWith dbTracer API.BSCreatingValueHandle

  trh <- liftIO $ TrH.newReadOnly dbEnvRo
  mbInitSlot <- liftIO $ TrH.submitReadOnly trh $ readDbSeqNoMaybeNull dbState
  initSlot <- liftIO $ maybe (throwIO LMDBErrUnableToReadSeqNo) (pure . dbsSeq) mbInitSlot
  vhStatusLock <- Status.new Open

  let
    -- \| Clean up a backing store value handle by committing its transaction
    -- handle.
    cleanup :: Cleanup m
    cleanup =
      m () -> Cleanup m
forall (m :: * -> *). m () -> Cleanup m
Cleanup (m () -> Cleanup m) -> m () -> Cleanup m
forall a b. (a -> b) -> a -> b
$
        IO () -> m ()
forall a. IO a -> m a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO (IO () -> m ()) -> IO () -> m ()
forall a b. (a -> b) -> a -> b
$
          TransactionHandle 'ReadOnly -> IO ()
forall (mode :: Mode). TransactionHandle mode -> IO ()
TrH.commit TransactionHandle 'ReadOnly
trh

    bsvhClose :: m ()
    bsvhClose =
      StatusLock m -> m () -> m () -> m ()
forall (m :: * -> *) a.
IOLike m =>
StatusLock m -> m a -> m a -> m a
Status.withReadAccess StatusLock m
dbStatusLock m ()
traceAlreadyClosed (m () -> m ()) -> m () -> m ()
forall a b. (a -> b) -> a -> b
$ do
        StatusLock m -> m () -> m ((), Status) -> m ()
forall (m :: * -> *) a.
IOLike m =>
StatusLock m -> m a -> m (a, Status) -> m a
Status.withWriteAccess StatusLock m
vhStatusLock m ()
traceTVHAlreadyClosed (m ((), Status) -> m ()) -> m ((), Status) -> m ()
forall a b. (a -> b) -> a -> b
$ do
          Tracer m BackingStoreValueHandleTrace
-> BackingStoreValueHandleTrace -> m ()
forall (m :: * -> *) a. Tracer m a -> a -> m ()
Trace.traceWith Tracer m BackingStoreValueHandleTrace
tracer BackingStoreValueHandleTrace
API.BSVHClosing
          Cleanup m -> m ()
forall (m :: * -> *). Cleanup m -> m ()
runCleanup Cleanup m
cleanup
          STM m () -> m ()
forall a. HasCallStack => STM m a -> m a
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
STM m a -> m a
IOLike.atomically (STM m () -> m ()) -> STM m () -> m ()
forall a b. (a -> b) -> a -> b
$ TVar m (Map Int (Cleanup m))
-> (Map Int (Cleanup m) -> Map Int (Cleanup m)) -> STM m ()
forall a. TVar m a -> (a -> a) -> STM m ()
forall (m :: * -> *) a.
MonadSTM m =>
TVar m a -> (a -> a) -> STM m ()
IOLike.modifyTVar' TVar m (Map Int (Cleanup m))
dbOpenHandles (Int -> Map Int (Cleanup m) -> Map Int (Cleanup m)
forall k a. Ord k => k -> Map k a -> Map k a
Map.delete Int
vhId)
          Tracer m BackingStoreValueHandleTrace
-> BackingStoreValueHandleTrace -> m ()
forall (m :: * -> *) a. Tracer m a -> a -> m ()
Trace.traceWith Tracer m BackingStoreValueHandleTrace
tracer BackingStoreValueHandleTrace
API.BSVHClosed
          ((), Status) -> m ((), Status)
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure ((), Status
Closed)
     where
      traceAlreadyClosed :: m ()
traceAlreadyClosed = Tracer m BackingStoreTrace -> BackingStoreTrace -> m ()
forall (m :: * -> *) a. Tracer m a -> a -> m ()
Trace.traceWith Tracer m BackingStoreTrace
dbTracer BackingStoreTrace
API.BSAlreadyClosed
      traceTVHAlreadyClosed :: m ()
traceTVHAlreadyClosed = Tracer m BackingStoreValueHandleTrace
-> BackingStoreValueHandleTrace -> m ()
forall (m :: * -> *) a. Tracer m a -> a -> m ()
Trace.traceWith Tracer m BackingStoreValueHandleTrace
tracer BackingStoreValueHandleTrace
API.BSVHAlreadyClosed

    bsvhRead :: l EmptyMK -> LedgerTables l KeysMK -> m (LedgerTables l ValuesMK)
    bsvhRead l EmptyMK
st LedgerTables l KeysMK
keys =
      StatusLock m
-> m (LedgerTables l ValuesMK)
-> m (LedgerTables l ValuesMK)
-> m (LedgerTables l ValuesMK)
forall (m :: * -> *) a.
IOLike m =>
StatusLock m -> m a -> m a -> m a
Status.withReadAccess StatusLock m
dbStatusLock (LMDBErr -> m (LedgerTables l ValuesMK)
forall e a. Exception e => e -> m a
forall (m :: * -> *) e a. (MonadThrow m, Exception e) => e -> m a
throwIO LMDBErr
LMDBErrClosed) (m (LedgerTables l ValuesMK) -> m (LedgerTables l ValuesMK))
-> m (LedgerTables l ValuesMK) -> m (LedgerTables l ValuesMK)
forall a b. (a -> b) -> a -> b
$ do
        StatusLock m
-> m (LedgerTables l ValuesMK)
-> m (LedgerTables l ValuesMK)
-> m (LedgerTables l ValuesMK)
forall (m :: * -> *) a.
IOLike m =>
StatusLock m -> m a -> m a -> m a
Status.withReadAccess StatusLock m
vhStatusLock (LMDBErr -> m (LedgerTables l ValuesMK)
forall e a. Exception e => e -> m a
forall (m :: * -> *) e a. (MonadThrow m, Exception e) => e -> m a
throwIO (Int -> LMDBErr
LMDBErrNoValueHandle Int
vhId)) (m (LedgerTables l ValuesMK) -> m (LedgerTables l ValuesMK))
-> m (LedgerTables l ValuesMK) -> m (LedgerTables l ValuesMK)
forall a b. (a -> b) -> a -> b
$ do
          Tracer m BackingStoreValueHandleTrace
-> BackingStoreValueHandleTrace -> m ()
forall (m :: * -> *) a. Tracer m a -> a -> m ()
Trace.traceWith Tracer m BackingStoreValueHandleTrace
tracer BackingStoreValueHandleTrace
API.BSVHReading
          res <-
            IO (LedgerTables l ValuesMK) -> m (LedgerTables l ValuesMK)
forall a. IO a -> m a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO (IO (LedgerTables l ValuesMK) -> m (LedgerTables l ValuesMK))
-> IO (LedgerTables l ValuesMK) -> m (LedgerTables l ValuesMK)
forall a b. (a -> b) -> a -> b
$
              TransactionHandle 'ReadOnly
-> Transaction 'ReadOnly (LedgerTables l ValuesMK)
-> IO (LedgerTables l ValuesMK)
forall (mode :: Mode) a.
TransactionHandle mode -> Transaction 'ReadOnly a -> IO a
TrH.submitReadOnly TransactionHandle 'ReadOnly
trh (Transaction 'ReadOnly (LedgerTables l ValuesMK)
 -> IO (LedgerTables l ValuesMK))
-> Transaction 'ReadOnly (LedgerTables l ValuesMK)
-> IO (LedgerTables l ValuesMK)
forall a b. (a -> b) -> a -> b
$
                (forall k v.
 LedgerTableConstraints' l k v =>
 LMDBMK k v -> KeysMK k v -> Transaction 'ReadOnly (ValuesMK k v))
-> LedgerTables l LMDBMK
-> LedgerTables l KeysMK
-> Transaction 'ReadOnly (LedgerTables l ValuesMK)
forall (f :: * -> *) (l :: LedgerStateKind) (mk1 :: MapKind)
       (mk2 :: MapKind) (mk3 :: MapKind).
(Applicative f, LedgerTableConstraints l) =>
(forall k v.
 LedgerTableConstraints' l k v =>
 mk1 k v -> mk2 k v -> f (mk3 k v))
-> LedgerTables l mk1
-> LedgerTables l mk2
-> f (LedgerTables l mk3)
ltzipWith2A (l EmptyMK
-> LMDBMK k v -> KeysMK k v -> Transaction 'ReadOnly (ValuesMK k v)
forall idx v k (mode :: Mode).
(IndexedMemPack idx v, MemPack k, Ord k) =>
idx -> LMDBMK k v -> KeysMK k v -> Transaction mode (ValuesMK k v)
readLMDBTable l EmptyMK
st) LedgerTables l LMDBMK
dbBackingTables LedgerTables l KeysMK
keys
          Trace.traceWith tracer API.BSVHRead
          pure res

    bsvhRangeRead ::
      l EmptyMK ->
      API.RangeQuery (LedgerTables l KeysMK) ->
      m (LedgerTables l ValuesMK)
    bsvhRangeRead l EmptyMK
st RangeQuery (LedgerTables l KeysMK)
rq =
      StatusLock m
-> m (LedgerTables l ValuesMK)
-> m (LedgerTables l ValuesMK)
-> m (LedgerTables l ValuesMK)
forall (m :: * -> *) a.
IOLike m =>
StatusLock m -> m a -> m a -> m a
Status.withReadAccess StatusLock m
dbStatusLock (LMDBErr -> m (LedgerTables l ValuesMK)
forall e a. Exception e => e -> m a
forall (m :: * -> *) e a. (MonadThrow m, Exception e) => e -> m a
throwIO LMDBErr
LMDBErrClosed) (m (LedgerTables l ValuesMK) -> m (LedgerTables l ValuesMK))
-> m (LedgerTables l ValuesMK) -> m (LedgerTables l ValuesMK)
forall a b. (a -> b) -> a -> b
$ do
        StatusLock m
-> m (LedgerTables l ValuesMK)
-> m (LedgerTables l ValuesMK)
-> m (LedgerTables l ValuesMK)
forall (m :: * -> *) a.
IOLike m =>
StatusLock m -> m a -> m a -> m a
Status.withReadAccess StatusLock m
vhStatusLock (LMDBErr -> m (LedgerTables l ValuesMK)
forall e a. Exception e => e -> m a
forall (m :: * -> *) e a. (MonadThrow m, Exception e) => e -> m a
throwIO (Int -> LMDBErr
LMDBErrNoValueHandle Int
vhId)) (m (LedgerTables l ValuesMK) -> m (LedgerTables l ValuesMK))
-> m (LedgerTables l ValuesMK) -> m (LedgerTables l ValuesMK)
forall a b. (a -> b) -> a -> b
$ do
          Tracer m BackingStoreValueHandleTrace
-> BackingStoreValueHandleTrace -> m ()
forall (m :: * -> *) a. Tracer m a -> a -> m ()
Trace.traceWith Tracer m BackingStoreValueHandleTrace
tracer BackingStoreValueHandleTrace
API.BSVHRangeReading
          res <-
            IO (LedgerTables l ValuesMK) -> m (LedgerTables l ValuesMK)
forall a. IO a -> m a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO (IO (LedgerTables l ValuesMK) -> m (LedgerTables l ValuesMK))
-> IO (LedgerTables l ValuesMK) -> m (LedgerTables l ValuesMK)
forall a b. (a -> b) -> a -> b
$
              TransactionHandle 'ReadOnly
-> Transaction 'ReadOnly (LedgerTables l ValuesMK)
-> IO (LedgerTables l ValuesMK)
forall (mode :: Mode) a.
TransactionHandle mode -> Transaction 'ReadOnly a -> IO a
TrH.submitReadOnly TransactionHandle 'ReadOnly
trh (Transaction 'ReadOnly (LedgerTables l ValuesMK)
 -> IO (LedgerTables l ValuesMK))
-> Transaction 'ReadOnly (LedgerTables l ValuesMK)
-> IO (LedgerTables l ValuesMK)
forall a b. (a -> b) -> a -> b
$
                let dbMK :: LMDBMK (TxIn l) (TxOut l)
dbMK = LedgerTables l LMDBMK -> LMDBMK (TxIn l) (TxOut l)
forall (l :: LedgerStateKind) (mk :: MapKind).
LedgerTables l mk -> mk (TxIn l) (TxOut l)
getLedgerTables LedgerTables l LMDBMK
dbBackingTables
                 in ValuesMK (TxIn l) (TxOut l) -> LedgerTables l ValuesMK
forall (l :: LedgerStateKind) (mk :: MapKind).
mk (TxIn l) (TxOut l) -> LedgerTables l mk
LedgerTables (ValuesMK (TxIn l) (TxOut l) -> LedgerTables l ValuesMK)
-> Transaction 'ReadOnly (ValuesMK (TxIn l) (TxOut l))
-> Transaction 'ReadOnly (LedgerTables l ValuesMK)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> RangeQuery (LedgerTables l KeysMK)
-> l EmptyMK
-> LMDBMK (TxIn l) (TxOut l)
-> Transaction 'ReadOnly (ValuesMK (TxIn l) (TxOut l))
forall (mode :: Mode) (l :: LedgerStateKind) idx.
(Ord (TxIn l), MemPack (TxIn l), IndexedMemPack idx (TxOut l)) =>
RangeQuery (LedgerTables l KeysMK)
-> idx
-> LMDBMK (TxIn l) (TxOut l)
-> Transaction mode (ValuesMK (TxIn l) (TxOut l))
rangeRead RangeQuery (LedgerTables l KeysMK)
rq l EmptyMK
st LMDBMK (TxIn l) (TxOut l)
dbMK
          Trace.traceWith tracer API.BSVHRangeRead
          pure res

    bsvhStat :: m API.Statistics
    bsvhStat =
      StatusLock m -> m Statistics -> m Statistics -> m Statistics
forall (m :: * -> *) a.
IOLike m =>
StatusLock m -> m a -> m a -> m a
Status.withReadAccess StatusLock m
dbStatusLock (LMDBErr -> m Statistics
forall e a. Exception e => e -> m a
forall (m :: * -> *) e a. (MonadThrow m, Exception e) => e -> m a
throwIO LMDBErr
LMDBErrClosed) (m Statistics -> m Statistics) -> m Statistics -> m Statistics
forall a b. (a -> b) -> a -> b
$ do
        StatusLock m -> m Statistics -> m Statistics -> m Statistics
forall (m :: * -> *) a.
IOLike m =>
StatusLock m -> m a -> m a -> m a
Status.withReadAccess StatusLock m
vhStatusLock (LMDBErr -> m Statistics
forall e a. Exception e => e -> m a
forall (m :: * -> *) e a. (MonadThrow m, Exception e) => e -> m a
throwIO (Int -> LMDBErr
LMDBErrNoValueHandle Int
vhId)) (m Statistics -> m Statistics) -> m Statistics -> m Statistics
forall a b. (a -> b) -> a -> b
$ do
          Tracer m BackingStoreValueHandleTrace
-> BackingStoreValueHandleTrace -> m ()
forall (m :: * -> *) a. Tracer m a -> a -> m ()
Trace.traceWith Tracer m BackingStoreValueHandleTrace
tracer BackingStoreValueHandleTrace
API.BSVHStatting
          let transaction :: Transaction 'ReadOnly Statistics
transaction = do
                DbSeqNo{dbsSeq} <- Database () DbSeqNo -> Transaction 'ReadOnly DbSeqNo
forall (mode :: Mode).
Database () DbSeqNo -> Transaction mode DbSeqNo
readDbSeqNo Database () DbSeqNo
dbState
                constn <- lttraverse (\(LMDBMK String
_ Database k v
dbx) -> Int -> K2 Int k v
forall k1 k2 a (b :: k1) (c :: k2). a -> K2 a b c
K2 (Int -> K2 Int k v)
-> Transaction 'ReadOnly Int -> Transaction 'ReadOnly (K2 Int k v)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Database k v -> Transaction 'ReadOnly Int
forall k v (mode :: Mode). Database k v -> Transaction mode Int
LMDB.size Database k v
dbx) dbBackingTables
                let n = LedgerTables l (K2 Int) -> Int
forall (l :: LedgerStateKind) a. LedgerTables l (K2 a) -> a
ltcollapse LedgerTables l (K2 Int)
constn
                pure $ API.Statistics dbsSeq n
          res <- IO Statistics -> m Statistics
forall a. IO a -> m a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO (IO Statistics -> m Statistics) -> IO Statistics -> m Statistics
forall a b. (a -> b) -> a -> b
$ TransactionHandle 'ReadOnly
-> Transaction 'ReadOnly Statistics -> IO Statistics
forall (mode :: Mode) a.
TransactionHandle mode -> Transaction 'ReadOnly a -> IO a
TrH.submitReadOnly TransactionHandle 'ReadOnly
trh Transaction 'ReadOnly Statistics
transaction
          Trace.traceWith tracer API.BSVHStatted
          pure res

    bsvhReadAll :: l EmptyMK -> m (LedgerTables l ValuesMK)
    bsvhReadAll l EmptyMK
st =
      StatusLock m
-> m (LedgerTables l ValuesMK)
-> m (LedgerTables l ValuesMK)
-> m (LedgerTables l ValuesMK)
forall (m :: * -> *) a.
IOLike m =>
StatusLock m -> m a -> m a -> m a
Status.withReadAccess StatusLock m
dbStatusLock (LMDBErr -> m (LedgerTables l ValuesMK)
forall e a. Exception e => e -> m a
forall (m :: * -> *) e a. (MonadThrow m, Exception e) => e -> m a
throwIO LMDBErr
LMDBErrClosed) (m (LedgerTables l ValuesMK) -> m (LedgerTables l ValuesMK))
-> m (LedgerTables l ValuesMK) -> m (LedgerTables l ValuesMK)
forall a b. (a -> b) -> a -> b
$ do
        StatusLock m
-> m (LedgerTables l ValuesMK)
-> m (LedgerTables l ValuesMK)
-> m (LedgerTables l ValuesMK)
forall (m :: * -> *) a.
IOLike m =>
StatusLock m -> m a -> m a -> m a
Status.withReadAccess StatusLock m
vhStatusLock (LMDBErr -> m (LedgerTables l ValuesMK)
forall e a. Exception e => e -> m a
forall (m :: * -> *) e a. (MonadThrow m, Exception e) => e -> m a
throwIO (Int -> LMDBErr
LMDBErrNoValueHandle Int
vhId)) (m (LedgerTables l ValuesMK) -> m (LedgerTables l ValuesMK))
-> m (LedgerTables l ValuesMK) -> m (LedgerTables l ValuesMK)
forall a b. (a -> b) -> a -> b
$ do
          Tracer m BackingStoreValueHandleTrace
-> BackingStoreValueHandleTrace -> m ()
forall (m :: * -> *) a. Tracer m a -> a -> m ()
Trace.traceWith Tracer m BackingStoreValueHandleTrace
tracer BackingStoreValueHandleTrace
API.BSVHRangeReading
          res <-
            IO (LedgerTables l ValuesMK) -> m (LedgerTables l ValuesMK)
forall a. IO a -> m a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO (IO (LedgerTables l ValuesMK) -> m (LedgerTables l ValuesMK))
-> IO (LedgerTables l ValuesMK) -> m (LedgerTables l ValuesMK)
forall a b. (a -> b) -> a -> b
$
              TransactionHandle 'ReadOnly
-> Transaction 'ReadOnly (LedgerTables l ValuesMK)
-> IO (LedgerTables l ValuesMK)
forall (mode :: Mode) a.
TransactionHandle mode -> Transaction 'ReadOnly a -> IO a
TrH.submitReadOnly TransactionHandle 'ReadOnly
trh (Transaction 'ReadOnly (LedgerTables l ValuesMK)
 -> IO (LedgerTables l ValuesMK))
-> Transaction 'ReadOnly (LedgerTables l ValuesMK)
-> IO (LedgerTables l ValuesMK)
forall a b. (a -> b) -> a -> b
$
                let dbMK :: LMDBMK (TxIn l) (TxOut l)
dbMK = LedgerTables l LMDBMK -> LMDBMK (TxIn l) (TxOut l)
forall (l :: LedgerStateKind) (mk :: MapKind).
LedgerTables l mk -> mk (TxIn l) (TxOut l)
getLedgerTables LedgerTables l LMDBMK
dbBackingTables
                 in ValuesMK (TxIn l) (TxOut l) -> LedgerTables l ValuesMK
forall (l :: LedgerStateKind) (mk :: MapKind).
mk (TxIn l) (TxOut l) -> LedgerTables l mk
LedgerTables (ValuesMK (TxIn l) (TxOut l) -> LedgerTables l ValuesMK)
-> Transaction 'ReadOnly (ValuesMK (TxIn l) (TxOut l))
-> Transaction 'ReadOnly (LedgerTables l ValuesMK)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Proxy l
-> l EmptyMK
-> LMDBMK (TxIn l) (TxOut l)
-> Transaction 'ReadOnly (ValuesMK (TxIn l) (TxOut l))
forall (l :: LedgerStateKind) idx (mode :: Mode).
(Ord (TxIn l), MemPack (TxIn l), IndexedMemPack idx (TxOut l)) =>
Proxy l
-> idx
-> LMDBMK (TxIn l) (TxOut l)
-> Transaction mode (ValuesMK (TxIn l) (TxOut l))
readAll (forall {k} (t :: k). Proxy t
forall (t :: LedgerStateKind). Proxy t
Proxy @l) l EmptyMK
st LMDBMK (TxIn l) (TxOut l)
dbMK
          Trace.traceWith tracer API.BSVHRangeRead
          pure res

    bsvh =
      API.BackingStoreValueHandle
        { bsvhAtSlot :: WithOrigin SlotNo
API.bsvhAtSlot = WithOrigin SlotNo
initSlot
        , bsvhClose :: m ()
API.bsvhClose = m ()
bsvhClose
        , bsvhRead :: ReadHint (LedgerTables l ValuesMK)
-> LedgerTables l KeysMK -> m (LedgerTables l ValuesMK)
API.bsvhRead = l EmptyMK -> LedgerTables l KeysMK -> m (LedgerTables l ValuesMK)
ReadHint (LedgerTables l ValuesMK)
-> LedgerTables l KeysMK -> m (LedgerTables l ValuesMK)
bsvhRead
        , bsvhReadAll :: ReadHint (LedgerTables l ValuesMK) -> m (LedgerTables l ValuesMK)
API.bsvhReadAll = l EmptyMK -> m (LedgerTables l ValuesMK)
ReadHint (LedgerTables l ValuesMK) -> m (LedgerTables l ValuesMK)
bsvhReadAll
        , bsvhRangeRead :: ReadHint (LedgerTables l ValuesMK)
-> RangeQuery (LedgerTables l KeysMK)
-> m (LedgerTables l ValuesMK)
API.bsvhRangeRead = l EmptyMK
-> RangeQuery (LedgerTables l KeysMK)
-> m (LedgerTables l ValuesMK)
ReadHint (LedgerTables l ValuesMK)
-> RangeQuery (LedgerTables l KeysMK)
-> m (LedgerTables l ValuesMK)
bsvhRangeRead
        , bsvhStat :: m Statistics
API.bsvhStat = m Statistics
bsvhStat
        }

  IOLike.atomically $ IOLike.modifyTVar' dbOpenHandles (Map.insert vhId cleanup)

  Trace.traceWith dbTracer API.BSCreatedValueHandle
  pure bsvh
 where
  Db
    { Environment 'ReadWrite
dbEnv :: forall (m :: * -> *) (l :: LedgerStateKind).
Db m l -> Environment 'ReadWrite
dbEnv :: Environment 'ReadWrite
dbEnv
    , Tracer m BackingStoreTrace
dbTracer :: forall (m :: * -> *) (l :: LedgerStateKind).
Db m l -> Tracer m BackingStoreTrace
dbTracer :: Tracer m BackingStoreTrace
dbTracer
    , Database () DbSeqNo
dbState :: forall (m :: * -> *) (l :: LedgerStateKind).
Db m l -> Database () DbSeqNo
dbState :: Database () DbSeqNo
dbState
    , TVar m (Map Int (Cleanup m))
dbOpenHandles :: forall (m :: * -> *) (l :: LedgerStateKind).
Db m l -> TVar m (Map Int (Cleanup m))
dbOpenHandles :: TVar m (Map Int (Cleanup m))
dbOpenHandles
    , LedgerTables l LMDBMK
dbBackingTables :: forall (m :: * -> *) (l :: LedgerStateKind).
Db m l -> LedgerTables l LMDBMK
dbBackingTables :: LedgerTables l LMDBMK
dbBackingTables
    , TVar m Int
dbNextId :: forall (m :: * -> *) (l :: LedgerStateKind). Db m l -> TVar m Int
dbNextId :: TVar m Int
dbNextId
    , StatusLock m
dbStatusLock :: forall (m :: * -> *) (l :: LedgerStateKind). Db m l -> StatusLock m
dbStatusLock :: StatusLock m
dbStatusLock
    } = Db m l
db

-- | A monadic action used for cleaning up resources.
newtype Cleanup m = Cleanup {forall (m :: * -> *). Cleanup m -> m ()
runCleanup :: m ()}

{-------------------------------------------------------------------------------
 Errors
-------------------------------------------------------------------------------}

-- | Errors that can be thrown by LMDB.
--
-- __WARNING__: these errors will be thrown in IO as having a corrupt database
-- is critical for the functioning of Consensus.
data LMDBErr
  = -- | The database state can not be found on-disk.
    LMDBErrNoDbSeqNo
  | -- | The sequence number of a @`Db`@ should be monotonically increasing
    -- across calls to @`bsWrite`@, since we use @`bsWrite`@ to flush
    -- /immutable/ changes. That is, we can only flush with a newer sequence
    -- number because the changes should be /immutable/. Note that this does
    -- not mean that values can not be changed in the future, only that we
    -- can not change values in the past.
    LMDBErrNonMonotonicSeq !(WithOrigin SlotNo) !(WithOrigin SlotNo)
  | -- | The database table that is being initialised is non-empty.
    LMDBErrInitialisingNonEmpty !String
  | -- | The database that is being initialized already had a DbSeqNo table
    LMDBErrInitialisingAlreadyHasState
  | -- | Trying to use a non-existing value handle.
    LMDBErrNoValueHandle !Int
  | -- | Couldn't create a value handle because we couldn't read the sequence
    -- number
    LMDBErrUnableToReadSeqNo
  | -- | Failed to read a value from a database table.
    LMDBErrBadRead
  | -- | Failed to read a range of values from a database table.
    LMDBErrBadRangeRead
  | -- | A database directory should not exist already.
    LMDBErrDirExists !FilePath
  | -- | A database directory should exist already.
    LMDBErrDirDoesntExist !FilePath
  | -- | The directory exists but is not an LMDB directory!
    LMDBErrDirIsNotLMDB !FilePath
  | -- | What should be a directory is in fact a file
    LMDBErrNotADir !FS.FsPath
  | -- | The database has been closed, so all backing store operations should
    -- throw an error.
    LMDBErrClosed

instance Exception LMDBErr

-- | Show instance for pretty printing @`LMDBErr`@s as error messages that
-- include: (i) an indication of the probable cause of the error, and
-- (ii) a descriptive error message for the specific @`LMDBErr`@.
instance Show LMDBErr where
  show :: LMDBErr -> String
show LMDBErr
dbErr =
    [String] -> String
forall a. Monoid a => [a] -> a
mconcat
      [ String
"[LMDB-ERROR] "
      , String
"The LMDB Backing store has encountered a fatal exception. "
      , String
"Possibly, the LMDB database is corrupted.\n"
      , String
"[ERROR-MSG] "
      , LMDBErr -> String
prettyPrintLMDBErr LMDBErr
dbErr
      ]

-- | Pretty print a @`LMDBErr`@ with a descriptive error message.
prettyPrintLMDBErr :: LMDBErr -> String
prettyPrintLMDBErr :: LMDBErr -> String
prettyPrintLMDBErr = \case
  LMDBErr
LMDBErrNoDbSeqNo ->
    String
"Can not find the database state on-disk."
  LMDBErrNonMonotonicSeq WithOrigin SlotNo
s1 WithOrigin SlotNo
s2 ->
    String
"Trying to write to the database with a non-monotonic sequence number: "
      String -> ShowS
forall a. Semigroup a => a -> a -> a
<> Bool -> ShowS -> ShowS
showParen Bool
True (WithOrigin SlotNo -> ShowS
forall a. Show a => a -> ShowS
shows WithOrigin SlotNo
s1) String
""
      String -> ShowS
forall a. Semigroup a => a -> a -> a
<> String
" is not <= "
      String -> ShowS
forall a. Semigroup a => a -> a -> a
<> Bool -> ShowS -> ShowS
showParen Bool
True (WithOrigin SlotNo -> ShowS
forall a. Show a => a -> ShowS
shows WithOrigin SlotNo
s2) String
""
  LMDBErrInitialisingNonEmpty String
s ->
    String
"The database table that is being initialised is non-empty: " String -> ShowS
forall a. Semigroup a => a -> a -> a
<> String
s
  LMDBErr
LMDBErrInitialisingAlreadyHasState ->
    String
"The database contains no values but still has a table with a sequence number."
  LMDBErrNoValueHandle Int
vh_id ->
    String
"Trying to use non-existing value handle: " String -> ShowS
forall a. Semigroup a => a -> a -> a
<> Int -> String
forall a. Show a => a -> String
show Int
vh_id
  LMDBErr
LMDBErrUnableToReadSeqNo ->
    String
"Reading the sequence number failed thus we couldn't create a value handle."
  LMDBErr
LMDBErrBadRead ->
    String
"Failed to read a value from a database table."
  LMDBErr
LMDBErrBadRangeRead ->
    String
"Failed to read a range of values from a database table."
  LMDBErrDirExists String
path ->
    String
"Database directory should not exist already: " String -> ShowS
forall a. Semigroup a => a -> a -> a
<> ShowS
forall a. Show a => a -> String
show String
path
  LMDBErrDirDoesntExist String
path ->
    String
"Database directory should already exist: " String -> ShowS
forall a. Semigroup a => a -> a -> a
<> ShowS
forall a. Show a => a -> String
show String
path
  LMDBErrDirIsNotLMDB String
path ->
    String
"Database directory doesn't contain an LMDB database: "
      String -> ShowS
forall a. Semigroup a => a -> a -> a
<> ShowS
forall a. Show a => a -> String
show String
path
      String -> ShowS
forall a. Semigroup a => a -> a -> a
<> String
"\nPre-UTxO-HD and In-Memory implementations are incompatible \
         \ with the LMDB implementation, please delete your ledger database \
         \ if you want to run with LMDB"
  LMDBErrNotADir FsPath
path ->
    String
"The path " String -> ShowS
forall a. Semigroup a => a -> a -> a
<> FsPath -> String
forall a. Show a => a -> String
show FsPath
path String -> ShowS
forall a. Semigroup a => a -> a -> a
<> String
" should be a directory but it is a file instead."
  LMDBErr
LMDBErrClosed -> String
"The database has been closed."