{-# LANGUAGE ConstraintKinds #-}
{-# LANGUAGE DeriveAnyClass #-}
{-# LANGUAGE DeriveGeneric #-}
{-# LANGUAGE FlexibleContexts #-}
{-# LANGUAGE LambdaCase #-}
{-# LANGUAGE NamedFieldPuns #-}
{-# LANGUAGE RecordWildCards #-}
{-# LANGUAGE ScopedTypeVariables #-}
{-# LANGUAGE StandaloneDeriving #-}
{-# LANGUAGE TypeApplications #-}
{-# LANGUAGE TypeFamilies #-}
{-# LANGUAGE TypeOperators #-}

-- | Thin wrapper around the LedgerDB
module Ouroboros.Consensus.Storage.ChainDB.Impl.LgrDB (
    LgrDB
    -- opaque
  , LedgerDB'
  , LgrDbSerialiseConstraints
    -- * Initialization
  , LgrDbArgs (..)
  , defaultArgs
  , openDB
    -- * 'TraceReplayEvent' decorator
  , LedgerDB.decorateReplayTracerWithGoal
    -- * Wrappers
  , currentPoint
  , getCurrent
  , getDiskPolicy
  , setCurrent
  , takeSnapshot
  , trimSnapshots
    -- * Validation
  , ValidateResult (..)
  , validate
    -- * Previously applied blocks
  , garbageCollectPrevApplied
  , getPrevApplied
    -- * Re-exports
  , LedgerDB.AnnLedgerError (..)
  , LedgerDB.DiskPolicy (..)
  , LedgerDB.DiskSnapshot
  , LedgerDB.ExceededRollback (..)
  , LedgerDB.TraceReplayEvent (..)
  , LedgerDB.TraceSnapshotEvent (..)
  , LedgerDB.ledgerDbCurrent
    -- * Exported for testing purposes
  , mkLgrDB
  ) where

import           Codec.Serialise (Serialise (decode))
import           Control.Monad.Trans.Class
import           Control.Tracer
import           Data.Foldable as Foldable (foldl')
import           Data.Set (Set)
import qualified Data.Set as Set
import           Data.Word (Word64)
import           GHC.Generics (Generic)
import           GHC.Stack (HasCallStack)
import           Ouroboros.Consensus.Block
import           Ouroboros.Consensus.Config
import           Ouroboros.Consensus.HeaderValidation
import           Ouroboros.Consensus.Ledger.Abstract
import           Ouroboros.Consensus.Ledger.Extended
import           Ouroboros.Consensus.Ledger.Inspect
import           Ouroboros.Consensus.Ledger.SupportsProtocol
import           Ouroboros.Consensus.Protocol.Abstract
import           Ouroboros.Consensus.Storage.ChainDB.API (ChainDbFailure (..))
import           Ouroboros.Consensus.Storage.ChainDB.Impl.BlockCache
                     (BlockCache)
import qualified Ouroboros.Consensus.Storage.ChainDB.Impl.BlockCache as BlockCache
import           Ouroboros.Consensus.Storage.ImmutableDB (ImmutableDB)
import           Ouroboros.Consensus.Storage.ImmutableDB.Stream
import           Ouroboros.Consensus.Storage.LedgerDB (LedgerDB')
import qualified Ouroboros.Consensus.Storage.LedgerDB as LedgerDB
import           Ouroboros.Consensus.Storage.Serialisation
import           Ouroboros.Consensus.Util.Args
import           Ouroboros.Consensus.Util.IOLike
import           System.FS.API (SomeHasFS (..), createDirectoryIfMissing)
import           System.FS.API.Types (FsError, mkFsPath)

-- | Thin wrapper around the ledger database
data LgrDB m blk = LgrDB {
      forall (m :: * -> *) blk.
LgrDB m blk -> StrictTVar m (LedgerDB' blk)
varDB          :: !(StrictTVar m (LedgerDB' blk))
      -- ^ INVARIANT: the tip of the 'LedgerDB' is always in sync with the tip
      -- of the current chain of the ChainDB.
    , forall (m :: * -> *) blk.
LgrDB m blk -> StrictTVar m (Set (RealPoint blk))
varPrevApplied :: !(StrictTVar m (Set (RealPoint blk)))
      -- ^ INVARIANT: this set contains only points that are in the
      -- VolatileDB.
      --
      -- INVARIANT: all points on the current chain fragment are in this set.
      --
      -- The VolatileDB might contain invalid blocks, these will not be in
      -- this set.
      --
      -- When a garbage-collection is performed on the VolatileDB, the points
      -- of the blocks eligible for garbage-collection should be removed from
      -- this set.
    , forall (m :: * -> *) blk. LgrDB m blk -> RealPoint blk -> m blk
resolveBlock   :: !(RealPoint blk -> m blk)
      -- ^ Read a block from disk
    , forall (m :: * -> *) blk.
LgrDB m blk -> LedgerDbCfg (ExtLedgerState blk)
cfg            :: !(LedgerDB.LedgerDbCfg (ExtLedgerState blk))
    , forall (m :: * -> *) blk. LgrDB m blk -> DiskPolicy
diskPolicy     :: !LedgerDB.DiskPolicy
    , forall (m :: * -> *) blk. LgrDB m blk -> SomeHasFS m
hasFS          :: !(SomeHasFS m)
    , forall (m :: * -> *) blk.
LgrDB m blk -> Tracer m (TraceSnapshotEvent blk)
tracer         :: !(Tracer m (LedgerDB.TraceSnapshotEvent blk))
    } deriving ((forall x. LgrDB m blk -> Rep (LgrDB m blk) x)
-> (forall x. Rep (LgrDB m blk) x -> LgrDB m blk)
-> Generic (LgrDB m blk)
forall x. Rep (LgrDB m blk) x -> LgrDB m blk
forall x. LgrDB m blk -> Rep (LgrDB m blk) x
forall a.
(forall x. a -> Rep a x) -> (forall x. Rep a x -> a) -> Generic a
forall (m :: * -> *) blk x. Rep (LgrDB m blk) x -> LgrDB m blk
forall (m :: * -> *) blk x. LgrDB m blk -> Rep (LgrDB m blk) x
$cfrom :: forall (m :: * -> *) blk x. LgrDB m blk -> Rep (LgrDB m blk) x
from :: forall x. LgrDB m blk -> Rep (LgrDB m blk) x
$cto :: forall (m :: * -> *) blk x. Rep (LgrDB m blk) x -> LgrDB m blk
to :: forall x. Rep (LgrDB m blk) x -> LgrDB m blk
Generic)

deriving instance (IOLike m, LedgerSupportsProtocol blk)
               => NoThunks (LgrDB m blk)
  -- use generic instance

-- | 'EncodeDisk' and 'DecodeDisk' constraints needed for the LgrDB.
type LgrDbSerialiseConstraints blk =
  ( Serialise      (HeaderHash  blk)
  , EncodeDisk blk (LedgerState blk)
  , DecodeDisk blk (LedgerState blk)
  , EncodeDisk blk (AnnTip      blk)
  , DecodeDisk blk (AnnTip      blk)
  , EncodeDisk blk (ChainDepState (BlockProtocol blk))
  , DecodeDisk blk (ChainDepState (BlockProtocol blk))
  )

{-------------------------------------------------------------------------------
  Initialization
-------------------------------------------------------------------------------}

data LgrDbArgs f m blk = LgrDbArgs {
      forall (f :: * -> *) (m :: * -> *) blk.
LgrDbArgs f m blk -> DiskPolicyArgs
lgrDiskPolicyArgs :: LedgerDB.DiskPolicyArgs
    , forall (f :: * -> *) (m :: * -> *) blk.
LgrDbArgs f m blk -> HKD f (m (ExtLedgerState blk))
lgrGenesis        :: HKD f (m (ExtLedgerState blk))
    , forall (f :: * -> *) (m :: * -> *) blk.
LgrDbArgs f m blk -> HKD f (SomeHasFS m)
lgrHasFS          :: HKD f (SomeHasFS m)
    , forall (f :: * -> *) (m :: * -> *) blk.
LgrDbArgs f m blk -> HKD f (LedgerDbCfg (ExtLedgerState blk))
lgrConfig         :: HKD f (LedgerDB.LedgerDbCfg (ExtLedgerState blk))
    , forall (f :: * -> *) (m :: * -> *) blk.
LgrDbArgs f m blk -> Tracer m (TraceSnapshotEvent blk)
lgrTracer         :: Tracer m (LedgerDB.TraceSnapshotEvent blk)
    }

-- | Default arguments
defaultArgs :: Applicative m => Incomplete LgrDbArgs m blk
defaultArgs :: forall (m :: * -> *) blk.
Applicative m =>
Incomplete LgrDbArgs m blk
defaultArgs = LgrDbArgs {
      lgrDiskPolicyArgs :: DiskPolicyArgs
lgrDiskPolicyArgs = DiskPolicyArgs
LedgerDB.defaultDiskPolicyArgs
    , lgrGenesis :: HKD Defaults (m (ExtLedgerState blk))
lgrGenesis        = HKD Defaults (m (ExtLedgerState blk))
Defaults (m (ExtLedgerState blk))
forall {k} (t :: k). Defaults t
noDefault
    , lgrHasFS :: HKD Defaults (SomeHasFS m)
lgrHasFS          = HKD Defaults (SomeHasFS m)
Defaults (SomeHasFS m)
forall {k} (t :: k). Defaults t
noDefault
    , lgrConfig :: HKD Defaults (LedgerDbCfg (ExtLedgerState blk))
lgrConfig         = HKD Defaults (LedgerDbCfg (ExtLedgerState blk))
Defaults (LedgerDbCfg (ExtLedgerState blk))
forall {k} (t :: k). Defaults t
noDefault
    , lgrTracer :: Tracer m (TraceSnapshotEvent blk)
lgrTracer         = Tracer m (TraceSnapshotEvent blk)
forall (m :: * -> *) a. Applicative m => Tracer m a
nullTracer
    }

-- | Open the ledger DB
--
-- In addition to the ledger DB also returns the number of immutable blocks
-- that were replayed.
openDB :: forall m blk.
          ( IOLike m
          , LedgerSupportsProtocol blk
          , LgrDbSerialiseConstraints blk
          , InspectLedger blk
          , HasCallStack
          )
       => Complete LgrDbArgs m blk
       -- ^ Stateless initializaton arguments
       -> Tracer m (LedgerDB.ReplayGoal blk -> LedgerDB.TraceReplayEvent blk)
       -- ^ Used to trace the progress while replaying blocks against the
       -- ledger.
       -> ImmutableDB m blk
       -- ^ Reference to the immutable DB
       --
       -- After reading a snapshot from disk, the ledger DB will be brought
       -- up to date with tip of the immutable DB. The corresponding ledger
       -- state can then be used as the starting point for chain selection in
       -- the ChainDB driver.
       -> (RealPoint blk -> m blk)
       -- ^ Read a block from disk
       --
       -- The block may be in the immutable DB or in the volatile DB; the ledger
       -- DB does not know where the boundary is at any given point.
       -> m (LgrDB m blk, Word64)
openDB :: forall (m :: * -> *) blk.
(IOLike m, LedgerSupportsProtocol blk,
 LgrDbSerialiseConstraints blk, InspectLedger blk, HasCallStack) =>
Complete LgrDbArgs m blk
-> Tracer m (ReplayGoal blk -> TraceReplayEvent blk)
-> ImmutableDB m blk
-> (RealPoint blk -> m blk)
-> m (LgrDB m blk, Word64)
openDB args :: Complete LgrDbArgs m blk
args@LgrDbArgs { lgrHasFS :: forall (f :: * -> *) (m :: * -> *) blk.
LgrDbArgs f m blk -> HKD f (SomeHasFS m)
lgrHasFS = lgrHasFS :: HKD Identity (SomeHasFS m)
lgrHasFS@(SomeHasFS HasFS m h
hasFS), Tracer m (TraceSnapshotEvent blk)
DiskPolicyArgs
HKD Identity (m (ExtLedgerState blk))
HKD Identity (LedgerDbCfg (ExtLedgerState blk))
lgrDiskPolicyArgs :: forall (f :: * -> *) (m :: * -> *) blk.
LgrDbArgs f m blk -> DiskPolicyArgs
lgrGenesis :: forall (f :: * -> *) (m :: * -> *) blk.
LgrDbArgs f m blk -> HKD f (m (ExtLedgerState blk))
lgrConfig :: forall (f :: * -> *) (m :: * -> *) blk.
LgrDbArgs f m blk -> HKD f (LedgerDbCfg (ExtLedgerState blk))
lgrTracer :: forall (f :: * -> *) (m :: * -> *) blk.
LgrDbArgs f m blk -> Tracer m (TraceSnapshotEvent blk)
lgrDiskPolicyArgs :: DiskPolicyArgs
lgrGenesis :: HKD Identity (m (ExtLedgerState blk))
lgrConfig :: HKD Identity (LedgerDbCfg (ExtLedgerState blk))
lgrTracer :: Tracer m (TraceSnapshotEvent blk)
.. } Tracer m (ReplayGoal blk -> TraceReplayEvent blk)
replayTracer ImmutableDB m blk
immutableDB RealPoint blk -> m blk
getBlock = do
    HasFS m h -> HasCallStack => Bool -> FsPath -> m ()
forall (m :: * -> *) h.
HasFS m h -> HasCallStack => Bool -> FsPath -> m ()
createDirectoryIfMissing HasFS m h
hasFS Bool
True (Context -> FsPath
mkFsPath [])
    (LedgerDB (ExtLedgerState blk)
db, Word64
replayed) <- Complete LgrDbArgs m blk
-> Tracer m (ReplayGoal blk -> TraceReplayEvent blk)
-> ImmutableDB m blk
-> m (LedgerDB (ExtLedgerState blk), Word64)
forall blk (m :: * -> *).
(IOLike m, LedgerSupportsProtocol blk,
 LgrDbSerialiseConstraints blk, InspectLedger blk, HasCallStack) =>
Complete LgrDbArgs m blk
-> Tracer m (ReplayGoal blk -> TraceReplayEvent blk)
-> ImmutableDB m blk
-> m (LedgerDB' blk, Word64)
initFromDisk Complete LgrDbArgs m blk
args Tracer m (ReplayGoal blk -> TraceReplayEvent blk)
replayTracer ImmutableDB m blk
immutableDB
    -- When initializing the ledger DB from disk we:
    --
    -- - Look for the newest valid snapshot, say 'Lbs', which corresponds to the
    --   application of a block in the immutable DB, say 'b'.
    --
    -- - Push onto the ledger DB all the ledger states that result from applying
    --   blocks found in the on-disk immutable DB, starting from the successor
    --   of 'b'.
    --
    -- The anchor of 'LedgerDB' must be the oldest point we can rollback to. So
    -- if we follow the procedure described above (that 'initFromDisk'
    -- implements), the newest ledger state in 'db', say 'Lbn' corresponds to
    -- the most recent block in the immutable DB. If this block is in the
    -- immutable DB, it means that at some point it was part of a chain that was
    -- >k blocks long. Thus 'Lbn' is the oldest point we can roll back to.
    -- Therefore, we need to make the newest state (current) of the ledger DB
    -- the anchor.
    let dbPrunedToImmDBTip :: LedgerDB (ExtLedgerState blk)
dbPrunedToImmDBTip = SecurityParam
-> LedgerDB (ExtLedgerState blk) -> LedgerDB (ExtLedgerState blk)
forall l. GetTip l => SecurityParam -> LedgerDB l -> LedgerDB l
LedgerDB.ledgerDbPrune (Word64 -> SecurityParam
SecurityParam Word64
0) LedgerDB (ExtLedgerState blk)
db
    (StrictTVar m (LedgerDB (ExtLedgerState blk))
varDB, StrictTVar m (Set (RealPoint blk))
varPrevApplied) <-
      (,) (StrictTVar m (LedgerDB (ExtLedgerState blk))
 -> StrictTVar m (Set (RealPoint blk))
 -> (StrictTVar m (LedgerDB (ExtLedgerState blk)),
     StrictTVar m (Set (RealPoint blk))))
-> m (StrictTVar m (LedgerDB (ExtLedgerState blk)))
-> m (StrictTVar m (Set (RealPoint blk))
      -> (StrictTVar m (LedgerDB (ExtLedgerState blk)),
          StrictTVar m (Set (RealPoint blk))))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> LedgerDB (ExtLedgerState blk)
-> m (StrictTVar m (LedgerDB (ExtLedgerState blk)))
forall (m :: * -> *) a.
(HasCallStack, MonadSTM m, NoThunks a) =>
a -> m (StrictTVar m a)
newTVarIO LedgerDB (ExtLedgerState blk)
dbPrunedToImmDBTip m (StrictTVar m (Set (RealPoint blk))
   -> (StrictTVar m (LedgerDB (ExtLedgerState blk)),
       StrictTVar m (Set (RealPoint blk))))
-> m (StrictTVar m (Set (RealPoint blk)))
-> m (StrictTVar m (LedgerDB (ExtLedgerState blk)),
      StrictTVar m (Set (RealPoint blk)))
forall a b. m (a -> b) -> m a -> m b
forall (f :: * -> *) a b. Applicative f => f (a -> b) -> f a -> f b
<*> Set (RealPoint blk) -> m (StrictTVar m (Set (RealPoint blk)))
forall (m :: * -> *) a.
(HasCallStack, MonadSTM m, NoThunks a) =>
a -> m (StrictTVar m a)
newTVarIO Set (RealPoint blk)
forall a. Set a
Set.empty
    (LgrDB m blk, Word64) -> m (LgrDB m blk, Word64)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (
        LgrDB {
            varDB :: StrictTVar m (LedgerDB (ExtLedgerState blk))
varDB          = StrictTVar m (LedgerDB (ExtLedgerState blk))
varDB
          , varPrevApplied :: StrictTVar m (Set (RealPoint blk))
varPrevApplied = StrictTVar m (Set (RealPoint blk))
varPrevApplied
          , resolveBlock :: RealPoint blk -> m blk
resolveBlock   = RealPoint blk -> m blk
getBlock
          , cfg :: LedgerDbCfg (ExtLedgerState blk)
cfg            = HKD Identity (LedgerDbCfg (ExtLedgerState blk))
LedgerDbCfg (ExtLedgerState blk)
lgrConfig
          , diskPolicy :: DiskPolicy
diskPolicy     = let k :: SecurityParam
k = LedgerDbCfg (ExtLedgerState blk) -> SecurityParam
forall l. LedgerDbCfg l -> SecurityParam
LedgerDB.ledgerDbCfgSecParam HKD Identity (LedgerDbCfg (ExtLedgerState blk))
LedgerDbCfg (ExtLedgerState blk)
lgrConfig
            in  SecurityParam -> DiskPolicyArgs -> DiskPolicy
LedgerDB.mkDiskPolicy SecurityParam
k DiskPolicyArgs
lgrDiskPolicyArgs
          , hasFS :: SomeHasFS m
hasFS          = SomeHasFS m
HKD Identity (SomeHasFS m)
lgrHasFS
          , tracer :: Tracer m (TraceSnapshotEvent blk)
tracer         = Tracer m (TraceSnapshotEvent blk)
lgrTracer
          }
      , Word64
replayed
      )

initFromDisk ::
     forall blk m.
     ( IOLike m
     , LedgerSupportsProtocol blk
     , LgrDbSerialiseConstraints blk
     , InspectLedger blk
     , HasCallStack
     )
  => Complete LgrDbArgs m blk
  -> Tracer m (LedgerDB.ReplayGoal blk -> LedgerDB.TraceReplayEvent blk)
  -> ImmutableDB m blk
  -> m (LedgerDB' blk, Word64)
initFromDisk :: forall blk (m :: * -> *).
(IOLike m, LedgerSupportsProtocol blk,
 LgrDbSerialiseConstraints blk, InspectLedger blk, HasCallStack) =>
Complete LgrDbArgs m blk
-> Tracer m (ReplayGoal blk -> TraceReplayEvent blk)
-> ImmutableDB m blk
-> m (LedgerDB' blk, Word64)
initFromDisk LgrDbArgs { lgrHasFS :: forall (f :: * -> *) (m :: * -> *) blk.
LgrDbArgs f m blk -> HKD f (SomeHasFS m)
lgrHasFS = HKD Identity (SomeHasFS m)
hasFS, Tracer m (TraceSnapshotEvent blk)
DiskPolicyArgs
HKD Identity (m (ExtLedgerState blk))
HKD Identity (LedgerDbCfg (ExtLedgerState blk))
lgrDiskPolicyArgs :: forall (f :: * -> *) (m :: * -> *) blk.
LgrDbArgs f m blk -> DiskPolicyArgs
lgrGenesis :: forall (f :: * -> *) (m :: * -> *) blk.
LgrDbArgs f m blk -> HKD f (m (ExtLedgerState blk))
lgrConfig :: forall (f :: * -> *) (m :: * -> *) blk.
LgrDbArgs f m blk -> HKD f (LedgerDbCfg (ExtLedgerState blk))
lgrTracer :: forall (f :: * -> *) (m :: * -> *) blk.
LgrDbArgs f m blk -> Tracer m (TraceSnapshotEvent blk)
lgrDiskPolicyArgs :: DiskPolicyArgs
lgrGenesis :: HKD Identity (m (ExtLedgerState blk))
lgrConfig :: HKD Identity (LedgerDbCfg (ExtLedgerState blk))
lgrTracer :: Tracer m (TraceSnapshotEvent blk)
.. }
             Tracer m (ReplayGoal blk -> TraceReplayEvent blk)
replayTracer
             ImmutableDB m blk
immutableDB = Proxy blk -> m (LedgerDB' blk, Word64) -> m (LedgerDB' blk, Word64)
forall (m :: * -> *) x blk.
(MonadCatch m, HasHeader blk) =>
Proxy blk -> m x -> m x
wrapFailure (forall t. Proxy t
forall {k} (t :: k). Proxy t
Proxy @blk) (m (LedgerDB' blk, Word64) -> m (LedgerDB' blk, Word64))
-> m (LedgerDB' blk, Word64) -> m (LedgerDB' blk, Word64)
forall a b. (a -> b) -> a -> b
$ do
    (InitLog blk
_initLog, LedgerDB' blk
db, Word64
replayed) <-
      Tracer m (ReplayGoal blk -> TraceReplayEvent blk)
-> Tracer m (TraceSnapshotEvent blk)
-> SomeHasFS m
-> (forall s. Decoder s (ExtLedgerState blk))
-> (forall s. Decoder s (HeaderHash blk))
-> LedgerDbCfg (ExtLedgerState blk)
-> m (ExtLedgerState blk)
-> StreamAPI m blk blk
-> m (InitLog blk, LedgerDB' blk, Word64)
forall (m :: * -> *) blk.
(IOLike m, LedgerSupportsProtocol blk, InspectLedger blk,
 HasCallStack) =>
Tracer m (ReplayGoal blk -> TraceReplayEvent blk)
-> Tracer m (TraceSnapshotEvent blk)
-> SomeHasFS m
-> (forall s. Decoder s (ExtLedgerState blk))
-> (forall s. Decoder s (HeaderHash blk))
-> LedgerDbCfg (ExtLedgerState blk)
-> m (ExtLedgerState blk)
-> StreamAPI m blk blk
-> m (InitLog blk, LedgerDB' blk, Word64)
LedgerDB.initLedgerDB
        Tracer m (ReplayGoal blk -> TraceReplayEvent blk)
replayTracer
        Tracer m (TraceSnapshotEvent blk)
lgrTracer
        SomeHasFS m
HKD Identity (SomeHasFS m)
hasFS
        (CodecConfig blk -> forall s. Decoder s (ExtLedgerState blk)
forall blk.
(DecodeDisk blk (LedgerState blk),
 DecodeDisk blk (ChainDepState (BlockProtocol blk)),
 DecodeDisk blk (AnnTip blk)) =>
CodecConfig blk -> forall s. Decoder s (ExtLedgerState blk)
decodeDiskExtLedgerState CodecConfig blk
ccfg)
        Decoder s (HeaderHash blk)
forall s. Decoder s (HeaderHash blk)
forall a s. Serialise a => Decoder s a
decode
        HKD Identity (LedgerDbCfg (ExtLedgerState blk))
LedgerDbCfg (ExtLedgerState blk)
lgrConfig
        m (ExtLedgerState blk)
HKD Identity (m (ExtLedgerState blk))
lgrGenesis
        (ImmutableDB m blk -> StreamAPI m blk blk
forall (m :: * -> *) blk.
(IOLike m, HasHeader blk) =>
ImmutableDB m blk -> StreamAPI m blk blk
streamAPI ImmutableDB m blk
immutableDB)
    (LedgerDB' blk, Word64) -> m (LedgerDB' blk, Word64)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (LedgerDB' blk
db, Word64
replayed)
  where
    ccfg :: CodecConfig blk
ccfg = TopLevelConfig blk -> CodecConfig blk
forall blk. TopLevelConfig blk -> CodecConfig blk
configCodec (TopLevelConfig blk -> CodecConfig blk)
-> TopLevelConfig blk -> CodecConfig blk
forall a b. (a -> b) -> a -> b
$ ExtLedgerCfg blk -> TopLevelConfig blk
forall blk. ExtLedgerCfg blk -> TopLevelConfig blk
getExtLedgerCfg (ExtLedgerCfg blk -> TopLevelConfig blk)
-> ExtLedgerCfg blk -> TopLevelConfig blk
forall a b. (a -> b) -> a -> b
$ LedgerDbCfg (ExtLedgerState blk) -> LedgerCfg (ExtLedgerState blk)
forall l. LedgerDbCfg l -> LedgerCfg l
LedgerDB.ledgerDbCfg HKD Identity (LedgerDbCfg (ExtLedgerState blk))
LedgerDbCfg (ExtLedgerState blk)
lgrConfig

-- | For testing purposes
mkLgrDB :: StrictTVar m (LedgerDB' blk)
        -> StrictTVar m (Set (RealPoint blk))
        -> (RealPoint blk -> m blk)
        -> Complete LgrDbArgs m blk
        -> SecurityParam
        -> LgrDB m blk
mkLgrDB :: forall (m :: * -> *) blk.
StrictTVar m (LedgerDB' blk)
-> StrictTVar m (Set (RealPoint blk))
-> (RealPoint blk -> m blk)
-> Complete LgrDbArgs m blk
-> SecurityParam
-> LgrDB m blk
mkLgrDB StrictTVar m (LedgerDB' blk)
varDB StrictTVar m (Set (RealPoint blk))
varPrevApplied RealPoint blk -> m blk
resolveBlock Complete LgrDbArgs m blk
args SecurityParam
k = LgrDB {Tracer m (TraceSnapshotEvent blk)
SomeHasFS m
StrictTVar m (Set (RealPoint blk))
StrictTVar m (LedgerDB' blk)
DiskPolicy
HKD Identity (SomeHasFS m)
HKD Identity (LedgerDbCfg (ExtLedgerState blk))
LedgerDbCfg (ExtLedgerState blk)
RealPoint blk -> m blk
varDB :: StrictTVar m (LedgerDB' blk)
varPrevApplied :: StrictTVar m (Set (RealPoint blk))
resolveBlock :: RealPoint blk -> m blk
cfg :: LedgerDbCfg (ExtLedgerState blk)
diskPolicy :: DiskPolicy
hasFS :: SomeHasFS m
tracer :: Tracer m (TraceSnapshotEvent blk)
varDB :: StrictTVar m (LedgerDB' blk)
varPrevApplied :: StrictTVar m (Set (RealPoint blk))
resolveBlock :: RealPoint blk -> m blk
cfg :: HKD Identity (LedgerDbCfg (ExtLedgerState blk))
hasFS :: HKD Identity (SomeHasFS m)
tracer :: Tracer m (TraceSnapshotEvent blk)
diskPolicy :: DiskPolicy
..}
  where
    LgrDbArgs {
        lgrConfig :: forall (f :: * -> *) (m :: * -> *) blk.
LgrDbArgs f m blk -> HKD f (LedgerDbCfg (ExtLedgerState blk))
lgrConfig         = HKD Identity (LedgerDbCfg (ExtLedgerState blk))
cfg
      , lgrDiskPolicyArgs :: forall (f :: * -> *) (m :: * -> *) blk.
LgrDbArgs f m blk -> DiskPolicyArgs
lgrDiskPolicyArgs = DiskPolicyArgs
diskPolicyArgs
      , lgrHasFS :: forall (f :: * -> *) (m :: * -> *) blk.
LgrDbArgs f m blk -> HKD f (SomeHasFS m)
lgrHasFS          = HKD Identity (SomeHasFS m)
hasFS
      , lgrTracer :: forall (f :: * -> *) (m :: * -> *) blk.
LgrDbArgs f m blk -> Tracer m (TraceSnapshotEvent blk)
lgrTracer         = Tracer m (TraceSnapshotEvent blk)
tracer
      } = Complete LgrDbArgs m blk
args
    diskPolicy :: DiskPolicy
diskPolicy = SecurityParam -> DiskPolicyArgs -> DiskPolicy
LedgerDB.mkDiskPolicy SecurityParam
k DiskPolicyArgs
diskPolicyArgs

{-------------------------------------------------------------------------------
  Wrappers
-------------------------------------------------------------------------------}

getCurrent :: IOLike m => LgrDB m blk -> STM m (LedgerDB' blk)
getCurrent :: forall (m :: * -> *) blk.
IOLike m =>
LgrDB m blk -> STM m (LedgerDB' blk)
getCurrent LgrDB{Tracer m (TraceSnapshotEvent blk)
SomeHasFS m
StrictTVar m (Set (RealPoint blk))
StrictTVar m (LedgerDB' blk)
DiskPolicy
LedgerDbCfg (ExtLedgerState blk)
RealPoint blk -> m blk
varDB :: forall (m :: * -> *) blk.
LgrDB m blk -> StrictTVar m (LedgerDB' blk)
varPrevApplied :: forall (m :: * -> *) blk.
LgrDB m blk -> StrictTVar m (Set (RealPoint blk))
resolveBlock :: forall (m :: * -> *) blk. LgrDB m blk -> RealPoint blk -> m blk
cfg :: forall (m :: * -> *) blk.
LgrDB m blk -> LedgerDbCfg (ExtLedgerState blk)
diskPolicy :: forall (m :: * -> *) blk. LgrDB m blk -> DiskPolicy
hasFS :: forall (m :: * -> *) blk. LgrDB m blk -> SomeHasFS m
tracer :: forall (m :: * -> *) blk.
LgrDB m blk -> Tracer m (TraceSnapshotEvent blk)
varDB :: StrictTVar m (LedgerDB' blk)
varPrevApplied :: StrictTVar m (Set (RealPoint blk))
resolveBlock :: RealPoint blk -> m blk
cfg :: LedgerDbCfg (ExtLedgerState blk)
diskPolicy :: DiskPolicy
hasFS :: SomeHasFS m
tracer :: Tracer m (TraceSnapshotEvent blk)
..} = StrictTVar m (LedgerDB' blk) -> STM m (LedgerDB' blk)
forall (m :: * -> *) a. MonadSTM m => StrictTVar m a -> STM m a
readTVar StrictTVar m (LedgerDB' blk)
varDB

-- | PRECONDITION: The new 'LedgerDB' must be the result of calling either
-- 'LedgerDB.ledgerDbSwitch' or 'LedgerDB.ledgerDbPushMany' on the current
-- 'LedgerDB'.
setCurrent :: IOLike m => LgrDB m blk -> LedgerDB' blk -> STM m ()
setCurrent :: forall (m :: * -> *) blk.
IOLike m =>
LgrDB m blk -> LedgerDB' blk -> STM m ()
setCurrent LgrDB{Tracer m (TraceSnapshotEvent blk)
SomeHasFS m
StrictTVar m (Set (RealPoint blk))
StrictTVar m (LedgerDB' blk)
DiskPolicy
LedgerDbCfg (ExtLedgerState blk)
RealPoint blk -> m blk
varDB :: forall (m :: * -> *) blk.
LgrDB m blk -> StrictTVar m (LedgerDB' blk)
varPrevApplied :: forall (m :: * -> *) blk.
LgrDB m blk -> StrictTVar m (Set (RealPoint blk))
resolveBlock :: forall (m :: * -> *) blk. LgrDB m blk -> RealPoint blk -> m blk
cfg :: forall (m :: * -> *) blk.
LgrDB m blk -> LedgerDbCfg (ExtLedgerState blk)
diskPolicy :: forall (m :: * -> *) blk. LgrDB m blk -> DiskPolicy
hasFS :: forall (m :: * -> *) blk. LgrDB m blk -> SomeHasFS m
tracer :: forall (m :: * -> *) blk.
LgrDB m blk -> Tracer m (TraceSnapshotEvent blk)
varDB :: StrictTVar m (LedgerDB' blk)
varPrevApplied :: StrictTVar m (Set (RealPoint blk))
resolveBlock :: RealPoint blk -> m blk
cfg :: LedgerDbCfg (ExtLedgerState blk)
diskPolicy :: DiskPolicy
hasFS :: SomeHasFS m
tracer :: Tracer m (TraceSnapshotEvent blk)
..} = StrictTVar m (LedgerDB' blk) -> LedgerDB' blk -> STM m ()
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
StrictTVar m a -> a -> STM m ()
writeTVar (StrictTVar m (LedgerDB' blk) -> LedgerDB' blk -> STM m ())
-> StrictTVar m (LedgerDB' blk) -> LedgerDB' blk -> STM m ()
forall a b. (a -> b) -> a -> b
$! StrictTVar m (LedgerDB' blk)
varDB

currentPoint :: forall blk. UpdateLedger blk => LedgerDB' blk -> Point blk
currentPoint :: forall blk. UpdateLedger blk => LedgerDB' blk -> Point blk
currentPoint = Point blk -> Point blk
forall {k1} {k2} (b :: k1) (b' :: k2).
Coercible (HeaderHash b) (HeaderHash b') =>
Point b -> Point b'
castPoint
             (Point blk -> Point blk)
-> (LedgerDB' blk -> Point blk) -> LedgerDB' blk -> Point blk
forall b c a. (b -> c) -> (a -> b) -> a -> c
. LedgerState blk -> Point blk
forall blk. UpdateLedger blk => LedgerState blk -> Point blk
ledgerTipPoint
             (LedgerState blk -> Point blk)
-> (LedgerDB' blk -> LedgerState blk) -> LedgerDB' blk -> Point blk
forall b c a. (b -> c) -> (a -> b) -> a -> c
. ExtLedgerState blk -> LedgerState blk
forall blk. ExtLedgerState blk -> LedgerState blk
ledgerState
             (ExtLedgerState blk -> LedgerState blk)
-> (LedgerDB' blk -> ExtLedgerState blk)
-> LedgerDB' blk
-> LedgerState blk
forall b c a. (b -> c) -> (a -> b) -> a -> c
. LedgerDB' blk -> ExtLedgerState blk
forall l. GetTip l => LedgerDB l -> l
LedgerDB.ledgerDbCurrent

takeSnapshot ::
     forall m blk.
     ( IOLike m
     , LgrDbSerialiseConstraints blk
     , HasHeader blk
     , IsLedger (LedgerState blk)
     )
  => LgrDB m blk -> m (Maybe (LedgerDB.DiskSnapshot, RealPoint blk))
takeSnapshot :: forall (m :: * -> *) blk.
(IOLike m, LgrDbSerialiseConstraints blk, HasHeader blk,
 IsLedger (LedgerState blk)) =>
LgrDB m blk -> m (Maybe (DiskSnapshot, RealPoint blk))
takeSnapshot lgrDB :: LgrDB m blk
lgrDB@LgrDB{ LedgerDbCfg (ExtLedgerState blk)
cfg :: forall (m :: * -> *) blk.
LgrDB m blk -> LedgerDbCfg (ExtLedgerState blk)
cfg :: LedgerDbCfg (ExtLedgerState blk)
cfg, Tracer m (TraceSnapshotEvent blk)
tracer :: forall (m :: * -> *) blk.
LgrDB m blk -> Tracer m (TraceSnapshotEvent blk)
tracer :: Tracer m (TraceSnapshotEvent blk)
tracer, SomeHasFS m
hasFS :: forall (m :: * -> *) blk. LgrDB m blk -> SomeHasFS m
hasFS :: SomeHasFS m
hasFS } = Proxy blk
-> m (Maybe (DiskSnapshot, RealPoint blk))
-> m (Maybe (DiskSnapshot, RealPoint blk))
forall (m :: * -> *) x blk.
(MonadCatch m, HasHeader blk) =>
Proxy blk -> m x -> m x
wrapFailure (forall t. Proxy t
forall {k} (t :: k). Proxy t
Proxy @blk) (m (Maybe (DiskSnapshot, RealPoint blk))
 -> m (Maybe (DiskSnapshot, RealPoint blk)))
-> m (Maybe (DiskSnapshot, RealPoint blk))
-> m (Maybe (DiskSnapshot, RealPoint blk))
forall a b. (a -> b) -> a -> b
$ do
    ExtLedgerState blk
ledgerDB <- LedgerDB (ExtLedgerState blk) -> ExtLedgerState blk
forall l. LedgerDB l -> l
LedgerDB.ledgerDbAnchor (LedgerDB (ExtLedgerState blk) -> ExtLedgerState blk)
-> m (LedgerDB (ExtLedgerState blk)) -> m (ExtLedgerState blk)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> STM m (LedgerDB (ExtLedgerState blk))
-> m (LedgerDB (ExtLedgerState blk))
forall a. HasCallStack => STM m a -> m a
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
STM m a -> m a
atomically (LgrDB m blk -> STM m (LedgerDB (ExtLedgerState blk))
forall (m :: * -> *) blk.
IOLike m =>
LgrDB m blk -> STM m (LedgerDB' blk)
getCurrent LgrDB m blk
lgrDB)
    Tracer m (TraceSnapshotEvent blk)
-> SomeHasFS m
-> (ExtLedgerState blk -> Encoding)
-> ExtLedgerState blk
-> m (Maybe (DiskSnapshot, RealPoint blk))
forall (m :: * -> *) blk.
(MonadThrow m, MonadMonotonicTime m, IsLedger (LedgerState blk)) =>
Tracer m (TraceSnapshotEvent blk)
-> SomeHasFS m
-> (ExtLedgerState blk -> Encoding)
-> ExtLedgerState blk
-> m (Maybe (DiskSnapshot, RealPoint blk))
LedgerDB.takeSnapshot
      Tracer m (TraceSnapshotEvent blk)
tracer
      SomeHasFS m
hasFS
      (CodecConfig blk -> ExtLedgerState blk -> Encoding
forall blk.
(EncodeDisk blk (LedgerState blk),
 EncodeDisk blk (ChainDepState (BlockProtocol blk)),
 EncodeDisk blk (AnnTip blk)) =>
CodecConfig blk -> ExtLedgerState blk -> Encoding
encodeDiskExtLedgerState CodecConfig blk
ccfg)
      ExtLedgerState blk
ledgerDB
  where
    ccfg :: CodecConfig blk
ccfg = TopLevelConfig blk -> CodecConfig blk
forall blk. TopLevelConfig blk -> CodecConfig blk
configCodec (TopLevelConfig blk -> CodecConfig blk)
-> TopLevelConfig blk -> CodecConfig blk
forall a b. (a -> b) -> a -> b
$ ExtLedgerCfg blk -> TopLevelConfig blk
forall blk. ExtLedgerCfg blk -> TopLevelConfig blk
getExtLedgerCfg (ExtLedgerCfg blk -> TopLevelConfig blk)
-> ExtLedgerCfg blk -> TopLevelConfig blk
forall a b. (a -> b) -> a -> b
$ LedgerDbCfg (ExtLedgerState blk) -> LedgerCfg (ExtLedgerState blk)
forall l. LedgerDbCfg l -> LedgerCfg l
LedgerDB.ledgerDbCfg LedgerDbCfg (ExtLedgerState blk)
cfg

trimSnapshots ::
     forall m blk. (MonadCatch m, HasHeader blk)
  => LgrDB m blk
  -> m [LedgerDB.DiskSnapshot]
trimSnapshots :: forall (m :: * -> *) blk.
(MonadCatch m, HasHeader blk) =>
LgrDB m blk -> m [DiskSnapshot]
trimSnapshots LgrDB { DiskPolicy
diskPolicy :: forall (m :: * -> *) blk. LgrDB m blk -> DiskPolicy
diskPolicy :: DiskPolicy
diskPolicy, Tracer m (TraceSnapshotEvent blk)
tracer :: forall (m :: * -> *) blk.
LgrDB m blk -> Tracer m (TraceSnapshotEvent blk)
tracer :: Tracer m (TraceSnapshotEvent blk)
tracer, SomeHasFS m
hasFS :: forall (m :: * -> *) blk. LgrDB m blk -> SomeHasFS m
hasFS :: SomeHasFS m
hasFS } = Proxy blk -> m [DiskSnapshot] -> m [DiskSnapshot]
forall (m :: * -> *) x blk.
(MonadCatch m, HasHeader blk) =>
Proxy blk -> m x -> m x
wrapFailure (forall t. Proxy t
forall {k} (t :: k). Proxy t
Proxy @blk) (m [DiskSnapshot] -> m [DiskSnapshot])
-> m [DiskSnapshot] -> m [DiskSnapshot]
forall a b. (a -> b) -> a -> b
$
    Tracer m (TraceSnapshotEvent blk)
-> SomeHasFS m -> DiskPolicy -> m [DiskSnapshot]
forall (m :: * -> *) r.
Monad m =>
Tracer m (TraceSnapshotEvent r)
-> SomeHasFS m -> DiskPolicy -> m [DiskSnapshot]
LedgerDB.trimSnapshots Tracer m (TraceSnapshotEvent blk)
tracer SomeHasFS m
hasFS DiskPolicy
diskPolicy

getDiskPolicy :: LgrDB m blk -> LedgerDB.DiskPolicy
getDiskPolicy :: forall (m :: * -> *) blk. LgrDB m blk -> DiskPolicy
getDiskPolicy = LgrDB m blk -> DiskPolicy
forall (m :: * -> *) blk. LgrDB m blk -> DiskPolicy
diskPolicy

{-------------------------------------------------------------------------------
  Validation
-------------------------------------------------------------------------------}

data ValidateResult blk =
    ValidateSuccessful       (LedgerDB'       blk)
  | ValidateLedgerError      (LedgerDB.AnnLedgerError' blk)
  | ValidateExceededRollBack LedgerDB.ExceededRollback

validate :: forall m blk. (IOLike m, LedgerSupportsProtocol blk, HasCallStack)
         => LgrDB m blk
         -> LedgerDB' blk
            -- ^ This is used as the starting point for validation, not the one
            -- in the 'LgrDB'.
         -> BlockCache blk
         -> Word64  -- ^ How many blocks to roll back
         -> (LedgerDB.UpdateLedgerDbTraceEvent blk -> m ())
         -> [Header blk]
         -> m (ValidateResult blk)
validate :: forall (m :: * -> *) blk.
(IOLike m, LedgerSupportsProtocol blk, HasCallStack) =>
LgrDB m blk
-> LedgerDB' blk
-> BlockCache blk
-> Word64
-> (UpdateLedgerDbTraceEvent blk -> m ())
-> [Header blk]
-> m (ValidateResult blk)
validate LgrDB{Tracer m (TraceSnapshotEvent blk)
SomeHasFS m
StrictTVar m (Set (RealPoint blk))
StrictTVar m (LedgerDB' blk)
DiskPolicy
LedgerDbCfg (ExtLedgerState blk)
ResolveBlock m blk
varDB :: forall (m :: * -> *) blk.
LgrDB m blk -> StrictTVar m (LedgerDB' blk)
varPrevApplied :: forall (m :: * -> *) blk.
LgrDB m blk -> StrictTVar m (Set (RealPoint blk))
resolveBlock :: forall (m :: * -> *) blk. LgrDB m blk -> RealPoint blk -> m blk
cfg :: forall (m :: * -> *) blk.
LgrDB m blk -> LedgerDbCfg (ExtLedgerState blk)
diskPolicy :: forall (m :: * -> *) blk. LgrDB m blk -> DiskPolicy
hasFS :: forall (m :: * -> *) blk. LgrDB m blk -> SomeHasFS m
tracer :: forall (m :: * -> *) blk.
LgrDB m blk -> Tracer m (TraceSnapshotEvent blk)
varDB :: StrictTVar m (LedgerDB' blk)
varPrevApplied :: StrictTVar m (Set (RealPoint blk))
resolveBlock :: ResolveBlock m blk
cfg :: LedgerDbCfg (ExtLedgerState blk)
diskPolicy :: DiskPolicy
hasFS :: SomeHasFS m
tracer :: Tracer m (TraceSnapshotEvent blk)
..} LedgerDB' blk
ledgerDB BlockCache blk
blockCache Word64
numRollbacks UpdateLedgerDbTraceEvent blk -> m ()
trace = \[Header blk]
hdrs -> do
    [Ap
   (ExceptT
      (AnnLedgerError (ExtLedgerState blk) blk)
      (ReaderT (ResolveBlock m blk) m))
   (ExtLedgerState blk)
   blk
   (ResolvesBlocks
      (ExceptT
         (AnnLedgerError (ExtLedgerState blk) blk)
         (ReaderT (ResolveBlock m blk) m))
      blk,
    ThrowsLedgerError
      (ExceptT
         (AnnLedgerError (ExtLedgerState blk) blk)
         (ReaderT (ResolveBlock m blk) m))
      (ExtLedgerState blk)
      blk)]
aps <- [Header blk]
-> Set (RealPoint blk)
-> [Ap
      (ExceptT
         (AnnLedgerError (ExtLedgerState blk) blk)
         (ReaderT (ResolveBlock m blk) m))
      (ExtLedgerState blk)
      blk
      (ResolvesBlocks
         (ExceptT
            (AnnLedgerError (ExtLedgerState blk) blk)
            (ReaderT (ResolveBlock m blk) m))
         blk,
       ThrowsLedgerError
         (ExceptT
            (AnnLedgerError (ExtLedgerState blk) blk)
            (ReaderT (ResolveBlock m blk) m))
         (ExtLedgerState blk)
         blk)]
forall (n :: * -> *) l.
(l ~ ExtLedgerState blk) =>
[Header blk]
-> Set (RealPoint blk)
-> [Ap n l blk (ResolvesBlocks n blk, ThrowsLedgerError n l blk)]
mkAps [Header blk]
hdrs (Set (RealPoint blk)
 -> [Ap
       (ExceptT
          (AnnLedgerError (ExtLedgerState blk) blk)
          (ReaderT (ResolveBlock m blk) m))
       (ExtLedgerState blk)
       blk
       (ResolvesBlocks
          (ExceptT
             (AnnLedgerError (ExtLedgerState blk) blk)
             (ReaderT (ResolveBlock m blk) m))
          blk,
        ThrowsLedgerError
          (ExceptT
             (AnnLedgerError (ExtLedgerState blk) blk)
             (ReaderT (ResolveBlock m blk) m))
          (ExtLedgerState blk)
          blk)])
-> m (Set (RealPoint blk))
-> m [Ap
        (ExceptT
           (AnnLedgerError (ExtLedgerState blk) blk)
           (ReaderT (ResolveBlock m blk) m))
        (ExtLedgerState blk)
        blk
        (ResolvesBlocks
           (ExceptT
              (AnnLedgerError (ExtLedgerState blk) blk)
              (ReaderT (ResolveBlock m blk) m))
           blk,
         ThrowsLedgerError
           (ExceptT
              (AnnLedgerError (ExtLedgerState blk) blk)
              (ReaderT (ResolveBlock m blk) m))
           (ExtLedgerState blk)
           blk)]
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> STM m (Set (RealPoint blk)) -> m (Set (RealPoint blk))
forall a. HasCallStack => STM m a -> m a
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
STM m a -> m a
atomically (StrictTVar m (Set (RealPoint blk)) -> STM m (Set (RealPoint blk))
forall (m :: * -> *) a. MonadSTM m => StrictTVar m a -> STM m a
readTVar StrictTVar m (Set (RealPoint blk))
varPrevApplied)
    ValidateResult blk
res <- (Either
   (AnnLedgerError (ExtLedgerState blk) blk)
   (Either ExceededRollback (LedgerDB' blk))
 -> ValidateResult blk)
-> m (Either
        (AnnLedgerError (ExtLedgerState blk) blk)
        (Either ExceededRollback (LedgerDB' blk)))
-> m (ValidateResult blk)
forall a b. (a -> b) -> m a -> m b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap Either
  (AnnLedgerError (ExtLedgerState blk) blk)
  (Either ExceededRollback (LedgerDB' blk))
-> ValidateResult blk
rewrap (m (Either
      (AnnLedgerError (ExtLedgerState blk) blk)
      (Either ExceededRollback (LedgerDB' blk)))
 -> m (ValidateResult blk))
-> m (Either
        (AnnLedgerError (ExtLedgerState blk) blk)
        (Either ExceededRollback (LedgerDB' blk)))
-> m (ValidateResult blk)
forall a b. (a -> b) -> a -> b
$ ResolveBlock m blk
-> ExceptT
     (AnnLedgerError (ExtLedgerState blk) blk)
     (ReaderT (ResolveBlock m blk) m)
     (Either ExceededRollback (LedgerDB' blk))
-> m (Either
        (AnnLedgerError (ExtLedgerState blk) blk)
        (Either ExceededRollback (LedgerDB' blk)))
forall (m :: * -> *) blk l a.
ResolveBlock m blk
-> ExceptT
     (AnnLedgerError l blk) (ReaderT (ResolveBlock m blk) m) a
-> m (Either (AnnLedgerError l blk) a)
LedgerDB.defaultResolveWithErrors ResolveBlock m blk
resolveBlock (ExceptT
   (AnnLedgerError (ExtLedgerState blk) blk)
   (ReaderT (ResolveBlock m blk) m)
   (Either ExceededRollback (LedgerDB' blk))
 -> m (Either
         (AnnLedgerError (ExtLedgerState blk) blk)
         (Either ExceededRollback (LedgerDB' blk))))
-> ExceptT
     (AnnLedgerError (ExtLedgerState blk) blk)
     (ReaderT (ResolveBlock m blk) m)
     (Either ExceededRollback (LedgerDB' blk))
-> m (Either
        (AnnLedgerError (ExtLedgerState blk) blk)
        (Either ExceededRollback (LedgerDB' blk)))
forall a b. (a -> b) -> a -> b
$
             LedgerDbCfg (ExtLedgerState blk)
-> Word64
-> (UpdateLedgerDbTraceEvent blk
    -> ExceptT
         (AnnLedgerError (ExtLedgerState blk) blk)
         (ReaderT (ResolveBlock m blk) m)
         ())
-> [Ap
      (ExceptT
         (AnnLedgerError (ExtLedgerState blk) blk)
         (ReaderT (ResolveBlock m blk) m))
      (ExtLedgerState blk)
      blk
      (ResolvesBlocks
         (ExceptT
            (AnnLedgerError (ExtLedgerState blk) blk)
            (ReaderT (ResolveBlock m blk) m))
         blk,
       ThrowsLedgerError
         (ExceptT
            (AnnLedgerError (ExtLedgerState blk) blk)
            (ReaderT (ResolveBlock m blk) m))
         (ExtLedgerState blk)
         blk)]
-> LedgerDB' blk
-> ExceptT
     (AnnLedgerError (ExtLedgerState blk) blk)
     (ReaderT (ResolveBlock m blk) m)
     (Either ExceededRollback (LedgerDB' blk))
forall l blk (m :: * -> *) (c :: Constraint).
(ApplyBlock l blk, Monad m, c) =>
LedgerDbCfg l
-> Word64
-> (UpdateLedgerDbTraceEvent blk -> m ())
-> [Ap m l blk c]
-> LedgerDB l
-> m (Either ExceededRollback (LedgerDB l))
LedgerDB.ledgerDbSwitch
               LedgerDbCfg (ExtLedgerState blk)
cfg
               Word64
numRollbacks
               (ReaderT (ResolveBlock m blk) m ()
-> ExceptT
     (AnnLedgerError (ExtLedgerState blk) blk)
     (ReaderT (ResolveBlock m blk) m)
     ()
forall (m :: * -> *) a.
Monad m =>
m a -> ExceptT (AnnLedgerError (ExtLedgerState blk) blk) m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (ReaderT (ResolveBlock m blk) m ()
 -> ExceptT
      (AnnLedgerError (ExtLedgerState blk) blk)
      (ReaderT (ResolveBlock m blk) m)
      ())
-> (UpdateLedgerDbTraceEvent blk
    -> ReaderT (ResolveBlock m blk) m ())
-> UpdateLedgerDbTraceEvent blk
-> ExceptT
     (AnnLedgerError (ExtLedgerState blk) blk)
     (ReaderT (ResolveBlock m blk) m)
     ()
forall b c a. (b -> c) -> (a -> b) -> a -> c
. m () -> ReaderT (ResolveBlock m blk) m ()
forall (m :: * -> *) a.
Monad m =>
m a -> ReaderT (ResolveBlock m blk) m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (m () -> ReaderT (ResolveBlock m blk) m ())
-> (UpdateLedgerDbTraceEvent blk -> m ())
-> UpdateLedgerDbTraceEvent blk
-> ReaderT (ResolveBlock m blk) m ()
forall b c a. (b -> c) -> (a -> b) -> a -> c
. UpdateLedgerDbTraceEvent blk -> m ()
trace)
               [Ap
   (ExceptT
      (AnnLedgerError (ExtLedgerState blk) blk)
      (ReaderT (ResolveBlock m blk) m))
   (ExtLedgerState blk)
   blk
   (ResolvesBlocks
      (ExceptT
         (AnnLedgerError (ExtLedgerState blk) blk)
         (ReaderT (ResolveBlock m blk) m))
      blk,
    ThrowsLedgerError
      (ExceptT
         (AnnLedgerError (ExtLedgerState blk) blk)
         (ReaderT (ResolveBlock m blk) m))
      (ExtLedgerState blk)
      blk)]
aps
               LedgerDB' blk
ledgerDB
    STM m () -> m ()
forall a. HasCallStack => STM m a -> m a
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
STM m a -> m a
atomically (STM m () -> m ()) -> STM m () -> m ()
forall a b. (a -> b) -> a -> b
$ StrictTVar m (Set (RealPoint blk))
-> (Set (RealPoint blk) -> Set (RealPoint blk)) -> STM m ()
forall (m :: * -> *) a.
MonadSTM m =>
StrictTVar m a -> (a -> a) -> STM m ()
modifyTVar StrictTVar m (Set (RealPoint blk))
varPrevApplied ((Set (RealPoint blk) -> Set (RealPoint blk)) -> STM m ())
-> (Set (RealPoint blk) -> Set (RealPoint blk)) -> STM m ()
forall a b. (a -> b) -> a -> b
$
      [RealPoint blk] -> Set (RealPoint blk) -> Set (RealPoint blk)
addPoints (ValidateResult blk -> [RealPoint blk] -> [RealPoint blk]
validBlockPoints ValidateResult blk
res ((Header blk -> RealPoint blk) -> [Header blk] -> [RealPoint blk]
forall a b. (a -> b) -> [a] -> [b]
map Header blk -> RealPoint blk
forall blk.
(HasHeader (Header blk), HasHeader blk) =>
Header blk -> RealPoint blk
headerRealPoint [Header blk]
hdrs))
    ValidateResult blk -> m (ValidateResult blk)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return ValidateResult blk
res
  where
    rewrap :: Either (LedgerDB.AnnLedgerError' blk) (Either LedgerDB.ExceededRollback (LedgerDB' blk))
           -> ValidateResult blk
    rewrap :: Either
  (AnnLedgerError (ExtLedgerState blk) blk)
  (Either ExceededRollback (LedgerDB' blk))
-> ValidateResult blk
rewrap (Left         AnnLedgerError (ExtLedgerState blk) blk
e)  = AnnLedgerError (ExtLedgerState blk) blk -> ValidateResult blk
forall blk. AnnLedgerError' blk -> ValidateResult blk
ValidateLedgerError      AnnLedgerError (ExtLedgerState blk) blk
e
    rewrap (Right (Left  ExceededRollback
e)) = ExceededRollback -> ValidateResult blk
forall blk. ExceededRollback -> ValidateResult blk
ValidateExceededRollBack ExceededRollback
e
    rewrap (Right (Right LedgerDB' blk
l)) = LedgerDB' blk -> ValidateResult blk
forall blk. LedgerDB' blk -> ValidateResult blk
ValidateSuccessful       LedgerDB' blk
l

    mkAps :: forall n l. l ~ ExtLedgerState blk
          => [Header blk]
          -> Set (RealPoint blk)
          -> [LedgerDB.Ap n l blk ( LedgerDB.ResolvesBlocks    n   blk
                                  , LedgerDB.ThrowsLedgerError n l blk
                                  )]
    mkAps :: forall (n :: * -> *) l.
(l ~ ExtLedgerState blk) =>
[Header blk]
-> Set (RealPoint blk)
-> [Ap n l blk (ResolvesBlocks n blk, ThrowsLedgerError n l blk)]
mkAps [Header blk]
hdrs Set (RealPoint blk)
prevApplied =
      [ case ( RealPoint blk -> Set (RealPoint blk) -> Bool
forall a. Ord a => a -> Set a -> Bool
Set.member (Header blk -> RealPoint blk
forall blk.
(HasHeader (Header blk), HasHeader blk) =>
Header blk -> RealPoint blk
headerRealPoint Header blk
hdr) Set (RealPoint blk)
prevApplied
             , HeaderHash blk -> BlockCache blk -> Maybe blk
forall blk.
HasHeader blk =>
HeaderHash blk -> BlockCache blk -> Maybe blk
BlockCache.lookup (Header blk -> HeaderHash blk
forall blk. HasHeader (Header blk) => Header blk -> HeaderHash blk
headerHash Header blk
hdr) BlockCache blk
blockCache
             ) of
          (Bool
False, Maybe blk
Nothing)  ->                   RealPoint blk
-> Ap n l blk (ResolvesBlocks n blk, ThrowsLedgerError n l blk)
forall blk (m :: * -> *) l.
RealPoint blk
-> Ap m l blk (ResolvesBlocks m blk, ThrowsLedgerError m l blk)
LedgerDB.ApplyRef   (Header blk -> RealPoint blk
forall blk.
(HasHeader (Header blk), HasHeader blk) =>
Header blk -> RealPoint blk
headerRealPoint Header blk
hdr)
          (Bool
True,  Maybe blk
Nothing)  -> Ap n l blk (ResolvesBlocks n blk)
-> Ap n l blk (ResolvesBlocks n blk, ThrowsLedgerError n l blk)
forall (c :: Constraint) (c1 :: Constraint) (m :: * -> *) l blk.
(c => c1) =>
Ap m l blk c1 -> Ap m l blk c
LedgerDB.Weaken (Ap n l blk (ResolvesBlocks n blk)
 -> Ap n l blk (ResolvesBlocks n blk, ThrowsLedgerError n l blk))
-> Ap n l blk (ResolvesBlocks n blk)
-> Ap n l blk (ResolvesBlocks n blk, ThrowsLedgerError n l blk)
forall a b. (a -> b) -> a -> b
$ RealPoint blk -> Ap n l blk (ResolvesBlocks n blk)
forall blk (m :: * -> *) l.
RealPoint blk -> Ap m l blk (ResolvesBlocks m blk)
LedgerDB.ReapplyRef (Header blk -> RealPoint blk
forall blk.
(HasHeader (Header blk), HasHeader blk) =>
Header blk -> RealPoint blk
headerRealPoint Header blk
hdr)
          (Bool
False, Just blk
blk) -> Ap n l blk (ThrowsLedgerError n l blk)
-> Ap n l blk (ResolvesBlocks n blk, ThrowsLedgerError n l blk)
forall (c :: Constraint) (c1 :: Constraint) (m :: * -> *) l blk.
(c => c1) =>
Ap m l blk c1 -> Ap m l blk c
LedgerDB.Weaken (Ap n l blk (ThrowsLedgerError n l blk)
 -> Ap n l blk (ResolvesBlocks n blk, ThrowsLedgerError n l blk))
-> Ap n l blk (ThrowsLedgerError n l blk)
-> Ap n l blk (ResolvesBlocks n blk, ThrowsLedgerError n l blk)
forall a b. (a -> b) -> a -> b
$ blk -> Ap n l blk (ThrowsLedgerError n l blk)
forall blk (m :: * -> *) l.
blk -> Ap m l blk (ThrowsLedgerError m l blk)
LedgerDB.ApplyVal   blk
blk
          (Bool
True,  Just blk
blk) -> Ap n l blk (() :: Constraint)
-> Ap n l blk (ResolvesBlocks n blk, ThrowsLedgerError n l blk)
forall (c :: Constraint) (c1 :: Constraint) (m :: * -> *) l blk.
(c => c1) =>
Ap m l blk c1 -> Ap m l blk c
LedgerDB.Weaken (Ap n l blk (() :: Constraint)
 -> Ap n l blk (ResolvesBlocks n blk, ThrowsLedgerError n l blk))
-> Ap n l blk (() :: Constraint)
-> Ap n l blk (ResolvesBlocks n blk, ThrowsLedgerError n l blk)
forall a b. (a -> b) -> a -> b
$ blk -> Ap n l blk (() :: Constraint)
forall blk (m :: * -> *) l. blk -> Ap m l blk (() :: Constraint)
LedgerDB.ReapplyVal blk
blk
      | Header blk
hdr <- [Header blk]
hdrs
      ]

    -- | Based on the 'ValidateResult', return the hashes corresponding to
    -- valid blocks.
    validBlockPoints :: ValidateResult blk -> [RealPoint blk] -> [RealPoint blk]
    validBlockPoints :: ValidateResult blk -> [RealPoint blk] -> [RealPoint blk]
validBlockPoints = \case
      ValidateExceededRollBack ExceededRollback
_ -> [RealPoint blk] -> [RealPoint blk] -> [RealPoint blk]
forall a b. a -> b -> a
const []
      ValidateSuccessful       LedgerDB' blk
_ -> [RealPoint blk] -> [RealPoint blk]
forall a. a -> a
id
      ValidateLedgerError      AnnLedgerError (ExtLedgerState blk) blk
e -> (RealPoint blk -> Bool) -> [RealPoint blk] -> [RealPoint blk]
forall a. (a -> Bool) -> [a] -> [a]
takeWhile (RealPoint blk -> RealPoint blk -> Bool
forall a. Eq a => a -> a -> Bool
/= AnnLedgerError (ExtLedgerState blk) blk -> RealPoint blk
forall l blk. AnnLedgerError l blk -> RealPoint blk
LedgerDB.annLedgerErrRef AnnLedgerError (ExtLedgerState blk) blk
e)

    addPoints :: [RealPoint blk]
              -> Set (RealPoint blk) -> Set (RealPoint blk)
    addPoints :: [RealPoint blk] -> Set (RealPoint blk) -> Set (RealPoint blk)
addPoints [RealPoint blk]
hs Set (RealPoint blk)
set = (Set (RealPoint blk) -> RealPoint blk -> Set (RealPoint blk))
-> Set (RealPoint blk) -> [RealPoint blk] -> Set (RealPoint blk)
forall b a. (b -> a -> b) -> b -> [a] -> b
forall (t :: * -> *) b a.
Foldable t =>
(b -> a -> b) -> b -> t a -> b
Foldable.foldl' ((RealPoint blk -> Set (RealPoint blk) -> Set (RealPoint blk))
-> Set (RealPoint blk) -> RealPoint blk -> Set (RealPoint blk)
forall a b c. (a -> b -> c) -> b -> a -> c
flip RealPoint blk -> Set (RealPoint blk) -> Set (RealPoint blk)
forall a. Ord a => a -> Set a -> Set a
Set.insert) Set (RealPoint blk)
set [RealPoint blk]
hs

{-------------------------------------------------------------------------------
  Previously applied blocks
-------------------------------------------------------------------------------}

getPrevApplied :: IOLike m => LgrDB m blk -> STM m (Set (RealPoint blk))
getPrevApplied :: forall (m :: * -> *) blk.
IOLike m =>
LgrDB m blk -> STM m (Set (RealPoint blk))
getPrevApplied LgrDB{Tracer m (TraceSnapshotEvent blk)
SomeHasFS m
StrictTVar m (Set (RealPoint blk))
StrictTVar m (LedgerDB' blk)
DiskPolicy
LedgerDbCfg (ExtLedgerState blk)
RealPoint blk -> m blk
varDB :: forall (m :: * -> *) blk.
LgrDB m blk -> StrictTVar m (LedgerDB' blk)
varPrevApplied :: forall (m :: * -> *) blk.
LgrDB m blk -> StrictTVar m (Set (RealPoint blk))
resolveBlock :: forall (m :: * -> *) blk. LgrDB m blk -> RealPoint blk -> m blk
cfg :: forall (m :: * -> *) blk.
LgrDB m blk -> LedgerDbCfg (ExtLedgerState blk)
diskPolicy :: forall (m :: * -> *) blk. LgrDB m blk -> DiskPolicy
hasFS :: forall (m :: * -> *) blk. LgrDB m blk -> SomeHasFS m
tracer :: forall (m :: * -> *) blk.
LgrDB m blk -> Tracer m (TraceSnapshotEvent blk)
varDB :: StrictTVar m (LedgerDB' blk)
varPrevApplied :: StrictTVar m (Set (RealPoint blk))
resolveBlock :: RealPoint blk -> m blk
cfg :: LedgerDbCfg (ExtLedgerState blk)
diskPolicy :: DiskPolicy
hasFS :: SomeHasFS m
tracer :: Tracer m (TraceSnapshotEvent blk)
..} = StrictTVar m (Set (RealPoint blk)) -> STM m (Set (RealPoint blk))
forall (m :: * -> *) a. MonadSTM m => StrictTVar m a -> STM m a
readTVar StrictTVar m (Set (RealPoint blk))
varPrevApplied

-- | Remove all points with a slot older than the given slot from the set of
-- previously applied points.
garbageCollectPrevApplied :: IOLike m => LgrDB m blk -> SlotNo -> STM m ()
garbageCollectPrevApplied :: forall (m :: * -> *) blk.
IOLike m =>
LgrDB m blk -> SlotNo -> STM m ()
garbageCollectPrevApplied LgrDB{Tracer m (TraceSnapshotEvent blk)
SomeHasFS m
StrictTVar m (Set (RealPoint blk))
StrictTVar m (LedgerDB' blk)
DiskPolicy
LedgerDbCfg (ExtLedgerState blk)
RealPoint blk -> m blk
varDB :: forall (m :: * -> *) blk.
LgrDB m blk -> StrictTVar m (LedgerDB' blk)
varPrevApplied :: forall (m :: * -> *) blk.
LgrDB m blk -> StrictTVar m (Set (RealPoint blk))
resolveBlock :: forall (m :: * -> *) blk. LgrDB m blk -> RealPoint blk -> m blk
cfg :: forall (m :: * -> *) blk.
LgrDB m blk -> LedgerDbCfg (ExtLedgerState blk)
diskPolicy :: forall (m :: * -> *) blk. LgrDB m blk -> DiskPolicy
hasFS :: forall (m :: * -> *) blk. LgrDB m blk -> SomeHasFS m
tracer :: forall (m :: * -> *) blk.
LgrDB m blk -> Tracer m (TraceSnapshotEvent blk)
varDB :: StrictTVar m (LedgerDB' blk)
varPrevApplied :: StrictTVar m (Set (RealPoint blk))
resolveBlock :: RealPoint blk -> m blk
cfg :: LedgerDbCfg (ExtLedgerState blk)
diskPolicy :: DiskPolicy
hasFS :: SomeHasFS m
tracer :: Tracer m (TraceSnapshotEvent blk)
..} SlotNo
slotNo = StrictTVar m (Set (RealPoint blk))
-> (Set (RealPoint blk) -> Set (RealPoint blk)) -> STM m ()
forall (m :: * -> *) a.
MonadSTM m =>
StrictTVar m a -> (a -> a) -> STM m ()
modifyTVar StrictTVar m (Set (RealPoint blk))
varPrevApplied ((Set (RealPoint blk) -> Set (RealPoint blk)) -> STM m ())
-> (Set (RealPoint blk) -> Set (RealPoint blk)) -> STM m ()
forall a b. (a -> b) -> a -> b
$
    (RealPoint blk -> Bool)
-> Set (RealPoint blk) -> Set (RealPoint blk)
forall a. (a -> Bool) -> Set a -> Set a
Set.dropWhileAntitone ((SlotNo -> SlotNo -> Bool
forall a. Ord a => a -> a -> Bool
< SlotNo
slotNo) (SlotNo -> Bool)
-> (RealPoint blk -> SlotNo) -> RealPoint blk -> Bool
forall b c a. (b -> c) -> (a -> b) -> a -> c
. RealPoint blk -> SlotNo
forall blk. RealPoint blk -> SlotNo
realPointSlot)

{-------------------------------------------------------------------------------
  Error handling
-------------------------------------------------------------------------------}

-- | Wrap exceptions that may indicate disk failure in a 'ChainDbFailure'
-- exception using the 'LgrDbFailure' constructor.
wrapFailure ::
     forall m x blk. (MonadCatch m, HasHeader blk)
  => Proxy blk
  -> m x
  -> m x
wrapFailure :: forall (m :: * -> *) x blk.
(MonadCatch m, HasHeader blk) =>
Proxy blk -> m x -> m x
wrapFailure Proxy blk
_ m x
k = m x -> (FsError -> m x) -> m x
forall e a. Exception e => m a -> (e -> m a) -> m a
forall (m :: * -> *) e a.
(MonadCatch m, Exception e) =>
m a -> (e -> m a) -> m a
catch m x
k FsError -> m x
rethrow
  where
    rethrow :: FsError -> m x
    rethrow :: FsError -> m x
rethrow FsError
err = ChainDbFailure blk -> m x
forall e a. Exception e => e -> m a
forall (m :: * -> *) e a. (MonadThrow m, Exception e) => e -> m a
throwIO (ChainDbFailure blk -> m x) -> ChainDbFailure blk -> m x
forall a b. (a -> b) -> a -> b
$ forall blk. FsError -> ChainDbFailure blk
LgrDbFailure @blk FsError
err