{-# LANGUAGE ConstraintKinds           #-}
{-# LANGUAGE ExistentialQuantification #-}
{-# LANGUAGE FlexibleContexts          #-}
{-# LANGUAGE LambdaCase                #-}
{-# LANGUAGE NamedFieldPuns            #-}
{-# LANGUAGE RankNTypes                #-}
{-# LANGUAGE RecordWildCards           #-}
{-# LANGUAGE ScopedTypeVariables       #-}
{-# LANGUAGE TupleSections             #-}
{-# LANGUAGE TypeApplications          #-}

-- | Immutable on-disk database of binary blobs
--
-- = Internal format
--
-- The API of the ImmutableDB uses 'SlotNo' to indicate a location in the
-- chain\/immutable database. To distinguish EBBs from regular blocks, the hash
-- is used (together they form a 'RealPoint'). The contents of the database are
-- not stored in one big file that is appended to in eternity, but a separate
-- file is created for each 'ChunkNo'.
--
-- Within each 'ChunkNo', the entries are numbered by 'RelativeSlot's. Each
-- 'SlotNo' can be converted to a combination of an 'ChunkNo' and a
-- 'RelativeSlot' (= 'ChunkSlot') and vice versa. This conversion depends on the
-- size of the chunks: 'ChunkSize'. This size may not be the same for each
-- chunk. When opening the database, the user must give a 'ChunkInfo' that will
-- be used to find out the size of each chunk.
--
-- For example:
--
-- > Chunks:         <──────── 0 ────────> <────── 1 ──────>
-- > chunk size:               4                   3
-- >                 ┌───┬───┬───┬───┬───┐ ┌───┬───┬───┬───┐
-- >                 │   │   │   │   │   │ │   │   │   │   │
-- >                 └───┴───┴───┴───┴───┘ └───┴───┴───┴───┘
-- > 'RelativeSlot':   0   1   2   3   4     0   1   2   3
-- > 'SlotNo':        EBB  0   1   2   3    EBB  4   5   6
--
-- Not all chunks can contain EBBs; see 'ChunkInfo' for details.
--
-- = Errors
--
-- Whenever an 'UnexpectedFailure' is thrown during an operation, e.g.,
-- 'appendBlock', the database will be automatically closed because we can not
-- guarantee a consistent state in the face of file system errors.
--
-- = Opening the database
--
-- The database can be closed and opened again. In case the database was closed
-- because of an unexpected error. When the database is opened again, invalid
-- data will be truncated from the database until a valid prefix is recovered.
--
-- = Concurrency
--
-- The same database should not be opened multiple times concurrently.
-- This is ensured by the file lock of the ChainDB.
--
-- The database can have multiple readers, but should only have one writer.
--
--
-- = Layout on disk
--
-- The database is structured on disk as follows:
--
-- > /
-- >   00000.chunk
-- >   00000.primary
-- >   00000.secondary
-- >   ..
-- >   00008.chunk
-- >   00008.primary
-- >   00008.secondary
--
-- For each chunk, there are three files on disk:
--
--   * A \"chunk file\" that stores the actual blocks. But nothing more, so
--     nothing is stored for empty slots.
--
--   * A \"secondary index file\" that stores information about each block: its
--     hash, the slot number or epoch number in case of an EBB, a checksum of
--     the block, the offset of the block in the chunk file, and more. This
--     index is sparse to save space.
--
--   * A \"primary index file\" that maps slots to offsets in the secondary
--     index file.
module Ouroboros.Consensus.Storage.ImmutableDB.Impl (
    -- * Opening the databse
    ImmutableDbArgs (..)
  , ImmutableDbSerialiseConstraints
  , defaultArgs
  , openDB
    -- * Re-exported
  , ChunkFileError (..)
  , Index.CacheConfig (..)
  , TraceChunkValidation (..)
  , TraceEvent (..)
  , ValidationPolicy (..)
    -- * Internals for testing purposes
  , Internal (..)
  , deleteAfter
  , openDBInternal
  ) where

import qualified Codec.CBOR.Write as CBOR
import           Control.Monad (replicateM_, unless, when)
import           Control.Monad.Except (runExceptT)
import           Control.Monad.State.Strict (get, lift, modify, put)
import           Control.Tracer (Tracer, nullTracer, traceWith)
import qualified Data.ByteString.Lazy as Lazy
import           GHC.Stack (HasCallStack)

import           Ouroboros.Consensus.Block hiding (headerHash)
import           Ouroboros.Consensus.Util (SomePair (..))
import           Ouroboros.Consensus.Util.Args
import           Ouroboros.Consensus.Util.IOLike
import           Ouroboros.Consensus.Util.ResourceRegistry

import           Ouroboros.Consensus.Storage.Common
import           Ouroboros.Consensus.Storage.FS.API
import           Ouroboros.Consensus.Storage.FS.API.Types hiding (allowExisting)
import           Ouroboros.Consensus.Storage.FS.CRC
import           Ouroboros.Consensus.Storage.Serialisation

import           Ouroboros.Consensus.Storage.ImmutableDB.API
import           Ouroboros.Consensus.Storage.ImmutableDB.Chunks
import           Ouroboros.Consensus.Storage.ImmutableDB.Impl.Index (Index)
import qualified Ouroboros.Consensus.Storage.ImmutableDB.Impl.Index as Index
import qualified Ouroboros.Consensus.Storage.ImmutableDB.Impl.Index.Primary as Primary
import           Ouroboros.Consensus.Storage.ImmutableDB.Impl.Index.Secondary
                     (BlockOffset (..), HeaderOffset (..), HeaderSize (..))
import qualified Ouroboros.Consensus.Storage.ImmutableDB.Impl.Index.Secondary as Secondary
import           Ouroboros.Consensus.Storage.ImmutableDB.Impl.Iterator
import           Ouroboros.Consensus.Storage.ImmutableDB.Impl.Parser
import           Ouroboros.Consensus.Storage.ImmutableDB.Impl.State
import           Ouroboros.Consensus.Storage.ImmutableDB.Impl.Types
import           Ouroboros.Consensus.Storage.ImmutableDB.Impl.Util
import           Ouroboros.Consensus.Storage.ImmutableDB.Impl.Validation

{------------------------------------------------------------------------------
  Opening the database
------------------------------------------------------------------------------}

data ImmutableDbArgs f m blk = ImmutableDbArgs {
      ImmutableDbArgs f m blk -> CacheConfig
immCacheConfig      :: Index.CacheConfig
    , ImmutableDbArgs f m blk -> HKD f (blk -> Bool)
immCheckIntegrity   :: HKD f (blk -> Bool)
    , ImmutableDbArgs f m blk -> HKD f ChunkInfo
immChunkInfo        :: HKD f ChunkInfo
    , ImmutableDbArgs f m blk -> HKD f (CodecConfig blk)
immCodecConfig      :: HKD f (CodecConfig blk)
    , ImmutableDbArgs f m blk -> SomeHasFS m
immHasFS            :: SomeHasFS m
    , ImmutableDbArgs f m blk -> HKD f (ResourceRegistry m)
immRegistry         :: HKD f (ResourceRegistry m)
    , ImmutableDbArgs f m blk -> Tracer m (TraceEvent blk)
immTracer           :: Tracer m (TraceEvent blk)
    , ImmutableDbArgs f m blk -> ValidationPolicy
immValidationPolicy :: ValidationPolicy
    }

-- | Default arguments
defaultArgs :: Applicative m => SomeHasFS m -> ImmutableDbArgs Defaults m blk
defaultArgs :: SomeHasFS m -> ImmutableDbArgs Defaults m blk
defaultArgs SomeHasFS m
immHasFS = ImmutableDbArgs :: forall (f :: * -> *) (m :: * -> *) blk.
CacheConfig
-> HKD f (blk -> Bool)
-> HKD f ChunkInfo
-> HKD f (CodecConfig blk)
-> SomeHasFS m
-> HKD f (ResourceRegistry m)
-> Tracer m (TraceEvent blk)
-> ValidationPolicy
-> ImmutableDbArgs f m blk
ImmutableDbArgs {
      immCacheConfig :: CacheConfig
immCacheConfig      = CacheConfig
cacheConfig
    , immCheckIntegrity :: HKD Defaults (blk -> Bool)
immCheckIntegrity   = HKD Defaults (blk -> Bool)
forall t. Defaults t
NoDefault
    , immChunkInfo :: HKD Defaults ChunkInfo
immChunkInfo        = HKD Defaults ChunkInfo
forall t. Defaults t
NoDefault
    , immCodecConfig :: HKD Defaults (CodecConfig blk)
immCodecConfig      = HKD Defaults (CodecConfig blk)
forall t. Defaults t
NoDefault
    , SomeHasFS m
immHasFS :: SomeHasFS m
immHasFS :: SomeHasFS m
immHasFS
    , immRegistry :: HKD Defaults (ResourceRegistry m)
immRegistry         = HKD Defaults (ResourceRegistry m)
forall t. Defaults t
NoDefault
    , immTracer :: Tracer m (TraceEvent blk)
immTracer           = Tracer m (TraceEvent blk)
forall (m :: * -> *) a. Applicative m => Tracer m a
nullTracer
    , immValidationPolicy :: ValidationPolicy
immValidationPolicy = ValidationPolicy
ValidateMostRecentChunk
    }
  where
    -- Cache 250 past chunks by default. This will take roughly 250 MB of RAM.
    -- At the time of writing (1/2020), there are 166 epochs, and we store one
    -- epoch per chunk, so even one year from now, we will be able to cache all
    -- chunks' indices in the chain.
    --
    -- If this number were too low, i.e., less than the number of chunks that
    -- that clients are requesting blocks from, we would constantly evict and
    -- reparse indices, causing a much higher CPU load.
    cacheConfig :: CacheConfig
cacheConfig = CacheConfig :: Word32 -> DiffTime -> CacheConfig
Index.CacheConfig {
          $sel:pastChunksToCache:CacheConfig :: Word32
pastChunksToCache = Word32
250
        , $sel:expireUnusedAfter:CacheConfig :: DiffTime
expireUnusedAfter = DiffTime
5 DiffTime -> DiffTime -> DiffTime
forall a. Num a => a -> a -> a
* DiffTime
60 -- Expire after 1 minute
        }

-- | 'EncodeDisk' and 'DecodeDisk' constraints needed for the ImmutableDB.
type ImmutableDbSerialiseConstraints blk =
  ( EncodeDisk blk blk
  , DecodeDisk blk (Lazy.ByteString -> blk)
  , DecodeDiskDep (NestedCtxt Header) blk
  , ReconstructNestedCtxt Header blk
  , HasBinaryBlockInfo blk
  )

{------------------------------------------------------------------------------
  Exposed internals and/or extra functionality for testing purposes
------------------------------------------------------------------------------}

data Internal m blk = Internal {
    -- | Delete everything in the database after the specified tip.
    --
    -- PRECONDITION: The tip must correspond to an existing block or genesis.
    --
    -- The correctness of open iterators is not guaranteed, they should be
    -- closed before calling this operation.
    --
    -- Throws a 'ClosedDBError' if the database is closed.
    Internal m blk -> HasCallStack => WithOrigin (Tip blk) -> m ()
deleteAfter_ :: HasCallStack => WithOrigin (Tip blk) -> m ()
  }

-- | Wrapper around 'deleteAfter_' to ensure 'HasCallStack' constraint
--
-- See documentation of 'deleteAfter_'.
deleteAfter :: HasCallStack => Internal m blk -> WithOrigin (Tip blk) -> m ()
deleteAfter :: Internal m blk -> WithOrigin (Tip blk) -> m ()
deleteAfter = Internal m blk -> WithOrigin (Tip blk) -> m ()
forall (m :: * -> *) blk.
Internal m blk -> HasCallStack => WithOrigin (Tip blk) -> m ()
deleteAfter_

{------------------------------------------------------------------------------
  ImmutableDB Implementation
------------------------------------------------------------------------------}

openDB ::
     forall m blk ans.
     ( IOLike m
     , GetPrevHash blk
     , ConvertRawHash blk
     , ImmutableDbSerialiseConstraints blk
     , HasCallStack
     )
  => ImmutableDbArgs Identity m blk
  -> (forall st. WithTempRegistry st m (ImmutableDB m blk, st) -> ans)
  -> ans
openDB :: ImmutableDbArgs Identity m blk
-> (forall st.
    WithTempRegistry st m (ImmutableDB m blk, st) -> ans)
-> ans
openDB ImmutableDbArgs Identity m blk
args forall st. WithTempRegistry st m (ImmutableDB m blk, st) -> ans
cont =
    ImmutableDbArgs Identity m blk
-> (forall h.
    WithTempRegistry
      (OpenState m blk h)
      m
      ((ImmutableDB m blk, Internal m blk), OpenState m blk h)
    -> ans)
-> ans
forall (m :: * -> *) blk ans.
(IOLike m, GetPrevHash blk, ConvertRawHash blk,
 ImmutableDbSerialiseConstraints blk, HasCallStack) =>
ImmutableDbArgs Identity m blk
-> (forall h.
    WithTempRegistry
      (OpenState m blk h)
      m
      ((ImmutableDB m blk, Internal m blk), OpenState m blk h)
    -> ans)
-> ans
openDBInternal ImmutableDbArgs Identity m blk
args (WithTempRegistry
  (OpenState m blk h) m (ImmutableDB m blk, OpenState m blk h)
-> ans
forall st. WithTempRegistry st m (ImmutableDB m blk, st) -> ans
cont (WithTempRegistry
   (OpenState m blk h) m (ImmutableDB m blk, OpenState m blk h)
 -> ans)
-> (WithTempRegistry
      (OpenState m blk h)
      m
      ((ImmutableDB m blk, Internal m blk), OpenState m blk h)
    -> WithTempRegistry
         (OpenState m blk h) m (ImmutableDB m blk, OpenState m blk h))
-> WithTempRegistry
     (OpenState m blk h)
     m
     ((ImmutableDB m blk, Internal m blk), OpenState m blk h)
-> ans
forall b c a. (b -> c) -> (a -> b) -> a -> c
. (((ImmutableDB m blk, Internal m blk), OpenState m blk h)
 -> (ImmutableDB m blk, OpenState m blk h))
-> WithTempRegistry
     (OpenState m blk h)
     m
     ((ImmutableDB m blk, Internal m blk), OpenState m blk h)
-> WithTempRegistry
     (OpenState m blk h) m (ImmutableDB m blk, OpenState m blk h)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap ((ImmutableDB m blk, Internal m blk), OpenState m blk h)
-> (ImmutableDB m blk, OpenState m blk h)
forall a b b. ((a, b), b) -> (a, b)
swizzle)
  where
    swizzle :: ((a, b), b) -> (a, b)
swizzle ((a
immdb, b
_internal), b
ost) = (a
immdb, b
ost)

-- | For testing purposes: exposes internals via 'Internal'
--
--
openDBInternal ::
     forall m blk ans.
     ( IOLike m
     , GetPrevHash blk
     , ConvertRawHash blk
     , ImmutableDbSerialiseConstraints blk
     , HasCallStack
     )
  => ImmutableDbArgs Identity m blk
  -> (forall h.
         WithTempRegistry
           (OpenState m blk h)
           m
           ((ImmutableDB m blk, Internal m blk), OpenState m blk h)
      -> ans
     )
  -> ans
openDBInternal :: ImmutableDbArgs Identity m blk
-> (forall h.
    WithTempRegistry
      (OpenState m blk h)
      m
      ((ImmutableDB m blk, Internal m blk), OpenState m blk h)
    -> ans)
-> ans
openDBInternal ImmutableDbArgs { immHasFS :: forall (f :: * -> *) (m :: * -> *) blk.
ImmutableDbArgs f m blk -> SomeHasFS m
immHasFS = SomeHasFS HasFS m h
hasFS, Tracer m (TraceEvent blk)
HKD Identity (CodecConfig blk)
HKD Identity ChunkInfo
HKD Identity (ResourceRegistry m)
HKD Identity (blk -> Bool)
ValidationPolicy
CacheConfig
immValidationPolicy :: ValidationPolicy
immTracer :: Tracer m (TraceEvent blk)
immRegistry :: HKD Identity (ResourceRegistry m)
immCodecConfig :: HKD Identity (CodecConfig blk)
immChunkInfo :: HKD Identity ChunkInfo
immCheckIntegrity :: HKD Identity (blk -> Bool)
immCacheConfig :: CacheConfig
immValidationPolicy :: forall (f :: * -> *) (m :: * -> *) blk.
ImmutableDbArgs f m blk -> ValidationPolicy
immTracer :: forall (f :: * -> *) (m :: * -> *) blk.
ImmutableDbArgs f m blk -> Tracer m (TraceEvent blk)
immRegistry :: forall (f :: * -> *) (m :: * -> *) blk.
ImmutableDbArgs f m blk -> HKD f (ResourceRegistry m)
immCodecConfig :: forall (f :: * -> *) (m :: * -> *) blk.
ImmutableDbArgs f m blk -> HKD f (CodecConfig blk)
immChunkInfo :: forall (f :: * -> *) (m :: * -> *) blk.
ImmutableDbArgs f m blk -> HKD f ChunkInfo
immCheckIntegrity :: forall (f :: * -> *) (m :: * -> *) blk.
ImmutableDbArgs f m blk -> HKD f (blk -> Bool)
immCacheConfig :: forall (f :: * -> *) (m :: * -> *) blk.
ImmutableDbArgs f m blk -> CacheConfig
.. } forall h.
WithTempRegistry
  (OpenState m blk h)
  m
  ((ImmutableDB m blk, Internal m blk), OpenState m blk h)
-> ans
cont = WithTempRegistry
  (OpenState m blk h)
  m
  ((ImmutableDB m blk, Internal m blk), OpenState m blk h)
-> ans
forall h.
WithTempRegistry
  (OpenState m blk h)
  m
  ((ImmutableDB m blk, Internal m blk), OpenState m blk h)
-> ans
cont (WithTempRegistry
   (OpenState m blk h)
   m
   ((ImmutableDB m blk, Internal m blk), OpenState m blk h)
 -> ans)
-> WithTempRegistry
     (OpenState m blk h)
     m
     ((ImmutableDB m blk, Internal m blk), OpenState m blk h)
-> ans
forall a b. (a -> b) -> a -> b
$ do
    m () -> WithTempRegistry (OpenState m blk h) m ()
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (m () -> WithTempRegistry (OpenState m blk h) m ())
-> m () -> WithTempRegistry (OpenState m blk h) m ()
forall a b. (a -> b) -> a -> b
$ HasFS m h -> Bool -> FsPath -> m ()
forall (m :: * -> *) h.
HasFS m h -> HasCallStack => Bool -> FsPath -> m ()
createDirectoryIfMissing HasFS m h
hasFS Bool
True ([String] -> FsPath
mkFsPath [])
    let validateEnv :: ValidateEnv m blk h
validateEnv = ValidateEnv :: forall (m :: * -> *) blk h.
HasFS m h
-> ChunkInfo
-> Tracer m (TraceEvent blk)
-> CacheConfig
-> CodecConfig blk
-> (blk -> Bool)
-> ValidateEnv m blk h
ValidateEnv {
            hasFS :: HasFS m h
hasFS          = HasFS m h
hasFS
          , chunkInfo :: ChunkInfo
chunkInfo      = HKD Identity ChunkInfo
ChunkInfo
immChunkInfo
          , tracer :: Tracer m (TraceEvent blk)
tracer         = Tracer m (TraceEvent blk)
immTracer
          , cacheConfig :: CacheConfig
cacheConfig    = CacheConfig
immCacheConfig
          , codecConfig :: CodecConfig blk
codecConfig    = HKD Identity (CodecConfig blk)
CodecConfig blk
immCodecConfig
          , checkIntegrity :: blk -> Bool
checkIntegrity = HKD Identity (blk -> Bool)
blk -> Bool
immCheckIntegrity
          }
    OpenState m blk h
ost <- ValidateEnv m blk h
-> ResourceRegistry m
-> ValidationPolicy
-> WithTempRegistry (OpenState m blk h) m (OpenState m blk h)
forall (m :: * -> *) blk h.
(IOLike m, GetPrevHash blk, HasBinaryBlockInfo blk,
 DecodeDisk blk (ByteString -> blk), ConvertRawHash blk, Eq h,
 HasCallStack) =>
ValidateEnv m blk h
-> ResourceRegistry m
-> ValidationPolicy
-> WithTempRegistry (OpenState m blk h) m (OpenState m blk h)
validateAndReopen ValidateEnv m blk h
validateEnv HKD Identity (ResourceRegistry m)
ResourceRegistry m
immRegistry ValidationPolicy
immValidationPolicy

    StrictMVar m (InternalState m blk h)
stVar <- m (StrictMVar m (InternalState m blk h))
-> WithTempRegistry
     (OpenState m blk h) m (StrictMVar m (InternalState m blk h))
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (m (StrictMVar m (InternalState m blk h))
 -> WithTempRegistry
      (OpenState m blk h) m (StrictMVar m (InternalState m blk h)))
-> m (StrictMVar m (InternalState m blk h))
-> WithTempRegistry
     (OpenState m blk h) m (StrictMVar m (InternalState m blk h))
forall a b. (a -> b) -> a -> b
$ InternalState m blk h -> m (StrictMVar m (InternalState m blk h))
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack, NoThunks a) =>
a -> m (StrictMVar m a)
newMVar (OpenState m blk h -> InternalState m blk h
forall (m :: * -> *) blk h.
OpenState m blk h -> InternalState m blk h
DbOpen OpenState m blk h
ost)

    let dbEnv :: ImmutableDBEnv m blk
dbEnv = ImmutableDBEnv :: forall (m :: * -> *) blk h.
Eq h =>
HasFS m h
-> StrictMVar m (InternalState m blk h)
-> (blk -> Bool)
-> ChunkInfo
-> Tracer m (TraceEvent blk)
-> CacheConfig
-> CodecConfig blk
-> ImmutableDBEnv m blk
ImmutableDBEnv {
            hasFS :: HasFS m h
hasFS            = HasFS m h
hasFS
          , varInternalState :: StrictMVar m (InternalState m blk h)
varInternalState = StrictMVar m (InternalState m blk h)
stVar
          , checkIntegrity :: blk -> Bool
checkIntegrity   = HKD Identity (blk -> Bool)
blk -> Bool
immCheckIntegrity
          , chunkInfo :: ChunkInfo
chunkInfo        = HKD Identity ChunkInfo
ChunkInfo
immChunkInfo
          , tracer :: Tracer m (TraceEvent blk)
tracer           = Tracer m (TraceEvent blk)
immTracer
          , cacheConfig :: CacheConfig
cacheConfig      = CacheConfig
immCacheConfig
          , codecConfig :: CodecConfig blk
codecConfig      = HKD Identity (CodecConfig blk)
CodecConfig blk
immCodecConfig
          }
        db :: ImmutableDB m blk
db = ImmutableDB :: forall (m :: * -> *) blk.
(HasCallStack => m ())
-> (HasCallStack => STM m (WithOrigin (Tip blk)))
-> (forall b.
    HasCallStack =>
    BlockComponent blk b
    -> RealPoint blk -> m (Either (MissingBlock blk) b))
-> (HasCallStack => blk -> m ())
-> (forall b.
    HasCallStack =>
    ResourceRegistry m
    -> BlockComponent blk b
    -> StreamFrom blk
    -> StreamTo blk
    -> m (Either (MissingBlock blk) (Iterator m blk b)))
-> ImmutableDB m blk
ImmutableDB {
            closeDB_ :: HasCallStack => m ()
closeDB_           = ImmutableDBEnv m blk -> m ()
forall (m :: * -> *) blk.
(HasCallStack, IOLike m) =>
ImmutableDBEnv m blk -> m ()
closeDBImpl           ImmutableDBEnv m blk
dbEnv
          , getTip_ :: HasCallStack => STM m (WithOrigin (Tip blk))
getTip_            = ImmutableDBEnv m blk -> STM m (WithOrigin (Tip blk))
forall (m :: * -> *) blk.
(HasCallStack, IOLike m, HasHeader blk) =>
ImmutableDBEnv m blk -> STM m (WithOrigin (Tip blk))
getTipImpl            ImmutableDBEnv m blk
dbEnv
          , getBlockComponent_ :: forall b.
HasCallStack =>
BlockComponent blk b
-> RealPoint blk -> m (Either (MissingBlock blk) b)
getBlockComponent_ = ImmutableDBEnv m blk
-> BlockComponent blk b
-> RealPoint blk
-> m (Either (MissingBlock blk) b)
forall (m :: * -> *) blk b.
(HasHeader blk, ReconstructNestedCtxt Header blk,
 DecodeDisk blk (ByteString -> blk),
 DecodeDiskDep (NestedCtxt Header) blk, IOLike m) =>
ImmutableDBEnv m blk
-> BlockComponent blk b
-> RealPoint blk
-> m (Either (MissingBlock blk) b)
getBlockComponentImpl ImmutableDBEnv m blk
dbEnv
          , appendBlock_ :: HasCallStack => blk -> m ()
appendBlock_       = ImmutableDBEnv m blk -> blk -> m ()
forall (m :: * -> *) blk.
(HasHeader blk, GetHeader blk, EncodeDisk blk blk,
 HasBinaryBlockInfo blk, IOLike m, HasCallStack) =>
ImmutableDBEnv m blk -> blk -> m ()
appendBlockImpl       ImmutableDBEnv m blk
dbEnv
          , stream_ :: forall b.
HasCallStack =>
ResourceRegistry m
-> BlockComponent blk b
-> StreamFrom blk
-> StreamTo blk
-> m (Either (MissingBlock blk) (Iterator m blk b))
stream_            = ImmutableDBEnv m blk
-> ResourceRegistry m
-> BlockComponent blk b
-> StreamFrom blk
-> StreamTo blk
-> m (Either (MissingBlock blk) (Iterator m blk b))
forall (m :: * -> *) blk b.
(IOLike m, HasHeader blk, DecodeDisk blk (ByteString -> blk),
 DecodeDiskDep (NestedCtxt Header) blk,
 ReconstructNestedCtxt Header blk, HasCallStack) =>
ImmutableDBEnv m blk
-> ResourceRegistry m
-> BlockComponent blk b
-> StreamFrom blk
-> StreamTo blk
-> m (Either (MissingBlock blk) (Iterator m blk b))
streamImpl            ImmutableDBEnv m blk
dbEnv
          }
        internal :: Internal m blk
internal = Internal :: forall (m :: * -> *) blk.
(HasCallStack => WithOrigin (Tip blk) -> m ()) -> Internal m blk
Internal {
            deleteAfter_ :: HasCallStack => WithOrigin (Tip blk) -> m ()
deleteAfter_ = ImmutableDBEnv m blk -> WithOrigin (Tip blk) -> m ()
forall (m :: * -> *) blk.
(HasCallStack, ConvertRawHash blk, IOLike m, HasHeader blk) =>
ImmutableDBEnv m blk -> WithOrigin (Tip blk) -> m ()
deleteAfterImpl ImmutableDBEnv m blk
dbEnv
          }

    ((ImmutableDB m blk, Internal m blk), OpenState m blk h)
-> WithTempRegistry
     (OpenState m blk h)
     m
     ((ImmutableDB m blk, Internal m blk), OpenState m blk h)
forall (m :: * -> *) a. Monad m => a -> m a
return ((ImmutableDB m blk
db, Internal m blk
internal), OpenState m blk h
ost)

closeDBImpl ::
     forall m blk. (HasCallStack, IOLike m)
  => ImmutableDBEnv m blk
  -> m ()
closeDBImpl :: ImmutableDBEnv m blk -> m ()
closeDBImpl ImmutableDBEnv { HasFS m h
hasFS :: HasFS m h
hasFS :: ()
hasFS, Tracer m (TraceEvent blk)
tracer :: Tracer m (TraceEvent blk)
tracer :: forall (m :: * -> *) blk.
ImmutableDBEnv m blk -> Tracer m (TraceEvent blk)
tracer, StrictMVar m (InternalState m blk h)
varInternalState :: StrictMVar m (InternalState m blk h)
varInternalState :: ()
varInternalState } = do
    InternalState m blk h
internalState <- StrictMVar m (InternalState m blk h) -> m (InternalState m blk h)
forall (m :: * -> *) a. MonadSTM m => StrictMVar m a -> m a
takeMVar StrictMVar m (InternalState m blk h)
varInternalState
    case InternalState m blk h
internalState of
      -- Already closed
      InternalState m blk h
DbClosed -> do
        StrictMVar m (InternalState m blk h)
-> InternalState m blk h -> m ()
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
StrictMVar m a -> a -> m ()
putMVar StrictMVar m (InternalState m blk h)
varInternalState InternalState m blk h
internalState
        Tracer m (TraceEvent blk) -> TraceEvent blk -> m ()
forall (m :: * -> *) a. Tracer m a -> a -> m ()
traceWith Tracer m (TraceEvent blk)
tracer (TraceEvent blk -> m ()) -> TraceEvent blk -> m ()
forall a b. (a -> b) -> a -> b
$ TraceEvent blk
forall blk. TraceEvent blk
DBAlreadyClosed
      DbOpen OpenState m blk h
openState -> do
        -- Close the database before doing the file-system operations so that
        -- in case these fail, we don't leave the database open.
        StrictMVar m (InternalState m blk h)
-> InternalState m blk h -> m ()
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
StrictMVar m a -> a -> m ()
putMVar StrictMVar m (InternalState m blk h)
varInternalState InternalState m blk h
forall (m :: * -> *) blk h. InternalState m blk h
DbClosed
        HasFS m h -> OpenState m blk h -> m ()
forall (m :: * -> *) h blk.
Monad m =>
HasFS m h -> OpenState m blk h -> m ()
cleanUp HasFS m h
hasFS OpenState m blk h
openState
        Tracer m (TraceEvent blk) -> TraceEvent blk -> m ()
forall (m :: * -> *) a. Tracer m a -> a -> m ()
traceWith Tracer m (TraceEvent blk)
tracer TraceEvent blk
forall blk. TraceEvent blk
DBClosed

deleteAfterImpl ::
     forall m blk. (HasCallStack, ConvertRawHash blk, IOLike m, HasHeader blk)
  => ImmutableDBEnv m blk
  -> WithOrigin (Tip blk)
  -> m ()
deleteAfterImpl :: ImmutableDBEnv m blk -> WithOrigin (Tip blk) -> m ()
deleteAfterImpl dbEnv :: ImmutableDBEnv m blk
dbEnv@ImmutableDBEnv { Tracer m (TraceEvent blk)
tracer :: Tracer m (TraceEvent blk)
tracer :: forall (m :: * -> *) blk.
ImmutableDBEnv m blk -> Tracer m (TraceEvent blk)
tracer, ChunkInfo
chunkInfo :: ChunkInfo
chunkInfo :: forall (m :: * -> *) blk. ImmutableDBEnv m blk -> ChunkInfo
chunkInfo } WithOrigin (Tip blk)
newTip =
  -- We're not using 'Index' in this function but truncating the index files
  -- directly.
  ImmutableDBEnv m blk
-> (forall h. Eq h => HasFS m h -> ModifyOpenState m blk h ())
-> m ()
forall (m :: * -> *) blk a.
(HasCallStack, IOLike m, StandardHash blk, Typeable blk) =>
ImmutableDBEnv m blk
-> (forall h. Eq h => HasFS m h -> ModifyOpenState m blk h a)
-> m a
modifyOpenState ImmutableDBEnv m blk
dbEnv ((forall h. Eq h => HasFS m h -> ModifyOpenState m blk h ())
 -> m ())
-> (forall h. Eq h => HasFS m h -> ModifyOpenState m blk h ())
-> m ()
forall a b. (a -> b) -> a -> b
$ \HasFS m h
hasFS -> do
    st :: OpenState m blk h
st@OpenState { Index m blk h
currentIndex :: forall (m :: * -> *) blk h. OpenState m blk h -> Index m blk h
currentIndex :: Index m blk h
currentIndex, WithOrigin (Tip blk)
currentTip :: forall (m :: * -> *) blk h.
OpenState m blk h -> WithOrigin (Tip blk)
currentTip :: WithOrigin (Tip blk)
currentTip } <- StateT
  (OpenState m blk h)
  (WithTempRegistry (OpenState m blk h) m)
  (OpenState m blk h)
forall s (m :: * -> *). MonadState s m => m s
get

    Bool -> ModifyOpenState m blk h () -> ModifyOpenState m blk h ()
forall (f :: * -> *). Applicative f => Bool -> f () -> f ()
when ((Tip blk -> CompareTip blk
forall blk. Tip blk -> CompareTip blk
CompareTip (Tip blk -> CompareTip blk)
-> WithOrigin (Tip blk) -> WithOrigin (CompareTip blk)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> WithOrigin (Tip blk)
newTip) WithOrigin (CompareTip blk) -> WithOrigin (CompareTip blk) -> Bool
forall a. Ord a => a -> a -> Bool
< (Tip blk -> CompareTip blk
forall blk. Tip blk -> CompareTip blk
CompareTip (Tip blk -> CompareTip blk)
-> WithOrigin (Tip blk) -> WithOrigin (CompareTip blk)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> WithOrigin (Tip blk)
currentTip)) (ModifyOpenState m blk h () -> ModifyOpenState m blk h ())
-> ModifyOpenState m blk h () -> ModifyOpenState m blk h ()
forall a b. (a -> b) -> a -> b
$ do
      WithTempRegistry (OpenState m blk h) m ()
-> ModifyOpenState m blk h ()
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (WithTempRegistry (OpenState m blk h) m ()
 -> ModifyOpenState m blk h ())
-> WithTempRegistry (OpenState m blk h) m ()
-> ModifyOpenState m blk h ()
forall a b. (a -> b) -> a -> b
$ m () -> WithTempRegistry (OpenState m blk h) m ()
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (m () -> WithTempRegistry (OpenState m blk h) m ())
-> m () -> WithTempRegistry (OpenState m blk h) m ()
forall a b. (a -> b) -> a -> b
$ do
        Tracer m (TraceEvent blk) -> TraceEvent blk -> m ()
forall (m :: * -> *) a. Tracer m a -> a -> m ()
traceWith Tracer m (TraceEvent blk)
tracer (TraceEvent blk -> m ()) -> TraceEvent blk -> m ()
forall a b. (a -> b) -> a -> b
$ WithOrigin (Tip blk) -> TraceEvent blk
forall blk. WithOrigin (Tip blk) -> TraceEvent blk
DeletingAfter WithOrigin (Tip blk)
newTip
        -- Release the open handles, as we might have to remove files that are
        -- currently opened.
        HasFS m h -> OpenState m blk h -> m ()
forall (m :: * -> *) h blk.
Monad m =>
HasFS m h -> OpenState m blk h -> m ()
cleanUp HasFS m h
hasFS OpenState m blk h
st
        HasFS m h -> OpenState m blk h -> WithOrigin ChunkSlot -> m ()
forall h.
HasFS m h -> OpenState m blk h -> WithOrigin ChunkSlot -> m ()
truncateTo HasFS m h
hasFS OpenState m blk h
st WithOrigin ChunkSlot
newTipChunkSlot
        -- Reset the index, as it can contain stale information. Also restarts
        -- the background thread expiring unused past chunks.
        Index m blk h -> ChunkNo -> m ()
forall (m :: * -> *) blk h.
Index m blk h -> HasCallStack => ChunkNo -> m ()
Index.restart Index m blk h
currentIndex ChunkNo
newChunk

      OpenState m blk h
ost <- WithTempRegistry (OpenState m blk h) m (OpenState m blk h)
-> StateT
     (OpenState m blk h)
     (WithTempRegistry (OpenState m blk h) m)
     (OpenState m blk h)
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (WithTempRegistry (OpenState m blk h) m (OpenState m blk h)
 -> StateT
      (OpenState m blk h)
      (WithTempRegistry (OpenState m blk h) m)
      (OpenState m blk h))
-> WithTempRegistry (OpenState m blk h) m (OpenState m blk h)
-> StateT
     (OpenState m blk h)
     (WithTempRegistry (OpenState m blk h) m)
     (OpenState m blk h)
forall a b. (a -> b) -> a -> b
$ HasFS m h
-> Index m blk h
-> ChunkNo
-> WithOrigin (Tip blk)
-> AllowExisting
-> WithTempRegistry (OpenState m blk h) m (OpenState m blk h)
forall (m :: * -> *) blk h.
(HasCallStack, IOLike m, Eq h) =>
HasFS m h
-> Index m blk h
-> ChunkNo
-> WithOrigin (Tip blk)
-> AllowExisting
-> WithTempRegistry (OpenState m blk h) m (OpenState m blk h)
mkOpenState HasFS m h
hasFS Index m blk h
currentIndex ChunkNo
newChunk WithOrigin (Tip blk)
newTip AllowExisting
allowExisting
      OpenState m blk h -> ModifyOpenState m blk h ()
forall s (m :: * -> *). MonadState s m => s -> m ()
put OpenState m blk h
ost
  where
    newTipChunkSlot :: WithOrigin ChunkSlot
    newTipChunkSlot :: WithOrigin ChunkSlot
newTipChunkSlot = ChunkInfo -> Tip blk -> ChunkSlot
forall blk. ChunkInfo -> Tip blk -> ChunkSlot
chunkSlotForTip ChunkInfo
chunkInfo (Tip blk -> ChunkSlot)
-> WithOrigin (Tip blk) -> WithOrigin ChunkSlot
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> WithOrigin (Tip blk)
newTip

    newChunk :: ChunkNo
    allowExisting :: AllowExisting
    (ChunkNo
newChunk, AllowExisting
allowExisting) = case WithOrigin ChunkSlot
newTipChunkSlot of
      WithOrigin ChunkSlot
Origin                        -> (ChunkNo
firstChunkNo, AllowExisting
MustBeNew)
      NotOrigin (ChunkSlot ChunkNo
chunk RelativeSlot
_) -> (ChunkNo
chunk, AllowExisting
AllowExisting)

    truncateTo ::
         HasFS m h
      -> OpenState m blk h
      -> WithOrigin ChunkSlot
      -> m ()
    truncateTo :: HasFS m h -> OpenState m blk h -> WithOrigin ChunkSlot -> m ()
truncateTo HasFS m h
hasFS OpenState {} = \case
      WithOrigin ChunkSlot
Origin                       ->
        HasFS m h -> ChunkNo -> m ()
forall (m :: * -> *) h.
(HasCallStack, Monad m) =>
HasFS m h -> ChunkNo -> m ()
removeFilesStartingFrom HasFS m h
hasFS ChunkNo
firstChunkNo
      NotOrigin (ChunkSlot ChunkNo
chunk RelativeSlot
relSlot) -> do
        HasFS m h -> ChunkNo -> m ()
forall (m :: * -> *) h.
(HasCallStack, Monad m) =>
HasFS m h -> ChunkNo -> m ()
removeFilesStartingFrom HasFS m h
hasFS (ChunkNo -> ChunkNo
nextChunkNo ChunkNo
chunk)

        -- Retrieve the needed info from the primary index file and then
        -- truncate it.
        PrimaryIndex
primaryIndex <- Proxy blk -> HasFS m h -> ChunkNo -> m PrimaryIndex
forall blk (m :: * -> *) h.
(HasCallStack, MonadThrow m, StandardHash blk, Typeable blk) =>
Proxy blk -> HasFS m h -> ChunkNo -> m PrimaryIndex
Primary.load (Proxy blk
forall k (t :: k). Proxy t
Proxy @blk) HasFS m h
hasFS ChunkNo
chunk
        HasFS m h -> ChunkNo -> RelativeSlot -> m ()
forall (m :: * -> *) h.
(HasCallStack, MonadThrow m) =>
HasFS m h -> ChunkNo -> RelativeSlot -> m ()
Primary.truncateToSlotFS HasFS m h
hasFS ChunkNo
chunk RelativeSlot
relSlot
        let lastSecondaryOffset :: Word32
lastSecondaryOffset = HasCallStack => PrimaryIndex -> RelativeSlot -> Word32
PrimaryIndex -> RelativeSlot -> Word32
Primary.offsetOfSlot PrimaryIndex
primaryIndex RelativeSlot
relSlot
            isEBB :: IsEBB
isEBB               = RelativeSlot -> IsEBB
relativeSlotIsEBB RelativeSlot
relSlot

        -- Retrieve the needed info from the secondary index file and then
        -- truncate it.
        (Entry blk
entry :: Secondary.Entry blk, BlockSize
blockSize) <-
          HasFS m h -> ChunkNo -> IsEBB -> Word32 -> m (Entry blk, BlockSize)
forall (m :: * -> *) blk h.
(HasCallStack, ConvertRawHash blk, MonadThrow m, StandardHash blk,
 Typeable blk) =>
HasFS m h -> ChunkNo -> IsEBB -> Word32 -> m (Entry blk, BlockSize)
Secondary.readEntry HasFS m h
hasFS ChunkNo
chunk IsEBB
isEBB Word32
lastSecondaryOffset
        Proxy blk -> HasFS m h -> ChunkNo -> Word32 -> m ()
forall (m :: * -> *) blk h.
(HasCallStack, ConvertRawHash blk, MonadThrow m) =>
Proxy blk -> HasFS m h -> ChunkNo -> Word32 -> m ()
Secondary.truncateToEntry (Proxy blk
forall k (t :: k). Proxy t
Proxy @blk) HasFS m h
hasFS ChunkNo
chunk Word32
lastSecondaryOffset

        -- Truncate the chunk file.
        case BlockSize
blockSize of
          -- The block is the last block in the chunk file, so no need to
          -- truncate
          BlockSize
Secondary.LastEntry      -> () -> m ()
forall (m :: * -> *) a. Monad m => a -> m a
return ()
          Secondary.BlockSize Word32
size ->
              HasFS m h -> FsPath -> OpenMode -> (Handle h -> m ()) -> m ()
forall (m :: * -> *) h a.
(HasCallStack, MonadThrow m) =>
HasFS m h -> FsPath -> OpenMode -> (Handle h -> m a) -> m a
withFile HasFS m h
hasFS FsPath
chunkFile (AllowExisting -> OpenMode
AppendMode AllowExisting
AllowExisting) ((Handle h -> m ()) -> m ()) -> (Handle h -> m ()) -> m ()
forall a b. (a -> b) -> a -> b
$ \Handle h
eHnd ->
                HasFS m h -> Handle h -> Word64 -> m ()
forall (m :: * -> *) h.
HasFS m h -> HasCallStack => Handle h -> Word64 -> m ()
hTruncate HasFS m h
hasFS Handle h
eHnd Word64
offset
            where
              chunkFile :: FsPath
chunkFile = ChunkNo -> FsPath
fsPathChunkFile ChunkNo
chunk
              offset :: Word64
offset    = BlockOffset -> Word64
unBlockOffset (Entry blk -> BlockOffset
forall blk. Entry blk -> BlockOffset
Secondary.blockOffset Entry blk
entry)
                        Word64 -> Word64 -> Word64
forall a. Num a => a -> a -> a
+ Word32 -> Word64
forall a b. (Integral a, Num b) => a -> b
fromIntegral Word32
size

getTipImpl ::
     forall m blk. (HasCallStack, IOLike m, HasHeader blk)
  => ImmutableDBEnv m blk
  -> STM m (WithOrigin (Tip blk))
getTipImpl :: ImmutableDBEnv m blk -> STM m (WithOrigin (Tip blk))
getTipImpl ImmutableDBEnv m blk
dbEnv = do
    SomePair HasFS m a
_hasFS OpenState { currentTip } <- ImmutableDBEnv m blk
-> STM m (SomePair (HasFS m) (OpenState m blk))
forall (m :: * -> *) blk.
(HasCallStack, IOLike m, StandardHash blk, Typeable blk) =>
ImmutableDBEnv m blk
-> STM m (SomePair (HasFS m) (OpenState m blk))
getOpenState ImmutableDBEnv m blk
dbEnv
    WithOrigin (Tip blk) -> STM m (WithOrigin (Tip blk))
forall (m :: * -> *) a. Monad m => a -> m a
return WithOrigin (Tip blk)
currentTip

getBlockComponentImpl ::
     forall m blk b.
     ( HasHeader blk
     , ReconstructNestedCtxt Header blk
     , DecodeDisk blk (Lazy.ByteString -> blk)
     , DecodeDiskDep (NestedCtxt Header) blk
     , IOLike m
     )
  => ImmutableDBEnv m blk
  -> BlockComponent blk b
  -> RealPoint blk
  -> m (Either (MissingBlock blk) b)
getBlockComponentImpl :: ImmutableDBEnv m blk
-> BlockComponent blk b
-> RealPoint blk
-> m (Either (MissingBlock blk) b)
getBlockComponentImpl ImmutableDBEnv m blk
dbEnv BlockComponent blk b
blockComponent RealPoint blk
pt =
    ImmutableDBEnv m blk
-> (forall h.
    HasFS m h -> OpenState m blk h -> m (Either (MissingBlock blk) b))
-> m (Either (MissingBlock blk) b)
forall (m :: * -> *) blk r.
(HasCallStack, IOLike m, StandardHash blk, Typeable blk) =>
ImmutableDBEnv m blk
-> (forall h. HasFS m h -> OpenState m blk h -> m r) -> m r
withOpenState ImmutableDBEnv m blk
dbEnv ((forall h.
  HasFS m h -> OpenState m blk h -> m (Either (MissingBlock blk) b))
 -> m (Either (MissingBlock blk) b))
-> (forall h.
    HasFS m h -> OpenState m blk h -> m (Either (MissingBlock blk) b))
-> m (Either (MissingBlock blk) b)
forall a b. (a -> b) -> a -> b
$ \HasFS m h
hasFS OpenState{Word32
WithOrigin (Tip blk)
Handle h
ChunkNo
BlockOffset
Index m blk h
currentSecondaryHandle :: forall (m :: * -> *) blk h. OpenState m blk h -> Handle h
currentPrimaryHandle :: forall (m :: * -> *) blk h. OpenState m blk h -> Handle h
currentChunkHandle :: forall (m :: * -> *) blk h. OpenState m blk h -> Handle h
currentSecondaryOffset :: forall (m :: * -> *) blk h. OpenState m blk h -> Word32
currentChunkOffset :: forall (m :: * -> *) blk h. OpenState m blk h -> BlockOffset
currentChunk :: forall (m :: * -> *) blk h. OpenState m blk h -> ChunkNo
currentIndex :: Index m blk h
currentTip :: WithOrigin (Tip blk)
currentSecondaryHandle :: Handle h
currentPrimaryHandle :: Handle h
currentChunkHandle :: Handle h
currentSecondaryOffset :: Word32
currentChunkOffset :: BlockOffset
currentChunk :: ChunkNo
currentTip :: forall (m :: * -> *) blk h.
OpenState m blk h -> WithOrigin (Tip blk)
currentIndex :: forall (m :: * -> *) blk h. OpenState m blk h -> Index m blk h
..} -> ExceptT (MissingBlock blk) m b -> m (Either (MissingBlock blk) b)
forall e (m :: * -> *) a. ExceptT e m a -> m (Either e a)
runExceptT (ExceptT (MissingBlock blk) m b -> m (Either (MissingBlock blk) b))
-> ExceptT (MissingBlock blk) m b
-> m (Either (MissingBlock blk) b)
forall a b. (a -> b) -> a -> b
$ do
      (ChunkSlot, (Entry blk, BlockSize), Word32)
slotInfo <- ChunkInfo
-> Index m blk h
-> WithOrigin (Tip blk)
-> RealPoint blk
-> ExceptT
     (MissingBlock blk) m (ChunkSlot, (Entry blk, BlockSize), Word32)
forall (m :: * -> *) blk h.
(HasCallStack, IOLike m, HasHeader blk) =>
ChunkInfo
-> Index m blk h
-> WithOrigin (Tip blk)
-> RealPoint blk
-> ExceptT
     (MissingBlock blk) m (ChunkSlot, (Entry blk, BlockSize), Word32)
getSlotInfo ChunkInfo
chunkInfo Index m blk h
currentIndex WithOrigin (Tip blk)
currentTip RealPoint blk
pt
      let (ChunkSlot ChunkNo
chunk RelativeSlot
_, (Entry blk
entry, BlockSize
blockSize), Word32
_secondaryOffset) = (ChunkSlot, (Entry blk, BlockSize), Word32)
slotInfo
          chunkFile :: FsPath
chunkFile = ChunkNo -> FsPath
fsPathChunkFile ChunkNo
chunk
          Secondary.Entry { BlockOffset
blockOffset :: BlockOffset
blockOffset :: forall blk. Entry blk -> BlockOffset
blockOffset } = Entry blk
entry

      -- TODO don't open the 'chunkFile' unless we need to. In practice,
      -- we only use this to read (raw) blocks or (raw) headers, which
      -- does require opening the 'chunkFile'. Related: #2227.
      m b -> ExceptT (MissingBlock blk) m b
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (m b -> ExceptT (MissingBlock blk) m b)
-> m b -> ExceptT (MissingBlock blk) m b
forall a b. (a -> b) -> a -> b
$ HasFS m h -> FsPath -> OpenMode -> (Handle h -> m b) -> m b
forall (m :: * -> *) h a.
(HasCallStack, MonadThrow m) =>
HasFS m h -> FsPath -> OpenMode -> (Handle h -> m a) -> m a
withFile HasFS m h
hasFS FsPath
chunkFile OpenMode
ReadMode ((Handle h -> m b) -> m b) -> (Handle h -> m b) -> m b
forall a b. (a -> b) -> a -> b
$ \Handle h
eHnd -> do

        Word32
actualBlockSize <- case BlockSize
blockSize of
          Secondary.BlockSize Word32
size
            -> Word32 -> m Word32
forall (m :: * -> *) a. Monad m => a -> m a
return Word32
size
          -- See the 'GetBlock' case for more info about
          -- 'Secondary.LastEntry'.
          BlockSize
Secondary.LastEntry
            | ChunkNo
chunk ChunkNo -> ChunkNo -> Bool
forall a. Eq a => a -> a -> Bool
== ChunkNo
currentChunk
            -> Word32 -> m Word32
forall (m :: * -> *) a. Monad m => a -> m a
return (Word32 -> m Word32) -> Word32 -> m Word32
forall a b. (a -> b) -> a -> b
$ BlockOffset -> Word32
forall a b. (Integral a, Num b) => a -> b
fromIntegral (BlockOffset -> Word32) -> BlockOffset -> Word32
forall a b. (a -> b) -> a -> b
$ BlockOffset
currentChunkOffset BlockOffset -> BlockOffset -> BlockOffset
forall a. Num a => a -> a -> a
- BlockOffset
blockOffset
            | Bool
otherwise
            -> do
              -- With cached indices, we'll never hit this case.
              Word64
offsetAfterLastBlock <- HasFS m h -> Handle h -> m Word64
forall (m :: * -> *) h.
HasFS m h -> HasCallStack => Handle h -> m Word64
hGetSize HasFS m h
hasFS Handle h
eHnd
              Word32 -> m Word32
forall (m :: * -> *) a. Monad m => a -> m a
return (Word32 -> m Word32) -> Word32 -> m Word32
forall a b. (a -> b) -> a -> b
$ Word64 -> Word32
forall a b. (Integral a, Num b) => a -> b
fromIntegral (Word64 -> Word32) -> Word64 -> Word32
forall a b. (a -> b) -> a -> b
$
                Word64
offsetAfterLastBlock Word64 -> Word64 -> Word64
forall a. Num a => a -> a -> a
- BlockOffset -> Word64
unBlockOffset BlockOffset
blockOffset

        HasFS m h
-> ChunkInfo
-> ChunkNo
-> CodecConfig blk
-> (blk -> Bool)
-> Handle h
-> WithBlockSize (Entry blk)
-> BlockComponent blk b
-> m b
forall (m :: * -> *) blk b h.
(HasHeader blk, ReconstructNestedCtxt Header blk,
 DecodeDisk blk (ByteString -> blk),
 DecodeDiskDep (NestedCtxt Header) blk, IOLike m) =>
HasFS m h
-> ChunkInfo
-> ChunkNo
-> CodecConfig blk
-> (blk -> Bool)
-> Handle h
-> WithBlockSize (Entry blk)
-> BlockComponent blk b
-> m b
extractBlockComponent
          HasFS m h
hasFS
          ChunkInfo
chunkInfo
          ChunkNo
chunk
          CodecConfig blk
codecConfig
          blk -> Bool
checkIntegrity
          Handle h
eHnd
          (Word32 -> Entry blk -> WithBlockSize (Entry blk)
forall a. Word32 -> a -> WithBlockSize a
WithBlockSize Word32
actualBlockSize Entry blk
entry)
          BlockComponent blk b
blockComponent
  where
    ImmutableDBEnv { ChunkInfo
chunkInfo, CodecConfig blk
codecConfig, blk -> Bool
checkIntegrity } = ImmutableDBEnv m blk
dbEnv

appendBlockImpl ::
     forall m blk.
     ( HasHeader blk
     , GetHeader blk
     , EncodeDisk blk blk
     , HasBinaryBlockInfo blk
     , IOLike m
     , HasCallStack
     )
  => ImmutableDBEnv m blk
  -> blk
  -> m ()
appendBlockImpl :: ImmutableDBEnv m blk -> blk -> m ()
appendBlockImpl ImmutableDBEnv m blk
dbEnv blk
blk =
    ImmutableDBEnv m blk
-> (forall h. Eq h => HasFS m h -> ModifyOpenState m blk h ())
-> m ()
forall (m :: * -> *) blk a.
(HasCallStack, IOLike m, StandardHash blk, Typeable blk) =>
ImmutableDBEnv m blk
-> (forall h. Eq h => HasFS m h -> ModifyOpenState m blk h a)
-> m a
modifyOpenState ImmutableDBEnv m blk
dbEnv ((forall h. Eq h => HasFS m h -> ModifyOpenState m blk h ())
 -> m ())
-> (forall h. Eq h => HasFS m h -> ModifyOpenState m blk h ())
-> m ()
forall a b. (a -> b) -> a -> b
$ \HasFS m h
hasFS -> do
      OpenState {
          currentTip :: forall (m :: * -> *) blk h.
OpenState m blk h -> WithOrigin (Tip blk)
currentTip   = WithOrigin (Tip blk)
initialTip
        , currentIndex :: forall (m :: * -> *) blk h. OpenState m blk h -> Index m blk h
currentIndex = Index m blk h
index
        , currentChunk :: forall (m :: * -> *) blk h. OpenState m blk h -> ChunkNo
currentChunk = ChunkNo
initialChunk
        } <- StateT
  (OpenState m blk h)
  (WithTempRegistry (OpenState m blk h) m)
  (OpenState m blk h)
forall s (m :: * -> *). MonadState s m => m s
get

      -- Check that we're not appending to the past
      let blockAfterTip :: Bool
blockAfterTip =
            CompareTip blk -> WithOrigin (CompareTip blk)
forall t. t -> WithOrigin t
NotOrigin (Tip blk -> CompareTip blk
forall blk. Tip blk -> CompareTip blk
CompareTip Tip blk
blockTip) WithOrigin (CompareTip blk) -> WithOrigin (CompareTip blk) -> Bool
forall a. Ord a => a -> a -> Bool
> (Tip blk -> CompareTip blk
forall blk. Tip blk -> CompareTip blk
CompareTip (Tip blk -> CompareTip blk)
-> WithOrigin (Tip blk) -> WithOrigin (CompareTip blk)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> WithOrigin (Tip blk)
initialTip)

      Bool -> ModifyOpenState m blk h () -> ModifyOpenState m blk h ()
forall (f :: * -> *). Applicative f => Bool -> f () -> f ()
unless Bool
blockAfterTip (ModifyOpenState m blk h () -> ModifyOpenState m blk h ())
-> ModifyOpenState m blk h () -> ModifyOpenState m blk h ()
forall a b. (a -> b) -> a -> b
$ WithTempRegistry (OpenState m blk h) m ()
-> ModifyOpenState m blk h ()
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (WithTempRegistry (OpenState m blk h) m ()
 -> ModifyOpenState m blk h ())
-> WithTempRegistry (OpenState m blk h) m ()
-> ModifyOpenState m blk h ()
forall a b. (a -> b) -> a -> b
$
        ApiMisuse blk -> WithTempRegistry (OpenState m blk h) m ()
forall (m :: * -> *) blk a.
(MonadThrow m, HasCallStack, StandardHash blk, Typeable blk) =>
ApiMisuse blk -> m a
throwApiMisuse (ApiMisuse blk -> WithTempRegistry (OpenState m blk h) m ())
-> ApiMisuse blk -> WithTempRegistry (OpenState m blk h) m ()
forall a b. (a -> b) -> a -> b
$
          RealPoint blk -> Point blk -> ApiMisuse blk
forall blk. RealPoint blk -> Point blk -> ApiMisuse blk
AppendBlockNotNewerThanTipError
            (blk -> RealPoint blk
forall blk. HasHeader blk => blk -> RealPoint blk
blockRealPoint blk
blk)
            (WithOrigin (Tip blk) -> Point blk
forall blk. WithOrigin (Tip blk) -> Point blk
tipToPoint WithOrigin (Tip blk)
initialTip)

      -- If the slot is in a chunk > the current one, we have to finalise the
      -- current one and start a new chunk file, possibly skipping some chunks.
      Bool -> ModifyOpenState m blk h () -> ModifyOpenState m blk h ()
forall (f :: * -> *). Applicative f => Bool -> f () -> f ()
when (ChunkNo
chunk ChunkNo -> ChunkNo -> Bool
forall a. Ord a => a -> a -> Bool
> ChunkNo
initialChunk) (ModifyOpenState m blk h () -> ModifyOpenState m blk h ())
-> ModifyOpenState m blk h () -> ModifyOpenState m blk h ()
forall a b. (a -> b) -> a -> b
$ do
        let newChunksToStart :: Int
            newChunksToStart :: Int
newChunksToStart = Word64 -> Int
forall a b. (Integral a, Num b) => a -> b
fromIntegral (Word64 -> Int) -> Word64 -> Int
forall a b. (a -> b) -> a -> b
$ ChunkNo -> ChunkNo -> Word64
countChunks ChunkNo
chunk ChunkNo
initialChunk
        Int -> ModifyOpenState m blk h () -> ModifyOpenState m blk h ()
forall (m :: * -> *) a. Applicative m => Int -> m a -> m ()
replicateM_ Int
newChunksToStart (ModifyOpenState m blk h () -> ModifyOpenState m blk h ())
-> ModifyOpenState m blk h () -> ModifyOpenState m blk h ()
forall a b. (a -> b) -> a -> b
$
          HasFS m h
-> Index m blk h
-> ChunkInfo
-> ChunkNo
-> ModifyOpenState m blk h ()
forall (m :: * -> *) h blk.
(HasCallStack, IOLike m, Eq h) =>
HasFS m h
-> Index m blk h
-> ChunkInfo
-> ChunkNo
-> ModifyOpenState m blk h ()
startNewChunk HasFS m h
hasFS Index m blk h
index ChunkInfo
chunkInfo ChunkNo
initialChunk

      -- We may have updated the state with 'startNewChunk', so get the
      -- (possibly) updated state.
      OpenState {
          WithOrigin (Tip blk)
currentTip :: WithOrigin (Tip blk)
currentTip :: forall (m :: * -> *) blk h.
OpenState m blk h -> WithOrigin (Tip blk)
currentTip
        , Handle h
currentChunkHandle :: Handle h
currentChunkHandle :: forall (m :: * -> *) blk h. OpenState m blk h -> Handle h
currentChunkHandle
        , BlockOffset
currentChunkOffset :: BlockOffset
currentChunkOffset :: forall (m :: * -> *) blk h. OpenState m blk h -> BlockOffset
currentChunkOffset
        , Handle h
currentSecondaryHandle :: Handle h
currentSecondaryHandle :: forall (m :: * -> *) blk h. OpenState m blk h -> Handle h
currentSecondaryHandle
        , Word32
currentSecondaryOffset :: Word32
currentSecondaryOffset :: forall (m :: * -> *) blk h. OpenState m blk h -> Word32
currentSecondaryOffset
        , Handle h
currentPrimaryHandle :: Handle h
currentPrimaryHandle :: forall (m :: * -> *) blk h. OpenState m blk h -> Handle h
currentPrimaryHandle
        } <- StateT
  (OpenState m blk h)
  (WithTempRegistry (OpenState m blk h) m)
  (OpenState m blk h)
forall s (m :: * -> *). MonadState s m => m s
get

      -- Compute the next empty slot @m@, if we need to write to slot @n@, we
      -- will need to backfill @n - m@ slots.
      let nextFreeRelSlot :: RelativeSlot
          nextFreeRelSlot :: RelativeSlot
nextFreeRelSlot =
            if ChunkNo
chunk ChunkNo -> ChunkNo -> Bool
forall a. Ord a => a -> a -> Bool
> ChunkNo
initialChunk
              -- If we had to start a new chunk, we start with slot 0. Note that
              -- in this case the 'currentTip' will refer to something in a
              -- chunk before 'currentChunk'.
              then ChunkInfo -> ChunkNo -> RelativeSlot
firstBlockOrEBB ChunkInfo
chunkInfo ChunkNo
chunk
              else case WithOrigin (Tip blk)
currentTip of
                WithOrigin (Tip blk)
Origin        -> ChunkInfo -> ChunkNo -> RelativeSlot
firstBlockOrEBB ChunkInfo
chunkInfo ChunkNo
firstChunkNo
                -- Invariant: the currently open chunk is never full
                NotOrigin Tip blk
tip -> HasCallStack => RelativeSlot -> RelativeSlot
RelativeSlot -> RelativeSlot
unsafeNextRelativeSlot (RelativeSlot -> RelativeSlot)
-> (ChunkSlot -> RelativeSlot) -> ChunkSlot -> RelativeSlot
forall b c a. (b -> c) -> (a -> b) -> a -> c
. ChunkSlot -> RelativeSlot
chunkRelative (ChunkSlot -> RelativeSlot) -> ChunkSlot -> RelativeSlot
forall a b. (a -> b) -> a -> b
$
                                   ChunkInfo -> Tip blk -> ChunkSlot
forall blk. ChunkInfo -> Tip blk -> ChunkSlot
chunkSlotForTip ChunkInfo
chunkInfo Tip blk
tip

      -- Append to the end of the chunk file.
      (Word64
blockSize, Word32
entrySize) <- WithTempRegistry (OpenState m blk h) m (Word64, Word32)
-> StateT
     (OpenState m blk h)
     (WithTempRegistry (OpenState m blk h) m)
     (Word64, Word32)
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (WithTempRegistry (OpenState m blk h) m (Word64, Word32)
 -> StateT
      (OpenState m blk h)
      (WithTempRegistry (OpenState m blk h) m)
      (Word64, Word32))
-> WithTempRegistry (OpenState m blk h) m (Word64, Word32)
-> StateT
     (OpenState m blk h)
     (WithTempRegistry (OpenState m blk h) m)
     (Word64, Word32)
forall a b. (a -> b) -> a -> b
$ m (Word64, Word32)
-> WithTempRegistry (OpenState m blk h) m (Word64, Word32)
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (m (Word64, Word32)
 -> WithTempRegistry (OpenState m blk h) m (Word64, Word32))
-> m (Word64, Word32)
-> WithTempRegistry (OpenState m blk h) m (Word64, Word32)
forall a b. (a -> b) -> a -> b
$ do

          -- Write to the chunk file
          let bytes :: ByteString
bytes = Encoding -> ByteString
CBOR.toLazyByteString (Encoding -> ByteString) -> Encoding -> ByteString
forall a b. (a -> b) -> a -> b
$ CodecConfig blk -> blk -> Encoding
forall blk a. EncodeDisk blk a => CodecConfig blk -> a -> Encoding
encodeDisk CodecConfig blk
codecConfig blk
blk
          (Word64
blockSize, CRC
crc) <- 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
currentChunkHandle ByteString
bytes

          -- Write to the secondary index file
          let entry :: Entry blk
entry = Entry :: forall blk.
BlockOffset
-> HeaderOffset
-> HeaderSize
-> CRC
-> HeaderHash blk
-> BlockOrEBB
-> Entry blk
Secondary.Entry {
                  blockOffset :: BlockOffset
blockOffset  = BlockOffset
currentChunkOffset
                , headerOffset :: HeaderOffset
headerOffset = Word16 -> HeaderOffset
HeaderOffset Word16
headerOffset
                , headerSize :: HeaderSize
headerSize   = Word16 -> HeaderSize
HeaderSize Word16
headerSize
                , checksum :: CRC
checksum     = CRC
crc
                , headerHash :: HeaderHash blk
headerHash   = Tip blk -> HeaderHash blk
forall blk. Tip blk -> HeaderHash blk
tipHash Tip blk
blockTip
                , blockOrEBB :: BlockOrEBB
blockOrEBB   = BlockOrEBB
blockOrEBB
                }
          Word32
entrySize <-
            Word64 -> Word32
forall a b. (Integral a, Num b) => a -> b
fromIntegral (Word64 -> Word32) -> m Word64 -> m Word32
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$>
              Index m blk h
-> ChunkNo -> Handle h -> WithBlockSize (Entry blk) -> m Word64
forall (m :: * -> *) blk h.
Index m blk h
-> HasCallStack =>
   ChunkNo -> Handle h -> WithBlockSize (Entry blk) -> m Word64
Index.appendEntry
                Index m blk h
index
                ChunkNo
chunk
                Handle h
currentSecondaryHandle
                (Word32 -> Entry blk -> WithBlockSize (Entry blk)
forall a. Word32 -> a -> WithBlockSize a
WithBlockSize (Word64 -> Word32
forall a b. (Integral a, Num b) => a -> b
fromIntegral Word64
blockSize) Entry blk
entry)

          -- Write to the primary index file
          let backfillOffsets :: [Word32]
backfillOffsets =
                RelativeSlot -> RelativeSlot -> Word32 -> [Word32]
Primary.backfill
                  RelativeSlot
relSlot
                  RelativeSlot
nextFreeRelSlot
                  Word32
currentSecondaryOffset
              offsets :: [Word32]
offsets = [Word32]
backfillOffsets [Word32] -> [Word32] -> [Word32]
forall a. Semigroup a => a -> a -> a
<> [Word32
currentSecondaryOffset Word32 -> Word32 -> Word32
forall a. Num a => a -> a -> a
+ Word32
entrySize]
          Index m blk h -> Handle h -> [Word32] -> m ()
forall (m :: * -> *) blk h.
Index m blk h
-> forall (f :: * -> *).
   (HasCallStack, Foldable f) =>
   Handle h -> f Word32 -> m ()
Index.appendOffsets Index m blk h
index Handle h
currentPrimaryHandle [Word32]
offsets

          (Word64, Word32) -> m (Word64, Word32)
forall (m :: * -> *) a. Monad m => a -> m a
return (Word64
blockSize, Word32
entrySize)

      (OpenState m blk h -> OpenState m blk h)
-> ModifyOpenState m blk h ()
forall s (m :: * -> *). MonadState s m => (s -> s) -> m ()
modify ((OpenState m blk h -> OpenState m blk h)
 -> ModifyOpenState m blk h ())
-> (OpenState m blk h -> OpenState m blk h)
-> ModifyOpenState m blk h ()
forall a b. (a -> b) -> a -> b
$ \OpenState m blk h
st -> OpenState m blk h
st
        { currentChunkOffset :: BlockOffset
currentChunkOffset     = BlockOffset
currentChunkOffset BlockOffset -> BlockOffset -> BlockOffset
forall a. Num a => a -> a -> a
+ Word64 -> BlockOffset
forall a b. (Integral a, Num b) => a -> b
fromIntegral Word64
blockSize
        , currentSecondaryOffset :: Word32
currentSecondaryOffset = Word32
currentSecondaryOffset Word32 -> Word32 -> Word32
forall a. Num a => a -> a -> a
+ Word32
entrySize
        , currentTip :: WithOrigin (Tip blk)
currentTip             = Tip blk -> WithOrigin (Tip blk)
forall t. t -> WithOrigin t
NotOrigin Tip blk
blockTip
        }
  where
    ImmutableDBEnv { ChunkInfo
chunkInfo, CodecConfig blk
codecConfig } = ImmutableDBEnv m blk
dbEnv

    newBlockIsEBB :: Maybe EpochNo
    newBlockIsEBB :: Maybe EpochNo
newBlockIsEBB = blk -> Maybe EpochNo
forall blk. GetHeader blk => blk -> Maybe EpochNo
blockIsEBB blk
blk

    blockOrEBB :: BlockOrEBB
    blockOrEBB :: BlockOrEBB
blockOrEBB = case Maybe EpochNo
newBlockIsEBB of
        Just EpochNo
epochNo -> EpochNo -> BlockOrEBB
EBB EpochNo
epochNo
        Maybe EpochNo
Nothing      -> SlotNo -> BlockOrEBB
Block (blk -> SlotNo
forall b. HasHeader b => b -> SlotNo
blockSlot blk
blk)

    ChunkSlot ChunkNo
chunk RelativeSlot
relSlot = ChunkInfo -> BlockOrEBB -> ChunkSlot
chunkSlotForBlockOrEBB ChunkInfo
chunkInfo BlockOrEBB
blockOrEBB

    blockTip :: Tip blk
    blockTip :: Tip blk
blockTip = blk -> Tip blk
forall blk. (HasHeader blk, GetHeader blk) => blk -> Tip blk
blockToTip blk
blk

    BinaryBlockInfo {Word16
headerSize :: BinaryBlockInfo -> Word16
headerOffset :: BinaryBlockInfo -> Word16
headerSize :: Word16
headerOffset :: Word16
..} = blk -> BinaryBlockInfo
forall blk. HasBinaryBlockInfo blk => blk -> BinaryBlockInfo
getBinaryBlockInfo blk
blk

startNewChunk ::
     forall m h blk. (HasCallStack, IOLike m, Eq h)
  => HasFS m h
  -> Index m blk h
  -> ChunkInfo
  -> ChunkNo  -- ^ Chunk containing the tip
  -> ModifyOpenState m blk h ()
startNewChunk :: HasFS m h
-> Index m blk h
-> ChunkInfo
-> ChunkNo
-> ModifyOpenState m blk h ()
startNewChunk HasFS m h
hasFS Index m blk h
index ChunkInfo
chunkInfo ChunkNo
tipChunk = do
    st :: OpenState m blk h
st@OpenState {Word32
WithOrigin (Tip blk)
Handle h
ChunkNo
BlockOffset
Index m blk h
currentIndex :: Index m blk h
currentTip :: WithOrigin (Tip blk)
currentSecondaryHandle :: Handle h
currentPrimaryHandle :: Handle h
currentChunkHandle :: Handle h
currentSecondaryOffset :: Word32
currentChunkOffset :: BlockOffset
currentChunk :: ChunkNo
currentSecondaryHandle :: forall (m :: * -> *) blk h. OpenState m blk h -> Handle h
currentPrimaryHandle :: forall (m :: * -> *) blk h. OpenState m blk h -> Handle h
currentChunkHandle :: forall (m :: * -> *) blk h. OpenState m blk h -> Handle h
currentSecondaryOffset :: forall (m :: * -> *) blk h. OpenState m blk h -> Word32
currentChunkOffset :: forall (m :: * -> *) blk h. OpenState m blk h -> BlockOffset
currentChunk :: forall (m :: * -> *) blk h. OpenState m blk h -> ChunkNo
currentTip :: forall (m :: * -> *) blk h.
OpenState m blk h -> WithOrigin (Tip blk)
currentIndex :: forall (m :: * -> *) blk h. OpenState m blk h -> Index m blk h
..} <- StateT
  (OpenState m blk h)
  (WithTempRegistry (OpenState m blk h) m)
  (OpenState m blk h)
forall s (m :: * -> *). MonadState s m => m s
get

    -- We have to take care when starting multiple new chunks in a row. In the
    -- first call the tip will be in the current chunk, but in subsequent
    -- calls, the tip will still be in an chunk in the past, not the
    -- 'currentChunk'. In that case, we can't use the relative slot of the
    -- tip, since it will point to a relative slot in a past chunk. So when
    -- the current (empty) chunk is not the chunk containing the tip, we use
    -- relative slot 0 to calculate how much to pad.
    let nextFreeRelSlot :: NextRelativeSlot
        nextFreeRelSlot :: NextRelativeSlot
nextFreeRelSlot = case WithOrigin (Tip blk)
currentTip of
          WithOrigin (Tip blk)
Origin ->
            RelativeSlot -> NextRelativeSlot
NextRelativeSlot (RelativeSlot -> NextRelativeSlot)
-> RelativeSlot -> NextRelativeSlot
forall a b. (a -> b) -> a -> b
$ ChunkInfo -> ChunkNo -> RelativeSlot
firstBlockOrEBB ChunkInfo
chunkInfo ChunkNo
firstChunkNo
          NotOrigin Tip blk
tip ->
            if ChunkNo
tipChunk ChunkNo -> ChunkNo -> Bool
forall a. Eq a => a -> a -> Bool
== ChunkNo
currentChunk then
              let ChunkSlot ChunkNo
_ RelativeSlot
relSlot = ChunkInfo -> Tip blk -> ChunkSlot
forall blk. ChunkInfo -> Tip blk -> ChunkSlot
chunkSlotForTip ChunkInfo
chunkInfo Tip blk
tip
              in HasCallStack => RelativeSlot -> NextRelativeSlot
RelativeSlot -> NextRelativeSlot
nextRelativeSlot RelativeSlot
relSlot
            else
              RelativeSlot -> NextRelativeSlot
NextRelativeSlot (RelativeSlot -> NextRelativeSlot)
-> RelativeSlot -> NextRelativeSlot
forall a b. (a -> b) -> a -> b
$ ChunkInfo -> ChunkNo -> RelativeSlot
firstBlockOrEBB ChunkInfo
chunkInfo ChunkNo
currentChunk

    let backfillOffsets :: [Word32]
backfillOffsets = ChunkInfo -> ChunkNo -> NextRelativeSlot -> Word32 -> [Word32]
Primary.backfillChunk
                            ChunkInfo
chunkInfo
                            ChunkNo
currentChunk
                            NextRelativeSlot
nextFreeRelSlot
                            Word32
currentSecondaryOffset

    WithTempRegistry (OpenState m blk h) m ()
-> ModifyOpenState m blk h ()
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (WithTempRegistry (OpenState m blk h) m ()
 -> ModifyOpenState m blk h ())
-> WithTempRegistry (OpenState m blk h) m ()
-> ModifyOpenState m blk h ()
forall a b. (a -> b) -> a -> b
$ m () -> WithTempRegistry (OpenState m blk h) m ()
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (m () -> WithTempRegistry (OpenState m blk h) m ())
-> m () -> WithTempRegistry (OpenState m blk h) m ()
forall a b. (a -> b) -> a -> b
$
      Index m blk h -> Handle h -> [Word32] -> m ()
forall (m :: * -> *) blk h.
Index m blk h
-> forall (f :: * -> *).
   (HasCallStack, Foldable f) =>
   Handle h -> f Word32 -> m ()
Index.appendOffsets Index m blk h
index Handle h
currentPrimaryHandle [Word32]
backfillOffsets
      m () -> m () -> m ()
forall (m :: * -> *) a b. MonadThrow m => m a -> m b -> m a
`finally` HasFS m h -> OpenState m blk h -> m ()
forall (m :: * -> *) h blk.
Monad m =>
HasFS m h -> OpenState m blk h -> m ()
closeOpenHandles HasFS m h
hasFS OpenState m blk h
st

    OpenState m blk h
st' <- WithTempRegistry (OpenState m blk h) m (OpenState m blk h)
-> StateT
     (OpenState m blk h)
     (WithTempRegistry (OpenState m blk h) m)
     (OpenState m blk h)
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (WithTempRegistry (OpenState m blk h) m (OpenState m blk h)
 -> StateT
      (OpenState m blk h)
      (WithTempRegistry (OpenState m blk h) m)
      (OpenState m blk h))
-> WithTempRegistry (OpenState m blk h) m (OpenState m blk h)
-> StateT
     (OpenState m blk h)
     (WithTempRegistry (OpenState m blk h) m)
     (OpenState m blk h)
forall a b. (a -> b) -> a -> b
$
      HasFS m h
-> Index m blk h
-> ChunkNo
-> WithOrigin (Tip blk)
-> AllowExisting
-> WithTempRegistry (OpenState m blk h) m (OpenState m blk h)
forall (m :: * -> *) blk h.
(HasCallStack, IOLike m, Eq h) =>
HasFS m h
-> Index m blk h
-> ChunkNo
-> WithOrigin (Tip blk)
-> AllowExisting
-> WithTempRegistry (OpenState m blk h) m (OpenState m blk h)
mkOpenState HasFS m h
hasFS Index m blk h
index (ChunkNo -> ChunkNo
nextChunkNo ChunkNo
currentChunk) WithOrigin (Tip blk)
currentTip AllowExisting
MustBeNew

    OpenState m blk h -> ModifyOpenState m blk h ()
forall s (m :: * -> *). MonadState s m => s -> m ()
put OpenState m blk h
st'