{-# LANGUAGE DeriveAnyClass      #-}
{-# LANGUAGE DeriveGeneric       #-}
{-# LANGUAGE FlexibleContexts    #-}
{-# LANGUAGE GADTs               #-}
{-# LANGUAGE LambdaCase          #-}
{-# LANGUAGE NamedFieldPuns      #-}
{-# LANGUAGE RecordWildCards     #-}
{-# LANGUAGE ScopedTypeVariables #-}
{-# LANGUAGE StandaloneDeriving  #-}
module Ouroboros.Consensus.Storage.ImmutableDB.Impl.Iterator (
    CurrentChunkInfo (..)
  , extractBlockComponent
  , getSlotInfo
  , streamImpl
  ) where

import qualified Codec.CBOR.Read as CBOR
import           Control.Monad.Except
import qualified Data.ByteString.Lazy as Lazy
import qualified Data.ByteString.Short as Short
import           Data.Foldable (find)
import           Data.Functor ((<&>))
import           Data.List.NonEmpty (NonEmpty)
import qualified Data.List.NonEmpty as NE
import           GHC.Generics (Generic)

import           Cardano.Prelude (forceElemsToWHNF)

import           GHC.Stack (HasCallStack)

import           Ouroboros.Consensus.Block hiding (headerHash)
import           Ouroboros.Consensus.Util.IOLike
import           Ouroboros.Consensus.Util.ResourceRegistry (ResourceKey,
                     ResourceRegistry, allocate, release, unsafeRelease)

import           Ouroboros.Consensus.Storage.Common
import           Ouroboros.Consensus.Storage.FS.API
import           Ouroboros.Consensus.Storage.FS.API.Types
import           Ouroboros.Consensus.Storage.FS.CRC
import           Ouroboros.Consensus.Storage.Serialisation

import           Ouroboros.Consensus.Storage.ImmutableDB.API hiding
                     (getBlockComponent)
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           Ouroboros.Consensus.Storage.ImmutableDB.Impl.Index.Primary
                     (SecondaryOffset)
import           Ouroboros.Consensus.Storage.ImmutableDB.Impl.Index.Secondary
                     (BlockOffset (..), BlockSize (..))
import qualified Ouroboros.Consensus.Storage.ImmutableDB.Impl.Index.Secondary as Secondary
import           Ouroboros.Consensus.Storage.ImmutableDB.Impl.State
import           Ouroboros.Consensus.Storage.ImmutableDB.Impl.Types
import           Ouroboros.Consensus.Storage.ImmutableDB.Impl.Util

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

-- | Internal handle to an iterator.
--
-- Note: in contrast to 'IteratorState', these fields remain static for the
-- lifetime of the iterator.
data IteratorHandle m blk h = IteratorHandle {
      IteratorHandle m blk h -> HasFS m h
ithHasFS    :: !(HasFS m h)
      -- ^ Bundled HasFS instance because of the existential @h@.
    , IteratorHandle m blk h -> Index m blk h
ithIndex    :: !(Index m blk h)
      -- ^ Bundled Index instance because of the existential @h@.
    , IteratorHandle m blk h
-> StrictTVar m (IteratorStateOrExhausted m blk h)
ithVarState :: !(StrictTVar m (IteratorStateOrExhausted m blk h))
      -- ^ The state of the iterator
    , IteratorHandle m blk h -> ChunkNo
ithEndChunk :: !ChunkNo
      -- ^ The chunk in which the last block to stream is located.
    , IteratorHandle m blk h -> HeaderHash blk
ithEndHash  :: !(HeaderHash blk)
      -- ^ The has of the last block the iterator should return.
    }

data IteratorStateOrExhausted m hash h =
    IteratorStateOpen !(IteratorState m hash h)
  | IteratorStateExhausted
  deriving ((forall x.
 IteratorStateOrExhausted m hash h
 -> Rep (IteratorStateOrExhausted m hash h) x)
-> (forall x.
    Rep (IteratorStateOrExhausted m hash h) x
    -> IteratorStateOrExhausted m hash h)
-> Generic (IteratorStateOrExhausted m hash h)
forall x.
Rep (IteratorStateOrExhausted m hash h) x
-> IteratorStateOrExhausted m hash h
forall x.
IteratorStateOrExhausted m hash h
-> Rep (IteratorStateOrExhausted m hash h) x
forall a.
(forall x. a -> Rep a x) -> (forall x. Rep a x -> a) -> Generic a
forall (m :: * -> *) hash h x.
Rep (IteratorStateOrExhausted m hash h) x
-> IteratorStateOrExhausted m hash h
forall (m :: * -> *) hash h x.
IteratorStateOrExhausted m hash h
-> Rep (IteratorStateOrExhausted m hash h) x
$cto :: forall (m :: * -> *) hash h x.
Rep (IteratorStateOrExhausted m hash h) x
-> IteratorStateOrExhausted m hash h
$cfrom :: forall (m :: * -> *) hash h x.
IteratorStateOrExhausted m hash h
-> Rep (IteratorStateOrExhausted m hash h) x
Generic, Context
-> IteratorStateOrExhausted m hash h -> IO (Maybe ThunkInfo)
Proxy (IteratorStateOrExhausted m hash h) -> String
(Context
 -> IteratorStateOrExhausted m hash h -> IO (Maybe ThunkInfo))
-> (Context
    -> IteratorStateOrExhausted m hash h -> IO (Maybe ThunkInfo))
-> (Proxy (IteratorStateOrExhausted m hash h) -> String)
-> NoThunks (IteratorStateOrExhausted m hash h)
forall a.
(Context -> a -> IO (Maybe ThunkInfo))
-> (Context -> a -> IO (Maybe ThunkInfo))
-> (Proxy a -> String)
-> NoThunks a
forall (m :: * -> *) hash h.
(StandardHash hash, IOLike m) =>
Context
-> IteratorStateOrExhausted m hash h -> IO (Maybe ThunkInfo)
forall (m :: * -> *) hash h.
(StandardHash hash, IOLike m) =>
Proxy (IteratorStateOrExhausted m hash h) -> String
showTypeOf :: Proxy (IteratorStateOrExhausted m hash h) -> String
$cshowTypeOf :: forall (m :: * -> *) hash h.
(StandardHash hash, IOLike m) =>
Proxy (IteratorStateOrExhausted m hash h) -> String
wNoThunks :: Context
-> IteratorStateOrExhausted m hash h -> IO (Maybe ThunkInfo)
$cwNoThunks :: forall (m :: * -> *) hash h.
(StandardHash hash, IOLike m) =>
Context
-> IteratorStateOrExhausted m hash h -> IO (Maybe ThunkInfo)
noThunks :: Context
-> IteratorStateOrExhausted m hash h -> IO (Maybe ThunkInfo)
$cnoThunks :: forall (m :: * -> *) hash h.
(StandardHash hash, IOLike m) =>
Context
-> IteratorStateOrExhausted m hash h -> IO (Maybe ThunkInfo)
NoThunks)

data IteratorState m blk h = IteratorState {
      IteratorState m blk h -> ChunkNo
itsChunk        :: !ChunkNo
      -- ^ The current chunk the iterator is streaming from.
    , IteratorState m blk h -> Handle h
itsChunkHandle  :: !(Handle h)
      -- ^ A handle to the chunk file corresponding with 'itsChunk'.
    , IteratorState m blk h -> ResourceKey m
itsChunkKey     :: !(ResourceKey m)
      -- ^ The 'ResourceKey' corresponding to the 'itsChunkHandle'. We use it to
      -- release the handle from the 'ResourceRegistry'.
      --
      -- NOTE: if we only close the handle but don't release the resource, the
      -- registry will still hold on to the (closed) handle/resource.
    , IteratorState m blk h -> NonEmpty (WithBlockSize (Entry blk))
itsChunkEntries :: !(NonEmpty (WithBlockSize (Secondary.Entry blk)))
      -- ^ The entries from the secondary index corresponding to the current
      -- chunk. The first entry in the list is the next one to stream.
      --
      -- Invariant: all the entries in this list must be included in the stream.
      -- In other words, entries corresponding to blocks after the end bound are
      -- not included in this list.
    }
  deriving ((forall x. IteratorState m blk h -> Rep (IteratorState m blk h) x)
-> (forall x.
    Rep (IteratorState m blk h) x -> IteratorState m blk h)
-> Generic (IteratorState m blk h)
forall x. Rep (IteratorState m blk h) x -> IteratorState m blk h
forall x. IteratorState m blk h -> Rep (IteratorState m blk h) x
forall a.
(forall x. a -> Rep a x) -> (forall x. Rep a x -> a) -> Generic a
forall (m :: * -> *) blk h x.
Rep (IteratorState m blk h) x -> IteratorState m blk h
forall (m :: * -> *) blk h x.
IteratorState m blk h -> Rep (IteratorState m blk h) x
$cto :: forall (m :: * -> *) blk h x.
Rep (IteratorState m blk h) x -> IteratorState m blk h
$cfrom :: forall (m :: * -> *) blk h x.
IteratorState m blk h -> Rep (IteratorState m blk h) x
Generic)

deriving instance (StandardHash blk, IOLike m) => NoThunks (IteratorState m blk h)

-- | Auxiliary data type that combines the 'currentChunk' and
-- 'currentChunkOffset' fields from 'OpenState'. This is used to avoid passing
-- the whole state around, and moreover, it avoids issues with existential @h@
-- type parameter.
data CurrentChunkInfo = CurrentChunkInfo !ChunkNo !BlockOffset

streamImpl ::
     forall m blk b.
     ( IOLike m
     , HasHeader blk
     , DecodeDisk blk (Lazy.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
-> ResourceRegistry m
-> BlockComponent blk b
-> StreamFrom blk
-> StreamTo blk
-> m (Either (MissingBlock blk) (Iterator m blk b))
streamImpl ImmutableDBEnv m blk
dbEnv ResourceRegistry m
registry BlockComponent blk b
blockComponent = \StreamFrom blk
from StreamTo blk
to ->
    ImmutableDBEnv m blk
-> (forall h.
    HasFS m h
    -> OpenState m blk h
    -> m (Either (MissingBlock blk) (Iterator m blk b)))
-> m (Either (MissingBlock blk) (Iterator m 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) (Iterator m blk b)))
 -> m (Either (MissingBlock blk) (Iterator m blk b)))
-> (forall h.
    HasFS m h
    -> OpenState m blk h
    -> m (Either (MissingBlock blk) (Iterator m blk b)))
-> m (Either (MissingBlock blk) (Iterator m blk b))
forall a b. (a -> b) -> a -> b
$ \HasFS m h
hasFS OpenState{SecondaryOffset
WithOrigin (Tip blk)
Handle h
ChunkNo
BlockOffset
Index m blk h
currentIndex :: forall (m :: * -> *) blk h. OpenState m blk h -> Index m blk h
currentTip :: forall (m :: * -> *) blk h.
OpenState m blk h -> WithOrigin (Tip blk)
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 -> SecondaryOffset
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 :: SecondaryOffset
currentChunkOffset :: BlockOffset
currentChunk :: ChunkNo
..} -> ExceptT (MissingBlock blk) m (Iterator m blk b)
-> m (Either (MissingBlock blk) (Iterator m blk b))
forall e (m :: * -> *) a. ExceptT e m a -> m (Either e a)
runExceptT (ExceptT (MissingBlock blk) m (Iterator m blk b)
 -> m (Either (MissingBlock blk) (Iterator m blk b)))
-> ExceptT (MissingBlock blk) m (Iterator m blk b)
-> m (Either (MissingBlock blk) (Iterator m blk b))
forall a b. (a -> b) -> a -> b
$ do
      Bool
-> ExceptT (MissingBlock blk) m ()
-> ExceptT (MissingBlock blk) m ()
forall (f :: * -> *). Applicative f => Bool -> f () -> f ()
unless (StreamFrom blk -> StreamTo blk -> Bool
forall blk.
StandardHash blk =>
StreamFrom blk -> StreamTo blk -> Bool
validBounds StreamFrom blk
from StreamTo blk
to) (ExceptT (MissingBlock blk) m ()
 -> ExceptT (MissingBlock blk) m ())
-> ExceptT (MissingBlock blk) m ()
-> ExceptT (MissingBlock blk) m ()
forall a b. (a -> b) -> a -> b
$
        m () -> ExceptT (MissingBlock blk) m ()
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (m () -> ExceptT (MissingBlock blk) m ())
-> m () -> ExceptT (MissingBlock blk) m ()
forall a b. (a -> b) -> a -> b
$ ApiMisuse blk -> m ()
forall (m :: * -> *) blk a.
(MonadThrow m, HasCallStack, StandardHash blk, Typeable blk) =>
ApiMisuse blk -> m a
throwApiMisuse (ApiMisuse blk -> m ()) -> ApiMisuse blk -> m ()
forall a b. (a -> b) -> a -> b
$ StreamFrom blk -> StreamTo blk -> ApiMisuse blk
forall blk. StreamFrom blk -> StreamTo blk -> ApiMisuse blk
InvalidIteratorRangeError StreamFrom blk
from StreamTo blk
to

      ChunkSlot
endChunkSlot <- Index m blk h
-> WithOrigin (Tip blk)
-> StreamTo blk
-> ExceptT (MissingBlock blk) m ChunkSlot
forall h.
HasCallStack =>
Index m blk h
-> WithOrigin (Tip blk)
-> StreamTo blk
-> ExceptT (MissingBlock blk) m ChunkSlot
checkUpperBound Index m blk h
currentIndex WithOrigin (Tip blk)
currentTip StreamTo blk
to

      --  When the lower bound is exclusive, we do the same as when it is
      --  inclusive. We set up the iterator to point at the lower bound. Only at
      --  the very end of this function do we advance it to the block after it,
      --  in the case of an exclusive lower bound.
      (SecondaryOffset
secondaryOffset, ChunkSlot
startChunkSlot) <-
        Index m blk h
-> WithOrigin (Tip blk)
-> StreamFrom blk
-> ExceptT (MissingBlock blk) m (SecondaryOffset, ChunkSlot)
forall h.
HasCallStack =>
Index m blk h
-> WithOrigin (Tip blk)
-> StreamFrom blk
-> ExceptT (MissingBlock blk) m (SecondaryOffset, ChunkSlot)
checkLowerBound Index m blk h
currentIndex WithOrigin (Tip blk)
currentTip StreamFrom blk
from

      m (Iterator m blk b)
-> ExceptT (MissingBlock blk) m (Iterator m blk b)
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (m (Iterator m blk b)
 -> ExceptT (MissingBlock blk) m (Iterator m blk b))
-> m (Iterator m blk b)
-> ExceptT (MissingBlock blk) m (Iterator m blk b)
forall a b. (a -> b) -> a -> b
$ do
        -- 'validBounds' will catch nearly all invalid ranges, except for one:
        -- streaming from the regular block to the EBB in the same slot. The
        -- EBB comes before the regular block, so these bounds are invalid.
        -- However, to distinguish the EBB from the regular block, as both
        -- have the same slot number, we need to look at the hashes.
        -- 'validateBounds' doesn't have enough information to do that.
        Bool -> m () -> m ()
forall (f :: * -> *). Applicative f => Bool -> f () -> f ()
when (ChunkSlot
startChunkSlot ChunkSlot -> ChunkSlot -> Bool
forall a. Ord a => a -> a -> Bool
> ChunkSlot
endChunkSlot) (m () -> m ()) -> m () -> m ()
forall a b. (a -> b) -> a -> b
$
          ApiMisuse blk -> m ()
forall (m :: * -> *) blk a.
(MonadThrow m, HasCallStack, StandardHash blk, Typeable blk) =>
ApiMisuse blk -> m a
throwApiMisuse (ApiMisuse blk -> m ()) -> ApiMisuse blk -> m ()
forall a b. (a -> b) -> a -> b
$ StreamFrom blk -> StreamTo blk -> ApiMisuse blk
forall blk. StreamFrom blk -> StreamTo blk -> ApiMisuse blk
InvalidIteratorRangeError StreamFrom blk
from StreamTo blk
to

        let ChunkSlot ChunkNo
startChunk RelativeSlot
startRelSlot = ChunkSlot
startChunkSlot
            startIsEBB :: IsEBB
startIsEBB = RelativeSlot -> IsEBB
relativeSlotIsEBB RelativeSlot
startRelSlot
            currentChunkInfo :: CurrentChunkInfo
currentChunkInfo = ChunkNo -> BlockOffset -> CurrentChunkInfo
CurrentChunkInfo ChunkNo
currentChunk BlockOffset
currentChunkOffset
            endHash :: HeaderHash blk
endHash = case StreamTo blk
to of
              StreamToInclusive (RealPoint SlotNo
_slot HeaderHash blk
hash) -> HeaderHash blk
hash

        IteratorState m blk h
iteratorState <-
          HasFS m h
-> Index m blk h
-> ResourceRegistry m
-> CurrentChunkInfo
-> HeaderHash blk
-> ChunkNo
-> SecondaryOffset
-> IsEBB
-> m (IteratorState m blk h)
forall blk (m :: * -> *) h.
(HasCallStack, HasHeader blk, IOLike m) =>
HasFS m h
-> Index m blk h
-> ResourceRegistry m
-> CurrentChunkInfo
-> HeaderHash blk
-> ChunkNo
-> SecondaryOffset
-> IsEBB
-> m (IteratorState m blk h)
iteratorStateForChunk
            HasFS m h
hasFS
            Index m blk h
currentIndex
            ResourceRegistry m
registry
            CurrentChunkInfo
currentChunkInfo
            HeaderHash blk
endHash
            ChunkNo
startChunk
            SecondaryOffset
secondaryOffset
            IsEBB
startIsEBB

        StrictTVar m (IteratorStateOrExhausted m blk h)
varIteratorState <- IteratorStateOrExhausted m blk h
-> m (StrictTVar m (IteratorStateOrExhausted m blk h))
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack, NoThunks a) =>
a -> m (StrictTVar m a)
newTVarIO (IteratorStateOrExhausted m blk h
 -> m (StrictTVar m (IteratorStateOrExhausted m blk h)))
-> IteratorStateOrExhausted m blk h
-> m (StrictTVar m (IteratorStateOrExhausted m blk h))
forall a b. (a -> b) -> a -> b
$ IteratorState m blk h -> IteratorStateOrExhausted m blk h
forall (m :: * -> *) hash h.
IteratorState m hash h -> IteratorStateOrExhausted m hash h
IteratorStateOpen IteratorState m blk h
iteratorState

        let ith :: IteratorHandle m blk h
ith = IteratorHandle :: forall (m :: * -> *) blk h.
HasFS m h
-> Index m blk h
-> StrictTVar m (IteratorStateOrExhausted m blk h)
-> ChunkNo
-> HeaderHash blk
-> IteratorHandle m blk h
IteratorHandle {
                ithHasFS :: HasFS m h
ithHasFS    = HasFS m h
hasFS
              , ithIndex :: Index m blk h
ithIndex    = Index m blk h
currentIndex
              , ithVarState :: StrictTVar m (IteratorStateOrExhausted m blk h)
ithVarState = StrictTVar m (IteratorStateOrExhausted m blk h)
varIteratorState
              , ithEndChunk :: ChunkNo
ithEndChunk = ChunkSlot -> ChunkNo
chunkIndex ChunkSlot
endChunkSlot
              , ithEndHash :: HeaderHash blk
ithEndHash  = HeaderHash blk
endHash
              }

        -- When streaming from an exclusive lower bound that is not genesis, we
        -- have opened the iterator at the bound itself, so we have to skip it
        -- first.
        case StreamFrom blk
from of
          StreamFromExclusive (BlockPoint {}) ->
            ResourceRegistry m
-> CurrentChunkInfo -> IteratorHandle m blk h -> m ()
forall (m :: * -> *) blk h.
(HasCallStack, IOLike m, HasHeader blk) =>
ResourceRegistry m
-> CurrentChunkInfo -> IteratorHandle m blk h -> m ()
stepIterator ResourceRegistry m
registry CurrentChunkInfo
currentChunkInfo IteratorHandle m blk h
ith
          StreamFrom blk
_otherwise -> () -> m ()
forall (m :: * -> *) a. Monad m => a -> m a
return ()

        Iterator m blk b -> m (Iterator m blk b)
forall (m :: * -> *) a. Monad m => a -> m a
return (Iterator m blk b -> m (Iterator m blk b))
-> Iterator m blk b -> m (Iterator m blk b)
forall a b. (a -> b) -> a -> b
$ IteratorHandle m blk h -> Iterator m blk b
forall h. IteratorHandle m blk h -> Iterator m blk b
mkIterator IteratorHandle m blk h
ith
  where
    ImmutableDBEnv { ChunkInfo
chunkInfo } = ImmutableDBEnv m blk
dbEnv

    -- | Check the upper bound: check whether it exists in the database (return
    -- a 'MissingBlock' otherwise), and return the corresponding 'ChunkSlot'.
    checkUpperBound ::
         HasCallStack
      => Index m blk h
      -> WithOrigin (Tip blk)  -- ^ Current tip
      -> StreamTo blk
      -> ExceptT (MissingBlock blk) m ChunkSlot
      -- ^ We can't return 'TipInfo' here because the secondary index does
      -- not give us block numbers
    checkUpperBound :: Index m blk h
-> WithOrigin (Tip blk)
-> StreamTo blk
-> ExceptT (MissingBlock blk) m ChunkSlot
checkUpperBound Index m blk h
index WithOrigin (Tip blk)
currentTip (StreamToInclusive RealPoint blk
endPt) = do
        (ChunkSlot
chunkSlot, (Entry blk, BlockSize)
_, SecondaryOffset
_) <- ChunkInfo
-> Index m blk h
-> WithOrigin (Tip blk)
-> RealPoint blk
-> ExceptT
     (MissingBlock blk)
     m
     (ChunkSlot, (Entry blk, BlockSize), SecondaryOffset)
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), SecondaryOffset)
getSlotInfo ChunkInfo
chunkInfo Index m blk h
index WithOrigin (Tip blk)
currentTip RealPoint blk
endPt
        ChunkSlot -> ExceptT (MissingBlock blk) m ChunkSlot
forall (m :: * -> *) a. Monad m => a -> m a
return ChunkSlot
chunkSlot

    -- | Check the lower bound: check whether it exists in the database (return
    -- a 'MissingBlock' otherwise), and return the corresponding 'ChunkSlot' and
    -- 'SecondaryOffset'.
    --
    -- PRECONDITION: the end bound has been checked already
    --
    -- PRECONDITION: the bounds passed the 'validBounds' check
    --
    -- Both preconditions combined guarantee us that the tip is not origin and
    -- that the lower bound is <= the tip.
    checkLowerBound ::
         HasCallStack
      => Index m blk h
      -> WithOrigin (Tip blk)  -- ^ Current tip
      -> StreamFrom blk
      -> ExceptT (MissingBlock blk) m (SecondaryOffset, ChunkSlot)
    checkLowerBound :: Index m blk h
-> WithOrigin (Tip blk)
-> StreamFrom blk
-> ExceptT (MissingBlock blk) m (SecondaryOffset, ChunkSlot)
checkLowerBound Index m blk h
index WithOrigin (Tip blk)
currentTip = \case
        StreamFromInclusive RealPoint blk
startPt -> do
          (ChunkSlot
chunkSlot, (Entry blk, BlockSize)
_, SecondaryOffset
secondaryOffset) <-
            ChunkInfo
-> Index m blk h
-> WithOrigin (Tip blk)
-> RealPoint blk
-> ExceptT
     (MissingBlock blk)
     m
     (ChunkSlot, (Entry blk, BlockSize), SecondaryOffset)
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), SecondaryOffset)
getSlotInfo ChunkInfo
chunkInfo Index m blk h
index WithOrigin (Tip blk)
currentTip RealPoint blk
startPt
          (SecondaryOffset, ChunkSlot)
-> ExceptT (MissingBlock blk) m (SecondaryOffset, ChunkSlot)
forall (m :: * -> *) a. Monad m => a -> m a
return (SecondaryOffset
secondaryOffset, ChunkSlot
chunkSlot)
        StreamFromExclusive Point blk
startPt -> case Point blk -> WithOrigin (RealPoint blk)
forall blk. Point blk -> WithOrigin (RealPoint blk)
pointToWithOriginRealPoint Point blk
startPt of
          WithOrigin (RealPoint blk)
Origin             -> m (SecondaryOffset, ChunkSlot)
-> ExceptT (MissingBlock blk) m (SecondaryOffset, ChunkSlot)
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (m (SecondaryOffset, ChunkSlot)
 -> ExceptT (MissingBlock blk) m (SecondaryOffset, ChunkSlot))
-> m (SecondaryOffset, ChunkSlot)
-> ExceptT (MissingBlock blk) m (SecondaryOffset, ChunkSlot)
forall a b. (a -> b) -> a -> b
$ Index m blk h -> m (SecondaryOffset, ChunkSlot)
forall h.
HasCallStack =>
Index m blk h -> m (SecondaryOffset, ChunkSlot)
findFirstBlock Index m blk h
index
          NotOrigin RealPoint blk
startPt' -> do
            (ChunkSlot
chunkSlot, (Entry blk, BlockSize)
_, SecondaryOffset
secondaryOffset) <-
              ChunkInfo
-> Index m blk h
-> WithOrigin (Tip blk)
-> RealPoint blk
-> ExceptT
     (MissingBlock blk)
     m
     (ChunkSlot, (Entry blk, BlockSize), SecondaryOffset)
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), SecondaryOffset)
getSlotInfo ChunkInfo
chunkInfo Index m blk h
index WithOrigin (Tip blk)
currentTip RealPoint blk
startPt'
            (SecondaryOffset, ChunkSlot)
-> ExceptT (MissingBlock blk) m (SecondaryOffset, ChunkSlot)
forall (m :: * -> *) a. Monad m => a -> m a
return (SecondaryOffset
secondaryOffset, ChunkSlot
chunkSlot)

    mkIterator :: IteratorHandle m blk h -> Iterator m blk b
    mkIterator :: IteratorHandle m blk h -> Iterator m blk b
mkIterator IteratorHandle m blk h
ith = Iterator :: forall (m :: * -> *) blk b.
(HasCallStack => m (IteratorResult b))
-> (HasCallStack => STM m (Maybe (RealPoint blk)))
-> (HasCallStack => m ())
-> Iterator m blk b
Iterator {
          iteratorNext :: HasCallStack => m (IteratorResult b)
iteratorNext    = ImmutableDBEnv m blk
-> IteratorHandle m blk h
-> ResourceRegistry m
-> BlockComponent blk b
-> m (IteratorResult b)
forall (m :: * -> *) blk b h.
(IOLike m, HasHeader blk, DecodeDisk blk (ByteString -> blk),
 DecodeDiskDep (NestedCtxt Header) blk,
 ReconstructNestedCtxt Header blk) =>
ImmutableDBEnv m blk
-> IteratorHandle m blk h
-> ResourceRegistry m
-> BlockComponent blk b
-> m (IteratorResult b)
iteratorNextImpl    ImmutableDBEnv m blk
dbEnv IteratorHandle m blk h
ith ResourceRegistry m
registry BlockComponent blk b
blockComponent
        , iteratorHasNext :: HasCallStack => STM m (Maybe (RealPoint blk))
iteratorHasNext = ImmutableDBEnv m blk
-> IteratorHandle m blk h -> STM m (Maybe (RealPoint blk))
forall (m :: * -> *) blk h.
IOLike m =>
ImmutableDBEnv m blk
-> IteratorHandle m blk h -> STM m (Maybe (RealPoint blk))
iteratorHasNextImpl ImmutableDBEnv m blk
dbEnv IteratorHandle m blk h
ith
        , iteratorClose :: HasCallStack => m ()
iteratorClose   = IteratorHandle m blk h -> m ()
forall (m :: * -> *) blk h.
(HasCallStack, IOLike m) =>
IteratorHandle m blk h -> m ()
iteratorCloseImpl         IteratorHandle m blk h
ith
        }

    -- | Find the 'SecondaryOffset' and 'ChunkSlot' corresponding to the first
    -- block in the ImmutableDB.
    --
    -- PRECONDITION: the ImmutableDB is not empty.
    findFirstBlock ::
         HasCallStack
      => Index m blk h
      -> m (SecondaryOffset, ChunkSlot)
    findFirstBlock :: Index m blk h -> m (SecondaryOffset, ChunkSlot)
findFirstBlock Index m blk h
index = ChunkNo -> m (SecondaryOffset, ChunkSlot)
go ChunkNo
firstChunkNo
      where
        go :: ChunkNo -> m (SecondaryOffset, ChunkSlot)
        go :: ChunkNo -> m (SecondaryOffset, ChunkSlot)
go ChunkNo
chunk = Index m blk h -> ChunkNo -> m (Maybe RelativeSlot)
forall (m :: * -> *) blk h.
Index m blk h -> HasCallStack => ChunkNo -> m (Maybe RelativeSlot)
Index.readFirstFilledSlot Index m blk h
index ChunkNo
chunk m (Maybe RelativeSlot)
-> (Maybe RelativeSlot -> m (SecondaryOffset, ChunkSlot))
-> m (SecondaryOffset, ChunkSlot)
forall (m :: * -> *) a b. Monad m => m a -> (a -> m b) -> m b
>>= \case
          Maybe RelativeSlot
Nothing      -> ChunkNo -> m (SecondaryOffset, ChunkSlot)
go (ChunkNo -> ChunkNo
nextChunkNo ChunkNo
chunk)
          Just RelativeSlot
relSlot -> (SecondaryOffset, ChunkSlot) -> m (SecondaryOffset, ChunkSlot)
forall (m :: * -> *) a. Monad m => a -> m a
return (SecondaryOffset
0, ChunkNo -> RelativeSlot -> ChunkSlot
chunkSlotForRelativeSlot ChunkNo
chunk RelativeSlot
relSlot)

-- | Get information about the block or EBB at the given slot with the given
-- hash. If no such block exists, because the slot is empty, it contains a block
-- and/or EBB with a different hash, or it is newer than the current tip, return
-- a 'MissingBlock'.
--
-- Return the 'ChunkSlot' corresponding to the block or EBB, the corresponding
-- entry (and 'BlockSize') from the secondary index file, and the
-- 'SecondaryOffset' of that entry.
--
-- The primary index is read to find out whether the slot is filled and what
-- the 'SecondaryOffset' is for the slot. The secondary index is read to check
-- the hash and to return the 'Secondary.Entry'.
getSlotInfo ::
     forall m blk h. (HasCallStack, IOLike m, HasHeader blk)
  => ChunkInfo
  -> Index m blk h
  -> WithOrigin (Tip blk)  -- ^ Current tip
  -> RealPoint blk
  -> ExceptT (MissingBlock blk) m
             ( ChunkSlot
             , (Secondary.Entry blk, BlockSize)
             , SecondaryOffset
             )
getSlotInfo :: ChunkInfo
-> Index m blk h
-> WithOrigin (Tip blk)
-> RealPoint blk
-> ExceptT
     (MissingBlock blk)
     m
     (ChunkSlot, (Entry blk, BlockSize), SecondaryOffset)
getSlotInfo ChunkInfo
chunkInfo Index m blk h
index WithOrigin (Tip blk)
currentTip pt :: RealPoint blk
pt@(RealPoint SlotNo
slot HeaderHash blk
hash) = do
    let (ChunkNo
chunk, Maybe ChunkSlot
mIfBoundary, ChunkSlot
ifRegular) =
          HasCallStack =>
ChunkInfo -> SlotNo -> (ChunkNo, Maybe ChunkSlot, ChunkSlot)
ChunkInfo -> SlotNo -> (ChunkNo, Maybe ChunkSlot, ChunkSlot)
chunkSlotForUnknownBlock ChunkInfo
chunkInfo SlotNo
slot

    case WithOrigin (Tip blk)
currentTip of
      NotOrigin (Tip { SlotNo
tipSlotNo :: forall blk. Tip blk -> SlotNo
tipSlotNo :: SlotNo
tipSlotNo })
        | SlotNo
slot SlotNo -> SlotNo -> Bool
forall a. Ord a => a -> a -> Bool
<= SlotNo
tipSlotNo
        -> () -> ExceptT (MissingBlock blk) m ()
forall (m :: * -> *) a. Monad m => a -> m a
return ()
      WithOrigin (Tip blk)
_otherwise
        -> MissingBlock blk -> ExceptT (MissingBlock blk) m ()
forall e (m :: * -> *) a. MonadError e m => e -> m a
throwError (MissingBlock blk -> ExceptT (MissingBlock blk) m ())
-> MissingBlock blk -> ExceptT (MissingBlock blk) m ()
forall a b. (a -> b) -> a -> b
$ RealPoint blk -> Point blk -> MissingBlock blk
forall blk. RealPoint blk -> Point blk -> MissingBlock blk
NewerThanTip RealPoint blk
pt (WithOrigin (Tip blk) -> Point blk
forall blk. WithOrigin (Tip blk) -> Point blk
tipToPoint WithOrigin (Tip blk)
currentTip)

    -- Obtain the offsets in the secondary index file from the primary index
    -- file. The block /could/ still correspond to an EBB, a regular block or
    -- both. We will know which one it is when we can check the hashes from
    -- the secondary index file with the hash we have.
    NonEmpty (IsEBB, SecondaryOffset)
toRead :: NonEmpty (IsEBB, SecondaryOffset) <- case Maybe ChunkSlot
mIfBoundary of
      Just ChunkSlot
ifBoundary -> do
        Two (Maybe SecondaryOffset)
offsets <- m (Two (Maybe SecondaryOffset))
-> ExceptT (MissingBlock blk) m (Two (Maybe SecondaryOffset))
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (m (Two (Maybe SecondaryOffset))
 -> ExceptT (MissingBlock blk) m (Two (Maybe SecondaryOffset)))
-> m (Two (Maybe SecondaryOffset))
-> ExceptT (MissingBlock blk) m (Two (Maybe SecondaryOffset))
forall a b. (a -> b) -> a -> b
$ Index m blk h
-> ChunkNo -> Two RelativeSlot -> m (Two (Maybe SecondaryOffset))
forall (m :: * -> *) blk h.
Index m blk h
-> forall (t :: * -> *).
   (HasCallStack, Traversable t) =>
   ChunkNo -> t RelativeSlot -> m (t (Maybe SecondaryOffset))
Index.readOffsets Index m blk h
index ChunkNo
chunk
                            (ChunkSlot -> RelativeSlot
chunkRelative (ChunkSlot -> RelativeSlot) -> Two ChunkSlot -> Two RelativeSlot
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> ChunkSlot -> ChunkSlot -> Two ChunkSlot
forall a. a -> a -> Two a
Two ChunkSlot
ifBoundary ChunkSlot
ifRegular)
        case Two (Maybe SecondaryOffset)
offsets of
          Two Maybe SecondaryOffset
Nothing Maybe SecondaryOffset
Nothing                   ->
            MissingBlock blk
-> ExceptT (MissingBlock blk) m (NonEmpty (IsEBB, SecondaryOffset))
forall e (m :: * -> *) a. MonadError e m => e -> m a
throwError (MissingBlock blk
 -> ExceptT
      (MissingBlock blk) m (NonEmpty (IsEBB, SecondaryOffset)))
-> MissingBlock blk
-> ExceptT (MissingBlock blk) m (NonEmpty (IsEBB, SecondaryOffset))
forall a b. (a -> b) -> a -> b
$ RealPoint blk -> MissingBlock blk
forall blk. RealPoint blk -> MissingBlock blk
EmptySlot RealPoint blk
pt
          Two (Just SecondaryOffset
ebbOffset) (Just SecondaryOffset
blkOffset) ->
            NonEmpty (IsEBB, SecondaryOffset)
-> ExceptT (MissingBlock blk) m (NonEmpty (IsEBB, SecondaryOffset))
forall (m :: * -> *) a. Monad m => a -> m a
return ((IsEBB
IsEBB, SecondaryOffset
ebbOffset) (IsEBB, SecondaryOffset)
-> [(IsEBB, SecondaryOffset)] -> NonEmpty (IsEBB, SecondaryOffset)
forall a. a -> [a] -> NonEmpty a
NE.:| [(IsEBB
IsNotEBB, SecondaryOffset
blkOffset)])
          Two (Just SecondaryOffset
ebbOffset) Maybe SecondaryOffset
Nothing          ->
            NonEmpty (IsEBB, SecondaryOffset)
-> ExceptT (MissingBlock blk) m (NonEmpty (IsEBB, SecondaryOffset))
forall (m :: * -> *) a. Monad m => a -> m a
return ((IsEBB
IsEBB, SecondaryOffset
ebbOffset) (IsEBB, SecondaryOffset)
-> [(IsEBB, SecondaryOffset)] -> NonEmpty (IsEBB, SecondaryOffset)
forall a. a -> [a] -> NonEmpty a
NE.:| [])
          Two Maybe SecondaryOffset
Nothing (Just SecondaryOffset
blkOffset)          ->
            NonEmpty (IsEBB, SecondaryOffset)
-> ExceptT (MissingBlock blk) m (NonEmpty (IsEBB, SecondaryOffset))
forall (m :: * -> *) a. Monad m => a -> m a
return ((IsEBB
IsNotEBB, SecondaryOffset
blkOffset) (IsEBB, SecondaryOffset)
-> [(IsEBB, SecondaryOffset)] -> NonEmpty (IsEBB, SecondaryOffset)
forall a. a -> [a] -> NonEmpty a
NE.:| [])
      Maybe ChunkSlot
Nothing -> do
        Maybe SecondaryOffset
offset <- m (Maybe SecondaryOffset)
-> ExceptT (MissingBlock blk) m (Maybe SecondaryOffset)
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (m (Maybe SecondaryOffset)
 -> ExceptT (MissingBlock blk) m (Maybe SecondaryOffset))
-> m (Maybe SecondaryOffset)
-> ExceptT (MissingBlock blk) m (Maybe SecondaryOffset)
forall a b. (a -> b) -> a -> b
$ Index m blk h
-> ChunkNo -> RelativeSlot -> m (Maybe SecondaryOffset)
forall (m :: * -> *) blk h.
Functor m =>
Index m blk h
-> ChunkNo -> RelativeSlot -> m (Maybe SecondaryOffset)
Index.readOffset Index m blk h
index ChunkNo
chunk (ChunkSlot -> RelativeSlot
chunkRelative ChunkSlot
ifRegular)
        case Maybe SecondaryOffset
offset of
          Maybe SecondaryOffset
Nothing        ->
            MissingBlock blk
-> ExceptT (MissingBlock blk) m (NonEmpty (IsEBB, SecondaryOffset))
forall e (m :: * -> *) a. MonadError e m => e -> m a
throwError (MissingBlock blk
 -> ExceptT
      (MissingBlock blk) m (NonEmpty (IsEBB, SecondaryOffset)))
-> MissingBlock blk
-> ExceptT (MissingBlock blk) m (NonEmpty (IsEBB, SecondaryOffset))
forall a b. (a -> b) -> a -> b
$ RealPoint blk -> MissingBlock blk
forall blk. RealPoint blk -> MissingBlock blk
EmptySlot RealPoint blk
pt
          Just SecondaryOffset
blkOffset ->
            NonEmpty (IsEBB, SecondaryOffset)
-> ExceptT (MissingBlock blk) m (NonEmpty (IsEBB, SecondaryOffset))
forall (m :: * -> *) a. Monad m => a -> m a
return ((IsEBB
IsNotEBB, SecondaryOffset
blkOffset) (IsEBB, SecondaryOffset)
-> [(IsEBB, SecondaryOffset)] -> NonEmpty (IsEBB, SecondaryOffset)
forall a. a -> [a] -> NonEmpty a
NE.:| [])

    NonEmpty (Entry blk, BlockSize)
entriesWithBlockSizes :: NonEmpty (Secondary.Entry blk, BlockSize) <-
      m (NonEmpty (Entry blk, BlockSize))
-> ExceptT (MissingBlock blk) m (NonEmpty (Entry blk, BlockSize))
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (m (NonEmpty (Entry blk, BlockSize))
 -> ExceptT (MissingBlock blk) m (NonEmpty (Entry blk, BlockSize)))
-> m (NonEmpty (Entry blk, BlockSize))
-> ExceptT (MissingBlock blk) m (NonEmpty (Entry blk, BlockSize))
forall a b. (a -> b) -> a -> b
$ Index m blk h
-> ChunkNo
-> NonEmpty (IsEBB, SecondaryOffset)
-> m (NonEmpty (Entry blk, BlockSize))
forall (m :: * -> *) blk h.
Index m blk h
-> forall (t :: * -> *).
   (HasCallStack, Traversable t) =>
   ChunkNo
   -> t (IsEBB, SecondaryOffset) -> m (t (Entry blk, BlockSize))
Index.readEntries Index m blk h
index ChunkNo
chunk NonEmpty (IsEBB, SecondaryOffset)
toRead

    -- Return the entry from the secondary index file that matches the
    -- expected hash.
    (SecondaryOffset
secondaryOffset, (Entry blk
entry, BlockSize
blockSize)) <-
      case ((SecondaryOffset, (Entry blk, BlockSize)) -> Bool)
-> NonEmpty (SecondaryOffset, (Entry blk, BlockSize))
-> Maybe (SecondaryOffset, (Entry blk, BlockSize))
forall (t :: * -> *) a. Foldable t => (a -> Bool) -> t a -> Maybe a
find ((HeaderHash blk -> HeaderHash blk -> Bool
forall a. Eq a => a -> a -> Bool
== HeaderHash blk
hash) (HeaderHash blk -> Bool)
-> ((SecondaryOffset, (Entry blk, BlockSize)) -> HeaderHash blk)
-> (SecondaryOffset, (Entry blk, BlockSize))
-> Bool
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Entry blk -> HeaderHash blk
forall blk. Entry blk -> HeaderHash blk
Secondary.headerHash (Entry blk -> HeaderHash blk)
-> ((SecondaryOffset, (Entry blk, BlockSize)) -> Entry blk)
-> (SecondaryOffset, (Entry blk, BlockSize))
-> HeaderHash blk
forall b c a. (b -> c) -> (a -> b) -> a -> c
. (Entry blk, BlockSize) -> Entry blk
forall a b. (a, b) -> a
fst ((Entry blk, BlockSize) -> Entry blk)
-> ((SecondaryOffset, (Entry blk, BlockSize))
    -> (Entry blk, BlockSize))
-> (SecondaryOffset, (Entry blk, BlockSize))
-> Entry blk
forall b c a. (b -> c) -> (a -> b) -> a -> c
. (SecondaryOffset, (Entry blk, BlockSize)) -> (Entry blk, BlockSize)
forall a b. (a, b) -> b
snd)
                (NonEmpty SecondaryOffset
-> NonEmpty (Entry blk, BlockSize)
-> NonEmpty (SecondaryOffset, (Entry blk, BlockSize))
forall a b. NonEmpty a -> NonEmpty b -> NonEmpty (a, b)
NE.zip (((IsEBB, SecondaryOffset) -> SecondaryOffset)
-> NonEmpty (IsEBB, SecondaryOffset) -> NonEmpty SecondaryOffset
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap (IsEBB, SecondaryOffset) -> SecondaryOffset
forall a b. (a, b) -> b
snd NonEmpty (IsEBB, SecondaryOffset)
toRead) NonEmpty (Entry blk, BlockSize)
entriesWithBlockSizes) of
        Just (SecondaryOffset, (Entry blk, BlockSize))
found -> (SecondaryOffset, (Entry blk, BlockSize))
-> ExceptT
     (MissingBlock blk) m (SecondaryOffset, (Entry blk, BlockSize))
forall (m :: * -> *) a. Monad m => a -> m a
return (SecondaryOffset, (Entry blk, BlockSize))
found
        Maybe (SecondaryOffset, (Entry blk, BlockSize))
Nothing    -> MissingBlock blk
-> ExceptT
     (MissingBlock blk) m (SecondaryOffset, (Entry blk, BlockSize))
forall e (m :: * -> *) a. MonadError e m => e -> m a
throwError (MissingBlock blk
 -> ExceptT
      (MissingBlock blk) m (SecondaryOffset, (Entry blk, BlockSize)))
-> MissingBlock blk
-> ExceptT
     (MissingBlock blk) m (SecondaryOffset, (Entry blk, BlockSize))
forall a b. (a -> b) -> a -> b
$ RealPoint blk -> NonEmpty (HeaderHash blk) -> MissingBlock blk
forall blk.
RealPoint blk -> NonEmpty (HeaderHash blk) -> MissingBlock blk
WrongHash RealPoint blk
pt NonEmpty (HeaderHash blk)
hashes
          where
            hashes :: NonEmpty (HeaderHash blk)
hashes = Entry blk -> HeaderHash blk
forall blk. Entry blk -> HeaderHash blk
Secondary.headerHash (Entry blk -> HeaderHash blk)
-> ((Entry blk, BlockSize) -> Entry blk)
-> (Entry blk, BlockSize)
-> HeaderHash blk
forall b c a. (b -> c) -> (a -> b) -> a -> c
. (Entry blk, BlockSize) -> Entry blk
forall a b. (a, b) -> a
fst ((Entry blk, BlockSize) -> HeaderHash blk)
-> NonEmpty (Entry blk, BlockSize) -> NonEmpty (HeaderHash blk)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> NonEmpty (Entry blk, BlockSize)
entriesWithBlockSizes

    -- Use the secondary index entry to determine whether the slot + hash
    -- correspond to an EBB or a regular block.
    let chunkSlot :: ChunkSlot
chunkSlot = case (Maybe ChunkSlot
mIfBoundary, Entry blk -> BlockOrEBB
forall blk. Entry blk -> BlockOrEBB
Secondary.blockOrEBB Entry blk
entry) of
                      (Just ChunkSlot
ifBoundary, EBB EpochNo
_) -> ChunkSlot
ifBoundary
                      (Maybe ChunkSlot, BlockOrEBB)
_otherwise               -> ChunkSlot
ifRegular
    (ChunkSlot, (Entry blk, BlockSize), SecondaryOffset)
-> ExceptT
     (MissingBlock blk)
     m
     (ChunkSlot, (Entry blk, BlockSize), SecondaryOffset)
forall (m :: * -> *) a. Monad m => a -> m a
return (ChunkSlot
chunkSlot, (Entry blk
entry, BlockSize
blockSize), SecondaryOffset
secondaryOffset)


-- | Move the iterator to the next position that can be read from,
-- advancing chunks if necessary. If no next position can be found, the
-- iterator is closed.
--
-- PRECONDITION: the given 'IteratorState' matches the one stored in the
-- given 'StrictTVar'.
--
-- PRECONDITION: the iterator is not exhausted.
stepIterator ::
     forall m blk h. (HasCallStack, IOLike m, HasHeader blk)
  => ResourceRegistry m
  -> CurrentChunkInfo
  -> IteratorHandle m blk h
  -> m ()
stepIterator :: ResourceRegistry m
-> CurrentChunkInfo -> IteratorHandle m blk h -> m ()
stepIterator ResourceRegistry m
registry CurrentChunkInfo
currentChunkInfo
             ith :: IteratorHandle m blk h
ith@IteratorHandle { HasFS m h
ithHasFS :: HasFS m h
ithHasFS :: forall (m :: * -> *) blk h. IteratorHandle m blk h -> HasFS m h
ithHasFS, Index m blk h
ithIndex :: Index m blk h
ithIndex :: forall (m :: * -> *) blk h. IteratorHandle m blk h -> Index m blk h
ithIndex, StrictTVar m (IteratorStateOrExhausted m blk h)
ithVarState :: StrictTVar m (IteratorStateOrExhausted m blk h)
ithVarState :: forall (m :: * -> *) blk h.
IteratorHandle m blk h
-> StrictTVar m (IteratorStateOrExhausted m blk h)
ithVarState, ChunkNo
ithEndChunk :: ChunkNo
ithEndChunk :: forall (m :: * -> *) blk h. IteratorHandle m blk h -> ChunkNo
ithEndChunk, HeaderHash blk
ithEndHash :: HeaderHash blk
ithEndHash :: forall (m :: * -> *) blk h.
IteratorHandle m blk h -> HeaderHash blk
ithEndHash } =
    STM m (IteratorStateOrExhausted m blk h)
-> m (IteratorStateOrExhausted m blk h)
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
STM m a -> m a
atomically (StrictTVar m (IteratorStateOrExhausted m blk h)
-> STM m (IteratorStateOrExhausted m blk h)
forall (m :: * -> *) a. MonadSTM m => StrictTVar m a -> STM m a
readTVar StrictTVar m (IteratorStateOrExhausted m blk h)
ithVarState) m (IteratorStateOrExhausted m blk h)
-> (IteratorStateOrExhausted m blk h -> m ()) -> m ()
forall (m :: * -> *) a b. Monad m => m a -> (a -> m b) -> m b
>>= \case
      IteratorStateOrExhausted m blk h
IteratorStateExhausted ->
        String -> m ()
forall a. HasCallStack => String -> a
error String
"precondition violated: iterator must not be exhausted"
      IteratorStateOpen its :: IteratorState m blk h
its@IteratorState { NonEmpty (WithBlockSize (Entry blk))
itsChunkEntries :: NonEmpty (WithBlockSize (Entry blk))
itsChunkEntries :: forall (m :: * -> *) blk h.
IteratorState m blk h -> NonEmpty (WithBlockSize (Entry blk))
itsChunkEntries, ResourceKey m
itsChunkKey :: ResourceKey m
itsChunkKey :: forall (m :: * -> *) blk h. IteratorState m blk h -> ResourceKey m
itsChunkKey, ChunkNo
itsChunk :: ChunkNo
itsChunk :: forall (m :: * -> *) blk h. IteratorState m blk h -> ChunkNo
itsChunk } ->
        case [WithBlockSize (Entry blk)]
-> Maybe (NonEmpty (WithBlockSize (Entry blk)))
forall a. [a] -> Maybe (NonEmpty a)
NE.nonEmpty (NonEmpty (WithBlockSize (Entry blk)) -> [WithBlockSize (Entry blk)]
forall a. NonEmpty a -> [a]
NE.tail NonEmpty (WithBlockSize (Entry blk))
itsChunkEntries) of
          -- There are entries left in this chunk, so continue. See the
          -- invariant on 'itsChunkEntries'
          Just NonEmpty (WithBlockSize (Entry blk))
itsChunkEntries' ->
            STM m () -> m ()
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 (IteratorStateOrExhausted m blk h)
-> IteratorStateOrExhausted m blk h -> STM m ()
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
StrictTVar m a -> a -> STM m ()
writeTVar StrictTVar m (IteratorStateOrExhausted m blk h)
ithVarState (IteratorStateOrExhausted m blk h -> STM m ())
-> IteratorStateOrExhausted m blk h -> STM m ()
forall a b. (a -> b) -> a -> b
$
              IteratorState m blk h -> IteratorStateOrExhausted m blk h
forall (m :: * -> *) hash h.
IteratorState m hash h -> IteratorStateOrExhausted m hash h
IteratorStateOpen IteratorState m blk h
its { itsChunkEntries :: NonEmpty (WithBlockSize (Entry blk))
itsChunkEntries = NonEmpty (WithBlockSize (Entry blk))
itsChunkEntries' }

          -- No more entries in this chunk, so open the next.
          Maybe (NonEmpty (WithBlockSize (Entry blk)))
Nothing -> do
            -- Release the resource, i.e., close the handle.
            m (Maybe (Context m)) -> m ()
forall (f :: * -> *) a. Functor f => f a -> f ()
void (m (Maybe (Context m)) -> m ()) -> m (Maybe (Context m)) -> m ()
forall a b. (a -> b) -> a -> b
$ ResourceKey m -> m (Maybe (Context m))
forall (m :: * -> *).
(IOLike m, HasCallStack) =>
ResourceKey m -> m (Maybe (Context m))
release ResourceKey m
itsChunkKey
            -- If this was the final chunk, close the iterator
            if ChunkNo
itsChunk ChunkNo -> ChunkNo -> Bool
forall a. Ord a => a -> a -> Bool
>= ChunkNo
ithEndChunk then
              IteratorHandle m blk h -> m ()
forall (m :: * -> *) blk h.
(HasCallStack, IOLike m) =>
IteratorHandle m blk h -> m ()
iteratorCloseImpl IteratorHandle m blk h
ith
            else
              ChunkNo -> m (IteratorState m blk h)
openNextChunk (ChunkNo -> ChunkNo
nextChunkNo ChunkNo
itsChunk) m (IteratorState m blk h)
-> (IteratorState m blk h -> m ()) -> m ()
forall (m :: * -> *) a b. Monad m => m a -> (a -> m b) -> m b
>>= \IteratorState m blk h
its' ->
                STM m () -> m ()
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 (IteratorStateOrExhausted m blk h)
-> IteratorStateOrExhausted m blk h -> STM m ()
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
StrictTVar m a -> a -> STM m ()
writeTVar StrictTVar m (IteratorStateOrExhausted m blk h)
ithVarState (IteratorStateOrExhausted m blk h -> STM m ())
-> IteratorStateOrExhausted m blk h -> STM m ()
forall a b. (a -> b) -> a -> b
$ IteratorState m blk h -> IteratorStateOrExhausted m blk h
forall (m :: * -> *) hash h.
IteratorState m hash h -> IteratorStateOrExhausted m hash h
IteratorStateOpen IteratorState m blk h
its'
  where
    openNextChunk ::
         ChunkNo    -- ^ The chunk to open
      -> m (IteratorState m blk h)
    openNextChunk :: ChunkNo -> m (IteratorState m blk h)
openNextChunk ChunkNo
chunk =
      Index m blk h -> ChunkNo -> m (Maybe RelativeSlot)
forall (m :: * -> *) blk h.
Index m blk h -> HasCallStack => ChunkNo -> m (Maybe RelativeSlot)
Index.readFirstFilledSlot Index m blk h
ithIndex ChunkNo
chunk m (Maybe RelativeSlot)
-> (Maybe RelativeSlot -> m (IteratorState m blk h))
-> m (IteratorState m blk h)
forall (m :: * -> *) a b. Monad m => m a -> (a -> m b) -> m b
>>= \case
        -- This chunk is empty, look in the next one.
        --
        -- We still haven't encountered the end bound, so this loop must end
        -- when we reach the non-empty chunk containing the end bound. This
        -- cannot loop forever as an error would be thrown when opening the
        -- index file(s) of a non-existing chunk.
        Maybe RelativeSlot
Nothing      -> ChunkNo -> m (IteratorState m blk h)
openNextChunk (ChunkNo -> ChunkNo
nextChunkNo ChunkNo
chunk)
        Just RelativeSlot
relSlot -> do
          -- Note that the only reason we actually open the primary index file
          -- is to see whether the first block in the chunk is an EBB or not.
          -- To see whether the chunk is empty, we could open the secondary
          -- index file directly and see whether it contains any blocks. The
          -- 'secondaryOffset' will be 0, as the first entry in the secondary
          -- index file always starts at offset 0. The same is true for
          -- 'findFirstFilledSlot'.
          let firstIsEBB :: IsEBB
firstIsEBB      = RelativeSlot -> IsEBB
relativeSlotIsEBB RelativeSlot
relSlot
              secondaryOffset :: SecondaryOffset
secondaryOffset = SecondaryOffset
0

          HasFS m h
-> Index m blk h
-> ResourceRegistry m
-> CurrentChunkInfo
-> HeaderHash blk
-> ChunkNo
-> SecondaryOffset
-> IsEBB
-> m (IteratorState m blk h)
forall blk (m :: * -> *) h.
(HasCallStack, HasHeader blk, IOLike m) =>
HasFS m h
-> Index m blk h
-> ResourceRegistry m
-> CurrentChunkInfo
-> HeaderHash blk
-> ChunkNo
-> SecondaryOffset
-> IsEBB
-> m (IteratorState m blk h)
iteratorStateForChunk
            HasFS m h
ithHasFS
            Index m blk h
ithIndex
            ResourceRegistry m
registry
            CurrentChunkInfo
currentChunkInfo
            HeaderHash blk
ithEndHash
            ChunkNo
chunk
            SecondaryOffset
secondaryOffset
            IsEBB
firstIsEBB


iteratorNextImpl ::
     forall m blk b h.
     ( IOLike m
     , HasHeader blk
     , DecodeDisk blk (Lazy.ByteString -> blk)
     , DecodeDiskDep (NestedCtxt Header) blk
     , ReconstructNestedCtxt Header blk
     )
  => ImmutableDBEnv m blk
  -> IteratorHandle m blk h
  -> ResourceRegistry m
  -> BlockComponent blk b
  -> m (IteratorResult b)
iteratorNextImpl :: ImmutableDBEnv m blk
-> IteratorHandle m blk h
-> ResourceRegistry m
-> BlockComponent blk b
-> m (IteratorResult b)
iteratorNextImpl ImmutableDBEnv m blk
dbEnv ith :: IteratorHandle m blk h
ith@IteratorHandle { HasFS m h
ithHasFS :: HasFS m h
ithHasFS :: forall (m :: * -> *) blk h. IteratorHandle m blk h -> HasFS m h
ithHasFS, StrictTVar m (IteratorStateOrExhausted m blk h)
ithVarState :: StrictTVar m (IteratorStateOrExhausted m blk h)
ithVarState :: forall (m :: * -> *) blk h.
IteratorHandle m blk h
-> StrictTVar m (IteratorStateOrExhausted m blk h)
ithVarState } ResourceRegistry m
registry BlockComponent blk b
blockComponent = do
    -- The idea is that if the state is not 'IteratorStateExhausted', then the
    -- head of 'itsChunkEntries' is always ready to be read. After extracting
    -- the block component, 'stepIterator' will advance the iterator to the next
    -- block.
    STM m (IteratorStateOrExhausted m blk h)
-> m (IteratorStateOrExhausted m blk h)
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
STM m a -> m a
atomically (StrictTVar m (IteratorStateOrExhausted m blk h)
-> STM m (IteratorStateOrExhausted m blk h)
forall (m :: * -> *) a. MonadSTM m => StrictTVar m a -> STM m a
readTVar StrictTVar m (IteratorStateOrExhausted m blk h)
ithVarState) m (IteratorStateOrExhausted m blk h)
-> (IteratorStateOrExhausted m blk h -> m (IteratorResult b))
-> m (IteratorResult b)
forall (m :: * -> *) a b. Monad m => m a -> (a -> m b) -> m b
>>= \case
      -- Iterator already closed
      IteratorStateOrExhausted m blk h
IteratorStateExhausted -> IteratorResult b -> m (IteratorResult b)
forall (m :: * -> *) a. Monad m => a -> m a
return IteratorResult b
forall b. IteratorResult b
IteratorExhausted
      IteratorStateOpen IteratorState { NonEmpty (WithBlockSize (Entry blk))
itsChunkEntries :: NonEmpty (WithBlockSize (Entry blk))
itsChunkEntries :: forall (m :: * -> *) blk h.
IteratorState m blk h -> NonEmpty (WithBlockSize (Entry blk))
itsChunkEntries, ChunkNo
itsChunk :: ChunkNo
itsChunk :: forall (m :: * -> *) blk h. IteratorState m blk h -> ChunkNo
itsChunk, Handle h
itsChunkHandle :: Handle h
itsChunkHandle :: forall (m :: * -> *) blk h. IteratorState m blk h -> Handle h
itsChunkHandle } ->
        ImmutableDBEnv m blk
-> (forall h.
    HasFS m h -> OpenState m blk h -> m (IteratorResult b))
-> m (IteratorResult 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 (IteratorResult b))
 -> m (IteratorResult b))
-> (forall h.
    HasFS m h -> OpenState m blk h -> m (IteratorResult b))
-> m (IteratorResult b)
forall a b. (a -> b) -> a -> b
$ \HasFS m h
_ OpenState m blk h
st -> do
          let entry :: WithBlockSize (Entry blk)
entry = NonEmpty (WithBlockSize (Entry blk)) -> WithBlockSize (Entry blk)
forall a. NonEmpty a -> a
NE.head NonEmpty (WithBlockSize (Entry blk))
itsChunkEntries
              currentChunkInfo :: CurrentChunkInfo
currentChunkInfo = ChunkNo -> BlockOffset -> CurrentChunkInfo
CurrentChunkInfo
                                   (OpenState m blk h -> ChunkNo
forall (m :: * -> *) blk h. OpenState m blk h -> ChunkNo
currentChunk OpenState m blk h
st)
                                   (OpenState m blk h -> BlockOffset
forall (m :: * -> *) blk h. OpenState m blk h -> BlockOffset
currentChunkOffset OpenState m blk h
st)
          b
b <-
            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
ithHasFS
              ChunkInfo
chunkInfo
              ChunkNo
itsChunk
              CodecConfig blk
codecConfig
              blk -> Bool
checkIntegrity
              Handle h
itsChunkHandle
              WithBlockSize (Entry blk)
entry
              BlockComponent blk b
blockComponent
          ResourceRegistry m
-> CurrentChunkInfo -> IteratorHandle m blk h -> m ()
forall (m :: * -> *) blk h.
(HasCallStack, IOLike m, HasHeader blk) =>
ResourceRegistry m
-> CurrentChunkInfo -> IteratorHandle m blk h -> m ()
stepIterator ResourceRegistry m
registry CurrentChunkInfo
currentChunkInfo IteratorHandle m blk h
ith
          IteratorResult b -> m (IteratorResult b)
forall (m :: * -> *) a. Monad m => a -> m a
return (IteratorResult b -> m (IteratorResult b))
-> IteratorResult b -> m (IteratorResult b)
forall a b. (a -> b) -> a -> b
$ b -> IteratorResult b
forall b. b -> IteratorResult b
IteratorResult b
b
  where
    ImmutableDBEnv { CodecConfig blk
codecConfig, ChunkInfo
chunkInfo, blk -> Bool
checkIntegrity } = ImmutableDBEnv m blk
dbEnv

iteratorHasNextImpl ::
     IOLike m
  => ImmutableDBEnv m blk
  -> IteratorHandle m blk h
  -> STM m (Maybe (RealPoint blk))
iteratorHasNextImpl :: ImmutableDBEnv m blk
-> IteratorHandle m blk h -> STM m (Maybe (RealPoint blk))
iteratorHasNextImpl ImmutableDBEnv { ChunkInfo
chunkInfo :: ChunkInfo
chunkInfo :: forall (m :: * -> *) blk. ImmutableDBEnv m blk -> ChunkInfo
chunkInfo } IteratorHandle { StrictTVar m (IteratorStateOrExhausted m blk h)
ithVarState :: StrictTVar m (IteratorStateOrExhausted m blk h)
ithVarState :: forall (m :: * -> *) blk h.
IteratorHandle m blk h
-> StrictTVar m (IteratorStateOrExhausted m blk h)
ithVarState } =
    StrictTVar m (IteratorStateOrExhausted m blk h)
-> STM m (IteratorStateOrExhausted m blk h)
forall (m :: * -> *) a. MonadSTM m => StrictTVar m a -> STM m a
readTVar StrictTVar m (IteratorStateOrExhausted m blk h)
ithVarState STM m (IteratorStateOrExhausted m blk h)
-> (IteratorStateOrExhausted m blk h -> Maybe (RealPoint blk))
-> STM m (Maybe (RealPoint blk))
forall (f :: * -> *) a b. Functor f => f a -> (a -> b) -> f b
<&> \case
      IteratorStateOrExhausted m blk h
IteratorStateExhausted -> Maybe (RealPoint blk)
forall a. Maybe a
Nothing
      IteratorStateOpen IteratorState { NonEmpty (WithBlockSize (Entry blk))
itsChunkEntries :: NonEmpty (WithBlockSize (Entry blk))
itsChunkEntries :: forall (m :: * -> *) blk h.
IteratorState m blk h -> NonEmpty (WithBlockSize (Entry blk))
itsChunkEntries } ->
          RealPoint blk -> Maybe (RealPoint blk)
forall a. a -> Maybe a
Just (SlotNo -> HeaderHash blk -> RealPoint blk
forall blk. SlotNo -> HeaderHash blk -> RealPoint blk
RealPoint SlotNo
slotNo (Entry blk -> HeaderHash blk
forall blk. Entry blk -> HeaderHash blk
Secondary.headerHash Entry blk
nextEntry))
        where
          WithBlockSize SecondaryOffset
_ Entry blk
nextEntry NE.:| [WithBlockSize (Entry blk)]
_ = NonEmpty (WithBlockSize (Entry blk))
itsChunkEntries

          slotNo :: SlotNo
          slotNo :: SlotNo
slotNo = ChunkInfo -> BlockOrEBB -> SlotNo
slotNoOfBlockOrEBB ChunkInfo
chunkInfo (Entry blk -> BlockOrEBB
forall blk. Entry blk -> BlockOrEBB
Secondary.blockOrEBB Entry blk
nextEntry)

iteratorCloseImpl ::
     (HasCallStack, IOLike m)
  => IteratorHandle m blk h
  -> m ()
iteratorCloseImpl :: IteratorHandle m blk h -> m ()
iteratorCloseImpl IteratorHandle { StrictTVar m (IteratorStateOrExhausted m blk h)
ithVarState :: StrictTVar m (IteratorStateOrExhausted m blk h)
ithVarState :: forall (m :: * -> *) blk h.
IteratorHandle m blk h
-> StrictTVar m (IteratorStateOrExhausted m blk h)
ithVarState } = do
    STM m (IteratorStateOrExhausted m blk h)
-> m (IteratorStateOrExhausted m blk h)
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
STM m a -> m a
atomically (StrictTVar m (IteratorStateOrExhausted m blk h)
-> STM m (IteratorStateOrExhausted m blk h)
forall (m :: * -> *) a. MonadSTM m => StrictTVar m a -> STM m a
readTVar StrictTVar m (IteratorStateOrExhausted m blk h)
ithVarState) m (IteratorStateOrExhausted m blk h)
-> (IteratorStateOrExhausted m blk h -> m ()) -> m ()
forall (m :: * -> *) a b. Monad m => m a -> (a -> m b) -> m b
>>= \case
      -- Already closed
      IteratorStateOrExhausted m blk h
IteratorStateExhausted -> () -> m ()
forall (m :: * -> *) a. Monad m => a -> m a
return ()
      IteratorStateOpen IteratorState { ResourceKey m
itsChunkKey :: ResourceKey m
itsChunkKey :: forall (m :: * -> *) blk h. IteratorState m blk h -> ResourceKey m
itsChunkKey } -> do
        -- First set it to Nothing to indicate it is closed, as the call to
        -- 'release' might fail, which would leave the iterator open in an
        -- invalid state.
        STM m () -> m ()
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 (IteratorStateOrExhausted m blk h)
-> IteratorStateOrExhausted m blk h -> STM m ()
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
StrictTVar m a -> a -> STM m ()
writeTVar StrictTVar m (IteratorStateOrExhausted m blk h)
ithVarState IteratorStateOrExhausted m blk h
forall (m :: * -> *) hash h. IteratorStateOrExhausted m hash h
IteratorStateExhausted
        -- TODO: we must use 'unsafeRelease' instead of 'release' because we
        -- might close the iterator from an /untracked thread/, i.e., a thread
        -- that was not spawned by the resource registry (or the thread that
        -- opened the resource registry) in which the handle was allocated.
        --
        -- This happens in the consensus tests (but not in the actual node),
        -- where the protocol threads that open iterators (BlockFetchServer
        -- and ChainSyncServer) are spawned using a different resource
        -- registry (A) than the one the ImmutableDB (and ChainDB) use (B).
        -- When the ChainDB is closed (by the thread that opened B), we're
        -- closing all open iterators, i.e., the iterators opened by the
        -- protocol threads. So we're releasing handles allocated in resource
        -- registry A from a thread tracked by resource registry B. See #1390.
        m (Maybe (Context m)) -> m ()
forall (f :: * -> *) a. Functor f => f a -> f ()
void (m (Maybe (Context m)) -> m ()) -> m (Maybe (Context m)) -> m ()
forall a b. (a -> b) -> a -> b
$ ResourceKey m -> m (Maybe (Context m))
forall (m :: * -> *).
IOLike m =>
ResourceKey m -> m (Maybe (Context m))
unsafeRelease ResourceKey m
itsChunkKey

iteratorStateForChunk ::
     (HasCallStack, HasHeader blk, IOLike m)
  => HasFS m h
  -> Index m blk h
  -> ResourceRegistry m
  -> CurrentChunkInfo
  -> HeaderHash blk
     -- ^ Hash of the end bound
  -> ChunkNo
  -> SecondaryOffset
     -- ^ Where to start in the secondary index
  -> IsEBB
     -- ^ Whether the first expected block will be an EBB or not.
  -> m (IteratorState m blk h)
iteratorStateForChunk :: HasFS m h
-> Index m blk h
-> ResourceRegistry m
-> CurrentChunkInfo
-> HeaderHash blk
-> ChunkNo
-> SecondaryOffset
-> IsEBB
-> m (IteratorState m blk h)
iteratorStateForChunk HasFS m h
hasFS Index m blk h
index ResourceRegistry m
registry
                      (CurrentChunkInfo ChunkNo
curChunk BlockOffset
curChunkOffset) HeaderHash blk
endHash
                      ChunkNo
chunk SecondaryOffset
secondaryOffset IsEBB
firstIsEBB = do
    -- Open the chunk file. Allocate the handle in the registry so that it
    -- will be closed in case of an exception.
    (ResourceKey m
key, Handle h
eHnd) <- ResourceRegistry m
-> (ResourceId -> m (Handle h))
-> (Handle h -> m ())
-> m (ResourceKey m, Handle h)
forall (m :: * -> *) a.
(IOLike m, HasCallStack) =>
ResourceRegistry m
-> (ResourceId -> m a) -> (a -> m ()) -> m (ResourceKey m, a)
allocate
      ResourceRegistry m
registry
      (\ResourceId
_key -> HasCallStack => FsPath -> OpenMode -> m (Handle h)
FsPath -> OpenMode -> m (Handle h)
hOpen (ChunkNo -> FsPath
fsPathChunkFile ChunkNo
chunk) OpenMode
ReadMode)
      HasCallStack => Handle h -> m ()
Handle h -> m ()
hClose

    -- If the last entry in @entries@ corresponds to the last block in the
    -- chunk, we cannot calculate the block size based on the next block.
    -- Instead, we calculate it based on the size of the chunk file.
    --
    -- IMPORTANT: for older chunks, this is fine, as the secondary index
    -- (entries) and the chunk file (size) are immutable. However, when doing
    -- this for the current chunk, there is a potential race condition between
    -- reading of the entries from the secondary index and obtaining the chunk
    -- file size: what if a new block was appended after reading the entries
    -- but before obtaining the chunk file size? Then the chunk file size will
    -- not correspond to the last entry we read, but to the block after it.
    -- Similarly if we switch the order of the two operations.
    --
    -- To avoid this race condition, we use the value of 'currentChunkOffset'
    -- from the state as the file size of the current chunk (stored in
    -- 'CurrentChunkInfo'). This value corresponds to the chunk file size at
    -- the time we /read the state/. We also know that the end bound of our
    -- iterator is always <= the tip from that same state, so all @entries@
    -- must be <= the tip from that state because we'll never stream beyond
    -- the tip. Remember that we only actually use the current chunk file size
    -- if the last entry we have read from the secondary index is the last
    -- entry in the file, in which case it would correspond to the tip from
    -- the state. In this case, the chunk file size (@curChunkOffset@) we are
    -- passed is consistent with the tip, as it was obtained from the same
    -- consistent state.
    Word64
chunkFileSize <- if ChunkNo
chunk ChunkNo -> ChunkNo -> Bool
forall a. Eq a => a -> a -> Bool
== ChunkNo
curChunk
      then Word64 -> m Word64
forall (m :: * -> *) a. Monad m => a -> m a
return (BlockOffset -> Word64
unBlockOffset BlockOffset
curChunkOffset)
      else HasCallStack => Handle h -> m Word64
Handle h -> m Word64
hGetSize Handle h
eHnd

    [WithBlockSize (Entry blk)]
entries <- Index m blk h
-> SecondaryOffset
-> ChunkNo
-> (Entry blk -> Bool)
-> Word64
-> IsEBB
-> m [WithBlockSize (Entry blk)]
forall (m :: * -> *) blk h.
Index m blk h
-> HasCallStack =>
   SecondaryOffset
   -> ChunkNo
   -> (Entry blk -> Bool)
   -> Word64
   -> IsEBB
   -> m [WithBlockSize (Entry blk)]
Index.readAllEntries Index m blk h
index SecondaryOffset
secondaryOffset ChunkNo
chunk
      ((HeaderHash blk -> HeaderHash blk -> Bool
forall a. Eq a => a -> a -> Bool
== HeaderHash blk
endHash) (HeaderHash blk -> Bool)
-> (Entry blk -> HeaderHash blk) -> Entry blk -> Bool
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Entry blk -> HeaderHash blk
forall blk. Entry blk -> HeaderHash blk
Secondary.headerHash) Word64
chunkFileSize IsEBB
firstIsEBB

    case [WithBlockSize (Entry blk)]
-> Maybe (NonEmpty (WithBlockSize (Entry blk)))
forall a. [a] -> Maybe (NonEmpty a)
NE.nonEmpty [WithBlockSize (Entry blk)]
entries of
      -- We still haven't encountered the end bound, so it cannot be
      -- that this non-empty chunk contains no entries <= the end bound.
      Maybe (NonEmpty (WithBlockSize (Entry blk)))
Nothing             -> String -> m (IteratorState m blk h)
forall a. HasCallStack => String -> a
error
        String
"impossible: there must be entries according to the primary index"

      Just NonEmpty (WithBlockSize (Entry blk))
itsChunkEntries -> IteratorState m blk h -> m (IteratorState m blk h)
forall (m :: * -> *) a. Monad m => a -> m a
return IteratorState :: forall (m :: * -> *) blk h.
ChunkNo
-> Handle h
-> ResourceKey m
-> NonEmpty (WithBlockSize (Entry blk))
-> IteratorState m blk h
IteratorState {
          itsChunk :: ChunkNo
itsChunk        = ChunkNo
chunk
        , itsChunkHandle :: Handle h
itsChunkHandle  = Handle h
eHnd
        , itsChunkKey :: ResourceKey m
itsChunkKey     = ResourceKey m
key
          -- Force so we don't store any thunks in the state
        , itsChunkEntries :: NonEmpty (WithBlockSize (Entry blk))
itsChunkEntries = NonEmpty (WithBlockSize (Entry blk))
-> NonEmpty (WithBlockSize (Entry blk))
forall (t :: * -> *) a. Foldable t => t a -> t a
forceElemsToWHNF NonEmpty (WithBlockSize (Entry blk))
itsChunkEntries
        }
  where
    HasFS { HasCallStack => FsPath -> OpenMode -> m (Handle h)
hOpen :: forall (m :: * -> *) h.
HasFS m h -> HasCallStack => FsPath -> OpenMode -> m (Handle h)
hOpen :: HasCallStack => FsPath -> OpenMode -> m (Handle h)
hOpen, HasCallStack => Handle h -> m ()
hClose :: forall (m :: * -> *) h.
HasFS m h -> HasCallStack => Handle h -> m ()
hClose :: HasCallStack => Handle h -> m ()
hClose, HasCallStack => Handle h -> m Word64
hGetSize :: forall (m :: * -> *) h.
HasFS m h -> HasCallStack => Handle h -> m Word64
hGetSize :: HasCallStack => Handle h -> m Word64
hGetSize } = HasFS m h
hasFS

extractBlockComponent ::
     forall m blk b h.
     ( HasHeader blk
     , ReconstructNestedCtxt Header blk
     , DecodeDisk blk (Lazy.ByteString -> blk)
     , DecodeDiskDep (NestedCtxt Header) blk
     , IOLike m
     )
  => HasFS m h
  -> ChunkInfo
  -> ChunkNo
  -> CodecConfig blk
  -> (blk -> Bool)
  -> Handle h
  -> WithBlockSize (Secondary.Entry blk)
  -> BlockComponent blk b
  -> m b
extractBlockComponent :: 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
ccfg blk -> Bool
checkIntegrity Handle h
eHnd
                      (WithBlockSize SecondaryOffset
blockSize Entry blk
entry) = BlockComponent blk b -> m b
forall b'. BlockComponent blk b' -> m b'
go
  where
    go :: forall b'. BlockComponent blk b' -> m b'
    go :: BlockComponent blk b' -> m b'
go = \case
        BlockComponent blk b'
GetHash          -> b' -> m b'
forall (m :: * -> *) a. Monad m => a -> m a
return b'
HeaderHash blk
headerHash
        BlockComponent blk b'
GetSlot          -> SlotNo -> m SlotNo
forall (m :: * -> *) a. Monad m => a -> m a
return SlotNo
slotNo
        BlockComponent blk b'
GetIsEBB         -> IsEBB -> m IsEBB
forall (m :: * -> *) a. Monad m => a -> m a
return (IsEBB -> m IsEBB) -> IsEBB -> m IsEBB
forall a b. (a -> b) -> a -> b
$ BlockOrEBB -> IsEBB
isBlockOrEBB BlockOrEBB
blockOrEBB
        BlockComponent blk b'
GetBlockSize     -> SecondaryOffset -> m SecondaryOffset
forall (m :: * -> *) a. Monad m => a -> m a
return SecondaryOffset
blockSize
        BlockComponent blk b'
GetHeaderSize    -> b' -> m b'
forall (m :: * -> *) a. Monad m => a -> m a
return (b' -> m b') -> b' -> m b'
forall a b. (a -> b) -> a -> b
$ Word16 -> b'
forall a b. (Integral a, Num b) => a -> b
fromIntegral (Word16 -> b') -> Word16 -> b'
forall a b. (a -> b) -> a -> b
$ HeaderSize -> Word16
Secondary.unHeaderSize HeaderSize
headerSize
        BlockComponent blk b'
GetRawBlock      -> m b'
m ByteString
readBlock
        BlockComponent blk b'
GetRawHeader     -> m b'
m ByteString
readHeader
        BlockComponent blk b'
GetNestedCtxt    -> m b'
m (SomeSecond (NestedCtxt Header) blk)
readNestedCtxt
        BlockComponent blk b'
GetBlock         -> do ByteString
rawBlk <- BlockComponent blk ByteString -> m ByteString
forall b'. BlockComponent blk b' -> m b'
go BlockComponent blk ByteString
forall blk. BlockComponent blk ByteString
GetRawBlock
                               ByteString -> m blk
parseBlock ByteString
rawBlk
        BlockComponent blk b'
GetHeader        -> do ByteString
rawHdr <- BlockComponent blk ByteString -> m ByteString
forall b'. BlockComponent blk b' -> m b'
go BlockComponent blk ByteString
forall blk. BlockComponent blk ByteString
GetRawHeader
                               SomeSecond (NestedCtxt Header) blk
ctxt   <- m (SomeSecond (NestedCtxt Header) blk)
readNestedCtxt
                               SomeSecond (NestedCtxt Header) blk -> ByteString -> m (Header blk)
parseHeader SomeSecond (NestedCtxt Header) blk
ctxt ByteString
rawHdr
        BlockComponent blk b'
GetVerifiedBlock -> do blk
blk <- BlockComponent blk blk -> m blk
forall b'. BlockComponent blk b' -> m b'
go BlockComponent blk blk
forall blk. BlockComponent blk blk
GetBlock
                               Bool -> m () -> m ()
forall (f :: * -> *). Applicative f => Bool -> f () -> f ()
unless (blk -> Bool
checkIntegrity blk
blk) (m () -> m ()) -> m () -> m ()
forall a b. (a -> b) -> a -> b
$
                                 UnexpectedFailure blk -> m ()
forall blk (m :: * -> *) a.
(StandardHash blk, Typeable blk, MonadThrow m) =>
UnexpectedFailure blk -> m a
throwUnexpectedFailure (UnexpectedFailure blk -> m ()) -> UnexpectedFailure blk -> m ()
forall a b. (a -> b) -> a -> b
$ RealPoint blk -> UnexpectedFailure blk
forall blk. RealPoint blk -> UnexpectedFailure blk
CorruptBlockError RealPoint blk
pt
                               blk -> m blk
forall (m :: * -> *) a. Monad m => a -> m a
return blk
blk
        GetPure b'
a        -> b' -> m b'
forall (m :: * -> *) a. Monad m => a -> m a
return b'
a
        GetApply BlockComponent blk (a -> b')
f BlockComponent blk a
bc    -> BlockComponent blk (a -> b') -> m (a -> b')
forall b'. BlockComponent blk b' -> m b'
go BlockComponent blk (a -> b')
f m (a -> b') -> m a -> m b'
forall (f :: * -> *) a b. Applicative f => f (a -> b) -> f a -> f b
<*> BlockComponent blk a -> m a
forall b'. BlockComponent blk b' -> m b'
go BlockComponent blk a
bc

    Secondary.Entry {
          BlockOffset
blockOffset :: forall blk. Entry blk -> BlockOffset
blockOffset :: BlockOffset
blockOffset
        , CRC
checksum :: forall blk. Entry blk -> CRC
checksum :: CRC
checksum
        , HeaderHash blk
headerHash :: HeaderHash blk
headerHash :: forall blk. Entry blk -> HeaderHash blk
headerHash
        , HeaderSize
headerSize :: forall blk. Entry blk -> HeaderSize
headerSize :: HeaderSize
headerSize
        , HeaderOffset
headerOffset :: forall blk. Entry blk -> HeaderOffset
headerOffset :: HeaderOffset
headerOffset
        , BlockOrEBB
blockOrEBB :: BlockOrEBB
blockOrEBB :: forall blk. Entry blk -> BlockOrEBB
blockOrEBB
        } = Entry blk
entry

    slotNo :: SlotNo
    slotNo :: SlotNo
slotNo = ChunkInfo -> BlockOrEBB -> SlotNo
slotNoOfBlockOrEBB ChunkInfo
chunkInfo BlockOrEBB
blockOrEBB

    pt :: RealPoint blk
    pt :: RealPoint blk
pt = SlotNo -> HeaderHash blk -> RealPoint blk
forall blk. SlotNo -> HeaderHash blk -> RealPoint blk
RealPoint SlotNo
slotNo HeaderHash blk
headerHash

    -- | We don't rely on the position of the handle, we always use
    -- 'hGetExactlyAtCRC', i.e. @pread@ for reading from a given offset.
    --
    -- In case the requested chunk is the current chunk, we will be reading
    -- from the chunk file while we're also writing to it. Are we guaranteed
    -- to read what have written? Duncan says: this is guaranteed at the OS
    -- level (POSIX), but not for Haskell handles, which might perform other
    -- buffering. However, the 'HasFS' implementation we're using uses POSIX
    -- file handles ("Ouroboros.Consensus.Storage.IO") so we're safe (other
    -- implementations of the 'HasFS' API guarantee this too).
    readBlock :: m Lazy.ByteString
    readBlock :: m ByteString
readBlock = do
        (ByteString
bl, CRC
checksum') <- HasFS m h -> Handle h -> Word64 -> AbsOffset -> m (ByteString, CRC)
forall (m :: * -> *) h.
(HasCallStack, MonadThrow m) =>
HasFS m h -> Handle h -> Word64 -> AbsOffset -> m (ByteString, CRC)
hGetExactlyAtCRC HasFS m h
hasFS Handle h
eHnd Word64
size AbsOffset
offset
        FsPath -> RealPoint blk -> CRC -> CRC -> m ()
forall blk (m :: * -> *).
(HasCallStack, HasHeader blk, MonadThrow m) =>
FsPath -> RealPoint blk -> CRC -> CRC -> m ()
checkChecksum FsPath
chunkFile RealPoint blk
pt CRC
checksum CRC
checksum'
        ByteString -> m ByteString
forall (m :: * -> *) a. Monad m => a -> m a
return ByteString
bl
      where
        size :: Word64
size      = SecondaryOffset -> Word64
forall a b. (Integral a, Num b) => a -> b
fromIntegral SecondaryOffset
blockSize
        offset :: AbsOffset
offset    = Word64 -> AbsOffset
AbsOffset (Word64 -> AbsOffset) -> Word64 -> AbsOffset
forall a b. (a -> b) -> a -> b
$ BlockOffset -> Word64
Secondary.unBlockOffset BlockOffset
blockOffset
        chunkFile :: FsPath
chunkFile = ChunkNo -> FsPath
fsPathChunkFile ChunkNo
chunk

    -- | We don't rely on the position of the handle, we always use
    -- 'hGetExactlyAt', i.e. @pread@ for reading from a given offset.
    readHeader :: m Lazy.ByteString
    readHeader :: m ByteString
readHeader =
        -- We cannot check the checksum in this case, as we're not reading the
        -- whole block
        HasFS m h -> Handle h -> Word64 -> AbsOffset -> m ByteString
forall (m :: * -> *) h.
(HasCallStack, MonadThrow m) =>
HasFS m h -> Handle h -> Word64 -> AbsOffset -> m ByteString
hGetExactlyAt HasFS m h
hasFS Handle h
eHnd Word64
size AbsOffset
offset
      where
        size :: Word64
size   = Word16 -> Word64
forall a b. (Integral a, Num b) => a -> b
fromIntegral (Word16 -> Word64) -> Word16 -> Word64
forall a b. (a -> b) -> a -> b
$ HeaderSize -> Word16
Secondary.unHeaderSize HeaderSize
headerSize
        offset :: AbsOffset
offset = Word64 -> AbsOffset
AbsOffset (Word64 -> AbsOffset) -> Word64 -> AbsOffset
forall a b. (a -> b) -> a -> b
$
          (BlockOffset -> Word64
Secondary.unBlockOffset BlockOffset
blockOffset) Word64 -> Word64 -> Word64
forall a. Num a => a -> a -> a
+
          Word16 -> Word64
forall a b. (Integral a, Num b) => a -> b
fromIntegral (HeaderOffset -> Word16
Secondary.unHeaderOffset HeaderOffset
headerOffset)

    readNestedCtxt :: m (SomeSecond (NestedCtxt Header) blk)
    readNestedCtxt :: m (SomeSecond (NestedCtxt Header) blk)
readNestedCtxt = do
        ShortByteString
bytes <- ByteString -> ShortByteString
Short.toShort (ByteString -> ShortByteString)
-> (ByteString -> ByteString) -> ByteString -> ShortByteString
forall b c a. (b -> c) -> (a -> b) -> a -> c
. ByteString -> ByteString
Lazy.toStrict (ByteString -> ShortByteString)
-> m ByteString -> m ShortByteString
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$>
                   HasFS m h -> Handle h -> Word64 -> AbsOffset -> m ByteString
forall (m :: * -> *) h.
(HasCallStack, MonadThrow m) =>
HasFS m h -> Handle h -> Word64 -> AbsOffset -> m ByteString
hGetExactlyAt HasFS m h
hasFS Handle h
eHnd Word64
size AbsOffset
offset
        SomeSecond (NestedCtxt Header) blk
-> m (SomeSecond (NestedCtxt Header) blk)
forall (m :: * -> *) a. Monad m => a -> m a
return (SomeSecond (NestedCtxt Header) blk
 -> m (SomeSecond (NestedCtxt Header) blk))
-> SomeSecond (NestedCtxt Header) blk
-> m (SomeSecond (NestedCtxt Header) blk)
forall a b. (a -> b) -> a -> b
$ Proxy (Header blk)
-> ShortByteString
-> SecondaryOffset
-> SomeSecond (NestedCtxt Header) blk
forall (f :: * -> *) blk (proxy :: * -> *).
ReconstructNestedCtxt f blk =>
proxy (f blk)
-> ShortByteString
-> SecondaryOffset
-> SomeSecond (NestedCtxt f) blk
reconstructNestedCtxt Proxy (Header blk)
p ShortByteString
bytes SecondaryOffset
blockSize
      where
        p :: Proxy (Header blk)
        p :: Proxy (Header blk)
p = Proxy (Header blk)
forall k (t :: k). Proxy t
Proxy

        size :: Word64
size   = Word8 -> Word64
forall a b. (Integral a, Num b) => a -> b
fromIntegral (PrefixLen -> Word8
getPrefixLen (Proxy (Header blk) -> PrefixLen
forall (f :: * -> *) blk (proxy :: * -> *).
ReconstructNestedCtxt f blk =>
proxy (f blk) -> PrefixLen
reconstructPrefixLen Proxy (Header blk)
p))
        offset :: AbsOffset
offset = Word64 -> AbsOffset
AbsOffset (Word64 -> AbsOffset) -> Word64 -> AbsOffset
forall a b. (a -> b) -> a -> b
$ BlockOffset -> Word64
Secondary.unBlockOffset BlockOffset
blockOffset

    parseBlock :: Lazy.ByteString -> m blk
    parseBlock :: ByteString -> m blk
parseBlock ByteString
bytes = ByteString
-> Either DeserialiseFailure (ByteString, ByteString -> blk)
-> m blk
forall b'.
ByteString
-> Either DeserialiseFailure (ByteString, ByteString -> b') -> m b'
throwParseErrors ByteString
bytes (Either DeserialiseFailure (ByteString, ByteString -> blk)
 -> m blk)
-> Either DeserialiseFailure (ByteString, ByteString -> blk)
-> m blk
forall a b. (a -> b) -> a -> b
$
        (forall s. Decoder s (ByteString -> blk))
-> ByteString
-> Either DeserialiseFailure (ByteString, ByteString -> blk)
forall a.
(forall s. Decoder s a)
-> ByteString -> Either DeserialiseFailure (ByteString, a)
CBOR.deserialiseFromBytes (CodecConfig blk -> forall s. Decoder s (ByteString -> blk)
forall blk a.
DecodeDisk blk a =>
CodecConfig blk -> forall s. Decoder s a
decodeDisk CodecConfig blk
ccfg) ByteString
bytes

    parseHeader ::
         SomeSecond (NestedCtxt Header) blk
      -> Lazy.ByteString
      -> m (Header blk)
    parseHeader :: SomeSecond (NestedCtxt Header) blk -> ByteString -> m (Header blk)
parseHeader (SomeSecond NestedCtxt Header blk b
ctxt) ByteString
bytes = ByteString
-> Either DeserialiseFailure (ByteString, ByteString -> Header blk)
-> m (Header blk)
forall b'.
ByteString
-> Either DeserialiseFailure (ByteString, ByteString -> b') -> m b'
throwParseErrors ByteString
bytes (Either DeserialiseFailure (ByteString, ByteString -> Header blk)
 -> m (Header blk))
-> Either DeserialiseFailure (ByteString, ByteString -> Header blk)
-> m (Header blk)
forall a b. (a -> b) -> a -> b
$
        (forall s. Decoder s (ByteString -> Header blk))
-> ByteString
-> Either DeserialiseFailure (ByteString, ByteString -> Header blk)
forall a.
(forall s. Decoder s a)
-> ByteString -> Either DeserialiseFailure (ByteString, a)
CBOR.deserialiseFromBytes
          ((\ByteString -> b
f -> DepPair (NestedCtxt Header blk) -> Header blk
forall (f :: * -> *) blk.
HasNestedContent f blk =>
DepPair (NestedCtxt f blk) -> f blk
nest (DepPair (NestedCtxt Header blk) -> Header blk)
-> (ByteString -> DepPair (NestedCtxt Header blk))
-> ByteString
-> Header blk
forall b c a. (b -> c) -> (a -> b) -> a -> c
. NestedCtxt Header blk b -> b -> DepPair (NestedCtxt Header blk)
forall (f :: * -> *) a. f a -> a -> DepPair f
DepPair NestedCtxt Header blk b
ctxt (b -> DepPair (NestedCtxt Header blk))
-> (ByteString -> b)
-> ByteString
-> DepPair (NestedCtxt Header blk)
forall b c a. (b -> c) -> (a -> b) -> a -> c
. ByteString -> b
f) ((ByteString -> b) -> ByteString -> Header blk)
-> Decoder s (ByteString -> b)
-> Decoder s (ByteString -> Header blk)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> CodecConfig blk
-> NestedCtxt Header blk b -> forall s. Decoder s (ByteString -> b)
forall (f :: * -> * -> *) blk a.
DecodeDiskDep f blk =>
CodecConfig blk -> f blk a -> forall s. Decoder s (ByteString -> a)
decodeDiskDep CodecConfig blk
ccfg NestedCtxt Header blk b
ctxt)
          ByteString
bytes

    throwParseErrors ::
         forall b'.
         Lazy.ByteString
      -> Either CBOR.DeserialiseFailure (Lazy.ByteString, Lazy.ByteString -> b')
      -> m b'
    throwParseErrors :: ByteString
-> Either DeserialiseFailure (ByteString, ByteString -> b') -> m b'
throwParseErrors ByteString
fullBytes = \case
        Right (ByteString
trailing, ByteString -> b'
f)
          | ByteString -> Bool
Lazy.null ByteString
trailing
          -> b' -> m b'
forall (m :: * -> *) a. Monad m => a -> m a
return (b' -> m b') -> b' -> m b'
forall a b. (a -> b) -> a -> b
$ ByteString -> b'
f ByteString
fullBytes
          | Bool
otherwise
          -> UnexpectedFailure blk -> m b'
forall blk (m :: * -> *) a.
(StandardHash blk, Typeable blk, MonadThrow m) =>
UnexpectedFailure blk -> m a
throwUnexpectedFailure (UnexpectedFailure blk -> m b') -> UnexpectedFailure blk -> m b'
forall a b. (a -> b) -> a -> b
$
               FsPath -> RealPoint blk -> ByteString -> UnexpectedFailure blk
forall blk.
FsPath -> RealPoint blk -> ByteString -> UnexpectedFailure blk
TrailingDataError (ChunkNo -> FsPath
fsPathChunkFile ChunkNo
chunk) RealPoint blk
pt ByteString
trailing
        Left DeserialiseFailure
err
          -> UnexpectedFailure blk -> m b'
forall blk (m :: * -> *) a.
(StandardHash blk, Typeable blk, MonadThrow m) =>
UnexpectedFailure blk -> m a
throwUnexpectedFailure (UnexpectedFailure blk -> m b') -> UnexpectedFailure blk -> m b'
forall a b. (a -> b) -> a -> b
$
               FsPath
-> RealPoint blk -> DeserialiseFailure -> UnexpectedFailure blk
forall blk.
FsPath
-> RealPoint blk -> DeserialiseFailure -> UnexpectedFailure blk
ParseError (ChunkNo -> FsPath
fsPathChunkFile ChunkNo
chunk) RealPoint blk
pt DeserialiseFailure
err