From 1abf132ad13fe344cb3241d600d18f8ce5276c73 Mon Sep 17 00:00:00 2001 From: kderme Date: Sun, 9 Feb 2020 16:24:52 +0200 Subject: [PATCH] Extend VolatileDB recovery --- .../Ouroboros/Consensus/Node/ExitFailure.hs | 126 ++++++++++++++++++ .../src/Ouroboros/Storage/ChainDB/API.hs | 1 - .../Ouroboros/Storage/ChainDB/Impl/VolDB.hs | 20 ++- .../src/Ouroboros/Storage/VolatileDB/Impl.hs | 85 ++++++++---- .../src/Ouroboros/Storage/VolatileDB/Types.hs | 47 +++---- .../src/Ouroboros/Storage/VolatileDB/Util.hs | 34 +++-- .../Test/Ouroboros/Storage/VolatileDB/Mock.hs | 2 +- .../Ouroboros/Storage/VolatileDB/Model.hs | 76 ++++------- .../Storage/VolatileDB/StateMachine.hs | 85 ++++-------- 9 files changed, 285 insertions(+), 191 deletions(-) create mode 100644 ouroboros-consensus/src/Ouroboros/Consensus/Node/ExitFailure.hs diff --git a/ouroboros-consensus/src/Ouroboros/Consensus/Node/ExitFailure.hs b/ouroboros-consensus/src/Ouroboros/Consensus/Node/ExitFailure.hs new file mode 100644 index 00000000000..a606084e9ac --- /dev/null +++ b/ouroboros-consensus/src/Ouroboros/Consensus/Node/ExitFailure.hs @@ -0,0 +1,126 @@ +{-# LANGUAGE LambdaCase #-} +{-# LANGUAGE NamedFieldPuns #-} +{-# LANGUAGE ScopedTypeVariables #-} +module Ouroboros.Consensus.Node.ExitFailure + ( -- * Pre-defined ExitFailures + ExitFailure + , defaultExitFailure + , configurationError + , noNetwork + , restartWithRecovery + , wrongDatabase + , diskFull + , insufficientPermissions + -- * Get the ExitFailure of an Exception + , toExitFailure + ) where + +import Control.Exception (SomeException, fromException) + +import Control.Monad.Class.MonadAsync (ExceptionInLinkedThread (..)) + +import Ouroboros.Storage.ChainDB.API (ChainDbFailure (..)) +import Ouroboros.Storage.FS.API.Types (FsError (..), FsErrorType (..)) +import Ouroboros.Storage.ImmutableDB.Types (ImmutableDBError) +import qualified Ouroboros.Storage.ImmutableDB.Types as ImmDB +import Ouroboros.Storage.VolatileDB.Types (VolatileDBError) +import qualified Ouroboros.Storage.VolatileDB.Types as VolDB + +import Ouroboros.Consensus.Node.DbMarker (DbMarkerError) +import Ouroboros.Consensus.Node.ProtocolInfo.Byron + (PBftLeaderCredentialsError) + +{------------------------------------------------------------------------------- + Pre-defined ExitFailures +-------------------------------------------------------------------------------} + +type ExitFailure = Int + +-- | Something went wrong, just restart the node. +defaultExitFailure :: ExitFailure +defaultExitFailure = 1 + +-- | Something is wrong with the node configuration, the user should check it. +-- +-- For example, for PBFT, it could be that the block signing key and the +-- delegation certificate do not match. +configurationError :: ExitFailure +configurationError = 2 + +-- | There is a problem with the network connection, the user should +-- investigate. +-- +-- TODO We're not yet returning this. +noNetwork :: ExitFailure +noNetwork = 3 + +-- | Something went wrong with the database, restart the node with recovery +-- enabled. +restartWithRecovery :: ExitFailure +restartWithRecovery = 4 + +-- | We were unable to open the database, probably the user is using the wrong +-- directory. See 'DbMarkerError' for details. +wrongDatabase :: ExitFailure +wrongDatabase = 5 + +-- | The disk is full, make some space before restarting the node. +diskFull :: ExitFailure +diskFull = 6 + +-- | The database folder doesn't have the right permissions. +insufficientPermissions :: ExitFailure +insufficientPermissions = 7 + +{------------------------------------------------------------------------------- + Get the ExitFailure of an Exception +-------------------------------------------------------------------------------} + +-- | Return the 'ExitFailure' (to be used in the @ExitFailure@ constructor of +-- 'System.Exit.ExitCode') for the given 'SomeException'. Defaults to +-- 'defaultExitFailure'. +toExitFailure :: SomeException -> ExitFailure +toExitFailure e + | Just (ExceptionInLinkedThread _ e') <- fromException e + = toExitFailure e' + | Just (_ :: DbMarkerError) <- fromException e + = wrongDatabase + | Just (e' :: ChainDbFailure) <- fromException e + = case e' of + ImmDbFailure ue -> immDbUnexpectedError ue + VolDbFailure ue -> volDbUnexpectedError ue + LgrDbFailure fe -> fsError fe + _ -> restartWithRecovery + | Just (_ :: PBftLeaderCredentialsError) <- fromException e + = configurationError + + -- The three exceptions below will always be wrapped in a + -- 'ChainDbFailure', but we include them just in case. + | Just (e' :: ImmutableDBError) <- fromException e + = case e' of + ImmDB.UnexpectedError ue -> immDbUnexpectedError ue + _ -> defaultExitFailure + | Just (e' :: VolatileDBError) <- fromException e + = case e' of + VolDB.UnexpectedError ue -> volDbUnexpectedError ue + _ -> defaultExitFailure + | Just (e' :: FsError) <- fromException e + = fsError e' + + | otherwise + = defaultExitFailure + where + immDbUnexpectedError :: ImmDB.UnexpectedError -> ExitFailure + immDbUnexpectedError = \case + ImmDB.FileSystemError fe -> fsError fe + _ -> restartWithRecovery + + volDbUnexpectedError :: VolDB.UnexpectedError -> ExitFailure + volDbUnexpectedError = \case + VolDB.FileSystemError fe -> fsError fe + + fsError :: FsError -> ExitFailure + fsError FsError { fsErrorType } = case fsErrorType of + FsDeviceFull -> diskFull + FsInsufficientPermissions -> insufficientPermissions + _ -> restartWithRecovery diff --git a/ouroboros-consensus/src/Ouroboros/Storage/ChainDB/API.hs b/ouroboros-consensus/src/Ouroboros/Storage/ChainDB/API.hs index 7331967ecf8..2248ac1a6ec 100644 --- a/ouroboros-consensus/src/Ouroboros/Storage/ChainDB/API.hs +++ b/ouroboros-consensus/src/Ouroboros/Storage/ChainDB/API.hs @@ -762,7 +762,6 @@ instance Exception ChainDbFailure where VolDbMissingBlock {} -> corruption VolDbFailure e -> case e of VolDB.FileSystemError fse -> fsError fse - VolDB.ParserError {} -> corruption LgrDbFailure fse -> fsError fse ChainDbMissingBlock {} -> corruption where diff --git a/ouroboros-consensus/src/Ouroboros/Storage/ChainDB/Impl/VolDB.hs b/ouroboros-consensus/src/Ouroboros/Storage/ChainDB/Impl/VolDB.hs index d88bf872af2..35673881930 100644 --- a/ouroboros-consensus/src/Ouroboros/Storage/ChainDB/Impl/VolDB.hs +++ b/ouroboros-consensus/src/Ouroboros/Storage/ChainDB/Impl/VolDB.hs @@ -30,7 +30,7 @@ module Ouroboros.Storage.ChainDB.Impl.VolDB ( , computePath , computePathSTM -- * Getting and parsing blocks - , BlockFileParserError (..) + , BlockFileParserError , getKnownBlock , getKnownHeader , getKnownBlockComponent @@ -134,7 +134,7 @@ instance NoUnexpectedThunks (VolDB m blk) where -- | Short-hand for events traced by the VolDB wrapper. type TraceEvent blk = - VolDB.TraceEvent (BlockFileParserError (HeaderHash blk)) (HeaderHash blk) + VolDB.TraceEvent Util.CBOR.ReadIncrementalErr (HeaderHash blk) {------------------------------------------------------------------------------- Initialization @@ -496,15 +496,13 @@ getBlockComponent db blockComponent hash = withDB db $ \vol -> Parsing -------------------------------------------------------------------------------} -data BlockFileParserError hash = - BlockReadErr Util.CBOR.ReadIncrementalErr - | BlockCorruptedErr hash - deriving (Eq, Show) +type BlockFileParserError hash = + VolDB.ParserError hash Util.CBOR.ReadIncrementalErr blockFileParser :: forall m blk. (IOLike m, HasHeader blk) => VolDbArgs m blk -> VolDB.Parser - (BlockFileParserError (HeaderHash blk)) + Util.CBOR.ReadIncrementalErr m (HeaderHash blk) blockFileParser VolDbArgs{..} = @@ -521,7 +519,7 @@ blockFileParser' :: forall m blk h. (IOLike m, HasHeader blk) -> (blk -> Bool) -> VolDB.BlockValidationPolicy -> VolDB.Parser - (BlockFileParserError (HeaderHash blk)) + Util.CBOR.ReadIncrementalErr m (HeaderHash blk) blockFileParser' hasFS isEBB encodeBlock decodeBlock validate validPolicy = @@ -545,14 +543,14 @@ blockFileParser' hasFS isEBB encodeBlock decodeBlock validate validPolicy = -> m (VolDB.ParsedInfo (HeaderHash blk), Maybe (BlockFileParserError (HeaderHash blk))) checkEntries parsed stream = S.next stream >>= \case - Left mbErr -> return (reverse parsed, BlockReadErr . fst <$> mbErr) + Left mbErr -> return (reverse parsed, VolDB.BlockReadErr . fst <$> mbErr) Right ((offset, (size, blk)), stream') | noValidation || (validate blk) -> let !blockInfo = extractInfo' blk newParsed = (offset, (VolDB.BlockSize size, blockInfo)) in checkEntries (newParsed : parsed) stream' Right ((_, (_, blk)), _) -> - let !bid = VolDB.bbid $ extractInfo' blk - in return (reverse parsed, Just (BlockCorruptedErr bid)) + let !bid = VolDB.bbid $ extractInfo' blk + in return (reverse parsed, Just (VolDB.BlockCorruptedErr bid)) {------------------------------------------------------------------------------- Error handling diff --git a/ouroboros-consensus/src/Ouroboros/Storage/VolatileDB/Impl.hs b/ouroboros-consensus/src/Ouroboros/Storage/VolatileDB/Impl.hs index 008f86a6375..6509d2dc615 100644 --- a/ouroboros-consensus/src/Ouroboros/Storage/VolatileDB/Impl.hs +++ b/ouroboros-consensus/src/Ouroboros/Storage/VolatileDB/Impl.hs @@ -84,6 +84,13 @@ -- There is an implicit ordering of block files, which is NOT alpharithmetic -- For example blocks-20.dat < blocks-100.dat -- +-- = Recovery +-- +-- The VolatileDB will always try to recover to a consistent state even if this +-- means deleting all of its contents. In order to achieve this, it truncates +-- files of blocks, if some blocks fail to parse, are invalid, or are +-- duplicated. The db also ignores any filenames which fail to parse. +-- module Ouroboros.Storage.VolatileDB.Impl ( -- * Opening a database openDB @@ -100,7 +107,6 @@ import Control.Monad import Control.Tracer (Tracer, traceWith) import qualified Data.ByteString.Builder as BS import Data.List (find, sortOn) -import Data.Map.Strict (Map) import qualified Data.Map.Strict as Map import Data.Maybe import Data.Set (Set) @@ -539,7 +545,8 @@ reOpenFile HasFS{..} _err VolatileDBEnv{..} st@InternalState{..} = do , _currentWriteOffset = 0 } -mkInternalStateDB :: ( HasCallStack +mkInternalStateDB :: forall m blockId e h. + ( HasCallStack , MonadThrow m , MonadCatch m , Ord blockId @@ -556,9 +563,16 @@ mkInternalStateDB hasFS@HasFS{..} err parser tracer maxBlocksPerFile = wrapFsError hasFsErr err $ do createDirectoryIfMissing True dbDir allFiles <- map toFsPath . Set.toList <$> listDirectory dbDir - filesWithIds <- fromEither err $ parseAllFds allFiles + filesWithIds <- logInvalidFiles $ parseAllFds allFiles mkInternalState hasFS err parser tracer maxBlocksPerFile filesWithIds where + -- | Logs about any invalid 'FsPath' and returns the valid ones. + logInvalidFiles :: ([(FileId, FsPath)], [FsPath]) -> m [(FileId, FsPath)] + logInvalidFiles (valid, invalid) = do + when (not $ null invalid) $ + traceWith tracer $ InvalidFileNames invalid + return valid + dbDir = mkFsPath [] toFsPath :: String -> FsPath @@ -594,8 +608,11 @@ mkInternalState hasFS err parser tracer n files = , Index.insert newIndex FileInfo.empty curMap , FileSize 0 ) - truncateOnError Nothing _ _ = return () - truncateOnError (Just e) file offset = do + truncateOnError :: Maybe (SlotOffset, ParserError blockId e) + -> FsPath + -> m () + truncateOnError Nothing _ = return () + truncateOnError (Just (offset, e)) file = do traceWith tracer $ Truncate e file offset -- The handle of the parser is closed at this point. We need -- to reopen the file in 'AppendMode' now (parser opens with @@ -653,23 +670,25 @@ mkInternalState hasFS err parser tracer n files = (blocks, mErr) <- parse parser file updateAndGo blocks mErr where - -- | Updates the state and call 'go' for the rest of the files. - updateAndGo :: [(SlotOffset, (BlockSize, BlockInfo blockId))] - -> Maybe e + -- | Updates the state, truncates if there are any parsing errors and + -- calls 'go' for the rest of the files. + updateAndGo :: ParsedInfo blockId + -> Maybe (ParserError blockId e) -> m (InternalState blockId h) updateAndGo blocks mErr = do - truncateOnError mErr file offset - newRevMap <- fromEither err $ reverseMap file currentRevMap fileMap + truncateOnError (fmap (offset,) mErr) file + truncateOnError mErr' file go newMap newRevMap newSuccMap newMaxSlot newHaveLessThanN rest where - offset = case reverse blocks of + (newRevMap, acceptedBlocks, mErr') = + reverseMap file currentRevMap blocks + offset = case reverse acceptedBlocks of [] -> 0 (slotOffset, (blockSize,_)) : _ -> -- The file offset is given by the offset of the last -- block plus its size. slotOffset + unBlockSize blockSize - fileMap = Map.fromList blocks - (fileInfo, maxSlotOfFile) = FileInfo.fromParsedInfo blocks + (fileInfo, maxSlotOfFile) = FileInfo.fromParsedInfo acceptedBlocks newMap = Index.insert fd fileInfo currentMap newMaxSlot = maxSlotList $ catMaybes [maxSlot, maxSlotOfFile] -- For each block we need to update the succesor Map of its @@ -678,7 +697,7 @@ mkInternalState hasFS err parser tracer n files = (\(_,(_, blockInfo)) succMap' -> insertMapSet succMap' (bbid blockInfo, bpreBid blockInfo)) succMap - blocks + acceptedBlocks newHaveLessThanN = if FileInfo.isFull n fileInfo then lessThanN else (fd, file, FileSize offset) : lessThanN @@ -758,28 +777,40 @@ getterSTM fromSt VolatileDBEnv{..} = do VolatileDbClosed -> EH.throwError' _dbErrSTM $ UserError ClosedDBError VolatileDbOpen st -> return $ fromSt st --- | For each block found in a parsed file, we insert its 'InternalBlockInfo'. --- If the block is already found in the 'ReverseIndex' or is duplicated in the --- same file, we abort and return an error. -reverseMap :: forall blockId. ( +-- | For each block found in a parsed file, we insert its 'InternalBlockInfo' +-- to the 'ReverseIndex'. +-- If a block is already found in the 'ReverseIndex' or is duplicated in the +-- same file, we stop and return an error, the offset to truncate and ignore +-- the rest blocks of the file. +reverseMap :: forall blockId e. ( Ord blockId , Typeable blockId , Show blockId ) => FsPath -> ReverseIndex blockId - -> Map SlotOffset (BlockSize, BlockInfo blockId) - -> Either VolatileDBError (ReverseIndex blockId) -reverseMap file revMap mp = foldM go revMap (Map.toList mp) + -> ParsedInfo blockId + -> ( ReverseIndex blockId + , ParsedInfo blockId + , Maybe (SlotOffset, ParserError blockId e)) +reverseMap file revMap = go revMap [] where go :: ReverseIndex blockId - -> (SlotOffset, (BlockSize, BlockInfo blockId)) - -> Either VolatileDBError (ReverseIndex blockId) - go rv (offset, (size, BlockInfo {..})) = case Map.lookup bbid rv of - Nothing -> Right $ Map.insert bbid internalBlockInfo rv - Just blockInfo -> Left $ UnexpectedError . ParserError - $ DuplicatedSlot bbid file (ibFile blockInfo) + -> ParsedInfo blockId -- accumulator of the accepted blocks. + -> ParsedInfo blockId + -> ( ReverseIndex blockId + , ParsedInfo blockId + , Maybe (SlotOffset, ParserError blockId e)) + go rv acc [] = (rv, reverse acc, Nothing) + go rv acc (parsedBlock : rest) = case Map.lookup bbid rv of + Nothing -> + go (Map.insert bbid internalBlockInfo rv) (parsedBlock : acc) rest + Just blockInfo -> + ( rv + , reverse acc + , Just (offset, DuplicatedSlot bbid file (ibFile blockInfo)) ) where + (offset, (size, BlockInfo {..})) = parsedBlock internalBlockInfo = InternalBlockInfo { ibFile = file , ibSlotOffset = offset diff --git a/ouroboros-consensus/src/Ouroboros/Storage/VolatileDB/Types.hs b/ouroboros-consensus/src/Ouroboros/Storage/VolatileDB/Types.hs index 59d38c67351..7c22cca1b0c 100644 --- a/ouroboros-consensus/src/Ouroboros/Storage/VolatileDB/Types.hs +++ b/ouroboros-consensus/src/Ouroboros/Storage/VolatileDB/Types.hs @@ -14,7 +14,6 @@ module Ouroboros.Storage.VolatileDB.Types import Control.Exception (Exception (..)) import Data.Map.Strict (Map) import Data.Set (Set) -import Data.Typeable import Data.Word (Word16, Word64) import GHC.Generics (Generic) @@ -63,7 +62,6 @@ data UserError = data UnexpectedError = FileSystemError FsError - | ParserError ParserError deriving (Show) instance Eq VolatileDBError where @@ -72,15 +70,13 @@ instance Eq VolatileDBError where instance Exception VolatileDBError where displayException = show -data ParserError = - forall blockId. (Typeable blockId, Eq blockId, Show blockId) => - DuplicatedSlot blockId FsPath FsPath - | InvalidFilename FsPath - -deriving instance Show ParserError - -instance Eq ParserError where - (==) = sameParseError +-- | This needs not be an 'Exception' instance, since we recover and don't +-- throw such errors. +data ParserError blockId e = + BlockReadErr e + | BlockCorruptedErr blockId + | DuplicatedSlot blockId FsPath FsPath + deriving (Eq, Show) sameVolatileDBError :: VolatileDBError -> VolatileDBError @@ -95,18 +91,6 @@ sameUnexpectedError :: UnexpectedError -> Bool sameUnexpectedError e1 e2 = case (e1, e2) of (FileSystemError fs1, FileSystemError fs2) -> sameFsError fs1 fs2 - (ParserError p1, ParserError p2) -> p1 == p2 - _ -> False - --- | This is not comparing the arguments of 'DuplicatedSlot', because it's not --- deterministic which duplication we find. In other words, it's possible that --- there are multiple pairs of duplicate blocks and our algorithm does not --- guarantee we always find the same. -sameParseError :: ParserError -> ParserError -> Bool -sameParseError e1 e2 = case (e1, e2) of - (DuplicatedSlot {}, DuplicatedSlot {}) -> True - (InvalidFilename str1, InvalidFilename str2) -> str1 == str2 - _ -> False newtype FileSize = FileSize {unFileSize :: Word64} deriving (Show, Generic, NoUnexpectedThunks) @@ -115,17 +99,19 @@ newtype BlockSize = BlockSize {unBlockSize :: Word64} newtype Parser e m blockId = Parser { -- | Parse block storage at the given path. - parse :: FsPath -> m (ParsedInfo blockId, Maybe e) + parse :: FsPath -> m (ParsedInfo blockId, Maybe (ParserError blockId e)) } -- | The offset of a slot in a file. type SlotOffset = Word64 -- | Information returned by the parser about a single file. +type ParsedInfo blockId = [ParsedBlockInfo blockId] + +-- | Information returned by the parser about a single block. -- --- The parser returns for each block, its size its blockId, its slot and its --- predecessor's blockId. -type ParsedInfo blockId = [(SlotOffset, (BlockSize, BlockInfo blockId))] +-- The parser returns for each block, its offset, its size and its 'BlockInfo' +type ParsedBlockInfo blockId = (SlotOffset, (BlockSize, BlockInfo blockId)) -- | The information that the user has to provide for each new block. data BlockInfo blockId = BlockInfo { @@ -153,10 +139,11 @@ data InternalBlockInfo blockId = InternalBlockInfo { Tracing ------------------------------------------------------------------------------} -data TraceEvent e hash +data TraceEvent e blockId = DBAlreadyClosed | DBAlreadyOpen - | BlockAlreadyHere hash + | BlockAlreadyHere blockId | TruncateCurrentFile FsPath - | Truncate e FsPath SlotOffset + | Truncate (ParserError blockId e) FsPath SlotOffset + | InvalidFileNames [FsPath] deriving (Eq, Generic, Show) diff --git a/ouroboros-consensus/src/Ouroboros/Storage/VolatileDB/Util.hs b/ouroboros-consensus/src/Ouroboros/Storage/VolatileDB/Util.hs index 4ed71704ddb..ee855374e5c 100644 --- a/ouroboros-consensus/src/Ouroboros/Storage/VolatileDB/Util.hs +++ b/ouroboros-consensus/src/Ouroboros/Storage/VolatileDB/Util.hs @@ -29,7 +29,10 @@ module Ouroboros.Storage.VolatileDB.Util ) where import Control.Monad +import Data.Bifunctor (first) +import Data.List (sortOn) import qualified Data.Map.Strict as Map +import Data.Maybe (fromMaybe) import Data.Set (Set) import qualified Data.Set as Set import Data.Text (Text) @@ -50,12 +53,10 @@ import Ouroboros.Storage.VolatileDB.Types FileId utilities ------------------------------------------------------------------------------} -parseFd :: FsPath -> Either VolatileDBError FileId -parseFd file = maybe err Right $ +parseFd :: FsPath -> Maybe FileId +parseFd file = parseFilename <=< lastMaybe $ fsPathToList file where - err = Left $ UnexpectedError $ ParserError $ InvalidFilename file - parseFilename :: Text -> Maybe FileId parseFilename = readMaybe . T.unpack @@ -64,22 +65,27 @@ parseFd file = maybe err Right $ . fst . T.breakOn "." --- | Parses the 'FileId' of each 'FsPath' and zips them together. --- When parsing fails, we abort with the corresponding parse error. -parseAllFds :: [FsPath] -> Either VolatileDBError [(FileId, FsPath)] -parseAllFds = mapM $ \f -> (,f) <$> parseFd f +-- | Parses the 'FileId' of each 'FsPath' and zips them together. Returns +-- the results sorted on the 'FileId'. +-- +-- Return separately any 'FsPath' which failed to parse. +parseAllFds :: [FsPath] -> ([(FileId, FsPath)], [FsPath]) +parseAllFds = first (sortOn fst) . foldr judge ([], []) + where + judge fsPath (parsed, notParsed) = case parseFd fsPath of + Nothing -> (parsed, fsPath : notParsed) + Just fileId -> ((fileId, fsPath) : parsed, notParsed) --- | When parsing fails, we abort with the corresponding parse error. -findLastFd :: [FsPath] -> Either VolatileDBError (Maybe FileId) -findLastFd = fmap safeMaximum . mapM parseFd +-- | This also returns any 'FsPath' which failed to parse. +findLastFd :: [FsPath] -> (Maybe FileId, [FsPath]) +findLastFd = first (fmap fst . lastMaybe) . parseAllFds filePath :: FileId -> FsPath filePath fd = mkFsPath ["blocks-" ++ show fd ++ ".dat"] unsafeParseFd :: FsPath -> FileId -unsafeParseFd file = either - (\_ -> error $ "Could not parse filename " <> show file) - id +unsafeParseFd file = fromMaybe + (error $ "Could not parse filename " <> show file) (parseFd file) {------------------------------------------------------------------------------ diff --git a/ouroboros-consensus/test-storage/Test/Ouroboros/Storage/VolatileDB/Mock.hs b/ouroboros-consensus/test-storage/Test/Ouroboros/Storage/VolatileDB/Mock.hs index f666ae83496..6fcfa744581 100644 --- a/ouroboros-consensus/test-storage/Test/Ouroboros/Storage/VolatileDB/Mock.hs +++ b/ouroboros-consensus/test-storage/Test/Ouroboros/Storage/VolatileDB/Mock.hs @@ -31,7 +31,7 @@ openDBMock err maxNumPerFile = do db dbVar = VolatileDB { closeDB = wrapModel' dbVar $ closeModel , isOpenDB = wrapModel' dbVar $ isOpenModel - , reOpenDB = wrapModel' dbVar $ reOpenModel err' + , reOpenDB = wrapModel' dbVar $ reOpenModel , getBlockComponent = wrapModel' dbVar .: (getBlockComponentModel err' . castBlockComponent) , putBlock = wrapModel' dbVar .: putBlockModel err' , garbageCollect = wrapModel' dbVar . garbageCollectModel err' diff --git a/ouroboros-consensus/test-storage/Test/Ouroboros/Storage/VolatileDB/Model.hs b/ouroboros-consensus/test-storage/Test/Ouroboros/Storage/VolatileDB/Model.hs index 00f79bfef25..ddbf3051603 100644 --- a/ouroboros-consensus/test-storage/Test/Ouroboros/Storage/VolatileDB/Model.hs +++ b/ouroboros-consensus/test-storage/Test/Ouroboros/Storage/VolatileDB/Model.hs @@ -42,14 +42,14 @@ import Data.Bifunctor import Data.ByteString.Builder import Data.ByteString.Lazy (ByteString) import qualified Data.ByteString.Lazy as BL -import Data.Either import Data.List (find, sortOn, splitAt) import Data.Map (Map) import qualified Data.Map as Map -import Data.Maybe (fromMaybe, isNothing, mapMaybe) +import Data.Maybe (fromMaybe, mapMaybe) import Data.Set (Set) import qualified Data.Set as Set import Data.Typeable +import Data.Word import Ouroboros.Network.Point (WithOrigin) @@ -71,8 +71,6 @@ type Index blockId = Map FsPath (MaxSlotNo, [(blockId, WithOrigin blockId)]) data DBModel blockId = DBModel { blocksPerFile :: Int -- ^ How many blocks each file has (should follow the real Impl). - , parseError :: Maybe ParserError - -- ^ An error indicates a broken db and that parsing will fail. , open :: Bool -- ^ Indicates if the db is open. , mp :: Map blockId (SlotNo, IsEBB, BinaryInfo ByteString) @@ -88,7 +86,6 @@ data DBModel blockId = DBModel { initDBModel :: Int -> DBModel blockId initDBModel bpf = DBModel { blocksPerFile = bpf - , parseError = Nothing , open = True , mp = Map.empty , latestGarbaged = Nothing @@ -134,11 +131,8 @@ unsafeGetBlocks :: DBModel blockId -> [(blockId, WithOrigin blockId)] unsafeGetBlocks file = snd . unsafeLookupIndex file --- | It throws an error if any file of the index does not parse. unsafeLastFd :: DBModel blockId -> Maybe FileId -unsafeLastFd DBModel {..} = - fromRight (error "filename in index didn't parse" ) $ - findLastFd (Map.keys index) +unsafeLastFd DBModel {..} = fst $ findLastFd (Map.keys index) getBlockId :: DBModel blockId -> [(blockId, WithOrigin blockId)] getBlockId DBModel {..} = concat $ snd <$> Map.elems index @@ -173,12 +167,11 @@ isOpenModel = do return open reOpenModel :: MonadState (DBModel blockId) m - => ThrowCantCatch VolatileDBError m - -> m () -reOpenModel err = do + => m () +reOpenModel = do dbm <- get dbm' <- if not $ open dbm - then recover err dbm + then recover dbm else return dbm put dbm' {open = True} @@ -349,7 +342,6 @@ runCorruptionModel corrs = do DropLastBytes _n -> dbm { mp = mp' , index = index' - , parseError = parseError' } where bids = unsafeGetBlocks dbm file @@ -360,23 +352,15 @@ runCorruptionModel corrs = do newMmax = snd <$> maxSlotList newBidsWithSlots index' = Map.insert file (maxSlotNoFromMaybe newMmax, newBids) index mp' = Map.withoutKeys mp (Set.fromList $ fst <$> droppedBids) - parseError' = if isNothing parseError - then Nothing else parseError - AppendBytes n -> + AppendBytes _n -> -- Appending doesn't actually change anything, since additional -- bytes will be truncated. We have taken care that additional -- bytes cannot parse as a block. If something like this happens, -- there are not much we can do anyway. - dbm {parseError = parseError'} - where - -- We predict what error the parser will throw. It's easier to do - -- this here, rather than on reopening. reopening will later actualy - -- throw the error. - parseError' = if n > 0 && isNothing parseError - then Nothing else parseError + dbm PutCorrupted _tb -> - -- Putting a corrupted block is a no-op since they will be truncated. - dbm + -- Putting a corrupted block is a no-op since they will be truncated. + dbm createFileModel :: forall blockId m. MonadState (DBModel blockId) m => m () @@ -384,42 +368,34 @@ createFileModel = do dbm <- get put dbm {index = openNewFile dbm} +-- | Creating invalid files is a no-op, since the parser ignores them. createInvalidFileModel :: forall blockId m. MonadState (DBModel blockId) m - => FsPath + => Word32 -> m () -createInvalidFileModel file = do - dbm <- get - put dbm {parseError = Just $ InvalidFilename file} +createInvalidFileModel _n = return () +-- | Inserting a duplicate block is a no-op, since the parser truncates them. duplicateBlockModel :: forall blockId m. ( MonadState (DBModel blockId) m , Typeable blockId , Eq blockId , Show blockId ) - => (FsPath, blockId) - -> m () -duplicateBlockModel (file, bid) = do - dbm <- get - let current = getCurrentFile dbm - put dbm {parseError = Just $ DuplicatedSlot bid file current} + => m () +duplicateBlockModel = return () recover :: MonadState (DBModel blockId) m - => ThrowCantCatch VolatileDBError m - -> DBModel blockId + => DBModel blockId -> m (DBModel blockId) -recover err dbm@DBModel {..} = - case parseError of - Just pError -> EH.throwError' err $ UnexpectedError $ ParserError pError - Nothing -> return dbm { - index = index' - , parseError = Nothing - -- Recalculate it from the index to match the real implementation - , maxSlotNo = maxSlotNoFromMaybe - $ safeMaximum - $ mapMaybe (\(mbS, _) -> maxSlotNoToMaybe mbS) - $ Map.elems index' - } +recover dbm@DBModel {..} = + return dbm { + index = index' + -- Recalculate it from the index to match the real implementation + , maxSlotNo = maxSlotNoFromMaybe + $ safeMaximum + $ mapMaybe (\(mbS, _) -> maxSlotNoToMaybe mbS) + $ Map.elems index' + } where lastFd = unsafeLastFd dbm ls = Map.toList index diff --git a/ouroboros-consensus/test-storage/Test/Ouroboros/Storage/VolatileDB/StateMachine.hs b/ouroboros-consensus/test-storage/Test/Ouroboros/Storage/VolatileDB/StateMachine.hs index c3a62c19eae..a9d186b19c5 100644 --- a/ouroboros-consensus/test-storage/Test/Ouroboros/Storage/VolatileDB/StateMachine.hs +++ b/ouroboros-consensus/test-storage/Test/Ouroboros/Storage/VolatileDB/StateMachine.hs @@ -35,7 +35,7 @@ import qualified Data.Map as M import Data.Maybe (fromJust, isJust, mapMaybe) import Data.Set (Set) import qualified Data.Set as S -import Data.TreeDiff (ToExpr (..), defaultExprViaShow) +import Data.TreeDiff (ToExpr (..)) import Data.Word import GHC.Generics import GHC.Stack @@ -156,7 +156,7 @@ data Cmd | AskIfMember [BlockId] | Corrupt Corruptions | CreateFile - | CreateInvalidFile + | CreateInvalidFile Word32 | DuplicateBlock FsPath TestBlock | GetSuccessors [Predecessor] | GetPredecessor [BlockId] @@ -199,9 +199,6 @@ deriving instance ToExpr (BinaryInfo ByteString) deriving instance ToExpr (ChainHash TestHeader) deriving instance ToExpr BlockNo -instance ToExpr ParserError where - toExpr = defaultExprViaShow - instance CommandNames (At Cmd) where cmdName (At cmd) = case cmd of GetBlockComponent {} -> "GetBlockComponent" @@ -228,8 +225,6 @@ instance CommandNames (At CmdErr) where data Model (r :: Type -> Type) = Model { dbModel :: DBModel BlockId -- ^ A model of the database. - , shouldEnd :: Bool - -- ^ Indicates that no further commands should be generated. } deriving (Generic, Show) type PureM = ExceptT VolatileDBError (State (DBModel BlockId)) @@ -246,9 +241,8 @@ data Event r = Event { lockstep :: forall r. Model r -> At CmdErr r - -> At Resp r -> Event r -lockstep model@Model {..} cmdErr (At resp) = Event { +lockstep model@Model {..} cmdErr = Event { eventBefore = model , eventCmd = cmdErr , eventAfter = model' @@ -256,12 +250,8 @@ lockstep model@Model {..} cmdErr (At resp) = Event { } where (mockResp, dbModel') = step model cmdErr - ends = case resp of - Resp (Left (UnexpectedError (ParserError _))) -> True - _ -> False model' = model { dbModel = dbModel' - , shouldEnd = ends } -- | Key property of the model is that we can go from real to mock responses. @@ -297,7 +287,7 @@ runPure dbm (CmdErr cmd err) = let blockInfo = mkBlockInfo tb blob = testBlockToBuilder tb Unit <$> putBlockModel tnc blockInfo blob - GetSuccessors bids -> do + GetSuccessors bids -> do successors <- getSuccessorsModel tnc return $ Successors $ successors <$> bids GetPredecessor bids -> do @@ -306,38 +296,35 @@ runPure dbm (CmdErr cmd err) = GarbageCollect slot -> Unit <$> garbageCollectModel tnc slot IsOpen -> Bl <$> isOpenModel Close -> Unit <$> closeModel - ReOpen -> Unit <$> reOpenModel tnc + ReOpen -> Unit <$> reOpenModel AskIfMember bids -> do isMember <- getIsMemberModel tnc return $ IsMember $ isMember <$> bids GetMaxSlotNo -> MaxSlot <$> getMaxSlotNoModel tnc Corrupt cors -> withClosedDB $ runCorruptionModel cors CreateFile -> withClosedDB createFileModel - CreateInvalidFile -> withClosedDB $ - createInvalidFileModel (mkFsPath ["invalidFileName.dat"]) - DuplicateBlock file tb -> withClosedDB $ - duplicateBlockModel (file, thHash $ testHeader tb) + CreateInvalidFile n -> withClosedDB $ createInvalidFileModel n + DuplicateBlock _ _ -> withClosedDB duplicateBlockModel withClosedDB :: PureM () -> PureM Success withClosedDB act = do closeModel act - reOpenModel tnc + reOpenModel return $ Unit () sm :: IOLike m - => Bool - -> StrictTVar m Errors + => StrictTVar m Errors -> VolatileDB BlockId m -> Internal.VolatileDBEnv m BlockId -> DBModel BlockId -> StateMachine Model (At CmdErr) m (At Resp) -sm terminatingCmd errorsVar db env dbm = StateMachine { +sm errorsVar db env dbm = StateMachine { initModel = initModelImpl dbm , transition = transitionImpl , precondition = preconditionImpl , postcondition = postconditionImpl - , generator = generatorImpl True terminatingCmd + , generator = generatorImpl True , shrinker = shrinkerImpl , semantics = semanticsImpl errorsVar db env , mock = mockImpl @@ -349,7 +336,6 @@ stateMachine :: IOLike m => DBModel BlockId -> StateMachine Model (At CmdErr) m (At Resp) stateMachine = sm - True (error "errorsVar unused") (error "semantics and DB used during command generation") (error "env used during command generation") @@ -357,16 +343,13 @@ stateMachine = sm initModelImpl :: DBModel BlockId -> Model r initModelImpl dbm = Model { dbModel = dbm - , shouldEnd = False } transitionImpl :: Model r -> At CmdErr r -> At Resp r -> Model r -transitionImpl model cmd = eventAfter . lockstep model cmd +transitionImpl model cmd _ = eventAfter $ lockstep model cmd preconditionImpl :: Model Symbolic -> At CmdErr Symbolic -> Logic -preconditionImpl Model{..} (At (CmdErr cmd err)) = - Not (Boolean shouldEnd) - .&& compatibleWithError +preconditionImpl Model{..} (At (CmdErr cmd err)) = compatibleWithError .&& case cmd of GetBlockComponent bid -> Boolean $ afterGC bid GetPredecessor bids -> forall bids (`elem` bidsInModel) @@ -378,7 +361,7 @@ preconditionImpl Model{..} (At (CmdErr cmd err)) = .&& forall (corruptionFiles cors) (`elem` getDBFiles dbModel) .&& forall (corruptions cors) corruptionPrecondition CreateFile -> isOpen - CreateInvalidFile -> isOpen + CreateInvalidFile _n -> isOpen DuplicateBlock file tb -> case fmap fst . snd <$> M.lookup file (index dbModel) of Nothing -> Bot @@ -427,17 +410,16 @@ postconditionImpl :: Model Concrete postconditionImpl model cmdErr resp = toMock (eventAfter ev) resp .== eventMockResp ev where - ev = lockstep model cmdErr resp + ev = lockstep model cmdErr -generatorCmdImpl :: Bool -> Model Symbolic -> Maybe (Gen (At Cmd Symbolic)) -generatorCmdImpl terminatingCmd m@Model {..} = - if shouldEnd then Nothing else Just $ do +generatorCmdImpl :: Model Symbolic -> Maybe (Gen (At Cmd Symbolic)) +generatorCmdImpl m@Model {..} = Just $ do blockId <- blockIdGenerator m blockIds <- listOf $ blockIdGenerator m slot <- arbitrary -- TODO: We may want to have more collisions. testBlock <- genBlock duplicate <- mkDuplicate testBlock - let allowDuplication = terminatingCmd && isJust duplicate + invalidId <- arbitrary At <$> frequency [ (150, return $ GetBlockComponent blockId) , (100, return $ GetBlockIds) @@ -450,6 +432,7 @@ generatorCmdImpl terminatingCmd m@Model {..} = , (50, return $ IsOpen) , (50, return $ Close) , (30, return CreateFile) + , (20, return $ CreateInvalidFile invalidId) -- When the db is Closed, we try to ReOpen it asap. -- This helps minimize TagClosedError and create more -- interesting tests. @@ -457,8 +440,7 @@ generatorCmdImpl terminatingCmd m@Model {..} = , (if null blockIds then 0 else 30, return $ AskIfMember blockIds) , (if null dbFiles then 0 else 100, Corrupt <$> generateCorruptions testBlock dbFiles) - , (if terminatingCmd then 1 else 0, return CreateInvalidFile) - , (if allowDuplication then 1 else 0, return $ fromJust duplicate) + , (if isJust duplicate then 20 else 0, return $ fromJust duplicate) ] where dbFiles = getDBFiles dbModel @@ -490,11 +472,10 @@ genBlock = do return testBlock generatorImpl :: Bool - -> Bool -> Model Symbolic -> Maybe (Gen (At CmdErr Symbolic)) -generatorImpl mkErr terminatingCmd m@Model {..} = do - genCmd <- generatorCmdImpl terminatingCmd m +generatorImpl mkErr m@Model {..} = do + genCmd <- generatorCmdImpl m Just $ do At cmd <- genCmd err <- if not (allowErrorFor cmd) then return Nothing @@ -598,14 +579,13 @@ runDB db cmd env@Internal.VolatileDBEnv{..} = case cmd of CreateFile -> do createFile env withClosedDB $ return () - CreateInvalidFile -> + CreateInvalidFile n -> withClosedDB $ - withFile _dbHasFS (mkFsPath ["invalidFileName.dat"]) - (AppendMode MustBeNew) $ \_hndl -> return () + withFile _dbHasFS (mkFsPath ["invalid-" ++ show n ++ "-file.dat" ]) + (AppendMode AllowExisting) $ \_hndl -> return () DuplicateBlock _file tb -> do _ <- withDBState $ \hasFS st -> - hPut hasFS (Internal._currentWriteHandle st) - (testBlockToBuilder tb) + hPut hasFS (Internal._currentWriteHandle st) (testBlockToBuilder tb) withClosedDB $ return () where @@ -650,7 +630,7 @@ prop_sequential = ValidateAll (db, env) <- run $ Internal.openDBFull hasFS EH.monadCatch ec parser nullTracer 3 - let sm' = sm True errorsVar db env dbm + let sm' = sm errorsVar db env dbm (hist, _model, res) <- runCommands sm' cmds run $ closeDB db return (hist, res) @@ -718,7 +698,6 @@ tag ls = C.classify , tagAppendRecover , tagIsClosedError , tagGarbageCollectThenReOpen - , tagImpossibleToRecover ] ls where @@ -802,12 +781,6 @@ tag ls = C.classify Left TagGarbageCollectThenReOpen _ -> Right $ tagGarbageCollectThenReOpen - tagImpossibleToRecover :: EventPred - tagImpossibleToRecover = C.predicate $ \ev -> - if shouldEnd (eventBefore ev) || shouldEnd (eventAfter ev) - then Left TagImpossibleToRecover - else Right tagImpossibleToRecover - getCmd :: Event r -> Cmd getCmd ev = cmd $ unAt (eventCmd ev) @@ -887,8 +860,6 @@ data Tag = -- > ReOpen | TagGarbageCollectThenReOpen - -- | Command which irreversibly corrupt the db. - | TagImpossibleToRecover deriving Show tagSimulatedErrors :: [Event Symbolic] -> [String] @@ -921,7 +892,7 @@ tagGetPredecessor = mapMaybe f execCmd :: Model Symbolic -> Command (At CmdErr) (At Resp) -> Event Symbolic -execCmd model (Command cmdErr resp _vars) = lockstep model cmdErr resp +execCmd model (Command cmdErr _resp _vars) = lockstep model cmdErr execCmds :: Model Symbolic -> Commands (At CmdErr) (At Resp) -> [Event Symbolic] execCmds model (Commands cs) = go model cs