{-# 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