{-# LANGUAGE DeriveAnyClass #-}
{-# LANGUAGE DeriveGeneric #-}
{-# LANGUAGE FlexibleContexts #-}
{-# LANGUAGE GADTs #-}
{-# LANGUAGE LambdaCase #-}
{-# LANGUAGE NamedFieldPuns #-}
{-# LANGUAGE ScopedTypeVariables #-}
{-# LANGUAGE StandaloneDeriving #-}
{-# LANGUAGE UndecidableInstances #-}

-- | In-memory model implementation of 'VolatileDB'
module Test.Ouroboros.Storage.VolatileDB.Model (
    DBModel (..)
  , initDBModel
    -- * Basic API
  , closeModel
  , filterByPredecessorModel
  , garbageCollectModel
  , getBlockComponentModel
  , getBlockInfoModel
  , getMaxSlotNoModel
  , isOpenModel
  , putBlockModel
  , reOpenModel
    -- * Corruptions
  , runCorruptionsModel
    -- * Exported for testing purposes
  , BlocksInFile (..)
  , blockHashes
  , blockIndex
  , getCurrentFile
  , getDBFileIds
  , getDBFiles
  ) where

import qualified Codec.CBOR.Write as CBOR
import           Control.Monad.Except (MonadError, throwError)
import qualified Data.ByteString.Lazy as BL
import           Data.Map.Strict (Map)
import qualified Data.Map.Strict as Map
import           Data.Maybe (fromMaybe)
import           Data.Set (Set)
import qualified Data.Set as Set
import           Data.TreeDiff
import           Data.Word (Word64)
import           GHC.Generics (Generic)
import           Ouroboros.Consensus.Block
import           Ouroboros.Consensus.Storage.Common (BlockComponent (..),
                     extractHeader)
import           Ouroboros.Consensus.Storage.Serialisation
                     (BinaryBlockInfo (..), EncodeDisk (..),
                     HasBinaryBlockInfo (..))
import           Ouroboros.Consensus.Storage.VolatileDB
import           Ouroboros.Consensus.Storage.VolatileDB.Impl.Types (FileId,
                     unBlocksPerFile)
import           Ouroboros.Consensus.Storage.VolatileDB.Impl.Util (filePath,
                     parseFd)
import           Ouroboros.Network.Block (MaxSlotNo (..))
import           System.FS.API.Types (FsPath)
import           Test.Ouroboros.Storage.TestBlock (Corruptions,
                     FileCorruption (..))

data DBModel blk = DBModel {
      forall blk. DBModel blk -> BlocksPerFile
blocksPerFile :: BlocksPerFile
      -- ^ How many blocks each file has (should follow the real
      -- implementation).
    , forall blk. DBModel blk -> Bool
open          :: Bool
      -- ^ Indicates if the DB is open.
    , forall blk. DBModel blk -> Map FileId (BlocksInFile blk)
fileIndex     :: Map FileId (BlocksInFile blk)
      -- ^ What each file contains in the real implementation.
      --
      -- INVARIANT: the map is never empty.
      --
      -- INVARIANT: the 'BlocksInFile' associated with the highest 'FileId'
      -- has always fewer than 'blocksPerFile' blocks.
    , forall blk. DBModel blk -> CodecConfig blk
codecConfig   :: CodecConfig blk
    }
  deriving ((forall x. DBModel blk -> Rep (DBModel blk) x)
-> (forall x. Rep (DBModel blk) x -> DBModel blk)
-> Generic (DBModel blk)
forall x. Rep (DBModel blk) x -> DBModel blk
forall x. DBModel blk -> Rep (DBModel blk) x
forall a.
(forall x. a -> Rep a x) -> (forall x. Rep a x -> a) -> Generic a
forall blk x. Rep (DBModel blk) x -> DBModel blk
forall blk x. DBModel blk -> Rep (DBModel blk) x
$cfrom :: forall blk x. DBModel blk -> Rep (DBModel blk) x
from :: forall x. DBModel blk -> Rep (DBModel blk) x
$cto :: forall blk x. Rep (DBModel blk) x -> DBModel blk
to :: forall x. Rep (DBModel blk) x -> DBModel blk
Generic)

deriving instance ( ToExpr blk
                  , ToExpr (CodecConfig blk)
                  ) => ToExpr (DBModel blk)

deriving instance (Show blk, Show (CodecConfig blk)) => Show (DBModel blk)

initDBModel :: BlocksPerFile -> CodecConfig blk -> DBModel blk
initDBModel :: forall blk. BlocksPerFile -> CodecConfig blk -> DBModel blk
initDBModel BlocksPerFile
blocksPerFile CodecConfig blk
codecConfig = DBModel {
      blocksPerFile :: BlocksPerFile
blocksPerFile = BlocksPerFile
blocksPerFile
    , open :: Bool
open          = Bool
True
    , fileIndex :: Map FileId (BlocksInFile blk)
fileIndex     = FileId -> BlocksInFile blk -> Map FileId (BlocksInFile blk)
forall k a. k -> a -> Map k a
Map.singleton FileId
0 BlocksInFile blk
forall blk. BlocksInFile blk
emptyFile
    , codecConfig :: CodecConfig blk
codecConfig   = CodecConfig blk
codecConfig
    }

blockIndex :: HasHeader blk => DBModel blk -> Map (HeaderHash blk) blk
blockIndex :: forall blk.
HasHeader blk =>
DBModel blk -> Map (HeaderHash blk) blk
blockIndex = (BlocksInFile blk -> Map (HeaderHash blk) blk)
-> Map FileId (BlocksInFile blk) -> Map (HeaderHash blk) blk
forall m a. Monoid m => (a -> m) -> Map FileId a -> m
forall (t :: * -> *) m a.
(Foldable t, Monoid m) =>
(a -> m) -> t a -> m
foldMap BlocksInFile blk -> Map (HeaderHash blk) blk
forall blk.
HasHeader blk =>
BlocksInFile blk -> Map (HeaderHash blk) blk
fileToBlockIndex (Map FileId (BlocksInFile blk) -> Map (HeaderHash blk) blk)
-> (DBModel blk -> Map FileId (BlocksInFile blk))
-> DBModel blk
-> Map (HeaderHash blk) blk
forall b c a. (b -> c) -> (a -> b) -> a -> c
. DBModel blk -> Map FileId (BlocksInFile blk)
forall blk. DBModel blk -> Map FileId (BlocksInFile blk)
fileIndex

blockHashes :: HasHeader blk => DBModel blk -> [HeaderHash blk]
blockHashes :: forall blk. HasHeader blk => DBModel blk -> [HeaderHash blk]
blockHashes = (BlocksInFile blk -> [HeaderHash blk])
-> Map FileId (BlocksInFile blk) -> [HeaderHash blk]
forall (t :: * -> *) a b. Foldable t => (a -> [b]) -> t a -> [b]
concatMap BlocksInFile blk -> [HeaderHash blk]
forall blk. HasHeader blk => BlocksInFile blk -> [HeaderHash blk]
fileHashes (Map FileId (BlocksInFile blk) -> [HeaderHash blk])
-> (DBModel blk -> Map FileId (BlocksInFile blk))
-> DBModel blk
-> [HeaderHash blk]
forall b c a. (b -> c) -> (a -> b) -> a -> c
. DBModel blk -> Map FileId (BlocksInFile blk)
forall blk. DBModel blk -> Map FileId (BlocksInFile blk)
fileIndex

getBlockToPredecessor ::
     GetPrevHash blk
  => DBModel blk
  -> Map (HeaderHash blk) (ChainHash blk)
getBlockToPredecessor :: forall blk.
GetPrevHash blk =>
DBModel blk -> Map (HeaderHash blk) (ChainHash blk)
getBlockToPredecessor DBModel { Map FileId (BlocksInFile blk)
fileIndex :: forall blk. DBModel blk -> Map FileId (BlocksInFile blk)
fileIndex :: Map FileId (BlocksInFile blk)
fileIndex } =
    (BlocksInFile blk -> Map (HeaderHash blk) (ChainHash blk))
-> Map FileId (BlocksInFile blk)
-> Map (HeaderHash blk) (ChainHash blk)
forall m a. Monoid m => (a -> m) -> Map FileId a -> m
forall (t :: * -> *) m a.
(Foldable t, Monoid m) =>
(a -> m) -> t a -> m
foldMap BlocksInFile blk -> Map (HeaderHash blk) (ChainHash blk)
forall blk.
GetPrevHash blk =>
BlocksInFile blk -> Map (HeaderHash blk) (ChainHash blk)
fileBlockToPredecessor Map FileId (BlocksInFile blk)
fileIndex

getCurrentFile :: DBModel blk -> FsPath
getCurrentFile :: forall blk. DBModel blk -> FsPath
getCurrentFile = FileId -> FsPath
filePath (FileId -> FsPath)
-> (DBModel blk -> FileId) -> DBModel blk -> FsPath
forall b c a. (b -> c) -> (a -> b) -> a -> c
. DBModel blk -> FileId
forall blk. DBModel blk -> FileId
getCurrentFileId

getCurrentFileId :: DBModel blk -> FileId
getCurrentFileId :: forall blk. DBModel blk -> FileId
getCurrentFileId =
      -- Relies on the first invariant of 'fileIndex'
      FileId
-> (((FileId, BlocksInFile blk), Map FileId (BlocksInFile blk))
    -> FileId)
-> Maybe
     ((FileId, BlocksInFile blk), Map FileId (BlocksInFile blk))
-> FileId
forall b a. b -> (a -> b) -> Maybe a -> b
maybe ([Char] -> FileId
forall a. HasCallStack => [Char] -> a
error [Char]
"empty fileIndex") ((FileId, BlocksInFile blk) -> FileId
forall a b. (a, b) -> a
fst ((FileId, BlocksInFile blk) -> FileId)
-> (((FileId, BlocksInFile blk), Map FileId (BlocksInFile blk))
    -> (FileId, BlocksInFile blk))
-> ((FileId, BlocksInFile blk), Map FileId (BlocksInFile blk))
-> FileId
forall b c a. (b -> c) -> (a -> b) -> a -> c
. ((FileId, BlocksInFile blk), Map FileId (BlocksInFile blk))
-> (FileId, BlocksInFile blk)
forall a b. (a, b) -> a
fst)
    (Maybe ((FileId, BlocksInFile blk), Map FileId (BlocksInFile blk))
 -> FileId)
-> (DBModel blk
    -> Maybe
         ((FileId, BlocksInFile blk), Map FileId (BlocksInFile blk)))
-> DBModel blk
-> FileId
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Map FileId (BlocksInFile blk)
-> Maybe
     ((FileId, BlocksInFile blk), Map FileId (BlocksInFile blk))
forall k a. Map k a -> Maybe ((k, a), Map k a)
Map.maxViewWithKey
    (Map FileId (BlocksInFile blk)
 -> Maybe
      ((FileId, BlocksInFile blk), Map FileId (BlocksInFile blk)))
-> (DBModel blk -> Map FileId (BlocksInFile blk))
-> DBModel blk
-> Maybe
     ((FileId, BlocksInFile blk), Map FileId (BlocksInFile blk))
forall b c a. (b -> c) -> (a -> b) -> a -> c
. DBModel blk -> Map FileId (BlocksInFile blk)
forall blk. DBModel blk -> Map FileId (BlocksInFile blk)
fileIndex

-- | Restore the invariants of 'fileIndex'
restoreInvariants :: DBModel blk -> DBModel blk
restoreInvariants :: forall blk. DBModel blk -> DBModel blk
restoreInvariants DBModel blk
dbm = case ((FileId, BlocksInFile blk), Map FileId (BlocksInFile blk))
-> (FileId, BlocksInFile blk)
forall a b. (a, b) -> a
fst (((FileId, BlocksInFile blk), Map FileId (BlocksInFile blk))
 -> (FileId, BlocksInFile blk))
-> Maybe
     ((FileId, BlocksInFile blk), Map FileId (BlocksInFile blk))
-> Maybe (FileId, BlocksInFile blk)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Map FileId (BlocksInFile blk)
-> Maybe
     ((FileId, BlocksInFile blk), Map FileId (BlocksInFile blk))
forall k a. Map k a -> Maybe ((k, a), Map k a)
Map.maxViewWithKey Map FileId (BlocksInFile blk)
fileIndex of
    Maybe (FileId, BlocksInFile blk)
Nothing
      -> DBModel blk
dbm {
           fileIndex = Map.insert 0 emptyFile fileIndex
         }
    Just (FileId
fileId, BlocksInFile blk
file)
      | let n :: FileId
n = BlocksInFile blk -> FileId
forall blk. BlocksInFile blk -> FileId
nbBlocksInFile BlocksInFile blk
file
      , FileId -> Word32
forall a b. (Integral a, Num b) => a -> b
fromIntegral FileId
n Word32 -> Word32 -> Bool
forall a. Eq a => a -> a -> Bool
== BlocksPerFile -> Word32
unBlocksPerFile BlocksPerFile
blocksPerFile
      , let newFileId :: FileId
newFileId = FileId
fileId FileId -> FileId -> FileId
forall a. Num a => a -> a -> a
+ FileId
1
        -- The file is full, start a new one
      -> DBModel blk
dbm {
           fileIndex = Map.insert newFileId emptyFile fileIndex
         }
      | Bool
otherwise
        -- The file is not full, continue appending to it
      -> DBModel blk
dbm
  where
    DBModel { BlocksPerFile
blocksPerFile :: forall blk. DBModel blk -> BlocksPerFile
blocksPerFile :: BlocksPerFile
blocksPerFile, Map FileId (BlocksInFile blk)
fileIndex :: forall blk. DBModel blk -> Map FileId (BlocksInFile blk)
fileIndex :: Map FileId (BlocksInFile blk)
fileIndex } = DBModel blk
dbm

whenOpen ::
     MonadError (VolatileDBError blk) m
  => DBModel blk
  -> a
  -> m a
whenOpen :: forall blk (m :: * -> *) a.
MonadError (VolatileDBError blk) m =>
DBModel blk -> a -> m a
whenOpen DBModel blk
dbm a
k
    | DBModel blk -> Bool
forall blk. DBModel blk -> Bool
open DBModel blk
dbm
    = a -> m a
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return a
k
    | Bool
otherwise
    = VolatileDBError blk -> m a
forall a. VolatileDBError blk -> m a
forall e (m :: * -> *) a. MonadError e m => e -> m a
throwError (VolatileDBError blk -> m a) -> VolatileDBError blk -> m a
forall a b. (a -> b) -> a -> b
$ ApiMisuse -> VolatileDBError blk
forall blk. ApiMisuse -> VolatileDBError blk
ApiMisuse (ApiMisuse -> VolatileDBError blk)
-> ApiMisuse -> VolatileDBError blk
forall a b. (a -> b) -> a -> b
$ Maybe SomeException -> ApiMisuse
ClosedDBError Maybe SomeException
forall a. Maybe a
Nothing

getDBFileIds :: DBModel blk -> [FileId]
getDBFileIds :: forall blk. DBModel blk -> [FileId]
getDBFileIds = Map FileId (BlocksInFile blk) -> [FileId]
forall k a. Map k a -> [k]
Map.keys (Map FileId (BlocksInFile blk) -> [FileId])
-> (DBModel blk -> Map FileId (BlocksInFile blk))
-> DBModel blk
-> [FileId]
forall b c a. (b -> c) -> (a -> b) -> a -> c
. DBModel blk -> Map FileId (BlocksInFile blk)
forall blk. DBModel blk -> Map FileId (BlocksInFile blk)
fileIndex

getDBFiles :: DBModel blk -> [FsPath]
getDBFiles :: forall blk. DBModel blk -> [FsPath]
getDBFiles = (FileId -> FsPath) -> [FileId] -> [FsPath]
forall a b. (a -> b) -> [a] -> [b]
map FileId -> FsPath
filePath ([FileId] -> [FsPath])
-> (DBModel blk -> [FileId]) -> DBModel blk -> [FsPath]
forall b c a. (b -> c) -> (a -> b) -> a -> c
. DBModel blk -> [FileId]
forall blk. DBModel blk -> [FileId]
getDBFileIds

{------------------------------------------------------------------------------
  BlocksInFile
------------------------------------------------------------------------------}

-- | The blocks in a file, in the same order as they would be written to the
-- file in the real implementation.
newtype BlocksInFile blk = BlocksInFile {
      forall blk. BlocksInFile blk -> [blk]
getBlocksInFile :: [blk]
    }
  deriving (FileId -> BlocksInFile blk -> ShowS
[BlocksInFile blk] -> ShowS
BlocksInFile blk -> [Char]
(FileId -> BlocksInFile blk -> ShowS)
-> (BlocksInFile blk -> [Char])
-> ([BlocksInFile blk] -> ShowS)
-> Show (BlocksInFile blk)
forall blk. Show blk => FileId -> BlocksInFile blk -> ShowS
forall blk. Show blk => [BlocksInFile blk] -> ShowS
forall blk. Show blk => BlocksInFile blk -> [Char]
forall a.
(FileId -> a -> ShowS) -> (a -> [Char]) -> ([a] -> ShowS) -> Show a
$cshowsPrec :: forall blk. Show blk => FileId -> BlocksInFile blk -> ShowS
showsPrec :: FileId -> BlocksInFile blk -> ShowS
$cshow :: forall blk. Show blk => BlocksInFile blk -> [Char]
show :: BlocksInFile blk -> [Char]
$cshowList :: forall blk. Show blk => [BlocksInFile blk] -> ShowS
showList :: [BlocksInFile blk] -> ShowS
Show, (forall x. BlocksInFile blk -> Rep (BlocksInFile blk) x)
-> (forall x. Rep (BlocksInFile blk) x -> BlocksInFile blk)
-> Generic (BlocksInFile blk)
forall x. Rep (BlocksInFile blk) x -> BlocksInFile blk
forall x. BlocksInFile blk -> Rep (BlocksInFile blk) x
forall a.
(forall x. a -> Rep a x) -> (forall x. Rep a x -> a) -> Generic a
forall blk x. Rep (BlocksInFile blk) x -> BlocksInFile blk
forall blk x. BlocksInFile blk -> Rep (BlocksInFile blk) x
$cfrom :: forall blk x. BlocksInFile blk -> Rep (BlocksInFile blk) x
from :: forall x. BlocksInFile blk -> Rep (BlocksInFile blk) x
$cto :: forall blk x. Rep (BlocksInFile blk) x -> BlocksInFile blk
to :: forall x. Rep (BlocksInFile blk) x -> BlocksInFile blk
Generic)

instance ToExpr blk => ToExpr (BlocksInFile blk)

emptyFile :: BlocksInFile blk
emptyFile :: forall blk. BlocksInFile blk
emptyFile = [blk] -> BlocksInFile blk
forall blk. [blk] -> BlocksInFile blk
BlocksInFile []

nbBlocksInFile :: BlocksInFile blk -> Int
nbBlocksInFile :: forall blk. BlocksInFile blk -> FileId
nbBlocksInFile = [blk] -> FileId
forall a. [a] -> FileId
forall (t :: * -> *) a. Foldable t => t a -> FileId
length ([blk] -> FileId)
-> (BlocksInFile blk -> [blk]) -> BlocksInFile blk -> FileId
forall b c a. (b -> c) -> (a -> b) -> a -> c
. BlocksInFile blk -> [blk]
forall blk. BlocksInFile blk -> [blk]
getBlocksInFile

appendBlock :: blk -> BlocksInFile blk -> BlocksInFile blk
appendBlock :: forall blk. blk -> BlocksInFile blk -> BlocksInFile blk
appendBlock blk
blk (BlocksInFile [blk]
blks) =
    [blk] -> BlocksInFile blk
forall blk. [blk] -> BlocksInFile blk
BlocksInFile ([blk]
blks [blk] -> [blk] -> [blk]
forall a. [a] -> [a] -> [a]
++ [blk
blk])

-- | The highest slot number in this file.
fileMaxSlotNo :: HasHeader blk => BlocksInFile blk -> MaxSlotNo
fileMaxSlotNo :: forall blk. HasHeader blk => BlocksInFile blk -> MaxSlotNo
fileMaxSlotNo = (blk -> MaxSlotNo) -> [blk] -> MaxSlotNo
forall m a. Monoid m => (a -> m) -> [a] -> m
forall (t :: * -> *) m a.
(Foldable t, Monoid m) =>
(a -> m) -> t a -> m
foldMap (SlotNo -> MaxSlotNo
MaxSlotNo (SlotNo -> MaxSlotNo) -> (blk -> SlotNo) -> blk -> MaxSlotNo
forall b c a. (b -> c) -> (a -> b) -> a -> c
. blk -> SlotNo
forall b. HasHeader b => b -> SlotNo
blockSlot) ([blk] -> MaxSlotNo)
-> (BlocksInFile blk -> [blk]) -> BlocksInFile blk -> MaxSlotNo
forall b c a. (b -> c) -> (a -> b) -> a -> c
. BlocksInFile blk -> [blk]
forall blk. BlocksInFile blk -> [blk]
getBlocksInFile

fileToBlockIndex ::
     HasHeader blk
  => BlocksInFile blk
  -> Map (HeaderHash blk) blk
fileToBlockIndex :: forall blk.
HasHeader blk =>
BlocksInFile blk -> Map (HeaderHash blk) blk
fileToBlockIndex = [(HeaderHash blk, blk)] -> Map (HeaderHash blk) blk
forall k a. Ord k => [(k, a)] -> Map k a
Map.fromList ([(HeaderHash blk, blk)] -> Map (HeaderHash blk) blk)
-> (BlocksInFile blk -> [(HeaderHash blk, blk)])
-> BlocksInFile blk
-> Map (HeaderHash blk) blk
forall b c a. (b -> c) -> (a -> b) -> a -> c
. (blk -> (HeaderHash blk, blk)) -> [blk] -> [(HeaderHash blk, blk)]
forall a b. (a -> b) -> [a] -> [b]
map blk -> (HeaderHash blk, blk)
forall {b}. HasHeader b => b -> (HeaderHash b, b)
addKey ([blk] -> [(HeaderHash blk, blk)])
-> (BlocksInFile blk -> [blk])
-> BlocksInFile blk
-> [(HeaderHash blk, blk)]
forall b c a. (b -> c) -> (a -> b) -> a -> c
. BlocksInFile blk -> [blk]
forall blk. BlocksInFile blk -> [blk]
getBlocksInFile
  where
    addKey :: b -> (HeaderHash b, b)
addKey b
blk = (b -> HeaderHash b
forall b. HasHeader b => b -> HeaderHash b
blockHash b
blk, b
blk)

fileHashes :: HasHeader blk => BlocksInFile blk -> [HeaderHash blk]
fileHashes :: forall blk. HasHeader blk => BlocksInFile blk -> [HeaderHash blk]
fileHashes = (blk -> HeaderHash blk) -> [blk] -> [HeaderHash blk]
forall a b. (a -> b) -> [a] -> [b]
map blk -> HeaderHash blk
forall b. HasHeader b => b -> HeaderHash b
blockHash ([blk] -> [HeaderHash blk])
-> (BlocksInFile blk -> [blk])
-> BlocksInFile blk
-> [HeaderHash blk]
forall b c a. (b -> c) -> (a -> b) -> a -> c
. BlocksInFile blk -> [blk]
forall blk. BlocksInFile blk -> [blk]
getBlocksInFile

fileBlockToPredecessor ::
    GetPrevHash blk
  => BlocksInFile blk
  -> Map (HeaderHash blk) (ChainHash blk)
fileBlockToPredecessor :: forall blk.
GetPrevHash blk =>
BlocksInFile blk -> Map (HeaderHash blk) (ChainHash blk)
fileBlockToPredecessor (BlocksInFile [blk]
blks) = [(HeaderHash blk, ChainHash blk)]
-> Map (HeaderHash blk) (ChainHash blk)
forall k a. Ord k => [(k, a)] -> Map k a
Map.fromList
    [ (blk -> HeaderHash blk
forall b. HasHeader b => b -> HeaderHash b
blockHash blk
blk, blk -> ChainHash blk
forall blk. GetPrevHash blk => blk -> ChainHash blk
blockPrevHash blk
blk)
    | blk
blk <- [blk]
blks
    ]

fileSize ::
     (Integral a, EncodeDisk blk blk)
  => CodecConfig blk
  -> BlocksInFile blk
  -> a
fileSize :: forall a blk.
(Integral a, EncodeDisk blk blk) =>
CodecConfig blk -> BlocksInFile blk -> a
fileSize CodecConfig blk
ccfg = [a] -> a
forall a. Num a => [a] -> a
forall (t :: * -> *) a. (Foldable t, Num a) => t a -> a
sum ([a] -> a) -> (BlocksInFile blk -> [a]) -> BlocksInFile blk -> a
forall b c a. (b -> c) -> (a -> b) -> a -> c
. (blk -> a) -> [blk] -> [a]
forall a b. (a -> b) -> [a] -> [b]
map (CodecConfig blk -> blk -> a
forall a blk.
(Integral a, EncodeDisk blk blk) =>
CodecConfig blk -> blk -> a
blockSize CodecConfig blk
ccfg) ([blk] -> [a])
-> (BlocksInFile blk -> [blk]) -> BlocksInFile blk -> [a]
forall b c a. (b -> c) -> (a -> b) -> a -> c
. BlocksInFile blk -> [blk]
forall blk. BlocksInFile blk -> [blk]
getBlocksInFile

-- | Only include blocks that come before the given offset.
--
-- > fileTruncateTo ccfg (fileSize blocks) blocks == blocks
fileTruncateTo ::
     forall blk. EncodeDisk blk blk
  => CodecConfig blk
  -> Word64
  -> BlocksInFile blk
  -> BlocksInFile blk
fileTruncateTo :: forall blk.
EncodeDisk blk blk =>
CodecConfig blk -> Word64 -> BlocksInFile blk -> BlocksInFile blk
fileTruncateTo CodecConfig blk
ccfg Word64
validUntil = [blk] -> BlocksInFile blk
forall blk. [blk] -> BlocksInFile blk
BlocksInFile ([blk] -> BlocksInFile blk)
-> (BlocksInFile blk -> [blk])
-> BlocksInFile blk
-> BlocksInFile blk
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Word64 -> [blk] -> [blk]
go Word64
0 ([blk] -> [blk])
-> (BlocksInFile blk -> [blk]) -> BlocksInFile blk -> [blk]
forall b c a. (b -> c) -> (a -> b) -> a -> c
. BlocksInFile blk -> [blk]
forall blk. BlocksInFile blk -> [blk]
getBlocksInFile
  where
    -- Invariant: offset <= validUntil
    go :: Word64 -> [blk] -> [blk]
    go :: Word64 -> [blk] -> [blk]
go Word64
offset = \case
      []
        -> []
      blk
blk:[blk]
blks
        | let newOffset :: Word64
newOffset = Word64
offset Word64 -> Word64 -> Word64
forall a. Num a => a -> a -> a
+ CodecConfig blk -> blk -> Word64
forall a blk.
(Integral a, EncodeDisk blk blk) =>
CodecConfig blk -> blk -> a
blockSize CodecConfig blk
ccfg blk
blk
        , Word64
newOffset Word64 -> Word64 -> Bool
forall a. Ord a => a -> a -> Bool
<= Word64
validUntil
        -> blk
blk blk -> [blk] -> [blk]
forall a. a -> [a] -> [a]
: Word64 -> [blk] -> [blk]
go Word64
newOffset [blk]
blks
        | Bool
otherwise
        -> []

blockSize :: (Integral a, EncodeDisk blk blk) => CodecConfig blk -> blk -> a
blockSize :: forall a blk.
(Integral a, EncodeDisk blk blk) =>
CodecConfig blk -> blk -> a
blockSize CodecConfig blk
ccfg =
      Int64 -> a
forall a b. (Integral a, Num b) => a -> b
fromIntegral
    (Int64 -> a) -> (blk -> Int64) -> blk -> a
forall b c a. (b -> c) -> (a -> b) -> a -> c
. ByteString -> Int64
BL.length
    (ByteString -> Int64) -> (blk -> ByteString) -> blk -> Int64
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Encoding -> ByteString
CBOR.toLazyByteString
    (Encoding -> ByteString) -> (blk -> Encoding) -> blk -> ByteString
forall b c a. (b -> c) -> (a -> b) -> a -> c
. CodecConfig blk -> blk -> Encoding
forall blk a. EncodeDisk blk a => CodecConfig blk -> a -> Encoding
encodeDisk CodecConfig blk
ccfg

{------------------------------------------------------------------------------
  Model API
------------------------------------------------------------------------------}

closeModel :: DBModel blk -> DBModel blk
closeModel :: forall blk. DBModel blk -> DBModel blk
closeModel DBModel blk
dbm = DBModel blk
dbm { open = False }

isOpenModel :: DBModel blk -> Bool
isOpenModel :: forall blk. DBModel blk -> Bool
isOpenModel = DBModel blk -> Bool
forall blk. DBModel blk -> Bool
open

reOpenModel :: DBModel blk -> DBModel blk
reOpenModel :: forall blk. DBModel blk -> DBModel blk
reOpenModel DBModel blk
dbm
    | DBModel blk -> Bool
forall blk. DBModel blk -> Bool
open DBModel blk
dbm
    = DBModel blk
dbm
    | Bool
otherwise
    = DBModel blk -> DBModel blk
forall blk. DBModel blk -> DBModel blk
restoreInvariants (DBModel blk -> DBModel blk) -> DBModel blk -> DBModel blk
forall a b. (a -> b) -> a -> b
$ DBModel blk
dbm { open = True }

getBlockComponentModel ::
     forall blk b.
     ( HasHeader blk
     , GetHeader blk
     , HasBinaryBlockInfo blk
     , EncodeDisk blk blk
     , HasNestedContent Header blk
     )
  => BlockComponent blk b
  -> HeaderHash blk
  -> DBModel blk
  -> Either (VolatileDBError blk) (Maybe b)
getBlockComponentModel :: forall blk b.
(HasHeader blk, GetHeader blk, HasBinaryBlockInfo blk,
 EncodeDisk blk blk, HasNestedContent Header blk) =>
BlockComponent blk b
-> HeaderHash blk
-> DBModel blk
-> Either (VolatileDBError blk) (Maybe b)
getBlockComponentModel BlockComponent blk b
blockComponent HeaderHash blk
hash dbm :: DBModel blk
dbm@DBModel { CodecConfig blk
codecConfig :: forall blk. DBModel blk -> CodecConfig blk
codecConfig :: CodecConfig blk
codecConfig } =
    DBModel blk -> Maybe b -> Either (VolatileDBError blk) (Maybe b)
forall blk (m :: * -> *) a.
MonadError (VolatileDBError blk) m =>
DBModel blk -> a -> m a
whenOpen DBModel blk
dbm (Maybe b -> Either (VolatileDBError blk) (Maybe b))
-> Maybe b -> Either (VolatileDBError blk) (Maybe b)
forall a b. (a -> b) -> a -> b
$
      (blk -> BlockComponent blk b -> b)
-> BlockComponent blk b -> blk -> b
forall a b c. (a -> b -> c) -> b -> a -> c
flip blk -> BlockComponent blk b -> b
forall b'. blk -> BlockComponent blk b' -> b'
go BlockComponent blk b
blockComponent (blk -> b) -> Maybe blk -> Maybe b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$>
        HeaderHash blk -> Map (HeaderHash blk) blk -> Maybe blk
forall k a. Ord k => k -> Map k a -> Maybe a
Map.lookup HeaderHash blk
hash (DBModel blk -> Map (HeaderHash blk) blk
forall blk.
HasHeader blk =>
DBModel blk -> Map (HeaderHash blk) blk
blockIndex DBModel blk
dbm)
  where
    go :: forall b'. blk -> BlockComponent blk b' -> b'
    go :: forall b'. blk -> BlockComponent blk b' -> b'
go blk
blk = \case
        BlockComponent blk b'
GetVerifiedBlock -> blk
b'
blk  -- We don't verify
        BlockComponent blk b'
GetBlock         -> blk
b'
blk
        BlockComponent blk b'
GetRawBlock      -> b'
ByteString
blockBytes
        BlockComponent blk b'
GetHeader        -> blk -> Header blk
forall blk. GetHeader blk => blk -> Header blk
getHeader blk
blk
        BlockComponent blk b'
GetRawHeader     -> BinaryBlockInfo -> ByteString -> ByteString
extractHeader BinaryBlockInfo
binaryBlockInfo ByteString
blockBytes
        BlockComponent blk b'
GetHash          -> blk -> HeaderHash blk
forall b. HasHeader b => b -> HeaderHash b
blockHash blk
blk
        BlockComponent blk b'
GetSlot          -> blk -> SlotNo
forall b. HasHeader b => b -> SlotNo
blockSlot blk
blk
        BlockComponent blk b'
GetIsEBB         -> blk -> IsEBB
forall blk. GetHeader blk => blk -> IsEBB
blockToIsEBB blk
blk
        BlockComponent blk b'
GetBlockSize     -> Int64 -> b'
forall a b. (Integral a, Num b) => a -> b
fromIntegral (Int64 -> b') -> Int64 -> b'
forall a b. (a -> b) -> a -> b
$ ByteString -> Int64
BL.length ByteString
blockBytes
        BlockComponent blk b'
GetHeaderSize    -> BinaryBlockInfo -> Word16
headerSize BinaryBlockInfo
binaryBlockInfo
        BlockComponent blk b'
GetNestedCtxt    -> case Header blk -> DepPair (NestedCtxt Header blk)
forall (f :: * -> *) blk.
HasNestedContent f blk =>
f blk -> DepPair (NestedCtxt f blk)
unnest (blk -> Header blk
forall blk. GetHeader blk => blk -> Header blk
getHeader blk
blk) of
                              DepPair NestedCtxt Header blk a
nestedCtxt a
_ -> NestedCtxt Header blk a -> SomeSecond (NestedCtxt Header) blk
forall {k1} {k2} (f :: k1 -> k2 -> *) (a :: k1) (b :: k2).
f a b -> SomeSecond f a
SomeSecond NestedCtxt Header blk a
nestedCtxt
        GetPure b'
a        -> b'
a
        GetApply BlockComponent blk (a1 -> b')
f BlockComponent blk a1
bc    -> blk -> BlockComponent blk (a1 -> b') -> a1 -> b'
forall b'. blk -> BlockComponent blk b' -> b'
go blk
blk BlockComponent blk (a1 -> b')
f (a1 -> b') -> a1 -> b'
forall a b. (a -> b) -> a -> b
$ blk -> BlockComponent blk a1 -> a1
forall b'. blk -> BlockComponent blk b' -> b'
go blk
blk BlockComponent blk a1
bc
      where
        binaryBlockInfo :: BinaryBlockInfo
binaryBlockInfo = blk -> BinaryBlockInfo
forall blk. HasBinaryBlockInfo blk => blk -> BinaryBlockInfo
getBinaryBlockInfo blk
blk
        blockBytes :: ByteString
blockBytes = Encoding -> ByteString
CBOR.toLazyByteString (Encoding -> ByteString) -> Encoding -> ByteString
forall a b. (a -> b) -> a -> b
$ CodecConfig blk -> blk -> Encoding
forall blk a. EncodeDisk blk a => CodecConfig blk -> a -> Encoding
encodeDisk CodecConfig blk
codecConfig blk
blk

putBlockModel ::
     forall blk. HasHeader blk
  => blk
  -> DBModel blk
  -> Either (VolatileDBError blk) (DBModel blk)
putBlockModel :: forall blk.
HasHeader blk =>
blk -> DBModel blk -> Either (VolatileDBError blk) (DBModel blk)
putBlockModel blk
blk DBModel blk
dbm = DBModel blk
-> DBModel blk -> Either (VolatileDBError blk) (DBModel blk)
forall blk (m :: * -> *) a.
MonadError (VolatileDBError blk) m =>
DBModel blk -> a -> m a
whenOpen DBModel blk
dbm (DBModel blk -> Either (VolatileDBError blk) (DBModel blk))
-> DBModel blk -> Either (VolatileDBError blk) (DBModel blk)
forall a b. (a -> b) -> a -> b
$
    case HeaderHash blk -> Map (HeaderHash blk) blk -> Maybe blk
forall k a. Ord k => k -> Map k a -> Maybe a
Map.lookup (blk -> HeaderHash blk
forall b. HasHeader b => b -> HeaderHash b
blockHash blk
blk) (DBModel blk -> Map (HeaderHash blk) blk
forall blk.
HasHeader blk =>
DBModel blk -> Map (HeaderHash blk) blk
blockIndex DBModel blk
dbm) of
      -- Block already stored
      Just blk
_  -> DBModel blk
dbm
      Maybe blk
Nothing -> DBModel blk -> DBModel blk
forall blk. DBModel blk -> DBModel blk
restoreInvariants (DBModel blk -> DBModel blk) -> DBModel blk -> DBModel blk
forall a b. (a -> b) -> a -> b
$ DBModel blk
dbm {
          -- The invariants guarantee that @getCurrentFileId dbm@ is in
          -- @fileIndex@.
          fileIndex  = Map.adjust
            (appendBlock blk)
            (getCurrentFileId dbm)
            fileIndex
        }
  where
    DBModel { Map FileId (BlocksInFile blk)
fileIndex :: forall blk. DBModel blk -> Map FileId (BlocksInFile blk)
fileIndex :: Map FileId (BlocksInFile blk)
fileIndex } = DBModel blk
dbm

garbageCollectModel ::
     forall blk. HasHeader blk
  => SlotNo
  -> DBModel blk
  -> Either (VolatileDBError blk) (DBModel blk)
garbageCollectModel :: forall blk.
HasHeader blk =>
SlotNo -> DBModel blk -> Either (VolatileDBError blk) (DBModel blk)
garbageCollectModel SlotNo
slot DBModel blk
dbm = DBModel blk
-> DBModel blk -> Either (VolatileDBError blk) (DBModel blk)
forall blk (m :: * -> *) a.
MonadError (VolatileDBError blk) m =>
DBModel blk -> a -> m a
whenOpen DBModel blk
dbm (DBModel blk -> Either (VolatileDBError blk) (DBModel blk))
-> DBModel blk -> Either (VolatileDBError blk) (DBModel blk)
forall a b. (a -> b) -> a -> b
$
     DBModel blk
dbm { fileIndex = fileIndex' }
  where
    (Map FileId (BlocksInFile blk)
_garbageCollected, Map FileId (BlocksInFile blk)
fileIndex') = (FileId -> BlocksInFile blk -> Bool)
-> Map FileId (BlocksInFile blk)
-> (Map FileId (BlocksInFile blk), Map FileId (BlocksInFile blk))
forall k a. (k -> a -> Bool) -> Map k a -> (Map k a, Map k a)
Map.partitionWithKey FileId -> BlocksInFile blk -> Bool
canGC (DBModel blk -> Map FileId (BlocksInFile blk)
forall blk. DBModel blk -> Map FileId (BlocksInFile blk)
fileIndex DBModel blk
dbm)

    canGC :: FileId -> BlocksInFile blk -> Bool
    canGC :: FileId -> BlocksInFile blk -> Bool
canGC FileId
fileId BlocksInFile blk
file =
      FileId
fileId FileId -> FileId -> Bool
forall a. Eq a => a -> a -> Bool
/= DBModel blk -> FileId
forall blk. DBModel blk -> FileId
getCurrentFileId DBModel blk
dbm Bool -> Bool -> Bool
&&
      BlocksInFile blk -> MaxSlotNo
forall blk. HasHeader blk => BlocksInFile blk -> MaxSlotNo
fileMaxSlotNo BlocksInFile blk
file MaxSlotNo -> MaxSlotNo -> Bool
forall a. Ord a => a -> a -> Bool
< SlotNo -> MaxSlotNo
MaxSlotNo SlotNo
slot

filterByPredecessorModel ::
     forall blk. GetPrevHash blk
  => DBModel blk
  -> Either (VolatileDBError blk) (ChainHash blk -> Set (HeaderHash blk))
filterByPredecessorModel :: forall blk.
GetPrevHash blk =>
DBModel blk
-> Either
     (VolatileDBError blk) (ChainHash blk -> Set (HeaderHash blk))
filterByPredecessorModel DBModel blk
dbm = DBModel blk
-> (ChainHash blk -> Set (HeaderHash blk))
-> Either
     (VolatileDBError blk) (ChainHash blk -> Set (HeaderHash blk))
forall blk (m :: * -> *) a.
MonadError (VolatileDBError blk) m =>
DBModel blk -> a -> m a
whenOpen DBModel blk
dbm ((ChainHash blk -> Set (HeaderHash blk))
 -> Either
      (VolatileDBError blk) (ChainHash blk -> Set (HeaderHash blk)))
-> (ChainHash blk -> Set (HeaderHash blk))
-> Either
     (VolatileDBError blk) (ChainHash blk -> Set (HeaderHash blk))
forall a b. (a -> b) -> a -> b
$ \ChainHash blk
predecessor ->
    Set (HeaderHash blk)
-> Maybe (Set (HeaderHash blk)) -> Set (HeaderHash blk)
forall a. a -> Maybe a -> a
fromMaybe Set (HeaderHash blk)
forall a. Set a
Set.empty (Maybe (Set (HeaderHash blk)) -> Set (HeaderHash blk))
-> Maybe (Set (HeaderHash blk)) -> Set (HeaderHash blk)
forall a b. (a -> b) -> a -> b
$ ChainHash blk
-> Map (ChainHash blk) (Set (HeaderHash blk))
-> Maybe (Set (HeaderHash blk))
forall k a. Ord k => k -> Map k a -> Maybe a
Map.lookup ChainHash blk
predecessor Map (ChainHash blk) (Set (HeaderHash blk))
successors
  where
    successors :: Map (ChainHash blk) (Set (HeaderHash blk))
    successors :: Map (ChainHash blk) (Set (HeaderHash blk))
successors = (HeaderHash blk
 -> ChainHash blk
 -> Map (ChainHash blk) (Set (HeaderHash blk))
 -> Map (ChainHash blk) (Set (HeaderHash blk)))
-> Map (ChainHash blk) (Set (HeaderHash blk))
-> Map (HeaderHash blk) (ChainHash blk)
-> Map (ChainHash blk) (Set (HeaderHash blk))
forall k a b. (k -> a -> b -> b) -> b -> Map k a -> b
Map.foldrWithKey'
      (\HeaderHash blk
hash ChainHash blk
prevHash ->
        (Set (HeaderHash blk)
 -> Set (HeaderHash blk) -> Set (HeaderHash blk))
-> ChainHash blk
-> Set (HeaderHash blk)
-> Map (ChainHash blk) (Set (HeaderHash blk))
-> Map (ChainHash blk) (Set (HeaderHash blk))
forall k a. Ord k => (a -> a -> a) -> k -> a -> Map k a -> Map k a
Map.insertWith Set (HeaderHash blk)
-> Set (HeaderHash blk) -> Set (HeaderHash blk)
forall a. Semigroup a => a -> a -> a
(<>) ChainHash blk
prevHash (HeaderHash blk -> Set (HeaderHash blk)
forall a. a -> Set a
Set.singleton HeaderHash blk
hash))
      Map (ChainHash blk) (Set (HeaderHash blk))
forall k a. Map k a
Map.empty
      (DBModel blk -> Map (HeaderHash blk) (ChainHash blk)
forall blk.
GetPrevHash blk =>
DBModel blk -> Map (HeaderHash blk) (ChainHash blk)
getBlockToPredecessor DBModel blk
dbm)

getBlockInfoModel ::
     (GetPrevHash blk, HasBinaryBlockInfo blk)
  => DBModel blk
  -> Either (VolatileDBError blk) (HeaderHash blk -> Maybe (BlockInfo blk))
getBlockInfoModel :: forall blk.
(GetPrevHash blk, HasBinaryBlockInfo blk) =>
DBModel blk
-> Either
     (VolatileDBError blk) (HeaderHash blk -> Maybe (BlockInfo blk))
getBlockInfoModel DBModel blk
dbm = DBModel blk
-> (HeaderHash blk -> Maybe (BlockInfo blk))
-> Either
     (VolatileDBError blk) (HeaderHash blk -> Maybe (BlockInfo blk))
forall blk (m :: * -> *) a.
MonadError (VolatileDBError blk) m =>
DBModel blk -> a -> m a
whenOpen DBModel blk
dbm ((HeaderHash blk -> Maybe (BlockInfo blk))
 -> Either
      (VolatileDBError blk) (HeaderHash blk -> Maybe (BlockInfo blk)))
-> (HeaderHash blk -> Maybe (BlockInfo blk))
-> Either
     (VolatileDBError blk) (HeaderHash blk -> Maybe (BlockInfo blk))
forall a b. (a -> b) -> a -> b
$ \HeaderHash blk
hash ->
    blk -> BlockInfo blk
forall blk.
(GetPrevHash blk, HasBinaryBlockInfo blk) =>
blk -> BlockInfo blk
extractBlockInfo (blk -> BlockInfo blk) -> Maybe blk -> Maybe (BlockInfo blk)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> HeaderHash blk -> Map (HeaderHash blk) blk -> Maybe blk
forall k a. Ord k => k -> Map k a -> Maybe a
Map.lookup HeaderHash blk
hash (DBModel blk -> Map (HeaderHash blk) blk
forall blk.
HasHeader blk =>
DBModel blk -> Map (HeaderHash blk) blk
blockIndex DBModel blk
dbm)

getMaxSlotNoModel ::
     HasHeader blk
  => DBModel blk
  -> Either (VolatileDBError blk) MaxSlotNo
getMaxSlotNoModel :: forall blk.
HasHeader blk =>
DBModel blk -> Either (VolatileDBError blk) MaxSlotNo
getMaxSlotNoModel DBModel blk
dbm = DBModel blk -> MaxSlotNo -> Either (VolatileDBError blk) MaxSlotNo
forall blk (m :: * -> *) a.
MonadError (VolatileDBError blk) m =>
DBModel blk -> a -> m a
whenOpen DBModel blk
dbm (MaxSlotNo -> Either (VolatileDBError blk) MaxSlotNo)
-> MaxSlotNo -> Either (VolatileDBError blk) MaxSlotNo
forall a b. (a -> b) -> a -> b
$
    (BlocksInFile blk -> MaxSlotNo)
-> Map FileId (BlocksInFile blk) -> MaxSlotNo
forall m a. Monoid m => (a -> m) -> Map FileId a -> m
forall (t :: * -> *) m a.
(Foldable t, Monoid m) =>
(a -> m) -> t a -> m
foldMap BlocksInFile blk -> MaxSlotNo
forall blk. HasHeader blk => BlocksInFile blk -> MaxSlotNo
fileMaxSlotNo (Map FileId (BlocksInFile blk) -> MaxSlotNo)
-> Map FileId (BlocksInFile blk) -> MaxSlotNo
forall a b. (a -> b) -> a -> b
$ DBModel blk -> Map FileId (BlocksInFile blk)
forall blk. DBModel blk -> Map FileId (BlocksInFile blk)
fileIndex DBModel blk
dbm

{------------------------------------------------------------------------------
  Corruptions
------------------------------------------------------------------------------}

runCorruptionsModel ::
     EncodeDisk blk blk
  => Corruptions
  -> DBModel blk
  -> DBModel blk
runCorruptionsModel :: forall blk.
EncodeDisk blk blk =>
Corruptions -> DBModel blk -> DBModel blk
runCorruptionsModel Corruptions
corrs DBModel blk
dbm = ((FileCorruption, FsPath) -> DBModel blk -> DBModel blk)
-> DBModel blk -> Corruptions -> DBModel blk
forall a b. (a -> b -> b) -> b -> NonEmpty a -> b
forall (t :: * -> *) a b.
Foldable t =>
(a -> b -> b) -> b -> t a -> b
foldr ((FileCorruption -> FsPath -> DBModel blk -> DBModel blk)
-> (FileCorruption, FsPath) -> DBModel blk -> DBModel blk
forall a b c. (a -> b -> c) -> (a, b) -> c
uncurry FileCorruption -> FsPath -> DBModel blk -> DBModel blk
forall blk.
EncodeDisk blk blk =>
FileCorruption -> FsPath -> DBModel blk -> DBModel blk
runCorruption) DBModel blk
dbm Corruptions
corrs

runCorruption ::
     EncodeDisk blk blk
  => FileCorruption
  -> FsPath
  -> DBModel blk
  -> DBModel blk
runCorruption :: forall blk.
EncodeDisk blk blk =>
FileCorruption -> FsPath -> DBModel blk -> DBModel blk
runCorruption FileCorruption
corruption FsPath
file dbm :: DBModel blk
dbm@DBModel { Map FileId (BlocksInFile blk)
fileIndex :: forall blk. DBModel blk -> Map FileId (BlocksInFile blk)
fileIndex :: Map FileId (BlocksInFile blk)
fileIndex, codecConfig :: forall blk. DBModel blk -> CodecConfig blk
codecConfig = CodecConfig blk
ccfg } =
    case FileCorruption
corruption of
      FileCorruption
DeleteFile      -> DBModel blk
dbm {
            fileIndex = Map.delete fileId fileIndex
          }
      DropLastBytes Word64
n -> DBModel blk
dbm {
            fileIndex = Map.adjust (fileTruncateTo ccfg validBytes) fileId fileIndex
          }
        where
          validBytes :: Word64
validBytes | Word64
n Word64 -> Word64 -> Bool
forall a. Ord a => a -> a -> Bool
> Word64
size  = Word64
0
                     | Bool
otherwise = Word64
size Word64 -> Word64 -> Word64
forall a. Num a => a -> a -> a
- Word64
n
      -- When we simulate corruption, we will do a bitflip in the filesystem. In
      -- the model, this corresponds to truncation, forcing the implementation
      -- to detect the corruption and to truncate accordingly.
      Corrupt Word64
offset  -> DBModel blk
dbm {
            fileIndex = Map.adjust (fileTruncateTo ccfg validBytes) fileId fileIndex
          }
        where
          validBytes :: Word64
validBytes = Word64
offset Word64 -> Word64 -> Word64
forall a. Integral a => a -> a -> a
`mod` Word64
size
  where
    fileId :: FileId
fileId = FsPath -> FileId
unsafeParseFd FsPath
file
    size :: Word64
size   = CodecConfig blk -> BlocksInFile blk -> Word64
forall a blk.
(Integral a, EncodeDisk blk blk) =>
CodecConfig blk -> BlocksInFile blk -> a
fileSize CodecConfig blk
ccfg (Map FileId (BlocksInFile blk)
fileIndex Map FileId (BlocksInFile blk) -> FileId -> BlocksInFile blk
forall k a. Ord k => Map k a -> k -> a
Map.! FileId
fileId)

unsafeParseFd :: FsPath -> FileId
unsafeParseFd :: FsPath -> FileId
unsafeParseFd FsPath
file = FileId -> Maybe FileId -> FileId
forall a. a -> Maybe a -> a
fromMaybe
    ([Char] -> FileId
forall a. HasCallStack => [Char] -> a
error ([Char] -> FileId) -> [Char] -> FileId
forall a b. (a -> b) -> a -> b
$ [Char]
"Could not parse filename " [Char] -> ShowS
forall a. Semigroup a => a -> a -> a
<> FsPath -> [Char]
forall a. Show a => a -> [Char]
show FsPath
file)
    (FsPath -> Maybe FileId
parseFd FsPath
file)