{-# LANGUAGE FlexibleContexts #-}
{-# LANGUAGE GADTs #-}
{-# LANGUAGE LambdaCase #-}
{-# LANGUAGE MultiWayIf #-}
{-# LANGUAGE NamedFieldPuns #-}
{-# LANGUAGE RecordWildCards #-}
{-# LANGUAGE ScopedTypeVariables #-}
{-# LANGUAGE TypeApplications #-}
{-# LANGUAGE TypeOperators #-}
{-# LANGUAGE UndecidableInstances #-}

-- | Operations involving chain selection: the initial chain selection and
-- adding a block.
module Ouroboros.Consensus.Storage.ChainDB.Impl.ChainSel (
    addBlockAsync
  , chainSelSync
  , chainSelectionForBlock
  , initialChainSelection
  , triggerChainSelectionAsync
    -- * Exported for testing purposes
  , olderThanK
  ) where

import           Cardano.Ledger.BaseTypes (unNonZero)
import           Control.Exception (assert)
import           Control.Monad (forM, forM_, when)
import           Control.Monad.Except ()
import           Control.Monad.Trans.Class (lift)
import           Control.Monad.Trans.State.Strict
import           Control.ResourceRegistry (ResourceRegistry, withRegistry)
import           Control.Tracer (Tracer, nullTracer, traceWith)
import           Data.Foldable (for_)
import           Data.Function (on)
import           Data.Functor.Contravariant ((>$<))
import           Data.List (sortBy)
import           Data.List.NonEmpty (NonEmpty)
import qualified Data.List.NonEmpty as NE
import           Data.Map.Strict (Map)
import qualified Data.Map.Strict as Map
import           Data.Maybe (fromJust, isJust, isNothing)
import           Data.Maybe.Strict (StrictMaybe (..), strictMaybeToMaybe)
import           Data.Set (Set)
import qualified Data.Set as Set
import           GHC.Stack (HasCallStack)
import           Ouroboros.Consensus.Block
import           Ouroboros.Consensus.Config
import           Ouroboros.Consensus.Fragment.Diff (ChainDiff (..))
import qualified Ouroboros.Consensus.Fragment.Diff as Diff
import           Ouroboros.Consensus.Fragment.Validated (ValidatedFragment)
import qualified Ouroboros.Consensus.Fragment.Validated as VF
import           Ouroboros.Consensus.Fragment.ValidatedDiff
                     (ValidatedChainDiff (..))
import qualified Ouroboros.Consensus.Fragment.ValidatedDiff as ValidatedDiff
import           Ouroboros.Consensus.HardFork.Abstract
import qualified Ouroboros.Consensus.HardFork.History as History
import           Ouroboros.Consensus.HeaderValidation (HeaderWithTime (..),
                     mkHeaderWithTime)
import           Ouroboros.Consensus.Ledger.Abstract
import           Ouroboros.Consensus.Ledger.Extended
import           Ouroboros.Consensus.Ledger.Inspect
import           Ouroboros.Consensus.Ledger.SupportsProtocol
import           Ouroboros.Consensus.Storage.ChainDB.API (AddBlockPromise (..),
                     AddBlockResult (..), BlockComponent (..), ChainType (..),
                     LoE (..))
import           Ouroboros.Consensus.Storage.ChainDB.API.Types.InvalidBlockPunishment
                     (InvalidBlockPunishment, noPunishment)
import qualified Ouroboros.Consensus.Storage.ChainDB.API.Types.InvalidBlockPunishment as InvalidBlockPunishment
import           Ouroboros.Consensus.Storage.ChainDB.Impl.BlockCache
                     (BlockCache)
import qualified Ouroboros.Consensus.Storage.ChainDB.Impl.BlockCache as BlockCache
import           Ouroboros.Consensus.Storage.ChainDB.Impl.Paths
                     (LookupBlockInfo)
import qualified Ouroboros.Consensus.Storage.ChainDB.Impl.Paths as Paths
import qualified Ouroboros.Consensus.Storage.ChainDB.Impl.Query as Query
import           Ouroboros.Consensus.Storage.ChainDB.Impl.Types
import           Ouroboros.Consensus.Storage.ImmutableDB (ImmutableDB)
import qualified Ouroboros.Consensus.Storage.ImmutableDB as ImmutableDB
import           Ouroboros.Consensus.Storage.LedgerDB
import qualified Ouroboros.Consensus.Storage.LedgerDB as LedgerDB
import           Ouroboros.Consensus.Storage.VolatileDB (VolatileDB)
import qualified Ouroboros.Consensus.Storage.VolatileDB as VolatileDB
import           Ouroboros.Consensus.Util
import           Ouroboros.Consensus.Util.AnchoredFragment
import           Ouroboros.Consensus.Util.Enclose (encloseWith)
import           Ouroboros.Consensus.Util.IOLike
import           Ouroboros.Consensus.Util.STM (WithFingerprint (..))
import           Ouroboros.Network.AnchoredFragment (Anchor, AnchoredFragment,
                     AnchoredSeq (..))
import qualified Ouroboros.Network.AnchoredFragment as AF
import qualified Ouroboros.Network.AnchoredSeq as AS
import           Ouroboros.Network.Protocol.LocalStateQuery.Type (Target (..))

-- | Perform the initial chain selection based on the tip of the ImmutableDB
-- and the contents of the VolatileDB.
--
-- Returns the chosen validated chain and corresponding ledger.
--
initialChainSelection ::
     forall m blk.
     ( IOLike m
     , LedgerSupportsProtocol blk
     , BlockSupportsDiffusionPipelining blk
     )
  => ImmutableDB m blk
  -> VolatileDB m blk
  -> LedgerDB.LedgerDB' m blk
  -> ResourceRegistry m
  -> Tracer m (TraceInitChainSelEvent blk)
  -> TopLevelConfig blk
  -> StrictTVar m (WithFingerprint (InvalidBlocks blk))
  -> LoE ()
  -> m (ChainAndLedger m blk)
initialChainSelection :: forall (m :: * -> *) blk.
(IOLike m, LedgerSupportsProtocol blk,
 BlockSupportsDiffusionPipelining blk) =>
ImmutableDB m blk
-> VolatileDB m blk
-> LedgerDB' m blk
-> ResourceRegistry m
-> Tracer m (TraceInitChainSelEvent blk)
-> TopLevelConfig blk
-> StrictTVar m (WithFingerprint (InvalidBlocks blk))
-> LoE ()
-> m (ChainAndLedger m blk)
initialChainSelection ImmutableDB m blk
immutableDB VolatileDB m blk
volatileDB LedgerDB' m blk
lgrDB ResourceRegistry m
rr Tracer m (TraceInitChainSelEvent blk)
tracer TopLevelConfig blk
cfg StrictTVar m (WithFingerprint (InvalidBlocks blk))
varInvalid
                      LoE ()
loE = do
    -- TODO: Improve the user experience by trimming any potential
    -- blocks from the future from the VolatileDB.
    --
    -- When we perform chain selection, it is theoretically possible
    -- that the blocks in the VolatileDB are from the future, if for
    -- some reason the clock of the node was set back (by a
    -- significant amount of time). This is a rare situation, but can
    -- arise for instance if the clock of the node was set in the
    -- **far** future. In this case, node will be disconnected from
    -- other peers when diffusing these blocks. Once the node is
    -- restarted with a synchronized clock, it will diffuse said
    -- blocks from the future again (assuming they're still from the
    -- future after restart), which will cause other nodes to
    -- disconnect. By trimming blocks from the future from the
    -- VolatileDB we can prevent this inconvenient, albeit extremely
    -- rare, situation. However, it does not pose any security risk,
    -- and a node operator can correct the problem by either wiping
    -- out the VolatileDB or waiting enough time until the blocks are
    -- not from the **far** future anymore.
    (i :: Anchor blk, succsOf) <- STM m (Anchor blk, ChainHash blk -> Set (HeaderHash blk))
-> m (Anchor blk, ChainHash blk -> Set (HeaderHash blk))
forall a. HasCallStack => STM m a -> m a
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
STM m a -> m a
atomically (STM m (Anchor blk, ChainHash blk -> Set (HeaderHash blk))
 -> m (Anchor blk, ChainHash blk -> Set (HeaderHash blk)))
-> STM m (Anchor blk, ChainHash blk -> Set (HeaderHash blk))
-> m (Anchor blk, ChainHash blk -> Set (HeaderHash blk))
forall a b. (a -> b) -> a -> b
$ do
      invalid <- WithFingerprint (InvalidBlocks blk) -> InvalidBlocks blk
forall a. WithFingerprint a -> a
forgetFingerprint (WithFingerprint (InvalidBlocks blk) -> InvalidBlocks blk)
-> STM m (WithFingerprint (InvalidBlocks blk))
-> STM m (InvalidBlocks blk)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> StrictTVar m (WithFingerprint (InvalidBlocks blk))
-> STM m (WithFingerprint (InvalidBlocks blk))
forall (m :: * -> *) a. MonadSTM m => StrictTVar m a -> STM m a
readTVar StrictTVar m (WithFingerprint (InvalidBlocks blk))
varInvalid
      (,)
        <$> ImmutableDB.getTipAnchor immutableDB
        <*> (ignoreInvalidSuc volatileDB invalid <$>
              VolatileDB.filterByPredecessor volatileDB)

    -- This is safe: the LedgerDB tip doesn't change in between the previous
    -- atomically block and this call to 'withTipForker'.
    --
    -- We don't use 'LedgerDB.withTipForker' here, because 'curForker' might be
    -- returned as part of the selected chain.
    curForker <- LedgerDB.getForkerAtTarget lgrDB rr VolatileTip >>= \case
      Left{} -> [Char] -> m (Forker' m blk)
forall a. HasCallStack => [Char] -> a
error [Char]
"Unreachable, VolatileTip MUST be in the LedgerDB"
      Right Forker' m blk
frk -> Forker' m blk -> m (Forker' m blk)
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure Forker' m blk
frk

    chains <- constructChains i succsOf

    -- We use the empty fragment anchored at @i@ as the current chain (and
    -- ledger) and the default in case there is no better candidate.
    let curChain          = Anchor (Header blk) -> AnchoredFragment (Header blk)
forall v a b. Anchorable v a b => a -> AnchoredSeq v a b
Empty (Anchor blk -> Anchor (Header blk)
forall a b. (HeaderHash a ~ HeaderHash b) => Anchor a -> Anchor b
AF.castAnchor Anchor blk
i)
    curChainAndLedger <- VF.newM curChain curForker

    case NE.nonEmpty (filter (preferAnchoredCandidate bcfg curChain) chains) of
      -- If there are no candidates, no chain selection is needed
      Maybe (NonEmpty (AnchoredFragment (Header blk)))
Nothing      -> ValidatedFragment (Header blk) (Forker' m blk)
-> m (ValidatedFragment (Header blk) (Forker' m blk))
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return ValidatedFragment (Header blk) (Forker' m blk)
curChainAndLedger
      Just NonEmpty (AnchoredFragment (Header blk))
chains' ->
        HasCallStack =>
ValidatedFragment (Header blk) (Forker' m blk)
-> NonEmpty (AnchoredFragment (Header blk))
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
ValidatedFragment (Header blk) (Forker' m blk)
-> NonEmpty (AnchoredFragment (Header blk))
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
chainSelection' ValidatedFragment (Header blk) (Forker' m blk)
curChainAndLedger NonEmpty (AnchoredFragment (Header blk))
chains' m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
-> (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk))
    -> m (ValidatedFragment (Header blk) (Forker' m blk)))
-> m (ValidatedFragment (Header blk) (Forker' m blk))
forall a b. m a -> (a -> m b) -> m b
forall (m :: * -> *) a b. Monad m => m a -> (a -> m b) -> m b
>>= \case
          -- The returned forker will be closed in 'openDBInternal'.
          Maybe (ValidatedChainDiff (Header blk) (Forker' m blk))
Nothing       -> ValidatedFragment (Header blk) (Forker' m blk)
-> m (ValidatedFragment (Header blk) (Forker' m blk))
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure ValidatedFragment (Header blk) (Forker' m blk)
curChainAndLedger
          Just ValidatedChainDiff (Header blk) (Forker' m blk)
newChain -> Forker' m blk -> m ()
forall (m :: * -> *) (l :: LedgerStateKind) blk.
Forker m l blk -> m ()
forkerClose Forker' m blk
curForker m ()
-> m (ValidatedFragment (Header blk) (Forker' m blk))
-> m (ValidatedFragment (Header blk) (Forker' m blk))
forall a b. m a -> m b -> m b
forall (m :: * -> *) a b. Monad m => m a -> m b -> m b
>> ValidatedChainDiff (Header blk) (Forker' m blk)
-> m (ValidatedFragment (Header blk) (Forker' m blk))
toChainAndLedger ValidatedChainDiff (Header blk) (Forker' m blk)
newChain
  where
    bcfg :: BlockConfig blk
    bcfg :: BlockConfig blk
bcfg = TopLevelConfig blk -> BlockConfig blk
forall blk. TopLevelConfig blk -> BlockConfig blk
configBlock TopLevelConfig blk
cfg

    SecurityParam NonZero Word64
k = TopLevelConfig blk -> SecurityParam
forall blk.
ConsensusProtocol (BlockProtocol blk) =>
TopLevelConfig blk -> SecurityParam
configSecurityParam TopLevelConfig blk
cfg

    -- | Turn the 'ValidatedChainDiff' into a 'ChainAndLedger'.
    --
    -- The rollback of the 'ChainDiff' must be empty, as the suffix starts
    -- from the tip of the ImmutableDB, and we can't roll back past that tip.
    -- This is guaranteed by the fact that all constructed candidates start
    -- from this tip.
    toChainAndLedger
      :: ValidatedChainDiff (Header blk) (Forker' m blk)
      -> m (ChainAndLedger m blk)
    toChainAndLedger :: ValidatedChainDiff (Header blk) (Forker' m blk)
-> m (ValidatedFragment (Header blk) (Forker' m blk))
toChainAndLedger (ValidatedChainDiff ChainDiff (Header blk)
chainDiff Forker' m blk
ledger) =
      case ChainDiff (Header blk)
chainDiff of
        ChainDiff Word64
rollback AnchoredFragment (Header blk)
suffix
          | Word64
rollback Word64 -> Word64 -> Bool
forall a. Eq a => a -> a -> Bool
== Word64
0
          -> AnchoredFragment (Header blk)
-> Forker' m blk
-> m (ValidatedFragment (Header blk) (Forker' m blk))
forall (m :: * -> *) l b.
(MonadSTM m, GetTipSTM m l, HasHeader b,
 HeaderHash b ~ HeaderHash l, HasCallStack) =>
AnchoredFragment b -> l -> m (ValidatedFragment b l)
VF.newM AnchoredFragment (Header blk)
suffix Forker' m blk
ledger
          | Bool
otherwise
          -> [Char] -> m (ValidatedFragment (Header blk) (Forker' m blk))
forall a. HasCallStack => [Char] -> a
error [Char]
"constructed an initial chain with rollback"

    -- | Use the VolatileDB to construct all chains starting from the tip of
    -- the ImmutableDB.
    constructChains ::
         Anchor blk -- ^ Tip of the ImmutableDB, @i@
      -> (ChainHash blk -> Set (HeaderHash blk))
      -> m [AnchoredFragment (Header blk)]
    constructChains :: Anchor blk
-> (ChainHash blk -> Set (HeaderHash blk))
-> m [AnchoredFragment (Header blk)]
constructChains Anchor blk
i ChainHash blk -> Set (HeaderHash blk)
succsOf = (StateT
   (Map (HeaderHash blk) (Header blk))
   m
   [AnchoredFragment (Header blk)]
 -> Map (HeaderHash blk) (Header blk)
 -> m [AnchoredFragment (Header blk)])
-> Map (HeaderHash blk) (Header blk)
-> StateT
     (Map (HeaderHash blk) (Header blk))
     m
     [AnchoredFragment (Header blk)]
-> m [AnchoredFragment (Header blk)]
forall a b c. (a -> b -> c) -> b -> a -> c
flip StateT
  (Map (HeaderHash blk) (Header blk))
  m
  [AnchoredFragment (Header blk)]
-> Map (HeaderHash blk) (Header blk)
-> m [AnchoredFragment (Header blk)]
forall (m :: * -> *) s a. Monad m => StateT s m a -> s -> m a
evalStateT Map (HeaderHash blk) (Header blk)
forall k a. Map k a
Map.empty (StateT
   (Map (HeaderHash blk) (Header blk))
   m
   [AnchoredFragment (Header blk)]
 -> m [AnchoredFragment (Header blk)])
-> StateT
     (Map (HeaderHash blk) (Header blk))
     m
     [AnchoredFragment (Header blk)]
-> m [AnchoredFragment (Header blk)]
forall a b. (a -> b) -> a -> b
$
        (NonEmpty (HeaderHash blk)
 -> StateT
      (Map (HeaderHash blk) (Header blk))
      m
      (AnchoredFragment (Header blk)))
-> [NonEmpty (HeaderHash blk)]
-> StateT
     (Map (HeaderHash blk) (Header blk))
     m
     [AnchoredFragment (Header blk)]
forall (t :: * -> *) (m :: * -> *) a b.
(Traversable t, Monad m) =>
(a -> m b) -> t a -> m (t b)
forall (m :: * -> *) a b. Monad m => (a -> m b) -> [a] -> m [b]
mapM NonEmpty (HeaderHash blk)
-> StateT
     (Map (HeaderHash blk) (Header blk))
     m
     (AnchoredFragment (Header blk))
constructChain [NonEmpty (HeaderHash blk)]
suffixesAfterI
      where
        -- We now prevent selecting more than k blocks in maximalCandidates
        -- when the LoE is enabled to avoid circumventing the LoE on startup.
        -- Shutting down a syncing node and then restarting it should not cause
        -- it to select the longest chain the VolDB, since that chain might be
        -- adversarial (ie the LoE did not allow the node to select it when it
        -- arrived).
        suffixesAfterI :: [NonEmpty (HeaderHash blk)]
        suffixesAfterI :: [NonEmpty (HeaderHash blk)]
suffixesAfterI = (ChainHash blk -> Set (HeaderHash blk))
-> Maybe Word64 -> Point blk -> [NonEmpty (HeaderHash blk)]
forall blk.
(ChainHash blk -> Set (HeaderHash blk))
-> Maybe Word64 -> Point blk -> [NonEmpty (HeaderHash blk)]
Paths.maximalCandidates ChainHash blk -> Set (HeaderHash blk)
succsOf (NonZero Word64 -> Word64
forall a. NonZero a -> a
unNonZero (NonZero Word64 -> Word64)
-> Maybe (NonZero Word64) -> Maybe Word64
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Maybe (NonZero Word64)
limit) (Anchor blk -> Point blk
forall block. Anchor block -> Point block
AF.anchorToPoint Anchor blk
i)
          where
            limit :: Maybe (NonZero Word64)
limit = case LoE ()
loE of
              LoE ()
LoEDisabled   -> Maybe (NonZero Word64)
forall a. Maybe a
Nothing
              LoEEnabled () -> NonZero Word64 -> Maybe (NonZero Word64)
forall a. a -> Maybe a
Just NonZero Word64
k

        constructChain ::
             NonEmpty (HeaderHash blk)
          -> StateT (Map (HeaderHash blk) (Header blk))
                    m
                    (AnchoredFragment (Header blk))
        constructChain :: NonEmpty (HeaderHash blk)
-> StateT
     (Map (HeaderHash blk) (Header blk))
     m
     (AnchoredFragment (Header blk))
constructChain NonEmpty (HeaderHash blk)
hashes =
            Anchor (Header blk)
-> [Header blk] -> AnchoredFragment (Header blk)
forall v a b. Anchorable v a b => a -> [b] -> AnchoredSeq v a b
AF.fromOldestFirst (Anchor blk -> Anchor (Header blk)
forall a b. (HeaderHash a ~ HeaderHash b) => Anchor a -> Anchor b
AF.castAnchor Anchor blk
i) ([Header blk] -> AnchoredFragment (Header blk))
-> StateT (Map (HeaderHash blk) (Header blk)) m [Header blk]
-> StateT
     (Map (HeaderHash blk) (Header blk))
     m
     (AnchoredFragment (Header blk))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$>
            (HeaderHash blk
 -> StateT (Map (HeaderHash blk) (Header blk)) m (Header blk))
-> [HeaderHash blk]
-> StateT (Map (HeaderHash blk) (Header blk)) m [Header blk]
forall (t :: * -> *) (m :: * -> *) a b.
(Traversable t, Monad m) =>
(a -> m b) -> t a -> m (t b)
forall (m :: * -> *) a b. Monad m => (a -> m b) -> [a] -> m [b]
mapM (VolatileDB m blk
-> HeaderHash blk
-> StateT (Map (HeaderHash blk) (Header blk)) m (Header blk)
forall (m :: * -> *) blk.
(MonadThrow m, HasHeader blk) =>
VolatileDB m blk
-> HeaderHash blk
-> StateT (Map (HeaderHash blk) (Header blk)) m (Header blk)
getKnownHeaderThroughCache VolatileDB m blk
volatileDB) (NonEmpty (HeaderHash blk) -> [HeaderHash blk]
forall a. NonEmpty a -> [a]
NE.toList NonEmpty (HeaderHash blk)
hashes)

    -- | Perform chain selection (including validation) on the given
    -- candidates.
    --
    -- PRECONDITION: all candidates are anchored at @i@.
    --
    -- PRECONDITION: all candidates must be preferred over the current chain.
    chainSelection' ::
         HasCallStack
      => ChainAndLedger m blk
         -- ^ The current chain and ledger, corresponding to
         -- @i@.
      -> NonEmpty (AnchoredFragment (Header blk))
         -- ^ Candidates anchored at @i@
      -> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
    chainSelection' :: HasCallStack =>
ValidatedFragment (Header blk) (Forker' m blk)
-> NonEmpty (AnchoredFragment (Header blk))
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
chainSelection' ValidatedFragment (Header blk) (Forker' m blk)
curChainAndLedger NonEmpty (AnchoredFragment (Header blk))
candidates =
        STM m (Point blk) -> m (Point blk)
forall a. HasCallStack => STM m a -> m a
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
STM m a -> m a
atomically (Forker' m blk -> STM m (Point blk)
forall (l :: LedgerStateKind) blk (m :: * -> *).
(GetTip l, HeaderHash l ~ HeaderHash blk, Functor (STM m)) =>
Forker m l blk -> STM m (Point blk)
forkerCurrentPoint Forker' m blk
ledger) m (Point blk)
-> (Point blk
    -> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk))))
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
forall a b. m a -> (a -> m b) -> m b
forall (m :: * -> *) a b. Monad m => m a -> (a -> m b) -> m b
>>= \Point blk
curpt ->
        Bool
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
forall a. HasCallStack => Bool -> a -> a
assert ((AnchoredFragment (Header blk) -> Bool)
-> NonEmpty (AnchoredFragment (Header blk)) -> Bool
forall (t :: * -> *) a. Foldable t => (a -> Bool) -> t a -> Bool
all ((Point blk
curpt Point blk -> Point blk -> Bool
forall a. Eq a => a -> a -> Bool
==) (Point blk -> Bool)
-> (AnchoredFragment (Header blk) -> Point blk)
-> AnchoredFragment (Header blk)
-> Bool
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Point (Header blk) -> Point blk
forall {k1} {k2} (b :: k1) (b' :: k2).
Coercible (HeaderHash b) (HeaderHash b') =>
Point b -> Point b'
castPoint (Point (Header blk) -> Point blk)
-> (AnchoredFragment (Header blk) -> Point (Header blk))
-> AnchoredFragment (Header blk)
-> Point blk
forall b c a. (b -> c) -> (a -> b) -> a -> c
. AnchoredFragment (Header blk) -> Point (Header blk)
forall block. AnchoredFragment block -> Point block
AF.anchorPoint) NonEmpty (AnchoredFragment (Header blk))
candidates) (m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
 -> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk))))
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
forall a b. (a -> b) -> a -> b
$
        Bool
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
forall a. HasCallStack => Bool -> a -> a
assert ((AnchoredFragment (Header blk) -> Bool)
-> NonEmpty (AnchoredFragment (Header blk)) -> Bool
forall (t :: * -> *) a. Foldable t => (a -> Bool) -> t a -> Bool
all (BlockConfig blk
-> AnchoredFragment (Header blk)
-> AnchoredFragment (Header blk)
-> Bool
forall blk (h :: * -> *) (h' :: * -> *).
(BlockSupportsProtocol blk, HasCallStack, GetHeader1 h,
 GetHeader1 h', HeaderHash (h blk) ~ HeaderHash (h' blk),
 HasHeader (h blk), HasHeader (h' blk)) =>
BlockConfig blk
-> AnchoredFragment (h blk) -> AnchoredFragment (h' blk) -> Bool
preferAnchoredCandidate BlockConfig blk
bcfg AnchoredFragment (Header blk)
curChain) NonEmpty (AnchoredFragment (Header blk))
candidates) (m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
 -> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk))))
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
forall a b. (a -> b) -> a -> b
$ do
          cse <- m (ChainSelEnv m blk)
chainSelEnv
          chainSelection cse rr (Diff.extend <$> candidates)
      where
        curChain :: AnchoredFragment (Header blk)
curChain = ValidatedFragment (Header blk) (Forker' m blk)
-> AnchoredFragment (Header blk)
forall b l. ValidatedFragment b l -> AnchoredFragment b
VF.validatedFragment ValidatedFragment (Header blk) (Forker' m blk)
curChainAndLedger
        ledger :: Forker' m blk
ledger   = ValidatedFragment (Header blk) (Forker' m blk) -> Forker' m blk
forall b l. ValidatedFragment b l -> l
VF.validatedLedger   ValidatedFragment (Header blk) (Forker' m blk)
curChainAndLedger
        chainSelEnv :: m (ChainSelEnv m blk)
chainSelEnv = do
          varTentativeState  <- TentativeHeaderState blk
-> m (StrictTVar m (TentativeHeaderState blk))
forall (m :: * -> *) a.
(HasCallStack, MonadSTM m, NoThunks a) =>
a -> m (StrictTVar m a)
newTVarIO (Proxy blk -> TentativeHeaderState blk
forall blk.
BlockSupportsDiffusionPipelining blk =>
Proxy blk -> TentativeHeaderState blk
initialTentativeHeaderState (forall t. Proxy t
forall {k} (t :: k). Proxy t
Proxy @blk))
          varTentativeHeader <- newTVarIO SNothing
          pure ChainSelEnv
            { lgrDB
            , bcfg
            , varInvalid
            , blockCache = BlockCache.empty
            , curChainAndLedger
            , validationTracer = InitChainSelValidation >$< tracer
              -- initial chain selection is not concerned about pipelining
            , pipeliningTracer = nullTracer
            , varTentativeState
            , varTentativeHeader
            , punish = Nothing
            , getTentativeFollowers = pure []
            }

-- | Add a block to the ChainDB, /asynchronously/.
--
-- This adds a 'BlockToAdd' corresponding to the given block to the
-- 'cdbChainSelQueue' queue. The entries in that queue are processed using
-- 'chainSelSync', see that function for more information.
--
-- When the queue is full, this function will still block.
--
-- Compared to a synchronous approach, the asynchronous counterpart
-- doesn't have the following disadvantage: when a thread adding a
-- block to the ChainDB is killed, which can happen when disconnecting
-- from the corresponding node, we might have written the block to
-- disk, but not updated the corresponding in-memory state (e.g., that
-- of the VolatileDB), leaving both out of sync.
--
-- With this asynchronous approach, threads adding blocks asynchronously can
-- be killed without worries, the background thread processing the blocks
-- synchronously won't be killed. Only when the whole ChainDB shuts down will
-- that background thread get killed. But since there will be no more
-- in-memory state, it can't get out of sync with the file system state. On
-- the next startup, a correct in-memory state will be reconstructed from the
-- file system state.
--
-- PRECONDITON: the block to be added must not be from the future.
-- See 'Ouroboros.Consensus.Storage.ChainDB.API.addBlockAsync'.
--
addBlockAsync ::
     forall m blk. (IOLike m, HasHeader blk)
  => ChainDbEnv m blk
  -> InvalidBlockPunishment m
  -> blk
  -> m (AddBlockPromise m blk)
addBlockAsync :: forall (m :: * -> *) blk.
(IOLike m, HasHeader blk) =>
ChainDbEnv m blk
-> InvalidBlockPunishment m -> blk -> m (AddBlockPromise m blk)
addBlockAsync CDB { Tracer m (TraceEvent blk)
cdbTracer :: Tracer m (TraceEvent blk)
cdbTracer :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> Tracer m (TraceEvent blk)
cdbTracer, ChainSelQueue m blk
cdbChainSelQueue :: forall (m :: * -> *) blk. ChainDbEnv m blk -> ChainSelQueue m blk
cdbChainSelQueue :: ChainSelQueue m blk
cdbChainSelQueue } =
    Tracer m (TraceAddBlockEvent blk)
-> ChainSelQueue m blk
-> InvalidBlockPunishment m
-> blk
-> m (AddBlockPromise m blk)
forall (m :: * -> *) blk.
(IOLike m, HasHeader blk) =>
Tracer m (TraceAddBlockEvent blk)
-> ChainSelQueue m blk
-> InvalidBlockPunishment m
-> blk
-> m (AddBlockPromise m blk)
addBlockToAdd (TraceAddBlockEvent blk -> TraceEvent blk
forall blk. TraceAddBlockEvent blk -> TraceEvent blk
TraceAddBlockEvent (TraceAddBlockEvent blk -> TraceEvent blk)
-> Tracer m (TraceEvent blk) -> Tracer m (TraceAddBlockEvent blk)
forall (f :: * -> *) a b. Contravariant f => (a -> b) -> f b -> f a
>$< Tracer m (TraceEvent blk)
cdbTracer) ChainSelQueue m blk
cdbChainSelQueue

-- | Schedule reprocessing of blocks postponed by the LoE.
triggerChainSelectionAsync ::
  forall m blk.
  IOLike m =>
  ChainDbEnv m blk ->
  m (ChainSelectionPromise m)
triggerChainSelectionAsync :: forall (m :: * -> *) blk.
IOLike m =>
ChainDbEnv m blk -> m (ChainSelectionPromise m)
triggerChainSelectionAsync CDB {Tracer m (TraceEvent blk)
cdbTracer :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> Tracer m (TraceEvent blk)
cdbTracer :: Tracer m (TraceEvent blk)
cdbTracer, ChainSelQueue m blk
cdbChainSelQueue :: forall (m :: * -> *) blk. ChainDbEnv m blk -> ChainSelQueue m blk
cdbChainSelQueue :: ChainSelQueue m blk
cdbChainSelQueue} =
  Tracer m (TraceAddBlockEvent blk)
-> ChainSelQueue m blk -> m (ChainSelectionPromise m)
forall (m :: * -> *) blk.
IOLike m =>
Tracer m (TraceAddBlockEvent blk)
-> ChainSelQueue m blk -> m (ChainSelectionPromise m)
addReprocessLoEBlocks (TraceAddBlockEvent blk -> TraceEvent blk
forall blk. TraceAddBlockEvent blk -> TraceEvent blk
TraceAddBlockEvent (TraceAddBlockEvent blk -> TraceEvent blk)
-> Tracer m (TraceEvent blk) -> Tracer m (TraceAddBlockEvent blk)
forall (f :: * -> *) a b. Contravariant f => (a -> b) -> f b -> f a
>$< Tracer m (TraceEvent blk)
cdbTracer) ChainSelQueue m blk
cdbChainSelQueue

-- | Add a block to the ChainDB, /synchronously/.
--
-- This is the only operation that actually changes the ChainDB. It will store
-- the block on disk and trigger chain selection, possibly switching to a
-- fork.
--
-- When the slot of the block is > the current slot, a chain selection will be
-- scheduled in the slot of the block.
chainSelSync ::
     forall m blk.
     ( IOLike m
     , LedgerSupportsProtocol blk
     , BlockSupportsDiffusionPipelining blk
     , InspectLedger blk
     , HasHardForkHistory blk
     , HasCallStack
     )
  => ChainDbEnv m blk
  -> ChainSelMessage m blk
  -> Electric m ()

-- Reprocess headers that were postponed by the LoE.
-- When we try to extend the current chain with a new block beyond the LoE
-- limit, the block will be added to the DB without modifying the chain.
-- When the LoE fragment advances later, these blocks have to be scheduled
-- for ChainSel again, but this does not happen automatically.
-- So we fetch all direct successors of each of the chain's blocks and run
-- ChainSel for them.
-- We run a background thread that polls the candidate fragments and sends
-- 'ChainSelReprocessLoEBlocks' whenever we receive a new header or lose a
-- peer.
-- If 'cdbLoE' is 'LoEDisabled', this task is skipped.
chainSelSync :: forall (m :: * -> *) blk.
(IOLike m, LedgerSupportsProtocol blk,
 BlockSupportsDiffusionPipelining blk, InspectLedger blk,
 HasHardForkHistory blk, HasCallStack) =>
ChainDbEnv m blk -> ChainSelMessage m blk -> Electric m ()
chainSelSync cdb :: ChainDbEnv m blk
cdb@CDB{m (LoE (AnchoredFragment (HeaderWithTime blk)))
Tracer m (TraceEvent blk)
DiffTime
ResourceRegistry m
StrictTVar m (m ())
StrictTVar m (Map FollowerKey (FollowerHandle m blk))
StrictTVar m (Map IteratorKey (m ()))
StrictTVar m (StrictMaybe (Header blk))
StrictTVar m ChainSelStarvation
StrictTVar m (TentativeHeaderState blk)
StrictTVar m (WithFingerprint (InvalidBlocks blk))
StrictTVar m FollowerKey
StrictTVar m IteratorKey
StrictTVar m (InternalChain blk)
Fuse m
TopLevelConfig blk
VolatileDB m blk
ImmutableDB m blk
LedgerDB' m blk
ChainSelQueue m blk
cdbChainSelQueue :: forall (m :: * -> *) blk. ChainDbEnv m blk -> ChainSelQueue m blk
cdbTracer :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> Tracer m (TraceEvent blk)
cdbImmutableDB :: ImmutableDB m blk
cdbVolatileDB :: VolatileDB m blk
cdbLedgerDB :: LedgerDB' m blk
cdbChain :: StrictTVar m (InternalChain blk)
cdbTentativeState :: StrictTVar m (TentativeHeaderState blk)
cdbTentativeHeader :: StrictTVar m (StrictMaybe (Header blk))
cdbIterators :: StrictTVar m (Map IteratorKey (m ()))
cdbFollowers :: StrictTVar m (Map FollowerKey (FollowerHandle m blk))
cdbTopLevelConfig :: TopLevelConfig blk
cdbInvalid :: StrictTVar m (WithFingerprint (InvalidBlocks blk))
cdbNextIteratorKey :: StrictTVar m IteratorKey
cdbNextFollowerKey :: StrictTVar m FollowerKey
cdbCopyFuse :: Fuse m
cdbChainSelFuse :: Fuse m
cdbTracer :: Tracer m (TraceEvent blk)
cdbRegistry :: ResourceRegistry m
cdbGcDelay :: DiffTime
cdbGcInterval :: DiffTime
cdbKillBgThreads :: StrictTVar m (m ())
cdbChainSelQueue :: ChainSelQueue m blk
cdbLoE :: m (LoE (AnchoredFragment (HeaderWithTime blk)))
cdbChainSelStarvation :: StrictTVar m ChainSelStarvation
cdbChainSelStarvation :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> StrictTVar m ChainSelStarvation
cdbLoE :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> m (LoE (AnchoredFragment (HeaderWithTime blk)))
cdbKillBgThreads :: forall (m :: * -> *) blk. ChainDbEnv m blk -> StrictTVar m (m ())
cdbGcInterval :: forall (m :: * -> *) blk. ChainDbEnv m blk -> DiffTime
cdbGcDelay :: forall (m :: * -> *) blk. ChainDbEnv m blk -> DiffTime
cdbRegistry :: forall (m :: * -> *) blk. ChainDbEnv m blk -> ResourceRegistry m
cdbChainSelFuse :: forall (m :: * -> *) blk. ChainDbEnv m blk -> Fuse m
cdbCopyFuse :: forall (m :: * -> *) blk. ChainDbEnv m blk -> Fuse m
cdbNextFollowerKey :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> StrictTVar m FollowerKey
cdbNextIteratorKey :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> StrictTVar m IteratorKey
cdbInvalid :: forall (m :: * -> *) blk.
ChainDbEnv m blk
-> StrictTVar m (WithFingerprint (InvalidBlocks blk))
cdbTopLevelConfig :: forall (m :: * -> *) blk. ChainDbEnv m blk -> TopLevelConfig blk
cdbFollowers :: forall (m :: * -> *) blk.
ChainDbEnv m blk
-> StrictTVar m (Map FollowerKey (FollowerHandle m blk))
cdbIterators :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> StrictTVar m (Map IteratorKey (m ()))
cdbTentativeHeader :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> StrictTVar m (StrictMaybe (Header blk))
cdbTentativeState :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> StrictTVar m (TentativeHeaderState blk)
cdbChain :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> StrictTVar m (InternalChain blk)
cdbLedgerDB :: forall (m :: * -> *) blk. ChainDbEnv m blk -> LedgerDB' m blk
cdbVolatileDB :: forall (m :: * -> *) blk. ChainDbEnv m blk -> VolatileDB m blk
cdbImmutableDB :: forall (m :: * -> *) blk. ChainDbEnv m blk -> ImmutableDB m blk
..} (ChainSelReprocessLoEBlocks StrictTMVar m ()
varProcessed) = do
    m (LoE (AnchoredFragment (HeaderWithTime blk)))
-> Electric m (LoE (AnchoredFragment (HeaderWithTime blk)))
forall (m :: * -> *) a. Monad m => m a -> Electric m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift m (LoE (AnchoredFragment (HeaderWithTime blk)))
cdbLoE Electric m (LoE (AnchoredFragment (HeaderWithTime blk)))
-> (LoE (AnchoredFragment (HeaderWithTime blk)) -> Electric m ())
-> Electric m ()
forall a b. Electric m a -> (a -> Electric m b) -> Electric m b
forall (m :: * -> *) a b. Monad m => m a -> (a -> m b) -> m b
>>= \case
      LoE (AnchoredFragment (HeaderWithTime blk))
LoEDisabled  -> () -> Electric m ()
forall a. a -> Electric m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure ()
      LoEEnabled AnchoredFragment (HeaderWithTime blk)
_ -> do
        (succsOf, chain) <- m (ChainHash blk -> Set (HeaderHash blk),
   AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
-> Electric
     m
     (ChainHash blk -> Set (HeaderHash blk),
      AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
forall (m :: * -> *) a. Monad m => m a -> Electric m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (m (ChainHash blk -> Set (HeaderHash blk),
    AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
 -> Electric
      m
      (ChainHash blk -> Set (HeaderHash blk),
       AnchoredSeq
         (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)))
-> m (ChainHash blk -> Set (HeaderHash blk),
      AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
-> Electric
     m
     (ChainHash blk -> Set (HeaderHash blk),
      AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
forall a b. (a -> b) -> a -> b
$ STM
  m
  (ChainHash blk -> Set (HeaderHash blk),
   AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
-> m (ChainHash blk -> Set (HeaderHash blk),
      AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
forall a. HasCallStack => STM m a -> m a
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
STM m a -> m a
atomically (STM
   m
   (ChainHash blk -> Set (HeaderHash blk),
    AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
 -> m (ChainHash blk -> Set (HeaderHash blk),
       AnchoredSeq
         (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)))
-> STM
     m
     (ChainHash blk -> Set (HeaderHash blk),
      AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
-> m (ChainHash blk -> Set (HeaderHash blk),
      AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
forall a b. (a -> b) -> a -> b
$ do
          invalid <- WithFingerprint (InvalidBlocks blk) -> InvalidBlocks blk
forall a. WithFingerprint a -> a
forgetFingerprint (WithFingerprint (InvalidBlocks blk) -> InvalidBlocks blk)
-> STM m (WithFingerprint (InvalidBlocks blk))
-> STM m (InvalidBlocks blk)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> StrictTVar m (WithFingerprint (InvalidBlocks blk))
-> STM m (WithFingerprint (InvalidBlocks blk))
forall (m :: * -> *) a. MonadSTM m => StrictTVar m a -> STM m a
readTVar StrictTVar m (WithFingerprint (InvalidBlocks blk))
cdbInvalid
          (,)
            <$> (ignoreInvalidSuc cdbVolatileDB invalid <$>
              VolatileDB.filterByPredecessor cdbVolatileDB)
            <*> Query.getCurrentChain cdb
        let
            succsOf' = Set (HeaderHash blk) -> [HeaderHash blk]
forall a. Set a -> [a]
Set.toList (Set (HeaderHash blk) -> [HeaderHash blk])
-> (Point (Header blk) -> Set (HeaderHash blk))
-> Point (Header blk)
-> [HeaderHash blk]
forall b c a. (b -> c) -> (a -> b) -> a -> c
. ChainHash blk -> Set (HeaderHash blk)
succsOf (ChainHash blk -> Set (HeaderHash blk))
-> (Point (Header blk) -> ChainHash blk)
-> Point (Header blk)
-> Set (HeaderHash blk)
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Point blk -> ChainHash blk
forall {k} (block :: k). Point block -> ChainHash block
pointHash (Point blk -> ChainHash blk)
-> (Point (Header blk) -> Point blk)
-> Point (Header blk)
-> ChainHash blk
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Point (Header blk) -> Point blk
forall {k1} {k2} (b :: k1) (b' :: k2).
Coercible (HeaderHash b) (HeaderHash b') =>
Point b -> Point b'
castPoint
            loeHashes = Point (Header blk) -> [HeaderHash blk]
succsOf' (AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> Point (Header blk)
forall block. AnchoredFragment block -> Point block
AF.anchorPoint AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
chain)
            firstHeader = (Anchor (Header blk) -> Maybe (Header blk))
-> (Header blk -> Maybe (Header blk))
-> Either (Anchor (Header blk)) (Header blk)
-> Maybe (Header blk)
forall a c b. (a -> c) -> (b -> c) -> Either a b -> c
either (Maybe (Header blk) -> Anchor (Header blk) -> Maybe (Header blk)
forall a b. a -> b -> a
const Maybe (Header blk)
forall a. Maybe a
Nothing) Header blk -> Maybe (Header blk)
forall a. a -> Maybe a
Just (Either (Anchor (Header blk)) (Header blk) -> Maybe (Header blk))
-> Either (Anchor (Header blk)) (Header blk) -> Maybe (Header blk)
forall a b. (a -> b) -> a -> b
$ AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> Either (Anchor (Header blk)) (Header blk)
forall v a b. Anchorable v a b => AnchoredSeq v a b -> Either a b
AF.last AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
chain
            -- We avoid the VolatileDB for the headers we already have in the chain
            getHeaderFromHash HeaderHash blk
hash =
              case Maybe (Header blk)
firstHeader of
                Just Header blk
header | Header blk -> HeaderHash blk
forall blk. HasHeader (Header blk) => Header blk -> HeaderHash blk
headerHash Header blk
header HeaderHash blk -> HeaderHash blk -> Bool
forall a. Eq a => a -> a -> Bool
== HeaderHash blk
hash -> Header blk -> m (Header blk)
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure Header blk
header
                Maybe (Header blk)
_ -> VolatileDB m blk
-> BlockComponent blk (Header blk)
-> HeaderHash blk
-> m (Header blk)
forall (m :: * -> *) blk b.
(MonadThrow m, HasHeader blk) =>
VolatileDB m blk -> BlockComponent blk b -> HeaderHash blk -> m b
VolatileDB.getKnownBlockComponent VolatileDB m blk
cdbVolatileDB BlockComponent blk (Header blk)
forall blk. BlockComponent blk (Header blk)
GetHeader HeaderHash blk
hash
        loeHeaders <- lift (mapM getHeaderFromHash loeHashes)
        for_ loeHeaders $ \Header blk
hdr ->
          ChainDbEnv m blk
-> BlockCache blk
-> Header blk
-> InvalidBlockPunishment m
-> Electric m ()
forall (m :: * -> *) blk.
(IOLike m, LedgerSupportsProtocol blk,
 BlockSupportsDiffusionPipelining blk, InspectLedger blk,
 HasHardForkHistory blk, HasCallStack) =>
ChainDbEnv m blk
-> BlockCache blk
-> Header blk
-> InvalidBlockPunishment m
-> Electric m ()
chainSelectionForBlock ChainDbEnv m blk
cdb BlockCache blk
forall blk. BlockCache blk
BlockCache.empty Header blk
hdr InvalidBlockPunishment m
forall (m :: * -> *). Applicative m => InvalidBlockPunishment m
noPunishment
    m () -> Electric m ()
forall (m :: * -> *) a. Monad m => m a -> Electric m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (m () -> Electric m ()) -> m () -> Electric m ()
forall a b. (a -> b) -> a -> b
$ 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
$ StrictTMVar m () -> () -> STM m ()
forall (m :: * -> *) a.
MonadSTM m =>
StrictTMVar m a -> a -> STM m ()
putTMVar StrictTMVar m ()
varProcessed ()

chainSelSync cdb :: ChainDbEnv m blk
cdb@CDB {m (LoE (AnchoredFragment (HeaderWithTime blk)))
Tracer m (TraceEvent blk)
DiffTime
ResourceRegistry m
StrictTVar m (m ())
StrictTVar m (Map FollowerKey (FollowerHandle m blk))
StrictTVar m (Map IteratorKey (m ()))
StrictTVar m (StrictMaybe (Header blk))
StrictTVar m ChainSelStarvation
StrictTVar m (TentativeHeaderState blk)
StrictTVar m (WithFingerprint (InvalidBlocks blk))
StrictTVar m FollowerKey
StrictTVar m IteratorKey
StrictTVar m (InternalChain blk)
Fuse m
TopLevelConfig blk
VolatileDB m blk
ImmutableDB m blk
LedgerDB' m blk
ChainSelQueue m blk
cdbChainSelQueue :: forall (m :: * -> *) blk. ChainDbEnv m blk -> ChainSelQueue m blk
cdbTracer :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> Tracer m (TraceEvent blk)
cdbChainSelStarvation :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> StrictTVar m ChainSelStarvation
cdbLoE :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> m (LoE (AnchoredFragment (HeaderWithTime blk)))
cdbKillBgThreads :: forall (m :: * -> *) blk. ChainDbEnv m blk -> StrictTVar m (m ())
cdbGcInterval :: forall (m :: * -> *) blk. ChainDbEnv m blk -> DiffTime
cdbGcDelay :: forall (m :: * -> *) blk. ChainDbEnv m blk -> DiffTime
cdbRegistry :: forall (m :: * -> *) blk. ChainDbEnv m blk -> ResourceRegistry m
cdbChainSelFuse :: forall (m :: * -> *) blk. ChainDbEnv m blk -> Fuse m
cdbCopyFuse :: forall (m :: * -> *) blk. ChainDbEnv m blk -> Fuse m
cdbNextFollowerKey :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> StrictTVar m FollowerKey
cdbNextIteratorKey :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> StrictTVar m IteratorKey
cdbInvalid :: forall (m :: * -> *) blk.
ChainDbEnv m blk
-> StrictTVar m (WithFingerprint (InvalidBlocks blk))
cdbTopLevelConfig :: forall (m :: * -> *) blk. ChainDbEnv m blk -> TopLevelConfig blk
cdbFollowers :: forall (m :: * -> *) blk.
ChainDbEnv m blk
-> StrictTVar m (Map FollowerKey (FollowerHandle m blk))
cdbIterators :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> StrictTVar m (Map IteratorKey (m ()))
cdbTentativeHeader :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> StrictTVar m (StrictMaybe (Header blk))
cdbTentativeState :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> StrictTVar m (TentativeHeaderState blk)
cdbChain :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> StrictTVar m (InternalChain blk)
cdbLedgerDB :: forall (m :: * -> *) blk. ChainDbEnv m blk -> LedgerDB' m blk
cdbVolatileDB :: forall (m :: * -> *) blk. ChainDbEnv m blk -> VolatileDB m blk
cdbImmutableDB :: forall (m :: * -> *) blk. ChainDbEnv m blk -> ImmutableDB m blk
cdbImmutableDB :: ImmutableDB m blk
cdbVolatileDB :: VolatileDB m blk
cdbLedgerDB :: LedgerDB' m blk
cdbChain :: StrictTVar m (InternalChain blk)
cdbTentativeState :: StrictTVar m (TentativeHeaderState blk)
cdbTentativeHeader :: StrictTVar m (StrictMaybe (Header blk))
cdbIterators :: StrictTVar m (Map IteratorKey (m ()))
cdbFollowers :: StrictTVar m (Map FollowerKey (FollowerHandle m blk))
cdbTopLevelConfig :: TopLevelConfig blk
cdbInvalid :: StrictTVar m (WithFingerprint (InvalidBlocks blk))
cdbNextIteratorKey :: StrictTVar m IteratorKey
cdbNextFollowerKey :: StrictTVar m FollowerKey
cdbCopyFuse :: Fuse m
cdbChainSelFuse :: Fuse m
cdbTracer :: Tracer m (TraceEvent blk)
cdbRegistry :: ResourceRegistry m
cdbGcDelay :: DiffTime
cdbGcInterval :: DiffTime
cdbKillBgThreads :: StrictTVar m (m ())
cdbChainSelQueue :: ChainSelQueue m blk
cdbLoE :: m (LoE (AnchoredFragment (HeaderWithTime blk)))
cdbChainSelStarvation :: StrictTVar m ChainSelStarvation
..} (ChainSelAddBlock BlockToAdd { blockToAdd :: forall (m :: * -> *) blk. BlockToAdd m blk -> blk
blockToAdd = blk
b, StrictTMVar m Bool
StrictTMVar m (AddBlockResult blk)
InvalidBlockPunishment m
blockPunish :: InvalidBlockPunishment m
varBlockWrittenToDisk :: StrictTMVar m Bool
varBlockProcessed :: StrictTMVar m (AddBlockResult blk)
varBlockProcessed :: forall (m :: * -> *) blk.
BlockToAdd m blk -> StrictTMVar m (AddBlockResult blk)
varBlockWrittenToDisk :: forall (m :: * -> *) blk. BlockToAdd m blk -> StrictTMVar m Bool
blockPunish :: forall (m :: * -> *) blk.
BlockToAdd m blk -> InvalidBlockPunishment m
.. }) = do
    (isMember, invalid, curChain) <- m (HeaderHash blk -> Bool, InvalidBlocks blk,
   AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
-> Electric
     m
     (HeaderHash blk -> Bool, InvalidBlocks blk,
      AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
forall (m :: * -> *) a. Monad m => m a -> Electric m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (m (HeaderHash blk -> Bool, InvalidBlocks blk,
    AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
 -> Electric
      m
      (HeaderHash blk -> Bool, InvalidBlocks blk,
       AnchoredSeq
         (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)))
-> m (HeaderHash blk -> Bool, InvalidBlocks blk,
      AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
-> Electric
     m
     (HeaderHash blk -> Bool, InvalidBlocks blk,
      AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
forall a b. (a -> b) -> a -> b
$ STM
  m
  (HeaderHash blk -> Bool, InvalidBlocks blk,
   AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
-> m (HeaderHash blk -> Bool, InvalidBlocks blk,
      AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
forall a. HasCallStack => STM m a -> m a
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
STM m a -> m a
atomically (STM
   m
   (HeaderHash blk -> Bool, InvalidBlocks blk,
    AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
 -> m (HeaderHash blk -> Bool, InvalidBlocks blk,
       AnchoredSeq
         (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)))
-> STM
     m
     (HeaderHash blk -> Bool, InvalidBlocks blk,
      AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
-> m (HeaderHash blk -> Bool, InvalidBlocks blk,
      AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
forall a b. (a -> b) -> a -> b
$ (,,)
      ((HeaderHash blk -> Bool)
 -> InvalidBlocks blk
 -> AnchoredSeq
      (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
 -> (HeaderHash blk -> Bool, InvalidBlocks blk,
     AnchoredSeq
       (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)))
-> STM m (HeaderHash blk -> Bool)
-> STM
     m
     (InvalidBlocks blk
      -> AnchoredSeq
           (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
      -> (HeaderHash blk -> Bool, InvalidBlocks blk,
          AnchoredSeq
            (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> VolatileDB m blk -> STM m (HeaderHash blk -> Bool)
forall (m :: * -> *) blk.
Functor (STM m) =>
VolatileDB m blk -> STM m (HeaderHash blk -> Bool)
VolatileDB.getIsMember          VolatileDB m blk
cdbVolatileDB
      STM
  m
  (InvalidBlocks blk
   -> AnchoredSeq
        (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
   -> (HeaderHash blk -> Bool, InvalidBlocks blk,
       AnchoredSeq
         (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)))
-> STM m (InvalidBlocks blk)
-> STM
     m
     (AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
      -> (HeaderHash blk -> Bool, InvalidBlocks blk,
          AnchoredSeq
            (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)))
forall a b. STM m (a -> b) -> STM m a -> STM m b
forall (f :: * -> *) a b. Applicative f => f (a -> b) -> f a -> f b
<*> (WithFingerprint (InvalidBlocks blk) -> InvalidBlocks blk
forall a. WithFingerprint a -> a
forgetFingerprint (WithFingerprint (InvalidBlocks blk) -> InvalidBlocks blk)
-> STM m (WithFingerprint (InvalidBlocks blk))
-> STM m (InvalidBlocks blk)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> StrictTVar m (WithFingerprint (InvalidBlocks blk))
-> STM m (WithFingerprint (InvalidBlocks blk))
forall (m :: * -> *) a. MonadSTM m => StrictTVar m a -> STM m a
readTVar StrictTVar m (WithFingerprint (InvalidBlocks blk))
cdbInvalid)
      STM
  m
  (AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
   -> (HeaderHash blk -> Bool, InvalidBlocks blk,
       AnchoredSeq
         (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)))
-> STM
     m
     (AnchoredSeq
        (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
-> STM
     m
     (HeaderHash blk -> Bool, InvalidBlocks blk,
      AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
forall a b. STM m (a -> b) -> STM m a -> STM m b
forall (f :: * -> *) a b. Applicative f => f (a -> b) -> f a -> f b
<*> ChainDbEnv m blk
-> STM
     m
     (AnchoredSeq
        (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
forall (m :: * -> *) blk.
(IOLike m, HasHeader (Header blk),
 ConsensusProtocol (BlockProtocol blk)) =>
ChainDbEnv m blk -> STM m (AnchoredFragment (Header blk))
Query.getCurrentChain           ChainDbEnv m blk
cdb

    let immBlockNo = AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> WithOrigin BlockNo
forall block. AnchoredFragment block -> WithOrigin BlockNo
AF.anchorBlockNo AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
curChain

    -- We follow the steps from section "## Adding a block" in ChainDB.md

    if
      | olderThanK hdr isEBB immBlockNo -> do
        lift $ traceWith addBlockTracer $ IgnoreBlockOlderThanK (blockRealPoint b)
        lift $ deliverWrittenToDisk False

      | isMember (blockHash b) -> do
        lift $ traceWith addBlockTracer $ IgnoreBlockAlreadyInVolatileDB (blockRealPoint b)
        lift $ deliverWrittenToDisk True

      | Just (InvalidBlockInfo reason _) <- Map.lookup (blockHash b) invalid -> do
        lift $ traceWith addBlockTracer $ IgnoreInvalidBlock (blockRealPoint b) reason
        lift $ deliverWrittenToDisk False

        -- We wouldn't know the block is invalid if its prefix was invalid,
        -- hence 'InvalidBlockPunishment.BlockItself'.
        lift $ InvalidBlockPunishment.enact
          blockPunish
          InvalidBlockPunishment.BlockItself

      -- The remaining cases
      | otherwise -> do
        let traceEv = RealPoint blk
-> BlockNo -> IsEBB -> Enclosing -> TraceAddBlockEvent blk
forall blk.
RealPoint blk
-> BlockNo -> IsEBB -> Enclosing -> TraceAddBlockEvent blk
AddedBlockToVolatileDB (blk -> RealPoint blk
forall blk. HasHeader blk => blk -> RealPoint blk
blockRealPoint blk
b) (blk -> BlockNo
forall b. HasHeader b => b -> BlockNo
blockNo blk
b) IsEBB
isEBB
        lift $ encloseWith (traceEv >$< addBlockTracer) $
          VolatileDB.putBlock cdbVolatileDB b
        lift $ deliverWrittenToDisk True
        chainSelectionForBlock cdb (BlockCache.singleton b) hdr blockPunish

    newTip <- lift $ atomically $ Query.getTipPoint cdb

    lift $ deliverProcessed newTip
  where
    addBlockTracer :: Tracer m (TraceAddBlockEvent blk)
    addBlockTracer :: Tracer m (TraceAddBlockEvent blk)
addBlockTracer = TraceAddBlockEvent blk -> TraceEvent blk
forall blk. TraceAddBlockEvent blk -> TraceEvent blk
TraceAddBlockEvent (TraceAddBlockEvent blk -> TraceEvent blk)
-> Tracer m (TraceEvent blk) -> Tracer m (TraceAddBlockEvent blk)
forall (f :: * -> *) a b. Contravariant f => (a -> b) -> f b -> f a
>$< Tracer m (TraceEvent blk)
cdbTracer

    hdr :: Header blk
    hdr :: Header blk
hdr = blk -> Header blk
forall blk. GetHeader blk => blk -> Header blk
getHeader blk
b

    isEBB :: IsEBB
    isEBB :: IsEBB
isEBB = Header blk -> IsEBB
forall blk. GetHeader blk => Header blk -> IsEBB
headerToIsEBB Header blk
hdr

    -- | Fill in the 'TMVar' for the 'varBlockWrittenToDisk' of the block's
    -- 'AddBlockPromise' with the given 'Bool'.
    deliverWrittenToDisk :: Bool -> m ()
    deliverWrittenToDisk :: Bool -> m ()
deliverWrittenToDisk Bool
writtenToDisk = 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
$
        StrictTMVar m Bool -> Bool -> STM m ()
forall (m :: * -> *) a.
MonadSTM m =>
StrictTMVar m a -> a -> STM m ()
putTMVar StrictTMVar m Bool
varBlockWrittenToDisk Bool
writtenToDisk

    -- | Fill in the 'TMVar' for the 'varBlockProcessed' of the block's
    -- 'AddBlockPromise' with the given tip.
    deliverProcessed :: Point blk -> m ()
    deliverProcessed :: Point blk -> m ()
deliverProcessed Point blk
tip = 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
$
        StrictTMVar m (AddBlockResult blk)
-> AddBlockResult blk -> STM m ()
forall (m :: * -> *) a.
MonadSTM m =>
StrictTMVar m a -> a -> STM m ()
putTMVar StrictTMVar m (AddBlockResult blk)
varBlockProcessed (Point blk -> AddBlockResult blk
forall blk. Point blk -> AddBlockResult blk
SuccesfullyAddedBlock Point blk
tip)

-- | Return 'True' when the given header should be ignored when adding it
-- because it is too old, i.e., we wouldn't be able to switch to a chain
-- containing the corresponding block because its block number is more than
-- @k@ blocks or exactly @k@ blocks back.
--
-- Special case: the header corresponds to an EBB which has the same block
-- number as the block @k@ blocks back (the most recent \"immutable\" block).
-- As EBBs share their block number with the block before them, the EBB is not
-- too old in that case and can be adopted as part of our chain.
--
-- This special case can occur, for example, when the VolatileDB is empty
-- (because of corruption). The \"immutable\" block is then also the tip of
-- the chain. If we then try to add the EBB after it, it will have the same
-- block number, so we must allow it.
olderThanK ::
     HasHeader (Header blk)
  => Header blk
     -- ^ Header of the block to add
  -> IsEBB
     -- ^ Whether the block is an EBB or not
  -> WithOrigin BlockNo
     -- ^ The block number of the most recent \"immutable\" block, i.e., the
     -- block @k@ blocks back.
  -> Bool
olderThanK :: forall blk.
HasHeader (Header blk) =>
Header blk -> IsEBB -> WithOrigin BlockNo -> Bool
olderThanK Header blk
hdr IsEBB
isEBB WithOrigin BlockNo
immBlockNo
    | BlockNo -> WithOrigin BlockNo
forall t. t -> WithOrigin t
NotOrigin BlockNo
bNo WithOrigin BlockNo -> WithOrigin BlockNo -> Bool
forall a. Eq a => a -> a -> Bool
== WithOrigin BlockNo
immBlockNo
    , IsEBB
isEBB IsEBB -> IsEBB -> Bool
forall a. Eq a => a -> a -> Bool
== IsEBB
IsEBB
    = Bool
False
    | Bool
otherwise
    = BlockNo -> WithOrigin BlockNo
forall t. t -> WithOrigin t
NotOrigin BlockNo
bNo WithOrigin BlockNo -> WithOrigin BlockNo -> Bool
forall a. Ord a => a -> a -> Bool
<= WithOrigin BlockNo
immBlockNo
  where
    bNo :: BlockNo
bNo = Header blk -> BlockNo
forall b. HasHeader b => b -> BlockNo
blockNo Header blk
hdr

-- | When we switch to a new selected chain, we are either extending the current
-- chain by adding blocks on top or we are switching to a fork.
data ChainSwitchType = AddingBlocks | SwitchingToAFork
  deriving (Int -> ChainSwitchType -> ShowS
[ChainSwitchType] -> ShowS
ChainSwitchType -> [Char]
(Int -> ChainSwitchType -> ShowS)
-> (ChainSwitchType -> [Char])
-> ([ChainSwitchType] -> ShowS)
-> Show ChainSwitchType
forall a.
(Int -> a -> ShowS) -> (a -> [Char]) -> ([a] -> ShowS) -> Show a
$cshowsPrec :: Int -> ChainSwitchType -> ShowS
showsPrec :: Int -> ChainSwitchType -> ShowS
$cshow :: ChainSwitchType -> [Char]
show :: ChainSwitchType -> [Char]
$cshowList :: [ChainSwitchType] -> ShowS
showList :: [ChainSwitchType] -> ShowS
Show, ChainSwitchType -> ChainSwitchType -> Bool
(ChainSwitchType -> ChainSwitchType -> Bool)
-> (ChainSwitchType -> ChainSwitchType -> Bool)
-> Eq ChainSwitchType
forall a. (a -> a -> Bool) -> (a -> a -> Bool) -> Eq a
$c== :: ChainSwitchType -> ChainSwitchType -> Bool
== :: ChainSwitchType -> ChainSwitchType -> Bool
$c/= :: ChainSwitchType -> ChainSwitchType -> Bool
/= :: ChainSwitchType -> ChainSwitchType -> Bool
Eq)

-- | Trigger chain selection for the given block.
--
-- PRECONDITION: the block is in the VolatileDB.
--
-- PRECONDITION: the slot of the block <= the current (wall) slot
--
-- = Constructing candidate fragments
--
-- The VolatileDB keeps a \"successors\" map in memory, telling us the hashes
-- of the known successors of any block, but it does not keep /headers/ in
-- memory, which are needed to construct candidate fargments. We try to reuse
-- the headers from the current chain fragment where possible, but it will not
-- contain all needed headers. This means that we will need to read some
-- blocks from disk and extract their headers. Under normal circumstances this
-- does not matter too much; although this will be done every time we add a
-- block, the expected number of headers to read from disk is very small:
--
-- * None if we stay on the current chain and this is just the next block
-- * A handful if we stay on the current chain and the block we just received
--   was a missing block and we already received some of its successors
-- * A handful if we switch to a short fork
--
-- This is expensive only
--
-- * on startup: in this case we need to read at least @k@ blocks from the
--   VolatileDB, and possibly more if there are some other chains in the
--   VolatileDB starting from the tip of the ImmutableDB
-- * when we switch to a distant fork
--
-- This cost is currently deemed acceptable.
chainSelectionForBlock ::
     forall m blk.
     ( IOLike m
     , LedgerSupportsProtocol blk
     , BlockSupportsDiffusionPipelining blk
     , InspectLedger blk
     , HasHardForkHistory blk
     , HasCallStack
     )
  => ChainDbEnv m blk
  -> BlockCache blk
  -> Header blk
  -> InvalidBlockPunishment m
  -> Electric m ()
chainSelectionForBlock :: forall (m :: * -> *) blk.
(IOLike m, LedgerSupportsProtocol blk,
 BlockSupportsDiffusionPipelining blk, InspectLedger blk,
 HasHardForkHistory blk, HasCallStack) =>
ChainDbEnv m blk
-> BlockCache blk
-> Header blk
-> InvalidBlockPunishment m
-> Electric m ()
chainSelectionForBlock cdb :: ChainDbEnv m blk
cdb@CDB{m (LoE (AnchoredFragment (HeaderWithTime blk)))
Tracer m (TraceEvent blk)
DiffTime
ResourceRegistry m
StrictTVar m (m ())
StrictTVar m (Map FollowerKey (FollowerHandle m blk))
StrictTVar m (Map IteratorKey (m ()))
StrictTVar m (StrictMaybe (Header blk))
StrictTVar m ChainSelStarvation
StrictTVar m (TentativeHeaderState blk)
StrictTVar m (WithFingerprint (InvalidBlocks blk))
StrictTVar m FollowerKey
StrictTVar m IteratorKey
StrictTVar m (InternalChain blk)
Fuse m
TopLevelConfig blk
VolatileDB m blk
ImmutableDB m blk
LedgerDB' m blk
ChainSelQueue m blk
cdbChainSelQueue :: forall (m :: * -> *) blk. ChainDbEnv m blk -> ChainSelQueue m blk
cdbTracer :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> Tracer m (TraceEvent blk)
cdbChainSelStarvation :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> StrictTVar m ChainSelStarvation
cdbLoE :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> m (LoE (AnchoredFragment (HeaderWithTime blk)))
cdbKillBgThreads :: forall (m :: * -> *) blk. ChainDbEnv m blk -> StrictTVar m (m ())
cdbGcInterval :: forall (m :: * -> *) blk. ChainDbEnv m blk -> DiffTime
cdbGcDelay :: forall (m :: * -> *) blk. ChainDbEnv m blk -> DiffTime
cdbRegistry :: forall (m :: * -> *) blk. ChainDbEnv m blk -> ResourceRegistry m
cdbChainSelFuse :: forall (m :: * -> *) blk. ChainDbEnv m blk -> Fuse m
cdbCopyFuse :: forall (m :: * -> *) blk. ChainDbEnv m blk -> Fuse m
cdbNextFollowerKey :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> StrictTVar m FollowerKey
cdbNextIteratorKey :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> StrictTVar m IteratorKey
cdbInvalid :: forall (m :: * -> *) blk.
ChainDbEnv m blk
-> StrictTVar m (WithFingerprint (InvalidBlocks blk))
cdbTopLevelConfig :: forall (m :: * -> *) blk. ChainDbEnv m blk -> TopLevelConfig blk
cdbFollowers :: forall (m :: * -> *) blk.
ChainDbEnv m blk
-> StrictTVar m (Map FollowerKey (FollowerHandle m blk))
cdbIterators :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> StrictTVar m (Map IteratorKey (m ()))
cdbTentativeHeader :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> StrictTVar m (StrictMaybe (Header blk))
cdbTentativeState :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> StrictTVar m (TentativeHeaderState blk)
cdbChain :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> StrictTVar m (InternalChain blk)
cdbLedgerDB :: forall (m :: * -> *) blk. ChainDbEnv m blk -> LedgerDB' m blk
cdbVolatileDB :: forall (m :: * -> *) blk. ChainDbEnv m blk -> VolatileDB m blk
cdbImmutableDB :: forall (m :: * -> *) blk. ChainDbEnv m blk -> ImmutableDB m blk
cdbImmutableDB :: ImmutableDB m blk
cdbVolatileDB :: VolatileDB m blk
cdbLedgerDB :: LedgerDB' m blk
cdbChain :: StrictTVar m (InternalChain blk)
cdbTentativeState :: StrictTVar m (TentativeHeaderState blk)
cdbTentativeHeader :: StrictTVar m (StrictMaybe (Header blk))
cdbIterators :: StrictTVar m (Map IteratorKey (m ()))
cdbFollowers :: StrictTVar m (Map FollowerKey (FollowerHandle m blk))
cdbTopLevelConfig :: TopLevelConfig blk
cdbInvalid :: StrictTVar m (WithFingerprint (InvalidBlocks blk))
cdbNextIteratorKey :: StrictTVar m IteratorKey
cdbNextFollowerKey :: StrictTVar m FollowerKey
cdbCopyFuse :: Fuse m
cdbChainSelFuse :: Fuse m
cdbTracer :: Tracer m (TraceEvent blk)
cdbRegistry :: ResourceRegistry m
cdbGcDelay :: DiffTime
cdbGcInterval :: DiffTime
cdbKillBgThreads :: StrictTVar m (m ())
cdbChainSelQueue :: ChainSelQueue m blk
cdbLoE :: m (LoE (AnchoredFragment (HeaderWithTime blk)))
cdbChainSelStarvation :: StrictTVar m ChainSelStarvation
..} BlockCache blk
blockCache Header blk
hdr InvalidBlockPunishment m
punish = m () -> Electric m ()
forall {k} (m :: k -> *) (a :: k). m a -> Electric m a
electric (m () -> Electric m ()) -> m () -> Electric m ()
forall a b. (a -> b) -> a -> b
$ (ResourceRegistry m -> m ()) -> m ()
forall (m :: * -> *) a.
(MonadSTM m, MonadMask m, MonadThread m, HasCallStack) =>
(ResourceRegistry m -> m a) -> m a
withRegistry ((ResourceRegistry m -> m ()) -> m ())
-> (ResourceRegistry m -> m ()) -> m ()
forall a b. (a -> b) -> a -> b
$ \ResourceRegistry m
rr -> do
    (invalid, succsOf, lookupBlockInfo, curChain, tipPoint)
      <- STM
  m
  (InvalidBlocks blk, ChainHash blk -> Set (HeaderHash blk),
   HeaderHash blk -> Maybe (BlockInfo blk),
   AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk),
   Point blk)
-> m (InvalidBlocks blk, ChainHash blk -> Set (HeaderHash blk),
      HeaderHash blk -> Maybe (BlockInfo blk),
      AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk),
      Point blk)
forall a. HasCallStack => STM m a -> m a
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
STM m a -> m a
atomically (STM
   m
   (InvalidBlocks blk, ChainHash blk -> Set (HeaderHash blk),
    HeaderHash blk -> Maybe (BlockInfo blk),
    AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk),
    Point blk)
 -> m (InvalidBlocks blk, ChainHash blk -> Set (HeaderHash blk),
       HeaderHash blk -> Maybe (BlockInfo blk),
       AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk),
       Point blk))
-> STM
     m
     (InvalidBlocks blk, ChainHash blk -> Set (HeaderHash blk),
      HeaderHash blk -> Maybe (BlockInfo blk),
      AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk),
      Point blk)
-> m (InvalidBlocks blk, ChainHash blk -> Set (HeaderHash blk),
      HeaderHash blk -> Maybe (BlockInfo blk),
      AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk),
      Point blk)
forall a b. (a -> b) -> a -> b
$ (,,,,)
          (InvalidBlocks blk
 -> (ChainHash blk -> Set (HeaderHash blk))
 -> (HeaderHash blk -> Maybe (BlockInfo blk))
 -> AnchoredSeq
      (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
 -> Point blk
 -> (InvalidBlocks blk, ChainHash blk -> Set (HeaderHash blk),
     HeaderHash blk -> Maybe (BlockInfo blk),
     AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk),
     Point blk))
-> STM m (InvalidBlocks blk)
-> STM
     m
     ((ChainHash blk -> Set (HeaderHash blk))
      -> (HeaderHash blk -> Maybe (BlockInfo blk))
      -> AnchoredSeq
           (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
      -> Point blk
      -> (InvalidBlocks blk, ChainHash blk -> Set (HeaderHash blk),
          HeaderHash blk -> Maybe (BlockInfo blk),
          AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk),
          Point blk))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> (WithFingerprint (InvalidBlocks blk) -> InvalidBlocks blk
forall a. WithFingerprint a -> a
forgetFingerprint (WithFingerprint (InvalidBlocks blk) -> InvalidBlocks blk)
-> STM m (WithFingerprint (InvalidBlocks blk))
-> STM m (InvalidBlocks blk)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> StrictTVar m (WithFingerprint (InvalidBlocks blk))
-> STM m (WithFingerprint (InvalidBlocks blk))
forall (m :: * -> *) a. MonadSTM m => StrictTVar m a -> STM m a
readTVar StrictTVar m (WithFingerprint (InvalidBlocks blk))
cdbInvalid)
          STM
  m
  ((ChainHash blk -> Set (HeaderHash blk))
   -> (HeaderHash blk -> Maybe (BlockInfo blk))
   -> AnchoredSeq
        (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
   -> Point blk
   -> (InvalidBlocks blk, ChainHash blk -> Set (HeaderHash blk),
       HeaderHash blk -> Maybe (BlockInfo blk),
       AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk),
       Point blk))
-> STM m (ChainHash blk -> Set (HeaderHash blk))
-> STM
     m
     ((HeaderHash blk -> Maybe (BlockInfo blk))
      -> AnchoredSeq
           (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
      -> Point blk
      -> (InvalidBlocks blk, ChainHash blk -> Set (HeaderHash blk),
          HeaderHash blk -> Maybe (BlockInfo blk),
          AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk),
          Point blk))
forall a b. STM m (a -> b) -> STM m a -> STM m b
forall (f :: * -> *) a b. Applicative f => f (a -> b) -> f a -> f b
<*> VolatileDB m blk
-> HasCallStack => STM m (ChainHash blk -> Set (HeaderHash blk))
forall (m :: * -> *) blk.
VolatileDB m blk
-> HasCallStack => STM m (ChainHash blk -> Set (HeaderHash blk))
VolatileDB.filterByPredecessor  VolatileDB m blk
cdbVolatileDB
          STM
  m
  ((HeaderHash blk -> Maybe (BlockInfo blk))
   -> AnchoredSeq
        (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
   -> Point blk
   -> (InvalidBlocks blk, ChainHash blk -> Set (HeaderHash blk),
       HeaderHash blk -> Maybe (BlockInfo blk),
       AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk),
       Point blk))
-> STM m (HeaderHash blk -> Maybe (BlockInfo blk))
-> STM
     m
     (AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
      -> Point blk
      -> (InvalidBlocks blk, ChainHash blk -> Set (HeaderHash blk),
          HeaderHash blk -> Maybe (BlockInfo blk),
          AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk),
          Point blk))
forall a b. STM m (a -> b) -> STM m a -> STM m b
forall (f :: * -> *) a b. Applicative f => f (a -> b) -> f a -> f b
<*> VolatileDB m blk
-> HasCallStack => STM m (HeaderHash blk -> Maybe (BlockInfo blk))
forall (m :: * -> *) blk.
VolatileDB m blk
-> HasCallStack => STM m (HeaderHash blk -> Maybe (BlockInfo blk))
VolatileDB.getBlockInfo         VolatileDB m blk
cdbVolatileDB
          STM
  m
  (AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
   -> Point blk
   -> (InvalidBlocks blk, ChainHash blk -> Set (HeaderHash blk),
       HeaderHash blk -> Maybe (BlockInfo blk),
       AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk),
       Point blk))
-> STM
     m
     (AnchoredSeq
        (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
-> STM
     m
     (Point blk
      -> (InvalidBlocks blk, ChainHash blk -> Set (HeaderHash blk),
          HeaderHash blk -> Maybe (BlockInfo blk),
          AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk),
          Point blk))
forall a b. STM m (a -> b) -> STM m a -> STM m b
forall (f :: * -> *) a b. Applicative f => f (a -> b) -> f a -> f b
<*> ChainDbEnv m blk
-> STM
     m
     (AnchoredSeq
        (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
forall (m :: * -> *) blk.
(IOLike m, HasHeader (Header blk),
 ConsensusProtocol (BlockProtocol blk)) =>
ChainDbEnv m blk -> STM m (AnchoredFragment (Header blk))
Query.getCurrentChain           ChainDbEnv m blk
cdb
          STM
  m
  (Point blk
   -> (InvalidBlocks blk, ChainHash blk -> Set (HeaderHash blk),
       HeaderHash blk -> Maybe (BlockInfo blk),
       AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk),
       Point blk))
-> STM m (Point blk)
-> STM
     m
     (InvalidBlocks blk, ChainHash blk -> Set (HeaderHash blk),
      HeaderHash blk -> Maybe (BlockInfo blk),
      AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk),
      Point blk)
forall a b. STM m (a -> b) -> STM m a -> STM m b
forall (f :: * -> *) a b. Applicative f => f (a -> b) -> f a -> f b
<*> ChainDbEnv m blk -> STM m (Point blk)
forall (m :: * -> *) blk.
(IOLike m, HasHeader (Header blk)) =>
ChainDbEnv m blk -> STM m (Point blk)
Query.getTipPoint               ChainDbEnv m blk
cdb
    -- This is safe: the LedgerDB tip doesn't change in between the previous
    -- atomically block and this call to 'withTipForker'.
    LedgerDB.withTipForker cdbLedgerDB rr $ \Forker m (ExtLedgerState blk) blk
curForker -> do
      curChainAndLedger :: ChainAndLedger m blk <-
            -- The current chain we're working with here is not longer than @k@
            -- blocks (see 'getCurrentChain' and 'cdbChain'), which is easier to
            -- reason about when doing chain selection, etc.
            Bool -> m (ChainAndLedger m blk) -> m (ChainAndLedger m blk)
forall a. HasCallStack => Bool -> a -> a
assert (Int -> Word64
forall a b. (Integral a, Num b) => a -> b
fromIntegral (AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> Int
forall v a b. Anchorable v a b => AnchoredSeq v a b -> Int
AF.length AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
curChain) Word64 -> Word64 -> Bool
forall a. Ord a => a -> a -> Bool
<= NonZero Word64 -> Word64
forall a. NonZero a -> a
unNonZero NonZero Word64
k) (m (ChainAndLedger m blk) -> m (ChainAndLedger m blk))
-> m (ChainAndLedger m blk) -> m (ChainAndLedger m blk)
forall a b. (a -> b) -> a -> b
$
            AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> Forker m (ExtLedgerState blk) blk -> m (ChainAndLedger m blk)
forall (m :: * -> *) l b.
(MonadSTM m, GetTipSTM m l, HasHeader b,
 HeaderHash b ~ HeaderHash l, HasCallStack) =>
AnchoredFragment b -> l -> m (ValidatedFragment b l)
VF.newM AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
curChain Forker m (ExtLedgerState blk) blk
curForker

      let
          immBlockNo :: WithOrigin BlockNo
          immBlockNo = AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> WithOrigin BlockNo
forall block. AnchoredFragment block -> WithOrigin BlockNo
AF.anchorBlockNo AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
curChain

          -- Let these two functions ignore invalid blocks
          lookupBlockInfo' = ChainDbEnv m blk
-> InvalidBlocks blk
-> (HeaderHash blk -> Maybe (BlockInfo blk))
-> HeaderHash blk
-> Maybe (BlockInfo blk)
forall blk (proxy :: * -> *) a.
HasHeader blk =>
proxy blk
-> InvalidBlocks blk
-> (HeaderHash blk -> Maybe a)
-> HeaderHash blk
-> Maybe a
ignoreInvalid    ChainDbEnv m blk
cdb InvalidBlocks blk
invalid HeaderHash blk -> Maybe (BlockInfo blk)
lookupBlockInfo
          succsOf'         = ChainDbEnv m blk
-> InvalidBlocks blk
-> (ChainHash blk -> Set (HeaderHash blk))
-> ChainHash blk
-> Set (HeaderHash blk)
forall blk (proxy :: * -> *).
HasHeader blk =>
proxy blk
-> InvalidBlocks blk
-> (ChainHash blk -> Set (HeaderHash blk))
-> ChainHash blk
-> Set (HeaderHash blk)
ignoreInvalidSuc ChainDbEnv m blk
cdb InvalidBlocks blk
invalid ChainHash blk -> Set (HeaderHash blk)
succsOf

      -- The preconditions
      assert (isJust $ lookupBlockInfo (headerHash hdr)) $ return ()

      let
        -- Trim the LoE fragment to be anchored in the immutable tip, ie the
        -- anchor of @curChain@. In particular, this establishes the property that
        -- it intersects with the current chain.
        sanitizeLoEFrag :: AnchoredFragment (HeaderWithTime blk)
                        -> AnchoredFragment (HeaderWithTime blk)
        sanitizeLoEFrag AnchoredFragment (HeaderWithTime blk)
loeFrag0 =
          case AnchoredFragment (HeaderWithTime blk)
-> Point (Header blk)
-> Maybe
     (AnchoredFragment (HeaderWithTime blk),
      AnchoredFragment (HeaderWithTime blk))
forall block1 block2.
(HasHeader block1, HeaderHash block1 ~ HeaderHash block2) =>
AnchoredFragment block1
-> Point block2
-> Maybe (AnchoredFragment block1, AnchoredFragment block1)
AF.splitAfterPoint AnchoredFragment (HeaderWithTime blk)
loeFrag0 (AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> Point (Header blk)
forall block. AnchoredFragment block -> Point block
AF.anchorPoint AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
curChain) of
              Just (AnchoredFragment (HeaderWithTime blk)
_, AnchoredFragment (HeaderWithTime blk)
frag) -> AnchoredFragment (HeaderWithTime blk)
frag
              -- As the (unsanitized) LoE fragment is rooted in a recent immutable
              -- tip, this case means that it doesn't intersect with the current
              -- chain. This can temporarily be the case; we are conservative and
              -- use the empty fragment anchored at the immutable tip for chain
              -- selection.
              Maybe
  (AnchoredFragment (HeaderWithTime blk),
   AnchoredFragment (HeaderWithTime blk))
Nothing        ->  Anchor (HeaderWithTime blk)
-> AnchoredFragment (HeaderWithTime blk)
forall v a b. Anchorable v a b => a -> AnchoredSeq v a b
AF.Empty (Anchor (HeaderWithTime blk)
 -> AnchoredFragment (HeaderWithTime blk))
-> Anchor (HeaderWithTime blk)
-> AnchoredFragment (HeaderWithTime blk)
forall a b. (a -> b) -> a -> b
$ Anchor (Header blk) -> Anchor (HeaderWithTime blk)
forall a b. (HeaderHash a ~ HeaderHash b) => Anchor a -> Anchor b
AF.castAnchor (Anchor (Header blk) -> Anchor (HeaderWithTime blk))
-> Anchor (Header blk) -> Anchor (HeaderWithTime blk)
forall a b. (a -> b) -> a -> b
$ AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> Anchor (Header blk)
forall v a b. AnchoredSeq v a b -> a
AF.anchor AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
curChain

      loeFrag <- fmap sanitizeLoEFrag <$> cdbLoE

      traceWith addBlockTracer
        (ChainSelectionLoEDebug curChain (AF.mapAnchoredFragment hwtHeader <$> loeFrag))

      if
        -- The chain might have grown since we added the block such that the
        -- block is older than @k@.
        | olderThanK hdr isEBB immBlockNo -> do
          traceWith addBlockTracer $ IgnoreBlockOlderThanK p

        -- The block is invalid
        | Just (InvalidBlockInfo reason _) <- Map.lookup (headerHash hdr) invalid -> do
          traceWith addBlockTracer $ IgnoreInvalidBlock p reason

          -- We wouldn't know the block is invalid if its prefix was invalid,
          -- hence 'InvalidBlockPunishment.BlockItself'.
          InvalidBlockPunishment.enact
            punish
            InvalidBlockPunishment.BlockItself

        -- The block fits onto the end of our current chain
        | pointHash tipPoint == headerPrevHash hdr -> do
          -- ### Add to current chain
          traceWith addBlockTracer (TryAddToCurrentChain p)
          addToCurrentChain rr succsOf' curChainAndLedger loeFrag

        -- The block is reachable from the current selection
        -- and it doesn't fit after the current selection
        | Just diff <- Paths.isReachable lookupBlockInfo' curChain p -> do
          -- ### Switch to a fork
          traceWith addBlockTracer (TrySwitchToAFork p diff)
          switchToAFork rr succsOf' lookupBlockInfo' curChainAndLedger loeFrag diff

        -- We cannot reach the block from the current selection
        | otherwise -> do
          -- ### Store but don't change the current chain
          traceWith addBlockTracer (StoreButDontChange p)

      -- Note that we may have extended the chain, but have not trimmed it to
      -- @k@ blocks/headers. That is the job of the background thread, which
      -- will first copy the blocks/headers to trim (from the end of the
      -- fragment) from the VolatileDB to the ImmutableDB.
  where
    SecurityParam NonZero Word64
k = TopLevelConfig blk -> SecurityParam
forall blk.
ConsensusProtocol (BlockProtocol blk) =>
TopLevelConfig blk -> SecurityParam
configSecurityParam TopLevelConfig blk
cdbTopLevelConfig

    p :: RealPoint blk
    p :: RealPoint blk
p = Header blk -> RealPoint blk
forall blk. HasHeader (Header blk) => Header blk -> RealPoint blk
headerRealPoint Header blk
hdr

    isEBB :: IsEBB
    isEBB :: IsEBB
isEBB = Header blk -> IsEBB
forall blk. GetHeader blk => Header blk -> IsEBB
headerToIsEBB Header blk
hdr

    addBlockTracer :: Tracer m (TraceAddBlockEvent blk)
    addBlockTracer :: Tracer m (TraceAddBlockEvent blk)
addBlockTracer = TraceAddBlockEvent blk -> TraceEvent blk
forall blk. TraceAddBlockEvent blk -> TraceEvent blk
TraceAddBlockEvent (TraceAddBlockEvent blk -> TraceEvent blk)
-> Tracer m (TraceEvent blk) -> Tracer m (TraceAddBlockEvent blk)
forall (f :: * -> *) a b. Contravariant f => (a -> b) -> f b -> f a
>$< Tracer m (TraceEvent blk)
cdbTracer

    mkChainSelEnv :: ChainAndLedger m blk -> ChainSelEnv m blk
    mkChainSelEnv :: ChainAndLedger m blk -> ChainSelEnv m blk
mkChainSelEnv ChainAndLedger m blk
curChainAndLedger = ChainSelEnv
      { lgrDB :: LedgerDB' m blk
lgrDB                 = LedgerDB' m blk
cdbLedgerDB
      , bcfg :: BlockConfig blk
bcfg                  = TopLevelConfig blk -> BlockConfig blk
forall blk. TopLevelConfig blk -> BlockConfig blk
configBlock TopLevelConfig blk
cdbTopLevelConfig
      , varInvalid :: StrictTVar m (WithFingerprint (InvalidBlocks blk))
varInvalid            = StrictTVar m (WithFingerprint (InvalidBlocks blk))
cdbInvalid
      , varTentativeState :: StrictTVar m (TentativeHeaderState blk)
varTentativeState     = StrictTVar m (TentativeHeaderState blk)
cdbTentativeState
      , varTentativeHeader :: StrictTVar m (StrictMaybe (Header blk))
varTentativeHeader    = StrictTVar m (StrictMaybe (Header blk))
cdbTentativeHeader
      , getTentativeFollowers :: STM m [FollowerHandle m blk]
getTentativeFollowers =
              (FollowerHandle m blk -> Bool)
-> [FollowerHandle m blk] -> [FollowerHandle m blk]
forall a. (a -> Bool) -> [a] -> [a]
filter ((ChainType
TentativeChain ChainType -> ChainType -> Bool
forall a. Eq a => a -> a -> Bool
==) (ChainType -> Bool)
-> (FollowerHandle m blk -> ChainType)
-> FollowerHandle m blk
-> Bool
forall b c a. (b -> c) -> (a -> b) -> a -> c
. FollowerHandle m blk -> ChainType
forall (m :: * -> *) blk. FollowerHandle m blk -> ChainType
fhChainType) ([FollowerHandle m blk] -> [FollowerHandle m blk])
-> (Map FollowerKey (FollowerHandle m blk)
    -> [FollowerHandle m blk])
-> Map FollowerKey (FollowerHandle m blk)
-> [FollowerHandle m blk]
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Map FollowerKey (FollowerHandle m blk) -> [FollowerHandle m blk]
forall k a. Map k a -> [a]
Map.elems
          (Map FollowerKey (FollowerHandle m blk) -> [FollowerHandle m blk])
-> STM m (Map FollowerKey (FollowerHandle m blk))
-> STM m [FollowerHandle m blk]
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> StrictTVar m (Map FollowerKey (FollowerHandle m blk))
-> STM m (Map FollowerKey (FollowerHandle m blk))
forall (m :: * -> *) a. MonadSTM m => StrictTVar m a -> STM m a
readTVar StrictTVar m (Map FollowerKey (FollowerHandle m blk))
cdbFollowers
      , blockCache :: BlockCache blk
blockCache            = BlockCache blk
blockCache
      , curChainAndLedger :: ChainAndLedger m blk
curChainAndLedger     = ChainAndLedger m blk
curChainAndLedger
      , validationTracer :: Tracer m (TraceValidationEvent blk)
validationTracer      =
          TraceAddBlockEvent blk -> TraceEvent blk
forall blk. TraceAddBlockEvent blk -> TraceEvent blk
TraceAddBlockEvent (TraceAddBlockEvent blk -> TraceEvent blk)
-> (TraceValidationEvent blk -> TraceAddBlockEvent blk)
-> TraceValidationEvent blk
-> TraceEvent blk
forall b c a. (b -> c) -> (a -> b) -> a -> c
. TraceValidationEvent blk -> TraceAddBlockEvent blk
forall blk. TraceValidationEvent blk -> TraceAddBlockEvent blk
AddBlockValidation (TraceValidationEvent blk -> TraceEvent blk)
-> Tracer m (TraceEvent blk) -> Tracer m (TraceValidationEvent blk)
forall (f :: * -> *) a b. Contravariant f => (a -> b) -> f b -> f a
>$< Tracer m (TraceEvent blk)
cdbTracer
      , pipeliningTracer :: Tracer m (TracePipeliningEvent blk)
pipeliningTracer       =
          TraceAddBlockEvent blk -> TraceEvent blk
forall blk. TraceAddBlockEvent blk -> TraceEvent blk
TraceAddBlockEvent (TraceAddBlockEvent blk -> TraceEvent blk)
-> (TracePipeliningEvent blk -> TraceAddBlockEvent blk)
-> TracePipeliningEvent blk
-> TraceEvent blk
forall b c a. (b -> c) -> (a -> b) -> a -> c
. TracePipeliningEvent blk -> TraceAddBlockEvent blk
forall blk. TracePipeliningEvent blk -> TraceAddBlockEvent blk
PipeliningEvent (TracePipeliningEvent blk -> TraceEvent blk)
-> Tracer m (TraceEvent blk) -> Tracer m (TracePipeliningEvent blk)
forall (f :: * -> *) a b. Contravariant f => (a -> b) -> f b -> f a
>$< Tracer m (TraceEvent blk)
cdbTracer
      , punish :: Maybe (RealPoint blk, InvalidBlockPunishment m)
punish                = (RealPoint blk, InvalidBlockPunishment m)
-> Maybe (RealPoint blk, InvalidBlockPunishment m)
forall a. a -> Maybe a
Just (RealPoint blk
p, InvalidBlockPunishment m
punish)
      }

    -- | PRECONDITION: the header @hdr@ (and block @b@) fit onto the end of
    -- the current chain.
    addToCurrentChain ::
         HasCallStack
      => ResourceRegistry m
      -> (ChainHash blk -> Set (HeaderHash blk))
      -> ChainAndLedger m blk
         -- ^ The current chain and ledger
      -> LoE (AnchoredFragment (HeaderWithTime blk))
         -- ^ LoE fragment
      -> m ()
    addToCurrentChain :: HasCallStack =>
ResourceRegistry m
-> (ChainHash blk -> Set (HeaderHash blk))
-> ChainAndLedger m blk
-> LoE (AnchoredFragment (HeaderWithTime blk))
-> m ()
addToCurrentChain ResourceRegistry m
rr ChainHash blk -> Set (HeaderHash blk)
succsOf ChainAndLedger m blk
curChainAndLedger LoE (AnchoredFragment (HeaderWithTime blk))
loeFrag = do
        -- Extensions of @B@ that do not exceed the LoE
        let suffixesAfterB :: [NonEmpty (HeaderHash blk)]
suffixesAfterB = (ChainHash blk -> Set (HeaderHash blk))
-> Maybe Word64 -> Point blk -> [NonEmpty (HeaderHash blk)]
forall blk.
(ChainHash blk -> Set (HeaderHash blk))
-> Maybe Word64 -> Point blk -> [NonEmpty (HeaderHash blk)]
Paths.maximalCandidates ChainHash blk -> Set (HeaderHash blk)
succsOf Maybe Word64
forall a. Maybe a
Nothing (RealPoint blk -> Point blk
forall blk. RealPoint blk -> Point blk
realPointToPoint RealPoint blk
p)

        -- Fragments that are anchored at @curHead@, i.e. suffixes of the
        -- current chain.
        candidates <- case [NonEmpty (HeaderHash blk)]
-> Maybe (NonEmpty (NonEmpty (HeaderHash blk)))
forall a. [a] -> Maybe (NonEmpty a)
NE.nonEmpty [NonEmpty (HeaderHash blk)]
suffixesAfterB of
          -- If there are no suffixes after @b@, just use the suffix just
          -- containing @b@ as the sole candidate.
          Maybe (NonEmpty (NonEmpty (HeaderHash blk)))
Nothing              ->
            NonEmpty
  (AnchoredSeq
     (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
-> m (NonEmpty
        (AnchoredSeq
           (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)))
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (NonEmpty
   (AnchoredSeq
      (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
 -> m (NonEmpty
         (AnchoredSeq
            (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))))
-> NonEmpty
     (AnchoredSeq
        (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
-> m (NonEmpty
        (AnchoredSeq
           (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)))
forall a b. (a -> b) -> a -> b
$ Anchor (Header blk)
-> [Header blk]
-> AnchoredSeq
     (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
forall v a b. Anchorable v a b => a -> [b] -> AnchoredSeq v a b
AF.fromOldestFirst Anchor (Header blk)
curHead [Header blk
hdr] AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> [AnchoredSeq
      (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)]
-> NonEmpty
     (AnchoredSeq
        (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
forall a. a -> [a] -> NonEmpty a
NE.:| []
          Just NonEmpty (NonEmpty (HeaderHash blk))
suffixesAfterB' ->
            -- We can start with an empty cache, because we're only looking
            -- up the headers /after/ b, so they won't be on the current
            -- chain.
            (StateT
   (Map (HeaderHash blk) (Header blk))
   m
   (NonEmpty
      (AnchoredSeq
         (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)))
 -> Map (HeaderHash blk) (Header blk)
 -> m (NonEmpty
         (AnchoredSeq
            (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))))
-> Map (HeaderHash blk) (Header blk)
-> StateT
     (Map (HeaderHash blk) (Header blk))
     m
     (NonEmpty
        (AnchoredSeq
           (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)))
-> m (NonEmpty
        (AnchoredSeq
           (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)))
forall a b c. (a -> b -> c) -> b -> a -> c
flip StateT
  (Map (HeaderHash blk) (Header blk))
  m
  (NonEmpty
     (AnchoredSeq
        (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)))
-> Map (HeaderHash blk) (Header blk)
-> m (NonEmpty
        (AnchoredSeq
           (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)))
forall (m :: * -> *) s a. Monad m => StateT s m a -> s -> m a
evalStateT Map (HeaderHash blk) (Header blk)
forall k a. Map k a
Map.empty (StateT
   (Map (HeaderHash blk) (Header blk))
   m
   (NonEmpty
      (AnchoredSeq
         (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)))
 -> m (NonEmpty
         (AnchoredSeq
            (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))))
-> StateT
     (Map (HeaderHash blk) (Header blk))
     m
     (NonEmpty
        (AnchoredSeq
           (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)))
-> m (NonEmpty
        (AnchoredSeq
           (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)))
forall a b. (a -> b) -> a -> b
$ NonEmpty (NonEmpty (HeaderHash blk))
-> (NonEmpty (HeaderHash blk)
    -> StateT
         (Map (HeaderHash blk) (Header blk))
         m
         (AnchoredSeq
            (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)))
-> StateT
     (Map (HeaderHash blk) (Header blk))
     m
     (NonEmpty
        (AnchoredSeq
           (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)))
forall (t :: * -> *) (m :: * -> *) a b.
(Traversable t, Monad m) =>
t a -> (a -> m b) -> m (t b)
forM NonEmpty (NonEmpty (HeaderHash blk))
suffixesAfterB' ((NonEmpty (HeaderHash blk)
  -> StateT
       (Map (HeaderHash blk) (Header blk))
       m
       (AnchoredSeq
          (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)))
 -> StateT
      (Map (HeaderHash blk) (Header blk))
      m
      (NonEmpty
         (AnchoredSeq
            (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))))
-> (NonEmpty (HeaderHash blk)
    -> StateT
         (Map (HeaderHash blk) (Header blk))
         m
         (AnchoredSeq
            (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)))
-> StateT
     (Map (HeaderHash blk) (Header blk))
     m
     (NonEmpty
        (AnchoredSeq
           (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)))
forall a b. (a -> b) -> a -> b
$ \NonEmpty (HeaderHash blk)
hashes -> do
              hdrs <- (HeaderHash blk
 -> StateT (Map (HeaderHash blk) (Header blk)) m (Header blk))
-> [HeaderHash blk]
-> StateT (Map (HeaderHash blk) (Header blk)) m [Header blk]
forall (t :: * -> *) (m :: * -> *) a b.
(Traversable t, Monad m) =>
(a -> m b) -> t a -> m (t b)
forall (m :: * -> *) a b. Monad m => (a -> m b) -> [a] -> m [b]
mapM (VolatileDB m blk
-> HeaderHash blk
-> StateT (Map (HeaderHash blk) (Header blk)) m (Header blk)
forall (m :: * -> *) blk.
(MonadThrow m, HasHeader blk) =>
VolatileDB m blk
-> HeaderHash blk
-> StateT (Map (HeaderHash blk) (Header blk)) m (Header blk)
getKnownHeaderThroughCache VolatileDB m blk
cdbVolatileDB) ([HeaderHash blk]
 -> StateT (Map (HeaderHash blk) (Header blk)) m [Header blk])
-> [HeaderHash blk]
-> StateT (Map (HeaderHash blk) (Header blk)) m [Header blk]
forall a b. (a -> b) -> a -> b
$
                        NonEmpty (HeaderHash blk) -> [HeaderHash blk]
forall a. NonEmpty a -> [a]
NE.toList NonEmpty (HeaderHash blk)
hashes
              return $ AF.fromOldestFirst curHead (hdr : hdrs)

        let chainDiffs = [ChainDiff (Header blk)]
-> Maybe (NonEmpty (ChainDiff (Header blk)))
forall a. [a] -> Maybe (NonEmpty a)
NE.nonEmpty
              ([ChainDiff (Header blk)]
 -> Maybe (NonEmpty (ChainDiff (Header blk))))
-> [ChainDiff (Header blk)]
-> Maybe (NonEmpty (ChainDiff (Header blk)))
forall a b. (a -> b) -> a -> b
$ (ChainDiff (Header blk) -> Bool)
-> [ChainDiff (Header blk)] -> [ChainDiff (Header blk)]
forall a. (a -> Bool) -> [a] -> [a]
filter (BlockConfig blk
-> AnchoredSeq
     (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> AnchoredSeq
     (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> Bool
forall blk (h :: * -> *) (h' :: * -> *).
(BlockSupportsProtocol blk, HasCallStack, GetHeader1 h,
 GetHeader1 h', HeaderHash (h blk) ~ HeaderHash (h' blk),
 HasHeader (h blk), HasHeader (h' blk)) =>
BlockConfig blk
-> AnchoredFragment (h blk) -> AnchoredFragment (h' blk) -> Bool
preferAnchoredCandidate (ChainSelEnv m blk -> BlockConfig blk
forall (m :: * -> *) blk. ChainSelEnv m blk -> BlockConfig blk
bcfg ChainSelEnv m blk
chainSelEnv) AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
curChain (AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
 -> Bool)
-> (ChainDiff (Header blk)
    -> AnchoredSeq
         (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
-> ChainDiff (Header blk)
-> Bool
forall b c a. (b -> c) -> (a -> b) -> a -> c
. ChainDiff (Header blk)
-> AnchoredSeq
     (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
forall b. ChainDiff b -> AnchoredFragment b
Diff.getSuffix)
              ([ChainDiff (Header blk)] -> [ChainDiff (Header blk)])
-> [ChainDiff (Header blk)] -> [ChainDiff (Header blk)]
forall a b. (a -> b) -> a -> b
$ (ChainDiff (Header blk) -> ChainDiff (Header blk))
-> [ChainDiff (Header blk)] -> [ChainDiff (Header blk)]
forall a b. (a -> b) -> [a] -> [b]
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap (LoE (AnchoredFragment (HeaderWithTime blk))
-> ChainAndLedger m blk
-> ChainDiff (Header blk)
-> ChainDiff (Header blk)
forall blk'.
(HasHeader blk', HeaderHash blk ~ HeaderHash blk') =>
LoE (AnchoredFragment blk')
-> ChainAndLedger m blk
-> ChainDiff (Header blk)
-> ChainDiff (Header blk)
trimToLoE LoE (AnchoredFragment (HeaderWithTime blk))
loeFrag ChainAndLedger m blk
curChainAndLedger)
              ([ChainDiff (Header blk)] -> [ChainDiff (Header blk)])
-> [ChainDiff (Header blk)] -> [ChainDiff (Header blk)]
forall a b. (a -> b) -> a -> b
$ (AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
 -> ChainDiff (Header blk))
-> [AnchoredSeq
      (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)]
-> [ChainDiff (Header blk)]
forall a b. (a -> b) -> [a] -> [b]
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> ChainDiff (Header blk)
forall b. AnchoredFragment b -> ChainDiff b
Diff.extend
              ([AnchoredSeq
    (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)]
 -> [ChainDiff (Header blk)])
-> [AnchoredSeq
      (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)]
-> [ChainDiff (Header blk)]
forall a b. (a -> b) -> a -> b
$ NonEmpty
  (AnchoredSeq
     (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
-> [AnchoredSeq
      (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)]
forall a. NonEmpty a -> [a]
NE.toList NonEmpty
  (AnchoredSeq
     (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
candidates
        -- All candidates are longer than the current chain, so they will be
        -- preferred over it, /unless/ the block we just added is an EBB,
        -- which has the same 'BlockNo' as the block before it, so when
        -- using the 'BlockNo' as the proxy for the length (note that some
        -- protocols might do it differently), the candidate with the EBB
        -- appended will not be preferred over the current chain.
        --
        -- The consequence of this is that when adding an EBB, it will not
        -- be selected by chain selection and thus not appended to the chain
        -- until the block after it is added, which will again result in a
        -- candidate preferred over the current chain. In this case, the
        -- candidate will be a two-block (the EBB and the new block)
        -- extension of the current chain.
        case chainDiffs of
          Maybe (NonEmpty (ChainDiff (Header blk)))
Nothing          -> () -> m ()
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return ()
          Just NonEmpty (ChainDiff (Header blk))
chainDiffs' ->
            ChainSelEnv m blk
-> ResourceRegistry m
-> NonEmpty (ChainDiff (Header blk))
-> m (Maybe
        (ValidatedChainDiff
           (Header blk) (Forker m (ExtLedgerState blk) blk)))
forall (m :: * -> *) blk.
(IOLike m, LedgerSupportsProtocol blk,
 BlockSupportsDiffusionPipelining blk, HasCallStack) =>
ChainSelEnv m blk
-> ResourceRegistry m
-> NonEmpty (ChainDiff (Header blk))
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
chainSelection ChainSelEnv m blk
chainSelEnv ResourceRegistry m
rr NonEmpty (ChainDiff (Header blk))
chainDiffs' m (Maybe
     (ValidatedChainDiff
        (Header blk) (Forker m (ExtLedgerState blk) blk)))
-> (Maybe
      (ValidatedChainDiff
         (Header blk) (Forker m (ExtLedgerState blk) blk))
    -> m ())
-> m ()
forall a b. m a -> (a -> m b) -> m b
forall (m :: * -> *) a b. Monad m => m a -> (a -> m b) -> m b
>>= \case
              Maybe
  (ValidatedChainDiff
     (Header blk) (Forker m (ExtLedgerState blk) blk))
Nothing ->
                () -> m ()
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return ()
              Just ValidatedChainDiff (Header blk) (Forker m (ExtLedgerState blk) blk)
validatedChainDiff ->
                HasCallStack =>
ValidatedChainDiff (Header blk) (Forker m (ExtLedgerState blk) blk)
-> StrictTVar m (StrictMaybe (Header blk))
-> ChainSwitchType
-> m ()
ValidatedChainDiff (Header blk) (Forker m (ExtLedgerState blk) blk)
-> StrictTVar m (StrictMaybe (Header blk))
-> ChainSwitchType
-> m ()
switchTo
                  ValidatedChainDiff (Header blk) (Forker m (ExtLedgerState blk) blk)
validatedChainDiff
                  (ChainSelEnv m blk -> StrictTVar m (StrictMaybe (Header blk))
forall (m :: * -> *) blk.
ChainSelEnv m blk -> StrictTVar m (StrictMaybe (Header blk))
varTentativeHeader ChainSelEnv m blk
chainSelEnv)
                  ChainSwitchType
AddingBlocks
      where
        chainSelEnv :: ChainSelEnv m blk
chainSelEnv = ChainAndLedger m blk -> ChainSelEnv m blk
mkChainSelEnv ChainAndLedger m blk
curChainAndLedger
        curChain :: AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
curChain    = ChainAndLedger m blk
-> AnchoredSeq
     (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
forall b l. ValidatedFragment b l -> AnchoredFragment b
VF.validatedFragment ChainAndLedger m blk
curChainAndLedger
        curHead :: Anchor (Header blk)
curHead     = AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> Anchor (Header blk)
forall v a b. Anchorable v a b => AnchoredSeq v a b -> a
AF.headAnchor AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
curChain

    -- | Trim the given candidate fragment to respect the LoE.
    --
    -- The returned fragment is such that:
    --
    -- - It is a prefix of the given fragment.
    -- - If it contains the tip of the LoE fragment, then it contains at most
    --   @k@ block after it.
    -- - If it does not contain the tip of the LoE fragment, then it is included
    --   in the LoE fragment.
    --
    -- The fragment is represented by the current chain and a diff with that
    -- current chain. It is tempting to only consider the suffix of the diff,
    -- but that would be incorrect, because the diff might not intersect with
    -- the LoE fragment, because the diff suffix is anchored somewhere on the
    -- current chain and LoE frag's tip might be older than that anchor.
    --
    -- PRECONDITIONS:
    --
    -- 1. The given 'ChainDiff' can apply on top of the given 'ChainAndLedger'.
    -- 2. The LoE fragment intersects with the current selection.
    trimToLoE ::
      (HasHeader blk', HeaderHash blk ~ HeaderHash blk') =>
      LoE (AnchoredFragment blk') ->
      ChainAndLedger m blk ->
      ChainDiff (Header blk) ->
      ChainDiff (Header blk)
    trimToLoE :: forall blk'.
(HasHeader blk', HeaderHash blk ~ HeaderHash blk') =>
LoE (AnchoredFragment blk')
-> ChainAndLedger m blk
-> ChainDiff (Header blk)
-> ChainDiff (Header blk)
trimToLoE LoE (AnchoredFragment blk')
LoEDisabled ChainAndLedger m blk
_ ChainDiff (Header blk)
diff = ChainDiff (Header blk)
diff
    trimToLoE (LoEEnabled AnchoredFragment blk'
loe) ChainAndLedger m blk
curChain ChainDiff (Header blk)
diff =
      case AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> ChainDiff (Header blk)
-> Maybe
     (AnchoredSeq
        (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
forall b.
HasHeader b =>
AnchoredFragment b -> ChainDiff b -> Maybe (AnchoredFragment b)
Diff.apply (ChainAndLedger m blk
-> AnchoredSeq
     (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
forall b l. ValidatedFragment b l -> AnchoredFragment b
VF.validatedFragment ChainAndLedger m blk
curChain) ChainDiff (Header blk)
diff of
        Maybe
  (AnchoredSeq
     (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
Nothing -> [Char] -> ChainDiff (Header blk)
forall a. HasCallStack => [Char] -> a
error [Char]
"trimToLoE: precondition 1 violated: the given 'ChainDiff' must apply on top of the given 'ChainAndLedger'"
        Just AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
cand ->
          case AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> AnchoredFragment blk'
-> Maybe
     (AnchoredSeq
        (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk),
      AnchoredFragment blk',
      AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk),
      AnchoredFragment blk')
forall block1 block2.
(HasHeader block1, HasHeader block2,
 HeaderHash block1 ~ HeaderHash block2) =>
AnchoredFragment block1
-> AnchoredFragment block2
-> Maybe
     (AnchoredFragment block1, AnchoredFragment block2,
      AnchoredFragment block1, AnchoredFragment block2)
AF.intersect AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
cand AnchoredFragment blk'
loe of
            Maybe
  (AnchoredSeq
     (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk),
   AnchoredFragment blk',
   AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk),
   AnchoredFragment blk')
Nothing -> [Char] -> ChainDiff (Header blk)
forall a. HasCallStack => [Char] -> a
error [Char]
"trimToLoE: precondition 2 violated: the LoE fragment must intersect with the current selection"
            Just (AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
candPrefix, AnchoredFragment blk'
_, AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
candSuffix, AnchoredFragment blk'
loeSuffix) ->
              let trimmedCandSuffix :: AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
trimmedCandSuffix = Int
-> AnchoredSeq
     (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> AnchoredSeq
     (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
forall v a b.
Anchorable v a b =>
Int -> AnchoredSeq v a b -> AnchoredSeq v a b
AF.takeOldest (Word64 -> Int
forall a b. (Integral a, Num b) => a -> b
fromIntegral (Word64 -> Int) -> Word64 -> Int
forall a b. (a -> b) -> a -> b
$ NonZero Word64 -> Word64
forall a. NonZero a -> a
unNonZero NonZero Word64
k) AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
candSuffix
                  trimmedCand :: AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
trimmedCand =
                    if AnchoredFragment blk' -> Bool
forall v a b. AnchoredSeq v a b -> Bool
AF.null AnchoredFragment blk'
loeSuffix
                      then Maybe
  (AnchoredSeq
     (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
-> AnchoredSeq
     (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
forall a. HasCallStack => Maybe a -> a
fromJust (Maybe
   (AnchoredSeq
      (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
 -> AnchoredSeq
      (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
-> Maybe
     (AnchoredSeq
        (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
-> AnchoredSeq
     (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
forall a b. (a -> b) -> a -> b
$ AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> AnchoredSeq
     (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> Maybe
     (AnchoredSeq
        (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
forall block.
HasHeader block =>
AnchoredFragment block
-> AnchoredFragment block -> Maybe (AnchoredFragment block)
AF.join AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
candPrefix AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
trimmedCandSuffix
                      else AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
candPrefix
              in AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> AnchoredSeq
     (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> ChainDiff (Header blk)
forall b b'.
(HasHeader b, HasHeader b', HeaderHash b ~ HeaderHash b',
 HasCallStack) =>
AnchoredFragment b' -> AnchoredFragment b -> ChainDiff b
Diff.diff (ChainAndLedger m blk
-> AnchoredSeq
     (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
forall b l. ValidatedFragment b l -> AnchoredFragment b
VF.validatedFragment ChainAndLedger m blk
curChain) AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
trimmedCand

    -- | We have found a 'ChainDiff' through the VolatileDB connecting the new
    -- block to the current chain. We'll call the intersection/anchor @x@.
    --
    -- We try to extend this path by looking for forks that start with the
    -- given block, then we do chain selection and /possibly/ try to switch to
    -- a new fork.
    switchToAFork ::
         HasCallStack
      => ResourceRegistry m
      -> (ChainHash blk -> Set (HeaderHash blk))
      -> LookupBlockInfo blk
      -> ChainAndLedger m blk
         -- ^ The current chain (anchored at @i@) and ledger
      -> LoE (AnchoredFragment (HeaderWithTime blk))
         -- ^ LoE fragment
      -> ChainDiff (HeaderFields blk)
         -- ^ Header fields for @(x,b]@
      -> m ()
    switchToAFork :: HasCallStack =>
ResourceRegistry m
-> (ChainHash blk -> Set (HeaderHash blk))
-> (HeaderHash blk -> Maybe (BlockInfo blk))
-> ChainAndLedger m blk
-> LoE (AnchoredFragment (HeaderWithTime blk))
-> ChainDiff (HeaderFields blk)
-> m ()
switchToAFork ResourceRegistry m
rr ChainHash blk -> Set (HeaderHash blk)
succsOf HeaderHash blk -> Maybe (BlockInfo blk)
lookupBlockInfo ChainAndLedger m blk
curChainAndLedger LoE (AnchoredFragment (HeaderWithTime blk))
loeFrag ChainDiff (HeaderFields blk)
diff = do
        -- We use a cache to avoid reading the headers from disk multiple
        -- times in case they're part of multiple forks that go through @b@.
        let initCache :: Map (HeaderHash blk) (Header blk)
initCache = HeaderHash blk -> Header blk -> Map (HeaderHash blk) (Header blk)
forall k a. k -> a -> Map k a
Map.singleton (Header blk -> HeaderHash blk
forall blk. HasHeader (Header blk) => Header blk -> HeaderHash blk
headerHash Header blk
hdr) Header blk
hdr
        chainDiffs <-
          -- 5. Filter out candidates that are not preferred over the current
          -- chain.
          --
          -- The suffixes all fork off from the current chain within @k@
          -- blocks, so it satisfies the precondition of 'preferCandidate'.
            ([ChainDiff (Header blk)] -> [ChainDiff (Header blk)])
-> m [ChainDiff (Header blk)] -> m [ChainDiff (Header blk)]
forall a b. (a -> b) -> m a -> m b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap
              ( (ChainDiff (Header blk) -> Bool)
-> [ChainDiff (Header blk)] -> [ChainDiff (Header blk)]
forall a. (a -> Bool) -> [a] -> [a]
filter
                  ( BlockConfig blk
-> AnchoredSeq
     (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> AnchoredSeq
     (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> Bool
forall blk (h :: * -> *) (h' :: * -> *).
(BlockSupportsProtocol blk, HasCallStack, GetHeader1 h,
 GetHeader1 h', HeaderHash (h blk) ~ HeaderHash (h' blk),
 HasHeader (h blk), HasHeader (h' blk)) =>
BlockConfig blk
-> AnchoredFragment (h blk) -> AnchoredFragment (h' blk) -> Bool
preferAnchoredCandidate (ChainSelEnv m blk -> BlockConfig blk
forall (m :: * -> *) blk. ChainSelEnv m blk -> BlockConfig blk
bcfg ChainSelEnv m blk
chainSelEnv) AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
curChain
                  (AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
 -> Bool)
-> (ChainDiff (Header blk)
    -> AnchoredSeq
         (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
-> ChainDiff (Header blk)
-> Bool
forall b c a. (b -> c) -> (a -> b) -> a -> c
. ChainDiff (Header blk)
-> AnchoredSeq
     (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
forall b. ChainDiff b -> AnchoredFragment b
Diff.getSuffix
                  )
              )
            -- 4. Trim fragments so that they follow the LoE, that is, they
            -- extend the LoE or are extended by the LoE. Filter them out
            -- otherwise.
          (m [ChainDiff (Header blk)] -> m [ChainDiff (Header blk)])
-> (ChainDiff (HeaderFields blk) -> m [ChainDiff (Header blk)])
-> ChainDiff (HeaderFields blk)
-> m [ChainDiff (Header blk)]
forall b c a. (b -> c) -> (a -> b) -> a -> c
. ([ChainDiff (Header blk)] -> [ChainDiff (Header blk)])
-> m [ChainDiff (Header blk)] -> m [ChainDiff (Header blk)]
forall a b. (a -> b) -> m a -> m b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap ((ChainDiff (Header blk) -> ChainDiff (Header blk))
-> [ChainDiff (Header blk)] -> [ChainDiff (Header blk)]
forall a b. (a -> b) -> [a] -> [b]
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap (LoE (AnchoredFragment (HeaderWithTime blk))
-> ChainAndLedger m blk
-> ChainDiff (Header blk)
-> ChainDiff (Header blk)
forall blk'.
(HasHeader blk', HeaderHash blk ~ HeaderHash blk') =>
LoE (AnchoredFragment blk')
-> ChainAndLedger m blk
-> ChainDiff (Header blk)
-> ChainDiff (Header blk)
trimToLoE LoE (AnchoredFragment (HeaderWithTime blk))
loeFrag ChainAndLedger m blk
curChainAndLedger))
            -- 3. Translate the 'HeaderFields' to 'Header' by reading the
            -- headers from disk.
          (m [ChainDiff (Header blk)] -> m [ChainDiff (Header blk)])
-> (ChainDiff (HeaderFields blk) -> m [ChainDiff (Header blk)])
-> ChainDiff (HeaderFields blk)
-> m [ChainDiff (Header blk)]
forall b c a. (b -> c) -> (a -> b) -> a -> c
. (StateT
   (Map (HeaderHash blk) (Header blk)) m [ChainDiff (Header blk)]
 -> Map (HeaderHash blk) (Header blk) -> m [ChainDiff (Header blk)])
-> Map (HeaderHash blk) (Header blk)
-> StateT
     (Map (HeaderHash blk) (Header blk)) m [ChainDiff (Header blk)]
-> m [ChainDiff (Header blk)]
forall a b c. (a -> b -> c) -> b -> a -> c
flip StateT
  (Map (HeaderHash blk) (Header blk)) m [ChainDiff (Header blk)]
-> Map (HeaderHash blk) (Header blk) -> m [ChainDiff (Header blk)]
forall (m :: * -> *) s a. Monad m => StateT s m a -> s -> m a
evalStateT Map (HeaderHash blk) (Header blk)
initCache
          (StateT
   (Map (HeaderHash blk) (Header blk)) m [ChainDiff (Header blk)]
 -> m [ChainDiff (Header blk)])
-> (ChainDiff (HeaderFields blk)
    -> StateT
         (Map (HeaderHash blk) (Header blk)) m [ChainDiff (Header blk)])
-> ChainDiff (HeaderFields blk)
-> m [ChainDiff (Header blk)]
forall b c a. (b -> c) -> (a -> b) -> a -> c
. (ChainDiff (HeaderFields blk)
 -> StateT
      (Map (HeaderHash blk) (Header blk)) m (ChainDiff (Header blk)))
-> [ChainDiff (HeaderFields blk)]
-> StateT
     (Map (HeaderHash blk) (Header blk)) m [ChainDiff (Header blk)]
forall (t :: * -> *) (m :: * -> *) a b.
(Traversable t, Monad m) =>
(a -> m b) -> t a -> m (t b)
forall (m :: * -> *) a b. Monad m => (a -> m b) -> [a] -> m [b]
mapM ChainDiff (HeaderFields blk)
-> StateT
     (Map (HeaderHash blk) (Header blk)) m (ChainDiff (Header blk))
translateToHeaders
            -- 2. Filter out candidates that are shorter than the current
            -- chain. We don't want to needlessly read the headers from disk
            -- for those candidates.
          ([ChainDiff (HeaderFields blk)]
 -> StateT
      (Map (HeaderHash blk) (Header blk)) m [ChainDiff (Header blk)])
-> (ChainDiff (HeaderFields blk) -> [ChainDiff (HeaderFields blk)])
-> ChainDiff (HeaderFields blk)
-> StateT
     (Map (HeaderHash blk) (Header blk)) m [ChainDiff (Header blk)]
forall b c a. (b -> c) -> (a -> b) -> a -> c
. (ChainDiff (HeaderFields blk) -> Bool)
-> NonEmpty (ChainDiff (HeaderFields blk))
-> [ChainDiff (HeaderFields blk)]
forall a. (a -> Bool) -> NonEmpty a -> [a]
NE.filter (Bool -> Bool
not (Bool -> Bool)
-> (ChainDiff (HeaderFields blk) -> Bool)
-> ChainDiff (HeaderFields blk)
-> Bool
forall b c a. (b -> c) -> (a -> b) -> a -> c
. ChainDiff (HeaderFields blk) -> Bool
forall b. HasHeader b => ChainDiff b -> Bool
Diff.rollbackExceedsSuffix)
            -- 1. Extend the diff with candidates fitting on @B@ and not exceeding the LoE
          (NonEmpty (ChainDiff (HeaderFields blk))
 -> [ChainDiff (HeaderFields blk)])
-> (ChainDiff (HeaderFields blk)
    -> NonEmpty (ChainDiff (HeaderFields blk)))
-> ChainDiff (HeaderFields blk)
-> [ChainDiff (HeaderFields blk)]
forall b c a. (b -> c) -> (a -> b) -> a -> c
. (ChainHash blk -> Set (HeaderHash blk))
-> (HeaderHash blk -> Maybe (BlockInfo blk))
-> ChainDiff (HeaderFields blk)
-> NonEmpty (ChainDiff (HeaderFields blk))
forall blk.
HasHeader blk =>
(ChainHash blk -> Set (HeaderHash blk))
-> LookupBlockInfo blk
-> ChainDiff (HeaderFields blk)
-> NonEmpty (ChainDiff (HeaderFields blk))
Paths.extendWithSuccessors ChainHash blk -> Set (HeaderHash blk)
succsOf HeaderHash blk -> Maybe (BlockInfo blk)
lookupBlockInfo
          (ChainDiff (HeaderFields blk) -> m [ChainDiff (Header blk)])
-> ChainDiff (HeaderFields blk) -> m [ChainDiff (Header blk)]
forall a b. (a -> b) -> a -> b
$ ChainDiff (HeaderFields blk)
diff

        case NE.nonEmpty chainDiffs of
          -- No candidates preferred over the current chain
          Maybe (NonEmpty (ChainDiff (Header blk)))
Nothing          -> () -> m ()
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return ()
          Just NonEmpty (ChainDiff (Header blk))
chainDiffs' ->
            ChainSelEnv m blk
-> ResourceRegistry m
-> NonEmpty (ChainDiff (Header blk))
-> m (Maybe
        (ValidatedChainDiff
           (Header blk) (Forker m (ExtLedgerState blk) blk)))
forall (m :: * -> *) blk.
(IOLike m, LedgerSupportsProtocol blk,
 BlockSupportsDiffusionPipelining blk, HasCallStack) =>
ChainSelEnv m blk
-> ResourceRegistry m
-> NonEmpty (ChainDiff (Header blk))
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
chainSelection ChainSelEnv m blk
chainSelEnv ResourceRegistry m
rr NonEmpty (ChainDiff (Header blk))
chainDiffs' m (Maybe
     (ValidatedChainDiff
        (Header blk) (Forker m (ExtLedgerState blk) blk)))
-> (Maybe
      (ValidatedChainDiff
         (Header blk) (Forker m (ExtLedgerState blk) blk))
    -> m ())
-> m ()
forall a b. m a -> (a -> m b) -> m b
forall (m :: * -> *) a b. Monad m => m a -> (a -> m b) -> m b
>>= \case
              Maybe
  (ValidatedChainDiff
     (Header blk) (Forker m (ExtLedgerState blk) blk))
Nothing                 ->
                () -> m ()
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return ()
              Just ValidatedChainDiff (Header blk) (Forker m (ExtLedgerState blk) blk)
validatedChainDiff ->
                HasCallStack =>
ValidatedChainDiff (Header blk) (Forker m (ExtLedgerState blk) blk)
-> StrictTVar m (StrictMaybe (Header blk))
-> ChainSwitchType
-> m ()
ValidatedChainDiff (Header blk) (Forker m (ExtLedgerState blk) blk)
-> StrictTVar m (StrictMaybe (Header blk))
-> ChainSwitchType
-> m ()
switchTo
                  ValidatedChainDiff (Header blk) (Forker m (ExtLedgerState blk) blk)
validatedChainDiff
                  (ChainSelEnv m blk -> StrictTVar m (StrictMaybe (Header blk))
forall (m :: * -> *) blk.
ChainSelEnv m blk -> StrictTVar m (StrictMaybe (Header blk))
varTentativeHeader ChainSelEnv m blk
chainSelEnv)
                  ChainSwitchType
SwitchingToAFork
      where
        chainSelEnv :: ChainSelEnv m blk
chainSelEnv = ChainAndLedger m blk -> ChainSelEnv m blk
mkChainSelEnv ChainAndLedger m blk
curChainAndLedger
        curChain :: AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
curChain    = ChainAndLedger m blk
-> AnchoredSeq
     (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
forall b l. ValidatedFragment b l -> AnchoredFragment b
VF.validatedFragment ChainAndLedger m blk
curChainAndLedger

    mkSelectionChangedInfo ::
         AnchoredFragment (Header blk) -- ^ old chain
      -> AnchoredFragment (Header blk) -- ^ new chain
      -> ExtLedgerState blk EmptyMK    -- ^ new tip
      -> SelectionChangedInfo blk
    mkSelectionChangedInfo :: AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> AnchoredSeq
     (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> ExtLedgerState blk EmptyMK
-> SelectionChangedInfo blk
mkSelectionChangedInfo AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
oldChain AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
newChain ExtLedgerState blk EmptyMK
newTip =
        SelectionChangedInfo {
            newTipPoint :: RealPoint blk
newTipPoint       = RealPoint (Header blk) -> RealPoint blk
forall blk blk'.
Coercible (HeaderHash blk) (HeaderHash blk') =>
RealPoint blk -> RealPoint blk'
castRealPoint RealPoint (Header blk)
tipPoint
          , newTipEpoch :: EpochNo
newTipEpoch       = EpochNo
tipEpoch
          , newTipSlotInEpoch :: Word64
newTipSlotInEpoch = Word64
tipSlotInEpoch
          , newTipTrigger :: RealPoint blk
newTipTrigger     = RealPoint blk
p
          , SelectView (BlockProtocol blk)
newTipSelectView :: SelectView (BlockProtocol blk)
newTipSelectView :: SelectView (BlockProtocol blk)
newTipSelectView
          , oldTipSelectView :: Maybe (SelectView (BlockProtocol blk))
oldTipSelectView  =
                  BlockConfig blk -> Header blk -> SelectView (BlockProtocol blk)
forall blk.
BlockSupportsProtocol blk =>
BlockConfig blk -> Header blk -> SelectView (BlockProtocol blk)
selectView (TopLevelConfig blk -> BlockConfig blk
forall blk. TopLevelConfig blk -> BlockConfig blk
configBlock TopLevelConfig blk
cfg)
              (Header blk -> SelectView (BlockProtocol blk))
-> Maybe (Header blk) -> Maybe (SelectView (BlockProtocol blk))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Either (Anchor (Header blk)) (Header blk) -> Maybe (Header blk)
forall a b. Either a b -> Maybe b
eitherToMaybe (AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> Either (Anchor (Header blk)) (Header blk)
forall v a b. Anchorable v a b => AnchoredSeq v a b -> Either a b
AF.head AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
oldChain)
          }
      where
        cfg :: TopLevelConfig blk
        cfg :: TopLevelConfig blk
cfg = TopLevelConfig blk
cdbTopLevelConfig

        ledger :: LedgerState blk EmptyMK
        ledger :: LedgerState blk EmptyMK
ledger = ExtLedgerState blk EmptyMK -> LedgerState blk EmptyMK
forall blk (mk :: MapKind).
ExtLedgerState blk mk -> LedgerState blk mk
ledgerState ExtLedgerState blk EmptyMK
newTip

        summary :: History.Summary (HardForkIndices blk)
        summary :: Summary (HardForkIndices blk)
summary = LedgerConfig blk
-> LedgerState blk EmptyMK -> Summary (HardForkIndices blk)
forall blk (mk :: MapKind).
HasHardForkHistory blk =>
LedgerConfig blk
-> LedgerState blk mk -> Summary (HardForkIndices blk)
forall (mk :: MapKind).
LedgerConfig blk
-> LedgerState blk mk -> Summary (HardForkIndices blk)
hardForkSummary
                    (TopLevelConfig blk -> LedgerConfig blk
forall blk. TopLevelConfig blk -> LedgerConfig blk
configLedger TopLevelConfig blk
cfg)
                    LedgerState blk EmptyMK
ledger

        (RealPoint (Header blk)
tipPoint, (EpochNo
tipEpoch, Word64
tipSlotInEpoch), SelectView (BlockProtocol blk)
newTipSelectView) =
          case AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> Either (Anchor (Header blk)) (Header blk)
forall v a b. Anchorable v a b => AnchoredSeq v a b -> Either a b
AF.head AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
newChain of
            Left Anchor (Header blk)
_anchor -> [Char]
-> (RealPoint (Header blk), (EpochNo, Word64),
    SelectView (BlockProtocol blk))
forall a. HasCallStack => [Char] -> a
error [Char]
"cannot have switched to an empty chain"
            Right Header blk
tipHdr ->
              let query :: Qry (EpochNo, Word64)
query        = SlotNo -> Qry (EpochNo, Word64)
History.slotToEpoch' (Header blk -> SlotNo
forall b. HasHeader b => b -> SlotNo
blockSlot Header blk
tipHdr)
                  tipEpochData :: (EpochNo, Word64)
tipEpochData = Qry (EpochNo, Word64)
-> Summary (HardForkIndices blk) -> (EpochNo, Word64)
forall a (xs :: [*]). HasCallStack => Qry a -> Summary xs -> a
History.runQueryPure Qry (EpochNo, Word64)
query Summary (HardForkIndices blk)
summary
                  sv :: SelectView (BlockProtocol blk)
sv           = BlockConfig blk -> Header blk -> SelectView (BlockProtocol blk)
forall blk.
BlockSupportsProtocol blk =>
BlockConfig blk -> Header blk -> SelectView (BlockProtocol blk)
selectView (TopLevelConfig blk -> BlockConfig blk
forall blk. TopLevelConfig blk -> BlockConfig blk
configBlock TopLevelConfig blk
cfg) Header blk
tipHdr
              in (Header blk -> RealPoint (Header blk)
forall blk. HasHeader blk => blk -> RealPoint blk
blockRealPoint Header blk
tipHdr, (EpochNo, Word64)
tipEpochData, SelectView (BlockProtocol blk)
sv)

    -- | Try to apply the given 'ChainDiff' on the current chain fragment. The
    -- 'LedgerDB' is updated in the same transaction.
    --
    -- Note that we /cannot/ have switched to a different current chain in the
    -- meantime, since this function will only be called by a single
    -- background thread.
    --
    -- It /is/ possible that the background thread copying headers older than
    -- @k@ from the VolatileDB to the ImmutableDB has removed some headers
    -- from the beginning of the current chain fragment, but does not affect
    -- us, as we cannot roll back more than @k@ headers anyway.
    switchTo
      :: HasCallStack
      => ValidatedChainDiff (Header blk) (Forker' m blk)
         -- ^ Chain and ledger to switch to
      -> StrictTVar m (StrictMaybe (Header blk))
         -- ^ Tentative header
      -> ChainSwitchType
      -> m ()
    switchTo :: HasCallStack =>
ValidatedChainDiff (Header blk) (Forker m (ExtLedgerState blk) blk)
-> StrictTVar m (StrictMaybe (Header blk))
-> ChainSwitchType
-> m ()
switchTo ValidatedChainDiff (Header blk) (Forker m (ExtLedgerState blk) blk)
vChainDiff StrictTVar m (StrictMaybe (Header blk))
varTentativeHeader ChainSwitchType
chainSwitchType = do
        Tracer m (TraceAddBlockEvent blk) -> TraceAddBlockEvent blk -> m ()
forall (m :: * -> *) a. Tracer m a -> a -> m ()
traceWith Tracer m (TraceAddBlockEvent blk)
addBlockTracer (TraceAddBlockEvent blk -> m ()) -> TraceAddBlockEvent blk -> m ()
forall a b. (a -> b) -> a -> b
$
            Point blk -> TraceAddBlockEvent blk
forall blk. Point blk -> TraceAddBlockEvent blk
ChangingSelection
          (Point blk -> TraceAddBlockEvent blk)
-> Point blk -> TraceAddBlockEvent blk
forall a b. (a -> b) -> a -> b
$ Point (Header blk) -> Point blk
forall {k1} {k2} (b :: k1) (b' :: k2).
Coercible (HeaderHash b) (HeaderHash b') =>
Point b -> Point b'
castPoint
          (Point (Header blk) -> Point blk)
-> Point (Header blk) -> Point blk
forall a b. (a -> b) -> a -> b
$ AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> Point (Header blk)
forall block.
HasHeader block =>
AnchoredFragment block -> Point block
AF.headPoint
          (AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
 -> Point (Header blk))
-> AnchoredSeq
     (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> Point (Header blk)
forall a b. (a -> b) -> a -> b
$ ChainDiff (Header blk)
-> AnchoredSeq
     (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
forall b. ChainDiff b -> AnchoredFragment b
getSuffix
          (ChainDiff (Header blk)
 -> AnchoredSeq
      (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
-> ChainDiff (Header blk)
-> AnchoredSeq
     (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
forall a b. (a -> b) -> a -> b
$ ValidatedChainDiff (Header blk) (Forker m (ExtLedgerState blk) blk)
-> ChainDiff (Header blk)
forall b l. ValidatedChainDiff b l -> ChainDiff b
getChainDiff ValidatedChainDiff (Header blk) (Forker m (ExtLedgerState blk) blk)
vChainDiff
        (curChain, newChain, events, prevTentativeHeader, newLedger) <- STM
  m
  (AnchoredSeq
     (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk),
   AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk),
   [LedgerEvent blk], StrictMaybe (Header blk),
   ExtLedgerState blk EmptyMK)
-> m (AnchoredSeq
        (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk),
      AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk),
      [LedgerEvent blk], StrictMaybe (Header blk),
      ExtLedgerState blk EmptyMK)
forall a. HasCallStack => STM m a -> m a
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
STM m a -> m a
atomically (STM
   m
   (AnchoredSeq
      (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk),
    AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk),
    [LedgerEvent blk], StrictMaybe (Header blk),
    ExtLedgerState blk EmptyMK)
 -> m (AnchoredSeq
         (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk),
       AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk),
       [LedgerEvent blk], StrictMaybe (Header blk),
       ExtLedgerState blk EmptyMK))
-> STM
     m
     (AnchoredSeq
        (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk),
      AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk),
      [LedgerEvent blk], StrictMaybe (Header blk),
      ExtLedgerState blk EmptyMK)
-> m (AnchoredSeq
        (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk),
      AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk),
      [LedgerEvent blk], StrictMaybe (Header blk),
      ExtLedgerState blk EmptyMK)
forall a b. (a -> b) -> a -> b
$ do
          InternalChain curChain curChainWithTime <- StrictTVar m (InternalChain blk) -> STM m (InternalChain blk)
forall (m :: * -> *) a. MonadSTM m => StrictTVar m a -> STM m a
readTVar StrictTVar m (InternalChain blk)
cdbChain -- Not Query.getCurrentChain!
          curLedger <- getVolatileTip cdbLedgerDB
          newLedger <- forkerGetLedgerState newForker
          case Diff.apply curChain chainDiff of
            -- Impossible, as described in the docstring
            Maybe
  (AnchoredSeq
     (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
Nothing       ->
              [Char]
-> STM
     m
     (AnchoredSeq
        (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk),
      AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk),
      [LedgerEvent blk], StrictMaybe (Header blk),
      ExtLedgerState blk EmptyMK)
forall a. HasCallStack => [Char] -> a
error [Char]
"chainDiff doesn't fit onto current chain"
            Just AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
newChain -> do
              let lcfg :: LedgerConfig blk
lcfg             = TopLevelConfig blk -> LedgerConfig blk
forall blk. TopLevelConfig blk -> LedgerConfig blk
configLedger TopLevelConfig blk
cdbTopLevelConfig
                  diffWithTime :: ChainDiff (HeaderWithTime blk)
diffWithTime     =
                    -- the new ledger state can translate the slots of the new
                    -- headers
                    (Header blk -> HeaderWithTime blk)
-> ChainDiff (Header blk) -> ChainDiff (HeaderWithTime blk)
forall a b.
(HasHeader b, HeaderHash a ~ HeaderHash b) =>
(a -> b) -> ChainDiff a -> ChainDiff b
Diff.map
                      (LedgerConfig blk
-> LedgerState blk EmptyMK -> Header blk -> HeaderWithTime blk
forall blk (mk :: MapKind).
(HasHardForkHistory blk, HasHeader (Header blk)) =>
LedgerConfig blk
-> LedgerState blk mk -> Header blk -> HeaderWithTime blk
mkHeaderWithTime
                         LedgerConfig blk
lcfg
                         (ExtLedgerState blk EmptyMK -> LedgerState blk EmptyMK
forall blk (mk :: MapKind).
ExtLedgerState blk mk -> LedgerState blk mk
ledgerState ExtLedgerState blk EmptyMK
newLedger)
                      )
                      ChainDiff (Header blk)
chainDiff
                  newChainWithTime :: AnchoredFragment (HeaderWithTime blk)
newChainWithTime =
                    case AnchoredFragment (HeaderWithTime blk)
-> ChainDiff (HeaderWithTime blk)
-> Maybe (AnchoredFragment (HeaderWithTime blk))
forall b.
HasHeader b =>
AnchoredFragment b -> ChainDiff b -> Maybe (AnchoredFragment b)
Diff.apply AnchoredFragment (HeaderWithTime blk)
curChainWithTime ChainDiff (HeaderWithTime blk)
diffWithTime of
                      Maybe (AnchoredFragment (HeaderWithTime blk))
Nothing -> [Char] -> AnchoredFragment (HeaderWithTime blk)
forall a. HasCallStack => [Char] -> a
error [Char]
"chainDiff failed for HeaderWithTime"
                      Just AnchoredFragment (HeaderWithTime blk)
x  -> AnchoredFragment (HeaderWithTime blk)
x

              StrictTVar m (InternalChain blk) -> InternalChain blk -> STM m ()
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
StrictTVar m a -> a -> STM m ()
writeTVar        StrictTVar m (InternalChain blk)
cdbChain (InternalChain blk -> STM m ()) -> InternalChain blk -> STM m ()
forall a b. (a -> b) -> a -> b
$ AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> AnchoredFragment (HeaderWithTime blk) -> InternalChain blk
forall blk.
AnchoredFragment (Header blk)
-> AnchoredFragment (HeaderWithTime blk) -> InternalChain blk
InternalChain AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
newChain AnchoredFragment (HeaderWithTime blk)
newChainWithTime
              Forker m (ExtLedgerState blk) blk -> STM m ()
forall (m :: * -> *) (l :: LedgerStateKind) blk.
Forker m l blk -> STM m ()
forkerCommit Forker m (ExtLedgerState blk) blk
newForker

              -- Inspect the new ledger for potential problems
              let events :: [LedgerEvent blk]
                  events :: [LedgerEvent blk]
events = TopLevelConfig blk
-> LedgerState blk EmptyMK
-> LedgerState blk EmptyMK
-> [LedgerEvent blk]
forall blk (mk1 :: MapKind) (mk2 :: MapKind).
InspectLedger blk =>
TopLevelConfig blk
-> LedgerState blk mk1 -> LedgerState blk mk2 -> [LedgerEvent blk]
forall (mk1 :: MapKind) (mk2 :: MapKind).
TopLevelConfig blk
-> LedgerState blk mk1 -> LedgerState blk mk2 -> [LedgerEvent blk]
inspectLedger
                             TopLevelConfig blk
cdbTopLevelConfig
                             (ExtLedgerState blk EmptyMK -> LedgerState blk EmptyMK
forall blk (mk :: MapKind).
ExtLedgerState blk mk -> LedgerState blk mk
ledgerState ExtLedgerState blk EmptyMK
curLedger)
                             (ExtLedgerState blk EmptyMK -> LedgerState blk EmptyMK
forall blk (mk :: MapKind).
ExtLedgerState blk mk -> LedgerState blk mk
ledgerState ExtLedgerState blk EmptyMK
newLedger)

              -- Clear the tentative header
              prevTentativeHeader <- StrictTVar m (StrictMaybe (Header blk))
-> StrictMaybe (Header blk) -> STM m (StrictMaybe (Header blk))
forall (m :: * -> *) a.
MonadSTM m =>
StrictTVar m a -> a -> STM m a
swapTVar StrictTVar m (StrictMaybe (Header blk))
varTentativeHeader StrictMaybe (Header blk)
forall a. StrictMaybe a
SNothing

              case chainSwitchType of
                -- When adding blocks, the intersection point of the old and new
                -- tentative/selected chain is not receding, in which case
                -- `fhSwitchFork` is unnecessary. In the case of pipelining a
                -- block, it would even result in rolling back by one block and
                -- rolling forward again.
                ChainSwitchType
AddingBlocks      -> () -> STM m ()
forall a. a -> STM m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure ()
                ChainSwitchType
SwitchingToAFork -> do
                  -- Update the followers
                  --
                  -- 'Follower.switchFork' needs to know the intersection point
                  -- (@ipoint@) between the old and the current chain.
                  let ipoint :: Point blk
ipoint = Point (Header blk) -> Point blk
forall {k1} {k2} (b :: k1) (b' :: k2).
Coercible (HeaderHash b) (HeaderHash b') =>
Point b -> Point b'
castPoint (Point (Header blk) -> Point blk)
-> Point (Header blk) -> Point blk
forall a b. (a -> b) -> a -> b
$ ChainDiff (Header blk) -> Point (Header blk)
forall b. ChainDiff b -> Point b
Diff.getAnchorPoint ChainDiff (Header blk)
chainDiff
                  followerHandles <- Map FollowerKey (FollowerHandle m blk) -> [FollowerHandle m blk]
forall k a. Map k a -> [a]
Map.elems (Map FollowerKey (FollowerHandle m blk) -> [FollowerHandle m blk])
-> STM m (Map FollowerKey (FollowerHandle m blk))
-> STM m [FollowerHandle m blk]
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> StrictTVar m (Map FollowerKey (FollowerHandle m blk))
-> STM m (Map FollowerKey (FollowerHandle m blk))
forall (m :: * -> *) a. MonadSTM m => StrictTVar m a -> STM m a
readTVar StrictTVar m (Map FollowerKey (FollowerHandle m blk))
cdbFollowers
                  forM_ followerHandles $ switchFollowerToFork curChain newChain ipoint

              return (curChain, newChain, events, prevTentativeHeader, newLedger)
        let mkTraceEvent = case ChainSwitchType
chainSwitchType of
              ChainSwitchType
AddingBlocks     -> [LedgerEvent blk]
-> SelectionChangedInfo blk
-> AnchoredSeq
     (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> AnchoredSeq
     (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> TraceAddBlockEvent blk
forall blk.
[LedgerEvent blk]
-> SelectionChangedInfo blk
-> AnchoredFragment (Header blk)
-> AnchoredFragment (Header blk)
-> TraceAddBlockEvent blk
AddedToCurrentChain
              ChainSwitchType
SwitchingToAFork -> [LedgerEvent blk]
-> SelectionChangedInfo blk
-> AnchoredSeq
     (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> AnchoredSeq
     (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> TraceAddBlockEvent blk
forall blk.
[LedgerEvent blk]
-> SelectionChangedInfo blk
-> AnchoredFragment (Header blk)
-> AnchoredFragment (Header blk)
-> TraceAddBlockEvent blk
SwitchedToAFork
            selChangedInfo = AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> AnchoredSeq
     (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> ExtLedgerState blk EmptyMK
-> SelectionChangedInfo blk
mkSelectionChangedInfo AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
curChain AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
newChain ExtLedgerState blk EmptyMK
newLedger
        traceWith addBlockTracer $
          mkTraceEvent events selChangedInfo curChain newChain
        whenJust (strictMaybeToMaybe prevTentativeHeader) $ traceWith $
          PipeliningEvent . OutdatedTentativeHeader >$< addBlockTracer

        forkerClose newForker

      where
        -- Given the current chain and the new chain as chain fragments, and the
        -- intersection point (an optimization, since it has already been
        -- computed when calling this function), returns a function that updates
        -- the state of a follower via its handle.
        switchFollowerToFork :: AnchoredFragment (Header blk)
-> AnchoredFragment block
-> Point blk
-> FollowerHandle m blk
-> STM m ()
switchFollowerToFork AnchoredFragment (Header blk)
curChain AnchoredFragment block
newChain Point blk
ipoint =
          let oldPoints :: Set (Point blk)
oldPoints = [Point blk] -> Set (Point blk)
forall a. Ord a => [a] -> Set a
Set.fromList ([Point blk] -> Set (Point blk))
-> (AnchoredFragment (Header blk) -> [Point blk])
-> AnchoredFragment (Header blk)
-> Set (Point blk)
forall b c a. (b -> c) -> (a -> b) -> a -> c
. (Header blk -> Point blk) -> [Header blk] -> [Point blk]
forall a b. (a -> b) -> [a] -> [b]
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap Header blk -> Point blk
forall blk. HasHeader (Header blk) => Header blk -> Point blk
headerPoint ([Header blk] -> [Point blk])
-> (AnchoredFragment (Header blk) -> [Header blk])
-> AnchoredFragment (Header blk)
-> [Point blk]
forall b c a. (b -> c) -> (a -> b) -> a -> c
. AnchoredFragment (Header blk) -> [Header blk]
forall v a b. AnchoredSeq v a b -> [b]
AS.toOldestFirst
                        (AnchoredFragment (Header blk) -> Set (Point blk))
-> AnchoredFragment (Header blk) -> Set (Point blk)
forall a b. (a -> b) -> a -> b
$ ChainDiff (Header blk) -> AnchoredFragment (Header blk)
forall b. ChainDiff b -> AnchoredFragment b
Diff.getSuffix
                        (ChainDiff (Header blk) -> AnchoredFragment (Header blk))
-> ChainDiff (Header blk) -> AnchoredFragment (Header blk)
forall a b. (a -> b) -> a -> b
$ AnchoredFragment block
-> AnchoredFragment (Header blk) -> ChainDiff (Header blk)
forall b b'.
(HasHeader b, HasHeader b', HeaderHash b ~ HeaderHash b',
 HasCallStack) =>
AnchoredFragment b' -> AnchoredFragment b -> ChainDiff b
Diff.diff AnchoredFragment block
newChain AnchoredFragment (Header blk)
curChain
          in Bool
-> (FollowerHandle m blk -> STM m ())
-> FollowerHandle m blk
-> STM m ()
forall a. HasCallStack => Bool -> a -> a
assert (Point block -> AnchoredFragment block -> Bool
forall block.
HasHeader block =>
Point block -> AnchoredFragment block -> Bool
AF.withinFragmentBounds (Point blk -> Point block
forall {k1} {k2} (b :: k1) (b' :: k2).
Coercible (HeaderHash b) (HeaderHash b') =>
Point b -> Point b'
castPoint Point blk
ipoint) AnchoredFragment block
newChain) ((FollowerHandle m blk -> STM m ())
 -> FollowerHandle m blk -> STM m ())
-> (FollowerHandle m blk -> STM m ())
-> FollowerHandle m blk
-> STM m ()
forall a b. (a -> b) -> a -> b
$
             \FollowerHandle m blk
followerHandle -> FollowerHandle m blk -> Point blk -> Set (Point blk) -> STM m ()
forall (m :: * -> *) blk.
FollowerHandle m blk -> Point blk -> Set (Point blk) -> STM m ()
fhSwitchFork FollowerHandle m blk
followerHandle Point blk
ipoint Set (Point blk)
oldPoints

        ValidatedChainDiff ChainDiff (Header blk)
chainDiff Forker m (ExtLedgerState blk) blk
newForker = ValidatedChainDiff (Header blk) (Forker m (ExtLedgerState blk) blk)
vChainDiff

    -- | We have a new block @b@ that doesn't fit onto the current chain, but
    -- we have found a 'ChainDiff' connecting it to the current chain via
    -- intersection point @x@. We may also have extended that 'ChainDiff' with
    -- more blocks fitting onto @b@, i.e., a suffix @s@.
    --
    -- We now translate that 'ChainDiff' from 'HeaderFields' to 'Header's by
    -- reading the headers from disk.
    --
    -- Note that we need to read the headers corresponding to the hashes
    -- @(x,b)@ and @(b,?]@ from disk. Not for @b@, as that's in our cache.
    translateToHeaders
      :: ChainDiff (HeaderFields blk)
      -> StateT (Map (HeaderHash blk) (Header blk))
                m
                (ChainDiff (Header blk))
         -- ^ Fork, anchored at @x@, contains (the header of) @b@ and ends
         -- with the suffix @s@.
    translateToHeaders :: ChainDiff (HeaderFields blk)
-> StateT
     (Map (HeaderHash blk) (Header blk)) m (ChainDiff (Header blk))
translateToHeaders =
        (HeaderFields blk
 -> StateT (Map (HeaderHash blk) (Header blk)) m (Header blk))
-> ChainDiff (HeaderFields blk)
-> StateT
     (Map (HeaderHash blk) (Header blk)) m (ChainDiff (Header blk))
forall a b (m :: * -> *).
(HasHeader b, HeaderHash a ~ HeaderHash b, Monad m) =>
(a -> m b) -> ChainDiff a -> m (ChainDiff b)
Diff.mapM (VolatileDB m blk
-> HeaderHash blk
-> StateT (Map (HeaderHash blk) (Header blk)) m (Header blk)
forall (m :: * -> *) blk.
(MonadThrow m, HasHeader blk) =>
VolatileDB m blk
-> HeaderHash blk
-> StateT (Map (HeaderHash blk) (Header blk)) m (Header blk)
getKnownHeaderThroughCache VolatileDB m blk
cdbVolatileDB (HeaderHash blk
 -> StateT (Map (HeaderHash blk) (Header blk)) m (Header blk))
-> (HeaderFields blk -> HeaderHash blk)
-> HeaderFields blk
-> StateT (Map (HeaderHash blk) (Header blk)) m (Header blk)
forall b c a. (b -> c) -> (a -> b) -> a -> c
. HeaderFields blk -> HeaderHash blk
forall k (b :: k). HeaderFields b -> HeaderHash b
headerFieldHash)

-- | Check whether the header for the hash is in the cache, if not, get
-- the corresponding header from the VolatileDB and store it in the cache.
--
-- PRECONDITION: the header (block) must exist in the VolatileDB.
getKnownHeaderThroughCache ::
     (MonadThrow m, HasHeader blk)
  => VolatileDB m blk
  -> HeaderHash blk
  -> StateT (Map (HeaderHash blk) (Header blk)) m (Header blk)
getKnownHeaderThroughCache :: forall (m :: * -> *) blk.
(MonadThrow m, HasHeader blk) =>
VolatileDB m blk
-> HeaderHash blk
-> StateT (Map (HeaderHash blk) (Header blk)) m (Header blk)
getKnownHeaderThroughCache VolatileDB m blk
volatileDB HeaderHash blk
hash = (Map (HeaderHash blk) (Header blk) -> Maybe (Header blk))
-> StateT
     (Map (HeaderHash blk) (Header blk)) m (Maybe (Header blk))
forall (m :: * -> *) s a. Monad m => (s -> a) -> StateT s m a
gets (HeaderHash blk
-> Map (HeaderHash blk) (Header blk) -> Maybe (Header blk)
forall k a. Ord k => k -> Map k a -> Maybe a
Map.lookup HeaderHash blk
hash) StateT (Map (HeaderHash blk) (Header blk)) m (Maybe (Header blk))
-> (Maybe (Header blk)
    -> StateT (Map (HeaderHash blk) (Header blk)) m (Header blk))
-> StateT (Map (HeaderHash blk) (Header blk)) m (Header blk)
forall a b.
StateT (Map (HeaderHash blk) (Header blk)) m a
-> (a -> StateT (Map (HeaderHash blk) (Header blk)) m b)
-> StateT (Map (HeaderHash blk) (Header blk)) m b
forall (m :: * -> *) a b. Monad m => m a -> (a -> m b) -> m b
>>= \case
    Just Header blk
hdr -> Header blk
-> StateT (Map (HeaderHash blk) (Header blk)) m (Header blk)
forall a. a -> StateT (Map (HeaderHash blk) (Header blk)) m a
forall (m :: * -> *) a. Monad m => a -> m a
return Header blk
hdr
    Maybe (Header blk)
Nothing  -> do
      hdr <- m (Header blk)
-> StateT (Map (HeaderHash blk) (Header blk)) m (Header blk)
forall (m :: * -> *) a.
Monad m =>
m a -> StateT (Map (HeaderHash blk) (Header blk)) m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (m (Header blk)
 -> StateT (Map (HeaderHash blk) (Header blk)) m (Header blk))
-> m (Header blk)
-> StateT (Map (HeaderHash blk) (Header blk)) m (Header blk)
forall a b. (a -> b) -> a -> b
$ VolatileDB m blk
-> BlockComponent blk (Header blk)
-> HeaderHash blk
-> m (Header blk)
forall (m :: * -> *) blk b.
(MonadThrow m, HasHeader blk) =>
VolatileDB m blk -> BlockComponent blk b -> HeaderHash blk -> m b
VolatileDB.getKnownBlockComponent VolatileDB m blk
volatileDB BlockComponent blk (Header blk)
forall blk. BlockComponent blk (Header blk)
GetHeader HeaderHash blk
hash
      modify (Map.insert hash hdr)
      return hdr

-- | Environment used by 'chainSelection' and related functions.
data ChainSelEnv m blk = ChainSelEnv
    { forall (m :: * -> *) blk. ChainSelEnv m blk -> LedgerDB' m blk
lgrDB                 :: LedgerDB.LedgerDB' m blk
    , forall (m :: * -> *) blk.
ChainSelEnv m blk -> Tracer m (TraceValidationEvent blk)
validationTracer      :: Tracer m (TraceValidationEvent blk)
    , forall (m :: * -> *) blk.
ChainSelEnv m blk -> Tracer m (TracePipeliningEvent blk)
pipeliningTracer      :: Tracer m (TracePipeliningEvent blk)
    , forall (m :: * -> *) blk. ChainSelEnv m blk -> BlockConfig blk
bcfg                  :: BlockConfig blk
    , forall (m :: * -> *) blk.
ChainSelEnv m blk
-> StrictTVar m (WithFingerprint (InvalidBlocks blk))
varInvalid            :: StrictTVar m (WithFingerprint (InvalidBlocks blk))
    , forall (m :: * -> *) blk.
ChainSelEnv m blk -> StrictTVar m (TentativeHeaderState blk)
varTentativeState     :: StrictTVar m (TentativeHeaderState blk)
    , forall (m :: * -> *) blk.
ChainSelEnv m blk -> StrictTVar m (StrictMaybe (Header blk))
varTentativeHeader    :: StrictTVar m (StrictMaybe (Header blk))
    , forall (m :: * -> *) blk.
ChainSelEnv m blk -> STM m [FollowerHandle m blk]
getTentativeFollowers :: STM m [FollowerHandle m blk]
    , forall (m :: * -> *) blk. ChainSelEnv m blk -> BlockCache blk
blockCache            :: BlockCache blk
    , forall (m :: * -> *) blk. ChainSelEnv m blk -> ChainAndLedger m blk
curChainAndLedger     :: ChainAndLedger m blk
      -- | The block that this chain selection invocation is processing, and the
      -- punish action for the peer that sent that block; see
      -- 'InvalidBlockPunishment'.
      --
      -- One subtlety:
      --
      -- o If a BlockFetch client adds an invalid block but that block isn't
      --   part of any desirable paths through the VolDB, then we won't attempt
      --   to validate it and so we won't discover it's invalid. The peer will
      --   not be punished. This seems acceptable, since it means we have turned
      --   our focus to a another peer offering better blocks and so this peer
      --   is no longer causing us BlockFetch work.
      --
      -- Thus invalid blocks can be skipped entirely. This is part of
      -- the reason we bothered to restrict the expressiveness of the
      -- 'InvalidBlockPunishment' combinators.
    , forall (m :: * -> *) blk.
ChainSelEnv m blk
-> Maybe (RealPoint blk, InvalidBlockPunishment m)
punish                :: Maybe (RealPoint blk, InvalidBlockPunishment m)
    }

-- | Perform chain selection with the given candidates. If a validated
-- candidate was chosen to replace the current chain, return it along with the
-- corresponding ledger.
--
-- PRECONDITION: all candidates must be preferred over the current chain.
--
-- PRECONDITION: the candidate chain diffs must fit on the (given) current
-- chain.
chainSelection ::
     forall m blk.
     ( IOLike m
     , LedgerSupportsProtocol blk
     , BlockSupportsDiffusionPipelining blk
     , HasCallStack
     )
  => ChainSelEnv m blk
  -> ResourceRegistry m
  -> NonEmpty (ChainDiff (Header blk))
  -> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
     -- ^ The (valid) chain diff and corresponding LedgerDB that was selected,
     -- or 'Nothing' if there is no valid chain diff preferred over the current
     -- chain.
chainSelection :: forall (m :: * -> *) blk.
(IOLike m, LedgerSupportsProtocol blk,
 BlockSupportsDiffusionPipelining blk, HasCallStack) =>
ChainSelEnv m blk
-> ResourceRegistry m
-> NonEmpty (ChainDiff (Header blk))
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
chainSelection ChainSelEnv m blk
chainSelEnv ResourceRegistry m
rr NonEmpty (ChainDiff (Header blk))
chainDiffs =
    Bool
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
forall a. HasCallStack => Bool -> a -> a
assert ((ChainDiff (Header blk) -> Bool)
-> NonEmpty (ChainDiff (Header blk)) -> Bool
forall (t :: * -> *) a. Foldable t => (a -> Bool) -> t a -> Bool
all (BlockConfig blk
-> AnchoredFragment (Header blk)
-> AnchoredFragment (Header blk)
-> Bool
forall blk (h :: * -> *) (h' :: * -> *).
(BlockSupportsProtocol blk, HasCallStack, GetHeader1 h,
 GetHeader1 h', HeaderHash (h blk) ~ HeaderHash (h' blk),
 HasHeader (h blk), HasHeader (h' blk)) =>
BlockConfig blk
-> AnchoredFragment (h blk) -> AnchoredFragment (h' blk) -> Bool
preferAnchoredCandidate BlockConfig blk
bcfg AnchoredFragment (Header blk)
curChain (AnchoredFragment (Header blk) -> Bool)
-> (ChainDiff (Header blk) -> AnchoredFragment (Header blk))
-> ChainDiff (Header blk)
-> Bool
forall b c a. (b -> c) -> (a -> b) -> a -> c
. ChainDiff (Header blk) -> AnchoredFragment (Header blk)
forall b. ChainDiff b -> AnchoredFragment b
Diff.getSuffix)
                NonEmpty (ChainDiff (Header blk))
chainDiffs) (m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
 -> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk))))
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
forall a b. (a -> b) -> a -> b
$
    Bool
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
forall a. HasCallStack => Bool -> a -> a
assert ((ChainDiff (Header blk) -> Bool)
-> NonEmpty (ChainDiff (Header blk)) -> Bool
forall (t :: * -> *) a. Foldable t => (a -> Bool) -> t a -> Bool
all (Maybe (AnchoredFragment (Header blk)) -> Bool
forall a. Maybe a -> Bool
isJust (Maybe (AnchoredFragment (Header blk)) -> Bool)
-> (ChainDiff (Header blk)
    -> Maybe (AnchoredFragment (Header blk)))
-> ChainDiff (Header blk)
-> Bool
forall b c a. (b -> c) -> (a -> b) -> a -> c
. AnchoredFragment (Header blk)
-> ChainDiff (Header blk) -> Maybe (AnchoredFragment (Header blk))
forall b.
HasHeader b =>
AnchoredFragment b -> ChainDiff b -> Maybe (AnchoredFragment b)
Diff.apply AnchoredFragment (Header blk)
curChain)
                NonEmpty (ChainDiff (Header blk))
chainDiffs) (m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
 -> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk))))
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
forall a b. (a -> b) -> a -> b
$
    [ChainDiff (Header blk)]
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
go ([ChainDiff (Header blk)] -> [ChainDiff (Header blk)]
sortCandidates (NonEmpty (ChainDiff (Header blk)) -> [ChainDiff (Header blk)]
forall a. NonEmpty a -> [a]
NE.toList NonEmpty (ChainDiff (Header blk))
chainDiffs))
  where
    ChainSelEnv {Maybe (RealPoint blk, InvalidBlockPunishment m)
Tracer m (TracePipeliningEvent blk)
Tracer m (TraceValidationEvent blk)
STM m [FollowerHandle m blk]
StrictTVar m (StrictMaybe (Header blk))
StrictTVar m (TentativeHeaderState blk)
StrictTVar m (WithFingerprint (InvalidBlocks blk))
BlockConfig blk
BlockCache blk
ChainAndLedger m blk
LedgerDB' m blk
lgrDB :: forall (m :: * -> *) blk. ChainSelEnv m blk -> LedgerDB' m blk
bcfg :: forall (m :: * -> *) blk. ChainSelEnv m blk -> BlockConfig blk
varInvalid :: forall (m :: * -> *) blk.
ChainSelEnv m blk
-> StrictTVar m (WithFingerprint (InvalidBlocks blk))
blockCache :: forall (m :: * -> *) blk. ChainSelEnv m blk -> BlockCache blk
curChainAndLedger :: forall (m :: * -> *) blk. ChainSelEnv m blk -> ChainAndLedger m blk
validationTracer :: forall (m :: * -> *) blk.
ChainSelEnv m blk -> Tracer m (TraceValidationEvent blk)
pipeliningTracer :: forall (m :: * -> *) blk.
ChainSelEnv m blk -> Tracer m (TracePipeliningEvent blk)
varTentativeState :: forall (m :: * -> *) blk.
ChainSelEnv m blk -> StrictTVar m (TentativeHeaderState blk)
varTentativeHeader :: forall (m :: * -> *) blk.
ChainSelEnv m blk -> StrictTVar m (StrictMaybe (Header blk))
punish :: forall (m :: * -> *) blk.
ChainSelEnv m blk
-> Maybe (RealPoint blk, InvalidBlockPunishment m)
getTentativeFollowers :: forall (m :: * -> *) blk.
ChainSelEnv m blk -> STM m [FollowerHandle m blk]
bcfg :: BlockConfig blk
lgrDB :: LedgerDB' m blk
validationTracer :: Tracer m (TraceValidationEvent blk)
pipeliningTracer :: Tracer m (TracePipeliningEvent blk)
varInvalid :: StrictTVar m (WithFingerprint (InvalidBlocks blk))
varTentativeState :: StrictTVar m (TentativeHeaderState blk)
varTentativeHeader :: StrictTVar m (StrictMaybe (Header blk))
getTentativeFollowers :: STM m [FollowerHandle m blk]
blockCache :: BlockCache blk
curChainAndLedger :: ChainAndLedger m blk
punish :: Maybe (RealPoint blk, InvalidBlockPunishment m)
..} = ChainSelEnv m blk
chainSelEnv

    curChain :: AnchoredFragment (Header blk)
curChain = ChainAndLedger m blk -> AnchoredFragment (Header blk)
forall b l. ValidatedFragment b l -> AnchoredFragment b
VF.validatedFragment ChainAndLedger m blk
curChainAndLedger

    sortCandidates :: [ChainDiff (Header blk)] -> [ChainDiff (Header blk)]
    sortCandidates :: [ChainDiff (Header blk)] -> [ChainDiff (Header blk)]
sortCandidates =
      (ChainDiff (Header blk) -> ChainDiff (Header blk) -> Ordering)
-> [ChainDiff (Header blk)] -> [ChainDiff (Header blk)]
forall a. (a -> a -> Ordering) -> [a] -> [a]
sortBy ((AnchoredFragment (Header blk)
 -> AnchoredFragment (Header blk) -> Ordering)
-> AnchoredFragment (Header blk)
-> AnchoredFragment (Header blk)
-> Ordering
forall a b c. (a -> b -> c) -> b -> a -> c
flip (BlockConfig blk
-> AnchoredFragment (Header blk)
-> AnchoredFragment (Header blk)
-> Ordering
forall blk (h :: * -> *).
(BlockSupportsProtocol blk, HasCallStack, GetHeader1 h,
 HasHeader (h blk)) =>
BlockConfig blk
-> AnchoredFragment (h blk) -> AnchoredFragment (h blk) -> Ordering
compareAnchoredFragments BlockConfig blk
bcfg) (AnchoredFragment (Header blk)
 -> AnchoredFragment (Header blk) -> Ordering)
-> (ChainDiff (Header blk) -> AnchoredFragment (Header blk))
-> ChainDiff (Header blk)
-> ChainDiff (Header blk)
-> Ordering
forall b c a. (b -> b -> c) -> (a -> b) -> a -> a -> c
`on` ChainDiff (Header blk) -> AnchoredFragment (Header blk)
forall b. ChainDiff b -> AnchoredFragment b
Diff.getSuffix)

    -- 1. Take the first candidate from the list of sorted candidates
    -- 2. Validate it
    --    - If it is invalid -> discard it and go to 1 with the rest of the
    --      list.
    --    - If it is valid and has the same tip -> return it
    --    - If it is valid, but is a prefix of the original ->
    --        add it to the list, sort it and go to 1. See the comment
    --        [Ouroboros] below.
    go ::
         [ChainDiff (Header blk)]
      -> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
    go :: [ChainDiff (Header blk)]
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
go []            = Maybe (ValidatedChainDiff (Header blk) (Forker' m blk))
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return Maybe (ValidatedChainDiff (Header blk) (Forker' m blk))
forall a. Maybe a
Nothing
    go (ChainDiff (Header blk)
candidate:[ChainDiff (Header blk)]
candidates0) = do
        mTentativeHeader <- m (Maybe (Header blk, TentativeHeaderState blk))
setTentativeHeader
        validateCandidate chainSelEnv rr candidate >>= \case
          ValidationResult m blk
InsufficientSuffix ->
            -- When the body of the tentative block turns out to be invalid, we
            -- have a valid *empty* prefix, as the tentative header fits on top
            -- of the current chain.
            Bool
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
forall a. HasCallStack => Bool -> a -> a
assert (Maybe (Header blk, TentativeHeaderState blk) -> Bool
forall a. Maybe a -> Bool
isNothing Maybe (Header blk, TentativeHeaderState blk)
mTentativeHeader) (m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
 -> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk))))
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
forall a b. (a -> b) -> a -> b
$ do
              candidates1 <- [ChainDiff (Header blk)] -> m [ChainDiff (Header blk)]
truncateRejectedBlocks [ChainDiff (Header blk)]
candidates0
              go (sortCandidates candidates1)
          FullyValid validatedCandidate :: ValidatedChainDiff (Header blk) (Forker' m blk)
validatedCandidate@(ValidatedChainDiff ChainDiff (Header blk)
candidate' Forker' m blk
_) ->
            -- The entire candidate is valid
            Bool
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
forall a. HasCallStack => Bool -> a -> a
assert (ChainDiff (Header blk) -> Point (Header blk)
forall b. HasHeader b => ChainDiff b -> Point b
Diff.getTip ChainDiff (Header blk)
candidate Point (Header blk) -> Point (Header blk) -> Bool
forall a. Eq a => a -> a -> Bool
== ChainDiff (Header blk) -> Point (Header blk)
forall b. HasHeader b => ChainDiff b -> Point b
Diff.getTip ChainDiff (Header blk)
candidate') (m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
 -> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk))))
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
forall a b. (a -> b) -> a -> b
$
            Maybe (ValidatedChainDiff (Header blk) (Forker' m blk))
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk))
 -> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk))))
-> Maybe (ValidatedChainDiff (Header blk) (Forker' m blk))
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
forall a b. (a -> b) -> a -> b
$ ValidatedChainDiff (Header blk) (Forker' m blk)
-> Maybe (ValidatedChainDiff (Header blk) (Forker' m blk))
forall a. a -> Maybe a
Just ValidatedChainDiff (Header blk) (Forker' m blk)
validatedCandidate
          ValidPrefix ChainDiff (Header blk)
candidate' -> do
            Maybe (Header blk, TentativeHeaderState blk)
-> ((Header blk, TentativeHeaderState blk) -> m ()) -> m ()
forall (f :: * -> *) a.
Applicative f =>
Maybe a -> (a -> f ()) -> f ()
whenJust Maybe (Header blk, TentativeHeaderState blk)
mTentativeHeader (Header blk, TentativeHeaderState blk) -> m ()
clearTentativeHeader
            -- Prefix of the candidate because it contained rejected blocks
            -- (invalid blocks). Note that the
            -- spec says go back to candidate selection,
            -- because there might still be some candidates that contain the
            -- same rejected block. To simplify the control flow, we do it
            -- differently: instead of recomputing the candidates taking
            -- rejected blocks into account, we just truncate the remaining
            -- candidates that contain rejected blocks.
            candidates1 <- [ChainDiff (Header blk)] -> m [ChainDiff (Header blk)]
truncateRejectedBlocks [ChainDiff (Header blk)]
candidates0
            -- Only include the prefix if it is still preferred over the current
            -- chain. When the candidate is now empty because of the truncation,
            -- it will be dropped here, as it will not be preferred over the
            -- current chain.
            let candidates2
                  | BlockConfig blk
-> AnchoredFragment (Header blk)
-> AnchoredFragment (Header blk)
-> Bool
forall blk (h :: * -> *) (h' :: * -> *).
(BlockSupportsProtocol blk, HasCallStack, GetHeader1 h,
 GetHeader1 h', HeaderHash (h blk) ~ HeaderHash (h' blk),
 HasHeader (h blk), HasHeader (h' blk)) =>
BlockConfig blk
-> AnchoredFragment (h blk) -> AnchoredFragment (h' blk) -> Bool
preferAnchoredCandidate BlockConfig blk
bcfg AnchoredFragment (Header blk)
curChain (ChainDiff (Header blk) -> AnchoredFragment (Header blk)
forall b. ChainDiff b -> AnchoredFragment b
Diff.getSuffix ChainDiff (Header blk)
candidate')
                  = ChainDiff (Header blk)
candidate'ChainDiff (Header blk)
-> [ChainDiff (Header blk)] -> [ChainDiff (Header blk)]
forall a. a -> [a] -> [a]
:[ChainDiff (Header blk)]
candidates1
                  | Bool
otherwise
                  = [ChainDiff (Header blk)]
candidates1
            go (sortCandidates candidates2)
      where
        -- | Set and return the tentative header, if applicable. Also return the
        -- new 'TentativeHeaderState' in case the corresponding block body turns
        -- out to be invalid.
        setTentativeHeader :: m (Maybe (Header blk, TentativeHeaderState blk))
        setTentativeHeader :: m (Maybe (Header blk, TentativeHeaderState blk))
setTentativeHeader = do
            pipeliningResult <-
                  (\TentativeHeaderState blk
ts -> BlockConfig blk
-> TentativeHeaderState blk
-> ChainDiff (Header blk)
-> Maybe (Header blk, TentativeHeaderState blk)
forall blk.
(HasHeader (Header blk), BlockSupportsDiffusionPipelining blk) =>
BlockConfig blk
-> TentativeHeaderState blk
-> ChainDiff (Header blk)
-> Maybe (Header blk, TentativeHeaderState blk)
isPipelineable BlockConfig blk
bcfg TentativeHeaderState blk
ts ChainDiff (Header blk)
candidate)
              (TentativeHeaderState blk
 -> Maybe (Header blk, TentativeHeaderState blk))
-> m (TentativeHeaderState blk)
-> m (Maybe (Header blk, TentativeHeaderState blk))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> StrictTVar m (TentativeHeaderState blk)
-> m (TentativeHeaderState blk)
forall (m :: * -> *) a. MonadSTM m => StrictTVar m a -> m a
readTVarIO StrictTVar m (TentativeHeaderState blk)
varTentativeState
            whenJust pipeliningResult $ \(Header blk
tentativeHeader, TentativeHeaderState blk
_) -> do
              let setTentative :: Enclosing -> TracePipeliningEvent blk
setTentative = Header blk -> Enclosing -> TracePipeliningEvent blk
forall blk. Header blk -> Enclosing -> TracePipeliningEvent blk
SetTentativeHeader Header blk
tentativeHeader
              Tracer m Enclosing -> m () -> m ()
forall (m :: * -> *) a.
Applicative m =>
Tracer m Enclosing -> m a -> m a
encloseWith (Enclosing -> TracePipeliningEvent blk
setTentative (Enclosing -> TracePipeliningEvent blk)
-> Tracer m (TracePipeliningEvent blk) -> Tracer m Enclosing
forall (f :: * -> *) a b. Contravariant f => (a -> b) -> f b -> f a
>$< Tracer m (TracePipeliningEvent blk)
pipeliningTracer) (m () -> m ()) -> m () -> m ()
forall a b. (a -> b) -> a -> b
$
                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 (StrictMaybe (Header blk))
-> StrictMaybe (Header blk) -> STM m ()
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
StrictTVar m a -> a -> STM m ()
writeTVar StrictTVar m (StrictMaybe (Header blk))
varTentativeHeader (StrictMaybe (Header blk) -> STM m ())
-> StrictMaybe (Header blk) -> STM m ()
forall a b. (a -> b) -> a -> b
$ Header blk -> StrictMaybe (Header blk)
forall a. a -> StrictMaybe a
SJust Header blk
tentativeHeader
                -- As we are only extending the existing chain, the intersection
                -- point is not receding, in which case fhSwitchFork is not
                -- necessary.

              -- Just in case, explicitly yield to ensure that a capability (by
              -- default, the node uses just two) has the opportunity to switch
              -- to a ChainSync server thread.
              m ()
forall (m :: * -> *). MonadFork m => m ()
yield
            pure pipeliningResult

        -- | Clear a tentative header that turned out to be invalid. Also, roll
        -- back the tentative followers.
        clearTentativeHeader :: (Header blk, TentativeHeaderState blk) -> m ()
        clearTentativeHeader :: (Header blk, TentativeHeaderState blk) -> m ()
clearTentativeHeader (Header blk
tentativeHeader, TentativeHeaderState blk
tentativeSt) = do
            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
$ do
              StrictTVar m (StrictMaybe (Header blk))
-> StrictMaybe (Header blk) -> STM m ()
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
StrictTVar m a -> a -> STM m ()
writeTVar StrictTVar m (StrictMaybe (Header blk))
varTentativeHeader StrictMaybe (Header blk)
forall a. StrictMaybe a
SNothing
              StrictTVar m (TentativeHeaderState blk)
-> TentativeHeaderState blk -> STM m ()
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
StrictTVar m a -> a -> STM m ()
writeTVar StrictTVar m (TentativeHeaderState blk)
varTentativeState  TentativeHeaderState blk
tentativeSt
              (FollowerHandle m blk -> STM m ()) -> STM m ()
forTentativeFollowers ((FollowerHandle m blk -> STM m ()) -> STM m ())
-> (FollowerHandle m blk -> STM m ()) -> STM m ()
forall a b. (a -> b) -> a -> b
$ \FollowerHandle m blk
followerHandle -> do
                let curTipPoint :: Point blk
curTipPoint = Point (Header blk) -> Point blk
forall {k1} {k2} (b :: k1) (b' :: k2).
Coercible (HeaderHash b) (HeaderHash b') =>
Point b -> Point b'
castPoint (Point (Header blk) -> Point blk)
-> Point (Header blk) -> Point blk
forall a b. (a -> b) -> a -> b
$ AnchoredFragment (Header blk) -> Point (Header blk)
forall block.
HasHeader block =>
AnchoredFragment block -> Point block
AF.headPoint AnchoredFragment (Header blk)
curChain
                    oldPoints :: Set (Point blk)
oldPoints = Point blk -> Set (Point blk)
forall a. a -> Set a
Set.singleton (Point blk -> Set (Point blk)) -> Point blk -> Set (Point blk)
forall a b. (a -> b) -> a -> b
$ Header blk -> Point blk
forall blk. HasHeader (Header blk) => Header blk -> Point blk
headerPoint Header blk
tentativeHeader
                FollowerHandle m blk -> Point blk -> Set (Point blk) -> STM m ()
forall (m :: * -> *) blk.
FollowerHandle m blk -> Point blk -> Set (Point blk) -> STM m ()
fhSwitchFork FollowerHandle m blk
followerHandle Point blk
curTipPoint Set (Point blk)
oldPoints
            Tracer m (TracePipeliningEvent blk)
-> TracePipeliningEvent blk -> m ()
forall (m :: * -> *) a. Tracer m a -> a -> m ()
traceWith Tracer m (TracePipeliningEvent blk)
pipeliningTracer (TracePipeliningEvent blk -> m ())
-> TracePipeliningEvent blk -> m ()
forall a b. (a -> b) -> a -> b
$ Header blk -> TracePipeliningEvent blk
forall blk. Header blk -> TracePipeliningEvent blk
TrapTentativeHeader Header blk
tentativeHeader
          where
            forTentativeFollowers :: (FollowerHandle m blk -> STM m ()) -> STM m ()
forTentativeFollowers FollowerHandle m blk -> STM m ()
f = STM m [FollowerHandle m blk]
getTentativeFollowers STM m [FollowerHandle m blk]
-> ([FollowerHandle m blk] -> STM m ()) -> STM m ()
forall a b. STM m a -> (a -> STM m b) -> STM m b
forall (m :: * -> *) a b. Monad m => m a -> (a -> m b) -> m b
>>= (FollowerHandle m blk -> STM m ())
-> [FollowerHandle m blk] -> STM m ()
forall (t :: * -> *) (m :: * -> *) a b.
(Foldable t, Monad m) =>
(a -> m b) -> t a -> m ()
mapM_ FollowerHandle m blk -> STM m ()
f

    -- | Truncate the given (remaining) candidates that contain rejected
    -- blocks. Discard them if they are truncated so much that they are no
    -- longer preferred over the current chain.
    --
    -- A block is rejected if it is invalid (present in 'varInvalid',
    -- i.e., 'cdbInvalid').
    truncateRejectedBlocks ::
         [ChainDiff (Header blk)]
      -> m [ChainDiff (Header blk)]
    truncateRejectedBlocks :: [ChainDiff (Header blk)] -> m [ChainDiff (Header blk)]
truncateRejectedBlocks [ChainDiff (Header blk)]
cands = do
      invalid <- STM m (WithFingerprint (InvalidBlocks blk))
-> m (WithFingerprint (InvalidBlocks blk))
forall a. HasCallStack => STM m a -> m a
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
STM m a -> m a
atomically (STM m (WithFingerprint (InvalidBlocks blk))
 -> m (WithFingerprint (InvalidBlocks blk)))
-> STM m (WithFingerprint (InvalidBlocks blk))
-> m (WithFingerprint (InvalidBlocks blk))
forall a b. (a -> b) -> a -> b
$ StrictTVar m (WithFingerprint (InvalidBlocks blk))
-> STM m (WithFingerprint (InvalidBlocks blk))
forall (m :: * -> *) a. MonadSTM m => StrictTVar m a -> STM m a
readTVar StrictTVar m (WithFingerprint (InvalidBlocks blk))
varInvalid
      let isRejected Header blk
hdr =
               HeaderHash blk -> InvalidBlocks blk -> Bool
forall k a. Ord k => k -> Map k a -> Bool
Map.member (Header blk -> HeaderHash blk
forall blk. HasHeader (Header blk) => Header blk -> HeaderHash blk
headerHash Header blk
hdr) (WithFingerprint (InvalidBlocks blk) -> InvalidBlocks blk
forall a. WithFingerprint a -> a
forgetFingerprint WithFingerprint (InvalidBlocks blk)
invalid)
      return $ filter (preferAnchoredCandidate bcfg curChain . Diff.getSuffix)
             $ map (Diff.takeWhileOldest (not . isRejected)) cands

    -- [Ouroboros]
    --
    -- Ouroboros says that when we are given an invalid chain by a peer, we
    -- should reject that peer's chain. However, since we're throwing all
    -- blocks together in the ChainDB, we can't tell which block or which
    -- chain came from which peer, so we can't simply reject a peer's chain.
    --
    -- It might be that a good peer gave us a valid chain, but another peer
    -- gave us an invalid block that fits onto the chain of the good peer. In
    -- that case, we do still want to adopt the chain of the good peer, which
    -- is a prefix of the chain that we constructed using all the blocks we
    -- found in the VolatileDB, including the invalid block.
    --
    -- This is the reason why we still take valid prefixes of a invalid chains
    -- into account during chain selection: they might correspond to the good
    -- peer's valid chain.

-- | Result of 'validateCandidate'.
data ValidationResult m blk =
      -- | The entire candidate fragment was valid.
      FullyValid (ValidatedChainDiff (Header blk) (Forker' m blk))

      -- | The candidate fragment contained invalid blocks that had to
      -- be truncated from the fragment.
    | ValidPrefix (ChainDiff (Header blk))

      -- | After truncating the invalid blocks from
      -- the 'ChainDiff', it no longer contains enough blocks in its suffix to
      -- compensate for the number of blocks it wants to roll back.
    | InsufficientSuffix

-- | Validate a candidate by applying its blocks to the ledger, and return a
-- 'ValidatedChainDiff' for it, i.e., a chain diff along with a ledger
-- corresponding to its tip (the most recent block).
--
-- PRECONDITION: the candidate (chain diff) must fit onto the given current
-- chain.
--
-- If all blocks in the fragment are valid, then the chain diff in the
-- returned 'ValidatedChainDiff' is the same as the given candidate chain
-- diff.
--
-- If a block in the fragment is invalid, then the fragment in the returned
-- 'ValidatedChainDiff' is a prefix of the given candidate chain diff (upto
-- the last valid block).
--
-- Note that this function returns a 'Forker', and that this forker should be
-- closed when it is no longer used!
ledgerValidateCandidate ::
     forall m blk.
     ( IOLike m
     , LedgerSupportsProtocol blk
     , HasCallStack
     )
  => ChainSelEnv m blk
  -> ResourceRegistry m
  -> ChainDiff (Header blk)
  -> m (ValidatedChainDiff (Header blk) (Forker' m blk))
ledgerValidateCandidate :: forall (m :: * -> *) blk.
(IOLike m, LedgerSupportsProtocol blk, HasCallStack) =>
ChainSelEnv m blk
-> ResourceRegistry m
-> ChainDiff (Header blk)
-> m (ValidatedChainDiff (Header blk) (Forker' m blk))
ledgerValidateCandidate ChainSelEnv m blk
chainSelEnv ResourceRegistry m
rr chainDiff :: ChainDiff (Header blk)
chainDiff@(ChainDiff Word64
rollback AnchoredFragment (Header blk)
suffix) =
    LedgerDB m (ExtLedgerState blk) blk
-> (ExtLedgerState blk ~ ExtLedgerState blk) =>
   ResourceRegistry m
   -> (TraceValidateEvent blk -> m ())
   -> BlockCache blk
   -> Word64
   -> [Header blk]
   -> m (ValidateResult m (ExtLedgerState blk) blk)
forall (m :: * -> *) (l :: LedgerStateKind) blk.
LedgerDB m l blk
-> (l ~ ExtLedgerState blk) =>
   ResourceRegistry m
   -> (TraceValidateEvent blk -> m ())
   -> BlockCache blk
   -> Word64
   -> [Header blk]
   -> m (ValidateResult m l blk)
LedgerDB.validateFork LedgerDB m (ExtLedgerState blk) blk
lgrDB ResourceRegistry m
rr TraceValidateEvent blk -> m ()
traceUpdate BlockCache blk
blockCache Word64
rollback [Header blk]
newBlocks m (ValidateResult m (ExtLedgerState blk) blk)
-> (ValidateResult m (ExtLedgerState blk) blk
    -> m (ValidatedChainDiff (Header blk) (Forker' m blk)))
-> m (ValidatedChainDiff (Header blk) (Forker' m blk))
forall a b. m a -> (a -> m b) -> m b
forall (m :: * -> *) a b. Monad m => m a -> (a -> m b) -> m b
>>= \case
      ValidateExceededRollBack {} ->
        -- Impossible: we asked the LedgerDB to roll back past the immutable
        -- tip, which is impossible, since the candidates we construct must
        -- connect to the immutable tip.
        [Char] -> m (ValidatedChainDiff (Header blk) (Forker' m blk))
forall a. HasCallStack => [Char] -> a
error [Char]
"found candidate requiring rolling back past the immutable tip"

      ValidateLedgerError (AnnLedgerError Forker' m blk
ledger' RealPoint blk
pt LedgerErr (ExtLedgerState blk)
e) -> do
        lastValid <- STM m (Point blk) -> m (Point blk)
forall a. HasCallStack => STM m a -> m a
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
STM m a -> m a
atomically (STM m (Point blk) -> m (Point blk))
-> STM m (Point blk) -> m (Point blk)
forall a b. (a -> b) -> a -> b
$ Forker' m blk -> STM m (Point blk)
forall (l :: LedgerStateKind) blk (m :: * -> *).
(GetTip l, HeaderHash l ~ HeaderHash blk, Functor (STM m)) =>
Forker m l blk -> STM m (Point blk)
forkerCurrentPoint Forker' m blk
ledger'
        let chainDiff' = Point (Header blk)
-> ChainDiff (Header blk) -> ChainDiff (Header blk)
forall b.
(HasHeader b, HasCallStack) =>
Point b -> ChainDiff b -> ChainDiff b
Diff.truncate (Point blk -> Point (Header blk)
forall {k1} {k2} (b :: k1) (b' :: k2).
Coercible (HeaderHash b) (HeaderHash b') =>
Point b -> Point b'
castPoint Point blk
lastValid) ChainDiff (Header blk)
chainDiff
        traceWith validationTracer (InvalidBlock e pt)
        addInvalidBlock e pt
        traceWith validationTracer (ValidCandidate (Diff.getSuffix chainDiff'))

        -- punish the peer who sent a block if it is invalid or a block from its
        -- prefix is invalid
        --
        -- Note that it is a chain selection invariant that all candidates
        -- involve the block being processed: see Lemma 11.1 (Properties of the
        -- set of candidates) in the Chain Selection chapter of the The Cardano
        -- Consensus and Storage Layer technical report.
        whenJust punish $ \(RealPoint blk
addedPt, InvalidBlockPunishment m
punishment) -> do
          let m :: m ()
m = InvalidBlockPunishment m -> Invalidity -> m ()
forall (m :: * -> *).
InvalidBlockPunishment m -> Invalidity -> m ()
InvalidBlockPunishment.enact InvalidBlockPunishment m
punishment
                (Invalidity -> m ()) -> Invalidity -> m ()
forall a b. (a -> b) -> a -> b
$ if RealPoint blk
addedPt RealPoint blk -> RealPoint blk -> Bool
forall a. Eq a => a -> a -> Bool
== RealPoint blk
pt
                  then Invalidity
InvalidBlockPunishment.BlockItself
                  else Invalidity
InvalidBlockPunishment.BlockPrefix
          case RealPoint blk -> SlotNo
forall blk. RealPoint blk -> SlotNo
realPointSlot RealPoint blk
pt SlotNo -> SlotNo -> Ordering
forall a. Ord a => a -> a -> Ordering
`compare` RealPoint blk -> SlotNo
forall blk. RealPoint blk -> SlotNo
realPointSlot RealPoint blk
addedPt of
            Ordering
LT -> m ()
m
            Ordering
GT -> () -> m ()
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure ()
            Ordering
EQ -> Bool -> m () -> m ()
forall (f :: * -> *). Applicative f => Bool -> f () -> f ()
when (Point blk
lastValid Point blk -> Point blk -> Bool
forall a. Eq a => a -> a -> Bool
/= RealPoint blk -> Point blk
forall blk. RealPoint blk -> Point blk
realPointToPoint RealPoint blk
addedPt) m ()
m
              -- If pt and addedPt have the same slot, and addedPt is the tip of
              -- the ledger that pt was validated against, then addedPt is an
              -- EBB and is valid.
              --
              -- Otherwise, either pt == addedPt or addedPt comes after pt, so
              -- we should punish. (Tacit assumption made here: it's impossible
              -- three blocks in a row have the same slot.)

        ValidatedDiff.newM chainDiff' ledger'

      ValidateSuccessful Forker' m blk
ledger' -> do
        Tracer m (TraceValidationEvent blk)
-> TraceValidationEvent blk -> m ()
forall (m :: * -> *) a. Tracer m a -> a -> m ()
traceWith Tracer m (TraceValidationEvent blk)
validationTracer (AnchoredFragment (Header blk) -> TraceValidationEvent blk
forall blk.
AnchoredFragment (Header blk) -> TraceValidationEvent blk
ValidCandidate AnchoredFragment (Header blk)
suffix)
        ChainDiff (Header blk)
-> Forker' m blk
-> m (ValidatedChainDiff (Header blk) (Forker' m blk))
forall (m :: * -> *) b l.
(MonadSTM m, GetTipSTM m l, HasHeader b,
 HeaderHash l ~ HeaderHash b, HasCallStack) =>
ChainDiff b -> l -> m (ValidatedChainDiff b l)
ValidatedDiff.newM ChainDiff (Header blk)
chainDiff Forker' m blk
ledger'
  where
    ChainSelEnv {
        LedgerDB m (ExtLedgerState blk) blk
lgrDB :: forall (m :: * -> *) blk. ChainSelEnv m blk -> LedgerDB' m blk
lgrDB :: LedgerDB m (ExtLedgerState blk) blk
lgrDB
      , Tracer m (TraceValidationEvent blk)
validationTracer :: forall (m :: * -> *) blk.
ChainSelEnv m blk -> Tracer m (TraceValidationEvent blk)
validationTracer :: Tracer m (TraceValidationEvent blk)
validationTracer
      , BlockCache blk
blockCache :: forall (m :: * -> *) blk. ChainSelEnv m blk -> BlockCache blk
blockCache :: BlockCache blk
blockCache
      , StrictTVar m (WithFingerprint (InvalidBlocks blk))
varInvalid :: forall (m :: * -> *) blk.
ChainSelEnv m blk
-> StrictTVar m (WithFingerprint (InvalidBlocks blk))
varInvalid :: StrictTVar m (WithFingerprint (InvalidBlocks blk))
varInvalid
      , Maybe (RealPoint blk, InvalidBlockPunishment m)
punish :: forall (m :: * -> *) blk.
ChainSelEnv m blk
-> Maybe (RealPoint blk, InvalidBlockPunishment m)
punish :: Maybe (RealPoint blk, InvalidBlockPunishment m)
punish
      } = ChainSelEnv m blk
chainSelEnv

    traceUpdate :: TraceValidateEvent blk -> m ()
traceUpdate = Tracer m (TraceValidateEvent blk) -> TraceValidateEvent blk -> m ()
forall (m :: * -> *) a. Tracer m a -> a -> m ()
traceWith (Tracer m (TraceValidateEvent blk)
 -> TraceValidateEvent blk -> m ())
-> Tracer m (TraceValidateEvent blk)
-> TraceValidateEvent blk
-> m ()
forall a b. (a -> b) -> a -> b
$ TraceValidateEvent blk -> TraceValidationEvent blk
forall blk. TraceValidateEvent blk -> TraceValidationEvent blk
UpdateLedgerDbTraceEvent (TraceValidateEvent blk -> TraceValidationEvent blk)
-> Tracer m (TraceValidationEvent blk)
-> Tracer m (TraceValidateEvent blk)
forall (f :: * -> *) a b. Contravariant f => (a -> b) -> f b -> f a
>$< Tracer m (TraceValidationEvent blk)
validationTracer

    newBlocks :: [Header blk]
    newBlocks :: [Header blk]
newBlocks = AnchoredFragment (Header blk) -> [Header blk]
forall v a b. AnchoredSeq v a b -> [b]
AF.toOldestFirst AnchoredFragment (Header blk)
suffix

    -- | Record the invalid block in 'cdbInvalid' and change its fingerprint.
    addInvalidBlock :: ExtValidationError blk -> RealPoint blk -> m ()
    addInvalidBlock :: ExtValidationError blk -> RealPoint blk -> m ()
addInvalidBlock ExtValidationError blk
e (RealPoint SlotNo
slot HeaderHash blk
hash) = 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 (WithFingerprint (InvalidBlocks blk))
-> (WithFingerprint (InvalidBlocks blk)
    -> WithFingerprint (InvalidBlocks blk))
-> STM m ()
forall (m :: * -> *) a.
MonadSTM m =>
StrictTVar m a -> (a -> a) -> STM m ()
modifyTVar StrictTVar m (WithFingerprint (InvalidBlocks blk))
varInvalid ((WithFingerprint (InvalidBlocks blk)
  -> WithFingerprint (InvalidBlocks blk))
 -> STM m ())
-> (WithFingerprint (InvalidBlocks blk)
    -> WithFingerprint (InvalidBlocks blk))
-> STM m ()
forall a b. (a -> b) -> a -> b
$ \(WithFingerprint InvalidBlocks blk
invalid Fingerprint
fp) ->
        InvalidBlocks blk
-> Fingerprint -> WithFingerprint (InvalidBlocks blk)
forall a. a -> Fingerprint -> WithFingerprint a
WithFingerprint
          (HeaderHash blk
-> InvalidBlockInfo blk -> InvalidBlocks blk -> InvalidBlocks blk
forall k a. Ord k => k -> a -> Map k a -> Map k a
Map.insert HeaderHash blk
hash (ExtValidationError blk -> SlotNo -> InvalidBlockInfo blk
forall blk.
ExtValidationError blk -> SlotNo -> InvalidBlockInfo blk
InvalidBlockInfo ExtValidationError blk
e SlotNo
slot) InvalidBlocks blk
invalid)
          (Fingerprint -> Fingerprint
forall a. Enum a => a -> a
succ Fingerprint
fp)

-- | Validate a candidate chain using 'ledgerValidateCandidate'.
validateCandidate ::
     ( IOLike m
     , LedgerSupportsProtocol blk
     , HasCallStack
     )
  => ChainSelEnv m blk
  -> ResourceRegistry m
  -> ChainDiff (Header blk)
  -> m (ValidationResult m blk)
validateCandidate :: forall (m :: * -> *) blk.
(IOLike m, LedgerSupportsProtocol blk, HasCallStack) =>
ChainSelEnv m blk
-> ResourceRegistry m
-> ChainDiff (Header blk)
-> m (ValidationResult m blk)
validateCandidate ChainSelEnv m blk
chainSelEnv ResourceRegistry m
rr ChainDiff (Header blk)
chainDiff =
    ChainSelEnv m blk
-> ResourceRegistry m
-> ChainDiff (Header blk)
-> m (ValidatedChainDiff (Header blk) (Forker' m blk))
forall (m :: * -> *) blk.
(IOLike m, LedgerSupportsProtocol blk, HasCallStack) =>
ChainSelEnv m blk
-> ResourceRegistry m
-> ChainDiff (Header blk)
-> m (ValidatedChainDiff (Header blk) (Forker' m blk))
ledgerValidateCandidate ChainSelEnv m blk
chainSelEnv ResourceRegistry m
rr ChainDiff (Header blk)
chainDiff m (ValidatedChainDiff (Header blk) (Forker' m blk))
-> (ValidatedChainDiff (Header blk) (Forker' m blk)
    -> m (ValidationResult m blk))
-> m (ValidationResult m blk)
forall a b. m a -> (a -> m b) -> m b
forall (m :: * -> *) a b. Monad m => m a -> (a -> m b) -> m b
>>= \case
      ValidatedChainDiff (Header blk) (Forker' m blk)
validatedChainDiff
        | ValidatedChainDiff (Header blk) (Forker' m blk) -> Bool
forall b l. HasHeader b => ValidatedChainDiff b l -> Bool
ValidatedDiff.rollbackExceedsSuffix ValidatedChainDiff (Header blk) (Forker' m blk)
validatedChainDiff
        -> ValidatedChainDiff (Header blk) (Forker' m blk) -> m ()
forall b (m :: * -> *) blk.
ValidatedChainDiff b (Forker' m blk) -> m ()
cleanup ValidatedChainDiff (Header blk) (Forker' m blk)
validatedChainDiff m () -> m (ValidationResult m blk) -> m (ValidationResult m blk)
forall a b. m a -> m b -> m b
forall (m :: * -> *) a b. Monad m => m a -> m b -> m b
>> ValidationResult m blk -> m (ValidationResult m blk)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return ValidationResult m blk
forall (m :: * -> *) blk. ValidationResult m blk
InsufficientSuffix

        | AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> Int
forall v a b. Anchorable v a b => AnchoredSeq v a b -> Int
AF.length (ChainDiff (Header blk)
-> AnchoredSeq
     (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
forall b. ChainDiff b -> AnchoredFragment b
Diff.getSuffix ChainDiff (Header blk)
chainDiff) Int -> Int -> Bool
forall a. Eq a => a -> a -> Bool
== AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> Int
forall v a b. Anchorable v a b => AnchoredSeq v a b -> Int
AF.length (ChainDiff (Header blk)
-> AnchoredSeq
     (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
forall b. ChainDiff b -> AnchoredFragment b
Diff.getSuffix ChainDiff (Header blk)
chainDiff')
        -- No truncation
        -> ValidationResult m blk -> m (ValidationResult m blk)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (ValidationResult m blk -> m (ValidationResult m blk))
-> ValidationResult m blk -> m (ValidationResult m blk)
forall a b. (a -> b) -> a -> b
$ ValidatedChainDiff (Header blk) (Forker' m blk)
-> ValidationResult m blk
forall (m :: * -> *) blk.
ValidatedChainDiff (Header blk) (Forker' m blk)
-> ValidationResult m blk
FullyValid ValidatedChainDiff (Header blk) (Forker' m blk)
validatedChainDiff

        | Bool
otherwise
        -- In case of invalid blocks, we throw away the ledger
        -- corresponding to the truncated fragment and will have to
        -- validate it again, even when it's the sole candidate.
        -> ValidationResult m blk -> m (ValidationResult m blk)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (ValidationResult m blk -> m (ValidationResult m blk))
-> ValidationResult m blk -> m (ValidationResult m blk)
forall a b. (a -> b) -> a -> b
$ ChainDiff (Header blk) -> ValidationResult m blk
forall (m :: * -> *) blk.
ChainDiff (Header blk) -> ValidationResult m blk
ValidPrefix ChainDiff (Header blk)
chainDiff'

        where
          chainDiff' :: ChainDiff (Header blk)
chainDiff' = ValidatedChainDiff (Header blk) (Forker' m blk)
-> ChainDiff (Header blk)
forall b l. ValidatedChainDiff b l -> ChainDiff b
ValidatedDiff.getChainDiff ValidatedChainDiff (Header blk) (Forker' m blk)
validatedChainDiff
  where
    -- If this function does not return a validated chain diff, then there is a
    -- leftover forker that we have to close so that its resources are correctly
    -- released.
    cleanup :: ValidatedChainDiff b (Forker' m blk) -> m ()
    cleanup :: forall b (m :: * -> *) blk.
ValidatedChainDiff b (Forker' m blk) -> m ()
cleanup = Forker m (ExtLedgerState blk) blk -> m ()
forall (m :: * -> *) (l :: LedgerStateKind) blk.
Forker m l blk -> m ()
forkerClose (Forker m (ExtLedgerState blk) blk -> m ())
-> (ValidatedChainDiff b (Forker m (ExtLedgerState blk) blk)
    -> Forker m (ExtLedgerState blk) blk)
-> ValidatedChainDiff b (Forker m (ExtLedgerState blk) blk)
-> m ()
forall b c a. (b -> c) -> (a -> b) -> a -> c
. ValidatedChainDiff b (Forker m (ExtLedgerState blk) blk)
-> Forker m (ExtLedgerState blk) blk
forall b l. ValidatedChainDiff b l -> l
getLedger

{-------------------------------------------------------------------------------
  'ChainAndLedger'
-------------------------------------------------------------------------------}

-- | Instantiate 'ValidatedFragment' in the way that chain selection requires.
type ChainAndLedger m blk = ValidatedFragment (Header blk) (Forker' m blk)

{-------------------------------------------------------------------------------
  Diffusion pipelining
-------------------------------------------------------------------------------}

-- | Check whether a 'ChainDiff' can be pipelined. If it can, the tentative
-- header as well as the new 'TentativeHeaderState' (to be used in case the
-- block body turns out to be invalid) is returned.
--
-- PRECONDITION: The 'ChainDiff' fits on top of the current chain and is better.
isPipelineable ::
     (HasHeader (Header blk), BlockSupportsDiffusionPipelining blk)
  => BlockConfig blk
  -> TentativeHeaderState blk
  -> ChainDiff (Header blk)
  -> Maybe (Header blk, TentativeHeaderState blk)
isPipelineable :: forall blk.
(HasHeader (Header blk), BlockSupportsDiffusionPipelining blk) =>
BlockConfig blk
-> TentativeHeaderState blk
-> ChainDiff (Header blk)
-> Maybe (Header blk, TentativeHeaderState blk)
isPipelineable BlockConfig blk
bcfg TentativeHeaderState blk
st ChainDiff {Word64
AnchoredFragment (Header blk)
getSuffix :: forall b. ChainDiff b -> AnchoredFragment b
getRollback :: Word64
getSuffix :: AnchoredFragment (Header blk)
getRollback :: forall b. ChainDiff b -> Word64
..}
  | -- we apply exactly one header
    AF.Empty Anchor (Header blk)
_ :> Header blk
hdr <- AnchoredFragment (Header blk)
getSuffix
  , Just TentativeHeaderState blk
st' <- BlockConfig blk
-> Header blk
-> TentativeHeaderState blk
-> Maybe (TentativeHeaderState blk)
forall blk.
BlockSupportsDiffusionPipelining blk =>
BlockConfig blk
-> Header blk
-> TentativeHeaderState blk
-> Maybe (TentativeHeaderState blk)
updateTentativeHeaderState BlockConfig blk
bcfg Header blk
hdr TentativeHeaderState blk
st
    -- ensure that the diff is applied to the chain tip
  , Word64
getRollback Word64 -> Word64 -> Bool
forall a. Eq a => a -> a -> Bool
== Word64
0
  = (Header blk, TentativeHeaderState blk)
-> Maybe (Header blk, TentativeHeaderState blk)
forall a. a -> Maybe a
Just (Header blk
hdr, TentativeHeaderState blk
st')
  | Bool
otherwise = Maybe (Header blk, TentativeHeaderState blk)
forall a. Maybe a
Nothing

{-------------------------------------------------------------------------------
  Helpers
-------------------------------------------------------------------------------}

-- | Wrap a @getter@ function so that it returns 'Nothing' for invalid blocks.
ignoreInvalid ::
     HasHeader blk
  => proxy blk
  -> InvalidBlocks blk
  -> (HeaderHash blk -> Maybe a)
  -> (HeaderHash blk -> Maybe a)
ignoreInvalid :: forall blk (proxy :: * -> *) a.
HasHeader blk =>
proxy blk
-> InvalidBlocks blk
-> (HeaderHash blk -> Maybe a)
-> HeaderHash blk
-> Maybe a
ignoreInvalid proxy blk
_ InvalidBlocks blk
invalid HeaderHash blk -> Maybe a
getter HeaderHash blk
hash
    | HeaderHash blk -> InvalidBlocks blk -> Bool
forall k a. Ord k => k -> Map k a -> Bool
Map.member HeaderHash blk
hash InvalidBlocks blk
invalid = Maybe a
forall a. Maybe a
Nothing
    | Bool
otherwise               = HeaderHash blk -> Maybe a
getter HeaderHash blk
hash

-- | Wrap a @successors@ function so that invalid blocks are not returned as
-- successors.
ignoreInvalidSuc ::
     HasHeader blk
  => proxy blk
  -> InvalidBlocks blk
  -> (ChainHash blk -> Set (HeaderHash blk))
  -> (ChainHash blk -> Set (HeaderHash blk))
ignoreInvalidSuc :: forall blk (proxy :: * -> *).
HasHeader blk =>
proxy blk
-> InvalidBlocks blk
-> (ChainHash blk -> Set (HeaderHash blk))
-> ChainHash blk
-> Set (HeaderHash blk)
ignoreInvalidSuc proxy blk
_ InvalidBlocks blk
invalid ChainHash blk -> Set (HeaderHash blk)
succsOf =
    (HeaderHash blk -> Bool)
-> Set (HeaderHash blk) -> Set (HeaderHash blk)
forall a. (a -> Bool) -> Set a -> Set a
Set.filter (HeaderHash blk -> InvalidBlocks blk -> Bool
forall k a. Ord k => k -> Map k a -> Bool
`Map.notMember` InvalidBlocks blk
invalid) (Set (HeaderHash blk) -> Set (HeaderHash blk))
-> (ChainHash blk -> Set (HeaderHash blk))
-> ChainHash blk
-> Set (HeaderHash blk)
forall b c a. (b -> c) -> (a -> b) -> a -> c
. ChainHash blk -> Set (HeaderHash blk)
succsOf