{-# LANGUAGE FlexibleContexts #-}
{-# LANGUAGE GADTs #-}
{-# LANGUAGE LambdaCase #-}
{-# LANGUAGE MultiWayIf #-}
{-# LANGUAGE NamedFieldPuns #-}
{-# LANGUAGE RecordWildCards #-}
{-# LANGUAGE ScopedTypeVariables #-}
{-# LANGUAGE TypeApplications #-}
{-# 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           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.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.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.LgrDB (LedgerDB',
                     LgrDB)
import qualified Ouroboros.Consensus.Storage.ChainDB.Impl.LgrDB as LgrDB
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.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

-- | 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
  -> LgrDB m blk
  -> Tracer m (TraceInitChainSelEvent blk)
  -> TopLevelConfig blk
  -> StrictTVar m (WithFingerprint (InvalidBlocks blk))
  -> LoE ()
  -> m (ChainAndLedger blk)
initialChainSelection :: forall (m :: * -> *) blk.
(IOLike m, LedgerSupportsProtocol blk,
 BlockSupportsDiffusionPipelining blk) =>
ImmutableDB m blk
-> VolatileDB m blk
-> LgrDB m blk
-> Tracer m (TraceInitChainSelEvent blk)
-> TopLevelConfig blk
-> StrictTVar m (WithFingerprint (InvalidBlocks blk))
-> LoE ()
-> m (ChainAndLedger blk)
initialChainSelection ImmutableDB m blk
immutableDB VolatileDB m blk
volatileDB LgrDB m blk
lgrDB 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.
    (Anchor blk
i :: Anchor blk, ChainHash blk -> Set (HeaderHash blk)
succsOf, LedgerDB' blk
ledger) <- STM
  m
  (Anchor blk, ChainHash blk -> Set (HeaderHash blk), LedgerDB' blk)
-> m (Anchor blk, ChainHash blk -> Set (HeaderHash blk),
      LedgerDB' 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), LedgerDB' blk)
 -> m (Anchor blk, ChainHash blk -> Set (HeaderHash blk),
       LedgerDB' blk))
-> STM
     m
     (Anchor blk, ChainHash blk -> Set (HeaderHash blk), LedgerDB' blk)
-> m (Anchor blk, ChainHash blk -> Set (HeaderHash blk),
      LedgerDB' blk)
forall a b. (a -> b) -> a -> b
$ do
      InvalidBlocks blk
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
      (,,)
        (Anchor blk
 -> (ChainHash blk -> Set (HeaderHash blk))
 -> LedgerDB' blk
 -> (Anchor blk, ChainHash blk -> Set (HeaderHash blk),
     LedgerDB' blk))
-> STM m (Anchor blk)
-> STM
     m
     ((ChainHash blk -> Set (HeaderHash blk))
      -> LedgerDB' blk
      -> (Anchor blk, ChainHash blk -> Set (HeaderHash blk),
          LedgerDB' blk))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> ImmutableDB m blk -> STM m (Anchor blk)
forall (m :: * -> *) blk.
(MonadSTM m, HasCallStack) =>
ImmutableDB m blk -> STM m (Anchor blk)
ImmutableDB.getTipAnchor ImmutableDB m blk
immutableDB
        STM
  m
  ((ChainHash blk -> Set (HeaderHash blk))
   -> LedgerDB' blk
   -> (Anchor blk, ChainHash blk -> Set (HeaderHash blk),
       LedgerDB' blk))
-> STM m (ChainHash blk -> Set (HeaderHash blk))
-> STM
     m
     (LedgerDB' blk
      -> (Anchor blk, ChainHash blk -> Set (HeaderHash blk),
          LedgerDB' 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
-> 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 VolatileDB m blk
volatileDB InvalidBlocks blk
invalid ((ChainHash blk -> Set (HeaderHash blk))
 -> ChainHash blk -> Set (HeaderHash blk))
-> STM m (ChainHash blk -> Set (HeaderHash blk))
-> STM m (ChainHash blk -> Set (HeaderHash blk))
forall (f :: * -> *) a b. Functor 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
volatileDB)
        STM
  m
  (LedgerDB' blk
   -> (Anchor blk, ChainHash blk -> Set (HeaderHash blk),
       LedgerDB' blk))
-> STM m (LedgerDB' blk)
-> STM
     m
     (Anchor blk, ChainHash blk -> Set (HeaderHash blk), LedgerDB' 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
<*> LgrDB m blk -> STM m (LedgerDB' blk)
forall (m :: * -> *) blk.
IOLike m =>
LgrDB m blk -> STM m (LedgerDB' blk)
LgrDB.getCurrent LgrDB m blk
lgrDB

    [AnchoredFragment (Header blk)]
chains <- Anchor blk
-> (ChainHash blk -> Set (HeaderHash blk))
-> m [AnchoredFragment (Header blk)]
constructChains Anchor blk
i ChainHash blk -> Set (HeaderHash blk)
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 :: AnchoredFragment (Header blk)
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 :: ChainAndLedger blk
curChainAndLedger = AnchoredFragment (Header blk)
-> LedgerDB' blk -> ChainAndLedger blk
forall l b.
(GetTip l, HasHeader b, HeaderHash b ~ HeaderHash l,
 HasCallStack) =>
AnchoredFragment b -> l -> ValidatedFragment b l
VF.ValidatedFragment AnchoredFragment (Header blk)
curChain LedgerDB' blk
ledger

    case [AnchoredFragment (Header blk)]
-> Maybe (NonEmpty (AnchoredFragment (Header blk)))
forall a. [a] -> Maybe (NonEmpty a)
NE.nonEmpty ((AnchoredFragment (Header blk) -> Bool)
-> [AnchoredFragment (Header blk)]
-> [AnchoredFragment (Header blk)]
forall a. (a -> Bool) -> [a] -> [a]
filter (BlockConfig blk
-> AnchoredFragment (Header blk)
-> AnchoredFragment (Header blk)
-> Bool
forall blk.
(BlockSupportsProtocol blk, HasCallStack) =>
BlockConfig blk
-> AnchoredFragment (Header blk)
-> AnchoredFragment (Header blk)
-> Bool
preferAnchoredCandidate BlockConfig blk
bcfg AnchoredFragment (Header blk)
curChain) [AnchoredFragment (Header blk)]
chains) of
      -- If there are no candidates, no chain selection is needed
      Maybe (NonEmpty (AnchoredFragment (Header blk)))
Nothing      -> ChainAndLedger blk -> m (ChainAndLedger blk)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return ChainAndLedger blk
curChainAndLedger
      Just NonEmpty (AnchoredFragment (Header blk))
chains' -> ChainAndLedger blk
-> (ValidatedChainDiff (Header blk) (LedgerDB' blk)
    -> ChainAndLedger blk)
-> Maybe (ValidatedChainDiff (Header blk) (LedgerDB' blk))
-> ChainAndLedger blk
forall b a. b -> (a -> b) -> Maybe a -> b
maybe ChainAndLedger blk
curChainAndLedger ValidatedChainDiff (Header blk) (LedgerDB' blk)
-> ChainAndLedger blk
toChainAndLedger (Maybe (ValidatedChainDiff (Header blk) (LedgerDB' blk))
 -> ChainAndLedger blk)
-> m (Maybe (ValidatedChainDiff (Header blk) (LedgerDB' blk)))
-> m (ChainAndLedger blk)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$>
        HasCallStack =>
ChainAndLedger blk
-> NonEmpty (AnchoredFragment (Header blk))
-> m (Maybe (ValidatedChainDiff (Header blk) (LedgerDB' blk)))
ChainAndLedger blk
-> NonEmpty (AnchoredFragment (Header blk))
-> m (Maybe (ValidatedChainDiff (Header blk) (LedgerDB' blk)))
chainSelection' ChainAndLedger blk
curChainAndLedger NonEmpty (AnchoredFragment (Header blk))
chains'
  where
    bcfg :: BlockConfig blk
    bcfg :: BlockConfig blk
bcfg = TopLevelConfig blk -> BlockConfig blk
forall blk. TopLevelConfig blk -> BlockConfig blk
configBlock TopLevelConfig blk
cfg

    SecurityParam 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) (LedgerDB' blk)
      -> ChainAndLedger blk
    toChainAndLedger :: ValidatedChainDiff (Header blk) (LedgerDB' blk)
-> ChainAndLedger blk
toChainAndLedger (ValidatedChainDiff ChainDiff (Header blk)
chainDiff LedgerDB' 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)
-> LedgerDB' blk -> ChainAndLedger blk
forall l b.
(GetTip l, HasHeader b, HeaderHash b ~ HeaderHash l,
 HasCallStack) =>
AnchoredFragment b -> l -> ValidatedFragment b l
VF.ValidatedFragment AnchoredFragment (Header blk)
suffix LedgerDB' blk
ledger
          | Bool
otherwise
          -> [Char] -> ChainAndLedger 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 Maybe Word64
limit (Anchor blk -> Point blk
forall block. Anchor block -> Point block
AF.anchorToPoint Anchor blk
i)
          where
            limit :: Maybe Word64
limit = case LoE ()
loE of
              LoE ()
LoEDisabled   -> Maybe Word64
forall a. Maybe a
Nothing
              LoEEnabled () -> Word64 -> Maybe Word64
forall a. a -> Maybe a
Just 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 blk
         -- ^ The current chain and ledger, corresponding to
         -- @i@.
      -> NonEmpty (AnchoredFragment (Header blk))
         -- ^ Candidates anchored at @i@
      -> m (Maybe (ValidatedChainDiff (Header blk) (LedgerDB' blk)))
    chainSelection' :: HasCallStack =>
ChainAndLedger blk
-> NonEmpty (AnchoredFragment (Header blk))
-> m (Maybe (ValidatedChainDiff (Header blk) (LedgerDB' blk)))
chainSelection' ChainAndLedger blk
curChainAndLedger NonEmpty (AnchoredFragment (Header blk))
candidates =
        Bool
-> m (Maybe (ValidatedChainDiff (Header blk) (LedgerDB' blk)))
-> m (Maybe (ValidatedChainDiff (Header blk) (LedgerDB' 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 ((LedgerDB' blk -> Point blk
forall blk. UpdateLedger blk => LedgerDB' blk -> Point blk
LgrDB.currentPoint LedgerDB' blk
ledger 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) (LedgerDB' blk)))
 -> m (Maybe (ValidatedChainDiff (Header blk) (LedgerDB' blk))))
-> m (Maybe (ValidatedChainDiff (Header blk) (LedgerDB' blk)))
-> m (Maybe (ValidatedChainDiff (Header blk) (LedgerDB' blk)))
forall a b. (a -> b) -> a -> b
$
        Bool
-> m (Maybe (ValidatedChainDiff (Header blk) (LedgerDB' blk)))
-> m (Maybe (ValidatedChainDiff (Header blk) (LedgerDB' 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.
(BlockSupportsProtocol blk, HasCallStack) =>
BlockConfig blk
-> AnchoredFragment (Header blk)
-> AnchoredFragment (Header blk)
-> Bool
preferAnchoredCandidate BlockConfig blk
bcfg AnchoredFragment (Header blk)
curChain) NonEmpty (AnchoredFragment (Header blk))
candidates) (m (Maybe (ValidatedChainDiff (Header blk) (LedgerDB' blk)))
 -> m (Maybe (ValidatedChainDiff (Header blk) (LedgerDB' blk))))
-> m (Maybe (ValidatedChainDiff (Header blk) (LedgerDB' blk)))
-> m (Maybe (ValidatedChainDiff (Header blk) (LedgerDB' blk)))
forall a b. (a -> b) -> a -> b
$ do
          ChainSelEnv m blk
cse <- m (ChainSelEnv m blk)
chainSelEnv
          ChainSelEnv m blk
-> NonEmpty (ChainDiff (Header blk))
-> m (Maybe (ValidatedChainDiff (Header blk) (LedgerDB' blk)))
forall (m :: * -> *) blk.
(IOLike m, LedgerSupportsProtocol blk,
 BlockSupportsDiffusionPipelining blk, HasCallStack) =>
ChainSelEnv m blk
-> NonEmpty (ChainDiff (Header blk))
-> m (Maybe (ValidatedChainDiff (Header blk) (LedgerDB' blk)))
chainSelection ChainSelEnv m blk
cse (AnchoredFragment (Header blk) -> ChainDiff (Header blk)
forall b. AnchoredFragment b -> ChainDiff b
Diff.extend (AnchoredFragment (Header blk) -> ChainDiff (Header blk))
-> NonEmpty (AnchoredFragment (Header blk))
-> NonEmpty (ChainDiff (Header blk))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> NonEmpty (AnchoredFragment (Header blk))
candidates)
      where
        curChain :: AnchoredFragment (Header blk)
curChain = ChainAndLedger blk -> AnchoredFragment (Header blk)
forall b l. ValidatedFragment b l -> AnchoredFragment b
VF.validatedFragment ChainAndLedger blk
curChainAndLedger
        ledger :: LedgerDB' blk
ledger   = ChainAndLedger blk -> LedgerDB' blk
forall b l. ValidatedFragment b l -> l
VF.validatedLedger   ChainAndLedger blk
curChainAndLedger
        chainSelEnv :: m (ChainSelEnv m blk)
chainSelEnv = do
          StrictTVar m (TentativeHeaderState blk)
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))
          StrictTVar m (StrictMaybe (Header blk))
varTentativeHeader <- StrictMaybe (Header blk)
-> m (StrictTVar m (StrictMaybe (Header blk)))
forall (m :: * -> *) a.
(HasCallStack, MonadSTM m, NoThunks a) =>
a -> m (StrictTVar m a)
newTVarIO StrictMaybe (Header blk)
forall a. StrictMaybe a
SNothing
          ChainSelEnv m blk -> m (ChainSelEnv m blk)
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure ChainSelEnv
            { LgrDB m blk
lgrDB :: LgrDB m blk
lgrDB :: LgrDB m blk
lgrDB
            , BlockConfig blk
bcfg :: BlockConfig blk
bcfg :: BlockConfig blk
bcfg
            , StrictTVar m (WithFingerprint (InvalidBlocks blk))
varInvalid :: StrictTVar m (WithFingerprint (InvalidBlocks blk))
varInvalid :: StrictTVar m (WithFingerprint (InvalidBlocks blk))
varInvalid
            , blockCache :: BlockCache blk
blockCache = BlockCache blk
forall blk. BlockCache blk
BlockCache.empty
            , ChainAndLedger blk
curChainAndLedger :: ChainAndLedger blk
curChainAndLedger :: ChainAndLedger blk
curChainAndLedger
            , validationTracer :: Tracer m (TraceValidationEvent blk)
validationTracer = TraceValidationEvent blk -> TraceInitChainSelEvent blk
forall blk. TraceValidationEvent blk -> TraceInitChainSelEvent blk
InitChainSelValidation (TraceValidationEvent blk -> TraceInitChainSelEvent blk)
-> Tracer m (TraceInitChainSelEvent blk)
-> Tracer m (TraceValidationEvent blk)
forall (f :: * -> *) a b. Contravariant f => (a -> b) -> f b -> f a
>$< Tracer m (TraceInitChainSelEvent blk)
tracer
              -- initial chain selection is not concerned about pipelining
            , pipeliningTracer :: Tracer m (TracePipeliningEvent blk)
pipeliningTracer = Tracer m (TracePipeliningEvent blk)
forall (m :: * -> *) a. Applicative m => Tracer m a
nullTracer
            , StrictTVar m (TentativeHeaderState blk)
varTentativeState :: StrictTVar m (TentativeHeaderState blk)
varTentativeState :: StrictTVar m (TentativeHeaderState blk)
varTentativeState
            , StrictTVar m (StrictMaybe (Header blk))
varTentativeHeader :: StrictTVar m (StrictMaybe (Header blk))
varTentativeHeader :: StrictTVar m (StrictMaybe (Header blk))
varTentativeHeader
            , punish :: Maybe (RealPoint blk, InvalidBlockPunishment m)
punish = Maybe (RealPoint blk, InvalidBlockPunishment m)
forall a. Maybe a
Nothing
            , getTentativeFollowers :: STM m [FollowerHandle m blk]
getTentativeFollowers = [FollowerHandle m blk] -> STM m [FollowerHandle m blk]
forall a. a -> STM m a
forall (f :: * -> *) a. Applicative f => a -> f a
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 (Header 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 (AnchoredFragment (Header blk))
StrictTVar m (TentativeHeaderState blk)
StrictTVar m (WithFingerprint (InvalidBlocks blk))
StrictTVar m FollowerKey
StrictTVar m IteratorKey
Fuse m
TopLevelConfig blk
VolatileDB m blk
ImmutableDB m blk
LgrDB 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
cdbLgrDB :: LgrDB m blk
cdbChain :: StrictTVar m (AnchoredFragment (Header 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 (Header blk)))
cdbImmutableDB :: forall (m :: * -> *) blk. ChainDbEnv m blk -> ImmutableDB m blk
cdbVolatileDB :: forall (m :: * -> *) blk. ChainDbEnv m blk -> VolatileDB m blk
cdbLgrDB :: forall (m :: * -> *) blk. ChainDbEnv m blk -> LgrDB m blk
cdbChain :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> StrictTVar m (AnchoredFragment (Header blk))
cdbTentativeState :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> StrictTVar m (TentativeHeaderState blk)
cdbTentativeHeader :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> StrictTVar m (StrictMaybe (Header blk))
cdbIterators :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> StrictTVar m (Map IteratorKey (m ()))
cdbFollowers :: forall (m :: * -> *) blk.
ChainDbEnv m blk
-> StrictTVar m (Map FollowerKey (FollowerHandle m blk))
cdbTopLevelConfig :: forall (m :: * -> *) blk. ChainDbEnv m blk -> TopLevelConfig blk
cdbInvalid :: forall (m :: * -> *) blk.
ChainDbEnv m blk
-> StrictTVar m (WithFingerprint (InvalidBlocks blk))
cdbNextIteratorKey :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> StrictTVar m IteratorKey
cdbNextFollowerKey :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> StrictTVar m FollowerKey
cdbCopyFuse :: forall (m :: * -> *) blk. ChainDbEnv m blk -> Fuse m
cdbChainSelFuse :: forall (m :: * -> *) blk. ChainDbEnv m blk -> Fuse m
cdbRegistry :: forall (m :: * -> *) blk. ChainDbEnv m blk -> ResourceRegistry m
cdbGcDelay :: forall (m :: * -> *) blk. ChainDbEnv m blk -> DiffTime
cdbGcInterval :: forall (m :: * -> *) blk. ChainDbEnv m blk -> DiffTime
cdbKillBgThreads :: forall (m :: * -> *) blk. ChainDbEnv m blk -> StrictTVar m (m ())
cdbLoE :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> m (LoE (AnchoredFragment (Header blk)))
..} (ChainSelReprocessLoEBlocks StrictTMVar m ()
varProcessed) = do
    m (LoE (AnchoredFragment (Header blk)))
-> Electric m (LoE (AnchoredFragment (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 (LoE (AnchoredFragment (Header blk)))
cdbLoE Electric m (LoE (AnchoredFragment (Header blk)))
-> (LoE (AnchoredFragment (Header 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 (Header blk))
LoEDisabled  -> () -> Electric m ()
forall a. a -> Electric m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure ()
      LoEEnabled AnchoredFragment (Header blk)
_ -> do
        (ChainHash blk -> Set (HeaderHash blk)
succsOf, AnchoredFragment (Header blk)
chain) <- m (ChainHash blk -> Set (HeaderHash blk),
   AnchoredFragment (Header blk))
-> Electric
     m
     (ChainHash blk -> Set (HeaderHash blk),
      AnchoredFragment (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),
    AnchoredFragment (Header blk))
 -> Electric
      m
      (ChainHash blk -> Set (HeaderHash blk),
       AnchoredFragment (Header blk)))
-> m (ChainHash blk -> Set (HeaderHash blk),
      AnchoredFragment (Header blk))
-> Electric
     m
     (ChainHash blk -> Set (HeaderHash blk),
      AnchoredFragment (Header blk))
forall a b. (a -> b) -> a -> b
$ STM
  m
  (ChainHash blk -> Set (HeaderHash blk),
   AnchoredFragment (Header blk))
-> m (ChainHash blk -> Set (HeaderHash blk),
      AnchoredFragment (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),
    AnchoredFragment (Header blk))
 -> m (ChainHash blk -> Set (HeaderHash blk),
       AnchoredFragment (Header blk)))
-> STM
     m
     (ChainHash blk -> Set (HeaderHash blk),
      AnchoredFragment (Header blk))
-> m (ChainHash blk -> Set (HeaderHash blk),
      AnchoredFragment (Header blk))
forall a b. (a -> b) -> a -> b
$ do
          InvalidBlocks blk
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
          (,)
            ((ChainHash blk -> Set (HeaderHash blk))
 -> AnchoredFragment (Header blk)
 -> (ChainHash blk -> Set (HeaderHash blk),
     AnchoredFragment (Header blk)))
-> STM m (ChainHash blk -> Set (HeaderHash blk))
-> STM
     m
     (AnchoredFragment (Header blk)
      -> (ChainHash blk -> Set (HeaderHash blk),
          AnchoredFragment (Header blk)))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> (VolatileDB 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 VolatileDB m blk
cdbVolatileDB InvalidBlocks blk
invalid ((ChainHash blk -> Set (HeaderHash blk))
 -> ChainHash blk -> Set (HeaderHash blk))
-> STM m (ChainHash blk -> Set (HeaderHash blk))
-> STM m (ChainHash blk -> Set (HeaderHash blk))
forall (f :: * -> *) a b. Functor 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
  (AnchoredFragment (Header blk)
   -> (ChainHash blk -> Set (HeaderHash blk),
       AnchoredFragment (Header blk)))
-> STM m (AnchoredFragment (Header blk))
-> STM
     m
     (ChainHash blk -> Set (HeaderHash blk),
      AnchoredFragment (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 (AnchoredFragment (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
            succsOf' :: Point (Header blk) -> [HeaderHash blk]
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 :: [HeaderHash blk]
loeHashes = Point (Header blk) -> [HeaderHash blk]
succsOf' (AnchoredFragment (Header blk) -> Point (Header blk)
forall block. AnchoredFragment block -> Point block
AF.anchorPoint AnchoredFragment (Header blk)
chain)
            firstHeader :: Maybe (Header blk)
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
$ AnchoredFragment (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 AnchoredFragment (Header blk)
chain
            -- We avoid the VolatileDB for the headers we already have in the chain
            getHeaderFromHash :: HeaderHash blk -> m (Header blk)
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
        [Header blk]
loeHeaders <- m [Header blk] -> Electric m [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 ((HeaderHash blk -> m (Header blk))
-> [HeaderHash 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 HeaderHash blk -> m (Header blk)
getHeaderFromHash [HeaderHash blk]
loeHashes)
        [Header blk] -> (Header blk -> Electric m ()) -> Electric m ()
forall (t :: * -> *) (f :: * -> *) a b.
(Foldable t, Applicative f) =>
t a -> (a -> f b) -> f ()
for_ [Header blk]
loeHeaders ((Header blk -> Electric m ()) -> Electric m ())
-> (Header blk -> Electric m ()) -> Electric m ()
forall a b. (a -> b) -> a -> b
$ \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 (Header 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 (AnchoredFragment (Header blk))
StrictTVar m (TentativeHeaderState blk)
StrictTVar m (WithFingerprint (InvalidBlocks blk))
StrictTVar m FollowerKey
StrictTVar m IteratorKey
Fuse m
TopLevelConfig blk
VolatileDB m blk
ImmutableDB m blk
LgrDB 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 :: forall (m :: * -> *) blk. ChainDbEnv m blk -> ImmutableDB m blk
cdbVolatileDB :: forall (m :: * -> *) blk. ChainDbEnv m blk -> VolatileDB m blk
cdbLgrDB :: forall (m :: * -> *) blk. ChainDbEnv m blk -> LgrDB m blk
cdbChain :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> StrictTVar m (AnchoredFragment (Header blk))
cdbTentativeState :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> StrictTVar m (TentativeHeaderState blk)
cdbTentativeHeader :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> StrictTVar m (StrictMaybe (Header blk))
cdbIterators :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> StrictTVar m (Map IteratorKey (m ()))
cdbFollowers :: forall (m :: * -> *) blk.
ChainDbEnv m blk
-> StrictTVar m (Map FollowerKey (FollowerHandle m blk))
cdbTopLevelConfig :: forall (m :: * -> *) blk. ChainDbEnv m blk -> TopLevelConfig blk
cdbInvalid :: forall (m :: * -> *) blk.
ChainDbEnv m blk
-> StrictTVar m (WithFingerprint (InvalidBlocks blk))
cdbNextIteratorKey :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> StrictTVar m IteratorKey
cdbNextFollowerKey :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> StrictTVar m FollowerKey
cdbCopyFuse :: forall (m :: * -> *) blk. ChainDbEnv m blk -> Fuse m
cdbChainSelFuse :: forall (m :: * -> *) blk. ChainDbEnv m blk -> Fuse m
cdbRegistry :: forall (m :: * -> *) blk. ChainDbEnv m blk -> ResourceRegistry m
cdbGcDelay :: forall (m :: * -> *) blk. ChainDbEnv m blk -> DiffTime
cdbGcInterval :: forall (m :: * -> *) blk. ChainDbEnv m blk -> DiffTime
cdbKillBgThreads :: forall (m :: * -> *) blk. ChainDbEnv m blk -> StrictTVar m (m ())
cdbLoE :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> m (LoE (AnchoredFragment (Header blk)))
cdbImmutableDB :: ImmutableDB m blk
cdbVolatileDB :: VolatileDB m blk
cdbLgrDB :: LgrDB m blk
cdbChain :: StrictTVar m (AnchoredFragment (Header 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 (Header blk)))
..} (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)
blockPunish :: forall (m :: * -> *) blk.
BlockToAdd m blk -> InvalidBlockPunishment m
varBlockWrittenToDisk :: forall (m :: * -> *) blk. BlockToAdd m blk -> StrictTMVar m Bool
varBlockProcessed :: forall (m :: * -> *) blk.
BlockToAdd m blk -> StrictTMVar m (AddBlockResult blk)
.. }) = do
    (HeaderHash blk -> Bool
isMember, InvalidBlocks blk
invalid, AnchoredFragment (Header blk)
curChain) <- m (HeaderHash blk -> Bool, InvalidBlocks blk,
   AnchoredFragment (Header blk))
-> Electric
     m
     (HeaderHash blk -> Bool, InvalidBlocks blk,
      AnchoredFragment (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,
    AnchoredFragment (Header blk))
 -> Electric
      m
      (HeaderHash blk -> Bool, InvalidBlocks blk,
       AnchoredFragment (Header blk)))
-> m (HeaderHash blk -> Bool, InvalidBlocks blk,
      AnchoredFragment (Header blk))
-> Electric
     m
     (HeaderHash blk -> Bool, InvalidBlocks blk,
      AnchoredFragment (Header blk))
forall a b. (a -> b) -> a -> b
$ STM
  m
  (HeaderHash blk -> Bool, InvalidBlocks blk,
   AnchoredFragment (Header blk))
-> m (HeaderHash blk -> Bool, InvalidBlocks blk,
      AnchoredFragment (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,
    AnchoredFragment (Header blk))
 -> m (HeaderHash blk -> Bool, InvalidBlocks blk,
       AnchoredFragment (Header blk)))
-> STM
     m
     (HeaderHash blk -> Bool, InvalidBlocks blk,
      AnchoredFragment (Header blk))
-> m (HeaderHash blk -> Bool, InvalidBlocks blk,
      AnchoredFragment (Header blk))
forall a b. (a -> b) -> a -> b
$ (,,)
      ((HeaderHash blk -> Bool)
 -> InvalidBlocks blk
 -> AnchoredFragment (Header blk)
 -> (HeaderHash blk -> Bool, InvalidBlocks blk,
     AnchoredFragment (Header blk)))
-> STM m (HeaderHash blk -> Bool)
-> STM
     m
     (InvalidBlocks blk
      -> AnchoredFragment (Header blk)
      -> (HeaderHash blk -> Bool, InvalidBlocks blk,
          AnchoredFragment (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
   -> AnchoredFragment (Header blk)
   -> (HeaderHash blk -> Bool, InvalidBlocks blk,
       AnchoredFragment (Header blk)))
-> STM m (InvalidBlocks blk)
-> STM
     m
     (AnchoredFragment (Header blk)
      -> (HeaderHash blk -> Bool, InvalidBlocks blk,
          AnchoredFragment (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
  (AnchoredFragment (Header blk)
   -> (HeaderHash blk -> Bool, InvalidBlocks blk,
       AnchoredFragment (Header blk)))
-> STM m (AnchoredFragment (Header blk))
-> STM
     m
     (HeaderHash blk -> Bool, InvalidBlocks blk,
      AnchoredFragment (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 (AnchoredFragment (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 :: WithOrigin BlockNo
immBlockNo = AnchoredFragment (Header blk) -> WithOrigin BlockNo
forall block. AnchoredFragment block -> WithOrigin BlockNo
AF.anchorBlockNo AnchoredFragment (Header blk)
curChain

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

    if
      | Header blk -> IsEBB -> WithOrigin BlockNo -> Bool
forall blk.
HasHeader (Header blk) =>
Header blk -> IsEBB -> WithOrigin BlockNo -> Bool
olderThanK Header blk
hdr IsEBB
isEBB WithOrigin BlockNo
immBlockNo -> do
        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
$ 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
$ RealPoint blk -> TraceAddBlockEvent blk
forall blk. RealPoint blk -> TraceAddBlockEvent blk
IgnoreBlockOlderThanK (blk -> RealPoint blk
forall blk. HasHeader blk => blk -> RealPoint blk
blockRealPoint blk
b)
        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
$ Bool -> m ()
deliverWrittenToDisk Bool
False

      | HeaderHash blk -> Bool
isMember (blk -> HeaderHash blk
forall b. HasHeader b => b -> HeaderHash b
blockHash blk
b) -> do
        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
$ 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
$ RealPoint blk -> TraceAddBlockEvent blk
forall blk. RealPoint blk -> TraceAddBlockEvent blk
IgnoreBlockAlreadyInVolatileDB (blk -> RealPoint blk
forall blk. HasHeader blk => blk -> RealPoint blk
blockRealPoint blk
b)
        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
$ Bool -> m ()
deliverWrittenToDisk Bool
True

      | Just (InvalidBlockInfo ExtValidationError blk
reason SlotNo
_) <- HeaderHash blk -> InvalidBlocks blk -> Maybe (InvalidBlockInfo blk)
forall k a. Ord k => k -> Map k a -> Maybe a
Map.lookup (blk -> HeaderHash blk
forall b. HasHeader b => b -> HeaderHash b
blockHash blk
b) InvalidBlocks blk
invalid -> do
        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
$ 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
$ RealPoint blk -> ExtValidationError blk -> TraceAddBlockEvent blk
forall blk.
RealPoint blk -> ExtValidationError blk -> TraceAddBlockEvent blk
IgnoreInvalidBlock (blk -> RealPoint blk
forall blk. HasHeader blk => blk -> RealPoint blk
blockRealPoint blk
b) ExtValidationError blk
reason
        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
$ Bool -> m ()
deliverWrittenToDisk Bool
False

        -- We wouldn't know the block is invalid if its prefix was invalid,
        -- hence 'InvalidBlockPunishment.BlockItself'.
        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
$ InvalidBlockPunishment m -> Invalidity -> m ()
forall (m :: * -> *).
InvalidBlockPunishment m -> Invalidity -> m ()
InvalidBlockPunishment.enact
          InvalidBlockPunishment m
blockPunish
          Invalidity
InvalidBlockPunishment.BlockItself

      -- The remaining cases
      | Bool
otherwise -> do
        let traceEv :: Enclosing -> TraceAddBlockEvent blk
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
        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
$ Tracer m Enclosing -> m () -> m ()
forall (m :: * -> *) a.
Applicative m =>
Tracer m Enclosing -> m a -> m a
encloseWith (Enclosing -> TraceAddBlockEvent blk
traceEv (Enclosing -> TraceAddBlockEvent blk)
-> Tracer m (TraceAddBlockEvent blk) -> Tracer m Enclosing
forall (f :: * -> *) a b. Contravariant f => (a -> b) -> f b -> f a
>$< Tracer m (TraceAddBlockEvent blk)
addBlockTracer) (m () -> m ()) -> m () -> m ()
forall a b. (a -> b) -> a -> b
$
          VolatileDB m blk -> HasCallStack => blk -> m ()
forall (m :: * -> *) blk.
VolatileDB m blk -> HasCallStack => blk -> m ()
VolatileDB.putBlock VolatileDB m blk
cdbVolatileDB blk
b
        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
$ Bool -> m ()
deliverWrittenToDisk Bool
True
        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 (blk -> BlockCache blk
forall blk. HasHeader blk => blk -> BlockCache blk
BlockCache.singleton blk
b) Header blk
hdr InvalidBlockPunishment m
blockPunish

    Point blk
newTip <- m (Point blk) -> Electric m (Point 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 (Point blk) -> Electric m (Point blk))
-> m (Point blk) -> Electric m (Point blk)
forall a b. (a -> b) -> a -> b
$ 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
$ 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

    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
$ Point blk -> m ()
deliverProcessed Point blk
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 (Header 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 (AnchoredFragment (Header blk))
StrictTVar m (TentativeHeaderState blk)
StrictTVar m (WithFingerprint (InvalidBlocks blk))
StrictTVar m FollowerKey
StrictTVar m IteratorKey
Fuse m
TopLevelConfig blk
VolatileDB m blk
ImmutableDB m blk
LgrDB 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 :: forall (m :: * -> *) blk. ChainDbEnv m blk -> ImmutableDB m blk
cdbVolatileDB :: forall (m :: * -> *) blk. ChainDbEnv m blk -> VolatileDB m blk
cdbLgrDB :: forall (m :: * -> *) blk. ChainDbEnv m blk -> LgrDB m blk
cdbChain :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> StrictTVar m (AnchoredFragment (Header blk))
cdbTentativeState :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> StrictTVar m (TentativeHeaderState blk)
cdbTentativeHeader :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> StrictTVar m (StrictMaybe (Header blk))
cdbIterators :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> StrictTVar m (Map IteratorKey (m ()))
cdbFollowers :: forall (m :: * -> *) blk.
ChainDbEnv m blk
-> StrictTVar m (Map FollowerKey (FollowerHandle m blk))
cdbTopLevelConfig :: forall (m :: * -> *) blk. ChainDbEnv m blk -> TopLevelConfig blk
cdbInvalid :: forall (m :: * -> *) blk.
ChainDbEnv m blk
-> StrictTVar m (WithFingerprint (InvalidBlocks blk))
cdbNextIteratorKey :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> StrictTVar m IteratorKey
cdbNextFollowerKey :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> StrictTVar m FollowerKey
cdbCopyFuse :: forall (m :: * -> *) blk. ChainDbEnv m blk -> Fuse m
cdbChainSelFuse :: forall (m :: * -> *) blk. ChainDbEnv m blk -> Fuse m
cdbRegistry :: forall (m :: * -> *) blk. ChainDbEnv m blk -> ResourceRegistry m
cdbGcDelay :: forall (m :: * -> *) blk. ChainDbEnv m blk -> DiffTime
cdbGcInterval :: forall (m :: * -> *) blk. ChainDbEnv m blk -> DiffTime
cdbKillBgThreads :: forall (m :: * -> *) blk. ChainDbEnv m blk -> StrictTVar m (m ())
cdbLoE :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> m (LoE (AnchoredFragment (Header blk)))
cdbImmutableDB :: ImmutableDB m blk
cdbVolatileDB :: VolatileDB m blk
cdbLgrDB :: LgrDB m blk
cdbChain :: StrictTVar m (AnchoredFragment (Header 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 (Header blk)))
..} 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
$ do
    (InvalidBlocks blk
invalid, ChainHash blk -> Set (HeaderHash blk)
succsOf', HeaderHash blk -> Maybe (BlockInfo blk)
lookupBlockInfo, HeaderHash blk -> Maybe (BlockInfo blk)
lookupBlockInfo', AnchoredFragment (Header blk)
curChain, Point blk
tipPoint, LedgerDB' blk
ledgerDB)
      <- STM
  m
  (InvalidBlocks blk, ChainHash blk -> Set (HeaderHash blk),
   HeaderHash blk -> Maybe (BlockInfo blk),
   HeaderHash blk -> Maybe (BlockInfo blk),
   AnchoredFragment (Header blk), Point blk, LedgerDB' blk)
-> m (InvalidBlocks blk, ChainHash blk -> Set (HeaderHash blk),
      HeaderHash blk -> Maybe (BlockInfo blk),
      HeaderHash blk -> Maybe (BlockInfo blk),
      AnchoredFragment (Header blk), Point blk, LedgerDB' 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),
    HeaderHash blk -> Maybe (BlockInfo blk),
    AnchoredFragment (Header blk), Point blk, LedgerDB' blk)
 -> m (InvalidBlocks blk, ChainHash blk -> Set (HeaderHash blk),
       HeaderHash blk -> Maybe (BlockInfo blk),
       HeaderHash blk -> Maybe (BlockInfo blk),
       AnchoredFragment (Header blk), Point blk, LedgerDB' blk))
-> STM
     m
     (InvalidBlocks blk, ChainHash blk -> Set (HeaderHash blk),
      HeaderHash blk -> Maybe (BlockInfo blk),
      HeaderHash blk -> Maybe (BlockInfo blk),
      AnchoredFragment (Header blk), Point blk, LedgerDB' blk)
-> m (InvalidBlocks blk, ChainHash blk -> Set (HeaderHash blk),
      HeaderHash blk -> Maybe (BlockInfo blk),
      HeaderHash blk -> Maybe (BlockInfo blk),
      AnchoredFragment (Header blk), Point blk, LedgerDB' blk)
forall a b. (a -> b) -> a -> b
$ do
          (InvalidBlocks blk
invalid, ChainHash blk -> Set (HeaderHash blk)
succsOf, HeaderHash blk -> Maybe (BlockInfo blk)
lookupBlockInfo, AnchoredFragment (Header blk)
curChain, Point blk
tipPoint, LedgerDB' blk
ledgerDB) <-
                (,,,,,)
            (InvalidBlocks blk
 -> (ChainHash blk -> Set (HeaderHash blk))
 -> (HeaderHash blk -> Maybe (BlockInfo blk))
 -> AnchoredFragment (Header blk)
 -> Point blk
 -> LedgerDB' blk
 -> (InvalidBlocks blk, ChainHash blk -> Set (HeaderHash blk),
     HeaderHash blk -> Maybe (BlockInfo blk),
     AnchoredFragment (Header blk), Point blk, LedgerDB' blk))
-> STM m (InvalidBlocks blk)
-> STM
     m
     ((ChainHash blk -> Set (HeaderHash blk))
      -> (HeaderHash blk -> Maybe (BlockInfo blk))
      -> AnchoredFragment (Header blk)
      -> Point blk
      -> LedgerDB' blk
      -> (InvalidBlocks blk, ChainHash blk -> Set (HeaderHash blk),
          HeaderHash blk -> Maybe (BlockInfo blk),
          AnchoredFragment (Header blk), Point blk, LedgerDB' 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))
   -> AnchoredFragment (Header blk)
   -> Point blk
   -> LedgerDB' blk
   -> (InvalidBlocks blk, ChainHash blk -> Set (HeaderHash blk),
       HeaderHash blk -> Maybe (BlockInfo blk),
       AnchoredFragment (Header blk), Point blk, LedgerDB' blk))
-> STM m (ChainHash blk -> Set (HeaderHash blk))
-> STM
     m
     ((HeaderHash blk -> Maybe (BlockInfo blk))
      -> AnchoredFragment (Header blk)
      -> Point blk
      -> LedgerDB' blk
      -> (InvalidBlocks blk, ChainHash blk -> Set (HeaderHash blk),
          HeaderHash blk -> Maybe (BlockInfo blk),
          AnchoredFragment (Header blk), Point blk, LedgerDB' 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))
   -> AnchoredFragment (Header blk)
   -> Point blk
   -> LedgerDB' blk
   -> (InvalidBlocks blk, ChainHash blk -> Set (HeaderHash blk),
       HeaderHash blk -> Maybe (BlockInfo blk),
       AnchoredFragment (Header blk), Point blk, LedgerDB' blk))
-> STM m (HeaderHash blk -> Maybe (BlockInfo blk))
-> STM
     m
     (AnchoredFragment (Header blk)
      -> Point blk
      -> LedgerDB' blk
      -> (InvalidBlocks blk, ChainHash blk -> Set (HeaderHash blk),
          HeaderHash blk -> Maybe (BlockInfo blk),
          AnchoredFragment (Header blk), Point blk, LedgerDB' 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
  (AnchoredFragment (Header blk)
   -> Point blk
   -> LedgerDB' blk
   -> (InvalidBlocks blk, ChainHash blk -> Set (HeaderHash blk),
       HeaderHash blk -> Maybe (BlockInfo blk),
       AnchoredFragment (Header blk), Point blk, LedgerDB' blk))
-> STM m (AnchoredFragment (Header blk))
-> STM
     m
     (Point blk
      -> LedgerDB' blk
      -> (InvalidBlocks blk, ChainHash blk -> Set (HeaderHash blk),
          HeaderHash blk -> Maybe (BlockInfo blk),
          AnchoredFragment (Header blk), Point blk, LedgerDB' 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 (AnchoredFragment (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
   -> LedgerDB' blk
   -> (InvalidBlocks blk, ChainHash blk -> Set (HeaderHash blk),
       HeaderHash blk -> Maybe (BlockInfo blk),
       AnchoredFragment (Header blk), Point blk, LedgerDB' blk))
-> STM m (Point blk)
-> STM
     m
     (LedgerDB' blk
      -> (InvalidBlocks blk, ChainHash blk -> Set (HeaderHash blk),
          HeaderHash blk -> Maybe (BlockInfo blk),
          AnchoredFragment (Header blk), Point blk, LedgerDB' 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
            STM
  m
  (LedgerDB' blk
   -> (InvalidBlocks blk, ChainHash blk -> Set (HeaderHash blk),
       HeaderHash blk -> Maybe (BlockInfo blk),
       AnchoredFragment (Header blk), Point blk, LedgerDB' blk))
-> STM m (LedgerDB' blk)
-> STM
     m
     (InvalidBlocks blk, ChainHash blk -> Set (HeaderHash blk),
      HeaderHash blk -> Maybe (BlockInfo blk),
      AnchoredFragment (Header blk), Point blk, LedgerDB' 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
<*> LgrDB m blk -> STM m (LedgerDB' blk)
forall (m :: * -> *) blk.
IOLike m =>
LgrDB m blk -> STM m (LedgerDB' blk)
LgrDB.getCurrent                LgrDB m blk
cdbLgrDB

          -- Let these two functions ignore invalid blocks
          let lookupBlockInfo' :: HeaderHash blk -> Maybe (BlockInfo blk)
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' :: ChainHash blk -> Set (HeaderHash blk)
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

          (InvalidBlocks blk, ChainHash blk -> Set (HeaderHash blk),
 HeaderHash blk -> Maybe (BlockInfo blk),
 HeaderHash blk -> Maybe (BlockInfo blk),
 AnchoredFragment (Header blk), Point blk, LedgerDB' blk)
-> STM
     m
     (InvalidBlocks blk, ChainHash blk -> Set (HeaderHash blk),
      HeaderHash blk -> Maybe (BlockInfo blk),
      HeaderHash blk -> Maybe (BlockInfo blk),
      AnchoredFragment (Header blk), Point blk, LedgerDB' blk)
forall a. a -> STM m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (InvalidBlocks blk
invalid, ChainHash blk -> Set (HeaderHash blk)
succsOf', HeaderHash blk -> Maybe (BlockInfo blk)
lookupBlockInfo, HeaderHash blk -> Maybe (BlockInfo blk)
lookupBlockInfo', AnchoredFragment (Header blk)
curChain, Point blk
tipPoint, LedgerDB' blk
ledgerDB)

    let curChainAndLedger :: ChainAndLedger blk
        curChainAndLedger :: ChainAndLedger blk
curChainAndLedger =
          -- 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 -> ChainAndLedger blk -> ChainAndLedger blk
forall a. HasCallStack => Bool -> a -> a
assert (Int -> Word64
forall a b. (Integral a, Num b) => a -> b
fromIntegral (AnchoredFragment (Header blk) -> Int
forall v a b. Anchorable v a b => AnchoredSeq v a b -> Int
AF.length AnchoredFragment (Header blk)
curChain) Word64 -> Word64 -> Bool
forall a. Ord a => a -> a -> Bool
<= Word64
k) (ChainAndLedger blk -> ChainAndLedger blk)
-> ChainAndLedger blk -> ChainAndLedger blk
forall a b. (a -> b) -> a -> b
$
          AnchoredFragment (Header blk)
-> LedgerDB' blk -> ChainAndLedger blk
forall l b.
(GetTip l, HasHeader b, HeaderHash b ~ HeaderHash l,
 HasCallStack) =>
AnchoredFragment b -> l -> ValidatedFragment b l
VF.ValidatedFragment AnchoredFragment (Header blk)
curChain LedgerDB' blk
ledgerDB

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

    -- The preconditions
    Bool -> m () -> m ()
forall a. HasCallStack => Bool -> a -> a
assert (Maybe (BlockInfo blk) -> Bool
forall a. Maybe a -> Bool
isJust (Maybe (BlockInfo blk) -> Bool) -> Maybe (BlockInfo blk) -> Bool
forall a b. (a -> b) -> a -> b
$ HeaderHash blk -> Maybe (BlockInfo blk)
lookupBlockInfo (Header blk -> HeaderHash blk
forall blk. HasHeader (Header blk) => Header blk -> HeaderHash blk
headerHash Header blk
hdr)) (m () -> m ()) -> m () -> m ()
forall a b. (a -> b) -> a -> b
$ () -> m ()
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
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 (Header blk) -> AnchoredFragment (Header blk)
sanitizeLoEFrag AnchoredFragment (Header blk)
loeFrag0 =
        case AnchoredFragment (Header blk)
-> Point (Header blk)
-> Maybe
     (AnchoredFragment (Header blk), AnchoredFragment (Header blk))
forall block1 block2.
(HasHeader block1, HeaderHash block1 ~ HeaderHash block2) =>
AnchoredFragment block1
-> Point block2
-> Maybe (AnchoredFragment block1, AnchoredFragment block1)
AF.splitAfterPoint AnchoredFragment (Header blk)
loeFrag0 (AnchoredFragment (Header blk) -> Point (Header blk)
forall block. AnchoredFragment block -> Point block
AF.anchorPoint AnchoredFragment (Header blk)
curChain) of
            Just (AnchoredFragment (Header blk)
_, AnchoredFragment (Header blk)
frag) -> AnchoredFragment (Header 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 (Header blk), AnchoredFragment (Header blk))
Nothing        -> Anchor (Header blk) -> AnchoredFragment (Header blk)
forall v a b. Anchorable v a b => a -> AnchoredSeq v a b
AF.Empty (AnchoredFragment (Header blk) -> Anchor (Header blk)
forall v a b. AnchoredSeq v a b -> a
AF.anchor AnchoredFragment (Header blk)
curChain)

    LoE (AnchoredFragment (Header blk))
loeFrag <- (AnchoredFragment (Header blk) -> AnchoredFragment (Header blk))
-> LoE (AnchoredFragment (Header blk))
-> LoE (AnchoredFragment (Header blk))
forall a b. (a -> b) -> LoE a -> LoE b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap AnchoredFragment (Header blk) -> AnchoredFragment (Header blk)
sanitizeLoEFrag (LoE (AnchoredFragment (Header blk))
 -> LoE (AnchoredFragment (Header blk)))
-> m (LoE (AnchoredFragment (Header blk)))
-> m (LoE (AnchoredFragment (Header blk)))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> m (LoE (AnchoredFragment (Header blk)))
cdbLoE

    Tracer m (TraceAddBlockEvent blk) -> TraceAddBlockEvent blk -> m ()
forall (m :: * -> *) a. Tracer m a -> a -> m ()
traceWith Tracer m (TraceAddBlockEvent blk)
addBlockTracer (AnchoredFragment (Header blk)
-> LoE (AnchoredFragment (Header blk)) -> TraceAddBlockEvent blk
forall blk.
AnchoredFragment (Header blk)
-> LoE (AnchoredFragment (Header blk)) -> TraceAddBlockEvent blk
ChainSelectionLoEDebug AnchoredFragment (Header blk)
curChain LoE (AnchoredFragment (Header blk))
loeFrag)

    if
      -- The chain might have grown since we added the block such that the
      -- block is older than @k@.
      | Header blk -> IsEBB -> WithOrigin BlockNo -> Bool
forall blk.
HasHeader (Header blk) =>
Header blk -> IsEBB -> WithOrigin BlockNo -> Bool
olderThanK Header blk
hdr IsEBB
isEBB WithOrigin BlockNo
immBlockNo -> 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
$ RealPoint blk -> TraceAddBlockEvent blk
forall blk. RealPoint blk -> TraceAddBlockEvent blk
IgnoreBlockOlderThanK RealPoint blk
p

      -- The block is invalid
      | Just (InvalidBlockInfo ExtValidationError blk
reason SlotNo
_) <- HeaderHash blk -> InvalidBlocks blk -> Maybe (InvalidBlockInfo blk)
forall k a. Ord k => k -> Map k a -> Maybe a
Map.lookup (Header blk -> HeaderHash blk
forall blk. HasHeader (Header blk) => Header blk -> HeaderHash blk
headerHash Header blk
hdr) InvalidBlocks blk
invalid -> 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
$ RealPoint blk -> ExtValidationError blk -> TraceAddBlockEvent blk
forall blk.
RealPoint blk -> ExtValidationError blk -> TraceAddBlockEvent blk
IgnoreInvalidBlock RealPoint blk
p ExtValidationError blk
reason

        -- We wouldn't know the block is invalid if its prefix was invalid,
        -- hence 'InvalidBlockPunishment.BlockItself'.
        InvalidBlockPunishment m -> Invalidity -> m ()
forall (m :: * -> *).
InvalidBlockPunishment m -> Invalidity -> m ()
InvalidBlockPunishment.enact
          InvalidBlockPunishment m
punish
          Invalidity
InvalidBlockPunishment.BlockItself

      -- The block fits onto the end of our current chain
      | Point blk -> ChainHash blk
forall {k} (block :: k). Point block -> ChainHash block
pointHash Point blk
tipPoint ChainHash blk -> ChainHash blk -> Bool
forall a. Eq a => a -> a -> Bool
== Header blk -> ChainHash blk
forall blk. GetPrevHash blk => Header blk -> ChainHash blk
headerPrevHash Header blk
hdr -> do
        -- ### Add to current chain
        Tracer m (TraceAddBlockEvent blk) -> TraceAddBlockEvent blk -> m ()
forall (m :: * -> *) a. Tracer m a -> a -> m ()
traceWith Tracer m (TraceAddBlockEvent blk)
addBlockTracer (RealPoint blk -> TraceAddBlockEvent blk
forall blk. RealPoint blk -> TraceAddBlockEvent blk
TryAddToCurrentChain RealPoint blk
p)
        HasCallStack =>
(ChainHash blk -> Set (HeaderHash blk))
-> ChainAndLedger blk
-> LoE (AnchoredFragment (Header blk))
-> m ()
(ChainHash blk -> Set (HeaderHash blk))
-> ChainAndLedger blk
-> LoE (AnchoredFragment (Header blk))
-> m ()
addToCurrentChain ChainHash blk -> Set (HeaderHash blk)
succsOf' ChainAndLedger blk
curChainAndLedger LoE (AnchoredFragment (Header blk))
loeFrag

      -- The block is reachable from the current selection
      -- and it doesn't fit after the current selection
      | Just ChainDiff (HeaderFields blk)
diff <- (HeaderHash blk -> Maybe (BlockInfo blk))
-> AnchoredFragment (Header blk)
-> RealPoint blk
-> Maybe (ChainDiff (HeaderFields blk))
forall blk.
(HasHeader blk, GetHeader blk) =>
LookupBlockInfo blk
-> AnchoredFragment (Header blk)
-> RealPoint blk
-> Maybe (ChainDiff (HeaderFields blk))
Paths.isReachable HeaderHash blk -> Maybe (BlockInfo blk)
lookupBlockInfo' AnchoredFragment (Header blk)
curChain RealPoint blk
p -> do
        -- ### Switch to a fork
        Tracer m (TraceAddBlockEvent blk) -> TraceAddBlockEvent blk -> m ()
forall (m :: * -> *) a. Tracer m a -> a -> m ()
traceWith Tracer m (TraceAddBlockEvent blk)
addBlockTracer (RealPoint blk
-> ChainDiff (HeaderFields blk) -> TraceAddBlockEvent blk
forall blk.
RealPoint blk
-> ChainDiff (HeaderFields blk) -> TraceAddBlockEvent blk
TrySwitchToAFork RealPoint blk
p ChainDiff (HeaderFields blk)
diff)
        HasCallStack =>
(ChainHash blk -> Set (HeaderHash blk))
-> (HeaderHash blk -> Maybe (BlockInfo blk))
-> ChainAndLedger blk
-> LoE (AnchoredFragment (Header blk))
-> ChainDiff (HeaderFields blk)
-> m ()
(ChainHash blk -> Set (HeaderHash blk))
-> (HeaderHash blk -> Maybe (BlockInfo blk))
-> ChainAndLedger blk
-> LoE (AnchoredFragment (Header blk))
-> ChainDiff (HeaderFields blk)
-> m ()
switchToAFork ChainHash blk -> Set (HeaderHash blk)
succsOf' HeaderHash blk -> Maybe (BlockInfo blk)
lookupBlockInfo' ChainAndLedger blk
curChainAndLedger LoE (AnchoredFragment (Header blk))
loeFrag ChainDiff (HeaderFields blk)
diff

      -- We cannot reach the block from the current selection
      | Bool
otherwise -> do
        -- ### Store but don't change the current chain
        Tracer m (TraceAddBlockEvent blk) -> TraceAddBlockEvent blk -> m ()
forall (m :: * -> *) a. Tracer m a -> a -> m ()
traceWith Tracer m (TraceAddBlockEvent blk)
addBlockTracer (RealPoint blk -> TraceAddBlockEvent blk
forall blk. RealPoint blk -> TraceAddBlockEvent blk
StoreButDontChange RealPoint blk
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 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), HasHeader 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 blk -> ChainSelEnv m blk
    mkChainSelEnv :: ChainAndLedger blk -> ChainSelEnv m blk
mkChainSelEnv ChainAndLedger blk
curChainAndLedger = ChainSelEnv
      { lgrDB :: LgrDB m blk
lgrDB                 = LgrDB m blk
cdbLgrDB
      , 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 blk
curChainAndLedger     = ChainAndLedger 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
      => (ChainHash blk -> Set (HeaderHash blk))
      -> ChainAndLedger blk
         -- ^ The current chain and ledger
      -> LoE (AnchoredFragment (Header blk))
         -- ^ LoE fragment
      -> m ()
    addToCurrentChain :: HasCallStack =>
(ChainHash blk -> Set (HeaderHash blk))
-> ChainAndLedger blk
-> LoE (AnchoredFragment (Header blk))
-> m ()
addToCurrentChain ChainHash blk -> Set (HeaderHash blk)
succsOf ChainAndLedger blk
curChainAndLedger LoE (AnchoredFragment (Header 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.
        NonEmpty (AnchoredFragment (Header blk))
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 (AnchoredFragment (Header blk))
-> m (NonEmpty (AnchoredFragment (Header blk)))
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (NonEmpty (AnchoredFragment (Header blk))
 -> m (NonEmpty (AnchoredFragment (Header blk))))
-> NonEmpty (AnchoredFragment (Header blk))
-> m (NonEmpty (AnchoredFragment (Header blk)))
forall a b. (a -> b) -> a -> b
$ (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 (Header blk)
curHead [Header blk
hdr]) AnchoredFragment (Header blk)
-> [AnchoredFragment (Header blk)]
-> NonEmpty (AnchoredFragment (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 (AnchoredFragment (Header blk)))
 -> Map (HeaderHash blk) (Header blk)
 -> m (NonEmpty (AnchoredFragment (Header blk))))
-> Map (HeaderHash blk) (Header blk)
-> StateT
     (Map (HeaderHash blk) (Header blk))
     m
     (NonEmpty (AnchoredFragment (Header blk)))
-> m (NonEmpty (AnchoredFragment (Header blk)))
forall a b c. (a -> b -> c) -> b -> a -> c
flip StateT
  (Map (HeaderHash blk) (Header blk))
  m
  (NonEmpty (AnchoredFragment (Header blk)))
-> Map (HeaderHash blk) (Header blk)
-> m (NonEmpty (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
   (NonEmpty (AnchoredFragment (Header blk)))
 -> m (NonEmpty (AnchoredFragment (Header blk))))
-> StateT
     (Map (HeaderHash blk) (Header blk))
     m
     (NonEmpty (AnchoredFragment (Header blk)))
-> m (NonEmpty (AnchoredFragment (Header blk)))
forall a b. (a -> b) -> a -> b
$ NonEmpty (NonEmpty (HeaderHash blk))
-> (NonEmpty (HeaderHash blk)
    -> StateT
         (Map (HeaderHash blk) (Header blk))
         m
         (AnchoredFragment (Header blk)))
-> StateT
     (Map (HeaderHash blk) (Header blk))
     m
     (NonEmpty (AnchoredFragment (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
       (AnchoredFragment (Header blk)))
 -> StateT
      (Map (HeaderHash blk) (Header blk))
      m
      (NonEmpty (AnchoredFragment (Header blk))))
-> (NonEmpty (HeaderHash blk)
    -> StateT
         (Map (HeaderHash blk) (Header blk))
         m
         (AnchoredFragment (Header blk)))
-> StateT
     (Map (HeaderHash blk) (Header blk))
     m
     (NonEmpty (AnchoredFragment (Header blk)))
forall a b. (a -> b) -> a -> b
$ \NonEmpty (HeaderHash blk)
hashes -> do
              [Header blk]
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
              AnchoredFragment (Header blk)
-> StateT
     (Map (HeaderHash blk) (Header blk))
     m
     (AnchoredFragment (Header blk))
forall a. a -> StateT (Map (HeaderHash blk) (Header blk)) m a
forall (m :: * -> *) a. Monad m => a -> m a
return (AnchoredFragment (Header blk)
 -> StateT
      (Map (HeaderHash blk) (Header blk))
      m
      (AnchoredFragment (Header blk)))
-> AnchoredFragment (Header blk)
-> StateT
     (Map (HeaderHash blk) (Header blk))
     m
     (AnchoredFragment (Header blk))
forall a b. (a -> b) -> a -> b
$ 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 (Header blk)
curHead (Header blk
hdr Header blk -> [Header blk] -> [Header blk]
forall a. a -> [a] -> [a]
: [Header blk]
hdrs)

        let chainDiffs :: Maybe (NonEmpty (ChainDiff (Header blk)))
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
-> AnchoredFragment (Header blk)
-> AnchoredFragment (Header blk)
-> Bool
forall blk.
(BlockSupportsProtocol blk, HasCallStack) =>
BlockConfig blk
-> AnchoredFragment (Header blk)
-> AnchoredFragment (Header blk)
-> Bool
preferAnchoredCandidate (ChainSelEnv m blk -> BlockConfig blk
forall (m :: * -> *) blk. ChainSelEnv m blk -> BlockConfig blk
bcfg ChainSelEnv m blk
chainSelEnv) 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)
              ([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 (Header blk))
-> ChainAndLedger blk
-> ChainDiff (Header blk)
-> ChainDiff (Header blk)
trimToLoE LoE (AnchoredFragment (Header blk))
loeFrag ChainAndLedger blk
curChainAndLedger)
              ([ChainDiff (Header blk)] -> [ChainDiff (Header blk)])
-> [ChainDiff (Header blk)] -> [ChainDiff (Header blk)]
forall a b. (a -> b) -> a -> b
$ (AnchoredFragment (Header blk) -> ChainDiff (Header blk))
-> [AnchoredFragment (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 AnchoredFragment (Header blk) -> ChainDiff (Header blk)
forall b. AnchoredFragment b -> ChainDiff b
Diff.extend
              ([AnchoredFragment (Header blk)] -> [ChainDiff (Header blk)])
-> [AnchoredFragment (Header blk)] -> [ChainDiff (Header blk)]
forall a b. (a -> b) -> a -> b
$ NonEmpty (AnchoredFragment (Header blk))
-> [AnchoredFragment (Header blk)]
forall a. NonEmpty a -> [a]
NE.toList NonEmpty (AnchoredFragment (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 Maybe (NonEmpty (ChainDiff (Header blk)))
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
-> NonEmpty (ChainDiff (Header blk))
-> m (Maybe (ValidatedChainDiff (Header blk) (LedgerDB' blk)))
forall (m :: * -> *) blk.
(IOLike m, LedgerSupportsProtocol blk,
 BlockSupportsDiffusionPipelining blk, HasCallStack) =>
ChainSelEnv m blk
-> NonEmpty (ChainDiff (Header blk))
-> m (Maybe (ValidatedChainDiff (Header blk) (LedgerDB' blk)))
chainSelection ChainSelEnv m blk
chainSelEnv NonEmpty (ChainDiff (Header blk))
chainDiffs' m (Maybe (ValidatedChainDiff (Header blk) (LedgerDB' blk)))
-> (Maybe (ValidatedChainDiff (Header blk) (LedgerDB' 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) (LedgerDB' blk))
Nothing ->
                () -> m ()
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return ()
              Just ValidatedChainDiff (Header blk) (LedgerDB' blk)
validatedChainDiff ->
                HasCallStack =>
ValidatedChainDiff (Header blk) (LedgerDB' blk)
-> StrictTVar m (StrictMaybe (Header blk))
-> ChainSwitchType
-> m ()
ValidatedChainDiff (Header blk) (LedgerDB' blk)
-> StrictTVar m (StrictMaybe (Header blk))
-> ChainSwitchType
-> m ()
switchTo
                  ValidatedChainDiff (Header blk) (LedgerDB' 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 blk -> ChainSelEnv m blk
mkChainSelEnv ChainAndLedger blk
curChainAndLedger
        curChain :: AnchoredFragment (Header blk)
curChain    = ChainAndLedger blk -> AnchoredFragment (Header blk)
forall b l. ValidatedFragment b l -> AnchoredFragment b
VF.validatedFragment ChainAndLedger blk
curChainAndLedger
        curHead :: Anchor (Header blk)
curHead     = AnchoredFragment (Header blk) -> Anchor (Header blk)
forall v a b. Anchorable v a b => AnchoredSeq v a b -> a
AF.headAnchor AnchoredFragment (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 ::
      LoE (AnchoredFragment (Header blk)) ->
      ChainAndLedger blk ->
      ChainDiff (Header blk) ->
      ChainDiff (Header blk)
    trimToLoE :: LoE (AnchoredFragment (Header blk))
-> ChainAndLedger blk
-> ChainDiff (Header blk)
-> ChainDiff (Header blk)
trimToLoE LoE (AnchoredFragment (Header blk))
LoEDisabled ChainAndLedger blk
_ ChainDiff (Header blk)
diff = ChainDiff (Header blk)
diff
    trimToLoE (LoEEnabled AnchoredFragment (Header blk)
loe) ChainAndLedger blk
curChain ChainDiff (Header blk)
diff =
      case AnchoredFragment (Header blk)
-> ChainDiff (Header blk) -> Maybe (AnchoredFragment (Header blk))
forall b.
HasHeader b =>
AnchoredFragment b -> ChainDiff b -> Maybe (AnchoredFragment b)
Diff.apply (ChainAndLedger blk -> AnchoredFragment (Header blk)
forall b l. ValidatedFragment b l -> AnchoredFragment b
VF.validatedFragment ChainAndLedger blk
curChain) ChainDiff (Header blk)
diff of
        Maybe (AnchoredFragment (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 AnchoredFragment (Header blk)
cand ->
          case AnchoredFragment (Header blk)
-> AnchoredFragment (Header blk)
-> Maybe
     (AnchoredFragment (Header blk), AnchoredFragment (Header blk),
      AnchoredFragment (Header blk), AnchoredFragment (Header 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 AnchoredFragment (Header blk)
cand AnchoredFragment (Header blk)
loe of
            Maybe
  (AnchoredFragment (Header blk), AnchoredFragment (Header blk),
   AnchoredFragment (Header blk), AnchoredFragment (Header 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 (AnchoredFragment (Header blk)
candPrefix, AnchoredFragment (Header blk)
_, AnchoredFragment (Header blk)
candSuffix, AnchoredFragment (Header blk)
loeSuffix) ->
              let trimmedCandSuffix :: AnchoredFragment (Header blk)
trimmedCandSuffix = Int
-> AnchoredFragment (Header blk) -> AnchoredFragment (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
k) AnchoredFragment (Header blk)
candSuffix
                  trimmedCand :: AnchoredFragment (Header blk)
trimmedCand =
                    if AnchoredFragment (Header blk) -> Bool
forall v a b. AnchoredSeq v a b -> Bool
AF.null AnchoredFragment (Header blk)
loeSuffix
                      then Maybe (AnchoredFragment (Header blk))
-> AnchoredFragment (Header blk)
forall a. HasCallStack => Maybe a -> a
fromJust (Maybe (AnchoredFragment (Header blk))
 -> AnchoredFragment (Header blk))
-> Maybe (AnchoredFragment (Header blk))
-> AnchoredFragment (Header blk)
forall a b. (a -> b) -> a -> b
$ AnchoredFragment (Header blk)
-> AnchoredFragment (Header blk)
-> Maybe (AnchoredFragment (Header blk))
forall block.
HasHeader block =>
AnchoredFragment block
-> AnchoredFragment block -> Maybe (AnchoredFragment block)
AF.join AnchoredFragment (Header blk)
candPrefix AnchoredFragment (Header blk)
trimmedCandSuffix
                      else AnchoredFragment (Header blk)
candPrefix
              in AnchoredFragment (Header blk)
-> 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 (ChainAndLedger blk -> AnchoredFragment (Header blk)
forall b l. ValidatedFragment b l -> AnchoredFragment b
VF.validatedFragment ChainAndLedger blk
curChain) AnchoredFragment (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
      => (ChainHash blk -> Set (HeaderHash blk))
      -> LookupBlockInfo blk
      -> ChainAndLedger blk
         -- ^ The current chain (anchored at @i@) and ledger
      -> LoE (AnchoredFragment (Header blk))
         -- ^ LoE fragment
      -> ChainDiff (HeaderFields blk)
         -- ^ Header fields for @(x,b]@
      -> m ()
    switchToAFork :: HasCallStack =>
(ChainHash blk -> Set (HeaderHash blk))
-> (HeaderHash blk -> Maybe (BlockInfo blk))
-> ChainAndLedger blk
-> LoE (AnchoredFragment (Header blk))
-> ChainDiff (HeaderFields blk)
-> m ()
switchToAFork ChainHash blk -> Set (HeaderHash blk)
succsOf HeaderHash blk -> Maybe (BlockInfo blk)
lookupBlockInfo ChainAndLedger blk
curChainAndLedger LoE (AnchoredFragment (Header 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
        [ChainDiff (Header blk)]
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
-> AnchoredFragment (Header blk)
-> AnchoredFragment (Header blk)
-> Bool
forall blk.
(BlockSupportsProtocol blk, HasCallStack) =>
BlockConfig blk
-> AnchoredFragment (Header blk)
-> AnchoredFragment (Header blk)
-> Bool
preferAnchoredCandidate (ChainSelEnv m blk -> BlockConfig blk
forall (m :: * -> *) blk. ChainSelEnv m blk -> BlockConfig blk
bcfg ChainSelEnv m blk
chainSelEnv) 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
                  )
              )
            -- 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 (Header blk))
-> ChainAndLedger blk
-> ChainDiff (Header blk)
-> ChainDiff (Header blk)
trimToLoE LoE (AnchoredFragment (Header blk))
loeFrag ChainAndLedger 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 [ChainDiff (Header blk)]
-> Maybe (NonEmpty (ChainDiff (Header blk)))
forall a. [a] -> Maybe (NonEmpty a)
NE.nonEmpty [ChainDiff (Header blk)]
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
-> NonEmpty (ChainDiff (Header blk))
-> m (Maybe (ValidatedChainDiff (Header blk) (LedgerDB' blk)))
forall (m :: * -> *) blk.
(IOLike m, LedgerSupportsProtocol blk,
 BlockSupportsDiffusionPipelining blk, HasCallStack) =>
ChainSelEnv m blk
-> NonEmpty (ChainDiff (Header blk))
-> m (Maybe (ValidatedChainDiff (Header blk) (LedgerDB' blk)))
chainSelection ChainSelEnv m blk
chainSelEnv NonEmpty (ChainDiff (Header blk))
chainDiffs' m (Maybe (ValidatedChainDiff (Header blk) (LedgerDB' blk)))
-> (Maybe (ValidatedChainDiff (Header blk) (LedgerDB' 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) (LedgerDB' blk))
Nothing                 ->
                () -> m ()
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return ()
              Just ValidatedChainDiff (Header blk) (LedgerDB' blk)
validatedChainDiff ->
                HasCallStack =>
ValidatedChainDiff (Header blk) (LedgerDB' blk)
-> StrictTVar m (StrictMaybe (Header blk))
-> ChainSwitchType
-> m ()
ValidatedChainDiff (Header blk) (LedgerDB' blk)
-> StrictTVar m (StrictMaybe (Header blk))
-> ChainSwitchType
-> m ()
switchTo
                  ValidatedChainDiff (Header blk) (LedgerDB' 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 blk -> ChainSelEnv m blk
mkChainSelEnv ChainAndLedger blk
curChainAndLedger
        curChain :: AnchoredFragment (Header blk)
curChain    = ChainAndLedger blk -> AnchoredFragment (Header blk)
forall b l. ValidatedFragment b l -> AnchoredFragment b
VF.validatedFragment ChainAndLedger blk
curChainAndLedger

    mkSelectionChangedInfo ::
         AnchoredFragment (Header blk) -- ^ old chain
      -> AnchoredFragment (Header blk) -- ^ new chain
      -> LedgerDB' blk                 -- ^ new LedgerDB
      -> SelectionChangedInfo blk
    mkSelectionChangedInfo :: AnchoredFragment (Header blk)
-> AnchoredFragment (Header blk)
-> LedgerDB' blk
-> SelectionChangedInfo blk
mkSelectionChangedInfo AnchoredFragment (Header blk)
oldChain AnchoredFragment (Header blk)
newChain LedgerDB' blk
newLedgerDB =
        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 (AnchoredFragment (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 AnchoredFragment (Header blk)
oldChain)
          }
      where
        cfg :: TopLevelConfig blk
        cfg :: TopLevelConfig blk
cfg = TopLevelConfig blk
cdbTopLevelConfig

        ledger :: LedgerState blk
        ledger :: LedgerState blk
ledger = ExtLedgerState blk -> LedgerState blk
forall blk. ExtLedgerState blk -> LedgerState blk
ledgerState (LedgerDB' blk -> ExtLedgerState blk
forall l. GetTip l => LedgerDB l -> l
LgrDB.ledgerDbCurrent LedgerDB' blk
newLedgerDB)

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

        (RealPoint (Header blk)
tipPoint, (EpochNo
tipEpoch, Word64
tipSlotInEpoch), SelectView (BlockProtocol blk)
newTipSelectView) =
          case AnchoredFragment (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 AnchoredFragment (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
    -- 'LgrDB.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) (LedgerDB' blk)
         -- ^ Chain and ledger to switch to
      -> StrictTVar m (StrictMaybe (Header blk))
         -- ^ Tentative header
      -> ChainSwitchType
      -> m ()
    switchTo :: HasCallStack =>
ValidatedChainDiff (Header blk) (LedgerDB' blk)
-> StrictTVar m (StrictMaybe (Header blk))
-> ChainSwitchType
-> m ()
switchTo ValidatedChainDiff (Header blk) (LedgerDB' 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
$ AnchoredFragment (Header blk) -> Point (Header blk)
forall block.
HasHeader block =>
AnchoredFragment block -> Point block
AF.headPoint
          (AnchoredFragment (Header blk) -> Point (Header blk))
-> AnchoredFragment (Header blk) -> Point (Header blk)
forall a b. (a -> b) -> a -> b
$ ChainDiff (Header blk) -> AnchoredFragment (Header blk)
forall b. ChainDiff b -> AnchoredFragment b
getSuffix
          (ChainDiff (Header blk) -> AnchoredFragment (Header blk))
-> ChainDiff (Header blk) -> AnchoredFragment (Header blk)
forall a b. (a -> b) -> a -> b
$ ValidatedChainDiff (Header blk) (LedgerDB' blk)
-> ChainDiff (Header blk)
forall b l. ValidatedChainDiff b l -> ChainDiff b
getChainDiff ValidatedChainDiff (Header blk) (LedgerDB' blk)
vChainDiff
        (AnchoredFragment (Header blk)
curChain, AnchoredFragment (Header blk)
newChain, [LedgerEvent blk]
events, StrictMaybe (Header blk)
prevTentativeHeader) <- STM
  m
  (AnchoredFragment (Header blk), AnchoredFragment (Header blk),
   [LedgerEvent blk], StrictMaybe (Header blk))
-> m (AnchoredFragment (Header blk), AnchoredFragment (Header blk),
      [LedgerEvent blk], StrictMaybe (Header blk))
forall a. HasCallStack => STM m a -> m a
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
STM m a -> m a
atomically (STM
   m
   (AnchoredFragment (Header blk), AnchoredFragment (Header blk),
    [LedgerEvent blk], StrictMaybe (Header blk))
 -> m (AnchoredFragment (Header blk), AnchoredFragment (Header blk),
       [LedgerEvent blk], StrictMaybe (Header blk)))
-> STM
     m
     (AnchoredFragment (Header blk), AnchoredFragment (Header blk),
      [LedgerEvent blk], StrictMaybe (Header blk))
-> m (AnchoredFragment (Header blk), AnchoredFragment (Header blk),
      [LedgerEvent blk], StrictMaybe (Header blk))
forall a b. (a -> b) -> a -> b
$ do
          AnchoredFragment (Header blk)
curChain  <- StrictTVar m (AnchoredFragment (Header blk))
-> STM m (AnchoredFragment (Header blk))
forall (m :: * -> *) a. MonadSTM m => StrictTVar m a -> STM m a
readTVar         StrictTVar m (AnchoredFragment (Header blk))
cdbChain -- Not Query.getCurrentChain!
          LedgerDB' blk
curLedger <- LgrDB m blk -> STM m (LedgerDB' blk)
forall (m :: * -> *) blk.
IOLike m =>
LgrDB m blk -> STM m (LedgerDB' blk)
LgrDB.getCurrent LgrDB m blk
cdbLgrDB
          case 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 ChainDiff (Header blk)
chainDiff of
            -- Impossible, as described in the docstring
            Maybe (AnchoredFragment (Header blk))
Nothing       ->
              [Char]
-> STM
     m
     (AnchoredFragment (Header blk), AnchoredFragment (Header blk),
      [LedgerEvent blk], StrictMaybe (Header blk))
forall a. HasCallStack => [Char] -> a
error [Char]
"chainDiff doesn't fit onto current chain"
            Just AnchoredFragment (Header blk)
newChain -> do
              StrictTVar m (AnchoredFragment (Header blk))
-> AnchoredFragment (Header blk) -> STM m ()
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
StrictTVar m a -> a -> STM m ()
writeTVar StrictTVar m (AnchoredFragment (Header blk))
cdbChain AnchoredFragment (Header blk)
newChain
              LgrDB m blk -> LedgerDB' blk -> STM m ()
forall (m :: * -> *) blk.
IOLike m =>
LgrDB m blk -> LedgerDB' blk -> STM m ()
LgrDB.setCurrent LgrDB m blk
cdbLgrDB LedgerDB' blk
newLedger

              -- Inspect the new ledger for potential problems
              let events :: [LedgerEvent blk]
                  events :: [LedgerEvent blk]
events = TopLevelConfig blk
-> LedgerState blk -> LedgerState blk -> [LedgerEvent blk]
forall blk.
InspectLedger blk =>
TopLevelConfig blk
-> LedgerState blk -> LedgerState blk -> [LedgerEvent blk]
inspectLedger
                             TopLevelConfig blk
cdbTopLevelConfig
                             (ExtLedgerState blk -> LedgerState blk
forall blk. ExtLedgerState blk -> LedgerState blk
ledgerState (ExtLedgerState blk -> LedgerState blk)
-> ExtLedgerState blk -> LedgerState blk
forall a b. (a -> b) -> a -> b
$ LedgerDB' blk -> ExtLedgerState blk
forall l. GetTip l => LedgerDB l -> l
LgrDB.ledgerDbCurrent LedgerDB' blk
curLedger)
                             (ExtLedgerState blk -> LedgerState blk
forall blk. ExtLedgerState blk -> LedgerState blk
ledgerState (ExtLedgerState blk -> LedgerState blk)
-> ExtLedgerState blk -> LedgerState blk
forall a b. (a -> b) -> a -> b
$ LedgerDB' blk -> ExtLedgerState blk
forall l. GetTip l => LedgerDB l -> l
LgrDB.ledgerDbCurrent LedgerDB' blk
newLedger)

              -- Clear the tentative header
              StrictMaybe (Header blk)
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
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
                  [FollowerHandle m blk]
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
                  [FollowerHandle m blk]
-> (FollowerHandle m blk -> STM m ()) -> STM m ()
forall (t :: * -> *) (m :: * -> *) a b.
(Foldable t, Monad m) =>
t a -> (a -> m b) -> m ()
forM_ [FollowerHandle m blk]
followerHandles ((FollowerHandle m blk -> STM m ()) -> STM m ())
-> (FollowerHandle m blk -> STM m ()) -> STM m ()
forall a b. (a -> b) -> a -> b
$ AnchoredFragment (Header blk)
-> AnchoredFragment (Header blk)
-> Point blk
-> FollowerHandle m blk
-> STM m ()
forall {blk} {block} {m :: * -> *}.
(HeaderHash blk ~ HeaderHash block, HasHeader block, HasHeader blk,
 HasHeader (Header blk)) =>
AnchoredFragment (Header blk)
-> AnchoredFragment block
-> Point blk
-> FollowerHandle m blk
-> STM m ()
switchFollowerToFork AnchoredFragment (Header blk)
curChain AnchoredFragment (Header blk)
newChain Point blk
ipoint

              (AnchoredFragment (Header blk), AnchoredFragment (Header blk),
 [LedgerEvent blk], StrictMaybe (Header blk))
-> STM
     m
     (AnchoredFragment (Header blk), AnchoredFragment (Header blk),
      [LedgerEvent blk], StrictMaybe (Header blk))
forall a. a -> STM m a
forall (m :: * -> *) a. Monad m => a -> m a
return (AnchoredFragment (Header blk)
curChain, AnchoredFragment (Header blk)
newChain, [LedgerEvent blk]
events, StrictMaybe (Header blk)
prevTentativeHeader)

        let mkTraceEvent :: [LedgerEvent blk]
-> SelectionChangedInfo blk
-> AnchoredFragment (Header blk)
-> AnchoredFragment (Header blk)
-> TraceAddBlockEvent blk
mkTraceEvent = case ChainSwitchType
chainSwitchType of
              ChainSwitchType
AddingBlocks     -> [LedgerEvent blk]
-> SelectionChangedInfo blk
-> AnchoredFragment (Header blk)
-> AnchoredFragment (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
-> AnchoredFragment (Header blk)
-> AnchoredFragment (Header blk)
-> TraceAddBlockEvent blk
forall blk.
[LedgerEvent blk]
-> SelectionChangedInfo blk
-> AnchoredFragment (Header blk)
-> AnchoredFragment (Header blk)
-> TraceAddBlockEvent blk
SwitchedToAFork
            selChangedInfo :: SelectionChangedInfo blk
selChangedInfo = AnchoredFragment (Header blk)
-> AnchoredFragment (Header blk)
-> LedgerDB' blk
-> SelectionChangedInfo blk
mkSelectionChangedInfo AnchoredFragment (Header blk)
curChain AnchoredFragment (Header blk)
newChain LedgerDB' blk
newLedger
        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
$
          [LedgerEvent blk]
-> SelectionChangedInfo blk
-> AnchoredFragment (Header blk)
-> AnchoredFragment (Header blk)
-> TraceAddBlockEvent blk
mkTraceEvent [LedgerEvent blk]
events SelectionChangedInfo blk
selChangedInfo AnchoredFragment (Header blk)
curChain AnchoredFragment (Header blk)
newChain
        Maybe (Header blk) -> (Header blk -> m ()) -> m ()
forall (f :: * -> *) a.
Applicative f =>
Maybe a -> (a -> f ()) -> f ()
whenJust (StrictMaybe (Header blk) -> Maybe (Header blk)
forall a. StrictMaybe a -> Maybe a
strictMaybeToMaybe StrictMaybe (Header blk)
prevTentativeHeader) ((Header blk -> m ()) -> m ()) -> (Header blk -> m ()) -> m ()
forall a b. (a -> b) -> a -> b
$ Tracer m (Header blk) -> Header blk -> m ()
forall (m :: * -> *) a. Tracer m a -> a -> m ()
traceWith (Tracer m (Header blk) -> Header blk -> m ())
-> Tracer m (Header blk) -> Header blk -> m ()
forall a b. (a -> b) -> a -> b
$
          TracePipeliningEvent blk -> TraceAddBlockEvent blk
forall blk. TracePipeliningEvent blk -> TraceAddBlockEvent blk
PipeliningEvent (TracePipeliningEvent blk -> TraceAddBlockEvent blk)
-> (Header blk -> TracePipeliningEvent blk)
-> Header blk
-> TraceAddBlockEvent blk
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Header blk -> TracePipeliningEvent blk
forall blk. Header blk -> TracePipeliningEvent blk
OutdatedTentativeHeader (Header blk -> TraceAddBlockEvent blk)
-> Tracer m (TraceAddBlockEvent blk) -> Tracer m (Header blk)
forall (f :: * -> *) a b. Contravariant f => (a -> b) -> f b -> f a
>$< Tracer m (TraceAddBlockEvent blk)
addBlockTracer

      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 LedgerDB' blk
newLedger = ValidatedChainDiff (Header blk) (LedgerDB' 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
      Header blk
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
      (Map (HeaderHash blk) (Header blk)
 -> Map (HeaderHash blk) (Header blk))
-> StateT (Map (HeaderHash blk) (Header blk)) m ()
forall (m :: * -> *) s. Monad m => (s -> s) -> StateT s m ()
modify (HeaderHash blk
-> Header blk
-> Map (HeaderHash blk) (Header blk)
-> Map (HeaderHash blk) (Header blk)
forall k a. Ord k => k -> a -> Map k a -> Map k a
Map.insert HeaderHash blk
hash 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

-- | Environment used by 'chainSelection' and related functions.
data ChainSelEnv m blk = ChainSelEnv
    { forall (m :: * -> *) blk. ChainSelEnv m blk -> LgrDB m blk
lgrDB                 :: LgrDB 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 blk
curChainAndLedger     :: ChainAndLedger 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
  -> NonEmpty (ChainDiff (Header blk))
  -> m (Maybe (ValidatedChainDiff (Header blk) (LedgerDB' 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
-> NonEmpty (ChainDiff (Header blk))
-> m (Maybe (ValidatedChainDiff (Header blk) (LedgerDB' blk)))
chainSelection ChainSelEnv m blk
chainSelEnv NonEmpty (ChainDiff (Header blk))
chainDiffs =
    Bool
-> m (Maybe (ValidatedChainDiff (Header blk) (LedgerDB' blk)))
-> m (Maybe (ValidatedChainDiff (Header blk) (LedgerDB' 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.
(BlockSupportsProtocol blk, HasCallStack) =>
BlockConfig blk
-> AnchoredFragment (Header blk)
-> AnchoredFragment (Header 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) (LedgerDB' blk)))
 -> m (Maybe (ValidatedChainDiff (Header blk) (LedgerDB' blk))))
-> m (Maybe (ValidatedChainDiff (Header blk) (LedgerDB' blk)))
-> m (Maybe (ValidatedChainDiff (Header blk) (LedgerDB' blk)))
forall a b. (a -> b) -> a -> b
$
    Bool
-> m (Maybe (ValidatedChainDiff (Header blk) (LedgerDB' blk)))
-> m (Maybe (ValidatedChainDiff (Header blk) (LedgerDB' 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) (LedgerDB' blk)))
 -> m (Maybe (ValidatedChainDiff (Header blk) (LedgerDB' blk))))
-> m (Maybe (ValidatedChainDiff (Header blk) (LedgerDB' blk)))
-> m (Maybe (ValidatedChainDiff (Header blk) (LedgerDB' blk)))
forall a b. (a -> b) -> a -> b
$
    [ChainDiff (Header blk)]
-> m (Maybe (ValidatedChainDiff (Header blk) (LedgerDB' 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 blk
LgrDB m blk
lgrDB :: forall (m :: * -> *) blk. ChainSelEnv m blk -> LgrDB 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 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 :: LgrDB 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 blk
punish :: Maybe (RealPoint blk, InvalidBlockPunishment m)
..} = ChainSelEnv m blk
chainSelEnv

    curChain :: AnchoredFragment (Header blk)
curChain = ChainAndLedger blk -> AnchoredFragment (Header blk)
forall b l. ValidatedFragment b l -> AnchoredFragment b
VF.validatedFragment ChainAndLedger 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.
(BlockSupportsProtocol blk, HasCallStack) =>
BlockConfig blk
-> AnchoredFragment (Header blk)
-> AnchoredFragment (Header 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) (LedgerDB' blk)))
    go :: [ChainDiff (Header blk)]
-> m (Maybe (ValidatedChainDiff (Header blk) (LedgerDB' blk)))
go []            = Maybe (ValidatedChainDiff (Header blk) (LedgerDB' blk))
-> m (Maybe (ValidatedChainDiff (Header blk) (LedgerDB' blk)))
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return Maybe (ValidatedChainDiff (Header blk) (LedgerDB' blk))
forall a. Maybe a
Nothing
    go (ChainDiff (Header blk)
candidate:[ChainDiff (Header blk)]
candidates0) = do
        Maybe (Header blk, TentativeHeaderState blk)
mTentativeHeader <- m (Maybe (Header blk, TentativeHeaderState blk))
setTentativeHeader
        ChainSelEnv m blk
-> ChainDiff (Header blk) -> m (ValidationResult blk)
forall (m :: * -> *) blk.
(IOLike m, LedgerSupportsProtocol blk, HasCallStack) =>
ChainSelEnv m blk
-> ChainDiff (Header blk) -> m (ValidationResult blk)
validateCandidate ChainSelEnv m blk
chainSelEnv ChainDiff (Header blk)
candidate m (ValidationResult blk)
-> (ValidationResult blk
    -> m (Maybe (ValidatedChainDiff (Header blk) (LedgerDB' blk))))
-> m (Maybe (ValidatedChainDiff (Header blk) (LedgerDB' 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
          ValidationResult 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) (LedgerDB' blk)))
-> m (Maybe (ValidatedChainDiff (Header blk) (LedgerDB' 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) (LedgerDB' blk)))
 -> m (Maybe (ValidatedChainDiff (Header blk) (LedgerDB' blk))))
-> m (Maybe (ValidatedChainDiff (Header blk) (LedgerDB' blk)))
-> m (Maybe (ValidatedChainDiff (Header blk) (LedgerDB' blk)))
forall a b. (a -> b) -> a -> b
$ do
              [ChainDiff (Header blk)]
candidates1 <- [ChainDiff (Header blk)] -> m [ChainDiff (Header blk)]
truncateRejectedBlocks [ChainDiff (Header blk)]
candidates0
              [ChainDiff (Header blk)]
-> m (Maybe (ValidatedChainDiff (Header blk) (LedgerDB' blk)))
go ([ChainDiff (Header blk)] -> [ChainDiff (Header blk)]
sortCandidates [ChainDiff (Header blk)]
candidates1)
          FullyValid validatedCandidate :: ValidatedChainDiff (Header blk) (LedgerDB' blk)
validatedCandidate@(ValidatedChainDiff ChainDiff (Header blk)
candidate' LedgerDB' blk
_) ->
            -- The entire candidate is valid
            Bool
-> m (Maybe (ValidatedChainDiff (Header blk) (LedgerDB' blk)))
-> m (Maybe (ValidatedChainDiff (Header blk) (LedgerDB' 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) (LedgerDB' blk)))
 -> m (Maybe (ValidatedChainDiff (Header blk) (LedgerDB' blk))))
-> m (Maybe (ValidatedChainDiff (Header blk) (LedgerDB' blk)))
-> m (Maybe (ValidatedChainDiff (Header blk) (LedgerDB' blk)))
forall a b. (a -> b) -> a -> b
$
            Maybe (ValidatedChainDiff (Header blk) (LedgerDB' blk))
-> m (Maybe (ValidatedChainDiff (Header blk) (LedgerDB' blk)))
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (Maybe (ValidatedChainDiff (Header blk) (LedgerDB' blk))
 -> m (Maybe (ValidatedChainDiff (Header blk) (LedgerDB' blk))))
-> Maybe (ValidatedChainDiff (Header blk) (LedgerDB' blk))
-> m (Maybe (ValidatedChainDiff (Header blk) (LedgerDB' blk)))
forall a b. (a -> b) -> a -> b
$ ValidatedChainDiff (Header blk) (LedgerDB' blk)
-> Maybe (ValidatedChainDiff (Header blk) (LedgerDB' blk))
forall a. a -> Maybe a
Just ValidatedChainDiff (Header blk) (LedgerDB' 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.
            [ChainDiff (Header blk)]
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 :: [ChainDiff (Header blk)]
candidates2
                  | BlockConfig blk
-> AnchoredFragment (Header blk)
-> AnchoredFragment (Header blk)
-> Bool
forall blk.
(BlockSupportsProtocol blk, HasCallStack) =>
BlockConfig blk
-> AnchoredFragment (Header blk)
-> AnchoredFragment (Header 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
            [ChainDiff (Header blk)]
-> m (Maybe (ValidatedChainDiff (Header blk) (LedgerDB' blk)))
go ([ChainDiff (Header blk)] -> [ChainDiff (Header blk)]
sortCandidates [ChainDiff (Header blk)]
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
            Maybe (Header blk, TentativeHeaderState blk)
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
            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)
pipeliningResult (((Header blk, TentativeHeaderState blk) -> m ()) -> m ())
-> ((Header blk, TentativeHeaderState blk) -> m ()) -> m ()
forall a b. (a -> b) -> a -> b
$ \(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
            Maybe (Header blk, TentativeHeaderState blk)
-> m (Maybe (Header blk, TentativeHeaderState blk))
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure Maybe (Header blk, TentativeHeaderState blk)
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
      WithFingerprint (InvalidBlocks blk)
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 -> Bool
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)
      [ChainDiff (Header blk)] -> m [ChainDiff (Header blk)]
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return ([ChainDiff (Header blk)] -> m [ChainDiff (Header blk)])
-> [ChainDiff (Header blk)] -> m [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
-> AnchoredFragment (Header blk)
-> AnchoredFragment (Header blk)
-> Bool
forall blk.
(BlockSupportsProtocol blk, HasCallStack) =>
BlockConfig blk
-> AnchoredFragment (Header blk)
-> AnchoredFragment (Header 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)
             ([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]
map ((Header blk -> Bool)
-> ChainDiff (Header blk) -> ChainDiff (Header blk)
forall b. HasHeader b => (b -> Bool) -> ChainDiff b -> ChainDiff b
Diff.takeWhileOldest (Bool -> Bool
not (Bool -> Bool) -> (Header blk -> Bool) -> Header blk -> Bool
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Header blk -> Bool
isRejected)) [ChainDiff (Header blk)]
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 blk =
      -- | The entire candidate fragment was valid.
      FullyValid (ValidatedChainDiff (Header blk) (LedgerDB' 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).
ledgerValidateCandidate ::
     forall m blk.
     ( IOLike m
     , LedgerSupportsProtocol blk
     , HasCallStack
     )
  => ChainSelEnv m blk
  -> ChainDiff (Header blk)
  -> m (ValidatedChainDiff (Header blk) (LedgerDB' blk))
ledgerValidateCandidate :: forall (m :: * -> *) blk.
(IOLike m, LedgerSupportsProtocol blk, HasCallStack) =>
ChainSelEnv m blk
-> ChainDiff (Header blk)
-> m (ValidatedChainDiff (Header blk) (LedgerDB' blk))
ledgerValidateCandidate ChainSelEnv m blk
chainSelEnv chainDiff :: ChainDiff (Header blk)
chainDiff@(ChainDiff Word64
rollback AnchoredFragment (Header blk)
suffix) =
    LgrDB m blk
-> LedgerDB' blk
-> BlockCache blk
-> Word64
-> (UpdateLedgerDbTraceEvent blk -> m ())
-> [Header blk]
-> m (ValidateResult blk)
forall (m :: * -> *) blk.
(IOLike m, LedgerSupportsProtocol blk, HasCallStack) =>
LgrDB m blk
-> LedgerDB' blk
-> BlockCache blk
-> Word64
-> (UpdateLedgerDbTraceEvent blk -> m ())
-> [Header blk]
-> m (ValidateResult blk)
LgrDB.validate LgrDB m blk
lgrDB LedgerDB' blk
curLedger BlockCache blk
blockCache Word64
rollback UpdateLedgerDbTraceEvent blk -> m ()
traceUpdate [Header blk]
newBlocks m (ValidateResult blk)
-> (ValidateResult blk
    -> m (ValidatedChainDiff (Header blk) (LedgerDB' blk)))
-> m (ValidatedChainDiff (Header blk) (LedgerDB' 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
      LgrDB.ValidateExceededRollBack {} ->
        -- Impossible: we asked the LgrDB 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) (LedgerDB' blk))
forall a. HasCallStack => [Char] -> a
error [Char]
"found candidate requiring rolling back past the immutable tip"

      LgrDB.ValidateLedgerError (LgrDB.AnnLedgerError LedgerDB' blk
ledger' RealPoint blk
pt LedgerErr (ExtLedgerState blk)
e) -> do
        let lastValid :: Point blk
lastValid  = LedgerDB' blk -> Point blk
forall blk. UpdateLedger blk => LedgerDB' blk -> Point blk
LgrDB.currentPoint LedgerDB' blk
ledger'
            chainDiff' :: ChainDiff (Header blk)
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
        Tracer m (TraceValidationEvent blk)
-> TraceValidationEvent blk -> m ()
forall (m :: * -> *) a. Tracer m a -> a -> m ()
traceWith Tracer m (TraceValidationEvent blk)
validationTracer (ExtValidationError blk -> RealPoint blk -> TraceValidationEvent blk
forall blk.
ExtValidationError blk -> RealPoint blk -> TraceValidationEvent blk
InvalidBlock LedgerErr (ExtLedgerState blk)
ExtValidationError blk
e RealPoint blk
pt)
        ExtValidationError blk -> RealPoint blk -> m ()
addInvalidBlock LedgerErr (ExtLedgerState blk)
ExtValidationError blk
e RealPoint blk
pt
        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 (ChainDiff (Header blk) -> AnchoredFragment (Header blk)
forall b. ChainDiff b -> AnchoredFragment b
Diff.getSuffix ChainDiff (Header blk)
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.
        Maybe (RealPoint blk, InvalidBlockPunishment m)
-> ((RealPoint blk, InvalidBlockPunishment m) -> m ()) -> m ()
forall (f :: * -> *) a.
Applicative f =>
Maybe a -> (a -> f ()) -> f ()
whenJust Maybe (RealPoint blk, InvalidBlockPunishment m)
punish (((RealPoint blk, InvalidBlockPunishment m) -> m ()) -> m ())
-> ((RealPoint blk, InvalidBlockPunishment m) -> m ()) -> m ()
forall a b. (a -> b) -> a -> b
$ \(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.)

        ValidatedChainDiff (Header blk) (LedgerDB' blk)
-> m (ValidatedChainDiff (Header blk) (LedgerDB' blk))
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (ValidatedChainDiff (Header blk) (LedgerDB' blk)
 -> m (ValidatedChainDiff (Header blk) (LedgerDB' blk)))
-> ValidatedChainDiff (Header blk) (LedgerDB' blk)
-> m (ValidatedChainDiff (Header blk) (LedgerDB' blk))
forall a b. (a -> b) -> a -> b
$ ChainDiff (Header blk)
-> LedgerDB' blk -> ValidatedChainDiff (Header blk) (LedgerDB' blk)
forall b l.
(GetTip l, HasHeader b, HeaderHash l ~ HeaderHash b,
 HasCallStack) =>
ChainDiff b -> l -> ValidatedChainDiff b l
ValidatedDiff.new ChainDiff (Header blk)
chainDiff' LedgerDB' blk
ledger'

      LgrDB.ValidateSuccessful LedgerDB' 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)
        ValidatedChainDiff (Header blk) (LedgerDB' blk)
-> m (ValidatedChainDiff (Header blk) (LedgerDB' blk))
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (ValidatedChainDiff (Header blk) (LedgerDB' blk)
 -> m (ValidatedChainDiff (Header blk) (LedgerDB' blk)))
-> ValidatedChainDiff (Header blk) (LedgerDB' blk)
-> m (ValidatedChainDiff (Header blk) (LedgerDB' blk))
forall a b. (a -> b) -> a -> b
$ ChainDiff (Header blk)
-> LedgerDB' blk -> ValidatedChainDiff (Header blk) (LedgerDB' blk)
forall b l.
(GetTip l, HasHeader b, HeaderHash l ~ HeaderHash b,
 HasCallStack) =>
ChainDiff b -> l -> ValidatedChainDiff b l
ValidatedDiff.new ChainDiff (Header blk)
chainDiff LedgerDB' blk
ledger'
  where
    ChainSelEnv {
        LgrDB m blk
lgrDB :: forall (m :: * -> *) blk. ChainSelEnv m blk -> LgrDB m blk
lgrDB :: LgrDB m blk
lgrDB
      , Tracer m (TraceValidationEvent blk)
validationTracer :: forall (m :: * -> *) blk.
ChainSelEnv m blk -> Tracer m (TraceValidationEvent blk)
validationTracer :: Tracer m (TraceValidationEvent blk)
validationTracer
      , ChainAndLedger blk
curChainAndLedger :: forall (m :: * -> *) blk. ChainSelEnv m blk -> ChainAndLedger blk
curChainAndLedger :: ChainAndLedger blk
curChainAndLedger
      , 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 :: UpdateLedgerDbTraceEvent blk -> m ()
traceUpdate = Tracer m (UpdateLedgerDbTraceEvent blk)
-> UpdateLedgerDbTraceEvent blk -> m ()
forall (m :: * -> *) a. Tracer m a -> a -> m ()
traceWith (Tracer m (UpdateLedgerDbTraceEvent blk)
 -> UpdateLedgerDbTraceEvent blk -> m ())
-> Tracer m (UpdateLedgerDbTraceEvent blk)
-> UpdateLedgerDbTraceEvent blk
-> m ()
forall a b. (a -> b) -> a -> b
$ UpdateLedgerDbTraceEvent blk -> TraceValidationEvent blk
forall blk.
UpdateLedgerDbTraceEvent blk -> TraceValidationEvent blk
UpdateLedgerDbTraceEvent (UpdateLedgerDbTraceEvent blk -> TraceValidationEvent blk)
-> Tracer m (TraceValidationEvent blk)
-> Tracer m (UpdateLedgerDbTraceEvent blk)
forall (f :: * -> *) a b. Contravariant f => (a -> b) -> f b -> f a
>$< Tracer m (TraceValidationEvent blk)
validationTracer

    curLedger :: LedgerDB' blk
    curLedger :: LedgerDB' blk
curLedger = ChainAndLedger blk -> LedgerDB' blk
forall b l. ValidatedFragment b l -> l
VF.validatedLedger ChainAndLedger blk
curChainAndLedger

    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
  -> ChainDiff (Header blk)
  -> m (ValidationResult blk)
validateCandidate :: forall (m :: * -> *) blk.
(IOLike m, LedgerSupportsProtocol blk, HasCallStack) =>
ChainSelEnv m blk
-> ChainDiff (Header blk) -> m (ValidationResult blk)
validateCandidate ChainSelEnv m blk
chainSelEnv ChainDiff (Header blk)
chainDiff =
    ChainSelEnv m blk
-> ChainDiff (Header blk)
-> m (ValidatedChainDiff (Header blk) (LedgerDB' blk))
forall (m :: * -> *) blk.
(IOLike m, LedgerSupportsProtocol blk, HasCallStack) =>
ChainSelEnv m blk
-> ChainDiff (Header blk)
-> m (ValidatedChainDiff (Header blk) (LedgerDB' blk))
ledgerValidateCandidate ChainSelEnv m blk
chainSelEnv ChainDiff (Header blk)
chainDiff m (ValidatedChainDiff (Header blk) (LedgerDB' blk))
-> (ValidatedChainDiff (Header blk) (LedgerDB' blk)
    -> m (ValidationResult blk))
-> m (ValidationResult 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) (LedgerDB' blk)
validatedChainDiff
        | ValidatedChainDiff (Header blk) (LedgerDB' blk) -> Bool
forall b l. HasHeader b => ValidatedChainDiff b l -> Bool
ValidatedDiff.rollbackExceedsSuffix ValidatedChainDiff (Header blk) (LedgerDB' blk)
validatedChainDiff
        -> ValidationResult blk -> m (ValidationResult blk)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return ValidationResult blk
forall blk. ValidationResult 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 blk -> m (ValidationResult blk)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (ValidationResult blk -> m (ValidationResult blk))
-> ValidationResult blk -> m (ValidationResult blk)
forall a b. (a -> b) -> a -> b
$ ValidatedChainDiff (Header blk) (LedgerDB' blk)
-> ValidationResult blk
forall blk.
ValidatedChainDiff (Header blk) (LedgerDB' blk)
-> ValidationResult blk
FullyValid ValidatedChainDiff (Header blk) (LedgerDB' 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 blk -> m (ValidationResult blk)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (ValidationResult blk -> m (ValidationResult blk))
-> ValidationResult blk -> m (ValidationResult blk)
forall a b. (a -> b) -> a -> b
$ ChainDiff (Header blk) -> ValidationResult blk
forall blk. ChainDiff (Header blk) -> ValidationResult blk
ValidPrefix ChainDiff (Header blk)
chainDiff'

        where
          chainDiff' :: ChainDiff (Header blk)
chainDiff' = ValidatedChainDiff (Header blk) (LedgerDB' blk)
-> ChainDiff (Header blk)
forall b l. ValidatedChainDiff b l -> ChainDiff b
ValidatedDiff.getChainDiff ValidatedChainDiff (Header blk) (LedgerDB' blk)
validatedChainDiff

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

-- | Instantiate 'ValidatedFragment' in the way that chain selection requires.
type ChainAndLedger blk = ValidatedFragment (Header blk) (LedgerDB' 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