{-# LANGUAGE BangPatterns #-}
{-# LANGUAGE DeriveAnyClass #-}
{-# LANGUAGE DeriveGeneric #-}
{-# LANGUAGE DerivingVia #-}
{-# LANGUAGE FlexibleContexts #-}
{-# LANGUAGE FlexibleInstances #-}
{-# LANGUAGE GeneralizedNewtypeDeriving #-}
{-# LANGUAGE LambdaCase #-}
{-# LANGUAGE MultiParamTypeClasses #-}
{-# LANGUAGE RankNTypes #-}
{-# LANGUAGE StandaloneDeriving #-}
{-# LANGUAGE TupleSections #-}
{-# LANGUAGE TypeData #-}
{-# LANGUAGE TypeFamilies #-}
{-# LANGUAGE UndecidableInstances #-}
{-# LANGUAGE ViewPatterns #-}

module Ouroboros.Consensus.Storage.LedgerDB.V2.InMemory
  ( Backend (..)
  , Args (InMemArgs)
  , Mem
  , YieldArgs (YieldInMemory)
  , SinkArgs (SinkInMemory)
  , mkInMemoryArgs
  ) where

import Cardano.Binary as CBOR
import Cardano.Slotting.Slot
import Codec.CBOR.Read
import qualified Codec.CBOR.Write as CBOR
import Codec.Serialise (decode)
import Control.Monad (replicateM_, unless)
import qualified Control.Monad as Monad
import Control.Monad.Class.MonadST
import Control.Monad.Class.MonadSTM
import Control.Monad.Class.MonadThrow
import Control.Monad.Except
import Control.Monad.State.Strict (execStateT)
import Control.Monad.Trans.Except
import Control.ResourceRegistry
import Control.Tracer
import Data.ByteString (ByteString)
import qualified Data.ByteString as BS
import Data.ByteString.Builder.Extra (defaultChunkSize)
import Data.Functor.Contravariant ((>$<))
import Data.Functor.Identity
import qualified Data.List as List
import qualified Data.Map.Strict as Map
import Data.Maybe
import Data.MemPack
import Data.Void
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.CBOR (readIncremental)
import Ouroboros.Consensus.Util.CRC
import Ouroboros.Consensus.Util.Enclose
import Ouroboros.Consensus.Util.IOLike
import Streaming
import qualified Streaming as S
import qualified Streaming.Prelude as S
import System.FS.API
import System.FS.CRC
import qualified System.FilePath as F
import Prelude hiding (read)

{-------------------------------------------------------------------------------
  InMemory implementation of LedgerTablesHandles
-------------------------------------------------------------------------------}

data LedgerTablesHandleState l
  = LedgerTablesHandleOpen !(LedgerTables l ValuesMK)
  | LedgerTablesHandleClosed
  deriving (forall x.
 LedgerTablesHandleState l -> Rep (LedgerTablesHandleState l) x)
-> (forall x.
    Rep (LedgerTablesHandleState l) x -> LedgerTablesHandleState l)
-> Generic (LedgerTablesHandleState l)
forall x.
Rep (LedgerTablesHandleState l) x -> LedgerTablesHandleState l
forall x.
LedgerTablesHandleState l -> Rep (LedgerTablesHandleState l) x
forall a.
(forall x. a -> Rep a x) -> (forall x. Rep a x -> a) -> Generic a
forall (l :: LedgerStateKind) x.
Rep (LedgerTablesHandleState l) x -> LedgerTablesHandleState l
forall (l :: LedgerStateKind) x.
LedgerTablesHandleState l -> Rep (LedgerTablesHandleState l) x
$cfrom :: forall (l :: LedgerStateKind) x.
LedgerTablesHandleState l -> Rep (LedgerTablesHandleState l) x
from :: forall x.
LedgerTablesHandleState l -> Rep (LedgerTablesHandleState l) x
$cto :: forall (l :: LedgerStateKind) x.
Rep (LedgerTablesHandleState l) x -> LedgerTablesHandleState l
to :: forall x.
Rep (LedgerTablesHandleState l) x -> LedgerTablesHandleState l
Generic

deriving instance NoThunks (LedgerTables l ValuesMK) => NoThunks (LedgerTablesHandleState l)

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

guardClosed :: LedgerTablesHandleState l -> (LedgerTables l ValuesMK -> a) -> a
guardClosed :: forall (l :: LedgerStateKind) a.
LedgerTablesHandleState l -> (LedgerTables l ValuesMK -> a) -> a
guardClosed LedgerTablesHandleState l
LedgerTablesHandleClosed LedgerTables l ValuesMK -> a
_ = String -> a
forall a. HasCallStack => String -> a
error (String -> a) -> String -> a
forall a b. (a -> b) -> a -> b
$ InMemoryClosedExn -> String
forall a. Show a => a -> String
show InMemoryClosedExn
InMemoryClosedExn
guardClosed (LedgerTablesHandleOpen LedgerTables l ValuesMK
st) LedgerTables l ValuesMK -> a
f = LedgerTables l ValuesMK -> a
f LedgerTables l ValuesMK
st

newInMemoryLedgerTablesHandle ::
  forall m l.
  ( IOLike m
  , HasLedgerTables l
  , CanUpgradeLedgerTables l
  , SerializeTablesWithHint l
  ) =>
  Tracer m LedgerDBV2Trace ->
  SomeHasFS m ->
  LedgerTables l ValuesMK ->
  m (LedgerTablesHandle m l)
newInMemoryLedgerTablesHandle :: forall (m :: * -> *) (l :: LedgerStateKind).
(IOLike m, HasLedgerTables l, CanUpgradeLedgerTables l,
 SerializeTablesWithHint l) =>
Tracer m LedgerDBV2Trace
-> SomeHasFS m
-> LedgerTables l ValuesMK
-> m (LedgerTablesHandle m l)
newInMemoryLedgerTablesHandle Tracer m LedgerDBV2Trace
tracer someFS :: SomeHasFS m
someFS@(SomeHasFS HasFS m h
hasFS) LedgerTables l ValuesMK
l = do
  !tv <- LedgerTablesHandleState l
-> m (StrictTVar m (LedgerTablesHandleState l))
forall (m :: * -> *) a.
(HasCallStack, MonadSTM m, NoThunks a) =>
a -> m (StrictTVar m a)
newTVarIO (LedgerTables l ValuesMK -> LedgerTablesHandleState l
forall (l :: LedgerStateKind).
LedgerTables l ValuesMK -> LedgerTablesHandleState l
LedgerTablesHandleOpen LedgerTables l ValuesMK
l)
  traceWith tracer TraceLedgerTablesHandleCreate
  pure
    LedgerTablesHandle
      { close = implClose tracer tv
      , duplicate = implDuplicate tracer tv someFS
      , read = implRead tv
      , readRange = implReadRange tv
      , readAll = implReadAll tv
      , pushDiffs = implPushDiffs tv
      , takeHandleSnapshot = implTakeHandleSnapshot tv hasFS
      , tablesSize = implTablesSize tv
      }

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

implClose ::
  IOLike m =>
  Tracer m LedgerDBV2Trace ->
  StrictTVar m (LedgerTablesHandleState l) ->
  m ()
implClose :: forall (m :: * -> *) (l :: LedgerStateKind).
IOLike m =>
Tracer m LedgerDBV2Trace
-> StrictTVar m (LedgerTablesHandleState l) -> m ()
implClose Tracer m LedgerDBV2Trace
tracer StrictTVar m (LedgerTablesHandleState l)
tv = do
  p <- STM m (LedgerTablesHandleState l) -> m (LedgerTablesHandleState l)
forall a. HasCallStack => STM m a -> m a
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
STM m a -> m a
atomically (STM m (LedgerTablesHandleState l)
 -> m (LedgerTablesHandleState l))
-> STM m (LedgerTablesHandleState l)
-> m (LedgerTablesHandleState l)
forall a b. (a -> b) -> a -> b
$ StrictTVar m (LedgerTablesHandleState l)
-> LedgerTablesHandleState l -> STM m (LedgerTablesHandleState l)
forall (m :: * -> *) a.
MonadSTM m =>
StrictTVar m a -> a -> STM m a
swapTVar StrictTVar m (LedgerTablesHandleState l)
tv LedgerTablesHandleState l
forall (l :: LedgerStateKind). LedgerTablesHandleState l
LedgerTablesHandleClosed
  case p of
    LedgerTablesHandleOpen{} -> Tracer m LedgerDBV2Trace -> LedgerDBV2Trace -> m ()
forall (m :: * -> *) a. Tracer m a -> a -> m ()
traceWith Tracer m LedgerDBV2Trace
tracer LedgerDBV2Trace
TraceLedgerTablesHandleClose
    LedgerTablesHandleState l
_ -> () -> m ()
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure ()

implDuplicate ::
  ( IOLike m
  , HasLedgerTables l
  , CanUpgradeLedgerTables l
  , SerializeTablesWithHint l
  ) =>
  Tracer m LedgerDBV2Trace ->
  StrictTVar m (LedgerTablesHandleState l) ->
  SomeHasFS m ->
  m (LedgerTablesHandle m l)
implDuplicate :: forall (m :: * -> *) (l :: LedgerStateKind).
(IOLike m, HasLedgerTables l, CanUpgradeLedgerTables l,
 SerializeTablesWithHint l) =>
Tracer m LedgerDBV2Trace
-> StrictTVar m (LedgerTablesHandleState l)
-> SomeHasFS m
-> m (LedgerTablesHandle m l)
implDuplicate Tracer m LedgerDBV2Trace
tracer StrictTVar m (LedgerTablesHandleState l)
tv SomeHasFS m
someFS = do
  hs <- StrictTVar m (LedgerTablesHandleState l)
-> m (LedgerTablesHandleState l)
forall (m :: * -> *) a. MonadSTM m => StrictTVar m a -> m a
readTVarIO StrictTVar m (LedgerTablesHandleState l)
tv
  !x <- guardClosed hs $ newInMemoryLedgerTablesHandle tracer someFS
  pure x

implRead ::
  ( IOLike m
  , HasLedgerTables l
  ) =>
  StrictTVar m (LedgerTablesHandleState l) ->
  l EmptyMK ->
  LedgerTables l KeysMK ->
  m (LedgerTables l ValuesMK)
implRead :: forall (m :: * -> *) (l :: LedgerStateKind).
(IOLike m, HasLedgerTables l) =>
StrictTVar m (LedgerTablesHandleState l)
-> l EmptyMK
-> LedgerTables l KeysMK
-> m (LedgerTables l ValuesMK)
implRead StrictTVar m (LedgerTablesHandleState l)
tv l EmptyMK
_ LedgerTables l KeysMK
keys = do
  hs <- StrictTVar m (LedgerTablesHandleState l)
-> m (LedgerTablesHandleState l)
forall (m :: * -> *) a. MonadSTM m => StrictTVar m a -> m a
readTVarIO StrictTVar m (LedgerTablesHandleState l)
tv
  guardClosed
    hs
    (pure . flip (ltliftA2 (\(ValuesMK Map k v
v) (KeysMK Set k
k) -> Map k v -> ValuesMK k v
forall k v. Map k v -> ValuesMK k v
ValuesMK (Map k v -> ValuesMK k v) -> Map k v -> ValuesMK k v
forall a b. (a -> b) -> a -> b
$ Map k v
v Map k v -> Set k -> Map k v
forall k a. Ord k => Map k a -> Set k -> Map k a
`Map.restrictKeys` Set k
k)) keys)

implReadRange ::
  (IOLike m, HasLedgerTables l) =>
  StrictTVar m (LedgerTablesHandleState l) ->
  l EmptyMK ->
  (Maybe (TxIn l), Int) ->
  m (LedgerTables l ValuesMK, Maybe (TxIn l))
implReadRange :: forall (m :: * -> *) (l :: LedgerStateKind).
(IOLike m, HasLedgerTables l) =>
StrictTVar m (LedgerTablesHandleState l)
-> l EmptyMK
-> (Maybe (TxIn l), Int)
-> m (LedgerTables l ValuesMK, Maybe (TxIn l))
implReadRange StrictTVar m (LedgerTablesHandleState l)
tv l EmptyMK
_ (Maybe (TxIn l)
f, Int
t) = do
  hs <- StrictTVar m (LedgerTablesHandleState l)
-> m (LedgerTablesHandleState l)
forall (m :: * -> *) a. MonadSTM m => StrictTVar m a -> m a
readTVarIO StrictTVar m (LedgerTablesHandleState l)
tv
  guardClosed
    hs
    ( \(LedgerTables (ValuesMK Map (TxIn l) (TxOut l)
m)) ->
        let m' :: Map (TxIn l) (TxOut l)
m' = Int -> Map (TxIn l) (TxOut l) -> Map (TxIn l) (TxOut l)
forall k a. Int -> Map k a -> Map k a
Map.take Int
t (Map (TxIn l) (TxOut l) -> Map (TxIn l) (TxOut l))
-> (Map (TxIn l) (TxOut l) -> Map (TxIn l) (TxOut l))
-> Map (TxIn l) (TxOut l)
-> Map (TxIn l) (TxOut l)
forall b c a. (b -> c) -> (a -> b) -> a -> c
. ((Map (TxIn l) (TxOut l) -> Map (TxIn l) (TxOut l))
-> (TxIn l -> Map (TxIn l) (TxOut l) -> Map (TxIn l) (TxOut l))
-> Maybe (TxIn l)
-> Map (TxIn l) (TxOut l)
-> Map (TxIn l) (TxOut l)
forall b a. b -> (a -> b) -> Maybe a -> b
maybe Map (TxIn l) (TxOut l) -> Map (TxIn l) (TxOut l)
forall a. a -> a
id (\TxIn l
g -> (Map (TxIn l) (TxOut l), Map (TxIn l) (TxOut l))
-> Map (TxIn l) (TxOut l)
forall a b. (a, b) -> b
snd ((Map (TxIn l) (TxOut l), Map (TxIn l) (TxOut l))
 -> Map (TxIn l) (TxOut l))
-> (Map (TxIn l) (TxOut l)
    -> (Map (TxIn l) (TxOut l), Map (TxIn l) (TxOut l)))
-> Map (TxIn l) (TxOut l)
-> Map (TxIn l) (TxOut l)
forall b c a. (b -> c) -> (a -> b) -> a -> c
. TxIn l
-> Map (TxIn l) (TxOut l)
-> (Map (TxIn l) (TxOut l), Map (TxIn l) (TxOut l))
forall k a. Ord k => k -> Map k a -> (Map k a, Map k a)
Map.split TxIn l
g) Maybe (TxIn l)
f) (Map (TxIn l) (TxOut l) -> Map (TxIn l) (TxOut l))
-> Map (TxIn l) (TxOut l) -> Map (TxIn l) (TxOut l)
forall a b. (a -> b) -> a -> b
$ Map (TxIn l) (TxOut l)
m
         in (LedgerTables l ValuesMK, Maybe (TxIn l))
-> m (LedgerTables l ValuesMK, Maybe (TxIn l))
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (ValuesMK (TxIn l) (TxOut l) -> LedgerTables l ValuesMK
forall (l :: LedgerStateKind) (mk :: * -> * -> *).
mk (TxIn l) (TxOut l) -> LedgerTables l mk
LedgerTables (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)
m'), (TxIn l, TxOut l) -> TxIn l
forall a b. (a, b) -> a
fst ((TxIn l, TxOut l) -> TxIn l)
-> Maybe (TxIn l, TxOut l) -> Maybe (TxIn l)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Map (TxIn l) (TxOut l) -> Maybe (TxIn l, TxOut l)
forall k a. Map k a -> Maybe (k, a)
Map.lookupMax Map (TxIn l) (TxOut l)
m')
    )

implReadAll ::
  IOLike m =>
  StrictTVar m (LedgerTablesHandleState l) ->
  l EmptyMK ->
  m (LedgerTables l ValuesMK)
implReadAll :: forall (m :: * -> *) (l :: LedgerStateKind).
IOLike m =>
StrictTVar m (LedgerTablesHandleState l)
-> l EmptyMK -> m (LedgerTables l ValuesMK)
implReadAll StrictTVar m (LedgerTablesHandleState l)
tv l EmptyMK
_ = do
  hs <- StrictTVar m (LedgerTablesHandleState l)
-> m (LedgerTablesHandleState l)
forall (m :: * -> *) a. MonadSTM m => StrictTVar m a -> m a
readTVarIO StrictTVar m (LedgerTablesHandleState l)
tv
  guardClosed hs pure

implPushDiffs ::
  ( IOLike m
  , HasLedgerTables l
  , CanUpgradeLedgerTables l
  ) =>
  StrictTVar m (LedgerTablesHandleState l) ->
  l mk1 ->
  l DiffMK ->
  m ()
implPushDiffs :: forall (m :: * -> *) (l :: LedgerStateKind) (mk1 :: * -> * -> *).
(IOLike m, HasLedgerTables l, CanUpgradeLedgerTables l) =>
StrictTVar m (LedgerTablesHandleState l)
-> l mk1 -> l DiffMK -> m ()
implPushDiffs StrictTVar m (LedgerTablesHandleState l)
tv l mk1
st0 !l DiffMK
diffs =
  STM m () -> m ()
forall a. HasCallStack => STM m a -> m a
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
STM m a -> m a
atomically (STM m () -> m ()) -> STM m () -> m ()
forall a b. (a -> b) -> a -> b
$
    StrictTVar m (LedgerTablesHandleState l)
-> (LedgerTablesHandleState l -> LedgerTablesHandleState l)
-> STM m ()
forall (m :: * -> *) a.
MonadSTM m =>
StrictTVar m a -> (a -> a) -> STM m ()
modifyTVar
      StrictTVar m (LedgerTablesHandleState l)
tv
      ( \LedgerTablesHandleState l
r ->
          LedgerTablesHandleState l
-> (LedgerTables l ValuesMK -> LedgerTablesHandleState l)
-> LedgerTablesHandleState l
forall (l :: LedgerStateKind) a.
LedgerTablesHandleState l -> (LedgerTables l ValuesMK -> a) -> a
guardClosed
            LedgerTablesHandleState l
r
            ( LedgerTables l ValuesMK -> LedgerTablesHandleState l
forall (l :: LedgerStateKind).
LedgerTables l ValuesMK -> LedgerTablesHandleState l
LedgerTablesHandleOpen
                (LedgerTables l ValuesMK -> LedgerTablesHandleState l)
-> (LedgerTables l ValuesMK -> LedgerTables l ValuesMK)
-> LedgerTables l ValuesMK
-> LedgerTablesHandleState l
forall b c a. (b -> c) -> (a -> b) -> a -> c
. (LedgerTables l ValuesMK
 -> LedgerTables l DiffMK -> LedgerTables l ValuesMK)
-> LedgerTables l DiffMK
-> LedgerTables l ValuesMK
-> LedgerTables l ValuesMK
forall a b c. (a -> b -> c) -> b -> a -> c
flip
                  ((forall k v.
 LedgerTableConstraints' l k v =>
 ValuesMK k v -> DiffMK k v -> ValuesMK k v)
-> LedgerTables l ValuesMK
-> LedgerTables l DiffMK
-> LedgerTables l ValuesMK
forall (l :: LedgerStateKind) (mk1 :: * -> * -> *)
       (mk2 :: * -> * -> *) (mk3 :: * -> * -> *).
LedgerTableConstraints l =>
(forall k v.
 LedgerTableConstraints' l k v =>
 mk1 k v -> mk2 k v -> mk3 k v)
-> LedgerTables l mk1 -> LedgerTables l mk2 -> LedgerTables l mk3
ltliftA2 (\(ValuesMK Map k v
vals) (DiffMK Diff k v
d) -> Map k v -> ValuesMK k v
forall k v. Map k v -> ValuesMK k v
ValuesMK (Map k v -> Diff k v -> Map k v
forall k v. Ord k => Map k v -> Diff k v -> Map k v
Diff.applyDiff Map k v
vals Diff k v
d)))
                  (l DiffMK -> LedgerTables l DiffMK
forall (mk :: * -> * -> *).
(CanMapMK mk, CanMapKeysMK mk, ZeroableMK mk) =>
l mk -> LedgerTables l mk
forall (l :: LedgerStateKind) (mk :: * -> * -> *).
(HasLedgerTables l, CanMapMK mk, CanMapKeysMK mk, ZeroableMK mk) =>
l mk -> LedgerTables l mk
projectLedgerTables l DiffMK
diffs)
                (LedgerTables l ValuesMK -> LedgerTables l ValuesMK)
-> (LedgerTables l ValuesMK -> LedgerTables l ValuesMK)
-> LedgerTables l ValuesMK
-> LedgerTables l ValuesMK
forall b c a. (b -> c) -> (a -> b) -> a -> c
. l mk1
-> l DiffMK -> LedgerTables l ValuesMK -> LedgerTables l ValuesMK
forall (mk1 :: * -> * -> *) (mk2 :: * -> * -> *).
l mk1
-> l mk2 -> LedgerTables l ValuesMK -> LedgerTables l ValuesMK
forall (l :: LedgerStateKind) (mk1 :: * -> * -> *)
       (mk2 :: * -> * -> *).
CanUpgradeLedgerTables l =>
l mk1
-> l mk2 -> LedgerTables l ValuesMK -> LedgerTables l ValuesMK
upgradeTables l mk1
st0 l DiffMK
diffs
            )
      )

implTakeHandleSnapshot ::
  (IOLike m, SerializeTablesWithHint l) =>
  StrictTVar m (LedgerTablesHandleState l) ->
  HasFS m h ->
  l EmptyMK ->
  String ->
  m (Maybe CRC)
implTakeHandleSnapshot :: forall (m :: * -> *) (l :: LedgerStateKind) h.
(IOLike m, SerializeTablesWithHint l) =>
StrictTVar m (LedgerTablesHandleState l)
-> HasFS m h -> l EmptyMK -> String -> m (Maybe CRC)
implTakeHandleSnapshot StrictTVar m (LedgerTablesHandleState l)
tv HasFS m h
hasFS l EmptyMK
hint String
snapshotName = 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
$ Context -> FsPath
mkFsPath [String
snapshotName]
  h <- StrictTVar m (LedgerTablesHandleState l)
-> m (LedgerTablesHandleState l)
forall (m :: * -> *) a. MonadSTM m => StrictTVar m a -> m a
readTVarIO StrictTVar m (LedgerTablesHandleState l)
tv
  guardClosed h $
    \LedgerTables l ValuesMK
values ->
      HasFS m h
-> FsPath
-> OpenMode
-> (Handle h -> m (Maybe CRC))
-> m (Maybe CRC)
forall (m :: * -> *) h a.
(HasCallStack, MonadThrow m) =>
HasFS m h -> FsPath -> OpenMode -> (Handle h -> m a) -> m a
withFile HasFS m h
hasFS (Context -> FsPath
mkFsPath [String
snapshotName, String
"tables"]) (AllowExisting -> OpenMode
WriteMode AllowExisting
MustBeNew) ((Handle h -> m (Maybe CRC)) -> m (Maybe CRC))
-> (Handle h -> m (Maybe CRC)) -> m (Maybe CRC)
forall a b. (a -> b) -> a -> b
$ \Handle h
hf ->
        ((Word64, CRC) -> Maybe CRC) -> m (Word64, CRC) -> m (Maybe CRC)
forall a b. (a -> b) -> m a -> m b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap (CRC -> Maybe CRC
forall a. a -> Maybe a
Just (CRC -> Maybe CRC)
-> ((Word64, CRC) -> CRC) -> (Word64, CRC) -> Maybe CRC
forall b c a. (b -> c) -> (a -> b) -> a -> c
. (Word64, CRC) -> CRC
forall a b. (a, b) -> b
snd) (m (Word64, CRC) -> m (Maybe CRC))
-> m (Word64, CRC) -> m (Maybe CRC)
forall a b. (a -> b) -> a -> b
$
          HasFS m h -> Handle h -> ByteString -> m (Word64, CRC)
forall (m :: * -> *) h.
(HasCallStack, Monad m) =>
HasFS m h -> Handle h -> ByteString -> m (Word64, CRC)
hPutAllCRC HasFS m h
hasFS Handle h
hf (ByteString -> m (Word64, CRC)) -> ByteString -> m (Word64, CRC)
forall a b. (a -> b) -> a -> b
$
            Encoding -> ByteString
CBOR.toLazyByteString (Encoding -> ByteString) -> Encoding -> ByteString
forall a b. (a -> b) -> a -> b
$
              l EmptyMK -> LedgerTables l ValuesMK -> Encoding
forall (l :: LedgerStateKind).
SerializeTablesWithHint l =>
l EmptyMK -> LedgerTables l ValuesMK -> Encoding
valuesMKEncoder l EmptyMK
hint LedgerTables l ValuesMK
values

implTablesSize ::
  IOLike m =>
  StrictTVar m (LedgerTablesHandleState l) ->
  m (Maybe Int)
implTablesSize :: forall (m :: * -> *) (l :: LedgerStateKind).
IOLike m =>
StrictTVar m (LedgerTablesHandleState l) -> m (Maybe Int)
implTablesSize StrictTVar m (LedgerTablesHandleState l)
tv = do
  hs <- StrictTVar m (LedgerTablesHandleState l)
-> m (LedgerTablesHandleState l)
forall (m :: * -> *) a. MonadSTM m => StrictTVar m a -> m a
readTVarIO StrictTVar m (LedgerTablesHandleState l)
tv
  guardClosed hs (pure . Just . Map.size . getValuesMK . getLedgerTables)

{-------------------------------------------------------------------------------
  Snapshots
-------------------------------------------------------------------------------}

snapshotManager ::
  ( IOLike m
  , LedgerDbSerialiseConstraints blk
  , LedgerSupportsProtocol blk
  ) =>
  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) =>
CodecConfig blk
-> Tracer m (TraceSnapshotEvent blk)
-> SomeHasFS m
-> SnapshotManager m m blk (StateRef m (ExtLedgerState blk))
snapshotManager 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 = SomeHasFS m
-> Tracer m (TraceSnapshotEvent blk) -> DiskSnapshot -> m ()
forall (m :: * -> *) blk.
(Monad m, HasCallStack) =>
SomeHasFS m
-> Tracer m (TraceSnapshotEvent blk) -> DiskSnapshot -> m ()
defaultDeleteSnapshot 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 #-}
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 = do
  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 :: LedgerStateKind) (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 :: LedgerStateKind).
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 :: LedgerStateKind).
StateRef m l -> l EmptyMK
state StateRef m (ExtLedgerState blk)
st
    crc2 <- takeHandleSnapshot (tables st) (state st) $ snapshotToDirName ds
    writeSnapshotMetadata shfs ds $
      SnapshotMetadata
        { snapshotBackend = UTxOHDMemSnapshot
        , snapshotChecksum = maybe crc1 (crcOfConcat crc1) crc2
        , snapshotTablesCodecVersion = TablesCodecVersion1
        }

-- | 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
  , LedgerSupportsInMemoryLedgerDB (LedgerState blk)
  ) =>
  Tracer m LedgerDBV2Trace ->
  ResourceRegistry m ->
  CodecConfig blk ->
  SomeHasFS m ->
  DiskSnapshot ->
  ExceptT (SnapshotFailure blk) m (LedgerSeq' m blk, RealPoint blk)
loadSnapshot :: forall blk (m :: * -> *).
(LedgerDbSerialiseConstraints blk, LedgerSupportsProtocol blk,
 IOLike m, LedgerSupportsInMemoryLedgerDB (LedgerState blk)) =>
Tracer m LedgerDBV2Trace
-> ResourceRegistry m
-> CodecConfig blk
-> SomeHasFS 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 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 /= UTxOHDMemSnapshot) $ do
    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, Identity crcTables) <-
        (ReadIncrementalErr -> SnapshotFailure blk)
-> ExceptT
     ReadIncrementalErr
     m
     (LedgerTables (ExtLedgerState blk) ValuesMK, Identity CRC)
-> ExceptT
     (SnapshotFailure blk)
     m
     (LedgerTables (ExtLedgerState blk) ValuesMK, Identity CRC)
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)
-> (ReadIncrementalErr -> ReadSnapshotErr)
-> ReadIncrementalErr
-> SnapshotFailure blk
forall b c a. (b -> c) -> (a -> b) -> a -> c
. ReadIncrementalErr -> ReadSnapshotErr
ReadSnapshotFailed) (ExceptT
   ReadIncrementalErr
   m
   (LedgerTables (ExtLedgerState blk) ValuesMK, Identity CRC)
 -> ExceptT
      (SnapshotFailure blk)
      m
      (LedgerTables (ExtLedgerState blk) ValuesMK, Identity CRC))
-> ExceptT
     ReadIncrementalErr
     m
     (LedgerTables (ExtLedgerState blk) ValuesMK, Identity CRC)
-> ExceptT
     (SnapshotFailure blk)
     m
     (LedgerTables (ExtLedgerState blk) ValuesMK, Identity CRC)
forall a b. (a -> b) -> a -> b
$
          m (Either
     ReadIncrementalErr
     (LedgerTables (ExtLedgerState blk) ValuesMK, Identity CRC))
-> ExceptT
     ReadIncrementalErr
     m
     (LedgerTables (ExtLedgerState blk) ValuesMK, Identity CRC)
forall e (m :: * -> *) a. m (Either e a) -> ExceptT e m a
ExceptT (m (Either
      ReadIncrementalErr
      (LedgerTables (ExtLedgerState blk) ValuesMK, Identity CRC))
 -> ExceptT
      ReadIncrementalErr
      m
      (LedgerTables (ExtLedgerState blk) ValuesMK, Identity CRC))
-> m (Either
        ReadIncrementalErr
        (LedgerTables (ExtLedgerState blk) ValuesMK, Identity CRC))
-> ExceptT
     ReadIncrementalErr
     m
     (LedgerTables (ExtLedgerState blk) ValuesMK, Identity CRC)
forall a b. (a -> b) -> a -> b
$
            SomeHasFS m
-> (CRC -> Identity CRC)
-> Decoder
     (PrimState m) (LedgerTables (ExtLedgerState blk) ValuesMK)
-> FsPath
-> m (Either
        ReadIncrementalErr
        (LedgerTables (ExtLedgerState blk) ValuesMK, Identity CRC))
forall (m :: * -> *) (f :: * -> *) a.
(IOLike m, Functor f) =>
SomeHasFS m
-> (CRC -> f CRC)
-> Decoder (PrimState m) a
-> FsPath
-> m (Either ReadIncrementalErr (a, f CRC))
readIncremental
              SomeHasFS m
fs
              CRC -> Identity CRC
forall a. a -> Identity a
Identity
              (ExtLedgerState blk EmptyMK
-> Decoder
     (PrimState m) (LedgerTables (ExtLedgerState blk) ValuesMK)
forall (l :: LedgerStateKind) s.
SerializeTablesWithHint l =>
l EmptyMK -> Decoder s (LedgerTables l ValuesMK)
valuesMKDecoder ExtLedgerState blk EmptyMK
extLedgerSt)
              (DiskSnapshot -> FsPath
snapshotToDirPath DiskSnapshot
ds FsPath -> FsPath -> FsPath
</> Context -> FsPath
mkFsPath [String
"tables"])
      let computedCRC = CRC -> CRC -> CRC
crcOfConcat CRC
checksumAsRead CRC
crcTables
      Monad.when (computedCRC /= snapshotChecksum snapshotMeta) $
        throwE $
          InitFailureRead $
            ReadSnapshotDataCorruption
      (,pt) <$> lift (empty extLedgerSt values (newInMemoryLedgerTablesHandle tracer fs))

type data Mem

instance
  ( IOLike m
  , LedgerDbSerialiseConstraints blk
  , LedgerSupportsProtocol blk
  , LedgerSupportsInMemoryLedgerDB (LedgerState blk)
  ) =>
  Backend m Mem blk
  where
  data Args m Mem = InMemArgs
  newtype Resources m Mem = Resources (SomeHasFS m)
    deriving newtype Context -> Resources m Mem -> IO (Maybe ThunkInfo)
Proxy (Resources m Mem) -> String
(Context -> Resources m Mem -> IO (Maybe ThunkInfo))
-> (Context -> Resources m Mem -> IO (Maybe ThunkInfo))
-> (Proxy (Resources m Mem) -> String)
-> NoThunks (Resources m Mem)
forall a.
(Context -> a -> IO (Maybe ThunkInfo))
-> (Context -> a -> IO (Maybe ThunkInfo))
-> (Proxy a -> String)
-> NoThunks a
forall (m :: * -> *).
Context -> Resources m Mem -> IO (Maybe ThunkInfo)
forall (m :: * -> *). Proxy (Resources m Mem) -> String
$cnoThunks :: forall (m :: * -> *).
Context -> Resources m Mem -> IO (Maybe ThunkInfo)
noThunks :: Context -> Resources m Mem -> IO (Maybe ThunkInfo)
$cwNoThunks :: forall (m :: * -> *).
Context -> Resources m Mem -> IO (Maybe ThunkInfo)
wNoThunks :: Context -> Resources m Mem -> IO (Maybe ThunkInfo)
$cshowTypeOf :: forall (m :: * -> *). Proxy (Resources m Mem) -> String
showTypeOf :: Proxy (Resources m Mem) -> String
NoThunks
  newtype Trace m Mem = NoTrace Void
    deriving newtype Int -> Trace m Mem -> ShowS
[Trace m Mem] -> ShowS
Trace m Mem -> String
(Int -> Trace m Mem -> ShowS)
-> (Trace m Mem -> String)
-> ([Trace m Mem] -> ShowS)
-> Show (Trace m Mem)
forall a.
(Int -> a -> ShowS) -> (a -> String) -> ([a] -> ShowS) -> Show a
forall (m :: * -> *). Int -> Trace m Mem -> ShowS
forall (m :: * -> *). [Trace m Mem] -> ShowS
forall (m :: * -> *). Trace m Mem -> String
$cshowsPrec :: forall (m :: * -> *). Int -> Trace m Mem -> ShowS
showsPrec :: Int -> Trace m Mem -> ShowS
$cshow :: forall (m :: * -> *). Trace m Mem -> String
show :: Trace m Mem -> String
$cshowList :: forall (m :: * -> *). [Trace m Mem] -> ShowS
showList :: [Trace m Mem] -> ShowS
Show

  mkResources :: Proxy blk
-> Tracer m LedgerDBV2Trace
-> Args m Mem
-> ResourceRegistry m
-> SomeHasFS m
-> m (Resources m Mem)
mkResources Proxy blk
_ Tracer m LedgerDBV2Trace
_ Args m Mem
_ ResourceRegistry m
_ = Resources m Mem -> m (Resources m Mem)
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (Resources m Mem -> m (Resources m Mem))
-> (SomeHasFS m -> Resources m Mem)
-> SomeHasFS m
-> m (Resources m Mem)
forall b c a. (b -> c) -> (a -> b) -> a -> c
. SomeHasFS m -> Resources m Mem
forall (m :: * -> *). SomeHasFS m -> Resources m Mem
Resources
  releaseResources :: Proxy blk -> Resources m Mem -> m ()
releaseResources Proxy blk
_ Resources m Mem
_ = () -> m ()
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure ()
  newHandleFromValues :: Tracer m LedgerDBV2Trace
-> ResourceRegistry m
-> Resources m Mem
-> ExtLedgerState blk ValuesMK
-> m (LedgerTablesHandle m (ExtLedgerState blk))
newHandleFromValues Tracer m LedgerDBV2Trace
tracer ResourceRegistry m
_ (Resources SomeHasFS m
shfs) =
    Tracer m LedgerDBV2Trace
-> SomeHasFS m
-> LedgerTables (ExtLedgerState blk) ValuesMK
-> m (LedgerTablesHandle m (ExtLedgerState blk))
forall (m :: * -> *) (l :: LedgerStateKind).
(IOLike m, HasLedgerTables l, CanUpgradeLedgerTables l,
 SerializeTablesWithHint l) =>
Tracer m LedgerDBV2Trace
-> SomeHasFS m
-> LedgerTables l ValuesMK
-> m (LedgerTablesHandle m l)
newInMemoryLedgerTablesHandle Tracer m LedgerDBV2Trace
tracer SomeHasFS m
shfs (LedgerTables (ExtLedgerState blk) ValuesMK
 -> m (LedgerTablesHandle m (ExtLedgerState blk)))
-> (ExtLedgerState blk ValuesMK
    -> LedgerTables (ExtLedgerState blk) ValuesMK)
-> ExtLedgerState blk ValuesMK
-> m (LedgerTablesHandle m (ExtLedgerState blk))
forall b c a. (b -> c) -> (a -> b) -> a -> c
. ExtLedgerState blk ValuesMK
-> LedgerTables (ExtLedgerState blk) ValuesMK
forall (l :: LedgerStateKind) (l' :: LedgerStateKind)
       (mk :: * -> * -> *).
(HasLedgerTables l, SameUtxoTypes l l', CanMapMK mk,
 CanMapKeysMK mk, ZeroableMK mk) =>
l mk -> LedgerTables l' mk
ltprj
  newHandleFromSnapshot :: Tracer m LedgerDBV2Trace
-> ResourceRegistry m
-> CodecConfig blk
-> SomeHasFS m
-> Resources m Mem
-> 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 Mem
_ DiskSnapshot
ds =
    Tracer m LedgerDBV2Trace
-> ResourceRegistry m
-> CodecConfig blk
-> SomeHasFS m
-> DiskSnapshot
-> ExceptT
     (SnapshotFailure blk) m (LedgerSeq' m blk, RealPoint blk)
forall blk (m :: * -> *).
(LedgerDbSerialiseConstraints blk, LedgerSupportsProtocol blk,
 IOLike m, LedgerSupportsInMemoryLedgerDB (LedgerState blk)) =>
Tracer m LedgerDBV2Trace
-> ResourceRegistry m
-> CodecConfig blk
-> SomeHasFS 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 DiskSnapshot
ds
  snapshotManager :: Proxy blk
-> Resources m Mem
-> CodecConfig blk
-> Tracer m (TraceSnapshotEvent blk)
-> SomeHasFS m
-> SnapshotManager m m blk (StateRef m (ExtLedgerState blk))
snapshotManager Proxy blk
_ Resources m Mem
_ =
    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) =>
CodecConfig blk
-> Tracer m (TraceSnapshotEvent blk)
-> SomeHasFS m
-> SnapshotManager m m blk (StateRef m (ExtLedgerState blk))
Ouroboros.Consensus.Storage.LedgerDB.V2.InMemory.snapshotManager

-- | Create arguments for initializing the LedgerDB using the InMemory backend.
mkInMemoryArgs ::
  ( IOLike m
  , LedgerDbSerialiseConstraints blk
  , LedgerSupportsProtocol blk
  , LedgerSupportsInMemoryLedgerDB (LedgerState blk)
  ) =>
  a -> (LedgerDbBackendArgs m blk, a)
mkInMemoryArgs :: forall (m :: * -> *) blk a.
(IOLike m, LedgerDbSerialiseConstraints blk,
 LedgerSupportsProtocol blk,
 LedgerSupportsInMemoryLedgerDB (LedgerState blk)) =>
a -> (LedgerDbBackendArgs m blk, a)
mkInMemoryArgs = (,) (LedgerDbBackendArgs m blk -> a -> (LedgerDbBackendArgs m blk, a))
-> LedgerDbBackendArgs m blk -> a -> (LedgerDbBackendArgs m blk, a)
forall a b. (a -> b) -> a -> b
$ SomeBackendArgs m blk -> LedgerDbBackendArgs m blk
forall (m :: * -> *) blk.
SomeBackendArgs m blk -> LedgerDbBackendArgs m blk
LedgerDbBackendArgsV2 (SomeBackendArgs m blk -> LedgerDbBackendArgs m blk)
-> SomeBackendArgs m blk -> LedgerDbBackendArgs m blk
forall a b. (a -> b) -> a -> b
$ Args m Mem -> SomeBackendArgs m blk
forall (m :: * -> *) backend blk.
Backend m backend blk =>
Args m backend -> SomeBackendArgs m blk
SomeBackendArgs Args m Mem
forall (m :: * -> *). Args m Mem
InMemArgs

instance IOLike m => StreamingBackend m Mem l where
  data YieldArgs m Mem l
    = -- \| Yield an in-memory snapshot
      YieldInMemory
        -- \| How to make a SomeHasFS for @m@
        (MountPoint -> SomeHasFS m)
        -- \| The file path at which the HasFS has to be opened
        FilePath
        (Decoders l)

  data SinkArgs m Mem l
    = SinkInMemory
        Int
        (TxIn l -> Encoding)
        (TxOut l -> Encoding)
        (SomeHasFS m)
        FilePath

  yield :: Proxy Mem -> YieldArgs m Mem l -> Yield m l
yield Proxy Mem
_ (YieldInMemory MountPoint -> SomeHasFS m
mkFs String
fp (Decoders forall s. Decoder s (TxIn l)
decK forall s. Decoder s (TxOut l)
decV)) =
    (MountPoint -> SomeHasFS m)
-> String
-> (forall s. Decoder s (TxIn l))
-> (forall s. Decoder s (TxOut l))
-> Yield m l
forall (m :: * -> *) (l :: LedgerStateKind).
(MonadThrow m, MonadST m) =>
(MountPoint -> SomeHasFS m)
-> String
-> (forall s. Decoder s (TxIn l))
-> (forall s. Decoder s (TxOut l))
-> Yield m l
yieldInMemoryS MountPoint -> SomeHasFS m
mkFs String
fp Decoder s (TxIn l)
forall s. Decoder s (TxIn l)
decK Decoder s (TxOut l)
forall s. Decoder s (TxOut l)
decV

  sink :: Proxy Mem -> SinkArgs m Mem l -> Sink m l
sink Proxy Mem
_ (SinkInMemory Int
chunkSize TxIn l -> Encoding
encK TxOut l -> Encoding
encV SomeHasFS m
shfs String
fp) =
    Int
-> (TxIn l -> Encoding)
-> (TxOut l -> Encoding)
-> SomeHasFS m
-> String
-> Sink m l
forall (m :: * -> *) (l :: LedgerStateKind).
MonadThrow m =>
Int
-> (TxIn l -> Encoding)
-> (TxOut l -> Encoding)
-> SomeHasFS m
-> String
-> Sink m l
sinkInMemoryS Int
chunkSize TxIn l -> Encoding
encK TxOut l -> Encoding
encV SomeHasFS m
shfs String
fp

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

streamingFile ::
  forall m.
  MonadThrow m =>
  SomeHasFS m ->
  FsPath ->
  ( Stream (Of ByteString) m (Maybe CRC) ->
    ExceptT DeserialiseFailure m (Stream (Of ByteString) m (Maybe CRC, Maybe CRC))
  ) ->
  ExceptT DeserialiseFailure m (Maybe CRC, Maybe CRC)
streamingFile :: forall (m :: * -> *).
MonadThrow m =>
SomeHasFS m
-> FsPath
-> (Stream (Of ByteString) m (Maybe CRC)
    -> ExceptT
         DeserialiseFailure
         m
         (Stream (Of ByteString) m (Maybe CRC, Maybe CRC)))
-> ExceptT DeserialiseFailure m (Maybe CRC, Maybe CRC)
streamingFile (SomeHasFS HasFS m h
fs') FsPath
path Stream (Of ByteString) m (Maybe CRC)
-> ExceptT
     DeserialiseFailure
     m
     (Stream (Of ByteString) m (Maybe CRC, Maybe CRC))
cont =
  m (Either DeserialiseFailure (Maybe CRC, Maybe CRC))
-> ExceptT DeserialiseFailure m (Maybe CRC, Maybe CRC)
forall e (m :: * -> *) a. m (Either e a) -> ExceptT e m a
ExceptT (m (Either DeserialiseFailure (Maybe CRC, Maybe CRC))
 -> ExceptT DeserialiseFailure m (Maybe CRC, Maybe CRC))
-> m (Either DeserialiseFailure (Maybe CRC, Maybe CRC))
-> ExceptT DeserialiseFailure m (Maybe CRC, Maybe CRC)
forall a b. (a -> b) -> a -> b
$ HasFS m h
-> FsPath
-> OpenMode
-> (Handle h
    -> m (Either DeserialiseFailure (Maybe CRC, Maybe CRC)))
-> m (Either DeserialiseFailure (Maybe CRC, Maybe CRC))
forall (m :: * -> *) h a.
(HasCallStack, MonadThrow m) =>
HasFS m h -> FsPath -> OpenMode -> (Handle h -> m a) -> m a
withFile HasFS m h
fs' FsPath
path OpenMode
ReadMode ((Handle h -> m (Either DeserialiseFailure (Maybe CRC, Maybe CRC)))
 -> m (Either DeserialiseFailure (Maybe CRC, Maybe CRC)))
-> (Handle h
    -> m (Either DeserialiseFailure (Maybe CRC, Maybe CRC)))
-> m (Either DeserialiseFailure (Maybe CRC, Maybe CRC))
forall a b. (a -> b) -> a -> b
$ \Handle h
hdl ->
    ExceptT DeserialiseFailure m (Maybe CRC, Maybe CRC)
-> m (Either DeserialiseFailure (Maybe CRC, Maybe CRC))
forall e (m :: * -> *) a. ExceptT e m a -> m (Either e a)
runExceptT (ExceptT DeserialiseFailure m (Maybe CRC, Maybe CRC)
 -> m (Either DeserialiseFailure (Maybe CRC, Maybe CRC)))
-> ExceptT DeserialiseFailure m (Maybe CRC, Maybe CRC)
-> m (Either DeserialiseFailure (Maybe CRC, Maybe CRC))
forall a b. (a -> b) -> a -> b
$ Stream (Of ByteString) m (Maybe CRC)
-> ExceptT
     DeserialiseFailure
     m
     (Stream (Of ByteString) m (Maybe CRC, Maybe CRC))
cont (Handle h -> CRC -> Stream (Of ByteString) m (Maybe CRC)
getBS Handle h
hdl CRC
initCRC) ExceptT
  DeserialiseFailure
  m
  (Stream (Of ByteString) m (Maybe CRC, Maybe CRC))
-> (Stream (Of ByteString) m (Maybe CRC, Maybe CRC)
    -> ExceptT DeserialiseFailure m (Maybe CRC, Maybe CRC))
-> ExceptT DeserialiseFailure m (Maybe CRC, Maybe CRC)
forall a b.
ExceptT DeserialiseFailure m a
-> (a -> ExceptT DeserialiseFailure m b)
-> ExceptT DeserialiseFailure m b
forall (m :: * -> *) a b. Monad m => m a -> (a -> m b) -> m b
>>= Stream (Of ByteString) m (Maybe CRC, Maybe CRC)
-> ExceptT DeserialiseFailure m (Maybe CRC, Maybe CRC)
forall {t :: (* -> *) -> * -> *} {m :: * -> *} {b}.
(MonadTrans t, Monad m, MonadError DeserialiseFailure (t m)) =>
Stream (Of ByteString) m b -> t m b
noRemainingBytes
 where
  getBS :: Handle h -> CRC -> Stream (Of ByteString) m (Maybe CRC)
getBS Handle h
h !CRC
crc = do
    bs <- m ByteString -> Stream (Of ByteString) m ByteString
forall (m :: * -> *) a.
Monad m =>
m a -> Stream (Of ByteString) m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
S.lift (m ByteString -> Stream (Of ByteString) m ByteString)
-> m ByteString -> Stream (Of ByteString) m ByteString
forall a b. (a -> b) -> a -> b
$ HasFS m h -> HasCallStack => Handle h -> Word64 -> m ByteString
forall (m :: * -> *) h.
HasFS m h -> HasCallStack => Handle h -> Word64 -> m ByteString
hGetSome HasFS m h
fs' Handle h
h (Int -> Word64
forall a b. (Integral a, Num b) => a -> b
fromIntegral Int
defaultChunkSize)
    if BS.null bs
      then pure (Just crc)
      else do
        S.yield bs
        getBS h $! updateCRC bs crc

  noRemainingBytes :: Stream (Of ByteString) m b -> t m b
noRemainingBytes Stream (Of ByteString) m b
s =
    m (Maybe (ByteString, Stream (Of ByteString) m b))
-> t m (Maybe (ByteString, Stream (Of ByteString) m b))
forall (m :: * -> *) a. Monad m => m a -> t m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (Stream (Of ByteString) m b
-> m (Maybe (ByteString, Stream (Of ByteString) m b))
forall (m :: * -> *) a r.
Monad m =>
Stream (Of a) m r -> m (Maybe (a, Stream (Of a) m r))
S.uncons Stream (Of ByteString) m b
s) t m (Maybe (ByteString, Stream (Of ByteString) m b))
-> (Maybe (ByteString, Stream (Of ByteString) m b) -> t m b)
-> t m b
forall a b. t m a -> (a -> t m b) -> t m b
forall (m :: * -> *) a b. Monad m => m a -> (a -> m b) -> m b
>>= \case
      Maybe (ByteString, Stream (Of ByteString) m b)
Nothing -> m b -> t m b
forall (m :: * -> *) a. Monad m => m a -> t m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (m b -> t m b) -> m b -> t m b
forall a b. (a -> b) -> a -> b
$ Stream (Of ByteString) m b -> m b
forall (m :: * -> *) a r. Monad m => Stream (Of a) m r -> m r
S.effects Stream (Of ByteString) m b
s
      Just (ByteString -> Bool
BS.null -> Bool
True, Stream (Of ByteString) m b
s') -> Stream (Of ByteString) m b -> t m b
noRemainingBytes Stream (Of ByteString) m b
s'
      Just (ByteString, Stream (Of ByteString) m b)
_ -> DeserialiseFailure -> t m b
forall a. DeserialiseFailure -> t m a
forall e (m :: * -> *) a. MonadError e m => e -> m a
throwError (DeserialiseFailure -> t m b) -> DeserialiseFailure -> t m b
forall a b. (a -> b) -> a -> b
$ ByteOffset -> String -> DeserialiseFailure
DeserialiseFailure ByteOffset
0 String
"Remaining bytes"

yieldCborMapS ::
  forall m a b.
  MonadST m =>
  (forall s. Decoder s a) ->
  (forall s. Decoder s b) ->
  Stream (Of ByteString) m (Maybe CRC) ->
  Stream (Of (a, b)) (ExceptT DeserialiseFailure m) (Stream (Of ByteString) m (Maybe CRC))
yieldCborMapS :: forall (m :: * -> *) a b.
MonadST m =>
(forall s. Decoder s a)
-> (forall s. Decoder s b)
-> Stream (Of ByteString) m (Maybe CRC)
-> Stream
     (Of (a, b))
     (ExceptT DeserialiseFailure m)
     (Stream (Of ByteString) m (Maybe CRC))
yieldCborMapS forall s. Decoder s a
decK forall s. Decoder s b
decV = StateT
  (Stream (Of ByteString) m (Maybe CRC))
  (Stream (Of (a, b)) (ExceptT DeserialiseFailure m))
  ()
-> Stream (Of ByteString) m (Maybe CRC)
-> Stream
     (Of (a, b))
     (ExceptT DeserialiseFailure m)
     (Stream (Of ByteString) m (Maybe CRC))
forall (m :: * -> *) s a. Monad m => StateT s m a -> s -> m s
execStateT (StateT
   (Stream (Of ByteString) m (Maybe CRC))
   (Stream (Of (a, b)) (ExceptT DeserialiseFailure m))
   ()
 -> Stream (Of ByteString) m (Maybe CRC)
 -> Stream
      (Of (a, b))
      (ExceptT DeserialiseFailure m)
      (Stream (Of ByteString) m (Maybe CRC)))
-> StateT
     (Stream (Of ByteString) m (Maybe CRC))
     (Stream (Of (a, b)) (ExceptT DeserialiseFailure m))
     ()
-> Stream (Of ByteString) m (Maybe CRC)
-> Stream
     (Of (a, b))
     (ExceptT DeserialiseFailure m)
     (Stream (Of ByteString) m (Maybe CRC))
forall a b. (a -> b) -> a -> b
$ do
  (forall a.
 ExceptT DeserialiseFailure m a
 -> Stream (Of (a, b)) (ExceptT DeserialiseFailure m) a)
-> StateT
     (Stream (Of ByteString) m (Maybe CRC))
     (ExceptT DeserialiseFailure m)
     (Maybe Int)
-> StateT
     (Stream (Of ByteString) m (Maybe CRC))
     (Stream (Of (a, b)) (ExceptT DeserialiseFailure m))
     (Maybe Int)
forall {k} (t :: (* -> *) -> k -> *) (m :: * -> *) (n :: * -> *)
       (b :: k).
(MFunctor t, Monad m) =>
(forall a. m a -> n a) -> t m b -> t n b
forall (m :: * -> *) (n :: * -> *) b.
Monad m =>
(forall a. m a -> n a)
-> StateT (Stream (Of ByteString) m (Maybe CRC)) m b
-> StateT (Stream (Of ByteString) m (Maybe CRC)) n b
hoist ExceptT DeserialiseFailure m a
-> Stream (Of (a, b)) (ExceptT DeserialiseFailure m) a
forall a.
ExceptT DeserialiseFailure m a
-> Stream (Of (a, b)) (ExceptT DeserialiseFailure m) a
forall (m :: * -> *) a. Monad m => m a -> Stream (Of (a, b)) m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (Decoder (PrimState m) Int
-> StateT
     (Stream (Of ByteString) m (Maybe CRC))
     (ExceptT DeserialiseFailure m)
     Int
forall {t :: (* -> *) -> * -> *} {m :: * -> *} {a} {b}.
(MonadTrans t, MonadST m, MonadError DeserialiseFailure (t m)) =>
Decoder (PrimState m) a
-> StateT (Stream (Of ByteString) m b) (t m) a
decodeCbor Decoder (PrimState m) Int
forall s. Decoder s Int
decodeListLen StateT
  (Stream (Of ByteString) m (Maybe CRC))
  (ExceptT DeserialiseFailure m)
  Int
-> StateT
     (Stream (Of ByteString) m (Maybe CRC))
     (ExceptT DeserialiseFailure m)
     (Maybe Int)
-> StateT
     (Stream (Of ByteString) m (Maybe CRC))
     (ExceptT DeserialiseFailure m)
     (Maybe Int)
forall a b.
StateT
  (Stream (Of ByteString) m (Maybe CRC))
  (ExceptT DeserialiseFailure m)
  a
-> StateT
     (Stream (Of ByteString) m (Maybe CRC))
     (ExceptT DeserialiseFailure m)
     b
-> StateT
     (Stream (Of ByteString) m (Maybe CRC))
     (ExceptT DeserialiseFailure m)
     b
forall (m :: * -> *) a b. Monad m => m a -> m b -> m b
>> Decoder (PrimState m) (Maybe Int)
-> StateT
     (Stream (Of ByteString) m (Maybe CRC))
     (ExceptT DeserialiseFailure m)
     (Maybe Int)
forall {t :: (* -> *) -> * -> *} {m :: * -> *} {a} {b}.
(MonadTrans t, MonadST m, MonadError DeserialiseFailure (t m)) =>
Decoder (PrimState m) a
-> StateT (Stream (Of ByteString) m b) (t m) a
decodeCbor Decoder (PrimState m) (Maybe Int)
forall s. Decoder s (Maybe Int)
decodeMapLenOrIndef) StateT
  (Stream (Of ByteString) m (Maybe CRC))
  (Stream (Of (a, b)) (ExceptT DeserialiseFailure m))
  (Maybe Int)
-> (Maybe Int
    -> StateT
         (Stream (Of ByteString) m (Maybe CRC))
         (Stream (Of (a, b)) (ExceptT DeserialiseFailure m))
         ())
-> StateT
     (Stream (Of ByteString) m (Maybe CRC))
     (Stream (Of (a, b)) (ExceptT DeserialiseFailure m))
     ()
forall a b.
StateT
  (Stream (Of ByteString) m (Maybe CRC))
  (Stream (Of (a, b)) (ExceptT DeserialiseFailure m))
  a
-> (a
    -> StateT
         (Stream (Of ByteString) m (Maybe CRC))
         (Stream (Of (a, b)) (ExceptT DeserialiseFailure m))
         b)
-> StateT
     (Stream (Of ByteString) m (Maybe CRC))
     (Stream (Of (a, b)) (ExceptT DeserialiseFailure m))
     b
forall (m :: * -> *) a b. Monad m => m a -> (a -> m b) -> m b
>>= \case
    Maybe Int
Nothing -> StateT
  (Stream (Of ByteString) m (Maybe CRC))
  (Stream (Of (a, b)) (ExceptT DeserialiseFailure m))
  ()
go
    Just Int
n -> Int
-> StateT
     (Stream (Of ByteString) m (Maybe CRC))
     (Stream (Of (a, b)) (ExceptT DeserialiseFailure m))
     ()
-> StateT
     (Stream (Of ByteString) m (Maybe CRC))
     (Stream (Of (a, b)) (ExceptT DeserialiseFailure m))
     ()
forall (m :: * -> *) a. Applicative m => Int -> m a -> m ()
replicateM_ Int
n StateT
  (Stream (Of ByteString) m (Maybe CRC))
  (Stream (Of (a, b)) (ExceptT DeserialiseFailure m))
  ()
yieldKV
 where
  yieldKV :: StateT
  (Stream (Of ByteString) m (Maybe CRC))
  (Stream (Of (a, b)) (ExceptT DeserialiseFailure m))
  ()
yieldKV = do
    kv <- (forall a.
 ExceptT DeserialiseFailure m a
 -> Stream (Of (a, b)) (ExceptT DeserialiseFailure m) a)
-> StateT
     (Stream (Of ByteString) m (Maybe CRC))
     (ExceptT DeserialiseFailure m)
     (a, b)
-> StateT
     (Stream (Of ByteString) m (Maybe CRC))
     (Stream (Of (a, b)) (ExceptT DeserialiseFailure m))
     (a, b)
forall {k} (t :: (* -> *) -> k -> *) (m :: * -> *) (n :: * -> *)
       (b :: k).
(MFunctor t, Monad m) =>
(forall a. m a -> n a) -> t m b -> t n b
forall (m :: * -> *) (n :: * -> *) b.
Monad m =>
(forall a. m a -> n a)
-> StateT (Stream (Of ByteString) m (Maybe CRC)) m b
-> StateT (Stream (Of ByteString) m (Maybe CRC)) n b
hoist ExceptT DeserialiseFailure m a
-> Stream (Of (a, b)) (ExceptT DeserialiseFailure m) a
forall a.
ExceptT DeserialiseFailure m a
-> Stream (Of (a, b)) (ExceptT DeserialiseFailure m) a
forall (m :: * -> *) a. Monad m => m a -> Stream (Of (a, b)) m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (StateT
   (Stream (Of ByteString) m (Maybe CRC))
   (ExceptT DeserialiseFailure m)
   (a, b)
 -> StateT
      (Stream (Of ByteString) m (Maybe CRC))
      (Stream (Of (a, b)) (ExceptT DeserialiseFailure m))
      (a, b))
-> StateT
     (Stream (Of ByteString) m (Maybe CRC))
     (ExceptT DeserialiseFailure m)
     (a, b)
-> StateT
     (Stream (Of ByteString) m (Maybe CRC))
     (Stream (Of (a, b)) (ExceptT DeserialiseFailure m))
     (a, b)
forall a b. (a -> b) -> a -> b
$ Decoder (PrimState m) (a, b)
-> StateT
     (Stream (Of ByteString) m (Maybe CRC))
     (ExceptT DeserialiseFailure m)
     (a, b)
forall {t :: (* -> *) -> * -> *} {m :: * -> *} {a} {b}.
(MonadTrans t, MonadST m, MonadError DeserialiseFailure (t m)) =>
Decoder (PrimState m) a
-> StateT (Stream (Of ByteString) m b) (t m) a
decodeCbor (Decoder (PrimState m) (a, b)
 -> StateT
      (Stream (Of ByteString) m (Maybe CRC))
      (ExceptT DeserialiseFailure m)
      (a, b))
-> Decoder (PrimState m) (a, b)
-> StateT
     (Stream (Of ByteString) m (Maybe CRC))
     (ExceptT DeserialiseFailure m)
     (a, b)
forall a b. (a -> b) -> a -> b
$ (,) (a -> b -> (a, b))
-> Decoder (PrimState m) a -> Decoder (PrimState m) (b -> (a, b))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Decoder (PrimState m) a
forall s. Decoder s a
decK Decoder (PrimState m) (b -> (a, b))
-> Decoder (PrimState m) b -> Decoder (PrimState m) (a, b)
forall a b.
Decoder (PrimState m) (a -> b)
-> Decoder (PrimState m) a -> Decoder (PrimState m) b
forall (f :: * -> *) a b. Applicative f => f (a -> b) -> f a -> f b
<*> Decoder (PrimState m) b
forall s. Decoder s b
decV
    lift $ S.yield kv

  go :: StateT
  (Stream (Of ByteString) m (Maybe CRC))
  (Stream (Of (a, b)) (ExceptT DeserialiseFailure m))
  ()
go = do
    doBreak <- (forall a.
 ExceptT DeserialiseFailure m a
 -> Stream (Of (a, b)) (ExceptT DeserialiseFailure m) a)
-> StateT
     (Stream (Of ByteString) m (Maybe CRC))
     (ExceptT DeserialiseFailure m)
     Bool
-> StateT
     (Stream (Of ByteString) m (Maybe CRC))
     (Stream (Of (a, b)) (ExceptT DeserialiseFailure m))
     Bool
forall {k} (t :: (* -> *) -> k -> *) (m :: * -> *) (n :: * -> *)
       (b :: k).
(MFunctor t, Monad m) =>
(forall a. m a -> n a) -> t m b -> t n b
forall (m :: * -> *) (n :: * -> *) b.
Monad m =>
(forall a. m a -> n a)
-> StateT (Stream (Of ByteString) m (Maybe CRC)) m b
-> StateT (Stream (Of ByteString) m (Maybe CRC)) n b
hoist ExceptT DeserialiseFailure m a
-> Stream (Of (a, b)) (ExceptT DeserialiseFailure m) a
forall a.
ExceptT DeserialiseFailure m a
-> Stream (Of (a, b)) (ExceptT DeserialiseFailure m) a
forall (m :: * -> *) a. Monad m => m a -> Stream (Of (a, b)) m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (StateT
   (Stream (Of ByteString) m (Maybe CRC))
   (ExceptT DeserialiseFailure m)
   Bool
 -> StateT
      (Stream (Of ByteString) m (Maybe CRC))
      (Stream (Of (a, b)) (ExceptT DeserialiseFailure m))
      Bool)
-> StateT
     (Stream (Of ByteString) m (Maybe CRC))
     (ExceptT DeserialiseFailure m)
     Bool
-> StateT
     (Stream (Of ByteString) m (Maybe CRC))
     (Stream (Of (a, b)) (ExceptT DeserialiseFailure m))
     Bool
forall a b. (a -> b) -> a -> b
$ Decoder (PrimState m) Bool
-> StateT
     (Stream (Of ByteString) m (Maybe CRC))
     (ExceptT DeserialiseFailure m)
     Bool
forall {t :: (* -> *) -> * -> *} {m :: * -> *} {a} {b}.
(MonadTrans t, MonadST m, MonadError DeserialiseFailure (t m)) =>
Decoder (PrimState m) a
-> StateT (Stream (Of ByteString) m b) (t m) a
decodeCbor Decoder (PrimState m) Bool
forall s. Decoder s Bool
decodeBreakOr
    unless doBreak $ yieldKV *> go

  decodeCbor :: Decoder (PrimState m) a
-> StateT (Stream (Of ByteString) m b) (t m) a
decodeCbor Decoder (PrimState m) a
dec =
    (Stream (Of ByteString) m b -> t m (a, Stream (Of ByteString) m b))
-> StateT (Stream (Of ByteString) m b) (t m) a
forall s (m :: * -> *) a. (s -> m (a, s)) -> StateT s m a
StateT ((Stream (Of ByteString) m b
  -> t m (a, Stream (Of ByteString) m b))
 -> StateT (Stream (Of ByteString) m b) (t m) a)
-> (Stream (Of ByteString) m b
    -> t m (a, Stream (Of ByteString) m b))
-> StateT (Stream (Of ByteString) m b) (t m) a
forall a b. (a -> b) -> a -> b
$ \Stream (Of ByteString) m b
s -> Stream (Of ByteString) m b
-> IDecode (PrimState m) a -> t m (a, Stream (Of ByteString) m b)
forall {t :: (* -> *) -> * -> *} {m :: * -> *} {b} {a}.
(MonadTrans t, MonadST m, MonadError DeserialiseFailure (t m)) =>
Stream (Of ByteString) m b
-> IDecode (PrimState m) a -> t m (a, Stream (Of ByteString) m b)
go' Stream (Of ByteString) m b
s (IDecode (PrimState m) a -> t m (a, Stream (Of ByteString) m b))
-> t m (IDecode (PrimState m) a)
-> t m (a, Stream (Of ByteString) m b)
forall (m :: * -> *) a b. Monad m => (a -> m b) -> m a -> m b
=<< m (IDecode (PrimState m) a) -> t m (IDecode (PrimState m) a)
forall (m :: * -> *) a. Monad m => m a -> t m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (ST (PrimState m) (IDecode (PrimState m) a)
-> m (IDecode (PrimState m) a)
forall a. ST (PrimState m) a -> m a
forall (m :: * -> *) a. MonadST m => ST (PrimState m) a -> m a
stToIO (Decoder (PrimState m) a
-> ST (PrimState m) (IDecode (PrimState m) a)
forall s a. Decoder s a -> ST s (IDecode s a)
deserialiseIncremental Decoder (PrimState m) a
dec))
   where
    go' :: Stream (Of ByteString) m b
-> IDecode (PrimState m) a -> t m (a, Stream (Of ByteString) m b)
go' Stream (Of ByteString) m b
s = \case
      Partial Maybe ByteString -> ST (PrimState m) (IDecode (PrimState m) a)
k ->
        m (Either b (ByteString, Stream (Of ByteString) m b))
-> t m (Either b (ByteString, Stream (Of ByteString) m b))
forall (m :: * -> *) a. Monad m => m a -> t m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (Stream (Of ByteString) m b
-> m (Either b (ByteString, Stream (Of ByteString) m b))
forall (m :: * -> *) a r.
Monad m =>
Stream (Of a) m r -> m (Either r (a, Stream (Of a) m r))
S.next Stream (Of ByteString) m b
s) t m (Either b (ByteString, Stream (Of ByteString) m b))
-> (Either b (ByteString, Stream (Of ByteString) m b)
    -> t m (a, Stream (Of ByteString) m b))
-> t m (a, Stream (Of ByteString) m b)
forall a b. t m a -> (a -> t m b) -> t m b
forall (m :: * -> *) a b. Monad m => m a -> (a -> m b) -> m b
>>= \case
          Right (ByteString
bs, Stream (Of ByteString) m b
s') -> Stream (Of ByteString) m b
-> IDecode (PrimState m) a -> t m (a, Stream (Of ByteString) m b)
go' Stream (Of ByteString) m b
s' (IDecode (PrimState m) a -> t m (a, Stream (Of ByteString) m b))
-> t m (IDecode (PrimState m) a)
-> t m (a, Stream (Of ByteString) m b)
forall (m :: * -> *) a b. Monad m => (a -> m b) -> m a -> m b
=<< m (IDecode (PrimState m) a) -> t m (IDecode (PrimState m) a)
forall (m :: * -> *) a. Monad m => m a -> t m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (ST (PrimState m) (IDecode (PrimState m) a)
-> m (IDecode (PrimState m) a)
forall a. ST (PrimState m) a -> m a
forall (m :: * -> *) a. MonadST m => ST (PrimState m) a -> m a
stToIO (Maybe ByteString -> ST (PrimState m) (IDecode (PrimState m) a)
k (ByteString -> Maybe ByteString
forall a. a -> Maybe a
Just ByteString
bs)))
          Left b
r -> Stream (Of ByteString) m b
-> IDecode (PrimState m) a -> t m (a, Stream (Of ByteString) m b)
go' (b -> Stream (Of ByteString) m b
forall a. a -> Stream (Of ByteString) m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure b
r) (IDecode (PrimState m) a -> t m (a, Stream (Of ByteString) m b))
-> t m (IDecode (PrimState m) a)
-> t m (a, Stream (Of ByteString) m b)
forall (m :: * -> *) a b. Monad m => (a -> m b) -> m a -> m b
=<< m (IDecode (PrimState m) a) -> t m (IDecode (PrimState m) a)
forall (m :: * -> *) a. Monad m => m a -> t m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (ST (PrimState m) (IDecode (PrimState m) a)
-> m (IDecode (PrimState m) a)
forall a. ST (PrimState m) a -> m a
forall (m :: * -> *) a. MonadST m => ST (PrimState m) a -> m a
stToIO (Maybe ByteString -> ST (PrimState m) (IDecode (PrimState m) a)
k Maybe ByteString
forall a. Maybe a
Nothing))
      Codec.CBOR.Read.Done ByteString
bs ByteOffset
_off a
a -> (a, Stream (Of ByteString) m b)
-> t m (a, Stream (Of ByteString) m b)
forall a. a -> t m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (a
a, ByteString -> Stream (Of ByteString) m ()
forall (m :: * -> *) a. Monad m => a -> Stream (Of a) m ()
S.yield ByteString
bs Stream (Of ByteString) m ()
-> Stream (Of ByteString) m b -> Stream (Of ByteString) m b
forall a b.
Stream (Of ByteString) m a
-> Stream (Of ByteString) m b -> Stream (Of ByteString) m b
forall (f :: * -> *) a b. Applicative f => f a -> f b -> f b
*> Stream (Of ByteString) m b
s)
      Codec.CBOR.Read.Fail ByteString
_bs ByteOffset
_off DeserialiseFailure
err -> DeserialiseFailure -> t m (a, Stream (Of ByteString) m b)
forall a. DeserialiseFailure -> t m a
forall e (m :: * -> *) a. MonadError e m => e -> m a
throwError DeserialiseFailure
err

yieldInMemoryS ::
  (MonadThrow m, MonadST m) =>
  (MountPoint -> SomeHasFS m) ->
  FilePath ->
  (forall s. Decoder s (TxIn l)) ->
  (forall s. Decoder s (TxOut l)) ->
  Yield m l
yieldInMemoryS :: forall (m :: * -> *) (l :: LedgerStateKind).
(MonadThrow m, MonadST m) =>
(MountPoint -> SomeHasFS m)
-> String
-> (forall s. Decoder s (TxIn l))
-> (forall s. Decoder s (TxOut l))
-> Yield m l
yieldInMemoryS MountPoint -> SomeHasFS m
mkFs (String -> (String, String)
F.splitFileName -> (String
fp, String
fn)) forall s. Decoder s (TxIn l)
decK forall s. Decoder s (TxOut l)
decV l EmptyMK
_ 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 =
  SomeHasFS m
-> FsPath
-> (Stream (Of ByteString) m (Maybe CRC)
    -> ExceptT
         DeserialiseFailure
         m
         (Stream (Of ByteString) m (Maybe CRC, Maybe CRC)))
-> ExceptT DeserialiseFailure m (Maybe CRC, Maybe CRC)
forall (m :: * -> *).
MonadThrow m =>
SomeHasFS m
-> FsPath
-> (Stream (Of ByteString) m (Maybe CRC)
    -> ExceptT
         DeserialiseFailure
         m
         (Stream (Of ByteString) m (Maybe CRC, Maybe CRC)))
-> ExceptT DeserialiseFailure m (Maybe CRC, Maybe CRC)
streamingFile (MountPoint -> SomeHasFS m
mkFs (MountPoint -> SomeHasFS m) -> MountPoint -> SomeHasFS m
forall a b. (a -> b) -> a -> b
$ String -> MountPoint
MountPoint String
fp) (Context -> FsPath
mkFsPath [String
fn]) ((Stream (Of ByteString) m (Maybe CRC)
  -> ExceptT
       DeserialiseFailure
       m
       (Stream (Of ByteString) m (Maybe CRC, Maybe CRC)))
 -> ExceptT DeserialiseFailure m (Maybe CRC, Maybe CRC))
-> (Stream (Of ByteString) m (Maybe CRC)
    -> ExceptT
         DeserialiseFailure
         m
         (Stream (Of ByteString) m (Maybe CRC, Maybe CRC)))
-> ExceptT DeserialiseFailure m (Maybe CRC, Maybe CRC)
forall a b. (a -> b) -> a -> b
$ \Stream (Of ByteString) m (Maybe CRC)
s -> do
    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 (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)))
-> 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))
forall a b. (a -> b) -> a -> b
$ (forall s. Decoder s (TxIn l))
-> (forall s. Decoder s (TxOut l))
-> Stream (Of ByteString) m (Maybe CRC)
-> Stream
     (Of (TxIn l, TxOut l))
     (ExceptT DeserialiseFailure m)
     (Stream (Of ByteString) m (Maybe CRC))
forall (m :: * -> *) a b.
MonadST m =>
(forall s. Decoder s a)
-> (forall s. Decoder s b)
-> Stream (Of ByteString) m (Maybe CRC)
-> Stream
     (Of (a, b))
     (ExceptT DeserialiseFailure m)
     (Stream (Of ByteString) m (Maybe CRC))
yieldCborMapS Decoder s (TxIn l)
forall s. Decoder s (TxIn l)
decK Decoder s (TxOut l)
forall s. Decoder s (TxOut l)
decV Stream (Of ByteString) m (Maybe CRC)
s

sinkInMemoryS ::
  forall m l.
  MonadThrow m =>
  Int ->
  (TxIn l -> Encoding) ->
  (TxOut l -> Encoding) ->
  SomeHasFS m ->
  FilePath ->
  Sink m l
sinkInMemoryS :: forall (m :: * -> *) (l :: LedgerStateKind).
MonadThrow m =>
Int
-> (TxIn l -> Encoding)
-> (TxOut l -> Encoding)
-> SomeHasFS m
-> String
-> Sink m l
sinkInMemoryS Int
writeChunkSize TxIn l -> Encoding
encK TxOut l -> Encoding
encV (SomeHasFS HasFS m h
fs) String
fp l EmptyMK
_ Stream
  (Of (TxIn l, TxOut l))
  (ExceptT DeserialiseFailure m)
  (Stream (Of ByteString) m (Maybe CRC))
s =
  m (Either
     DeserialiseFailure
     (Stream (Of ByteString) m (Maybe CRC, Maybe CRC)))
-> ExceptT
     DeserialiseFailure
     m
     (Stream (Of ByteString) m (Maybe CRC, Maybe CRC))
forall e (m :: * -> *) a. m (Either e a) -> ExceptT e m a
ExceptT (m (Either
      DeserialiseFailure
      (Stream (Of ByteString) m (Maybe CRC, Maybe CRC)))
 -> ExceptT
      DeserialiseFailure
      m
      (Stream (Of ByteString) m (Maybe CRC, Maybe CRC)))
-> m (Either
        DeserialiseFailure
        (Stream (Of ByteString) m (Maybe CRC, Maybe CRC)))
-> ExceptT
     DeserialiseFailure
     m
     (Stream (Of ByteString) m (Maybe CRC, Maybe CRC))
forall a b. (a -> b) -> a -> b
$ HasFS m h
-> FsPath
-> OpenMode
-> (Handle h
    -> m (Either
            DeserialiseFailure
            (Stream (Of ByteString) m (Maybe CRC, Maybe CRC))))
-> m (Either
        DeserialiseFailure
        (Stream (Of ByteString) m (Maybe CRC, Maybe CRC)))
forall (m :: * -> *) h a.
(HasCallStack, MonadThrow m) =>
HasFS m h -> FsPath -> OpenMode -> (Handle h -> m a) -> m a
withFile HasFS m h
fs (Context -> FsPath
mkFsPath [String
fp]) (AllowExisting -> OpenMode
WriteMode AllowExisting
MustBeNew) ((Handle h
  -> m (Either
          DeserialiseFailure
          (Stream (Of ByteString) m (Maybe CRC, Maybe CRC))))
 -> m (Either
         DeserialiseFailure
         (Stream (Of ByteString) m (Maybe CRC, Maybe CRC))))
-> (Handle h
    -> m (Either
            DeserialiseFailure
            (Stream (Of ByteString) m (Maybe CRC, Maybe CRC))))
-> m (Either
        DeserialiseFailure
        (Stream (Of ByteString) m (Maybe CRC, Maybe CRC)))
forall a b. (a -> b) -> a -> b
$ \Handle h
hdl -> do
    let bs :: ByteString
bs = Encoding -> ByteString
toStrictByteString (Word -> Encoding
encodeListLen Word
1 Encoding -> Encoding -> Encoding
forall a. Semigroup a => a -> a -> a
<> Encoding
encodeMapLenIndef)
    let !crc0 :: CRC
crc0 = ByteString -> CRC -> CRC
forall a. CRC32 a => a -> CRC -> CRC
updateCRC ByteString
bs CRC
initCRC
    m Word64 -> m ()
forall (f :: * -> *) a. Functor f => f a -> f ()
void (m Word64 -> m ()) -> m Word64 -> m ()
forall a b. (a -> b) -> a -> b
$ HasFS m h -> HasCallStack => Handle h -> ByteString -> m Word64
forall (m :: * -> *) h.
HasFS m h -> HasCallStack => Handle h -> ByteString -> m Word64
hPutSome HasFS m h
fs Handle h
hdl ByteString
bs
    e <- ExceptT
  DeserialiseFailure m (Stream (Of ByteString) m (Maybe CRC), CRC)
-> m (Either
        DeserialiseFailure (Stream (Of ByteString) m (Maybe CRC), CRC))
forall e (m :: * -> *) a. ExceptT e m a -> m (Either e a)
runExceptT (ExceptT
   DeserialiseFailure m (Stream (Of ByteString) m (Maybe CRC), CRC)
 -> m (Either
         DeserialiseFailure (Stream (Of ByteString) m (Maybe CRC), CRC)))
-> ExceptT
     DeserialiseFailure m (Stream (Of ByteString) m (Maybe CRC), CRC)
-> m (Either
        DeserialiseFailure (Stream (Of ByteString) m (Maybe CRC), CRC))
forall a b. (a -> b) -> a -> b
$ Handle h
-> CRC
-> 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), CRC)
go Handle h
hdl CRC
crc0 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
    case e of
      Left DeserialiseFailure
err -> Either
  DeserialiseFailure
  (Stream (Of ByteString) m (Maybe CRC, Maybe CRC))
-> m (Either
        DeserialiseFailure
        (Stream (Of ByteString) m (Maybe CRC, Maybe CRC)))
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (Either
   DeserialiseFailure
   (Stream (Of ByteString) m (Maybe CRC, Maybe CRC))
 -> m (Either
         DeserialiseFailure
         (Stream (Of ByteString) m (Maybe CRC, Maybe CRC))))
-> Either
     DeserialiseFailure
     (Stream (Of ByteString) m (Maybe CRC, Maybe CRC))
-> m (Either
        DeserialiseFailure
        (Stream (Of ByteString) m (Maybe CRC, Maybe CRC)))
forall a b. (a -> b) -> a -> b
$ DeserialiseFailure
-> Either
     DeserialiseFailure
     (Stream (Of ByteString) m (Maybe CRC, Maybe CRC))
forall a b. a -> Either a b
Left DeserialiseFailure
err
      Right (Stream (Of ByteString) m (Maybe CRC)
r, CRC
crc1) -> do
        let bs1 :: ByteString
bs1 = Encoding -> ByteString
toStrictByteString Encoding
encodeBreak
        m Word64 -> m ()
forall (f :: * -> *) a. Functor f => f a -> f ()
void (m Word64 -> m ()) -> m Word64 -> m ()
forall a b. (a -> b) -> a -> b
$ HasFS m h -> HasCallStack => Handle h -> ByteString -> m Word64
forall (m :: * -> *) h.
HasFS m h -> HasCallStack => Handle h -> ByteString -> m Word64
hPutSome HasFS m h
fs Handle h
hdl ByteString
bs1
        let !crc2 :: CRC
crc2 = ByteString -> CRC -> CRC
forall a. CRC32 a => a -> CRC -> CRC
updateCRC ByteString
bs1 CRC
crc1
        Either
  DeserialiseFailure
  (Stream (Of ByteString) m (Maybe CRC, Maybe CRC))
-> m (Either
        DeserialiseFailure
        (Stream (Of ByteString) m (Maybe CRC, Maybe CRC)))
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (Either
   DeserialiseFailure
   (Stream (Of ByteString) m (Maybe CRC, Maybe CRC))
 -> m (Either
         DeserialiseFailure
         (Stream (Of ByteString) m (Maybe CRC, Maybe CRC))))
-> Either
     DeserialiseFailure
     (Stream (Of ByteString) m (Maybe CRC, Maybe CRC))
-> m (Either
        DeserialiseFailure
        (Stream (Of ByteString) m (Maybe CRC, Maybe CRC)))
forall a b. (a -> b) -> a -> b
$ Stream (Of ByteString) m (Maybe CRC, Maybe CRC)
-> Either
     DeserialiseFailure
     (Stream (Of ByteString) m (Maybe CRC, Maybe CRC))
forall a b. b -> Either a b
Right ((Maybe CRC -> (Maybe CRC, Maybe CRC))
-> Stream (Of ByteString) m (Maybe CRC)
-> Stream (Of ByteString) m (Maybe CRC, Maybe CRC)
forall a b.
(a -> b)
-> Stream (Of ByteString) m a -> Stream (Of ByteString) m b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap (,CRC -> Maybe CRC
forall a. a -> Maybe a
Just CRC
crc2) Stream (Of ByteString) m (Maybe CRC)
r)
 where
  go :: Handle h
-> CRC
-> 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), CRC)
go Handle h
tb !CRC
crc Int
0 [(TxIn l, TxOut l)]
m Stream
  (Of (TxIn l, TxOut l))
  (ExceptT DeserialiseFailure m)
  (Stream (Of ByteString) m (Maybe CRC))
s' = do
    let bs :: ByteString
bs = Encoding -> ByteString
toStrictByteString (Encoding -> ByteString) -> Encoding -> ByteString
forall a b. (a -> b) -> a -> b
$ [Encoding] -> Encoding
forall a. Monoid a => [a] -> a
mconcat [TxIn l -> Encoding
encK TxIn l
k Encoding -> Encoding -> Encoding
forall a. Semigroup a => a -> a -> a
<> TxOut l -> Encoding
encV TxOut l
v | (TxIn l
k, TxOut l
v) <- [(TxIn l, TxOut l)] -> [(TxIn l, TxOut l)]
forall a. [a] -> [a]
reverse [(TxIn l, TxOut l)]
m]
    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
$ m Word64 -> m ()
forall (f :: * -> *) a. Functor f => f a -> f ()
void (m Word64 -> m ()) -> m Word64 -> m ()
forall a b. (a -> b) -> a -> b
$ HasFS m h -> HasCallStack => Handle h -> ByteString -> m Word64
forall (m :: * -> *) h.
HasFS m h -> HasCallStack => Handle h -> ByteString -> m Word64
hPutSome HasFS m h
fs Handle h
tb ByteString
bs
    let !crc1 :: CRC
crc1 = ByteString -> CRC -> CRC
forall a. CRC32 a => a -> CRC -> CRC
updateCRC ByteString
bs CRC
crc
    Handle h
-> CRC
-> 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), CRC)
go Handle h
tb CRC
crc1 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 Handle h
tb !CRC
crc 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
        let bs :: ByteString
bs = Encoding -> ByteString
toStrictByteString (Encoding -> ByteString) -> Encoding -> ByteString
forall a b. (a -> b) -> a -> b
$ [Encoding] -> Encoding
forall a. Monoid a => [a] -> a
mconcat [TxIn l -> Encoding
encK TxIn l
k Encoding -> Encoding -> Encoding
forall a. Semigroup a => a -> a -> a
<> TxOut l -> Encoding
encV TxOut l
v | (TxIn l
k, TxOut l
v) <- [(TxIn l, TxOut l)] -> [(TxIn l, TxOut l)]
forall a. [a] -> [a]
reverse [(TxIn l, TxOut l)]
m]
        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
$ m Word64 -> m ()
forall (f :: * -> *) a. Functor f => f a -> f ()
void (m Word64 -> m ()) -> m Word64 -> m ()
forall a b. (a -> b) -> a -> b
$ HasFS m h -> HasCallStack => Handle h -> ByteString -> m Word64
forall (m :: * -> *) h.
HasFS m h -> HasCallStack => Handle h -> ByteString -> m Word64
hPutSome HasFS m h
fs Handle h
tb ByteString
bs
        let !crc1 :: CRC
crc1 = ByteString -> CRC -> CRC
forall a. CRC32 a => a -> CRC -> CRC
updateCRC ByteString
bs CRC
crc
        (,CRC
crc1) (Stream (Of ByteString) m (Maybe CRC)
 -> (Stream (Of ByteString) m (Maybe CRC), CRC))
-> ExceptT
     DeserialiseFailure m (Stream (Of ByteString) m (Maybe CRC))
-> ExceptT
     DeserialiseFailure m (Stream (Of ByteString) m (Maybe CRC), CRC)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> 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'') -> Handle h
-> CRC
-> 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), CRC)
go Handle h
tb CRC
crc (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''