{-# LANGUAGE BangPatterns #-}
{-# LANGUAGE DeriveGeneric #-}
{-# LANGUAGE RankNTypes #-}
{-# LANGUAGE ScopedTypeVariables #-}

-- | LedgerDB initialization either from a LedgerState or from a DiskSnapshot
module Ouroboros.Consensus.Storage.LedgerDB.Init (
    -- * Initialization
    InitLog (..)
  , ReplayStart (..)
  , initLedgerDB
    -- * Trace
  , ReplayGoal (..)
  , TraceReplayEvent (..)
  , decorateReplayTracerWithGoal
  , decorateReplayTracerWithStart
  ) where

import           Codec.Serialise.Decoding (Decoder)
import           Control.Monad (when)
import           Control.Monad.Except (ExceptT, runExceptT, throwError,
                     withExceptT)
import           Control.Monad.Trans.Class (lift)
import           Control.Tracer
import           Data.Word
import           GHC.Generics (Generic)
import           GHC.Stack
import           Ouroboros.Consensus.Block
import           Ouroboros.Consensus.Ledger.Abstract
import           Ouroboros.Consensus.Ledger.Extended
import           Ouroboros.Consensus.Ledger.Inspect
import           Ouroboros.Consensus.Ledger.SupportsProtocol
import           Ouroboros.Consensus.Storage.ImmutableDB.Stream
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.IOLike
import           Ouroboros.Network.Block (Point (Point))
import           System.FS.API

{-------------------------------------------------------------------------------
  Initialize the DB
-------------------------------------------------------------------------------}

-- | Initialization log
--
-- The initialization log records which snapshots from disk were considered,
-- in which order, and why some snapshots were rejected. It is primarily useful
-- for monitoring purposes.
data InitLog blk =
    -- | Defaulted to initialization from genesis
    --
    -- NOTE: Unless the blockchain is near genesis, we should see this /only/
    -- if data corrupted occurred.
    InitFromGenesis

    -- | Used a snapshot corresponding to the specified tip
  | InitFromSnapshot DiskSnapshot (RealPoint blk)

    -- | Initialization skipped a snapshot
    --
    -- We record the reason why it was skipped.
    --
    -- NOTE: We should /only/ see this if data corrupted occurred.
  | InitFailure DiskSnapshot (SnapshotFailure blk) (InitLog blk)
  deriving (Int -> InitLog blk -> ShowS
[InitLog blk] -> ShowS
InitLog blk -> String
(Int -> InitLog blk -> ShowS)
-> (InitLog blk -> String)
-> ([InitLog blk] -> ShowS)
-> Show (InitLog blk)
forall blk. StandardHash blk => Int -> InitLog blk -> ShowS
forall blk. StandardHash blk => [InitLog blk] -> ShowS
forall blk. StandardHash blk => InitLog blk -> String
forall a.
(Int -> a -> ShowS) -> (a -> String) -> ([a] -> ShowS) -> Show a
$cshowsPrec :: forall blk. StandardHash blk => Int -> InitLog blk -> ShowS
showsPrec :: Int -> InitLog blk -> ShowS
$cshow :: forall blk. StandardHash blk => InitLog blk -> String
show :: InitLog blk -> String
$cshowList :: forall blk. StandardHash blk => [InitLog blk] -> ShowS
showList :: [InitLog blk] -> ShowS
Show, InitLog blk -> InitLog blk -> Bool
(InitLog blk -> InitLog blk -> Bool)
-> (InitLog blk -> InitLog blk -> Bool) -> Eq (InitLog blk)
forall blk. StandardHash blk => InitLog blk -> InitLog blk -> Bool
forall a. (a -> a -> Bool) -> (a -> a -> Bool) -> Eq a
$c== :: forall blk. StandardHash blk => InitLog blk -> InitLog blk -> Bool
== :: InitLog blk -> InitLog blk -> Bool
$c/= :: forall blk. StandardHash blk => InitLog blk -> InitLog blk -> Bool
/= :: InitLog blk -> InitLog blk -> Bool
Eq, (forall x. InitLog blk -> Rep (InitLog blk) x)
-> (forall x. Rep (InitLog blk) x -> InitLog blk)
-> Generic (InitLog blk)
forall x. Rep (InitLog blk) x -> InitLog blk
forall x. InitLog blk -> Rep (InitLog blk) x
forall a.
(forall x. a -> Rep a x) -> (forall x. Rep a x -> a) -> Generic a
forall blk x. Rep (InitLog blk) x -> InitLog blk
forall blk x. InitLog blk -> Rep (InitLog blk) x
$cfrom :: forall blk x. InitLog blk -> Rep (InitLog blk) x
from :: forall x. InitLog blk -> Rep (InitLog blk) x
$cto :: forall blk x. Rep (InitLog blk) x -> InitLog blk
to :: forall x. Rep (InitLog blk) x -> InitLog blk
Generic)

-- | Initialize the ledger DB from the most recent snapshot on disk
--
-- If no such snapshot can be found, use the genesis ledger DB. Returns the
-- initialized DB as well as the block reference corresponding to the snapshot
-- we found on disk (the latter primarily for testing/monitoring purposes).
--
-- We do /not/ catch any exceptions thrown during streaming; should any be
-- thrown, it is the responsibility of the 'ChainDB' to catch these
-- and trigger (further) validation. We only discard snapshots if
--
-- * We cannot deserialise them, or
-- * they are /ahead/ of the chain
--
-- It is possible that the Ledger DB will not be able to roll back @k@ blocks
-- after initialization if the chain has been truncated (data corruption).
--
-- We do /not/ attempt to use multiple ledger states from disk to construct the
-- ledger DB. Instead we load only a /single/ ledger state from disk, and
-- /compute/ all subsequent ones. This is important, because the ledger states
-- obtained in this way will (hopefully) share much of their memory footprint
-- with their predecessors.
initLedgerDB ::
     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) -- ^ Genesis ledger state
  -> StreamAPI m blk blk
  -> m (InitLog blk, LedgerDB' blk, Word64)
initLedgerDB :: 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)
initLedgerDB Tracer m (ReplayGoal blk -> TraceReplayEvent blk)
replayTracer
             Tracer m (TraceSnapshotEvent blk)
tracer
             SomeHasFS m
hasFS
             forall s. Decoder s (ExtLedgerState blk)
decLedger
             forall s. Decoder s (HeaderHash blk)
decHash
             LedgerDbCfg (ExtLedgerState blk)
cfg
             m (ExtLedgerState blk)
getGenesisLedger
             StreamAPI m blk blk
stream = do
    [DiskSnapshot]
snapshots <- SomeHasFS m -> m [DiskSnapshot]
forall (m :: * -> *). Monad m => SomeHasFS m -> m [DiskSnapshot]
listSnapshots SomeHasFS m
hasFS
    (InitLog blk -> InitLog blk)
-> [DiskSnapshot] -> m (InitLog blk, LedgerDB' blk, Word64)
tryNewestFirst InitLog blk -> InitLog blk
forall a. a -> a
id [DiskSnapshot]
snapshots
  where
    tryNewestFirst :: (InitLog blk -> InitLog blk)
                   -> [DiskSnapshot]
                   -> m (InitLog blk, LedgerDB' blk, Word64)
    tryNewestFirst :: (InitLog blk -> InitLog blk)
-> [DiskSnapshot] -> m (InitLog blk, LedgerDB' blk, Word64)
tryNewestFirst InitLog blk -> InitLog blk
acc [] = do
        -- We're out of snapshots. Start at genesis
        Tracer m (ReplayGoal blk -> TraceReplayEvent blk)
-> (ReplayGoal blk -> TraceReplayEvent blk) -> m ()
forall (m :: * -> *) a. Tracer m a -> a -> m ()
traceWith Tracer m (ReplayGoal blk -> TraceReplayEvent blk)
replayTracer ReplayGoal blk -> TraceReplayEvent blk
forall blk. ReplayGoal blk -> TraceReplayEvent blk
ReplayFromGenesis
        LedgerDB' blk
initDb <- ExtLedgerState blk -> LedgerDB' blk
forall l. GetTip l => l -> LedgerDB l
ledgerDbWithAnchor (ExtLedgerState blk -> LedgerDB' blk)
-> m (ExtLedgerState blk) -> m (LedgerDB' blk)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> m (ExtLedgerState blk)
getGenesisLedger
        let replayTracer' :: Tracer
  m (ReplayStart blk -> ReplayGoal blk -> TraceReplayEvent blk)
replayTracer' = Point blk
-> Tracer m (ReplayGoal blk -> TraceReplayEvent blk)
-> Tracer
     m (ReplayStart blk -> ReplayGoal blk -> TraceReplayEvent blk)
forall blk (m :: * -> *).
Point blk
-> Tracer m (ReplayGoal blk -> TraceReplayEvent blk)
-> Tracer
     m (ReplayStart blk -> ReplayGoal blk -> TraceReplayEvent blk)
decorateReplayTracerWithStart (WithOrigin (Block SlotNo (HeaderHash blk)) -> Point blk
forall {k} (block :: k).
WithOrigin (Block SlotNo (HeaderHash block)) -> Point block
Point WithOrigin (Block SlotNo (HeaderHash blk))
forall t. WithOrigin t
Origin) Tracer m (ReplayGoal blk -> TraceReplayEvent blk)
replayTracer
        Either (SnapshotFailure blk) (LedgerDB' blk, Word64)
ml     <- ExceptT (SnapshotFailure blk) m (LedgerDB' blk, Word64)
-> m (Either (SnapshotFailure blk) (LedgerDB' blk, Word64))
forall e (m :: * -> *) a. ExceptT e m a -> m (Either e a)
runExceptT (ExceptT (SnapshotFailure blk) m (LedgerDB' blk, Word64)
 -> m (Either (SnapshotFailure blk) (LedgerDB' blk, Word64)))
-> ExceptT (SnapshotFailure blk) m (LedgerDB' blk, Word64)
-> m (Either (SnapshotFailure blk) (LedgerDB' blk, Word64))
forall a b. (a -> b) -> a -> b
$ Tracer
  m (ReplayStart blk -> ReplayGoal blk -> TraceReplayEvent blk)
-> LedgerDbCfg (ExtLedgerState blk)
-> StreamAPI m blk blk
-> LedgerDB' blk
-> ExceptT (SnapshotFailure blk) m (LedgerDB' blk, Word64)
forall (m :: * -> *) blk.
(Monad m, LedgerSupportsProtocol blk, InspectLedger blk,
 HasCallStack) =>
Tracer
  m (ReplayStart blk -> ReplayGoal blk -> TraceReplayEvent blk)
-> LedgerDbCfg (ExtLedgerState blk)
-> StreamAPI m blk blk
-> LedgerDB' blk
-> ExceptT (SnapshotFailure blk) m (LedgerDB' blk, Word64)
initStartingWith Tracer
  m (ReplayStart blk -> ReplayGoal blk -> TraceReplayEvent blk)
replayTracer' LedgerDbCfg (ExtLedgerState blk)
cfg StreamAPI m blk blk
stream LedgerDB' blk
initDb
        case Either (SnapshotFailure blk) (LedgerDB' blk, Word64)
ml of
          Left SnapshotFailure blk
_  -> String -> m (InitLog blk, LedgerDB' blk, Word64)
forall a. HasCallStack => String -> a
error String
"invariant violation: invalid current chain"
          Right (LedgerDB' blk
l, Word64
replayed) -> (InitLog blk, LedgerDB' blk, Word64)
-> m (InitLog blk, LedgerDB' blk, Word64)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (InitLog blk -> InitLog blk
acc InitLog blk
forall blk. InitLog blk
InitFromGenesis, LedgerDB' blk
l, Word64
replayed)
    tryNewestFirst InitLog blk -> InitLog blk
acc (DiskSnapshot
s:[DiskSnapshot]
ss) = do
        -- If we fail to use this snapshot, delete it and try an older one
        Either (SnapshotFailure blk) (RealPoint blk, LedgerDB' blk, Word64)
ml <- ExceptT
  (SnapshotFailure blk) m (RealPoint blk, LedgerDB' blk, Word64)
-> m (Either
        (SnapshotFailure blk) (RealPoint blk, LedgerDB' blk, Word64))
forall e (m :: * -> *) a. ExceptT e m a -> m (Either e a)
runExceptT (ExceptT
   (SnapshotFailure blk) m (RealPoint blk, LedgerDB' blk, Word64)
 -> m (Either
         (SnapshotFailure blk) (RealPoint blk, LedgerDB' blk, Word64)))
-> ExceptT
     (SnapshotFailure blk) m (RealPoint blk, LedgerDB' blk, Word64)
-> m (Either
        (SnapshotFailure blk) (RealPoint blk, LedgerDB' blk, Word64))
forall a b. (a -> b) -> a -> b
$ Tracer m (ReplayGoal blk -> TraceReplayEvent blk)
-> SomeHasFS m
-> (forall s. Decoder s (ExtLedgerState blk))
-> (forall s. Decoder s (HeaderHash blk))
-> LedgerDbCfg (ExtLedgerState blk)
-> StreamAPI m blk blk
-> DiskSnapshot
-> ExceptT
     (SnapshotFailure blk) m (RealPoint blk, LedgerDB' blk, Word64)
forall (m :: * -> *) blk.
(IOLike m, LedgerSupportsProtocol blk, InspectLedger blk,
 HasCallStack) =>
Tracer m (ReplayGoal blk -> TraceReplayEvent blk)
-> SomeHasFS m
-> (forall s. Decoder s (ExtLedgerState blk))
-> (forall s. Decoder s (HeaderHash blk))
-> LedgerDbCfg (ExtLedgerState blk)
-> StreamAPI m blk blk
-> DiskSnapshot
-> ExceptT
     (SnapshotFailure blk) m (RealPoint blk, LedgerDB' blk, Word64)
initFromSnapshot
                             Tracer m (ReplayGoal blk -> TraceReplayEvent blk)
replayTracer
                             SomeHasFS m
hasFS
                             Decoder s (ExtLedgerState blk)
forall s. Decoder s (ExtLedgerState blk)
decLedger
                             Decoder s (HeaderHash blk)
forall s. Decoder s (HeaderHash blk)
decHash
                             LedgerDbCfg (ExtLedgerState blk)
cfg
                             StreamAPI m blk blk
stream
                             DiskSnapshot
s
        case Either (SnapshotFailure blk) (RealPoint blk, LedgerDB' blk, Word64)
ml of
          Left SnapshotFailure blk
err -> do
            Bool -> m () -> m ()
forall (f :: * -> *). Applicative f => Bool -> f () -> f ()
when (DiskSnapshot -> Bool
diskSnapshotIsTemporary DiskSnapshot
s) (m () -> m ()) -> m () -> m ()
forall a b. (a -> b) -> a -> b
$
              -- We don't delete permanent snapshots, even if we couldn't parse
              -- them
              SomeHasFS m -> DiskSnapshot -> m ()
forall (m :: * -> *).
HasCallStack =>
SomeHasFS m -> DiskSnapshot -> m ()
deleteSnapshot SomeHasFS m
hasFS DiskSnapshot
s
            Tracer m (TraceSnapshotEvent blk) -> TraceSnapshotEvent blk -> m ()
forall (m :: * -> *) a. Tracer m a -> a -> m ()
traceWith Tracer m (TraceSnapshotEvent blk)
tracer (TraceSnapshotEvent blk -> m ()) -> TraceSnapshotEvent blk -> m ()
forall a b. (a -> b) -> a -> b
$ DiskSnapshot -> SnapshotFailure blk -> TraceSnapshotEvent blk
forall blk.
DiskSnapshot -> SnapshotFailure blk -> TraceSnapshotEvent blk
InvalidSnapshot DiskSnapshot
s SnapshotFailure blk
err
            (InitLog blk -> InitLog blk)
-> [DiskSnapshot] -> m (InitLog blk, LedgerDB' blk, Word64)
tryNewestFirst (InitLog blk -> InitLog blk
acc (InitLog blk -> InitLog blk)
-> (InitLog blk -> InitLog blk) -> InitLog blk -> InitLog blk
forall b c a. (b -> c) -> (a -> b) -> a -> c
. DiskSnapshot -> SnapshotFailure blk -> InitLog blk -> InitLog blk
forall blk.
DiskSnapshot -> SnapshotFailure blk -> InitLog blk -> InitLog blk
InitFailure DiskSnapshot
s SnapshotFailure blk
err) [DiskSnapshot]
ss
          Right (RealPoint blk
r, LedgerDB' blk
l, Word64
replayed) ->
            (InitLog blk, LedgerDB' blk, Word64)
-> m (InitLog blk, LedgerDB' blk, Word64)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (InitLog blk -> InitLog blk
acc (DiskSnapshot -> RealPoint blk -> InitLog blk
forall blk. DiskSnapshot -> RealPoint blk -> InitLog blk
InitFromSnapshot DiskSnapshot
s RealPoint blk
r), LedgerDB' blk
l, Word64
replayed)

{-------------------------------------------------------------------------------
  Internal: initialize using the given snapshot
-------------------------------------------------------------------------------}

-- | Attempt to initialize the ledger DB from the given snapshot
--
-- If the chain DB or ledger layer reports an error, the whole thing is aborted
-- and an error is returned. This should not throw any errors itself (ignoring
-- unexpected exceptions such as asynchronous exceptions, of course).
initFromSnapshot ::
     forall m blk. (
         IOLike m
       , LedgerSupportsProtocol blk
       , InspectLedger blk
       , HasCallStack
       )
  => Tracer m (ReplayGoal blk -> TraceReplayEvent blk)
  -> SomeHasFS m
  -> (forall s. Decoder s (ExtLedgerState blk))
  -> (forall s. Decoder s (HeaderHash blk))
  -> LedgerDbCfg (ExtLedgerState blk)
  -> StreamAPI m blk blk
  -> DiskSnapshot
  -> ExceptT (SnapshotFailure blk) m (RealPoint blk, LedgerDB' blk, Word64)
initFromSnapshot :: forall (m :: * -> *) blk.
(IOLike m, LedgerSupportsProtocol blk, InspectLedger blk,
 HasCallStack) =>
Tracer m (ReplayGoal blk -> TraceReplayEvent blk)
-> SomeHasFS m
-> (forall s. Decoder s (ExtLedgerState blk))
-> (forall s. Decoder s (HeaderHash blk))
-> LedgerDbCfg (ExtLedgerState blk)
-> StreamAPI m blk blk
-> DiskSnapshot
-> ExceptT
     (SnapshotFailure blk) m (RealPoint blk, LedgerDB' blk, Word64)
initFromSnapshot Tracer m (ReplayGoal blk -> TraceReplayEvent blk)
tracer SomeHasFS m
hasFS forall s. Decoder s (ExtLedgerState blk)
decLedger forall s. Decoder s (HeaderHash blk)
decHash LedgerDbCfg (ExtLedgerState blk)
cfg StreamAPI m blk blk
stream DiskSnapshot
ss = do
    ExtLedgerState blk
initSS <- (ReadIncrementalErr -> SnapshotFailure blk)
-> ExceptT ReadIncrementalErr m (ExtLedgerState blk)
-> ExceptT (SnapshotFailure blk) m (ExtLedgerState blk)
forall (m :: * -> *) e e' a.
Functor m =>
(e -> e') -> ExceptT e m a -> ExceptT e' m a
withExceptT ReadIncrementalErr -> SnapshotFailure blk
forall blk. ReadIncrementalErr -> SnapshotFailure blk
InitFailureRead (ExceptT ReadIncrementalErr m (ExtLedgerState blk)
 -> ExceptT (SnapshotFailure blk) m (ExtLedgerState blk))
-> ExceptT ReadIncrementalErr m (ExtLedgerState blk)
-> ExceptT (SnapshotFailure blk) m (ExtLedgerState blk)
forall a b. (a -> b) -> a -> b
$
                SomeHasFS m
-> (forall s. Decoder s (ExtLedgerState blk))
-> (forall s. Decoder s (HeaderHash blk))
-> DiskSnapshot
-> ExceptT ReadIncrementalErr m (ExtLedgerState blk)
forall (m :: * -> *) blk.
IOLike m =>
SomeHasFS m
-> (forall s. Decoder s (ExtLedgerState blk))
-> (forall s. Decoder s (HeaderHash blk))
-> DiskSnapshot
-> ExceptT ReadIncrementalErr m (ExtLedgerState blk)
readSnapshot SomeHasFS m
hasFS Decoder s (ExtLedgerState blk)
forall s. Decoder s (ExtLedgerState blk)
decLedger Decoder s (HeaderHash blk)
forall s. Decoder s (HeaderHash blk)
decHash DiskSnapshot
ss
    let replayStart :: Point blk
replayStart = Point (ExtLedgerState blk) -> Point blk
forall {k1} {k2} (b :: k1) (b' :: k2).
Coercible (HeaderHash b) (HeaderHash b') =>
Point b -> Point b'
castPoint (Point (ExtLedgerState blk) -> Point blk)
-> Point (ExtLedgerState blk) -> Point blk
forall a b. (a -> b) -> a -> b
$ ExtLedgerState blk -> Point (ExtLedgerState blk)
forall l. GetTip l => l -> Point l
getTip ExtLedgerState blk
initSS
    case Point blk -> WithOrigin (RealPoint blk)
forall blk. Point blk -> WithOrigin (RealPoint blk)
pointToWithOriginRealPoint Point blk
replayStart of
      WithOrigin (RealPoint blk)
Origin -> SnapshotFailure blk
-> ExceptT
     (SnapshotFailure blk) m (RealPoint blk, LedgerDB' blk, Word64)
forall a. SnapshotFailure blk -> ExceptT (SnapshotFailure blk) m a
forall e (m :: * -> *) a. MonadError e m => e -> m a
throwError SnapshotFailure blk
forall blk. SnapshotFailure blk
InitFailureGenesis
      NotOrigin RealPoint blk
realReplayStart -> do
        let tracer' :: Tracer
  m (ReplayStart blk -> ReplayGoal blk -> TraceReplayEvent blk)
tracer' = Point blk
-> Tracer m (ReplayGoal blk -> TraceReplayEvent blk)
-> Tracer
     m (ReplayStart blk -> ReplayGoal blk -> TraceReplayEvent blk)
forall blk (m :: * -> *).
Point blk
-> Tracer m (ReplayGoal blk -> TraceReplayEvent blk)
-> Tracer
     m (ReplayStart blk -> ReplayGoal blk -> TraceReplayEvent blk)
decorateReplayTracerWithStart Point blk
replayStart Tracer m (ReplayGoal blk -> TraceReplayEvent blk)
tracer
        m () -> ExceptT (SnapshotFailure blk) m ()
forall (m :: * -> *) a.
Monad m =>
m a -> ExceptT (SnapshotFailure blk) m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (m () -> ExceptT (SnapshotFailure blk) m ())
-> m () -> ExceptT (SnapshotFailure blk) m ()
forall a b. (a -> b) -> a -> b
$ Tracer
  m (ReplayStart blk -> ReplayGoal blk -> TraceReplayEvent blk)
-> (ReplayStart blk -> ReplayGoal blk -> TraceReplayEvent blk)
-> m ()
forall (m :: * -> *) a. Tracer m a -> a -> m ()
traceWith Tracer
  m (ReplayStart blk -> ReplayGoal blk -> TraceReplayEvent blk)
tracer' ((ReplayStart blk -> ReplayGoal blk -> TraceReplayEvent blk)
 -> m ())
-> (ReplayStart blk -> ReplayGoal blk -> TraceReplayEvent blk)
-> m ()
forall a b. (a -> b) -> a -> b
$ DiskSnapshot
-> ReplayStart blk -> ReplayGoal blk -> TraceReplayEvent blk
forall blk.
DiskSnapshot
-> ReplayStart blk -> ReplayGoal blk -> TraceReplayEvent blk
ReplayFromSnapshot DiskSnapshot
ss
        (LedgerDB' blk
initDB, Word64
replayed) <-
          Tracer
  m (ReplayStart blk -> ReplayGoal blk -> TraceReplayEvent blk)
-> LedgerDbCfg (ExtLedgerState blk)
-> StreamAPI m blk blk
-> LedgerDB' blk
-> ExceptT (SnapshotFailure blk) m (LedgerDB' blk, Word64)
forall (m :: * -> *) blk.
(Monad m, LedgerSupportsProtocol blk, InspectLedger blk,
 HasCallStack) =>
Tracer
  m (ReplayStart blk -> ReplayGoal blk -> TraceReplayEvent blk)
-> LedgerDbCfg (ExtLedgerState blk)
-> StreamAPI m blk blk
-> LedgerDB' blk
-> ExceptT (SnapshotFailure blk) m (LedgerDB' blk, Word64)
initStartingWith
            Tracer
  m (ReplayStart blk -> ReplayGoal blk -> TraceReplayEvent blk)
tracer'
            LedgerDbCfg (ExtLedgerState blk)
cfg
            StreamAPI m blk blk
stream
            (ExtLedgerState blk -> LedgerDB' blk
forall l. GetTip l => l -> LedgerDB l
ledgerDbWithAnchor ExtLedgerState blk
initSS)
        (RealPoint blk, LedgerDB' blk, Word64)
-> ExceptT
     (SnapshotFailure blk) m (RealPoint blk, LedgerDB' blk, Word64)
forall a. a -> ExceptT (SnapshotFailure blk) m a
forall (m :: * -> *) a. Monad m => a -> m a
return (RealPoint blk
realReplayStart, LedgerDB' blk
initDB, Word64
replayed)

-- | Attempt to initialize the ledger DB starting from the given ledger DB
initStartingWith ::
     forall m blk. (
         Monad m
       , LedgerSupportsProtocol blk
       , InspectLedger blk
       , HasCallStack
       )
  => Tracer m (ReplayStart blk -> ReplayGoal blk -> TraceReplayEvent blk)
  -> LedgerDbCfg (ExtLedgerState blk)
  -> StreamAPI m blk blk
  -> LedgerDB' blk
  -> ExceptT (SnapshotFailure blk) m (LedgerDB' blk, Word64)
initStartingWith :: forall (m :: * -> *) blk.
(Monad m, LedgerSupportsProtocol blk, InspectLedger blk,
 HasCallStack) =>
Tracer
  m (ReplayStart blk -> ReplayGoal blk -> TraceReplayEvent blk)
-> LedgerDbCfg (ExtLedgerState blk)
-> StreamAPI m blk blk
-> LedgerDB' blk
-> ExceptT (SnapshotFailure blk) m (LedgerDB' blk, Word64)
initStartingWith Tracer
  m (ReplayStart blk -> ReplayGoal blk -> TraceReplayEvent blk)
tracer LedgerDbCfg (ExtLedgerState blk)
cfg StreamAPI m blk blk
stream LedgerDB' blk
initDb = do
    StreamAPI m blk blk
-> Point blk
-> (RealPoint blk -> SnapshotFailure blk)
-> (LedgerDB' blk, Word64)
-> (blk -> (LedgerDB' blk, Word64) -> m (LedgerDB' blk, Word64))
-> ExceptT (SnapshotFailure blk) m (LedgerDB' blk, Word64)
forall (m :: * -> *) blk e b a.
(Monad m, HasCallStack) =>
StreamAPI m blk b
-> Point blk
-> (RealPoint blk -> e)
-> a
-> (b -> a -> m a)
-> ExceptT e m a
streamAll StreamAPI m blk blk
stream (Point (ExtLedgerState blk) -> Point blk
forall {k1} {k2} (b :: k1) (b' :: k2).
Coercible (HeaderHash b) (HeaderHash b') =>
Point b -> Point b'
castPoint (LedgerDB' blk -> Point (ExtLedgerState blk)
forall l. GetTip l => LedgerDB l -> Point l
ledgerDbTip LedgerDB' blk
initDb))
      RealPoint blk -> SnapshotFailure blk
forall blk. RealPoint blk -> SnapshotFailure blk
InitFailureTooRecent
      (LedgerDB' blk
initDb, Word64
0)
      blk -> (LedgerDB' blk, Word64) -> m (LedgerDB' blk, Word64)
push
  where
    push :: blk -> (LedgerDB' blk, Word64) -> m (LedgerDB' blk, Word64)
    push :: blk -> (LedgerDB' blk, Word64) -> m (LedgerDB' blk, Word64)
push blk
blk !(!LedgerDB' blk
db, !Word64
replayed) = do
        !LedgerDB' blk
db' <- LedgerDbCfg (ExtLedgerState blk)
-> Ap m (ExtLedgerState blk) blk (() :: Constraint)
-> LedgerDB' blk
-> m (LedgerDB' blk)
forall (m :: * -> *) (c :: Constraint) l blk.
(ApplyBlock l blk, Monad m, c) =>
LedgerDbCfg l -> Ap m l blk c -> LedgerDB l -> m (LedgerDB l)
ledgerDbPush LedgerDbCfg (ExtLedgerState blk)
cfg (blk -> Ap m (ExtLedgerState blk) blk (() :: Constraint)
forall blk (m :: * -> *) l. blk -> Ap m l blk (() :: Constraint)
ReapplyVal blk
blk) LedgerDB' blk
db

        let replayed' :: Word64
            !replayed' :: Word64
replayed' = Word64
replayed Word64 -> Word64 -> Word64
forall a. Num a => a -> a -> a
+ Word64
1

            events :: [LedgerEvent blk]
            events :: [LedgerEvent blk]
events = TopLevelConfig blk
-> LedgerState blk -> LedgerState blk -> [LedgerEvent blk]
forall blk.
InspectLedger blk =>
TopLevelConfig blk
-> LedgerState blk -> LedgerState blk -> [LedgerEvent blk]
inspectLedger
                       (ExtLedgerCfg blk -> TopLevelConfig blk
forall blk. ExtLedgerCfg blk -> TopLevelConfig blk
getExtLedgerCfg (LedgerDbCfg (ExtLedgerState blk) -> LedgerCfg (ExtLedgerState blk)
forall l. LedgerDbCfg l -> LedgerCfg l
ledgerDbCfg LedgerDbCfg (ExtLedgerState blk)
cfg))
                       (ExtLedgerState blk -> LedgerState blk
forall blk. ExtLedgerState blk -> LedgerState blk
ledgerState (LedgerDB' blk -> ExtLedgerState blk
forall l. GetTip l => LedgerDB l -> l
ledgerDbCurrent LedgerDB' blk
db))
                       (ExtLedgerState blk -> LedgerState blk
forall blk. ExtLedgerState blk -> LedgerState blk
ledgerState (LedgerDB' blk -> ExtLedgerState blk
forall l. GetTip l => LedgerDB l -> l
ledgerDbCurrent LedgerDB' blk
db'))

        Tracer
  m (ReplayStart blk -> ReplayGoal blk -> TraceReplayEvent blk)
-> (ReplayStart blk -> ReplayGoal blk -> TraceReplayEvent blk)
-> m ()
forall (m :: * -> *) a. Tracer m a -> a -> m ()
traceWith Tracer
  m (ReplayStart blk -> ReplayGoal blk -> TraceReplayEvent blk)
tracer (RealPoint blk
-> [LedgerEvent blk]
-> ReplayStart blk
-> ReplayGoal blk
-> TraceReplayEvent blk
forall blk.
RealPoint blk
-> [LedgerEvent blk]
-> ReplayStart blk
-> ReplayGoal blk
-> TraceReplayEvent blk
ReplayedBlock (blk -> RealPoint blk
forall blk. HasHeader blk => blk -> RealPoint blk
blockRealPoint blk
blk) [LedgerEvent blk]
events)
        (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')

{-------------------------------------------------------------------------------
  Trace events
-------------------------------------------------------------------------------}

-- | Add the tip of the Immutable DB to the trace event
--
-- Between the tip of the immutable DB and the point of the starting block,
-- the node could (if it so desired) easily compute a "percentage complete".
decorateReplayTracerWithGoal ::
     Point blk -- ^ Tip of the ImmutableDB
  -> Tracer m (TraceReplayEvent blk)
  -> Tracer m (ReplayGoal blk -> TraceReplayEvent blk)
decorateReplayTracerWithGoal :: forall blk (m :: * -> *).
Point blk
-> Tracer m (TraceReplayEvent blk)
-> Tracer m (ReplayGoal blk -> TraceReplayEvent blk)
decorateReplayTracerWithGoal Point blk
immTip = ((ReplayGoal blk -> TraceReplayEvent blk) -> TraceReplayEvent blk)
-> Tracer m (TraceReplayEvent blk)
-> Tracer m (ReplayGoal blk -> TraceReplayEvent blk)
forall a' a. (a' -> a) -> Tracer m a -> Tracer m a'
forall (f :: * -> *) a' a.
Contravariant f =>
(a' -> a) -> f a -> f a'
contramap ((ReplayGoal blk -> TraceReplayEvent blk)
-> ReplayGoal blk -> TraceReplayEvent blk
forall a b. (a -> b) -> a -> b
$ (Point blk -> ReplayGoal blk
forall blk. Point blk -> ReplayGoal blk
ReplayGoal Point blk
immTip))

-- | Add the block at which a replay started.
--
-- This allows to compute a "percentage complete" when tracing the events.
decorateReplayTracerWithStart ::
     Point blk -- ^ Starting point of the replay
  -> Tracer m (ReplayGoal blk -> TraceReplayEvent blk)
  -> Tracer m (ReplayStart blk -> ReplayGoal blk -> TraceReplayEvent blk)
decorateReplayTracerWithStart :: forall blk (m :: * -> *).
Point blk
-> Tracer m (ReplayGoal blk -> TraceReplayEvent blk)
-> Tracer
     m (ReplayStart blk -> ReplayGoal blk -> TraceReplayEvent blk)
decorateReplayTracerWithStart Point blk
start = ((ReplayStart blk -> ReplayGoal blk -> TraceReplayEvent blk)
 -> ReplayGoal blk -> TraceReplayEvent blk)
-> Tracer m (ReplayGoal blk -> TraceReplayEvent blk)
-> Tracer
     m (ReplayStart blk -> ReplayGoal blk -> TraceReplayEvent blk)
forall a' a. (a' -> a) -> Tracer m a -> Tracer m a'
forall (f :: * -> *) a' a.
Contravariant f =>
(a' -> a) -> f a -> f a'
contramap ((ReplayStart blk -> ReplayGoal blk -> TraceReplayEvent blk)
-> ReplayStart blk -> ReplayGoal blk -> TraceReplayEvent blk
forall a b. (a -> b) -> a -> b
$ (Point blk -> ReplayStart blk
forall blk. Point blk -> ReplayStart blk
ReplayStart Point blk
start))

-- | Which point the replay started from
newtype ReplayStart blk = ReplayStart (Point blk) deriving (ReplayStart blk -> ReplayStart blk -> Bool
(ReplayStart blk -> ReplayStart blk -> Bool)
-> (ReplayStart blk -> ReplayStart blk -> Bool)
-> Eq (ReplayStart blk)
forall blk.
StandardHash blk =>
ReplayStart blk -> ReplayStart blk -> Bool
forall a. (a -> a -> Bool) -> (a -> a -> Bool) -> Eq a
$c== :: forall blk.
StandardHash blk =>
ReplayStart blk -> ReplayStart blk -> Bool
== :: ReplayStart blk -> ReplayStart blk -> Bool
$c/= :: forall blk.
StandardHash blk =>
ReplayStart blk -> ReplayStart blk -> Bool
/= :: ReplayStart blk -> ReplayStart blk -> Bool
Eq, Int -> ReplayStart blk -> ShowS
[ReplayStart blk] -> ShowS
ReplayStart blk -> String
(Int -> ReplayStart blk -> ShowS)
-> (ReplayStart blk -> String)
-> ([ReplayStart blk] -> ShowS)
-> Show (ReplayStart blk)
forall blk. StandardHash blk => Int -> ReplayStart blk -> ShowS
forall blk. StandardHash blk => [ReplayStart blk] -> ShowS
forall blk. StandardHash blk => ReplayStart blk -> String
forall a.
(Int -> a -> ShowS) -> (a -> String) -> ([a] -> ShowS) -> Show a
$cshowsPrec :: forall blk. StandardHash blk => Int -> ReplayStart blk -> ShowS
showsPrec :: Int -> ReplayStart blk -> ShowS
$cshow :: forall blk. StandardHash blk => ReplayStart blk -> String
show :: ReplayStart blk -> String
$cshowList :: forall blk. StandardHash blk => [ReplayStart blk] -> ShowS
showList :: [ReplayStart blk] -> ShowS
Show)

-- | Which point the replay is expected to end at
newtype ReplayGoal blk = ReplayGoal (Point blk) deriving (ReplayGoal blk -> ReplayGoal blk -> Bool
(ReplayGoal blk -> ReplayGoal blk -> Bool)
-> (ReplayGoal blk -> ReplayGoal blk -> Bool)
-> Eq (ReplayGoal blk)
forall blk.
StandardHash blk =>
ReplayGoal blk -> ReplayGoal blk -> Bool
forall a. (a -> a -> Bool) -> (a -> a -> Bool) -> Eq a
$c== :: forall blk.
StandardHash blk =>
ReplayGoal blk -> ReplayGoal blk -> Bool
== :: ReplayGoal blk -> ReplayGoal blk -> Bool
$c/= :: forall blk.
StandardHash blk =>
ReplayGoal blk -> ReplayGoal blk -> Bool
/= :: ReplayGoal blk -> ReplayGoal blk -> Bool
Eq, Int -> ReplayGoal blk -> ShowS
[ReplayGoal blk] -> ShowS
ReplayGoal blk -> String
(Int -> ReplayGoal blk -> ShowS)
-> (ReplayGoal blk -> String)
-> ([ReplayGoal blk] -> ShowS)
-> Show (ReplayGoal blk)
forall blk. StandardHash blk => Int -> ReplayGoal blk -> ShowS
forall blk. StandardHash blk => [ReplayGoal blk] -> ShowS
forall blk. StandardHash blk => ReplayGoal blk -> String
forall a.
(Int -> a -> ShowS) -> (a -> String) -> ([a] -> ShowS) -> Show a
$cshowsPrec :: forall blk. StandardHash blk => Int -> ReplayGoal blk -> ShowS
showsPrec :: Int -> ReplayGoal blk -> ShowS
$cshow :: forall blk. StandardHash blk => ReplayGoal blk -> String
show :: ReplayGoal blk -> String
$cshowList :: forall blk. StandardHash blk => [ReplayGoal blk] -> ShowS
showList :: [ReplayGoal blk] -> ShowS
Show)

-- | Events traced while replaying blocks against the ledger to bring it up to
-- date w.r.t. the tip of the ImmutableDB during initialisation. As this
-- process takes a while, we trace events to inform higher layers of our
-- progress.
data TraceReplayEvent blk
  = -- | There were no LedgerDB snapshots on disk, so we're replaying all blocks
    -- starting from Genesis against the initial ledger.
    ReplayFromGenesis
        (ReplayGoal blk)  -- ^ the block at the tip of the ImmutableDB
    -- | There was a LedgerDB snapshot on disk corresponding to the given tip.
    -- We're replaying more recent blocks against it.
  | ReplayFromSnapshot
        DiskSnapshot
        (ReplayStart blk) -- ^ the block at which this replay started
        (ReplayGoal blk)  -- ^ the block at the tip of the ImmutableDB
  -- | We replayed the given block (reference) on the genesis snapshot during
  -- the initialisation of the LedgerDB. Used during ImmutableDB replay.
  | ReplayedBlock
        (RealPoint blk)   -- ^ the block being replayed
        [LedgerEvent blk]
        (ReplayStart blk) -- ^ the block at which this replay started
        (ReplayGoal blk)  -- ^ the block at the tip of the ImmutableDB
  deriving ((forall x. TraceReplayEvent blk -> Rep (TraceReplayEvent blk) x)
-> (forall x. Rep (TraceReplayEvent blk) x -> TraceReplayEvent blk)
-> Generic (TraceReplayEvent blk)
forall x. Rep (TraceReplayEvent blk) x -> TraceReplayEvent blk
forall x. TraceReplayEvent blk -> Rep (TraceReplayEvent blk) x
forall a.
(forall x. a -> Rep a x) -> (forall x. Rep a x -> a) -> Generic a
forall blk x. Rep (TraceReplayEvent blk) x -> TraceReplayEvent blk
forall blk x. TraceReplayEvent blk -> Rep (TraceReplayEvent blk) x
$cfrom :: forall blk x. TraceReplayEvent blk -> Rep (TraceReplayEvent blk) x
from :: forall x. TraceReplayEvent blk -> Rep (TraceReplayEvent blk) x
$cto :: forall blk x. Rep (TraceReplayEvent blk) x -> TraceReplayEvent blk
to :: forall x. Rep (TraceReplayEvent blk) x -> TraceReplayEvent blk
Generic, TraceReplayEvent blk -> TraceReplayEvent blk -> Bool
(TraceReplayEvent blk -> TraceReplayEvent blk -> Bool)
-> (TraceReplayEvent blk -> TraceReplayEvent blk -> Bool)
-> Eq (TraceReplayEvent blk)
forall blk.
(StandardHash blk, InspectLedger blk) =>
TraceReplayEvent blk -> TraceReplayEvent blk -> Bool
forall a. (a -> a -> Bool) -> (a -> a -> Bool) -> Eq a
$c== :: forall blk.
(StandardHash blk, InspectLedger blk) =>
TraceReplayEvent blk -> TraceReplayEvent blk -> Bool
== :: TraceReplayEvent blk -> TraceReplayEvent blk -> Bool
$c/= :: forall blk.
(StandardHash blk, InspectLedger blk) =>
TraceReplayEvent blk -> TraceReplayEvent blk -> Bool
/= :: TraceReplayEvent blk -> TraceReplayEvent blk -> Bool
Eq, Int -> TraceReplayEvent blk -> ShowS
[TraceReplayEvent blk] -> ShowS
TraceReplayEvent blk -> String
(Int -> TraceReplayEvent blk -> ShowS)
-> (TraceReplayEvent blk -> String)
-> ([TraceReplayEvent blk] -> ShowS)
-> Show (TraceReplayEvent blk)
forall blk.
(StandardHash blk, InspectLedger blk) =>
Int -> TraceReplayEvent blk -> ShowS
forall blk.
(StandardHash blk, InspectLedger blk) =>
[TraceReplayEvent blk] -> ShowS
forall blk.
(StandardHash blk, InspectLedger blk) =>
TraceReplayEvent blk -> String
forall a.
(Int -> a -> ShowS) -> (a -> String) -> ([a] -> ShowS) -> Show a
$cshowsPrec :: forall blk.
(StandardHash blk, InspectLedger blk) =>
Int -> TraceReplayEvent blk -> ShowS
showsPrec :: Int -> TraceReplayEvent blk -> ShowS
$cshow :: forall blk.
(StandardHash blk, InspectLedger blk) =>
TraceReplayEvent blk -> String
show :: TraceReplayEvent blk -> String
$cshowList :: forall blk.
(StandardHash blk, InspectLedger blk) =>
[TraceReplayEvent blk] -> ShowS
showList :: [TraceReplayEvent blk] -> ShowS
Show)