{-# 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 Cardano.Prelude (forceElemsToWHNF)
import qualified Codec.CBOR.Read as CBOR
import Control.Monad (unless, void, when)
import Control.Monad.Except (ExceptT, runExceptT, throwError)
import Control.Monad.Trans.Class (lift)
import Control.ResourceRegistry
( ResourceKey
, ResourceRegistry
, allocate
, release
, unsafeRelease
)
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 GHC.Stack (HasCallStack)
import Ouroboros.Consensus.Block hiding (headerHash)
import Ouroboros.Consensus.Storage.Common
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.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
import Ouroboros.Consensus.Storage.Serialisation
import Ouroboros.Consensus.Util.IOLike
import Ouroboros.Network.SizeInBytes
import System.FS.API.Lazy
import System.FS.CRC
data IteratorHandle m blk h = IteratorHandle
{ forall (m :: * -> *) blk h. IteratorHandle m blk h -> HasFS m h
ithHasFS :: !(HasFS m h)
, forall (m :: * -> *) blk h. IteratorHandle m blk h -> Index m blk h
ithIndex :: !(Index m blk h)
, forall (m :: * -> *) blk h.
IteratorHandle m blk h
-> StrictTVar m (IteratorStateOrExhausted m blk h)
ithVarState :: !(StrictTVar m (IteratorStateOrExhausted m blk h))
, forall (m :: * -> *) blk h. IteratorHandle m blk h -> ChunkNo
ithEndChunk :: !ChunkNo
, forall (m :: * -> *) blk h.
IteratorHandle m blk h -> HeaderHash blk
ithEndHash :: !(HeaderHash blk)
}
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
$cfrom :: forall (m :: * -> *) hash h x.
IteratorStateOrExhausted m hash h
-> Rep (IteratorStateOrExhausted m hash h) x
from :: forall 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
to :: forall x.
Rep (IteratorStateOrExhausted m hash h) x
-> IteratorStateOrExhausted m hash h
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
$cnoThunks :: 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)
$cwNoThunks :: forall (m :: * -> *) hash h.
(StandardHash hash, IOLike m) =>
Context
-> IteratorStateOrExhausted m hash h -> IO (Maybe ThunkInfo)
wNoThunks :: Context
-> IteratorStateOrExhausted m hash h -> IO (Maybe ThunkInfo)
$cshowTypeOf :: forall (m :: * -> *) hash h.
(StandardHash hash, IOLike m) =>
Proxy (IteratorStateOrExhausted m hash h) -> String
showTypeOf :: Proxy (IteratorStateOrExhausted m hash h) -> String
NoThunks)
data IteratorState m blk h = IteratorState
{ forall (m :: * -> *) blk h. IteratorState m blk h -> ChunkNo
itsChunk :: !ChunkNo
, forall (m :: * -> *) blk h. IteratorState m blk h -> Handle h
itsChunkHandle :: !(Handle h)
, forall (m :: * -> *) blk h. IteratorState m blk h -> ResourceKey m
itsChunkKey :: !(ResourceKey m)
, forall (m :: * -> *) blk h.
IteratorState m blk h -> NonEmpty (WithBlockSize (Entry blk))
itsChunkEntries :: !(NonEmpty (WithBlockSize (Secondary.Entry blk)))
}
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
$cfrom :: forall (m :: * -> *) blk h x.
IteratorState m blk h -> Rep (IteratorState m blk h) x
from :: forall 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
to :: forall x. Rep (IteratorState m blk h) x -> IteratorState m blk h
Generic
deriving instance (StandardHash blk, IOLike m) => NoThunks (IteratorState m blk h)
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 :: 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 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
currentChunk :: forall (m :: * -> *) blk h. OpenState m blk h -> ChunkNo
currentChunkOffset :: forall (m :: * -> *) blk h. OpenState m blk h -> BlockOffset
currentChunk :: ChunkNo
currentChunkOffset :: BlockOffset
currentSecondaryOffset :: SecondaryOffset
currentChunkHandle :: Handle h
currentPrimaryHandle :: Handle h
currentSecondaryHandle :: Handle h
currentTip :: WithOrigin (Tip blk)
currentIndex :: 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
..} -> 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 (m :: * -> *) a.
Monad m =>
m a -> ExceptT (MissingBlock blk) m a
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
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
(secondaryOffset, startChunkSlot) <-
checkLowerBound currentIndex currentTip from
lift $ do
when (startChunkSlot > endChunkSlot) $
throwApiMisuse $
InvalidIteratorRangeError from to
let ChunkSlot startChunk startRelSlot = startChunkSlot
startIsEBB = RelativeSlot -> IsEBB
relativeSlotIsEBB RelativeSlot
startRelSlot
currentChunkInfo = ChunkNo -> BlockOffset -> CurrentChunkInfo
CurrentChunkInfo ChunkNo
currentChunk BlockOffset
currentChunkOffset
endHash = case StreamTo blk
to of
StreamToInclusive (RealPoint SlotNo
_slot HeaderHash blk
hash) -> HeaderHash blk
hash
iteratorState <-
iteratorStateForChunk
hasFS
currentIndex
registry
currentChunkInfo
endHash
startChunk
secondaryOffset
startIsEBB
varIteratorState <- newTVarIO $ IteratorStateOpen iteratorState
let ith =
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
}
case 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 a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return ()
return $ mkIterator ith
where
ImmutableDBEnv{ChunkInfo
chunkInfo :: ChunkInfo
chunkInfo :: forall (m :: * -> *) blk. ImmutableDBEnv m blk -> ChunkInfo
chunkInfo} = ImmutableDBEnv m blk
dbEnv
checkUpperBound ::
HasCallStack =>
Index m blk h ->
WithOrigin (Tip blk) ->
StreamTo blk ->
ExceptT (MissingBlock blk) m ChunkSlot
checkUpperBound :: forall h.
HasCallStack =>
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, _, _) <- 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
return chunkSlot
checkLowerBound ::
HasCallStack =>
Index m blk h ->
WithOrigin (Tip blk) ->
StreamFrom blk ->
ExceptT (MissingBlock blk) m (SecondaryOffset, ChunkSlot)
checkLowerBound :: forall h.
HasCallStack =>
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, _, 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
return (secondaryOffset, 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 (m :: * -> *) a.
Monad m =>
m a -> ExceptT (MissingBlock blk) m a
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, _, 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'
return (secondaryOffset, chunkSlot)
mkIterator :: IteratorHandle m blk h -> Iterator m blk b
mkIterator :: forall h. IteratorHandle m blk h -> Iterator m blk b
mkIterator IteratorHandle m blk h
ith =
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
}
findFirstBlock ::
HasCallStack =>
Index m blk h ->
m (SecondaryOffset, ChunkSlot)
findFirstBlock :: forall h.
HasCallStack =>
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 -> HasCallStack => 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 a b. m a -> (a -> m b) -> m b
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 a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (SecondaryOffset
0, ChunkNo -> RelativeSlot -> ChunkSlot
chunkSlotForRelativeSlot ChunkNo
chunk RelativeSlot
relSlot)
getSlotInfo ::
forall m blk h.
(HasCallStack, IOLike m, HasHeader blk) =>
ChunkInfo ->
Index m blk h ->
WithOrigin (Tip blk) ->
RealPoint blk ->
ExceptT
(MissingBlock blk)
m
( ChunkSlot
, (Secondary.Entry blk, BlockSize)
, SecondaryOffset
)
getSlotInfo :: 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 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 :: SlotNo
tipSlotNo :: forall blk. Tip blk -> SlotNo
tipSlotNo})
| SlotNo
slot SlotNo -> SlotNo -> Bool
forall a. Ord a => a -> a -> Bool
<= SlotNo
tipSlotNo ->
() -> ExceptT (MissingBlock blk) m ()
forall a. a -> ExceptT (MissingBlock blk) m a
forall (m :: * -> *) a. Monad m => a -> m a
return ()
WithOrigin (Tip blk)
_otherwise ->
MissingBlock blk -> ExceptT (MissingBlock blk) m ()
forall a. MissingBlock blk -> ExceptT (MissingBlock blk) m a
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)
toRead :: NonEmpty (IsEBB, SecondaryOffset) <- case Maybe ChunkSlot
mIfBoundary of
Just ChunkSlot
ifBoundary -> do
let relatives :: Two RelativeSlot
relatives@(Two RelativeSlot
relb RelativeSlot
relr) = 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
(offsets, s) <- m (Two (Maybe SecondaryOffset), Maybe (StrictSeq SecondaryOffset))
-> ExceptT
(MissingBlock blk)
m
(Two (Maybe SecondaryOffset), Maybe (StrictSeq SecondaryOffset))
forall (m :: * -> *) a.
Monad m =>
m a -> ExceptT (MissingBlock blk) m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (m (Two (Maybe SecondaryOffset), Maybe (StrictSeq SecondaryOffset))
-> ExceptT
(MissingBlock blk)
m
(Two (Maybe SecondaryOffset), Maybe (StrictSeq SecondaryOffset)))
-> m (Two (Maybe SecondaryOffset),
Maybe (StrictSeq SecondaryOffset))
-> ExceptT
(MissingBlock blk)
m
(Two (Maybe SecondaryOffset), Maybe (StrictSeq SecondaryOffset))
forall a b. (a -> b) -> a -> b
$ Index m blk h
-> forall (t :: * -> *).
(HasCallStack, Traversable t) =>
ChunkNo
-> t RelativeSlot
-> m (t (Maybe SecondaryOffset), Maybe (StrictSeq SecondaryOffset))
forall (m :: * -> *) blk h.
Index m blk h
-> forall (t :: * -> *).
(HasCallStack, Traversable t) =>
ChunkNo
-> t RelativeSlot
-> m (t (Maybe SecondaryOffset), Maybe (StrictSeq SecondaryOffset))
Index.readOffsets Index m blk h
index ChunkNo
chunk Two RelativeSlot
relatives
case offsets of
Two Maybe SecondaryOffset
Nothing Maybe SecondaryOffset
Nothing ->
MissingBlock blk
-> ExceptT (MissingBlock blk) m (NonEmpty (IsEBB, SecondaryOffset))
forall a. MissingBlock blk -> ExceptT (MissingBlock blk) m a
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
-> ChunkNo
-> [RelativeSlot]
-> Maybe (StrictSeq SecondaryOffset)
-> MissingBlock blk
forall blk.
RealPoint blk
-> ChunkNo
-> [RelativeSlot]
-> Maybe (StrictSeq SecondaryOffset)
-> MissingBlock blk
EmptySlot RealPoint blk
pt ChunkNo
chunk [RelativeSlot
relb, RelativeSlot
relr] Maybe (StrictSeq SecondaryOffset)
s
Two (Just SecondaryOffset
ebbOffset) (Just SecondaryOffset
blkOffset) ->
NonEmpty (IsEBB, SecondaryOffset)
-> ExceptT (MissingBlock blk) m (NonEmpty (IsEBB, SecondaryOffset))
forall a. a -> ExceptT (MissingBlock blk) m a
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 a. a -> ExceptT (MissingBlock blk) m a
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 a. a -> ExceptT (MissingBlock blk) m a
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
let relr :: RelativeSlot
relr = ChunkSlot -> RelativeSlot
chunkRelative ChunkSlot
ifRegular
(offset, s) <- m (Maybe SecondaryOffset, Maybe (StrictSeq SecondaryOffset))
-> ExceptT
(MissingBlock blk)
m
(Maybe SecondaryOffset, Maybe (StrictSeq SecondaryOffset))
forall (m :: * -> *) a.
Monad m =>
m a -> ExceptT (MissingBlock blk) m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (m (Maybe SecondaryOffset, Maybe (StrictSeq SecondaryOffset))
-> ExceptT
(MissingBlock blk)
m
(Maybe SecondaryOffset, Maybe (StrictSeq SecondaryOffset)))
-> m (Maybe SecondaryOffset, Maybe (StrictSeq SecondaryOffset))
-> ExceptT
(MissingBlock blk)
m
(Maybe SecondaryOffset, Maybe (StrictSeq SecondaryOffset))
forall a b. (a -> b) -> a -> b
$ Index m blk h
-> ChunkNo
-> RelativeSlot
-> m (Maybe SecondaryOffset, Maybe (StrictSeq SecondaryOffset))
forall (m :: * -> *) blk h.
Functor m =>
Index m blk h
-> ChunkNo
-> RelativeSlot
-> m (Maybe SecondaryOffset, Maybe (StrictSeq SecondaryOffset))
Index.readOffset Index m blk h
index ChunkNo
chunk RelativeSlot
relr
case offset of
Maybe SecondaryOffset
Nothing ->
MissingBlock blk
-> ExceptT (MissingBlock blk) m (NonEmpty (IsEBB, SecondaryOffset))
forall a. MissingBlock blk -> ExceptT (MissingBlock blk) m a
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
-> ChunkNo
-> [RelativeSlot]
-> Maybe (StrictSeq SecondaryOffset)
-> MissingBlock blk
forall blk.
RealPoint blk
-> ChunkNo
-> [RelativeSlot]
-> Maybe (StrictSeq SecondaryOffset)
-> MissingBlock blk
EmptySlot RealPoint blk
pt ChunkNo
chunk [RelativeSlot
relr] Maybe (StrictSeq SecondaryOffset)
s
Just SecondaryOffset
blkOffset ->
NonEmpty (IsEBB, SecondaryOffset)
-> ExceptT (MissingBlock blk) m (NonEmpty (IsEBB, SecondaryOffset))
forall a. a -> ExceptT (MissingBlock blk) m a
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.:| [])
entriesWithBlockSizes :: NonEmpty (Secondary.Entry blk, BlockSize) <-
lift $ Index.readEntries index chunk toRead
(secondaryOffset, (entry, blockSize)) <-
case find
((== hash) . Secondary.headerHash . fst . snd)
(NE.zip (fmap snd toRead) entriesWithBlockSizes) of
Just (SecondaryOffset, (Entry blk, BlockSize))
found -> (SecondaryOffset, (Entry blk, BlockSize))
-> ExceptT
(MissingBlock blk) m (SecondaryOffset, (Entry blk, BlockSize))
forall a. a -> ExceptT (MissingBlock blk) m a
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 a. MissingBlock blk -> ExceptT (MissingBlock blk) m a
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
let 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
return (chunkSlot, (entry, blockSize), secondaryOffset)
stepIterator ::
forall m blk h.
(HasCallStack, IOLike m, HasHeader blk) =>
ResourceRegistry m ->
CurrentChunkInfo ->
IteratorHandle m blk h ->
m ()
stepIterator :: forall (m :: * -> *) blk h.
(HasCallStack, IOLike m, HasHeader blk) =>
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 :: forall (m :: * -> *) blk h. IteratorHandle m blk h -> HasFS m h
ithHasFS :: HasFS m h
ithHasFS, Index m blk h
ithIndex :: forall (m :: * -> *) blk h. IteratorHandle m blk h -> Index m blk h
ithIndex :: Index m blk h
ithIndex, 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)
ithVarState, ChunkNo
ithEndChunk :: forall (m :: * -> *) blk h. IteratorHandle m blk h -> ChunkNo
ithEndChunk :: ChunkNo
ithEndChunk, HeaderHash blk
ithEndHash :: forall (m :: * -> *) blk h.
IteratorHandle m blk h -> HeaderHash blk
ithEndHash :: HeaderHash blk
ithEndHash} =
STM m (IteratorStateOrExhausted m blk h)
-> m (IteratorStateOrExhausted m blk h)
forall a. HasCallStack => STM m a -> m a
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 a b. m a -> (a -> m b) -> m b
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 :: forall (m :: * -> *) blk h.
IteratorState m blk h -> NonEmpty (WithBlockSize (Entry blk))
itsChunkEntries :: NonEmpty (WithBlockSize (Entry blk))
itsChunkEntries, ResourceKey m
itsChunkKey :: forall (m :: * -> *) blk h. IteratorState m blk h -> ResourceKey m
itsChunkKey :: ResourceKey m
itsChunkKey, ChunkNo
itsChunk :: forall (m :: * -> *) blk h. IteratorState m blk h -> ChunkNo
itsChunk :: 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
Just NonEmpty (WithBlockSize (Entry blk))
itsChunkEntries' ->
STM m () -> m ()
forall a. HasCallStack => STM m a -> m a
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
STM m a -> m a
atomically (STM m () -> m ()) -> STM m () -> m ()
forall a b. (a -> b) -> a -> b
$
StrictTVar m (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 = itsChunkEntries'}
Maybe (NonEmpty (WithBlockSize (Entry blk)))
Nothing -> do
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 :: * -> *).
(MonadMask m, MonadSTM m, MonadThread m, HasCallStack) =>
ResourceKey m -> m (Maybe (Context m))
release ResourceKey m
itsChunkKey
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 a b. m a -> (a -> m b) -> m b
forall (m :: * -> *) a b. Monad m => m a -> (a -> m b) -> m b
>>= \IteratorState m blk h
its' ->
STM m () -> m ()
forall a. HasCallStack => STM m a -> m a
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
STM m a -> m a
atomically (STM m () -> m ()) -> STM m () -> m ()
forall a b. (a -> b) -> a -> b
$ StrictTVar m (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 ->
m (IteratorState m blk h)
openNextChunk :: ChunkNo -> m (IteratorState m blk h)
openNextChunk ChunkNo
chunk =
Index m blk h -> HasCallStack => 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 a b. m a -> (a -> m b) -> m b
forall (m :: * -> *) a b. Monad m => m a -> (a -> m b) -> m b
>>= \case
Maybe RelativeSlot
Nothing -> ChunkNo -> m (IteratorState m blk h)
openNextChunk (ChunkNo -> ChunkNo
nextChunkNo ChunkNo
chunk)
Just RelativeSlot
relSlot -> do
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 :: 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 ith :: IteratorHandle m blk h
ith@IteratorHandle{HasFS m h
ithHasFS :: forall (m :: * -> *) blk h. IteratorHandle m blk h -> HasFS m h
ithHasFS :: HasFS m h
ithHasFS, 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)
ithVarState} ResourceRegistry m
registry BlockComponent blk b
blockComponent = do
STM m (IteratorStateOrExhausted m blk h)
-> m (IteratorStateOrExhausted m blk h)
forall a. HasCallStack => STM m a -> m a
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 a b. m a -> (a -> m b) -> m b
forall (m :: * -> *) a b. Monad m => m a -> (a -> m b) -> m b
>>= \case
IteratorStateOrExhausted m blk h
IteratorStateExhausted -> IteratorResult b -> m (IteratorResult b)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return IteratorResult b
forall b. IteratorResult b
IteratorExhausted
IteratorStateOpen IteratorState{NonEmpty (WithBlockSize (Entry blk))
itsChunkEntries :: forall (m :: * -> *) blk h.
IteratorState m blk h -> NonEmpty (WithBlockSize (Entry blk))
itsChunkEntries :: NonEmpty (WithBlockSize (Entry blk))
itsChunkEntries, ChunkNo
itsChunk :: forall (m :: * -> *) blk h. IteratorState m blk h -> ChunkNo
itsChunk :: ChunkNo
itsChunk, Handle h
itsChunkHandle :: forall (m :: * -> *) blk h. IteratorState m blk h -> Handle h
itsChunkHandle :: 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 <-
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
stepIterator registry currentChunkInfo ith
return $ IteratorResult b
where
ImmutableDBEnv{CodecConfig blk
codecConfig :: CodecConfig blk
codecConfig :: forall (m :: * -> *) blk. ImmutableDBEnv m blk -> CodecConfig blk
codecConfig, ChunkInfo
chunkInfo :: forall (m :: * -> *) blk. ImmutableDBEnv m blk -> ChunkInfo
chunkInfo :: ChunkInfo
chunkInfo, blk -> Bool
checkIntegrity :: blk -> Bool
checkIntegrity :: forall (m :: * -> *) blk. ImmutableDBEnv m blk -> blk -> Bool
checkIntegrity} = ImmutableDBEnv m blk
dbEnv
iteratorHasNextImpl ::
IOLike m =>
ImmutableDBEnv m blk ->
IteratorHandle m blk h ->
STM m (Maybe (RealPoint blk))
iteratorHasNextImpl :: forall (m :: * -> *) blk h.
IOLike m =>
ImmutableDBEnv m blk
-> IteratorHandle m blk h -> STM m (Maybe (RealPoint blk))
iteratorHasNextImpl ImmutableDBEnv{ChunkInfo
chunkInfo :: forall (m :: * -> *) blk. ImmutableDBEnv m blk -> ChunkInfo
chunkInfo :: ChunkInfo
chunkInfo} IteratorHandle{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)
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 :: forall (m :: * -> *) blk h.
IteratorState m blk h -> NonEmpty (WithBlockSize (Entry blk))
itsChunkEntries :: 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 :: forall (m :: * -> *) blk h.
(HasCallStack, IOLike m) =>
IteratorHandle m blk h -> m ()
iteratorCloseImpl IteratorHandle{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)
ithVarState} = do
STM m (IteratorStateOrExhausted m blk h)
-> m (IteratorStateOrExhausted m blk h)
forall a. HasCallStack => STM m a -> m a
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 a b. m a -> (a -> m b) -> m b
forall (m :: * -> *) a b. Monad m => m a -> (a -> m b) -> m b
>>= \case
IteratorStateOrExhausted m blk h
IteratorStateExhausted -> () -> m ()
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return ()
IteratorStateOpen IteratorState{ResourceKey m
itsChunkKey :: forall (m :: * -> *) blk h. IteratorState m blk h -> ResourceKey m
itsChunkKey :: ResourceKey m
itsChunkKey} -> do
STM m () -> m ()
forall a. HasCallStack => STM m a -> m a
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
STM m a -> m a
atomically (STM m () -> m ()) -> STM m () -> m ()
forall a b. (a -> b) -> a -> b
$ StrictTVar m (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
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 :: * -> *).
(MonadMask m, MonadSTM 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 ->
ChunkNo ->
SecondaryOffset ->
IsEBB ->
m (IteratorState m blk h)
iteratorStateForChunk :: 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
index
ResourceRegistry m
registry
(CurrentChunkInfo ChunkNo
curChunk BlockOffset
curChunkOffset)
HeaderHash blk
endHash
ChunkNo
chunk
SecondaryOffset
secondaryOffset
IsEBB
firstIsEBB = do
(key, eHnd) <-
ResourceRegistry m
-> (ResourceId -> m (Handle h))
-> (Handle h -> m ())
-> m (ResourceKey m, Handle h)
forall (m :: * -> *) a.
(MonadSTM m, MonadMask m, MonadThread 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
chunkFileSize <-
if chunk == curChunk
then return (unBlockOffset curChunkOffset)
else hGetSize eHnd
entries <-
Index.readAllEntries
index
secondaryOffset
chunk
((== endHash) . Secondary.headerHash)
chunkFileSize
firstIsEBB
case NE.nonEmpty entries of
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 a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return
IteratorState
{ itsChunk :: ChunkNo
itsChunk = ChunkNo
chunk
, itsChunkHandle :: Handle h
itsChunkHandle = Handle h
eHnd
, itsChunkKey :: ResourceKey m
itsChunkKey = ResourceKey m
key
,
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 :: HasCallStack => FsPath -> OpenMode -> m (Handle h)
hOpen :: forall (m :: * -> *) h.
HasFS m h -> HasCallStack => FsPath -> OpenMode -> m (Handle h)
hOpen, HasCallStack => Handle h -> m ()
hClose :: HasCallStack => Handle h -> m ()
hClose :: forall (m :: * -> *) h.
HasFS m h -> HasCallStack => Handle h -> m ()
hClose, HasCallStack => Handle h -> m Word64
hGetSize :: HasCallStack => Handle h -> m Word64
hGetSize :: forall (m :: * -> *) h.
HasFS m h -> 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
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 :: forall b'. BlockComponent blk b' -> m b'
go = \case
BlockComponent blk b'
GetHash -> b' -> m b'
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return b'
HeaderHash blk
headerHash
BlockComponent blk b'
GetSlot -> b' -> m b'
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return b'
SlotNo
slotNo
BlockComponent blk b'
GetIsEBB -> IsEBB -> m IsEBB
forall a. a -> m a
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 -> SizeInBytes -> m SizeInBytes
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (SizeInBytes -> m SizeInBytes) -> SizeInBytes -> m SizeInBytes
forall a b. (a -> b) -> a -> b
$ SecondaryOffset -> SizeInBytes
SizeInBytes SecondaryOffset
blockSize
BlockComponent blk b'
GetHeaderSize -> b' -> m b'
forall a. a -> m a
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
rawBlk <- BlockComponent blk ByteString -> m ByteString
forall b'. BlockComponent blk b' -> m b'
go BlockComponent blk ByteString
forall blk. BlockComponent blk ByteString
GetRawBlock
parseBlock rawBlk
BlockComponent blk b'
GetHeader -> do
rawHdr <- BlockComponent blk ByteString -> m ByteString
forall b'. BlockComponent blk b' -> m b'
go BlockComponent blk ByteString
forall blk. BlockComponent blk ByteString
GetRawHeader
ctxt <- readNestedCtxt
parseHeader ctxt rawHdr
BlockComponent blk b'
GetVerifiedBlock -> do
blk <- BlockComponent blk blk -> m blk
forall b'. BlockComponent blk b' -> m b'
go BlockComponent blk blk
forall blk. BlockComponent blk blk
GetBlock
unless (checkIntegrity blk) $
throwUnexpectedFailure $
CorruptBlockError pt
return blk
GetPure b'
a -> b' -> m b'
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return b'
a
GetApply BlockComponent blk (a1 -> b')
f BlockComponent blk a1
bc -> BlockComponent blk (a1 -> b') -> m (a1 -> b')
forall b'. BlockComponent blk b' -> m b'
go BlockComponent blk (a1 -> b')
f m (a1 -> b') -> m a1 -> m b'
forall a b. m (a -> b) -> m a -> m b
forall (f :: * -> *) a b. Applicative f => f (a -> b) -> f a -> f b
<*> BlockComponent blk a1 -> m a1
forall b'. BlockComponent blk b' -> m b'
go BlockComponent blk a1
bc
Secondary.Entry
{ BlockOffset
blockOffset :: BlockOffset
blockOffset :: forall blk. Entry blk -> BlockOffset
blockOffset
, CRC
checksum :: CRC
checksum :: forall blk. Entry blk -> CRC
checksum
, HeaderHash blk
headerHash :: forall blk. Entry blk -> HeaderHash blk
headerHash :: HeaderHash blk
headerHash
, HeaderSize
headerSize :: HeaderSize
headerSize :: forall blk. Entry blk -> HeaderSize
headerSize
, HeaderOffset
headerOffset :: HeaderOffset
headerOffset :: forall blk. Entry blk -> HeaderOffset
headerOffset
, BlockOrEBB
blockOrEBB :: forall blk. Entry blk -> BlockOrEBB
blockOrEBB :: 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
readBlock :: m Lazy.ByteString
readBlock :: m ByteString
readBlock = do
(bl, 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
checkChecksum chunkFile pt checksum checksum'
return 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
readHeader :: m Lazy.ByteString
readHeader :: m ByteString
readHeader =
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
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
return $ reconstructNestedCtxt p bytes (SizeInBytes 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 (proxy :: * -> *). 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 a.
CodecConfig blk
-> NestedCtxt Header blk a -> forall s. Decoder s (ByteString -> a)
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 :: forall b'.
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 a. a -> m a
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