{-# LANGUAGE BangPatterns #-}
{-# LANGUAGE DeriveAnyClass #-}
{-# LANGUAGE DeriveGeneric #-}
{-# LANGUAGE DerivingVia #-}
{-# LANGUAGE FlexibleContexts #-}
{-# LANGUAGE FlexibleInstances #-}
{-# LANGUAGE GADTs #-}
{-# LANGUAGE LambdaCase #-}
{-# LANGUAGE MultiParamTypeClasses #-}
{-# LANGUAGE NamedFieldPuns #-}
{-# LANGUAGE ScopedTypeVariables #-}
{-# LANGUAGE StandaloneDeriving #-}
{-# LANGUAGE TupleSections #-}
{-# LANGUAGE TypeApplications #-}
{-# LANGUAGE TypeData #-}
{-# LANGUAGE TypeFamilies #-}
{-# LANGUAGE UndecidableInstances #-}
{-# LANGUAGE ViewPatterns #-}
-- Needed for @NoThunks (Table m k v b)@
{-# OPTIONS_GHC -Wno-orphans #-}

-- | Implementation of the 'LedgerTablesHandle' interface with LSM trees.
module Ouroboros.Consensus.Storage.LedgerDB.V2.LSM
  ( -- * Backend API
    LSM
  , Backend (..)
  , Args (LSMArgs)
  , mkLSMArgs
  , stdMkBlockIOFS

    -- * Streaming
  , YieldArgs (YieldLSM)
  , mkLSMYieldArgs
  , SinkArgs (SinkLSM)
  , mkLSMSinkArgs

    -- * Exported for tests
  , LSM.Salt
  , SomeHasFSAndBlockIO (..)
  ) where

import Codec.Serialise (decode)
import qualified Control.Monad as Monad
import Control.Monad.Trans (lift)
import Control.Monad.Trans.Except
import Control.ResourceRegistry
import Control.Tracer
import qualified Data.Foldable as Foldable
import Data.Functor.Contravariant ((>$<))
import qualified Data.List as List
import qualified Data.Map.Strict as Map
import Data.Maybe
import Data.MemPack
import qualified Data.Primitive.ByteArray as PBA
import qualified Data.Set as Set
import Data.String (fromString)
import qualified Data.Text as T
import qualified Data.Text as Text
import Data.Typeable
import qualified Data.Vector as V
import qualified Data.Vector.Mutable as VM
import qualified Data.Vector.Primitive as VP
import Data.Void
import Database.LSMTree (Salt, Session, Table)
import qualified Database.LSMTree as LSM
import GHC.Generics
import NoThunks.Class
import Ouroboros.Consensus.Block
import Ouroboros.Consensus.Ledger.Abstract
import Ouroboros.Consensus.Ledger.Extended
import Ouroboros.Consensus.Ledger.SupportsProtocol
import qualified Ouroboros.Consensus.Ledger.Tables.Diff as Diff
import Ouroboros.Consensus.Ledger.Tables.Utils
import Ouroboros.Consensus.Storage.LedgerDB.API
import Ouroboros.Consensus.Storage.LedgerDB.Args
import Ouroboros.Consensus.Storage.LedgerDB.Snapshots
import Ouroboros.Consensus.Storage.LedgerDB.V2.Backend
import Ouroboros.Consensus.Storage.LedgerDB.V2.LedgerSeq
import Ouroboros.Consensus.Util (chunks)
import Ouroboros.Consensus.Util.CRC
import Ouroboros.Consensus.Util.Enclose
import Ouroboros.Consensus.Util.IOLike
import Ouroboros.Consensus.Util.IndexedMemPack
import qualified Streaming as S
import qualified Streaming.Prelude as S
import System.FS.API
import qualified System.FS.BlockIO.API as BIO
import System.FS.BlockIO.IO
import System.FilePath (splitDirectories, splitFileName)
import System.Random
import Prelude hiding (read)

-- | Type alias for convenience
type UTxOTable m = Table m TxInBytes TxOutBytes Void

instance NoThunks (Table m txin txout Void) where
  showTypeOf :: Proxy (Table m txin txout Void) -> String
showTypeOf Proxy (Table m txin txout Void)
_ = String
"Table"
  wNoThunks :: Context -> Table m txin txout Void -> IO (Maybe ThunkInfo)
wNoThunks Context
_ Table m txin txout Void
_ = Maybe ThunkInfo -> IO (Maybe ThunkInfo)
forall a. a -> IO a
forall (f :: * -> *) a. Applicative f => a -> f a
pure Maybe ThunkInfo
forall a. Maybe a
Nothing

data LSMClosedExn = LSMClosedExn
  deriving (Int -> LSMClosedExn -> ShowS
[LSMClosedExn] -> ShowS
LSMClosedExn -> String
(Int -> LSMClosedExn -> ShowS)
-> (LSMClosedExn -> String)
-> ([LSMClosedExn] -> ShowS)
-> Show LSMClosedExn
forall a.
(Int -> a -> ShowS) -> (a -> String) -> ([a] -> ShowS) -> Show a
$cshowsPrec :: Int -> LSMClosedExn -> ShowS
showsPrec :: Int -> LSMClosedExn -> ShowS
$cshow :: LSMClosedExn -> String
show :: LSMClosedExn -> String
$cshowList :: [LSMClosedExn] -> ShowS
showList :: [LSMClosedExn] -> ShowS
Show, Show LSMClosedExn
Typeable LSMClosedExn
(Typeable LSMClosedExn, Show LSMClosedExn) =>
(LSMClosedExn -> SomeException)
-> (SomeException -> Maybe LSMClosedExn)
-> (LSMClosedExn -> String)
-> (LSMClosedExn -> Bool)
-> Exception LSMClosedExn
SomeException -> Maybe LSMClosedExn
LSMClosedExn -> Bool
LSMClosedExn -> String
LSMClosedExn -> SomeException
forall e.
(Typeable e, Show e) =>
(e -> SomeException)
-> (SomeException -> Maybe e)
-> (e -> String)
-> (e -> Bool)
-> Exception e
$ctoException :: LSMClosedExn -> SomeException
toException :: LSMClosedExn -> SomeException
$cfromException :: SomeException -> Maybe LSMClosedExn
fromException :: SomeException -> Maybe LSMClosedExn
$cdisplayException :: LSMClosedExn -> String
displayException :: LSMClosedExn -> String
$cbacktraceDesired :: LSMClosedExn -> Bool
backtraceDesired :: LSMClosedExn -> Bool
Exception)

{-------------------------------------------------------------------------------
  TxOuts
-------------------------------------------------------------------------------}

newtype TxOutBytes = TxOutBytes {TxOutBytes -> RawBytes
unTxOutBytes :: LSM.RawBytes}

toTxOutBytes :: IndexedMemPack (l EmptyMK) (TxOut l) => l EmptyMK -> TxOut l -> TxOutBytes
toTxOutBytes :: forall (l :: (* -> * -> *) -> *).
IndexedMemPack (l EmptyMK) (TxOut l) =>
l EmptyMK -> TxOut l -> TxOutBytes
toTxOutBytes l EmptyMK
st TxOut l
txout =
  let barr :: ByteArray
barr = Bool -> l EmptyMK -> TxOut l -> ByteArray
forall a idx.
(IndexedMemPack idx a, HasCallStack) =>
Bool -> idx -> a -> ByteArray
indexedPackByteArray Bool
True l EmptyMK
st TxOut l
txout
   in RawBytes -> TxOutBytes
TxOutBytes (RawBytes -> TxOutBytes) -> RawBytes -> TxOutBytes
forall a b. (a -> b) -> a -> b
$ Vector Word8 -> RawBytes
LSM.RawBytes (Int -> Int -> ByteArray -> Vector Word8
forall a. Int -> Int -> ByteArray -> Vector a
VP.Vector Int
0 (ByteArray -> Int
PBA.sizeofByteArray ByteArray
barr) ByteArray
barr)

fromTxOutBytes :: IndexedMemPack (l EmptyMK) (TxOut l) => l EmptyMK -> TxOutBytes -> TxOut l
fromTxOutBytes :: forall (l :: (* -> * -> *) -> *).
IndexedMemPack (l EmptyMK) (TxOut l) =>
l EmptyMK -> TxOutBytes -> TxOut l
fromTxOutBytes l EmptyMK
st (TxOutBytes (LSM.RawBytes Vector Word8
vec)) =
  case l EmptyMK -> Vector Word8 -> Either SomeError (TxOut l)
forall idx a b.
(IndexedMemPack idx a, Buffer b, HasCallStack) =>
idx -> b -> Either SomeError a
indexedUnpackEither l EmptyMK
st Vector Word8
vec of
    Left SomeError
err ->
      String -> TxOut l
forall a. HasCallStack => String -> a
error (String -> TxOut l) -> String -> TxOut l
forall a b. (a -> b) -> a -> b
$
        Context -> String
unlines
          [ String
"There was an error deserializing a TxOut from the LSM backend."
          , String
"This will likely result in a restart-crash loop."
          , String
"The error: " String -> ShowS
forall a. Semigroup a => a -> a -> a
<> SomeError -> String
forall a. Show a => a -> String
show SomeError
err
          ]
    Right TxOut l
v -> TxOut l
v

instance LSM.SerialiseValue TxOutBytes where
  serialiseValue :: TxOutBytes -> RawBytes
serialiseValue = TxOutBytes -> RawBytes
unTxOutBytes
  deserialiseValue :: RawBytes -> TxOutBytes
deserialiseValue = RawBytes -> TxOutBytes
TxOutBytes

deriving via LSM.ResolveAsFirst TxOutBytes instance LSM.ResolveValue TxOutBytes

{-------------------------------------------------------------------------------
  TxIns
-------------------------------------------------------------------------------}

newtype TxInBytes = TxInBytes {TxInBytes -> RawBytes
unTxInBytes :: LSM.RawBytes}

toTxInBytes :: MemPack (TxIn l) => Proxy l -> TxIn l -> TxInBytes
toTxInBytes :: forall (l :: (* -> * -> *) -> *).
MemPack (TxIn l) =>
Proxy l -> TxIn l -> TxInBytes
toTxInBytes Proxy l
_ TxIn l
txin =
  let barr :: ByteArray
barr = Bool -> TxIn l -> ByteArray
forall a. (MemPack a, HasCallStack) => Bool -> a -> ByteArray
packByteArray Bool
True TxIn l
txin
   in RawBytes -> TxInBytes
TxInBytes (RawBytes -> TxInBytes) -> RawBytes -> TxInBytes
forall a b. (a -> b) -> a -> b
$ Vector Word8 -> RawBytes
LSM.RawBytes (Int -> Int -> ByteArray -> Vector Word8
forall a. Int -> Int -> ByteArray -> Vector a
VP.Vector Int
0 (ByteArray -> Int
PBA.sizeofByteArray ByteArray
barr) ByteArray
barr)

fromTxInBytes :: MemPack (TxIn l) => Proxy l -> TxInBytes -> TxIn l
fromTxInBytes :: forall (l :: (* -> * -> *) -> *).
MemPack (TxIn l) =>
Proxy l -> TxInBytes -> TxIn l
fromTxInBytes Proxy l
_ (TxInBytes (LSM.RawBytes Vector Word8
vec)) =
  case Vector Word8 -> Either SomeError (TxIn l)
forall a b.
(MemPack a, Buffer b, HasCallStack) =>
b -> Either SomeError a
unpackEither Vector Word8
vec of
    Left SomeError
err ->
      String -> TxIn l
forall a. HasCallStack => String -> a
error (String -> TxIn l) -> String -> TxIn l
forall a b. (a -> b) -> a -> b
$
        Context -> String
unlines
          [ String
"There was an error deserializing a TxIn from the LSM backend."
          , String
"This will likely result in a restart-crash loop."
          , String
"The error: " String -> ShowS
forall a. Semigroup a => a -> a -> a
<> SomeError -> String
forall a. Show a => a -> String
show SomeError
err
          ]
    Right TxIn l
v -> TxIn l
v

instance LSM.SerialiseKey TxInBytes where
  serialiseKey :: TxInBytes -> RawBytes
serialiseKey = TxInBytes -> RawBytes
unTxInBytes
  deserialiseKey :: RawBytes -> TxInBytes
deserialiseKey = RawBytes -> TxInBytes
TxInBytes

{-------------------------------------------------------------------------------
  LedgerTablesHandle
-------------------------------------------------------------------------------}

newLSMLedgerTablesHandle ::
  forall m l.
  ( IOLike m
  , HasLedgerTables l
  , IndexedMemPack (l EmptyMK) (TxOut l)
  ) =>
  Tracer m LedgerDBV2Trace ->
  ResourceRegistry m ->
  (ResourceKey m, UTxOTable m) ->
  m (LedgerTablesHandle m l)
newLSMLedgerTablesHandle :: forall (m :: * -> *) (l :: (* -> * -> *) -> *).
(IOLike m, HasLedgerTables l,
 IndexedMemPack (l EmptyMK) (TxOut l)) =>
Tracer m LedgerDBV2Trace
-> ResourceRegistry m
-> (ResourceKey m, UTxOTable m)
-> m (LedgerTablesHandle m l)
newLSMLedgerTablesHandle Tracer m LedgerDBV2Trace
tracer ResourceRegistry m
rr (ResourceKey m
resKey, UTxOTable m
t) = do
  Tracer m LedgerDBV2Trace -> LedgerDBV2Trace -> m ()
forall (m :: * -> *) a. Tracer m a -> a -> m ()
traceWith Tracer m LedgerDBV2Trace
tracer LedgerDBV2Trace
TraceLedgerTablesHandleCreate
  LedgerTablesHandle m l -> m (LedgerTablesHandle m l)
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure
    LedgerTablesHandle
      { close :: m ()
close = ResourceKey m -> m ()
forall (m :: * -> *). IOLike m => ResourceKey m -> m ()
implClose ResourceKey m
resKey
      , duplicate :: m (LedgerTablesHandle m l)
duplicate = ResourceRegistry m
-> UTxOTable m
-> Tracer m LedgerDBV2Trace
-> m (LedgerTablesHandle m l)
forall (m :: * -> *) (l :: (* -> * -> *) -> *).
(IOLike m, HasLedgerTables l,
 IndexedMemPack (l EmptyMK) (TxOut l)) =>
ResourceRegistry m
-> UTxOTable m
-> Tracer m LedgerDBV2Trace
-> m (LedgerTablesHandle m l)
implDuplicate ResourceRegistry m
rr UTxOTable m
t Tracer m LedgerDBV2Trace
tracer
      , read :: l EmptyMK -> LedgerTables l KeysMK -> m (LedgerTables l ValuesMK)
read = UTxOTable m
-> l EmptyMK
-> LedgerTables l KeysMK
-> m (LedgerTables l ValuesMK)
forall (m :: * -> *) (l :: (* -> * -> *) -> *).
(IOLike m, HasLedgerTables l,
 IndexedMemPack (l EmptyMK) (TxOut l)) =>
UTxOTable m
-> l EmptyMK
-> LedgerTables l KeysMK
-> m (LedgerTables l ValuesMK)
implRead UTxOTable m
t
      , readRange :: l EmptyMK
-> (Maybe (TxIn l), Int)
-> m (LedgerTables l ValuesMK, Maybe (TxIn l))
readRange = UTxOTable m
-> l EmptyMK
-> (Maybe (TxIn l), Int)
-> m (LedgerTables l ValuesMK, Maybe (TxIn l))
forall (m :: * -> *) (l :: (* -> * -> *) -> *).
(IOLike m, IndexedMemPack (l EmptyMK) (TxOut l),
 HasLedgerTables l) =>
UTxOTable m
-> l EmptyMK
-> (Maybe (TxIn l), Int)
-> m (LedgerTables l ValuesMK, Maybe (TxIn l))
implReadRange UTxOTable m
t
      , readAll :: l EmptyMK -> m (LedgerTables l ValuesMK)
readAll = UTxOTable m -> l EmptyMK -> m (LedgerTables l ValuesMK)
forall (m :: * -> *) (l :: (* -> * -> *) -> *).
(IOLike m, HasLedgerTables l,
 IndexedMemPack (l EmptyMK) (TxOut l)) =>
UTxOTable m -> l EmptyMK -> m (LedgerTables l ValuesMK)
implReadAll UTxOTable m
t
      , pushDiffs :: forall (mk :: * -> * -> *). l mk -> l DiffMK -> m ()
pushDiffs = UTxOTable m -> l mk -> l DiffMK -> m ()
forall (m :: * -> *) (l :: (* -> * -> *) -> *) (mk :: * -> * -> *).
(IOLike m, HasLedgerTables l,
 IndexedMemPack (l EmptyMK) (TxOut l)) =>
UTxOTable m -> l mk -> l DiffMK -> m ()
implPushDiffs UTxOTable m
t
      , takeHandleSnapshot :: l EmptyMK -> String -> m (Maybe CRC)
takeHandleSnapshot = UTxOTable m -> l EmptyMK -> String -> m (Maybe CRC)
forall (m :: * -> *) t a.
IOLike m =>
UTxOTable m -> t -> String -> m (Maybe a)
implTakeHandleSnapshot UTxOTable m
t
      , tablesSize :: m (Maybe Int)
tablesSize = Maybe Int -> m (Maybe Int)
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure Maybe Int
forall a. Maybe a
Nothing
      }

{-# INLINE implClose #-}
{-# INLINE implDuplicate #-}
{-# INLINE implRead #-}
{-# INLINE implReadRange #-}
{-# INLINE implReadAll #-}
{-# INLINE implPushDiffs #-}
{-# INLINE implTakeHandleSnapshot #-}

implClose :: IOLike m => ResourceKey m -> m ()
implClose :: forall (m :: * -> *). IOLike m => ResourceKey m -> m ()
implClose = m (Maybe (Context m)) -> m ()
forall (f :: * -> *) a. Functor f => f a -> f ()
Monad.void (m (Maybe (Context m)) -> m ())
-> (ResourceKey m -> m (Maybe (Context m)))
-> ResourceKey m
-> m ()
forall b c a. (b -> c) -> (a -> b) -> a -> c
. ResourceKey m -> m (Maybe (Context m))
forall (m :: * -> *).
(MonadMask m, MonadSTM m, MonadThread m, HasCallStack) =>
ResourceKey m -> m (Maybe (Context m))
release

implDuplicate ::
  ( IOLike m
  , HasLedgerTables l
  , IndexedMemPack (l EmptyMK) (TxOut l)
  ) =>
  ResourceRegistry m ->
  UTxOTable m ->
  Tracer m LedgerDBV2Trace ->
  m (LedgerTablesHandle m l)
implDuplicate :: forall (m :: * -> *) (l :: (* -> * -> *) -> *).
(IOLike m, HasLedgerTables l,
 IndexedMemPack (l EmptyMK) (TxOut l)) =>
ResourceRegistry m
-> UTxOTable m
-> Tracer m LedgerDBV2Trace
-> m (LedgerTablesHandle m l)
implDuplicate ResourceRegistry m
rr UTxOTable m
t Tracer m LedgerDBV2Trace
tracer = do
  table <-
    ResourceRegistry m
-> (ResourceId -> m (UTxOTable m))
-> (UTxOTable m -> m ())
-> m (ResourceKey m, UTxOTable m)
forall (m :: * -> *) a.
(MonadSTM m, MonadMask m, MonadThread m, HasCallStack) =>
ResourceRegistry m
-> (ResourceId -> m a) -> (a -> m ()) -> m (ResourceKey m, a)
allocate
      ResourceRegistry m
rr
      (\ResourceId
_ -> UTxOTable m -> m (UTxOTable m)
forall (m :: * -> *) k v b.
IOLike m =>
Table m k v b -> m (Table m k v b)
LSM.duplicate UTxOTable m
t)
      ( \UTxOTable m
t' -> do
          Tracer m LedgerDBV2Trace -> LedgerDBV2Trace -> m ()
forall (m :: * -> *) a. Tracer m a -> a -> m ()
traceWith Tracer m LedgerDBV2Trace
tracer LedgerDBV2Trace
TraceLedgerTablesHandleClose
          UTxOTable m -> m ()
forall (m :: * -> *) k v b. IOLike m => Table m k v b -> m ()
LSM.closeTable UTxOTable m
t'
      )
  newLSMLedgerTablesHandle tracer rr table

implRead ::
  forall m l.
  ( IOLike m
  , HasLedgerTables l
  , IndexedMemPack (l EmptyMK) (TxOut l)
  ) =>
  UTxOTable m -> l EmptyMK -> LedgerTables l KeysMK -> m (LedgerTables l ValuesMK)
implRead :: forall (m :: * -> *) (l :: (* -> * -> *) -> *).
(IOLike m, HasLedgerTables l,
 IndexedMemPack (l EmptyMK) (TxOut l)) =>
UTxOTable m
-> l EmptyMK
-> LedgerTables l KeysMK
-> m (LedgerTables l ValuesMK)
implRead UTxOTable m
t l EmptyMK
st (LedgerTables (KeysMK Set (TxIn l)
keys)) = do
  let vec' :: Vector TxInBytes
vec' = (forall s. ST s (MVector s TxInBytes)) -> Vector TxInBytes
forall a. (forall s. ST s (MVector s a)) -> Vector a
V.create ((forall s. ST s (MVector s TxInBytes)) -> Vector TxInBytes)
-> (forall s. ST s (MVector s TxInBytes)) -> Vector TxInBytes
forall a b. (a -> b) -> a -> b
$ do
        vec <- Int -> ST s (MVector (PrimState (ST s)) TxInBytes)
forall (m :: * -> *) a.
PrimMonad m =>
Int -> m (MVector (PrimState m) a)
VM.new (Set (TxIn l) -> Int
forall a. Set a -> Int
Set.size Set (TxIn l)
keys)
        Monad.foldM_
          (\Int
i TxIn l
x -> MVector (PrimState (ST s)) TxInBytes -> Int -> TxInBytes -> ST s ()
forall (m :: * -> *) a.
PrimMonad m =>
MVector (PrimState m) a -> Int -> a -> m ()
VM.write MVector s TxInBytes
MVector (PrimState (ST s)) TxInBytes
vec Int
i (Proxy l -> TxIn l -> TxInBytes
forall (l :: (* -> * -> *) -> *).
MemPack (TxIn l) =>
Proxy l -> TxIn l -> TxInBytes
toTxInBytes (forall {k} (t :: k). Proxy t
forall (t :: (* -> * -> *) -> *). Proxy t
Proxy @l) TxIn l
x) ST s () -> ST s Int -> ST s Int
forall a b. ST s a -> ST s b -> ST s b
forall (m :: * -> *) a b. Monad m => m a -> m b -> m b
>> Int -> ST s Int
forall a. a -> ST s a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (Int
i Int -> Int -> Int
forall a. Num a => a -> a -> a
+ Int
1))
          0
          keys
        pure vec
  res <- UTxOTable m
-> Vector TxInBytes
-> m (Vector (LookupResult TxOutBytes (BlobRef m Void)))
forall (m :: * -> *) k v b.
(IOLike m, SerialiseKey k, SerialiseValue v, ResolveValue v) =>
Table m k v b
-> Vector k -> m (Vector (LookupResult v (BlobRef m b)))
LSM.lookups UTxOTable m
t Vector TxInBytes
vec'
  pure
    . LedgerTables
    . ValuesMK
    . Foldable.foldl'
      ( \Map (TxIn l) (TxOut l)
m (TxInBytes
k, LookupResult TxOutBytes (BlobRef m Void)
item) ->
          case LookupResult TxOutBytes (BlobRef m Void)
item of
            LSM.Found TxOutBytes
v -> 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 (Proxy l -> TxInBytes -> TxIn l
forall (l :: (* -> * -> *) -> *).
MemPack (TxIn l) =>
Proxy l -> TxInBytes -> TxIn l
fromTxInBytes (forall {k} (t :: k). Proxy t
forall (t :: (* -> * -> *) -> *). Proxy t
Proxy @l) TxInBytes
k) (l EmptyMK -> TxOutBytes -> TxOut l
forall (l :: (* -> * -> *) -> *).
IndexedMemPack (l EmptyMK) (TxOut l) =>
l EmptyMK -> TxOutBytes -> TxOut l
fromTxOutBytes l EmptyMK
st TxOutBytes
v) Map (TxIn l) (TxOut l)
m
            LookupResult TxOutBytes (BlobRef m Void)
LSM.NotFound -> Map (TxIn l) (TxOut l)
m
            LSM.FoundWithBlob{} -> Map (TxIn l) (TxOut l)
m
      )
      Map.empty
    $ V.zip vec' res

implReadRange ::
  forall m l.
  (IOLike m, IndexedMemPack (l EmptyMK) (TxOut l)) =>
  HasLedgerTables l =>
  UTxOTable m ->
  l EmptyMK ->
  (Maybe (TxIn l), Int) ->
  m (LedgerTables l ValuesMK, Maybe (TxIn l))
implReadRange :: forall (m :: * -> *) (l :: (* -> * -> *) -> *).
(IOLike m, IndexedMemPack (l EmptyMK) (TxOut l),
 HasLedgerTables l) =>
UTxOTable m
-> l EmptyMK
-> (Maybe (TxIn l), Int)
-> m (LedgerTables l ValuesMK, Maybe (TxIn l))
implReadRange UTxOTable m
table l EmptyMK
st (Maybe (TxIn l)
mPrev, Int
num) = do
  entries <- m (Vector (Entry TxInBytes TxOutBytes (BlobRef m Void)))
-> (TxIn l
    -> m (Vector (Entry TxInBytes TxOutBytes (BlobRef m Void))))
-> Maybe (TxIn l)
-> m (Vector (Entry TxInBytes TxOutBytes (BlobRef m Void)))
forall b a. b -> (a -> b) -> Maybe a -> b
maybe m (Vector (Entry TxInBytes TxOutBytes (BlobRef m Void)))
cursorFromStart TxIn l -> m (Vector (Entry TxInBytes TxOutBytes (BlobRef m Void)))
cursorFromKey Maybe (TxIn l)
mPrev
  pure
    ( LedgerTables
        . ValuesMK
        . V.foldl'
          ( \Map (TxIn l) (TxOut l)
m -> \case
              LSM.Entry TxInBytes
k TxOutBytes
v -> 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 (Proxy l -> TxInBytes -> TxIn l
forall (l :: (* -> * -> *) -> *).
MemPack (TxIn l) =>
Proxy l -> TxInBytes -> TxIn l
fromTxInBytes (forall {k} (t :: k). Proxy t
forall (t :: (* -> * -> *) -> *). Proxy t
Proxy @l) TxInBytes
k) (l EmptyMK -> TxOutBytes -> TxOut l
forall (l :: (* -> * -> *) -> *).
IndexedMemPack (l EmptyMK) (TxOut l) =>
l EmptyMK -> TxOutBytes -> TxOut l
fromTxOutBytes l EmptyMK
st TxOutBytes
v) Map (TxIn l) (TxOut l)
m
              LSM.EntryWithBlob{} -> Map (TxIn l) (TxOut l)
m
          )
          Map.empty
        $ entries
    , case snd <$> V.unsnoc entries of
        Maybe (Entry TxInBytes TxOutBytes (BlobRef m Void))
Nothing -> Maybe (TxIn l)
forall a. Maybe a
Nothing
        Just (LSM.Entry TxInBytes
k TxOutBytes
_) -> TxIn l -> Maybe (TxIn l)
forall a. a -> Maybe a
Just (Proxy l -> TxInBytes -> TxIn l
forall (l :: (* -> * -> *) -> *).
MemPack (TxIn l) =>
Proxy l -> TxInBytes -> TxIn l
fromTxInBytes (forall {k} (t :: k). Proxy t
forall (t :: (* -> * -> *) -> *). Proxy t
Proxy @l) TxInBytes
k)
        Just (LSM.EntryWithBlob TxInBytes
k TxOutBytes
_ BlobRef m Void
_) -> TxIn l -> Maybe (TxIn l)
forall a. a -> Maybe a
Just (Proxy l -> TxInBytes -> TxIn l
forall (l :: (* -> * -> *) -> *).
MemPack (TxIn l) =>
Proxy l -> TxInBytes -> TxIn l
fromTxInBytes (forall {k} (t :: k). Proxy t
forall (t :: (* -> * -> *) -> *). Proxy t
Proxy @l) TxInBytes
k)
    )
 where
  cursorFromStart :: m (Vector (Entry TxInBytes TxOutBytes (BlobRef m Void)))
cursorFromStart = UTxOTable m
-> (Cursor m TxInBytes TxOutBytes Void
    -> m (Vector (Entry TxInBytes TxOutBytes (BlobRef m Void))))
-> m (Vector (Entry TxInBytes TxOutBytes (BlobRef m Void)))
forall (m :: * -> *) k v b a.
(IOLike m, ResolveValue v) =>
Table m k v b -> (Cursor m k v b -> m a) -> m a
LSM.withCursor UTxOTable m
table (Int
-> Cursor m TxInBytes TxOutBytes Void
-> m (Vector (Entry TxInBytes TxOutBytes (BlobRef m Void)))
forall (m :: * -> *) k v b.
(IOLike m, SerialiseKey k, SerialiseValue v, ResolveValue v) =>
Int -> Cursor m k v b -> m (Vector (Entry k v (BlobRef m b)))
LSM.take Int
num)
  -- Here we ask for one value more and we drop one value because the
  -- cursor returns also the key at which it was opened.
  cursorFromKey :: TxIn l -> m (Vector (Entry TxInBytes TxOutBytes (BlobRef m Void)))
cursorFromKey TxIn l
k = (Vector (Entry TxInBytes TxOutBytes (BlobRef m Void))
 -> Vector (Entry TxInBytes TxOutBytes (BlobRef m Void)))
-> m (Vector (Entry TxInBytes TxOutBytes (BlobRef m Void)))
-> m (Vector (Entry TxInBytes TxOutBytes (BlobRef m Void)))
forall a b. (a -> b) -> m a -> m b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap (Int
-> Vector (Entry TxInBytes TxOutBytes (BlobRef m Void))
-> Vector (Entry TxInBytes TxOutBytes (BlobRef m Void))
forall a. Int -> Vector a -> Vector a
V.drop Int
1) (m (Vector (Entry TxInBytes TxOutBytes (BlobRef m Void)))
 -> m (Vector (Entry TxInBytes TxOutBytes (BlobRef m Void))))
-> m (Vector (Entry TxInBytes TxOutBytes (BlobRef m Void)))
-> m (Vector (Entry TxInBytes TxOutBytes (BlobRef m Void)))
forall a b. (a -> b) -> a -> b
$ UTxOTable m
-> TxInBytes
-> (Cursor m TxInBytes TxOutBytes Void
    -> m (Vector (Entry TxInBytes TxOutBytes (BlobRef m Void))))
-> m (Vector (Entry TxInBytes TxOutBytes (BlobRef m Void)))
forall (m :: * -> *) k v b a.
(IOLike m, SerialiseKey k, ResolveValue v) =>
Table m k v b -> k -> (Cursor m k v b -> m a) -> m a
LSM.withCursorAtOffset UTxOTable m
table (Proxy l -> TxIn l -> TxInBytes
forall (l :: (* -> * -> *) -> *).
MemPack (TxIn l) =>
Proxy l -> TxIn l -> TxInBytes
toTxInBytes (forall {k} (t :: k). Proxy t
forall (t :: (* -> * -> *) -> *). Proxy t
Proxy @l) TxIn l
k) (Int
-> Cursor m TxInBytes TxOutBytes Void
-> m (Vector (Entry TxInBytes TxOutBytes (BlobRef m Void)))
forall (m :: * -> *) k v b.
(IOLike m, SerialiseKey k, SerialiseValue v, ResolveValue v) =>
Int -> Cursor m k v b -> m (Vector (Entry k v (BlobRef m b)))
LSM.take (Int
 -> Cursor m TxInBytes TxOutBytes Void
 -> m (Vector (Entry TxInBytes TxOutBytes (BlobRef m Void))))
-> Int
-> Cursor m TxInBytes TxOutBytes Void
-> m (Vector (Entry TxInBytes TxOutBytes (BlobRef m Void)))
forall a b. (a -> b) -> a -> b
$ Int
num Int -> Int -> Int
forall a. Num a => a -> a -> a
+ Int
1)

implReadAll ::
  ( IOLike m
  , HasLedgerTables l
  , IndexedMemPack (l EmptyMK) (TxOut l)
  ) =>
  UTxOTable m ->
  l EmptyMK ->
  m (LedgerTables l ValuesMK)
implReadAll :: forall (m :: * -> *) (l :: (* -> * -> *) -> *).
(IOLike m, HasLedgerTables l,
 IndexedMemPack (l EmptyMK) (TxOut l)) =>
UTxOTable m -> l EmptyMK -> m (LedgerTables l ValuesMK)
implReadAll UTxOTable m
t l EmptyMK
st =
  let readAll' :: Maybe (TxIn l) -> m (LedgerTables l ValuesMK)
readAll' Maybe (TxIn l)
m = do
        (v, n) <- UTxOTable m
-> l EmptyMK
-> (Maybe (TxIn l), Int)
-> m (LedgerTables l ValuesMK, Maybe (TxIn l))
forall (m :: * -> *) (l :: (* -> * -> *) -> *).
(IOLike m, IndexedMemPack (l EmptyMK) (TxOut l),
 HasLedgerTables l) =>
UTxOTable m
-> l EmptyMK
-> (Maybe (TxIn l), Int)
-> m (LedgerTables l ValuesMK, Maybe (TxIn l))
implReadRange UTxOTable m
t l EmptyMK
st (Maybe (TxIn l)
m, Int
100000)
        maybe (pure v) (fmap (ltliftA2 unionValues v) . readAll' . Just) n
   in Maybe (TxIn l) -> m (LedgerTables l ValuesMK)
readAll' Maybe (TxIn l)
forall a. Maybe a
Nothing

implPushDiffs ::
  forall m l mk.
  ( IOLike m
  , HasLedgerTables l
  , IndexedMemPack (l EmptyMK) (TxOut l)
  ) =>
  UTxOTable m -> l mk -> l DiffMK -> m ()
implPushDiffs :: forall (m :: * -> *) (l :: (* -> * -> *) -> *) (mk :: * -> * -> *).
(IOLike m, HasLedgerTables l,
 IndexedMemPack (l EmptyMK) (TxOut l)) =>
UTxOTable m -> l mk -> l DiffMK -> m ()
implPushDiffs UTxOTable m
t l mk
_ !l DiffMK
st1 = do
  let LedgerTables (DiffMK (Diff.Diff Map (TxIn l) (Delta (TxOut l))
diffs)) = l DiffMK -> LedgerTables l DiffMK
forall (mk :: * -> * -> *).
(CanMapMK mk, CanMapKeysMK mk, ZeroableMK mk) =>
l mk -> LedgerTables l mk
forall (l :: (* -> * -> *) -> *) (mk :: * -> * -> *).
(HasLedgerTables l, CanMapMK mk, CanMapKeysMK mk, ZeroableMK mk) =>
l mk -> LedgerTables l mk
projectLedgerTables l DiffMK
st1
  let vec :: Vector (TxInBytes, Update TxOutBytes Void)
vec = (forall s. ST s (MVector s (TxInBytes, Update TxOutBytes Void)))
-> Vector (TxInBytes, Update TxOutBytes Void)
forall a. (forall s. ST s (MVector s a)) -> Vector a
V.create ((forall s. ST s (MVector s (TxInBytes, Update TxOutBytes Void)))
 -> Vector (TxInBytes, Update TxOutBytes Void))
-> (forall s. ST s (MVector s (TxInBytes, Update TxOutBytes Void)))
-> Vector (TxInBytes, Update TxOutBytes Void)
forall a b. (a -> b) -> a -> b
$ do
        vec' <- Int
-> ST
     s (MVector (PrimState (ST s)) (TxInBytes, Update TxOutBytes Void))
forall (m :: * -> *) a.
PrimMonad m =>
Int -> m (MVector (PrimState m) a)
VM.new (Map (TxIn l) (Delta (TxOut l)) -> Int
forall k a. Map k a -> Int
Map.size Map (TxIn l) (Delta (TxOut l))
diffs)
        Monad.foldM_
          (\Int
idx (TxIn l
k, Delta (TxOut l)
item) -> MVector (PrimState (ST s)) (TxInBytes, Update TxOutBytes Void)
-> Int -> (TxInBytes, Update TxOutBytes Void) -> ST s ()
forall (m :: * -> *) a.
PrimMonad m =>
MVector (PrimState m) a -> Int -> a -> m ()
VM.write MVector s (TxInBytes, Update TxOutBytes Void)
MVector (PrimState (ST s)) (TxInBytes, Update TxOutBytes Void)
vec' Int
idx (Proxy l -> TxIn l -> TxInBytes
forall (l :: (* -> * -> *) -> *).
MemPack (TxIn l) =>
Proxy l -> TxIn l -> TxInBytes
toTxInBytes (forall {k} (t :: k). Proxy t
forall (t :: (* -> * -> *) -> *). Proxy t
Proxy @l) TxIn l
k, (Delta (TxOut l) -> Update TxOutBytes Void
f Delta (TxOut l)
item)) ST s () -> ST s Int -> ST s Int
forall a b. ST s a -> ST s b -> ST s b
forall (m :: * -> *) a b. Monad m => m a -> m b -> m b
>> Int -> ST s Int
forall a. a -> ST s a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (Int
idx Int -> Int -> Int
forall a. Num a => a -> a -> a
+ Int
1))
          0
          $ Map.toList diffs
        pure vec'
  UTxOTable m -> Vector (TxInBytes, Update TxOutBytes Void) -> m ()
forall (m :: * -> *) k v b.
(IOLike m, SerialiseKey k, SerialiseValue v, ResolveValue v,
 SerialiseValue b) =>
Table m k v b -> Vector (k, Update v b) -> m ()
LSM.updates UTxOTable m
t Vector (TxInBytes, Update TxOutBytes Void)
vec
 where
  f :: Delta (TxOut l) -> Update TxOutBytes Void
f (Diff.Insert TxOut l
v) = TxOutBytes -> Maybe Void -> Update TxOutBytes Void
forall v b. v -> Maybe b -> Update v b
LSM.Insert (l EmptyMK -> TxOut l -> TxOutBytes
forall (l :: (* -> * -> *) -> *).
IndexedMemPack (l EmptyMK) (TxOut l) =>
l EmptyMK -> TxOut l -> TxOutBytes
toTxOutBytes (l DiffMK -> l EmptyMK
forall (l :: (* -> * -> *) -> *) (mk :: * -> * -> *).
HasLedgerTables l =>
l mk -> l EmptyMK
forgetLedgerTables l DiffMK
st1) TxOut l
v) Maybe Void
forall a. Maybe a
Nothing
  f Delta (TxOut l)
Diff.Delete = Update TxOutBytes Void
forall v b. Update v b
LSM.Delete

implTakeHandleSnapshot :: IOLike m => UTxOTable m -> t -> String -> m (Maybe a)
implTakeHandleSnapshot :: forall (m :: * -> *) t a.
IOLike m =>
UTxOTable m -> t -> String -> m (Maybe a)
implTakeHandleSnapshot UTxOTable m
t t
_ String
snapshotName = do
  SnapshotName -> SnapshotLabel -> UTxOTable m -> m ()
forall (m :: * -> *) k v b.
IOLike m =>
SnapshotName -> SnapshotLabel -> Table m k v b -> m ()
LSM.saveSnapshot
    (String -> SnapshotName
forall a. IsString a => String -> a
fromString String
snapshotName)
    (Text -> SnapshotLabel
LSM.SnapshotLabel (Text -> SnapshotLabel) -> Text -> SnapshotLabel
forall a b. (a -> b) -> a -> b
$ String -> Text
Text.pack (String -> Text) -> String -> Text
forall a b. (a -> b) -> a -> b
$ String
"UTxO table")
    UTxOTable m
t
  Maybe a -> m (Maybe a)
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure Maybe a
forall a. Maybe a
Nothing

{-------------------------------------------------------------------------------
  SnapshotManager
-------------------------------------------------------------------------------}

-- | Snapshots in LSM trees are split in two parts for now:
--
-- - The @state@ and @meta@ files in the usual location (@./ledger/<slotno>@ in
--   the ChainDB).
--
-- - The ledger tables, which are stored in the LSM-trees session directory,
--   under a @./lsm/snapshots/<slotno>@ directory.
--
-- Note that the name of the folder in which the @state@ file is and the name of
-- the snapshot in the LSM-trees directory have to match. This means that if the
-- user adds a suffix to the snapshot renaming the directory
-- @./ledger/<slotno>@, they will also have to rename the directory
-- @./lsm/snapshots/<slotno>@. Otherwise the initialization logic will exit with
-- failure saying that the snapshot was not found.
--
-- There is [an issue open in
-- LSM-trees](https://github.com/IntersectMBO/lsm-tree/issues/272) such that the
-- ledger tables part of the snapshot could also be stored in the
-- @./ledger/<slotno>@ directory, but it is not implemented yet.
snapshotManager ::
  ( IOLike m
  , LedgerDbSerialiseConstraints blk
  , LedgerSupportsProtocol blk
  ) =>
  Session m ->
  CodecConfig blk ->
  Tracer m (TraceSnapshotEvent blk) ->
  SomeHasFS m ->
  SnapshotManager m m blk (StateRef m (ExtLedgerState blk))
snapshotManager :: forall (m :: * -> *) blk.
(IOLike m, LedgerDbSerialiseConstraints blk,
 LedgerSupportsProtocol blk) =>
Session m
-> CodecConfig blk
-> Tracer m (TraceSnapshotEvent blk)
-> SomeHasFS m
-> SnapshotManager m m blk (StateRef m (ExtLedgerState blk))
snapshotManager Session m
session CodecConfig blk
ccfg Tracer m (TraceSnapshotEvent blk)
tracer SomeHasFS m
fs =
  SnapshotManager
    { listSnapshots :: m [DiskSnapshot]
listSnapshots = SomeHasFS m -> m [DiskSnapshot]
forall (m :: * -> *). Monad m => SomeHasFS m -> m [DiskSnapshot]
defaultListSnapshots SomeHasFS m
fs
    , deleteSnapshot :: DiskSnapshot -> m ()
deleteSnapshot = Session m
-> SomeHasFS m
-> Tracer m (TraceSnapshotEvent blk)
-> DiskSnapshot
-> m ()
forall (m :: * -> *) blk.
IOLike m =>
Session m
-> SomeHasFS m
-> Tracer m (TraceSnapshotEvent blk)
-> DiskSnapshot
-> m ()
implDeleteSnapshot Session m
session SomeHasFS m
fs Tracer m (TraceSnapshotEvent blk)
tracer
    , takeSnapshot :: Maybe String
-> StateRef m (ExtLedgerState blk)
-> m (Maybe (DiskSnapshot, RealPoint blk))
takeSnapshot = CodecConfig blk
-> Tracer m (TraceSnapshotEvent blk)
-> SomeHasFS m
-> Maybe String
-> StateRef m (ExtLedgerState blk)
-> m (Maybe (DiskSnapshot, RealPoint blk))
forall (m :: * -> *) blk.
(IOLike m, LedgerDbSerialiseConstraints blk,
 LedgerSupportsProtocol blk) =>
CodecConfig blk
-> Tracer m (TraceSnapshotEvent blk)
-> SomeHasFS m
-> Maybe String
-> StateRef m (ExtLedgerState blk)
-> m (Maybe (DiskSnapshot, RealPoint blk))
implTakeSnapshot CodecConfig blk
ccfg Tracer m (TraceSnapshotEvent blk)
tracer SomeHasFS m
fs
    }

{-# INLINE implTakeSnapshot #-}
{-# INLINE implDeleteSnapshot #-}

implTakeSnapshot ::
  ( IOLike m
  , LedgerDbSerialiseConstraints blk
  , LedgerSupportsProtocol blk
  ) =>
  CodecConfig blk ->
  Tracer m (TraceSnapshotEvent blk) ->
  SomeHasFS m ->
  Maybe String ->
  StateRef m (ExtLedgerState blk) ->
  m (Maybe (DiskSnapshot, RealPoint blk))
implTakeSnapshot :: forall (m :: * -> *) blk.
(IOLike m, LedgerDbSerialiseConstraints blk,
 LedgerSupportsProtocol blk) =>
CodecConfig blk
-> Tracer m (TraceSnapshotEvent blk)
-> SomeHasFS m
-> Maybe String
-> StateRef m (ExtLedgerState blk)
-> m (Maybe (DiskSnapshot, RealPoint blk))
implTakeSnapshot CodecConfig blk
ccfg Tracer m (TraceSnapshotEvent blk)
tracer shfs :: SomeHasFS m
shfs@(SomeHasFS HasFS m h
hasFs) Maybe String
suffix StateRef m (ExtLedgerState blk)
st =
  case Point blk -> WithOrigin (RealPoint blk)
forall blk. Point blk -> WithOrigin (RealPoint blk)
pointToWithOriginRealPoint (Point (ExtLedgerState blk) -> Point blk
forall {k1} {k2} (b :: k1) (b' :: k2).
Coercible (HeaderHash b) (HeaderHash b') =>
Point b -> Point b'
castPoint (ExtLedgerState blk EmptyMK -> Point (ExtLedgerState blk)
forall (mk :: * -> * -> *).
ExtLedgerState blk mk -> Point (ExtLedgerState blk)
forall (l :: (* -> * -> *) -> *) (mk :: * -> * -> *).
GetTip l =>
l mk -> Point l
getTip (ExtLedgerState blk EmptyMK -> Point (ExtLedgerState blk))
-> ExtLedgerState blk EmptyMK -> Point (ExtLedgerState blk)
forall a b. (a -> b) -> a -> b
$ StateRef m (ExtLedgerState blk) -> ExtLedgerState blk EmptyMK
forall (m :: * -> *) (l :: (* -> * -> *) -> *).
StateRef m l -> l EmptyMK
state StateRef m (ExtLedgerState blk)
st)) of
    WithOrigin (RealPoint blk)
Origin -> Maybe (DiskSnapshot, RealPoint blk)
-> m (Maybe (DiskSnapshot, RealPoint blk))
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return Maybe (DiskSnapshot, RealPoint blk)
forall a. Maybe a
Nothing
    NotOrigin RealPoint blk
t -> do
      let number :: Word64
number = SlotNo -> Word64
unSlotNo (RealPoint blk -> SlotNo
forall blk. RealPoint blk -> SlotNo
realPointSlot RealPoint blk
t)
          snapshot :: DiskSnapshot
snapshot = Word64 -> Maybe String -> DiskSnapshot
DiskSnapshot Word64
number Maybe String
suffix
      diskSnapshots <- SomeHasFS m -> m [DiskSnapshot]
forall (m :: * -> *). Monad m => SomeHasFS m -> m [DiskSnapshot]
defaultListSnapshots SomeHasFS m
shfs
      if List.any (== DiskSnapshot number suffix) diskSnapshots
        then
          return Nothing
        else do
          encloseTimedWith (TookSnapshot snapshot t >$< tracer) $
            writeSnapshot snapshot
          return $ Just (snapshot, t)
 where
  writeSnapshot :: DiskSnapshot -> m ()
writeSnapshot DiskSnapshot
ds = do
    HasFS m h -> HasCallStack => Bool -> FsPath -> m ()
forall (m :: * -> *) h.
HasFS m h -> HasCallStack => Bool -> FsPath -> m ()
createDirectoryIfMissing HasFS m h
hasFs Bool
True (FsPath -> m ()) -> FsPath -> m ()
forall a b. (a -> b) -> a -> b
$ DiskSnapshot -> FsPath
snapshotToDirPath DiskSnapshot
ds
    crc1 <- SomeHasFS m
-> (ExtLedgerState blk EmptyMK -> Encoding)
-> FsPath
-> ExtLedgerState blk EmptyMK
-> m CRC
forall (m :: * -> *) blk.
MonadThrow m =>
SomeHasFS m
-> (ExtLedgerState blk EmptyMK -> Encoding)
-> FsPath
-> ExtLedgerState blk EmptyMK
-> m CRC
writeExtLedgerState SomeHasFS m
shfs (CodecConfig blk -> ExtLedgerState blk EmptyMK -> Encoding
forall blk.
(EncodeDisk blk (LedgerState blk EmptyMK),
 EncodeDisk blk (ChainDepState (BlockProtocol blk)),
 EncodeDisk blk (AnnTip blk)) =>
CodecConfig blk -> ExtLedgerState blk EmptyMK -> Encoding
encodeDiskExtLedgerState CodecConfig blk
ccfg) (DiskSnapshot -> FsPath
snapshotToStatePath DiskSnapshot
ds) (ExtLedgerState blk EmptyMK -> m CRC)
-> ExtLedgerState blk EmptyMK -> m CRC
forall a b. (a -> b) -> a -> b
$ StateRef m (ExtLedgerState blk) -> ExtLedgerState blk EmptyMK
forall (m :: * -> *) (l :: (* -> * -> *) -> *).
StateRef m l -> l EmptyMK
state StateRef m (ExtLedgerState blk)
st
    crc2 <- takeHandleSnapshot (tables st) (state st) $ snapshotToDirName ds
    writeSnapshotMetadata shfs ds $
      SnapshotMetadata
        { snapshotBackend = UTxOHDLSMSnapshot
        , snapshotChecksum = maybe crc1 (crcOfConcat crc1) crc2
        , snapshotTablesCodecVersion = TablesCodecVersion1
        }

-- | Delete snapshot from disk and also from the LSM tree database.
implDeleteSnapshot ::
  IOLike m =>
  Session m ->
  SomeHasFS m ->
  Tracer m (TraceSnapshotEvent blk) ->
  DiskSnapshot ->
  m ()
implDeleteSnapshot :: forall (m :: * -> *) blk.
IOLike m =>
Session m
-> SomeHasFS m
-> Tracer m (TraceSnapshotEvent blk)
-> DiskSnapshot
-> m ()
implDeleteSnapshot
  Session m
session
  (SomeHasFS HasFS{HasCallStack => FsPath -> m Bool
doesDirectoryExist :: HasCallStack => FsPath -> m Bool
doesDirectoryExist :: forall (m :: * -> *) h.
HasFS m h -> HasCallStack => FsPath -> m Bool
doesDirectoryExist, HasCallStack => FsPath -> m ()
removeDirectoryRecursive :: HasCallStack => FsPath -> m ()
removeDirectoryRecursive :: forall (m :: * -> *) h. HasFS m h -> HasCallStack => FsPath -> m ()
removeDirectoryRecursive})
  Tracer m (TraceSnapshotEvent blk)
tracer
  DiskSnapshot
ss = do
    m ()
deleteState m () -> m () -> m ()
forall a b. m a -> m b -> m a
forall (m :: * -> *) a b. MonadThrow m => m a -> m b -> m a
`finally` m ()
deleteLsmTable
    Tracer m (TraceSnapshotEvent blk) -> TraceSnapshotEvent blk -> m ()
forall (m :: * -> *) a. Tracer m a -> a -> m ()
traceWith Tracer m (TraceSnapshotEvent blk)
tracer (DiskSnapshot -> TraceSnapshotEvent blk
forall blk. DiskSnapshot -> TraceSnapshotEvent blk
DeletedSnapshot DiskSnapshot
ss)
   where
    deleteState :: m ()
deleteState = do
      let p :: FsPath
p = DiskSnapshot -> FsPath
snapshotToDirPath DiskSnapshot
ss
      exists <- HasCallStack => FsPath -> m Bool
FsPath -> m Bool
doesDirectoryExist FsPath
p
      Monad.when exists (removeDirectoryRecursive p)

    deleteLsmTable :: m ()
deleteLsmTable =
      Session m -> SnapshotName -> m ()
forall (m :: * -> *). IOLike m => Session m -> SnapshotName -> m ()
LSM.deleteSnapshot
        Session m
session
        (String -> SnapshotName
forall a. IsString a => String -> a
fromString (String -> SnapshotName) -> String -> SnapshotName
forall a b. (a -> b) -> a -> b
$ Word64 -> String
forall a. Show a => a -> String
show (DiskSnapshot -> Word64
dsNumber DiskSnapshot
ss) String -> ShowS
forall a. Semigroup a => a -> a -> a
<> String -> ShowS -> Maybe String -> String
forall b a. b -> (a -> b) -> Maybe a -> b
maybe String
"" (String
"_" String -> ShowS
forall a. Semigroup a => a -> a -> a
<>) (DiskSnapshot -> Maybe String
dsSuffix DiskSnapshot
ss))

{-------------------------------------------------------------------------------
  Creating the first handle
-------------------------------------------------------------------------------}

-- | Read snapshot from disk.
--
--   Fail on data corruption, i.e. when the checksum of the read data differs
--   from the one tracked by @'DiskSnapshot'@.
loadSnapshot ::
  forall blk m.
  ( LedgerDbSerialiseConstraints blk
  , LedgerSupportsProtocol blk
  , IOLike m
  ) =>
  Tracer m LedgerDBV2Trace ->
  ResourceRegistry m ->
  CodecConfig blk ->
  SomeHasFS m ->
  Session m ->
  DiskSnapshot ->
  ExceptT (SnapshotFailure blk) m (LedgerSeq' m blk, RealPoint blk)
loadSnapshot :: forall blk (m :: * -> *).
(LedgerDbSerialiseConstraints blk, LedgerSupportsProtocol blk,
 IOLike m) =>
Tracer m LedgerDBV2Trace
-> ResourceRegistry m
-> CodecConfig blk
-> SomeHasFS m
-> Session m
-> DiskSnapshot
-> ExceptT
     (SnapshotFailure blk) m (LedgerSeq' m blk, RealPoint blk)
loadSnapshot Tracer m LedgerDBV2Trace
tracer ResourceRegistry m
rr CodecConfig blk
ccfg SomeHasFS m
fs Session m
session DiskSnapshot
ds =
  do
    snapshotMeta <-
      (MetadataErr -> SnapshotFailure blk)
-> ExceptT MetadataErr m SnapshotMetadata
-> ExceptT (SnapshotFailure blk) m SnapshotMetadata
forall (m :: * -> *) e e' a.
Functor m =>
(e -> e') -> ExceptT e m a -> ExceptT e' m a
withExceptT (ReadSnapshotErr -> SnapshotFailure blk
forall blk. ReadSnapshotErr -> SnapshotFailure blk
InitFailureRead (ReadSnapshotErr -> SnapshotFailure blk)
-> (MetadataErr -> ReadSnapshotErr)
-> MetadataErr
-> SnapshotFailure blk
forall b c a. (b -> c) -> (a -> b) -> a -> c
. FsPath -> MetadataErr -> ReadSnapshotErr
ReadMetadataError (DiskSnapshot -> FsPath
snapshotToMetadataPath DiskSnapshot
ds)) (ExceptT MetadataErr m SnapshotMetadata
 -> ExceptT (SnapshotFailure blk) m SnapshotMetadata)
-> ExceptT MetadataErr m SnapshotMetadata
-> ExceptT (SnapshotFailure blk) m SnapshotMetadata
forall a b. (a -> b) -> a -> b
$
        SomeHasFS m
-> DiskSnapshot -> ExceptT MetadataErr m SnapshotMetadata
forall (m :: * -> *).
IOLike m =>
SomeHasFS m
-> DiskSnapshot -> ExceptT MetadataErr m SnapshotMetadata
loadSnapshotMetadata SomeHasFS m
fs DiskSnapshot
ds
    Monad.when (snapshotBackend snapshotMeta /= UTxOHDLSMSnapshot) $
      throwE $
        InitFailureRead $
          ReadMetadataError (snapshotToMetadataPath ds) MetadataBackendMismatch
    (extLedgerSt, checksumAsRead) <-
      withExceptT
        (InitFailureRead . ReadSnapshotFailed)
        $ readExtLedgerState fs (decodeDiskExtLedgerState ccfg) decode (snapshotToStatePath ds)
    case pointToWithOriginRealPoint (castPoint (getTip extLedgerSt)) of
      WithOrigin (RealPoint blk)
Origin -> SnapshotFailure blk
-> ExceptT
     (SnapshotFailure blk) m (LedgerSeq' m blk, RealPoint blk)
forall (m :: * -> *) e a. Monad m => e -> ExceptT e m a
throwE SnapshotFailure blk
forall blk. SnapshotFailure blk
InitFailureGenesis
      NotOrigin RealPoint blk
pt -> do
        values <-
          m (ResourceKey m, Table m TxInBytes TxOutBytes Void)
-> ExceptT
     (SnapshotFailure blk)
     m
     (ResourceKey m, Table m TxInBytes TxOutBytes Void)
forall (m :: * -> *) a.
Monad m =>
m a -> ExceptT (SnapshotFailure blk) m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (m (ResourceKey m, Table m TxInBytes TxOutBytes Void)
 -> ExceptT
      (SnapshotFailure blk)
      m
      (ResourceKey m, Table m TxInBytes TxOutBytes Void))
-> m (ResourceKey m, Table m TxInBytes TxOutBytes Void)
-> ExceptT
     (SnapshotFailure blk)
     m
     (ResourceKey m, Table m TxInBytes TxOutBytes Void)
forall a b. (a -> b) -> a -> b
$
            ResourceRegistry m
-> (ResourceId -> m (Table m TxInBytes TxOutBytes Void))
-> (Table m TxInBytes TxOutBytes Void -> m ())
-> m (ResourceKey m, Table m TxInBytes TxOutBytes Void)
forall (m :: * -> *) a.
(MonadSTM m, MonadMask m, MonadThread m, HasCallStack) =>
ResourceRegistry m
-> (ResourceId -> m a) -> (a -> m ()) -> m (ResourceKey m, a)
allocate
              ResourceRegistry m
rr
              ( \ResourceId
_ ->
                  Session m
-> SnapshotName
-> SnapshotLabel
-> m (Table m TxInBytes TxOutBytes Void)
forall (m :: * -> *) k v b.
(IOLike m, ResolveValue v) =>
Session m -> SnapshotName -> SnapshotLabel -> m (Table m k v b)
LSM.openTableFromSnapshot
                    Session m
session
                    (String -> SnapshotName
forall a. IsString a => String -> a
fromString (String -> SnapshotName) -> String -> SnapshotName
forall a b. (a -> b) -> a -> b
$ DiskSnapshot -> String
snapshotToDirName DiskSnapshot
ds)
                    (Text -> SnapshotLabel
LSM.SnapshotLabel (Text -> SnapshotLabel) -> Text -> SnapshotLabel
forall a b. (a -> b) -> a -> b
$ String -> Text
Text.pack (String -> Text) -> String -> Text
forall a b. (a -> b) -> a -> b
$ String
"UTxO table")
              )
              ( \Table m TxInBytes TxOutBytes Void
t -> do
                  Tracer m LedgerDBV2Trace -> LedgerDBV2Trace -> m ()
forall (m :: * -> *) a. Tracer m a -> a -> m ()
traceWith Tracer m LedgerDBV2Trace
tracer LedgerDBV2Trace
TraceLedgerTablesHandleClose
                  Table m TxInBytes TxOutBytes Void -> m ()
forall (m :: * -> *) k v b. IOLike m => Table m k v b -> m ()
LSM.closeTable Table m TxInBytes TxOutBytes Void
t
              )
        Monad.when
          (checksumAsRead /= snapshotChecksum snapshotMeta)
          $ throwE
          $ InitFailureRead
            ReadSnapshotDataCorruption
        (,pt)
          <$> lift (empty extLedgerSt values (newLSMLedgerTablesHandle tracer rr))

-- | Create the initial LSM table from values, which should happen only at
-- Genesis.
tableFromValuesMK ::
  forall m l.
  (IOLike m, IndexedMemPack (l EmptyMK) (TxOut l), MemPack (TxIn l)) =>
  Tracer m LedgerDBV2Trace ->
  ResourceRegistry m ->
  Session m ->
  l EmptyMK ->
  LedgerTables l ValuesMK ->
  m (ResourceKey m, UTxOTable m)
tableFromValuesMK :: forall (m :: * -> *) (l :: (* -> * -> *) -> *).
(IOLike m, IndexedMemPack (l EmptyMK) (TxOut l),
 MemPack (TxIn l)) =>
Tracer m LedgerDBV2Trace
-> ResourceRegistry m
-> Session m
-> l EmptyMK
-> LedgerTables l ValuesMK
-> m (ResourceKey m, UTxOTable m)
tableFromValuesMK Tracer m LedgerDBV2Trace
tracer ResourceRegistry m
rr Session m
session l EmptyMK
st (LedgerTables (ValuesMK Map (TxIn l) (TxOut l)
values)) = do
  res@(_, table) <-
    ResourceRegistry m
-> (ResourceId -> m (Table m TxInBytes TxOutBytes Void))
-> (Table m TxInBytes TxOutBytes Void -> m ())
-> m (ResourceKey m, Table m TxInBytes TxOutBytes Void)
forall (m :: * -> *) a.
(MonadSTM m, MonadMask m, MonadThread m, HasCallStack) =>
ResourceRegistry m
-> (ResourceId -> m a) -> (a -> m ()) -> m (ResourceKey m, a)
allocate
      ResourceRegistry m
rr
      ( \ResourceId
_ ->
          TableConfig -> Session m -> m (Table m TxInBytes TxOutBytes Void)
forall (m :: * -> *) k v b.
IOLike m =>
TableConfig -> Session m -> m (Table m k v b)
LSM.newTableWith (TableConfig
LSM.defaultTableConfig{LSM.confFencePointerIndex = LSM.OrdinaryIndex}) Session m
session
      )
      ( \Table m TxInBytes TxOutBytes Void
tb -> do
          Tracer m LedgerDBV2Trace -> LedgerDBV2Trace -> m ()
forall (m :: * -> *) a. Tracer m a -> a -> m ()
traceWith Tracer m LedgerDBV2Trace
tracer LedgerDBV2Trace
TraceLedgerTablesHandleClose
          Table m TxInBytes TxOutBytes Void -> m ()
forall (m :: * -> *) k v b. IOLike m => Table m k v b -> m ()
LSM.closeTable Table m TxInBytes TxOutBytes Void
tb
      )
  mapM_ (go table) $ chunks 1000 $ Map.toList values
  pure res
 where
  go :: Table m TxInBytes TxOutBytes Void -> [(TxIn l, TxOut l)] -> m ()
go Table m TxInBytes TxOutBytes Void
table [(TxIn l, TxOut l)]
items =
    Table m TxInBytes TxOutBytes Void
-> Vector (TxInBytes, TxOutBytes, Maybe Void) -> m ()
forall (m :: * -> *) k v b.
(IOLike m, SerialiseKey k, SerialiseValue v, ResolveValue v,
 SerialiseValue b) =>
Table m k v b -> Vector (k, v, Maybe b) -> m ()
LSM.inserts Table m TxInBytes TxOutBytes Void
table (Vector (TxInBytes, TxOutBytes, Maybe Void) -> m ())
-> Vector (TxInBytes, TxOutBytes, Maybe Void) -> m ()
forall a b. (a -> b) -> a -> b
$
      Int
-> [(TxInBytes, TxOutBytes, Maybe Void)]
-> Vector (TxInBytes, TxOutBytes, Maybe Void)
forall a. Int -> [a] -> Vector a
V.fromListN ([(TxIn l, TxOut l)] -> Int
forall a. [a] -> Int
forall (t :: * -> *) a. Foldable t => t a -> Int
length [(TxIn l, TxOut l)]
items) ([(TxInBytes, TxOutBytes, Maybe Void)]
 -> Vector (TxInBytes, TxOutBytes, Maybe Void))
-> [(TxInBytes, TxOutBytes, Maybe Void)]
-> Vector (TxInBytes, TxOutBytes, Maybe Void)
forall a b. (a -> b) -> a -> b
$
        ((TxIn l, TxOut l) -> (TxInBytes, TxOutBytes, Maybe Void))
-> [(TxIn l, TxOut l)] -> [(TxInBytes, TxOutBytes, Maybe Void)]
forall a b. (a -> b) -> [a] -> [b]
map (\(TxIn l
k, TxOut l
v) -> (Proxy l -> TxIn l -> TxInBytes
forall (l :: (* -> * -> *) -> *).
MemPack (TxIn l) =>
Proxy l -> TxIn l -> TxInBytes
toTxInBytes (forall {k} (t :: k). Proxy t
forall (t :: (* -> * -> *) -> *). Proxy t
Proxy @l) TxIn l
k, l EmptyMK -> TxOut l -> TxOutBytes
forall (l :: (* -> * -> *) -> *).
IndexedMemPack (l EmptyMK) (TxOut l) =>
l EmptyMK -> TxOut l -> TxOutBytes
toTxOutBytes l EmptyMK
st TxOut l
v, Maybe Void
forall a. Maybe a
Nothing)) [(TxIn l, TxOut l)]
items

{-------------------------------------------------------------------------------
  Helpers
-------------------------------------------------------------------------------}

stdMkBlockIOFS ::
  FilePath -> ResourceRegistry IO -> IO (ResourceKey IO, SomeHasFSAndBlockIO IO)
stdMkBlockIOFS :: String
-> ResourceRegistry IO
-> IO (ResourceKey IO, SomeHasFSAndBlockIO IO)
stdMkBlockIOFS String
fastStoragePath ResourceRegistry IO
rr = do
  (rk1, bio) <-
    ResourceRegistry IO
-> (ResourceId -> IO (HasFS IO HandleIO, HasBlockIO IO HandleIO))
-> ((HasFS IO HandleIO, HasBlockIO IO HandleIO) -> IO ())
-> IO (ResourceKey IO, (HasFS IO HandleIO, HasBlockIO IO HandleIO))
forall (m :: * -> *) a.
(MonadSTM m, MonadMask m, MonadThread m, HasCallStack) =>
ResourceRegistry m
-> (ResourceId -> m a) -> (a -> m ()) -> m (ResourceKey m, a)
allocate
      ResourceRegistry IO
rr
      (\ResourceId
_ -> MountPoint
-> IOCtxParams -> IO (HasFS IO HandleIO, HasBlockIO IO HandleIO)
ioHasBlockIO (String -> MountPoint
MountPoint String
fastStoragePath) IOCtxParams
defaultIOCtxParams)
      (HasBlockIO IO HandleIO -> IO ()
HasBlockIO IO HandleIO -> HasCallStack => IO ()
forall (m :: * -> *) h. HasBlockIO m h -> HasCallStack => m ()
BIO.close (HasBlockIO IO HandleIO -> IO ())
-> ((HasFS IO HandleIO, HasBlockIO IO HandleIO)
    -> HasBlockIO IO HandleIO)
-> (HasFS IO HandleIO, HasBlockIO IO HandleIO)
-> IO ()
forall b c a. (b -> c) -> (a -> b) -> a -> c
. (HasFS IO HandleIO, HasBlockIO IO HandleIO)
-> HasBlockIO IO HandleIO
forall a b. (a, b) -> b
snd)
  pure (rk1, uncurry SomeHasFSAndBlockIO bio)

{-------------------------------------------------------------------------------
  Backend
-------------------------------------------------------------------------------}

type data LSM

-- | Create arguments for initializing the LedgerDB using the LSM-trees backend.
mkLSMArgs ::
  ( LedgerSupportsProtocol blk
  , LedgerDbSerialiseConstraints blk
  ) =>
  Proxy blk -> FilePath -> FilePath -> StdGen -> (LedgerDbBackendArgs IO blk, StdGen)
mkLSMArgs :: forall blk.
(LedgerSupportsProtocol blk, LedgerDbSerialiseConstraints blk) =>
Proxy blk
-> String
-> String
-> StdGen
-> (LedgerDbBackendArgs IO blk, StdGen)
mkLSMArgs Proxy blk
_ String
fp String
fastStorage StdGen
gen =
  let (Word64
lsmSalt, StdGen
gen') = StdGen -> (Word64, StdGen)
forall g. RandomGen g => g -> (Word64, g)
genWord64 StdGen
gen
   in ( SomeBackendArgs IO blk -> LedgerDbBackendArgs IO blk
forall (m :: * -> *) blk.
SomeBackendArgs m blk -> LedgerDbBackendArgs m blk
LedgerDbBackendArgsV2 (SomeBackendArgs IO blk -> LedgerDbBackendArgs IO blk)
-> SomeBackendArgs IO blk -> LedgerDbBackendArgs IO blk
forall a b. (a -> b) -> a -> b
$
          Args IO LSM -> SomeBackendArgs IO blk
forall (m :: * -> *) backend blk.
Backend m backend blk =>
Args m backend -> SomeBackendArgs m blk
SomeBackendArgs (Args IO LSM -> SomeBackendArgs IO blk)
-> Args IO LSM -> SomeBackendArgs IO blk
forall a b. (a -> b) -> a -> b
$
            FsPath
-> Word64
-> (ResourceRegistry IO
    -> IO (ResourceKey IO, SomeHasFSAndBlockIO IO))
-> Args IO LSM
forall (m :: * -> *).
FsPath
-> Word64
-> (ResourceRegistry m -> m (ResourceKey m, SomeHasFSAndBlockIO m))
-> Args m LSM
LSMArgs (Context -> FsPath
mkFsPath (Context -> FsPath) -> Context -> FsPath
forall a b. (a -> b) -> a -> b
$ String -> Context
splitDirectories String
fp) Word64
lsmSalt (String
-> ResourceRegistry IO
-> IO (ResourceKey IO, SomeHasFSAndBlockIO IO)
stdMkBlockIOFS String
fastStorage)
      , StdGen
gen'
      )

instance
  ( LedgerSupportsProtocol blk
  , IOLike m
  , LedgerDbSerialiseConstraints blk
  , HasLedgerTables (LedgerState blk)
  ) =>
  Backend m LSM blk
  where
  data Args m LSM
    = LSMArgs
        FsPath
        -- \^ The file path relative to the fast storage directory in which the LSM
        -- trees database will be located.
        Salt
        (ResourceRegistry m -> m (ResourceKey m, SomeHasFSAndBlockIO m))

  data Resources m LSM = LSMResources
    { forall (m :: * -> *). Resources m LSM -> ResourceKey m
sessionKey :: !(ResourceKey m)
    , forall (m :: * -> *). Resources m LSM -> Session m
sessionResource :: !(Session m)
    , forall (m :: * -> *). Resources m LSM -> ResourceKey m
blockIOKey :: !(ResourceKey m)
    }
    deriving (forall x. Resources m LSM -> Rep (Resources m LSM) x)
-> (forall x. Rep (Resources m LSM) x -> Resources m LSM)
-> Generic (Resources m LSM)
forall x. Rep (Resources m LSM) x -> Resources m LSM
forall x. Resources m LSM -> Rep (Resources m LSM) x
forall a.
(forall x. a -> Rep a x) -> (forall x. Rep a x -> a) -> Generic a
forall (m :: * -> *) x. Rep (Resources m LSM) x -> Resources m LSM
forall (m :: * -> *) x. Resources m LSM -> Rep (Resources m LSM) x
$cfrom :: forall (m :: * -> *) x. Resources m LSM -> Rep (Resources m LSM) x
from :: forall x. Resources m LSM -> Rep (Resources m LSM) x
$cto :: forall (m :: * -> *) x. Rep (Resources m LSM) x -> Resources m LSM
to :: forall x. Rep (Resources m LSM) x -> Resources m LSM
Generic

  data Trace m LSM
    = LSMTreeTrace !LSM.LSMTreeTrace
    deriving Int -> Trace m LSM -> ShowS
[Trace m LSM] -> ShowS
Trace m LSM -> String
(Int -> Trace m LSM -> ShowS)
-> (Trace m LSM -> String)
-> ([Trace m LSM] -> ShowS)
-> Show (Trace m LSM)
forall a.
(Int -> a -> ShowS) -> (a -> String) -> ([a] -> ShowS) -> Show a
forall (m :: * -> *). Int -> Trace m LSM -> ShowS
forall (m :: * -> *). [Trace m LSM] -> ShowS
forall (m :: * -> *). Trace m LSM -> String
$cshowsPrec :: forall (m :: * -> *). Int -> Trace m LSM -> ShowS
showsPrec :: Int -> Trace m LSM -> ShowS
$cshow :: forall (m :: * -> *). Trace m LSM -> String
show :: Trace m LSM -> String
$cshowList :: forall (m :: * -> *). [Trace m LSM] -> ShowS
showList :: [Trace m LSM] -> ShowS
Show

  mkResources :: Proxy blk
-> Tracer m LedgerDBV2Trace
-> Args m LSM
-> ResourceRegistry m
-> SomeHasFS m
-> m (Resources m LSM)
mkResources Proxy blk
_ Tracer m LedgerDBV2Trace
trcr (LSMArgs FsPath
path Word64
salt ResourceRegistry m -> m (ResourceKey m, SomeHasFSAndBlockIO m)
mkFS) ResourceRegistry m
reg SomeHasFS m
_ = do
    (rk1, SomeHasFSAndBlockIO fs blockio) <- ResourceRegistry m -> m (ResourceKey m, SomeHasFSAndBlockIO m)
mkFS ResourceRegistry m
reg
    session <-
      allocate
        reg
        ( \ResourceId
_ ->
            Tracer m LSMTreeTrace
-> HasFS m h -> HasBlockIO m h -> Word64 -> FsPath -> m (Session m)
forall (m :: * -> *) h.
(IOLike m, Typeable h) =>
Tracer m LSMTreeTrace
-> HasFS m h -> HasBlockIO m h -> Word64 -> FsPath -> m (Session m)
LSM.openSession
              (SomeBackendTrace -> LedgerDBV2Trace
BackendTrace (SomeBackendTrace -> LedgerDBV2Trace)
-> (LSMTreeTrace -> SomeBackendTrace)
-> LSMTreeTrace
-> LedgerDBV2Trace
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Trace (ZonkAny 0) LSM -> SomeBackendTrace
forall (m :: * -> *) backend.
Show (Trace m backend) =>
Trace m backend -> SomeBackendTrace
SomeBackendTrace (Trace (ZonkAny 0) LSM -> SomeBackendTrace)
-> (LSMTreeTrace -> Trace (ZonkAny 0) LSM)
-> LSMTreeTrace
-> SomeBackendTrace
forall b c a. (b -> c) -> (a -> b) -> a -> c
. LSMTreeTrace -> Trace (ZonkAny 0) LSM
forall (m :: * -> *). LSMTreeTrace -> Trace m LSM
LSMTreeTrace (LSMTreeTrace -> LedgerDBV2Trace)
-> Tracer m LedgerDBV2Trace -> Tracer m LSMTreeTrace
forall (f :: * -> *) a b. Contravariant f => (a -> b) -> f b -> f a
>$< Tracer m LedgerDBV2Trace
trcr)
              HasFS m h
fs
              HasBlockIO m h
blockio
              Word64
salt
              FsPath
path
        )
        LSM.closeSession
    pure (LSMResources (fst session) (snd session) rk1)

  releaseResources :: Proxy blk -> Resources m LSM -> m ()
releaseResources Proxy blk
_ Resources m LSM
l = do
    m (Maybe (Context m)) -> m ()
forall (f :: * -> *) a. Functor f => f a -> f ()
Monad.void (m (Maybe (Context m)) -> m ())
-> (Resources m LSM -> m (Maybe (Context m)))
-> Resources m LSM
-> m ()
forall b c a. (b -> c) -> (a -> b) -> a -> c
. ResourceKey m -> m (Maybe (Context m))
forall (m :: * -> *).
(MonadMask m, MonadSTM m, MonadThread m, HasCallStack) =>
ResourceKey m -> m (Maybe (Context m))
release (ResourceKey m -> m (Maybe (Context m)))
-> (Resources m LSM -> ResourceKey m)
-> Resources m LSM
-> m (Maybe (Context m))
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Resources m LSM -> ResourceKey m
forall (m :: * -> *). Resources m LSM -> ResourceKey m
sessionKey (Resources m LSM -> m ()) -> Resources m LSM -> m ()
forall a b. (a -> b) -> a -> b
$ Resources m LSM
l
    m (Maybe (Context m)) -> m ()
forall (f :: * -> *) a. Functor f => f a -> f ()
Monad.void (m (Maybe (Context m)) -> m ())
-> (Resources m LSM -> m (Maybe (Context m)))
-> Resources m LSM
-> m ()
forall b c a. (b -> c) -> (a -> b) -> a -> c
. ResourceKey m -> m (Maybe (Context m))
forall (m :: * -> *).
(MonadMask m, MonadSTM m, MonadThread m, HasCallStack) =>
ResourceKey m -> m (Maybe (Context m))
release (ResourceKey m -> m (Maybe (Context m)))
-> (Resources m LSM -> ResourceKey m)
-> Resources m LSM
-> m (Maybe (Context m))
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Resources m LSM -> ResourceKey m
forall (m :: * -> *). Resources m LSM -> ResourceKey m
blockIOKey (Resources m LSM -> m ()) -> Resources m LSM -> m ()
forall a b. (a -> b) -> a -> b
$ Resources m LSM
l

  newHandleFromSnapshot :: Tracer m LedgerDBV2Trace
-> ResourceRegistry m
-> CodecConfig blk
-> SomeHasFS m
-> Resources m LSM
-> DiskSnapshot
-> ExceptT
     (SnapshotFailure blk) m (LedgerSeq' m blk, RealPoint blk)
newHandleFromSnapshot Tracer m LedgerDBV2Trace
trcr ResourceRegistry m
reg CodecConfig blk
ccfg SomeHasFS m
shfs Resources m LSM
res DiskSnapshot
ds = do
    Tracer m LedgerDBV2Trace
-> ResourceRegistry m
-> CodecConfig blk
-> SomeHasFS m
-> Session m
-> DiskSnapshot
-> ExceptT
     (SnapshotFailure blk) m (LedgerSeq' m blk, RealPoint blk)
forall blk (m :: * -> *).
(LedgerDbSerialiseConstraints blk, LedgerSupportsProtocol blk,
 IOLike m) =>
Tracer m LedgerDBV2Trace
-> ResourceRegistry m
-> CodecConfig blk
-> SomeHasFS m
-> Session m
-> DiskSnapshot
-> ExceptT
     (SnapshotFailure blk) m (LedgerSeq' m blk, RealPoint blk)
loadSnapshot Tracer m LedgerDBV2Trace
trcr ResourceRegistry m
reg CodecConfig blk
ccfg SomeHasFS m
shfs (Resources m LSM -> Session m
forall (m :: * -> *). Resources m LSM -> Session m
sessionResource Resources m LSM
res) DiskSnapshot
ds

  newHandleFromValues :: Tracer m LedgerDBV2Trace
-> ResourceRegistry m
-> Resources m LSM
-> ExtLedgerState blk ValuesMK
-> m (LedgerTablesHandle m (ExtLedgerState blk))
newHandleFromValues Tracer m LedgerDBV2Trace
trcr ResourceRegistry m
reg Resources m LSM
res ExtLedgerState blk ValuesMK
st = do
    table <-
      Tracer m LedgerDBV2Trace
-> ResourceRegistry m
-> Session m
-> ExtLedgerState blk EmptyMK
-> LedgerTables (ExtLedgerState blk) ValuesMK
-> m (ResourceKey m, UTxOTable m)
forall (m :: * -> *) (l :: (* -> * -> *) -> *).
(IOLike m, IndexedMemPack (l EmptyMK) (TxOut l),
 MemPack (TxIn l)) =>
Tracer m LedgerDBV2Trace
-> ResourceRegistry m
-> Session m
-> l EmptyMK
-> LedgerTables l ValuesMK
-> m (ResourceKey m, UTxOTable m)
tableFromValuesMK Tracer m LedgerDBV2Trace
trcr ResourceRegistry m
reg (Resources m LSM -> Session m
forall (m :: * -> *). Resources m LSM -> Session m
sessionResource Resources m LSM
res) (ExtLedgerState blk ValuesMK -> ExtLedgerState blk EmptyMK
forall (l :: (* -> * -> *) -> *) (mk :: * -> * -> *).
HasLedgerTables l =>
l mk -> l EmptyMK
forgetLedgerTables ExtLedgerState blk ValuesMK
st) (ExtLedgerState blk ValuesMK
-> LedgerTables (ExtLedgerState blk) ValuesMK
forall (l :: (* -> * -> *) -> *) (l' :: (* -> * -> *) -> *)
       (mk :: * -> * -> *).
(HasLedgerTables l, SameUtxoTypes l l', CanMapMK mk,
 CanMapKeysMK mk, ZeroableMK mk) =>
l mk -> LedgerTables l' mk
ltprj ExtLedgerState blk ValuesMK
st)
    newLSMLedgerTablesHandle trcr reg table

  snapshotManager :: Proxy blk
-> Resources m LSM
-> CodecConfig blk
-> Tracer m (TraceSnapshotEvent blk)
-> SomeHasFS m
-> SnapshotManager m m blk (StateRef m (ExtLedgerState blk))
snapshotManager Proxy blk
_ Resources m LSM
res = Session m
-> CodecConfig blk
-> Tracer m (TraceSnapshotEvent blk)
-> SomeHasFS m
-> SnapshotManager m m blk (StateRef m (ExtLedgerState blk))
forall (m :: * -> *) blk.
(IOLike m, LedgerDbSerialiseConstraints blk,
 LedgerSupportsProtocol blk) =>
Session m
-> CodecConfig blk
-> Tracer m (TraceSnapshotEvent blk)
-> SomeHasFS m
-> SnapshotManager m m blk (StateRef m (ExtLedgerState blk))
Ouroboros.Consensus.Storage.LedgerDB.V2.LSM.snapshotManager (Resources m LSM -> Session m
forall (m :: * -> *). Resources m LSM -> Session m
sessionResource Resources m LSM
res)

instance
  ( MemPack (TxIn l)
  , IndexedMemPack (l EmptyMK) (TxOut l)
  , IOLike m
  ) =>
  StreamingBackend m LSM l
  where
  data YieldArgs m LSM l
    = -- \| Yield an LSM snapshot
      YieldLSM
        Int
        (LedgerTablesHandle m l)

  data SinkArgs m LSM l
    = SinkLSM
        -- \| Chunk size
        Int
        -- \| Snap name
        String
        (Session m)

  yield :: Proxy LSM -> YieldArgs m LSM l -> Yield m l
yield Proxy LSM
_ (YieldLSM Int
chunkSize LedgerTablesHandle m l
hdl) = Int -> LedgerTablesHandle m l -> Yield m l
forall (m :: * -> *) (l :: (* -> * -> *) -> *).
Monad m =>
Int -> LedgerTablesHandle m l -> Yield m l
yieldLsmS Int
chunkSize LedgerTablesHandle m l
hdl

  sink :: Proxy LSM -> SinkArgs m LSM l -> Sink m l
sink Proxy LSM
_ (SinkLSM Int
chunkSize String
snapName Session m
session) = Int -> String -> Session m -> Sink m l
forall (m :: * -> *) (l :: (* -> * -> *) -> *).
(MonadAsync m, MonadMVar m, MonadThrow (STM m), MonadMask m,
 MonadST m, MonadEvaluate m, MemPack (TxIn l),
 IndexedMemPack (l EmptyMK) (TxOut l)) =>
Int -> String -> Session m -> Sink m l
sinkLsmS Int
chunkSize String
snapName Session m
session

data SomeHasFSAndBlockIO m where
  SomeHasFSAndBlockIO ::
    (Eq h, Typeable h) => HasFS m h -> BIO.HasBlockIO m h -> SomeHasFSAndBlockIO m

instance IOLike m => NoThunks (Resources m LSM) where
  wNoThunks :: Context -> Resources m LSM -> IO (Maybe ThunkInfo)
wNoThunks Context
ctxt (LSMResources ResourceKey m
sk Session m
_ ResourceKey m
bk) = Context -> ResourceKey m -> IO (Maybe ThunkInfo)
forall a. NoThunks a => Context -> a -> IO (Maybe ThunkInfo)
wNoThunks Context
ctxt ResourceKey m
sk IO (Maybe ThunkInfo)
-> IO (Maybe ThunkInfo) -> IO (Maybe ThunkInfo)
forall a b. IO a -> IO b -> IO b
forall (m :: * -> *) a b. Monad m => m a -> m b -> m b
>> Context -> ResourceKey m -> IO (Maybe ThunkInfo)
forall a. NoThunks a => Context -> a -> IO (Maybe ThunkInfo)
wNoThunks Context
ctxt ResourceKey m
bk

{-------------------------------------------------------------------------------
  Streaming
-------------------------------------------------------------------------------}

yieldLsmS ::
  Monad m =>
  Int ->
  LedgerTablesHandle m l ->
  Yield m l
yieldLsmS :: forall (m :: * -> *) (l :: (* -> * -> *) -> *).
Monad m =>
Int -> LedgerTablesHandle m l -> Yield m l
yieldLsmS Int
readChunkSize LedgerTablesHandle m l
tb 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 ((Maybe (TxIn l), Int)
-> Stream
     (Of (TxIn l, TxOut l))
     (ExceptT DeserialiseFailure m)
     (Stream (Of ByteString) m (Maybe CRC))
go (Maybe (TxIn l)
forall a. Maybe a
Nothing, Int
readChunkSize))
  lift $ S.effects r
 where
  go :: (Maybe (TxIn l), Int)
-> Stream
     (Of (TxIn l, TxOut l))
     (ExceptT DeserialiseFailure m)
     (Stream (Of ByteString) m (Maybe CRC))
go (Maybe (TxIn l), Int)
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
$ LedgerTablesHandle m l
-> l EmptyMK
-> (Maybe (TxIn l), Int)
-> m (LedgerTables l ValuesMK, Maybe (TxIn l))
forall (m :: * -> *) (l :: (* -> * -> *) -> *).
LedgerTablesHandle m l
-> l EmptyMK
-> (Maybe (TxIn l), Int)
-> m (LedgerTables l ValuesMK, Maybe (TxIn l))
readRange LedgerTablesHandle m l
tb l EmptyMK
hint (Maybe (TxIn l), Int)
p
    if Map.null values
      then pure $ pure Nothing
      else do
        S.each $ Map.toList values
        go (mx, readChunkSize)

sinkLsmS ::
  forall m l.
  ( MonadAsync m
  , MonadMVar m
  , MonadThrow (STM m)
  , MonadMask m
  , MonadST m
  , MonadEvaluate m
  , MemPack (TxIn l)
  , IndexedMemPack (l EmptyMK) (TxOut l)
  ) =>
  Int ->
  String ->
  Session m ->
  Sink m l
sinkLsmS :: forall (m :: * -> *) (l :: (* -> * -> *) -> *).
(MonadAsync m, MonadMVar m, MonadThrow (STM m), MonadMask m,
 MonadST m, MonadEvaluate m, MemPack (TxIn l),
 IndexedMemPack (l EmptyMK) (TxOut l)) =>
Int -> String -> Session m -> Sink m l
sinkLsmS Int
writeChunkSize String
snapName Session m
session l EmptyMK
st Stream
  (Of (TxIn l, TxOut l))
  (ExceptT DeserialiseFailure m)
  (Stream (Of ByteString) m (Maybe CRC))
s = do
  tb :: UTxOTable m <- m (UTxOTable m) -> ExceptT DeserialiseFailure m (UTxOTable 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 (UTxOTable m) -> ExceptT DeserialiseFailure m (UTxOTable m))
-> m (UTxOTable m) -> ExceptT DeserialiseFailure m (UTxOTable m)
forall a b. (a -> b) -> a -> b
$ Session m -> m (UTxOTable m)
forall (m :: * -> *) k v b.
IOLike m =>
Session m -> m (Table m k v b)
LSM.newTable Session m
session
  r <- go tb writeChunkSize mempty s
  lift $
    LSM.saveSnapshot
      (LSM.toSnapshotName snapName)
      (LSM.SnapshotLabel $ T.pack "UTxO table")
      tb
  lift $ LSM.closeTable tb
  pure (fmap (,Nothing) r)
 where
  go :: UTxOTable m
-> Int
-> [(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 UTxOTable m
tb Int
0 [(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
$
      UTxOTable m -> Vector (TxInBytes, TxOutBytes, Maybe Void) -> m ()
forall (m :: * -> *) k v b.
(IOLike m, SerialiseKey k, SerialiseValue v, ResolveValue v,
 SerialiseValue b) =>
Table m k v b -> Vector (k, v, Maybe b) -> m ()
LSM.inserts UTxOTable m
tb (Vector (TxInBytes, TxOutBytes, Maybe Void) -> m ())
-> Vector (TxInBytes, TxOutBytes, Maybe Void) -> m ()
forall a b. (a -> b) -> a -> b
$
        [(TxInBytes, TxOutBytes, Maybe Void)]
-> Vector (TxInBytes, TxOutBytes, Maybe Void)
forall a. [a] -> Vector a
V.fromList [(Proxy l -> TxIn l -> TxInBytes
forall (l :: (* -> * -> *) -> *).
MemPack (TxIn l) =>
Proxy l -> TxIn l -> TxInBytes
toTxInBytes (forall {k} (t :: k). Proxy t
forall (t :: (* -> * -> *) -> *). Proxy t
Proxy @l) TxIn l
k, l EmptyMK -> TxOut l -> TxOutBytes
forall (l :: (* -> * -> *) -> *).
IndexedMemPack (l EmptyMK) (TxOut l) =>
l EmptyMK -> TxOut l -> TxOutBytes
toTxOutBytes l EmptyMK
st TxOut l
v, Maybe Void
forall a. Maybe a
Nothing) | (TxIn l
k, TxOut l
v) <- [(TxIn l, TxOut l)]
m]
    UTxOTable m
-> Int
-> [(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 UTxOTable m
tb Int
writeChunkSize [(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 UTxOTable m
tb Int
n [(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
$
          UTxOTable m -> Vector (TxInBytes, TxOutBytes, Maybe Void) -> m ()
forall (m :: * -> *) k v b.
(IOLike m, SerialiseKey k, SerialiseValue v, ResolveValue v,
 SerialiseValue b) =>
Table m k v b -> Vector (k, v, Maybe b) -> m ()
LSM.inserts UTxOTable m
tb (Vector (TxInBytes, TxOutBytes, Maybe Void) -> m ())
-> Vector (TxInBytes, TxOutBytes, Maybe Void) -> m ()
forall a b. (a -> b) -> a -> b
$
            [(TxInBytes, TxOutBytes, Maybe Void)]
-> Vector (TxInBytes, TxOutBytes, Maybe Void)
forall a. [a] -> Vector a
V.fromList
              [(Proxy l -> TxIn l -> TxInBytes
forall (l :: (* -> * -> *) -> *).
MemPack (TxIn l) =>
Proxy l -> TxIn l -> TxInBytes
toTxInBytes (forall {k} (t :: k). Proxy t
forall (t :: (* -> * -> *) -> *). Proxy t
Proxy @l) TxIn l
k, l EmptyMK -> TxOut l -> TxOutBytes
forall (l :: (* -> * -> *) -> *).
IndexedMemPack (l EmptyMK) (TxOut l) =>
l EmptyMK -> TxOut l -> TxOutBytes
toTxOutBytes l EmptyMK
st TxOut l
v, Maybe Void
forall a. Maybe a
Nothing) | (TxIn l
k, TxOut l
v) <- [(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, TxOut l)
item, Stream
  (Of (TxIn l, TxOut l))
  (ExceptT DeserialiseFailure m)
  (Stream (Of ByteString) m (Maybe CRC))
s'') -> UTxOTable m
-> Int
-> [(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 UTxOTable m
tb (Int
n Int -> Int -> Int
forall a. Num a => a -> a -> a
- Int
1) ((TxIn l, TxOut l)
item (TxIn l, TxOut l) -> [(TxIn l, TxOut l)] -> [(TxIn l, TxOut l)]
forall a. a -> [a] -> [a]
: [(TxIn l, TxOut l)]
m) Stream
  (Of (TxIn l, TxOut l))
  (ExceptT DeserialiseFailure m)
  (Stream (Of ByteString) m (Maybe CRC))
s''

-- | Create Yield arguments for LSM
mkLSMYieldArgs ::
  ( IOLike m
  , HasLedgerTables l
  , IndexedMemPack (l EmptyMK) (TxOut l)
  ) =>
  -- | The filepath in which the LSM database lives. Must not have a trailing slash!
  FilePath ->
  -- | The complete name of the snapshot to open, so @<slotno>[_<suffix>]@.
  String ->
  -- | Usually 'stdMkBlockIOFS'
  (FilePath -> ResourceRegistry m -> m (a, SomeHasFSAndBlockIO m)) ->
  -- | Usually 'newStdGen'
  (m StdGen) ->
  l EmptyMK ->
  ResourceRegistry m ->
  m (YieldArgs m LSM l)
mkLSMYieldArgs :: forall (m :: * -> *) (l :: (* -> * -> *) -> *) a.
(IOLike m, HasLedgerTables l,
 IndexedMemPack (l EmptyMK) (TxOut l)) =>
String
-> String
-> (String -> ResourceRegistry m -> m (a, SomeHasFSAndBlockIO m))
-> m StdGen
-> l EmptyMK
-> ResourceRegistry m
-> m (YieldArgs m LSM l)
mkLSMYieldArgs String
fp String
snapName String -> ResourceRegistry m -> m (a, SomeHasFSAndBlockIO m)
mkFS m StdGen
mkGen l EmptyMK
_ ResourceRegistry m
reg = do
  (_, SomeHasFSAndBlockIO hasFS blockIO) <- String -> ResourceRegistry m -> m (a, SomeHasFSAndBlockIO m)
mkFS String
fp ResourceRegistry m
reg
  salt <- fst . genWord64 <$> mkGen
  (_, session) <-
    allocate reg (\ResourceId
_ -> Tracer m LSMTreeTrace
-> HasFS m h -> HasBlockIO m h -> Word64 -> FsPath -> m (Session m)
forall (m :: * -> *) h.
(IOLike m, Typeable h) =>
Tracer m LSMTreeTrace
-> HasFS m h -> HasBlockIO m h -> Word64 -> FsPath -> m (Session m)
LSM.openSession Tracer m LSMTreeTrace
forall (m :: * -> *) a. Applicative m => Tracer m a
nullTracer HasFS m h
hasFS HasBlockIO m h
blockIO Word64
salt (Context -> FsPath
mkFsPath [])) LSM.closeSession
  tb <-
    allocate
      reg
      ( \ResourceId
_ ->
          Session m
-> SnapshotName
-> SnapshotLabel
-> m (Table m TxInBytes TxOutBytes Void)
forall (m :: * -> *) k v b.
(IOLike m, ResolveValue v) =>
Session m -> SnapshotName -> SnapshotLabel -> m (Table m k v b)
LSM.openTableFromSnapshot
            Session m
session
            (String -> SnapshotName
LSM.toSnapshotName String
snapName)
            (Text -> SnapshotLabel
LSM.SnapshotLabel (Text -> SnapshotLabel) -> Text -> SnapshotLabel
forall a b. (a -> b) -> a -> b
$ String -> Text
T.pack String
"UTxO table")
      )
      LSM.closeTable
  YieldLSM 1000 <$> newLSMLedgerTablesHandle nullTracer reg tb

-- | Create Sink arguments for LSM
mkLSMSinkArgs ::
  IOLike m =>
  -- | The filepath in which the LSM database should be opened. Must not have a trailing slash!
  FilePath ->
  -- | The complete name of the snapshot to be created, so @<slotno>[_<suffix>]@.
  String ->
  -- | Usually 'stdMkBlockIOFS'
  (FilePath -> ResourceRegistry m -> m (a, SomeHasFSAndBlockIO m)) ->
  -- | Usually 'newStdGen'
  (m StdGen) ->
  l EmptyMK ->
  ResourceRegistry m ->
  m (SinkArgs m LSM l)
mkLSMSinkArgs :: forall (m :: * -> *) a (l :: (* -> * -> *) -> *).
IOLike m =>
String
-> String
-> (String -> ResourceRegistry m -> m (a, SomeHasFSAndBlockIO m))
-> m StdGen
-> l EmptyMK
-> ResourceRegistry m
-> m (SinkArgs m LSM l)
mkLSMSinkArgs
  (String -> (String, String)
splitFileName -> (String
fp, String
lsmDir))
  String
snapName
  String -> ResourceRegistry m -> m (a, SomeHasFSAndBlockIO m)
mkFS
  m StdGen
mkGen
  l EmptyMK
_
  ResourceRegistry m
reg =
    do
      (_, SomeHasFSAndBlockIO hasFS blockIO) <- String -> ResourceRegistry m -> m (a, SomeHasFSAndBlockIO m)
mkFS String
fp ResourceRegistry m
reg
      removeDirectoryRecursive hasFS lsmFsPath
      createDirectory hasFS lsmFsPath
      salt <- fst . genWord64 <$> mkGen
      (_, session) <-
        allocate reg (\ResourceId
_ -> Tracer m LSMTreeTrace
-> HasFS m h -> HasBlockIO m h -> Word64 -> FsPath -> m (Session m)
forall (m :: * -> *) h.
(IOLike m, Typeable h) =>
Tracer m LSMTreeTrace
-> HasFS m h -> HasBlockIO m h -> Word64 -> FsPath -> m (Session m)
LSM.newSession Tracer m LSMTreeTrace
forall (m :: * -> *) a. Applicative m => Tracer m a
nullTracer HasFS m h
hasFS HasBlockIO m h
blockIO Word64
salt FsPath
lsmFsPath) LSM.closeSession
      pure (SinkLSM 1000 snapName session)
   where
    lsmFsPath :: FsPath
lsmFsPath = Context -> FsPath
mkFsPath [String
lsmDir]