Skip to content

Commit

Permalink
Extend VolatileDB recovery
Browse files Browse the repository at this point in the history
  • Loading branch information
kderme committed Feb 12, 2020
1 parent 2c8df88 commit 1abf132
Show file tree
Hide file tree
Showing 9 changed files with 285 additions and 191 deletions.
126 changes: 126 additions & 0 deletions 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
1 change: 0 additions & 1 deletion ouroboros-consensus/src/Ouroboros/Storage/ChainDB/API.hs
Expand Up @@ -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
Expand Down
20 changes: 9 additions & 11 deletions ouroboros-consensus/src/Ouroboros/Storage/ChainDB/Impl/VolDB.hs
Expand Up @@ -30,7 +30,7 @@ module Ouroboros.Storage.ChainDB.Impl.VolDB (
, computePath
, computePathSTM
-- * Getting and parsing blocks
, BlockFileParserError (..)
, BlockFileParserError
, getKnownBlock
, getKnownHeader
, getKnownBlockComponent
Expand Down Expand Up @@ -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
Expand Down Expand Up @@ -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{..} =
Expand All @@ -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 =
Expand All @@ -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
Expand Down
85 changes: 58 additions & 27 deletions ouroboros-consensus/src/Ouroboros/Storage/VolatileDB/Impl.hs
Expand Up @@ -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
Expand All @@ -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)
Expand Down Expand Up @@ -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
Expand All @@ -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
Expand Down Expand Up @@ -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
Expand Down Expand Up @@ -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
Expand All @@ -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
Expand Down Expand Up @@ -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
Expand Down

0 comments on commit 1abf132

Please sign in to comment.