Skip to content

Commit

Permalink
Allow skipping snapshot checksum check
Browse files Browse the repository at this point in the history
  • Loading branch information
geo2a committed Nov 29, 2024
1 parent 7c8d483 commit 5d39721
Show file tree
Hide file tree
Showing 8 changed files with 101 additions and 29 deletions.
12 changes: 9 additions & 3 deletions ouroboros-consensus-cardano/app/DBAnalyser/Parsers.hs
Original file line number Diff line number Diff line change
@@ -1,6 +1,7 @@
{-# LANGUAGE ApplicativeDo #-}
{-# LANGUAGE CPP #-}
{-# LANGUAGE LambdaCase #-}
{-# LANGUAGE ApplicativeDo #-}
{-# LANGUAGE CPP #-}
{-# LANGUAGE LambdaCase #-}
{-# LANGUAGE PatternSynonyms #-}

module DBAnalyser.Parsers (
BlockType (..)
Expand All @@ -21,6 +22,7 @@ import Options.Applicative
import Ouroboros.Consensus.Block
import Ouroboros.Consensus.Byron.Node (PBftSignatureThreshold (..))
import Ouroboros.Consensus.Shelley.Node (Nonce (..))
import Ouroboros.Consensus.Storage.LedgerDB.Snapshots (pattern DiskSnapshotChecksum, pattern NoDiskSnapshotChecksum)

{-------------------------------------------------------------------------------
Parsing
Expand All @@ -44,6 +46,10 @@ parseDBAnalyserConfig = DBAnalyserConfig
<*> parseValidationPolicy
<*> parseAnalysis
<*> parseLimit
<*> flag NoDiskSnapshotChecksum DiskSnapshotChecksum (mconcat [
long "disk-snapshot-checksum"
, help "Check the '.checksum' file if reading a ledger snapshot"
])

parseSelectDB :: Parser SelectDB
parseSelectDB =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2,6 +2,7 @@
{-# LANGUAGE GADTs #-}
{-# LANGUAGE LambdaCase #-}
{-# LANGUAGE NamedFieldPuns #-}
{-# LANGUAGE PatternSynonyms #-}
{-# LANGUAGE RankNTypes #-}
{-# LANGUAGE ScopedTypeVariables #-}

Expand Down Expand Up @@ -51,7 +52,7 @@ analyse ::
=> DBAnalyserConfig
-> Args blk
-> IO (Maybe AnalysisResult)
analyse DBAnalyserConfig{analysis, confLimit, dbDir, selectDB, validation, verbose} args =
analyse DBAnalyserConfig{analysis, confLimit, dbDir, selectDB, validation, verbose, diskSnapshotChecksum} args =
withRegistry $ \registry -> do
lock <- newMVar ()
chainDBTracer <- mkTracer lock verbose
Expand Down Expand Up @@ -92,6 +93,7 @@ analyse DBAnalyserConfig{analysis, confLimit, dbDir, selectDB, validation, verbo
ledgerDbFS
(decodeDiskExtLedgerState $ configCodec cfg)
decode
diskSnapshotChecksum
(DiskSnapshot slot (Just "db-analyser"))
-- TODO @readSnapshot@ has type @ExceptT ReadIncrementalErr m
-- (ExtLedgerState blk)@ but it also throws exceptions! This makes
Expand Down
Original file line number Diff line number Diff line change
@@ -1,20 +1,23 @@
{-# LANGUAGE DataKinds #-}
{-# LANGUAGE GeneralizedNewtypeDeriving #-}

module Cardano.Tools.DBAnalyser.Types (module Cardano.Tools.DBAnalyser.Types) where

import Data.Word
import Ouroboros.Consensus.Block
import Ouroboros.Consensus.Util (Flag)

data SelectDB =
SelectImmutableDB (WithOrigin SlotNo)

data DBAnalyserConfig = DBAnalyserConfig {
dbDir :: FilePath
, verbose :: Bool
, selectDB :: SelectDB
, validation :: Maybe ValidateBlocks
, analysis :: AnalysisName
, confLimit :: Limit
dbDir :: FilePath
, verbose :: Bool
, selectDB :: SelectDB
, validation :: Maybe ValidateBlocks
, analysis :: AnalysisName
, confLimit :: Limit
, diskSnapshotChecksum :: Flag "DiskSnapshotChecksum"
}

data AnalysisName =
Expand Down
5 changes: 5 additions & 0 deletions ouroboros-consensus-cardano/test/tools-test/Main.hs
Original file line number Diff line number Diff line change
@@ -1,3 +1,5 @@
{-# LANGUAGE PatternSynonyms #-}

module Main (main) where

import qualified Cardano.Tools.DBAnalyser.Block.Cardano as Cardano
Expand All @@ -8,6 +10,8 @@ import qualified Cardano.Tools.DBSynthesizer.Run as DBSynthesizer
import Cardano.Tools.DBSynthesizer.Types
import Ouroboros.Consensus.Block
import Ouroboros.Consensus.Cardano.Block
import Ouroboros.Consensus.Storage.LedgerDB.Snapshots
(pattern NoDiskSnapshotChecksum)
import qualified Test.Cardano.Tools.Headers
import Test.Tasty
import Test.Tasty.HUnit
Expand Down Expand Up @@ -68,6 +72,7 @@ testAnalyserConfig =
, validation = Just ValidateAllBlocks
, analysis = CountBlocks
, confLimit = Unlimited
, diskSnapshotChecksum = NoDiskSnapshotChecksum
}

testBlockArgs :: Cardano.Args (CardanoBlock StandardCrypto)
Expand Down
Original file line number Diff line number Diff line change
@@ -1,3 +1,5 @@
{-# LANGUAGE PatternSynonyms #-}

-- | The Ledger DB is responsible for the following tasks:
--
-- - __Maintaining the in-memory ledger state at the tip__: When we try to
Expand Down Expand Up @@ -135,6 +137,8 @@ module Ouroboros.Consensus.Storage.LedgerDB (
, diskSnapshotIsTemporary
, listSnapshots
, readSnapshot
, pattern DiskSnapshotChecksum
, pattern NoDiskSnapshotChecksum
-- ** Write to disk
, takeSnapshot
, trimSnapshots
Expand Down Expand Up @@ -174,8 +178,10 @@ import Ouroboros.Consensus.Storage.LedgerDB.Snapshots
(DiskSnapshot (..), SnapshotFailure (..),
TraceSnapshotEvent (..), decodeSnapshotBackwardsCompatible,
deleteSnapshot, diskSnapshotIsTemporary, encodeSnapshot,
listSnapshots, readSnapshot, snapshotToFileName,
snapshotToPath, takeSnapshot, trimSnapshots, writeSnapshot)
listSnapshots, readSnapshot,
pattern DiskSnapshotChecksum, pattern NoDiskSnapshotChecksum,
snapshotToFileName, snapshotToPath, takeSnapshot, trimSnapshots,
writeSnapshot)
import Ouroboros.Consensus.Storage.LedgerDB.Update
(AnnLedgerError (..), AnnLedgerError', Ap (..),
ExceededRollback (..), PushGoal (..), PushStart (..),
Expand Down
Original file line number Diff line number Diff line change
@@ -1,4 +1,5 @@
{-# LANGUAGE BangPatterns #-}
{-# LANGUAGE DataKinds #-}
{-# LANGUAGE DeriveGeneric #-}
{-# LANGUAGE RankNTypes #-}
{-# LANGUAGE ScopedTypeVariables #-}
Expand Down Expand Up @@ -35,6 +36,7 @@ import Ouroboros.Consensus.Storage.LedgerDB.LedgerDB
import Ouroboros.Consensus.Storage.LedgerDB.Query
import Ouroboros.Consensus.Storage.LedgerDB.Snapshots
import Ouroboros.Consensus.Storage.LedgerDB.Update
import Ouroboros.Consensus.Util (Flag)
import Ouroboros.Consensus.Util.IOLike
import Ouroboros.Network.Block (Point (Point))
import System.FS.API
Expand Down Expand Up @@ -112,12 +114,13 @@ initLedgerDB replayTracer
getGenesisLedger
stream = do
snapshots <- listSnapshots hasFS
tryNewestFirst id snapshots
tryNewestFirst DiskSnapshotChecksum id snapshots
where
tryNewestFirst :: (InitLog blk -> InitLog blk)
tryNewestFirst :: Flag "DiskSnapshotChecksum"
-> (InitLog blk -> InitLog blk)
-> [DiskSnapshot]
-> m (InitLog blk, LedgerDB' blk, Word64)
tryNewestFirst acc [] = do
tryNewestFirst _ acc [] = do
-- We're out of snapshots. Start at genesis
traceWith replayTracer ReplayFromGenesis
initDb <- ledgerDbWithAnchor <$> getGenesisLedger
Expand All @@ -126,8 +129,7 @@ initLedgerDB replayTracer
case ml of
Left _ -> error "invariant violation: invalid current chain"
Right (l, replayed) -> return (acc InitFromGenesis, l, replayed)
tryNewestFirst acc (s:ss) = do
-- If we fail to use this snapshot, delete it and try an older one
tryNewestFirst doChecksum acc allSnapshot@(s:ss) = do
ml <- runExceptT $ initFromSnapshot
replayTracer
hasFS
Expand All @@ -136,14 +138,23 @@ initLedgerDB replayTracer
cfg
stream
s
doChecksum
case ml of
-- If a checksum file is missing for a snapshot,
-- issue a warning and retry the same snapshot
-- ignoring the checksum
Left (InitFailureRead ReadSnapshotNoChecksumFile{}) -> do
traceWith tracer $ SnapshotMissingChecksum s
tryNewestFirst NoDiskSnapshotChecksum acc allSnapshot
-- If we fail to use this snapshot for any other reason, delete it and try an older one
Left err -> do
when (diskSnapshotIsTemporary s) $
-- We don't delete permanent snapshots, even if we couldn't parse
-- them
deleteSnapshot hasFS s
traceWith tracer $ InvalidSnapshot s err
tryNewestFirst (acc . InitFailure s err) ss
-- always reset checksum flag after failure
tryNewestFirst DiskSnapshotChecksum (acc . InitFailure s err) ss
Right (r, l, replayed) ->
return (acc (InitFromSnapshot s r), l, replayed)

Expand All @@ -170,10 +181,11 @@ initFromSnapshot ::
-> LedgerDbCfg (ExtLedgerState blk)
-> StreamAPI m blk blk
-> DiskSnapshot
-> Flag "DiskSnapshotChecksum"
-> ExceptT (SnapshotFailure blk) m (RealPoint blk, LedgerDB' blk, Word64)
initFromSnapshot tracer hasFS decLedger decHash cfg stream ss = do
initFromSnapshot tracer hasFS decLedger decHash cfg stream ss doChecksum = do
initSS <- withExceptT InitFailureRead $
readSnapshot hasFS decLedger decHash ss
readSnapshot hasFS decLedger decHash doChecksum ss
let replayStart = castPoint $ getTip initSS
case pointToWithOriginRealPoint replayStart of
Origin -> throwError InitFailureGenesis
Expand Down
Original file line number Diff line number Diff line change
@@ -1,11 +1,12 @@
{-# LANGUAGE BangPatterns #-}
{-# LANGUAGE DataKinds #-}
{-# LANGUAGE DeriveGeneric #-}
{-# LANGUAGE FlexibleContexts #-}
{-# LANGUAGE FlexibleInstances #-}
{-# LANGUAGE KindSignatures #-}
{-# LANGUAGE LambdaCase #-}
{-# LANGUAGE MultiParamTypeClasses #-}
{-# LANGUAGE NamedFieldPuns #-}
{-# LANGUAGE PatternSynonyms #-}
{-# LANGUAGE RankNTypes #-}
{-# LANGUAGE RecordWildCards #-}
{-# LANGUAGE ScopedTypeVariables #-}
Expand All @@ -14,9 +15,12 @@
module Ouroboros.Consensus.Storage.LedgerDB.Snapshots (
DiskSnapshot (..)
-- * Read from disk
, ReadSnapshotErr (..)
, SnapshotFailure (..)
, diskSnapshotIsTemporary
, listSnapshots
, pattern DiskSnapshotChecksum
, pattern NoDiskSnapshotChecksum
, readSnapshot
-- * Write to disk
, takeSnapshot
Expand Down Expand Up @@ -44,6 +48,7 @@ import qualified Data.ByteString.Builder as BS
import qualified Data.ByteString.Char8 as BSC
import qualified Data.ByteString.Lazy as BSL
import Data.Char (ord)
import Data.Coerce (coerce)
import Data.Functor.Contravariant ((>$<))
import qualified Data.List as List
import Data.Maybe (isJust, mapMaybe)
Expand All @@ -57,6 +62,7 @@ import Ouroboros.Consensus.Block
import Ouroboros.Consensus.Ledger.Abstract
import Ouroboros.Consensus.Ledger.Extended
import Ouroboros.Consensus.Storage.LedgerDB.DiskPolicy
import Ouroboros.Consensus.Util (Flag (..))
import Ouroboros.Consensus.Util.CBOR (ReadIncrementalErr,
decodeWithOrigin, readIncremental)
import Ouroboros.Consensus.Util.Enclose
Expand Down Expand Up @@ -90,7 +96,9 @@ data TraceSnapshotEvent blk
| TookSnapshot DiskSnapshot (RealPoint blk) EnclosingTimed
-- ^ A snapshot was written to disk.
| DeletedSnapshot DiskSnapshot
-- ^ An old or invalid on-disk snapshot was deleted
-- ^ An old or invalid on-disk snapshot was deleted.
| SnapshotMissingChecksum DiskSnapshot
-- ^ The checksum file for a snapshot was missing and was not checked
deriving (Generic, Eq, Show)

-- | Take a snapshot of the /oldest ledger state/ in the ledger DB
Expand Down Expand Up @@ -206,6 +214,10 @@ data ReadSnapshotErr =
| ReadSnapshotInvalidChecksumFile FsPath
deriving (Eq, Show)

pattern DiskSnapshotChecksum, NoDiskSnapshotChecksum :: Flag "DiskSnapshotChecksum"
pattern DiskSnapshotChecksum = Flag True
pattern NoDiskSnapshotChecksum = Flag False

-- | Read snapshot from disk.
--
-- Fail on data corruption, i.e. when the checksum of the read data differs
Expand All @@ -215,15 +227,23 @@ readSnapshot ::
=> SomeHasFS m
-> (forall s. Decoder s (ExtLedgerState blk))
-> (forall s. Decoder s (HeaderHash blk))
-> Flag "DiskSnapshotChecksum"
-> DiskSnapshot
-> ExceptT ReadSnapshotErr m (ExtLedgerState blk)
readSnapshot someHasFS decLedger decHash snapshotName = do
!snapshotCRC <- readCRC someHasFS (snapshotToChecksumPath snapshotName)
(ledgerState, checksumAsRead) <- withExceptT ReadSnapshotFailed . ExceptT $
readIncremental someHasFS decoder (snapshotToPath snapshotName)
if checksumAsRead /= snapshotCRC
then throwError ReadSnapshotDataCorruption
else pure ledgerState
readSnapshot someHasFS decLedger decHash doChecksum snapshotName = do
if coerce doChecksum then do
!snapshotCRC <- readCRC someHasFS (snapshotToChecksumPath snapshotName)
(ledgerState, checksumAsRead) <- withExceptT ReadSnapshotFailed . ExceptT $
readIncremental someHasFS decoder (snapshotToPath snapshotName)
if checksumAsRead /= snapshotCRC
then throwError ReadSnapshotDataCorruption
else pure ledgerState
else do
-- TODO: consider threading the doChecksum flag into readIncremental
-- to skip the checksum computation
(ledgerState, _) <- withExceptT ReadSnapshotFailed . ExceptT $
readIncremental someHasFS decoder (snapshotToPath snapshotName)
pure ledgerState
where
decoder :: Decoder s (ExtLedgerState blk)
decoder = decodeSnapshotBackwardsCompatible (Proxy @blk) decLedger decHash
Expand Down
Original file line number Diff line number Diff line change
@@ -1,5 +1,6 @@
{-# LANGUAGE BangPatterns #-}
{-# LANGUAGE ConstraintKinds #-}
{-# LANGUAGE DataKinds #-}
{-# LANGUAGE DeriveAnyClass #-}
{-# LANGUAGE DeriveGeneric #-}
{-# LANGUAGE DerivingStrategies #-}
Expand Down Expand Up @@ -77,6 +78,8 @@ module Ouroboros.Consensus.Util (
, electric
, newFuse
, withFuse
-- * Type-safe boolean flags
, Flag (..)
) where

import Cardano.Crypto.Hash (Hash, HashAlgorithm, hashFromBytes,
Expand All @@ -102,6 +105,7 @@ import Data.Void
import Data.Word (Word64)
import GHC.Generics (Generic)
import GHC.Stack
import GHC.TypeLits (Symbol)
import Ouroboros.Consensus.Util.IOLike
import Ouroboros.Network.Protocol.LocalStateQuery.Codec (Some (..))
import Ouroboros.Network.Util.ShowProxy (ShowProxy (..))
Expand Down Expand Up @@ -450,3 +454,17 @@ withFuse (Fuse name m) (Electric io) = do
newtype FuseBlownException = FuseBlownException Text
deriving (Show)
deriving anyclass (Exception)

{-------------------------------------------------------------------------------
Type-safe boolean flags
-------------------------------------------------------------------------------}

-- | Type-safe boolean flags with type level tags
--
-- It is recommended to create pattern synonyms for the true and false values.
-- Use 'coerce' to unwrap for use in e.g. if-statements.
--
-- See 'Ouroboros.Consensus.Storage.LedgerDB.Snapshots.DiskSnapshotChecksum'
-- for an example.
newtype Flag (name :: Symbol) = Flag Bool
deriving (Eq, Show, Generic)

0 comments on commit 5d39721

Please sign in to comment.