{-# LANGUAGE DataKinds #-}
{-# LANGUAGE DeriveAnyClass #-}
{-# LANGUAGE DeriveGeneric #-}
{-# LANGUAGE DerivingVia #-}
{-# LANGUAGE FlexibleContexts #-}
{-# LANGUAGE FlexibleInstances #-}
{-# LANGUAGE LambdaCase #-}
{-# LANGUAGE MultiParamTypeClasses #-}
{-# LANGUAGE NamedFieldPuns #-}
{-# LANGUAGE PatternSynonyms #-}
{-# LANGUAGE ScopedTypeVariables #-}
{-# LANGUAGE TupleSections #-}
{-# LANGUAGE TypeApplications #-}
{-# LANGUAGE TypeData #-}
{-# LANGUAGE TypeFamilies #-}
{-# LANGUAGE TypeOperators #-}
{-# LANGUAGE UndecidableInstances #-}
module Ouroboros.Consensus.Storage.LedgerDB.V1.BackingStore.Impl.LMDB
(
LMDB
, Backend (..)
, Args (LMDBBackingStoreArgs)
, LMDBLimits (LMDBLimits, lmdbMapSize, lmdbMaxDatabases, lmdbMaxReaders)
, mkLMDBArgs
, YieldArgs (YieldLMDB)
, mkLMDBYieldArgs
, SinkArgs (SinkLMDB)
, mkLMDBSinkArgs
, LMDBErr (..)
) where
import Cardano.Slotting.Slot (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 Control.Monad.Trans (lift)
import Control.ResourceRegistry
import qualified Control.Tracer as Trace
import Data.Bifunctor (first)
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.SOP.Dict as Dict
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.Block
import Ouroboros.Consensus.Ledger.Basics
import qualified Ouroboros.Consensus.Ledger.Tables.Diff as Diff
import Ouroboros.Consensus.Ledger.Tables.Utils (emptyLedgerTables)
import Ouroboros.Consensus.Storage.LedgerDB.API
import Ouroboros.Consensus.Storage.LedgerDB.Args
import Ouroboros.Consensus.Storage.LedgerDB.Snapshots
( SnapshotBackend (..)
)
import qualified Ouroboros.Consensus.Storage.LedgerDB.V1.Args as V1
import Ouroboros.Consensus.Storage.LedgerDB.V1.BackingStore
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 (..)
, PrimState
, bracket
)
import Ouroboros.Consensus.Util.IndexedMemPack
import qualified Streaming as S
import qualified Streaming.Prelude as S
import System.Directory
import qualified System.FS.API as FS
import System.FS.IO
import qualified System.FilePath as FilePath
import System.IO.Temp
data Db m l = Db
{ forall (m :: * -> *) (l :: LedgerStateKind).
Db m l -> Environment 'ReadWrite
dbEnv :: !(LMDB.Environment LMDB.ReadWrite)
, forall (m :: * -> *) (l :: LedgerStateKind).
Db m l -> Database () DbSeqNo
dbState :: !(LMDB.Database () DbSeqNo)
, forall (m :: * -> *) (l :: LedgerStateKind).
Db m l -> LedgerTables l LMDBMK
dbBackingTables :: !(LedgerTables l LMDBMK)
, 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)
, forall (m :: * -> *) (l :: LedgerStateKind).
Db m l -> TVar m (Map Int (Cleanup m))
dbOpenHandles :: !(IOLike.TVar m (Map Int (Cleanup m)))
, 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 #-}
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
}
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
data LMDBMK k v = LMDBMK !String !(LMDB.Database k v)
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 ::
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), Maybe (TxIn 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), Maybe (TxIn 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))
-> (Map (TxIn l) (TxOut l), Maybe (TxIn l))
-> (ValuesMK (TxIn l) (TxOut l), Maybe (TxIn l))
forall a b c. (a -> b) -> (a, c) -> (b, c)
forall (p :: * -> * -> *) a b c.
Bifunctor p =>
(a -> b) -> p a c -> p b c
first 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), Maybe (TxIn l))
-> (ValuesMK (TxIn l) (TxOut l), Maybe (TxIn l)))
-> Transaction mode (Map (TxIn l) (TxOut l), Maybe (TxIn l))
-> Transaction mode (ValuesMK (TxIn l) (TxOut l), Maybe (TxIn 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), Maybe (TxIn 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), Maybe (TxIn l))
-> Transaction mode (Map (TxIn l) (TxOut l), Maybe (TxIn 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, Maybe (TxIn l)
forall a. Maybe a
Nothing)
Just TxIn l
lastExcludedKey ->
Maybe (TxIn l, Bound)
-> Transaction mode (Map (TxIn l) (TxOut l), Maybe (TxIn l))
runCursorHelper (Maybe (TxIn l, Bound)
-> Transaction mode (Map (TxIn l) (TxOut l), Maybe (TxIn l)))
-> Maybe (TxIn l, Bound)
-> Transaction mode (Map (TxIn l) (TxOut l), Maybe (TxIn 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) ->
LMDB.Transaction mode (Map (TxIn l) (TxOut l), Maybe (TxIn l))
runCursorHelper :: Maybe (TxIn l, Bound)
-> Transaction mode (Map (TxIn l) (TxOut l), Maybe (TxIn l))
runCursorHelper Maybe (TxIn l, Bound)
lb =
idx
-> CursorM
(TxIn l) (TxOut l) mode (Map (TxIn l) (TxOut l), Maybe (TxIn l))
-> Database (TxIn l) (TxOut l)
-> Transaction mode (Map (TxIn l) (TxOut l), Maybe (TxIn 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), Maybe (TxIn l))
forall (m :: Mode -> * -> *) k v (mode :: Mode).
(CursorConstraints m k v mode, Ord k) =>
Maybe (k, Bound) -> Int -> m mode (Map k v, Maybe k)
LMDB.Cursor.cgetManyAndLast 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
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
data GuardDbDir = DirMustExist | DirMustNotExist
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
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
initFromVals ::
forall l m.
(HasLedgerTables l, MonadIO m, MemPackIdx l EmptyMK ~ l EmptyMK) =>
Trace.Tracer m API.BackingStoreTrace ->
WithOrigin SlotNo ->
LedgerTables l ValuesMK ->
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 :: * -> * -> *)
(mk2 :: * -> * -> *) (mk3 :: * -> * -> *).
(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
initFromLMDBs ::
(MonadIO m, IOLike m) =>
Trace.Tracer m API.BackingStoreTrace ->
LMDBLimits ->
API.SnapshotsFS m ->
FS.FsPath ->
API.LiveLMDBFS m ->
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
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
lmdbCopy ::
MonadIO m =>
FS.FsPath ->
Trace.Tracer m API.BackingStoreTrace ->
LMDB.Environment LMDB.ReadWrite ->
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
newLMDBBackingStore ::
forall m l.
( HasCallStack
, HasLedgerTables l
, MonadIO m
, IOLike m
, MemPackIdx l EmptyMK ~ l EmptyMK
) =>
Trace.Tracer m API.BackingStoreTrace ->
LMDBLimits ->
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
dbFilePath <- checkAndOpenDbDirWithRetry DirMustNotExist liveFS' path
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 ()
dbEnv <- liftIO $ LMDB.openEnvironment dbFilePath (unLMDBLimits limits)
dbState <- liftIO $ LMDB.readWriteTransaction dbEnv $ LMDB.getDatabase (Just "_dbstate")
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
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
-> BackingStore
m
(LedgerTables l KeysMK)
(TxIn l)
(LedgerTables l ValuesMK)
(LedgerTables l DiffMK)
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) (TxIn l) (LedgerTables l ValuesMK))
bsValueHandle = StatusLock m
-> m (BackingStoreValueHandle
m (LedgerTables l KeysMK) (TxIn l) (LedgerTables l ValuesMK))
-> m (BackingStoreValueHandle
m (LedgerTables l KeysMK) (TxIn l) (LedgerTables l ValuesMK))
-> m (BackingStoreValueHandle
m (LedgerTables l KeysMK) (TxIn l) (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) (TxIn l) (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) (TxIn l) (LedgerTables l ValuesMK))
-> m (BackingStoreValueHandle
m (LedgerTables l KeysMK) (TxIn l) (LedgerTables l ValuesMK)))
-> m (BackingStoreValueHandle
m (LedgerTables l KeysMK) (TxIn l) (LedgerTables l ValuesMK))
-> m (BackingStoreValueHandle
m (LedgerTables l KeysMK) (TxIn l) (LedgerTables l ValuesMK))
forall a b. (a -> b) -> a -> b
$ do
Db m l
-> m (BackingStoreValueHandle
m (LedgerTables l KeysMK) (TxIn l) (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
$
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 :: * -> * -> *)
(mk2 :: * -> * -> *) (mk3 :: * -> * -> *).
(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) (TxIn l) (LedgerTables l ValuesMK))
API.bsValueHandle = m (BackingStoreValueHandle
m (LedgerTables l KeysMK) (TxIn l) (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
mkLMDBBackingStoreValueHandle ::
forall l m.
(HasLedgerTables l, MonadIO m, IOLike m, HasCallStack, MemPackIdx l EmptyMK ~ l EmptyMK) =>
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
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 :: * -> * -> *)
(mk2 :: * -> * -> *) (mk3 :: * -> * -> *).
(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, Maybe (TxIn l))
bsvhRangeRead l EmptyMK
st RangeQuery (LedgerTables l KeysMK)
rq =
StatusLock m
-> m (LedgerTables l ValuesMK, Maybe (TxIn l))
-> m (LedgerTables l ValuesMK, Maybe (TxIn l))
-> m (LedgerTables l ValuesMK, Maybe (TxIn l))
forall (m :: * -> *) a.
IOLike m =>
StatusLock m -> m a -> m a -> m a
Status.withReadAccess StatusLock m
dbStatusLock (LMDBErr -> m (LedgerTables l ValuesMK, Maybe (TxIn l))
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, Maybe (TxIn l))
-> m (LedgerTables l ValuesMK, Maybe (TxIn l)))
-> m (LedgerTables l ValuesMK, Maybe (TxIn l))
-> m (LedgerTables l ValuesMK, Maybe (TxIn l))
forall a b. (a -> b) -> a -> b
$ do
StatusLock m
-> m (LedgerTables l ValuesMK, Maybe (TxIn l))
-> m (LedgerTables l ValuesMK, Maybe (TxIn l))
-> m (LedgerTables l ValuesMK, Maybe (TxIn l))
forall (m :: * -> *) a.
IOLike m =>
StatusLock m -> m a -> m a -> m a
Status.withReadAccess StatusLock m
vhStatusLock (LMDBErr -> m (LedgerTables l ValuesMK, Maybe (TxIn l))
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, Maybe (TxIn l))
-> m (LedgerTables l ValuesMK, Maybe (TxIn l)))
-> m (LedgerTables l ValuesMK, Maybe (TxIn l))
-> m (LedgerTables l ValuesMK, Maybe (TxIn l))
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, Maybe (TxIn l))
-> m (LedgerTables l ValuesMK, Maybe (TxIn l))
forall a. IO a -> m a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO (IO (LedgerTables l ValuesMK, Maybe (TxIn l))
-> m (LedgerTables l ValuesMK, Maybe (TxIn l)))
-> IO (LedgerTables l ValuesMK, Maybe (TxIn l))
-> m (LedgerTables l ValuesMK, Maybe (TxIn l))
forall a b. (a -> b) -> a -> b
$
TransactionHandle 'ReadOnly
-> Transaction 'ReadOnly (LedgerTables l ValuesMK, Maybe (TxIn l))
-> IO (LedgerTables l ValuesMK, Maybe (TxIn l))
forall (mode :: Mode) a.
TransactionHandle mode -> Transaction 'ReadOnly a -> IO a
TrH.submitReadOnly TransactionHandle 'ReadOnly
trh (Transaction 'ReadOnly (LedgerTables l ValuesMK, Maybe (TxIn l))
-> IO (LedgerTables l ValuesMK, Maybe (TxIn l)))
-> Transaction 'ReadOnly (LedgerTables l ValuesMK, Maybe (TxIn l))
-> IO (LedgerTables l ValuesMK, Maybe (TxIn l))
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 :: * -> * -> *).
LedgerTables l mk -> mk (TxIn l) (TxOut l)
getLedgerTables LedgerTables l LMDBMK
dbBackingTables
in (ValuesMK (TxIn l) (TxOut l) -> LedgerTables l ValuesMK)
-> (ValuesMK (TxIn l) (TxOut l), Maybe (TxIn l))
-> (LedgerTables l ValuesMK, Maybe (TxIn l))
forall a b c. (a -> b) -> (a, c) -> (b, c)
forall (p :: * -> * -> *) a b c.
Bifunctor p =>
(a -> b) -> p a c -> p b c
first ValuesMK (TxIn l) (TxOut l) -> LedgerTables l ValuesMK
forall (l :: LedgerStateKind) (mk :: * -> * -> *).
mk (TxIn l) (TxOut l) -> LedgerTables l mk
LedgerTables ((ValuesMK (TxIn l) (TxOut l), Maybe (TxIn l))
-> (LedgerTables l ValuesMK, Maybe (TxIn l)))
-> Transaction
'ReadOnly (ValuesMK (TxIn l) (TxOut l), Maybe (TxIn l))
-> Transaction 'ReadOnly (LedgerTables l ValuesMK, Maybe (TxIn l))
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), Maybe (TxIn 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), Maybe (TxIn 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 :: * -> * -> *).
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 :: * -> * -> *).
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, Maybe (TxIn l))
API.bsvhRangeRead = l EmptyMK
-> RangeQuery (LedgerTables l KeysMK)
-> m (LedgerTables l ValuesMK, Maybe (TxIn l))
ReadHint (LedgerTables l ValuesMK)
-> RangeQuery (LedgerTables l KeysMK)
-> m (LedgerTables l ValuesMK, Maybe (TxIn l))
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
newtype Cleanup m = Cleanup {forall (m :: * -> *). Cleanup m -> m ()
runCleanup :: m ()}
data LMDBErr
=
LMDBErrNoDbSeqNo
|
LMDBErrNonMonotonicSeq !(WithOrigin SlotNo) !(WithOrigin SlotNo)
|
LMDBErrInitialisingNonEmpty !String
|
LMDBErrInitialisingAlreadyHasState
|
LMDBErrNoValueHandle !Int
|
LMDBErrUnableToReadSeqNo
|
LMDBErrBadRead
|
LMDBErrBadRangeRead
|
LMDBErrDirExists !FilePath
|
LMDBErrDirDoesntExist !FilePath
|
LMDBErrDirIsNotLMDB !FilePath
|
LMDBErrNotADir !FS.FsPath
|
LMDBErrClosed
instance Exception 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
]
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."
type data LMDB
instance
( HasLedgerTables l
, MonadIO m
, IOLike m
, MemPackIdx l EmptyMK ~ l EmptyMK
) =>
Backend m LMDB l
where
data Args m LMDB
= LMDBBackingStoreArgs FilePath LMDBLimits (Dict.Dict MonadIOPrim m)
data Trace m LMDB
= OnDiskBackingStoreInitialise LMDB.Limits
| OnDiskBackingStoreTrace BackingStoreTrace
deriving (Trace m LMDB -> Trace m LMDB -> Bool
(Trace m LMDB -> Trace m LMDB -> Bool)
-> (Trace m LMDB -> Trace m LMDB -> Bool) -> Eq (Trace m LMDB)
forall a. (a -> a -> Bool) -> (a -> a -> Bool) -> Eq a
forall (m :: * -> *). Trace m LMDB -> Trace m LMDB -> Bool
$c== :: forall (m :: * -> *). Trace m LMDB -> Trace m LMDB -> Bool
== :: Trace m LMDB -> Trace m LMDB -> Bool
$c/= :: forall (m :: * -> *). Trace m LMDB -> Trace m LMDB -> Bool
/= :: Trace m LMDB -> Trace m LMDB -> Bool
Eq, Int -> Trace m LMDB -> ShowS
[Trace m LMDB] -> ShowS
Trace m LMDB -> String
(Int -> Trace m LMDB -> ShowS)
-> (Trace m LMDB -> String)
-> ([Trace m LMDB] -> ShowS)
-> Show (Trace m LMDB)
forall a.
(Int -> a -> ShowS) -> (a -> String) -> ([a] -> ShowS) -> Show a
forall (m :: * -> *). Int -> Trace m LMDB -> ShowS
forall (m :: * -> *). [Trace m LMDB] -> ShowS
forall (m :: * -> *). Trace m LMDB -> String
$cshowsPrec :: forall (m :: * -> *). Int -> Trace m LMDB -> ShowS
showsPrec :: Int -> Trace m LMDB -> ShowS
$cshow :: forall (m :: * -> *). Trace m LMDB -> String
show :: Trace m LMDB -> String
$cshowList :: forall (m :: * -> *). [Trace m LMDB] -> ShowS
showList :: [Trace m LMDB] -> ShowS
Show)
isRightBackendForSnapshot :: Proxy l -> Args m LMDB -> SnapshotBackend -> Bool
isRightBackendForSnapshot Proxy l
_ Args m LMDB
_ SnapshotBackend
UTxOHDLMDBSnapshot = Bool
True
isRightBackendForSnapshot Proxy l
_ Args m LMDB
_ SnapshotBackend
_ = Bool
False
newBackingStoreInitialiser :: Tracer m SomeBackendTrace
-> Args m LMDB -> SnapshotsFS m -> BackingStoreInitialiser m l
newBackingStoreInitialiser Tracer m SomeBackendTrace
trcr (LMDBBackingStoreArgs String
fs LMDBLimits
limits Dict MonadIOPrim m
Dict.Dict) =
Tracer m BackingStoreTrace
-> LMDBLimits
-> LiveLMDBFS m
-> SnapshotsFS m
-> BackingStoreInitialiser m l
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
(Trace (ZonkAny 0) LMDB -> SomeBackendTrace
forall (m :: * -> *) backend.
Show (Trace m backend) =>
Trace m backend -> SomeBackendTrace
SomeBackendTrace (Trace (ZonkAny 0) LMDB -> SomeBackendTrace)
-> (BackingStoreTrace -> Trace (ZonkAny 0) LMDB)
-> BackingStoreTrace
-> SomeBackendTrace
forall b c a. (b -> c) -> (a -> b) -> a -> c
. BackingStoreTrace -> Trace (ZonkAny 0) LMDB
forall (m :: * -> *). BackingStoreTrace -> Trace m LMDB
OnDiskBackingStoreTrace (BackingStoreTrace -> SomeBackendTrace)
-> Tracer m SomeBackendTrace -> Tracer m BackingStoreTrace
forall (f :: * -> *) a b. Contravariant f => (a -> b) -> f b -> f a
>$< Tracer m SomeBackendTrace
trcr)
LMDBLimits
limits
(SomeHasFS m -> LiveLMDBFS m
forall (m :: * -> *). SomeHasFS m -> LiveLMDBFS m
LiveLMDBFS (SomeHasFS m -> LiveLMDBFS m) -> SomeHasFS m -> LiveLMDBFS m
forall a b. (a -> b) -> a -> b
$ HasFS m HandleIO -> SomeHasFS m
forall h (m :: * -> *). Eq h => HasFS m h -> SomeHasFS m
FS.SomeHasFS (HasFS m HandleIO -> SomeHasFS m)
-> HasFS m HandleIO -> SomeHasFS m
forall a b. (a -> b) -> a -> b
$ MountPoint -> HasFS m HandleIO
forall (m :: * -> *).
(MonadIO m, PrimState IO ~ PrimState m) =>
MountPoint -> HasFS m HandleIO
ioHasFS (MountPoint -> HasFS m HandleIO) -> MountPoint -> HasFS m HandleIO
forall a b. (a -> b) -> a -> b
$ String -> MountPoint
FS.MountPoint String
fs)
mkLMDBArgs ::
( MonadIOPrim m
, HasLedgerTables (LedgerState blk)
, IOLike m
) =>
V1.FlushFrequency -> FilePath -> LMDBLimits -> a -> (LedgerDbBackendArgs m blk, a)
mkLMDBArgs :: forall (m :: * -> *) blk a.
(MonadIOPrim m, HasLedgerTables (LedgerState blk), IOLike m) =>
FlushFrequency
-> String -> LMDBLimits -> a -> (LedgerDbBackendArgs m blk, a)
mkLMDBArgs FlushFrequency
flushing String
lmdbPath LMDBLimits
limits =
(,) (LedgerDbBackendArgs m blk -> a -> (LedgerDbBackendArgs m blk, a))
-> LedgerDbBackendArgs m blk -> a -> (LedgerDbBackendArgs m blk, a)
forall a b. (a -> b) -> a -> b
$
LedgerDbBackendArgs m (ExtLedgerState blk)
-> LedgerDbBackendArgs m blk
forall (m :: * -> *) blk.
LedgerDbBackendArgs m (ExtLedgerState blk)
-> LedgerDbBackendArgs m blk
LedgerDbBackendArgsV1 (LedgerDbBackendArgs m (ExtLedgerState blk)
-> LedgerDbBackendArgs m blk)
-> LedgerDbBackendArgs m (ExtLedgerState blk)
-> LedgerDbBackendArgs m blk
forall a b. (a -> b) -> a -> b
$
FlushFrequency
-> SomeBackendArgs m (ExtLedgerState blk)
-> LedgerDbBackendArgs m (ExtLedgerState blk)
forall (m :: * -> *) (l :: LedgerStateKind).
FlushFrequency -> SomeBackendArgs m l -> LedgerDbBackendArgs m l
V1.V1Args FlushFrequency
flushing (SomeBackendArgs m (ExtLedgerState blk)
-> LedgerDbBackendArgs m (ExtLedgerState blk))
-> SomeBackendArgs m (ExtLedgerState blk)
-> LedgerDbBackendArgs m (ExtLedgerState blk)
forall a b. (a -> b) -> a -> b
$
Args m LMDB -> SomeBackendArgs m (ExtLedgerState blk)
forall (m :: * -> *) backend (l :: LedgerStateKind).
Backend m backend l =>
Args m backend -> SomeBackendArgs m l
SomeBackendArgs (Args m LMDB -> SomeBackendArgs m (ExtLedgerState blk))
-> Args m LMDB -> SomeBackendArgs m (ExtLedgerState blk)
forall a b. (a -> b) -> a -> b
$
String -> LMDBLimits -> Dict MonadIOPrim m -> Args m LMDB
forall (m :: * -> *).
String -> LMDBLimits -> Dict MonadIOPrim m -> Args m LMDB
LMDBBackingStoreArgs String
lmdbPath LMDBLimits
limits Dict MonadIOPrim m
forall {k} (c :: k -> Constraint) (a :: k). c a => Dict c a
Dict.Dict
class (MonadIO m, PrimState m ~ PrimState IO) => MonadIOPrim m
instance (MonadIO m, PrimState m ~ PrimState IO) => MonadIOPrim m
instance (Ord (TxIn l), GetTip l, Monad m) => StreamingBackend m LMDB l where
data SinkArgs m LMDB l
= SinkLMDB
Int
( SlotNo ->
(l EmptyMK, l EmptyMK) ->
LedgerTables l DiffMK ->
m ()
)
(l EmptyMK -> m ())
data YieldArgs m LMDB l
= YieldLMDB
Int
(LedgerBackingStoreValueHandle m l)
yield :: Proxy LMDB -> YieldArgs m LMDB l -> Yield m l
yield Proxy LMDB
_ (YieldLMDB Int
chunkSize LedgerBackingStoreValueHandle m l
valueHandle) = Int -> LedgerBackingStoreValueHandle m l -> Yield m l
forall (m :: * -> *) (l :: LedgerStateKind).
Monad m =>
Int -> LedgerBackingStoreValueHandle m l -> Yield m l
yieldLmdbS Int
chunkSize LedgerBackingStoreValueHandle m l
valueHandle
sink :: Proxy LMDB -> SinkArgs m LMDB l -> Sink m l
sink Proxy LMDB
_ (SinkLMDB Int
chunkSize SlotNo -> (l EmptyMK, l EmptyMK) -> LedgerTables l DiffMK -> m ()
write l EmptyMK -> m ()
copy) = Int
-> (SlotNo
-> (l EmptyMK, l EmptyMK) -> LedgerTables l DiffMK -> m ())
-> (l EmptyMK -> m ())
-> Sink m l
forall (m :: * -> *) (l :: LedgerStateKind).
(Ord (TxIn l), GetTip l, Monad m) =>
Int
-> (SlotNo
-> (l EmptyMK, l EmptyMK) -> LedgerTables l DiffMK -> m ())
-> (l EmptyMK -> m ())
-> Sink m l
sinkLmdbS Int
chunkSize SlotNo -> (l EmptyMK, l EmptyMK) -> LedgerTables l DiffMK -> m ()
write l EmptyMK -> m ()
copy
sinkLmdbS ::
forall m l.
(Ord (TxIn l), GetTip l, Monad m) =>
Int ->
(SlotNo -> (l EmptyMK, l EmptyMK) -> LedgerTables l DiffMK -> m ()) ->
(l EmptyMK -> m ()) ->
Sink m l
sinkLmdbS :: forall (m :: * -> *) (l :: LedgerStateKind).
(Ord (TxIn l), GetTip l, Monad m) =>
Int
-> (SlotNo
-> (l EmptyMK, l EmptyMK) -> LedgerTables l DiffMK -> m ())
-> (l EmptyMK -> m ())
-> Sink m l
sinkLmdbS Int
writeChunkSize SlotNo -> (l EmptyMK, l EmptyMK) -> LedgerTables l DiffMK -> m ()
bs l EmptyMK -> m ()
copyTo l EmptyMK
hint Stream
(Of (TxIn l, TxOut l))
(ExceptT DeserialiseFailure m)
(Stream (Of ByteString) m (Maybe CRC))
s = do
r <- Int
-> Map (TxIn l) (TxOut l)
-> Stream
(Of (TxIn l, TxOut l))
(ExceptT DeserialiseFailure m)
(Stream (Of ByteString) m (Maybe CRC))
-> ExceptT
DeserialiseFailure m (Stream (Of ByteString) m (Maybe CRC))
go Int
writeChunkSize Map (TxIn l) (TxOut l)
forall a. Monoid a => a
mempty Stream
(Of (TxIn l, TxOut l))
(ExceptT DeserialiseFailure m)
(Stream (Of ByteString) m (Maybe CRC))
s
lift $ copyTo hint
pure (fmap (,Nothing) r)
where
sl :: SlotNo
sl = SlotNo -> (SlotNo -> SlotNo) -> WithOrigin SlotNo -> SlotNo
forall b t. b -> (t -> b) -> WithOrigin t -> b
withOrigin (String -> SlotNo
forall a. HasCallStack => String -> a
error String
"unreachable") SlotNo -> SlotNo
forall a. a -> a
id (WithOrigin SlotNo -> SlotNo) -> WithOrigin SlotNo -> SlotNo
forall a b. (a -> b) -> a -> b
$ Point l -> WithOrigin SlotNo
forall {k} (block :: k). Point block -> WithOrigin SlotNo
pointSlot (Point l -> WithOrigin SlotNo) -> Point l -> WithOrigin SlotNo
forall a b. (a -> b) -> a -> b
$ l EmptyMK -> Point l
forall (mk :: * -> * -> *). l mk -> Point l
forall (l :: LedgerStateKind) (mk :: * -> * -> *).
GetTip l =>
l mk -> Point l
getTip l EmptyMK
hint
go :: Int
-> Map (TxIn l) (TxOut l)
-> Stream
(Of (TxIn l, TxOut l))
(ExceptT DeserialiseFailure m)
(Stream (Of ByteString) m (Maybe CRC))
-> ExceptT
DeserialiseFailure m (Stream (Of ByteString) m (Maybe CRC))
go Int
0 Map (TxIn l) (TxOut l)
m Stream
(Of (TxIn l, TxOut l))
(ExceptT DeserialiseFailure m)
(Stream (Of ByteString) m (Maybe CRC))
s' = do
m () -> ExceptT DeserialiseFailure m ()
forall (m :: * -> *) a.
Monad m =>
m a -> ExceptT DeserialiseFailure m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (m () -> ExceptT DeserialiseFailure m ())
-> m () -> ExceptT DeserialiseFailure m ()
forall a b. (a -> b) -> a -> b
$ SlotNo -> (l EmptyMK, l EmptyMK) -> LedgerTables l DiffMK -> m ()
bs SlotNo
sl (l EmptyMK
hint, l EmptyMK
hint) (DiffMK (TxIn l) (TxOut l) -> LedgerTables l DiffMK
forall (l :: LedgerStateKind) (mk :: * -> * -> *).
mk (TxIn l) (TxOut l) -> LedgerTables l mk
LedgerTables (DiffMK (TxIn l) (TxOut l) -> LedgerTables l DiffMK)
-> DiffMK (TxIn l) (TxOut l) -> LedgerTables l DiffMK
forall a b. (a -> b) -> a -> b
$ Diff (TxIn l) (TxOut l) -> DiffMK (TxIn l) (TxOut l)
forall k v. Diff k v -> DiffMK k v
DiffMK (Diff (TxIn l) (TxOut l) -> DiffMK (TxIn l) (TxOut l))
-> Diff (TxIn l) (TxOut l) -> DiffMK (TxIn l) (TxOut l)
forall a b. (a -> b) -> a -> b
$ Map (TxIn l) (TxOut l) -> Diff (TxIn l) (TxOut l)
forall k v. Map k v -> Diff k v
Diff.fromMapInserts Map (TxIn l) (TxOut l)
m)
Int
-> Map (TxIn l) (TxOut l)
-> Stream
(Of (TxIn l, TxOut l))
(ExceptT DeserialiseFailure m)
(Stream (Of ByteString) m (Maybe CRC))
-> ExceptT
DeserialiseFailure m (Stream (Of ByteString) m (Maybe CRC))
go Int
writeChunkSize Map (TxIn l) (TxOut l)
forall a. Monoid a => a
mempty Stream
(Of (TxIn l, TxOut l))
(ExceptT DeserialiseFailure m)
(Stream (Of ByteString) m (Maybe CRC))
s'
go Int
n Map (TxIn l) (TxOut l)
m Stream
(Of (TxIn l, TxOut l))
(ExceptT DeserialiseFailure m)
(Stream (Of ByteString) m (Maybe CRC))
s' = do
mbs <- Stream
(Of (TxIn l, TxOut l))
(ExceptT DeserialiseFailure m)
(Stream (Of ByteString) m (Maybe CRC))
-> ExceptT
DeserialiseFailure
m
(Maybe
((TxIn l, TxOut l),
Stream
(Of (TxIn l, TxOut l))
(ExceptT DeserialiseFailure m)
(Stream (Of ByteString) m (Maybe CRC))))
forall (m :: * -> *) a r.
Monad m =>
Stream (Of a) m r -> m (Maybe (a, Stream (Of a) m r))
S.uncons Stream
(Of (TxIn l, TxOut l))
(ExceptT DeserialiseFailure m)
(Stream (Of ByteString) m (Maybe CRC))
s'
case mbs of
Maybe
((TxIn l, TxOut l),
Stream
(Of (TxIn l, TxOut l))
(ExceptT DeserialiseFailure m)
(Stream (Of ByteString) m (Maybe CRC)))
Nothing -> do
m () -> ExceptT DeserialiseFailure m ()
forall (m :: * -> *) a.
Monad m =>
m a -> ExceptT DeserialiseFailure m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (m () -> ExceptT DeserialiseFailure m ())
-> m () -> ExceptT DeserialiseFailure m ()
forall a b. (a -> b) -> a -> b
$ SlotNo -> (l EmptyMK, l EmptyMK) -> LedgerTables l DiffMK -> m ()
bs SlotNo
sl (l EmptyMK
hint, l EmptyMK
hint) (DiffMK (TxIn l) (TxOut l) -> LedgerTables l DiffMK
forall (l :: LedgerStateKind) (mk :: * -> * -> *).
mk (TxIn l) (TxOut l) -> LedgerTables l mk
LedgerTables (DiffMK (TxIn l) (TxOut l) -> LedgerTables l DiffMK)
-> DiffMK (TxIn l) (TxOut l) -> LedgerTables l DiffMK
forall a b. (a -> b) -> a -> b
$ Diff (TxIn l) (TxOut l) -> DiffMK (TxIn l) (TxOut l)
forall k v. Diff k v -> DiffMK k v
DiffMK (Diff (TxIn l) (TxOut l) -> DiffMK (TxIn l) (TxOut l))
-> Diff (TxIn l) (TxOut l) -> DiffMK (TxIn l) (TxOut l)
forall a b. (a -> b) -> a -> b
$ Map (TxIn l) (TxOut l) -> Diff (TxIn l) (TxOut l)
forall k v. Map k v -> Diff k v
Diff.fromMapInserts Map (TxIn l) (TxOut l)
m)
Stream
(Of (TxIn l, TxOut l))
(ExceptT DeserialiseFailure m)
(Stream (Of ByteString) m (Maybe CRC))
-> ExceptT
DeserialiseFailure m (Stream (Of ByteString) m (Maybe CRC))
forall (m :: * -> *) a r. Monad m => Stream (Of a) m r -> m r
S.effects Stream
(Of (TxIn l, TxOut l))
(ExceptT DeserialiseFailure m)
(Stream (Of ByteString) m (Maybe CRC))
s'
Just ((TxIn l
k, TxOut l
v), Stream
(Of (TxIn l, TxOut l))
(ExceptT DeserialiseFailure m)
(Stream (Of ByteString) m (Maybe CRC))
s'') ->
Int
-> Map (TxIn l) (TxOut l)
-> Stream
(Of (TxIn l, TxOut l))
(ExceptT DeserialiseFailure m)
(Stream (Of ByteString) m (Maybe CRC))
-> ExceptT
DeserialiseFailure m (Stream (Of ByteString) m (Maybe CRC))
go (Int
n Int -> Int -> Int
forall a. Num a => a -> a -> a
- Int
1) (TxIn l
-> TxOut l -> Map (TxIn l) (TxOut l) -> Map (TxIn l) (TxOut l)
forall k a. Ord k => k -> a -> Map k a -> Map k a
Map.insert TxIn l
k TxOut l
v Map (TxIn l) (TxOut l)
m) Stream
(Of (TxIn l, TxOut l))
(ExceptT DeserialiseFailure m)
(Stream (Of ByteString) m (Maybe CRC))
s''
yieldLmdbS ::
Monad m =>
Int ->
LedgerBackingStoreValueHandle m l ->
Yield m l
yieldLmdbS :: forall (m :: * -> *) (l :: LedgerStateKind).
Monad m =>
Int -> LedgerBackingStoreValueHandle m l -> Yield m l
yieldLmdbS Int
readChunkSize LedgerBackingStoreValueHandle m l
bsvh l EmptyMK
hint Stream
(Of (TxIn l, TxOut l))
(ExceptT DeserialiseFailure m)
(Stream (Of ByteString) m (Maybe CRC))
-> ExceptT
DeserialiseFailure
m
(Stream (Of ByteString) m (Maybe CRC, Maybe CRC))
k = do
r <- Stream
(Of (TxIn l, TxOut l))
(ExceptT DeserialiseFailure m)
(Stream (Of ByteString) m (Maybe CRC))
-> ExceptT
DeserialiseFailure
m
(Stream (Of ByteString) m (Maybe CRC, Maybe CRC))
k (RangeQuery (LedgerTables l KeysMK)
-> Stream
(Of (TxIn l, TxOut l))
(ExceptT DeserialiseFailure m)
(Stream (Of ByteString) m (Maybe CRC))
go (Maybe (LedgerTables l KeysMK)
-> Int -> RangeQuery (LedgerTables l KeysMK)
forall keys. Maybe keys -> Int -> RangeQuery keys
RangeQuery Maybe (LedgerTables l KeysMK)
forall a. Maybe a
Nothing Int
readChunkSize))
lift $ S.effects r
where
go :: RangeQuery (LedgerTables l KeysMK)
-> Stream
(Of (TxIn l, TxOut l))
(ExceptT DeserialiseFailure m)
(Stream (Of ByteString) m (Maybe CRC))
go RangeQuery (LedgerTables l KeysMK)
p = do
(LedgerTables (ValuesMK values), mx) <- ExceptT
DeserialiseFailure m (LedgerTables l ValuesMK, Maybe (TxIn l))
-> Stream
(Of (TxIn l, TxOut l))
(ExceptT DeserialiseFailure m)
(LedgerTables l ValuesMK, Maybe (TxIn l))
forall (m :: * -> *) a.
Monad m =>
m a -> Stream (Of (TxIn l, TxOut l)) m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (ExceptT
DeserialiseFailure m (LedgerTables l ValuesMK, Maybe (TxIn l))
-> Stream
(Of (TxIn l, TxOut l))
(ExceptT DeserialiseFailure m)
(LedgerTables l ValuesMK, Maybe (TxIn l)))
-> ExceptT
DeserialiseFailure m (LedgerTables l ValuesMK, Maybe (TxIn l))
-> Stream
(Of (TxIn l, TxOut l))
(ExceptT DeserialiseFailure m)
(LedgerTables l ValuesMK, Maybe (TxIn l))
forall a b. (a -> b) -> a -> b
$ m (LedgerTables l ValuesMK, Maybe (TxIn l))
-> ExceptT
DeserialiseFailure m (LedgerTables l ValuesMK, Maybe (TxIn l))
forall (m :: * -> *) a.
Monad m =>
m a -> ExceptT DeserialiseFailure m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
S.lift (m (LedgerTables l ValuesMK, Maybe (TxIn l))
-> ExceptT
DeserialiseFailure m (LedgerTables l ValuesMK, Maybe (TxIn l)))
-> m (LedgerTables l ValuesMK, Maybe (TxIn l))
-> ExceptT
DeserialiseFailure m (LedgerTables l ValuesMK, Maybe (TxIn l))
forall a b. (a -> b) -> a -> b
$ LedgerBackingStoreValueHandle m l
-> ReadHint (LedgerTables l ValuesMK)
-> RangeQuery (LedgerTables l KeysMK)
-> m (LedgerTables l ValuesMK, Maybe (TxIn l))
forall (m :: * -> *) keys key values.
BackingStoreValueHandle m keys key values
-> ReadHint values -> RangeQuery keys -> m (values, Maybe key)
bsvhRangeRead LedgerBackingStoreValueHandle m l
bsvh l EmptyMK
ReadHint (LedgerTables l ValuesMK)
hint RangeQuery (LedgerTables l KeysMK)
p
case mx of
Maybe (TxIn l)
Nothing -> Stream (Of ByteString) m (Maybe CRC)
-> Stream
(Of (TxIn l, TxOut l))
(ExceptT DeserialiseFailure m)
(Stream (Of ByteString) m (Maybe CRC))
forall a.
a -> Stream (Of (TxIn l, TxOut l)) (ExceptT DeserialiseFailure m) a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (Stream (Of ByteString) m (Maybe CRC)
-> Stream
(Of (TxIn l, TxOut l))
(ExceptT DeserialiseFailure m)
(Stream (Of ByteString) m (Maybe CRC)))
-> Stream (Of ByteString) m (Maybe CRC)
-> Stream
(Of (TxIn l, TxOut l))
(ExceptT DeserialiseFailure m)
(Stream (Of ByteString) m (Maybe CRC))
forall a b. (a -> b) -> a -> b
$ Maybe CRC -> Stream (Of ByteString) m (Maybe CRC)
forall a. a -> Stream (Of ByteString) m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure Maybe CRC
forall a. Maybe a
Nothing
Just TxIn l
x -> do
[(TxIn l, TxOut l)]
-> Stream (Of (TxIn l, TxOut l)) (ExceptT DeserialiseFailure m) ()
forall (m :: * -> *) (f :: * -> *) a.
(Monad m, Foldable f) =>
f a -> Stream (Of a) m ()
S.each ([(TxIn l, TxOut l)]
-> Stream (Of (TxIn l, TxOut l)) (ExceptT DeserialiseFailure m) ())
-> [(TxIn l, TxOut l)]
-> Stream (Of (TxIn l, TxOut l)) (ExceptT DeserialiseFailure m) ()
forall a b. (a -> b) -> a -> b
$ Map (TxIn l) (TxOut l) -> [(TxIn l, TxOut l)]
forall k a. Map k a -> [(k, a)]
Map.toList Map (TxIn l) (TxOut l)
values
RangeQuery (LedgerTables l KeysMK)
-> Stream
(Of (TxIn l, TxOut l))
(ExceptT DeserialiseFailure m)
(Stream (Of ByteString) m (Maybe CRC))
go (Maybe (LedgerTables l KeysMK)
-> Int -> RangeQuery (LedgerTables l KeysMK)
forall keys. Maybe keys -> Int -> RangeQuery keys
RangeQuery (LedgerTables l KeysMK -> Maybe (LedgerTables l KeysMK)
forall a. a -> Maybe a
Just (LedgerTables l KeysMK -> Maybe (LedgerTables l KeysMK))
-> (Set (TxIn l) -> LedgerTables l KeysMK)
-> Set (TxIn l)
-> Maybe (LedgerTables l KeysMK)
forall b c a. (b -> c) -> (a -> b) -> a -> c
. KeysMK (TxIn l) (TxOut l) -> LedgerTables l KeysMK
forall (l :: LedgerStateKind) (mk :: * -> * -> *).
mk (TxIn l) (TxOut l) -> LedgerTables l mk
LedgerTables (KeysMK (TxIn l) (TxOut l) -> LedgerTables l KeysMK)
-> (Set (TxIn l) -> KeysMK (TxIn l) (TxOut l))
-> Set (TxIn l)
-> LedgerTables l KeysMK
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Set (TxIn l) -> KeysMK (TxIn l) (TxOut l)
forall k v. Set k -> KeysMK k v
KeysMK (Set (TxIn l) -> Maybe (LedgerTables l KeysMK))
-> Set (TxIn l) -> Maybe (LedgerTables l KeysMK)
forall a b. (a -> b) -> a -> b
$ TxIn l -> Set (TxIn l)
forall a. a -> Set a
Set.singleton TxIn l
x) Int
readChunkSize)
mkLMDBYieldArgs ::
forall l.
( HasCallStack
, HasLedgerTables l
, MemPackIdx l EmptyMK ~ l EmptyMK
) =>
FilePath ->
LMDBLimits ->
l EmptyMK ->
ResourceRegistry IO ->
IO (YieldArgs IO LMDB l)
mkLMDBYieldArgs :: forall (l :: LedgerStateKind).
(HasCallStack, HasLedgerTables l,
MemPackIdx l EmptyMK ~ l EmptyMK) =>
String
-> LMDBLimits
-> l EmptyMK
-> ResourceRegistry IO
-> IO (YieldArgs IO LMDB l)
mkLMDBYieldArgs String
fp LMDBLimits
limits l EmptyMK
hint ResourceRegistry IO
reg = do
let (String
dbPath, String
snapName) = String -> (String, String)
FilePath.splitFileName String
fp
tempDir <- IO String
getCanonicalTemporaryDirectory
let lmdbTemp = String
tempDir String -> ShowS
FilePath.</> String
"lmdb_streaming_in"
removePathForcibly lmdbTemp
_ <-
allocate
reg
(\ResourceId
_ -> String -> IO ()
createDirectory String
lmdbTemp)
(\()
_ -> String -> IO ()
removePathForcibly String
lmdbTemp)
(_, bs) <-
allocate
reg
( \ResourceId
_ -> do
Tracer IO BackingStoreTrace
-> LMDBLimits
-> LiveLMDBFS IO
-> SnapshotsFS IO
-> InitFrom (LedgerTables l ValuesMK)
-> IO
(BackingStore
IO
(LedgerTables l KeysMK)
(TxIn l)
(LedgerTables l ValuesMK)
(LedgerTables l DiffMK))
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 IO BackingStoreTrace
forall (m :: * -> *) a. Applicative m => Tracer m a
Trace.nullTracer
LMDBLimits
limits
(SomeHasFS IO -> LiveLMDBFS IO
forall (m :: * -> *). SomeHasFS m -> LiveLMDBFS m
LiveLMDBFS (SomeHasFS IO -> LiveLMDBFS IO) -> SomeHasFS IO -> LiveLMDBFS IO
forall a b. (a -> b) -> a -> b
$ HasFS IO HandleIO -> SomeHasFS IO
forall h (m :: * -> *). Eq h => HasFS m h -> SomeHasFS m
FS.SomeHasFS (HasFS IO HandleIO -> SomeHasFS IO)
-> HasFS IO HandleIO -> SomeHasFS IO
forall a b. (a -> b) -> a -> b
$ MountPoint -> HasFS IO HandleIO
forall (m :: * -> *).
(MonadIO m, PrimState IO ~ PrimState m) =>
MountPoint -> HasFS m HandleIO
ioHasFS (MountPoint -> HasFS IO HandleIO)
-> MountPoint -> HasFS IO HandleIO
forall a b. (a -> b) -> a -> b
$ String -> MountPoint
FS.MountPoint String
lmdbTemp)
(SomeHasFS IO -> SnapshotsFS IO
forall (m :: * -> *). SomeHasFS m -> SnapshotsFS m
SnapshotsFS (SomeHasFS IO -> SnapshotsFS IO) -> SomeHasFS IO -> SnapshotsFS IO
forall a b. (a -> b) -> a -> b
$ HasFS IO HandleIO -> SomeHasFS IO
forall h (m :: * -> *). Eq h => HasFS m h -> SomeHasFS m
FS.SomeHasFS (HasFS IO HandleIO -> SomeHasFS IO)
-> HasFS IO HandleIO -> SomeHasFS IO
forall a b. (a -> b) -> a -> b
$ MountPoint -> HasFS IO HandleIO
forall (m :: * -> *).
(MonadIO m, PrimState IO ~ PrimState m) =>
MountPoint -> HasFS m HandleIO
ioHasFS (MountPoint -> HasFS IO HandleIO)
-> MountPoint -> HasFS IO HandleIO
forall a b. (a -> b) -> a -> b
$ String -> MountPoint
FS.MountPoint String
dbPath)
(InitHint (LedgerTables l ValuesMK)
-> FsPath -> InitFrom (LedgerTables l ValuesMK)
forall values. InitHint values -> FsPath -> InitFrom values
InitFromCopy l EmptyMK
InitHint (LedgerTables l ValuesMK)
hint ([String] -> FsPath
FS.mkFsPath [String
snapName]))
)
bsClose
(_, bsvh) <- allocate reg (\ResourceId
_ -> BackingStore
IO
(LedgerTables l KeysMK)
(TxIn l)
(LedgerTables l ValuesMK)
(LedgerTables l DiffMK)
-> IO
(BackingStoreValueHandle
IO (LedgerTables l KeysMK) (TxIn l) (LedgerTables l ValuesMK))
forall (m :: * -> *) keys key values diff.
BackingStore m keys key values diff
-> m (BackingStoreValueHandle m keys key values)
bsValueHandle BackingStore
IO
(LedgerTables l KeysMK)
(TxIn l)
(LedgerTables l ValuesMK)
(LedgerTables l DiffMK)
bs) bsvhClose
pure (YieldLMDB 1000 bsvh)
mkLMDBSinkArgs ::
forall l.
( HasCallStack
, HasLedgerTables l
, MemPackIdx l EmptyMK ~ l EmptyMK
) =>
FilePath ->
LMDBLimits ->
l EmptyMK ->
ResourceRegistry IO ->
IO (SinkArgs IO LMDB l)
mkLMDBSinkArgs :: forall (l :: LedgerStateKind).
(HasCallStack, HasLedgerTables l,
MemPackIdx l EmptyMK ~ l EmptyMK) =>
String
-> LMDBLimits
-> l EmptyMK
-> ResourceRegistry IO
-> IO (SinkArgs IO LMDB l)
mkLMDBSinkArgs String
fp LMDBLimits
limits l EmptyMK
hint ResourceRegistry IO
reg = do
let (String
snapDir, String
snapName) = String -> (String, String)
FilePath.splitFileName String
fp
tempDir <- IO String
getCanonicalTemporaryDirectory
let lmdbTemp = String
tempDir String -> ShowS
FilePath.</> String
"lmdb_streaming_out"
removePathForcibly lmdbTemp
_ <- allocate reg (\ResourceId
_ -> String -> IO ()
createDirectory String
lmdbTemp) (\()
_ -> String -> IO ()
removePathForcibly String
lmdbTemp)
(_, bs) <-
allocate
reg
( \ResourceId
_ ->
Tracer IO BackingStoreTrace
-> LMDBLimits
-> LiveLMDBFS IO
-> SnapshotsFS IO
-> InitFrom (LedgerTables l ValuesMK)
-> IO
(BackingStore
IO
(LedgerTables l KeysMK)
(TxIn l)
(LedgerTables l ValuesMK)
(LedgerTables l DiffMK))
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 IO BackingStoreTrace
forall (m :: * -> *) a. Applicative m => Tracer m a
Trace.nullTracer
LMDBLimits
limits
(SomeHasFS IO -> LiveLMDBFS IO
forall (m :: * -> *). SomeHasFS m -> LiveLMDBFS m
LiveLMDBFS (SomeHasFS IO -> LiveLMDBFS IO) -> SomeHasFS IO -> LiveLMDBFS IO
forall a b. (a -> b) -> a -> b
$ HasFS IO HandleIO -> SomeHasFS IO
forall h (m :: * -> *). Eq h => HasFS m h -> SomeHasFS m
FS.SomeHasFS (HasFS IO HandleIO -> SomeHasFS IO)
-> HasFS IO HandleIO -> SomeHasFS IO
forall a b. (a -> b) -> a -> b
$ MountPoint -> HasFS IO HandleIO
forall (m :: * -> *).
(MonadIO m, PrimState IO ~ PrimState m) =>
MountPoint -> HasFS m HandleIO
ioHasFS (MountPoint -> HasFS IO HandleIO)
-> MountPoint -> HasFS IO HandleIO
forall a b. (a -> b) -> a -> b
$ String -> MountPoint
FS.MountPoint String
lmdbTemp)
(SomeHasFS IO -> SnapshotsFS IO
forall (m :: * -> *). SomeHasFS m -> SnapshotsFS m
SnapshotsFS (SomeHasFS IO -> SnapshotsFS IO) -> SomeHasFS IO -> SnapshotsFS IO
forall a b. (a -> b) -> a -> b
$ HasFS IO HandleIO -> SomeHasFS IO
forall h (m :: * -> *). Eq h => HasFS m h -> SomeHasFS m
FS.SomeHasFS (HasFS IO HandleIO -> SomeHasFS IO)
-> HasFS IO HandleIO -> SomeHasFS IO
forall a b. (a -> b) -> a -> b
$ MountPoint -> HasFS IO HandleIO
forall (m :: * -> *).
(MonadIO m, PrimState IO ~ PrimState m) =>
MountPoint -> HasFS m HandleIO
ioHasFS (MountPoint -> HasFS IO HandleIO)
-> MountPoint -> HasFS IO HandleIO
forall a b. (a -> b) -> a -> b
$ String -> MountPoint
FS.MountPoint String
snapDir)
(WithOrigin SlotNo
-> InitHint (LedgerTables l ValuesMK)
-> LedgerTables l ValuesMK
-> InitFrom (LedgerTables l ValuesMK)
forall values.
WithOrigin SlotNo -> InitHint values -> values -> InitFrom values
InitFromValues (SlotNo -> WithOrigin SlotNo
forall t. t -> WithOrigin t
At SlotNo
0) l EmptyMK
InitHint (LedgerTables l ValuesMK)
hint LedgerTables l ValuesMK
forall (mk :: * -> * -> *) (l :: LedgerStateKind).
(ZeroableMK mk, LedgerTableConstraints l) =>
LedgerTables l mk
emptyLedgerTables)
)
bsClose
pure $ SinkLMDB 1000 (bsWrite bs) (\l EmptyMK
h -> BackingStore
IO
(LedgerTables l KeysMK)
(TxIn l)
(LedgerTables l ValuesMK)
(LedgerTables l DiffMK)
-> SerializeTablesHint (LedgerTables l ValuesMK) -> FsPath -> IO ()
forall (m :: * -> *) keys key values diff.
BackingStore m keys key values diff
-> SerializeTablesHint values -> FsPath -> m ()
bsCopy BackingStore
IO
(LedgerTables l KeysMK)
(TxIn l)
(LedgerTables l ValuesMK)
(LedgerTables l DiffMK)
bs l EmptyMK
SerializeTablesHint (LedgerTables l ValuesMK)
h ([String] -> FsPath
FS.mkFsPath [String
snapName, String
"tables"]))