{-# LANGUAGE DerivingStrategies #-}
{-# LANGUAGE FlexibleContexts #-}
{-# LANGUAGE GADTs #-}
{-# LANGUAGE NamedFieldPuns #-}
{-# LANGUAGE RankNTypes #-}
{-# LANGUAGE ScopedTypeVariables #-}
{-# LANGUAGE StandaloneDeriving #-}
{-# LANGUAGE UndecidableInstances #-}

-- | Implementation of the GDD governor
--
-- The GDD governor is the component responsible for identifying and
-- disconnecting peers offering sparser chains than the best. This has the
-- effect of unblocking the Limit on Eagerness, since removing disagreeing
-- peers allows the current selection to advance. See
-- 'Ouroboros.Consensus.Storage.ChainDB.API.LoE' for more details.
--
-- The GDD governor, invoked with 'runGDDGovernor', is supposed to run in a background
-- thread. It evaluates candidate chains whenever they change, or whenever a
-- peer claims to have no more headers, or whenever a peer starts sending
-- headers beyond the forecast horizon.
--
-- Whenever GDD disconnects peers, and as a result the youngest header present
-- in all candidate fragments changes, the chain selection is updated.
--
module Ouroboros.Consensus.Genesis.Governor (
    DensityBounds (..)
  , GDDDebugInfo (..)
  , GDDStateView (..)
  , TraceGDDEvent (..)
  , densityDisconnect
  , gddWatcher
  , sharedCandidatePrefix
  ) where

import           Control.Monad (guard, void, when)
import           Control.Tracer (Tracer, traceWith)
import           Data.Bifunctor (second)
import           Data.Containers.ListUtils (nubOrd)
import           Data.Foldable (for_, toList)
import           Data.Functor.Compose (Compose (..))
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 (mapMaybe, maybeToList)
import           Data.Maybe.Strict (StrictMaybe)
import           Data.Word (Word64)
import           Ouroboros.Consensus.Block
import           Ouroboros.Consensus.Config (TopLevelConfig, configLedger,
                     configSecurityParam)
import           Ouroboros.Consensus.Config.SecurityParam
                     (SecurityParam (SecurityParam))
import           Ouroboros.Consensus.HardFork.Abstract (HasHardForkHistory (..))
import           Ouroboros.Consensus.HardFork.History.Qry (qryFromExpr,
                     runQuery, slotToGenesisWindow)
import           Ouroboros.Consensus.Ledger.Extended (ExtLedgerState,
                     ledgerState)
import           Ouroboros.Consensus.Ledger.SupportsProtocol
                     (LedgerSupportsProtocol)
import           Ouroboros.Consensus.MiniProtocol.ChainSync.Client
                     (ChainSyncClientHandle (..), ChainSyncState (..))
import           Ouroboros.Consensus.Node.GsmState
import           Ouroboros.Consensus.Storage.ChainDB.API (ChainDB)
import qualified Ouroboros.Consensus.Storage.ChainDB.API as ChainDB
import           Ouroboros.Consensus.Util (eitherToMaybe, whenJust)
import           Ouroboros.Consensus.Util.AnchoredFragment (stripCommonPrefix)
import           Ouroboros.Consensus.Util.IOLike
import           Ouroboros.Consensus.Util.STM (Watcher (..))
import           Ouroboros.Network.AnchoredFragment (AnchoredFragment)
import qualified Ouroboros.Network.AnchoredFragment as AF

-- | A 'Watcher' that evaluates the GDD rule whenever necessary, writing the LoE
-- fragment to @varLoEFrag@, and then triggering ChainSel to reprocess all
-- blocks that had previously been postponed by the LoE.
--
-- Evaluating the GDD rule might cause peers to be disconnected if they have
-- sparser chains than the best chain.
gddWatcher ::
     forall m blk peer.
     ( IOLike m
     , Ord peer
     , LedgerSupportsProtocol blk
     , HasHardForkHistory blk
     )
  => TopLevelConfig blk
  -> Tracer m (TraceGDDEvent peer blk)
  -> ChainDB m blk
  -> STM m GsmState
  -> STM m (Map peer (ChainSyncClientHandle m blk))
     -- ^ The ChainSync handles. We trigger the GDD whenever our 'GsmState'
     -- changes, and when 'Syncing', whenever any of the candidate fragments
     -- changes. Also, we use this to disconnect from peers with insufficient
     -- densities.
  -> StrictTVar m (AnchoredFragment (Header blk))
     -- ^ The LoE fragment. It starts at a (recent) immutable tip and ends at
     -- the common intersection of the candidate fragments.
  -> Watcher m
       (GsmState, GDDStateView m blk peer)
       (Map peer (StrictMaybe (WithOrigin SlotNo), Bool))
gddWatcher :: forall (m :: * -> *) blk peer.
(IOLike m, Ord peer, LedgerSupportsProtocol blk,
 HasHardForkHistory blk) =>
TopLevelConfig blk
-> Tracer m (TraceGDDEvent peer blk)
-> ChainDB m blk
-> STM m GsmState
-> STM m (Map peer (ChainSyncClientHandle m blk))
-> StrictTVar m (AnchoredFragment (Header blk))
-> Watcher
     m
     (GsmState, GDDStateView m blk peer)
     (Map peer (StrictMaybe (WithOrigin SlotNo), Bool))
gddWatcher TopLevelConfig blk
cfg Tracer m (TraceGDDEvent peer blk)
tracer ChainDB m blk
chainDb STM m GsmState
getGsmState STM m (Map peer (ChainSyncClientHandle m blk))
getHandles StrictTVar m (AnchoredFragment (Header blk))
varLoEFrag =
    Watcher {
        wInitial :: Maybe (Map peer (StrictMaybe (WithOrigin SlotNo), Bool))
wInitial = Maybe (Map peer (StrictMaybe (WithOrigin SlotNo), Bool))
forall a. Maybe a
Nothing
      , wReader :: STM m (GsmState, GDDStateView m blk peer)
wReader  = (,) (GsmState
 -> GDDStateView m blk peer -> (GsmState, GDDStateView m blk peer))
-> STM m GsmState
-> STM
     m (GDDStateView m blk peer -> (GsmState, GDDStateView m blk peer))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> STM m GsmState
getGsmState STM
  m (GDDStateView m blk peer -> (GsmState, GDDStateView m blk peer))
-> STM m (GDDStateView m blk peer)
-> STM m (GsmState, GDDStateView m blk peer)
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
<*> STM m (GDDStateView m blk peer)
getGDDStateView
      , (GsmState, GDDStateView m blk peer)
-> Map peer (StrictMaybe (WithOrigin SlotNo), Bool)
wFingerprint :: (GsmState, GDDStateView m blk peer)
-> Map peer (StrictMaybe (WithOrigin SlotNo), Bool)
wFingerprint :: (GsmState, GDDStateView m blk peer)
-> Map peer (StrictMaybe (WithOrigin SlotNo), Bool)
wFingerprint
      , (GsmState, GDDStateView m blk peer) -> m ()
wNotify :: (GsmState, GDDStateView m blk peer) -> m ()
wNotify :: (GsmState, GDDStateView m blk peer) -> m ()
wNotify
      }
  where
    getGDDStateView :: STM m (GDDStateView m blk peer)
    getGDDStateView :: STM m (GDDStateView m blk peer)
getGDDStateView = do
        AnchoredFragment (Header blk)
curChain          <- ChainDB m blk -> STM m (AnchoredFragment (Header blk))
forall (m :: * -> *) blk.
ChainDB m blk -> STM m (AnchoredFragment (Header blk))
ChainDB.getCurrentChain ChainDB m blk
chainDb
        ExtLedgerState blk
immutableLedgerSt <- ChainDB m blk -> STM m (ExtLedgerState blk)
forall (m :: * -> *) blk.
Monad (STM m) =>
ChainDB m blk -> STM m (ExtLedgerState blk)
ChainDB.getImmutableLedger ChainDB m blk
chainDb
        Map peer (ChainSyncClientHandle m blk)
handles           <- STM m (Map peer (ChainSyncClientHandle m blk))
getHandles
        Map peer (ChainSyncState blk)
states            <- (ChainSyncClientHandle m blk -> STM m (ChainSyncState blk))
-> Map peer (ChainSyncClientHandle m blk)
-> STM m (Map peer (ChainSyncState blk))
forall (t :: * -> *) (f :: * -> *) a b.
(Traversable t, Applicative f) =>
(a -> f b) -> t a -> f (t b)
forall (f :: * -> *) a b.
Applicative f =>
(a -> f b) -> Map peer a -> f (Map peer b)
traverse (StrictTVar m (ChainSyncState blk) -> STM m (ChainSyncState blk)
forall (m :: * -> *) a. MonadSTM m => StrictTVar m a -> STM m a
readTVar (StrictTVar m (ChainSyncState blk) -> STM m (ChainSyncState blk))
-> (ChainSyncClientHandle m blk
    -> StrictTVar m (ChainSyncState blk))
-> ChainSyncClientHandle m blk
-> STM m (ChainSyncState blk)
forall b c a. (b -> c) -> (a -> b) -> a -> c
. ChainSyncClientHandle m blk -> StrictTVar m (ChainSyncState blk)
forall (m :: * -> *) blk.
ChainSyncClientHandle m blk -> StrictTVar m (ChainSyncState blk)
cschState) Map peer (ChainSyncClientHandle m blk)
handles
        GDDStateView m blk peer -> STM m (GDDStateView m blk peer)
forall a. a -> STM m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure GDDStateView {
            gddCtxCurChain :: AnchoredFragment (Header blk)
gddCtxCurChain          = AnchoredFragment (Header blk)
curChain
          , gddCtxImmutableLedgerSt :: ExtLedgerState blk
gddCtxImmutableLedgerSt = ExtLedgerState blk
immutableLedgerSt
          , gddCtxKillActions :: Map peer (m ())
gddCtxKillActions       = (ChainSyncClientHandle m blk -> m ())
-> Map peer (ChainSyncClientHandle m blk) -> Map peer (m ())
forall a b k. (a -> b) -> Map k a -> Map k b
Map.map ChainSyncClientHandle m blk -> m ()
forall (m :: * -> *) blk. ChainSyncClientHandle m blk -> m ()
cschGDDKill Map peer (ChainSyncClientHandle m blk)
handles
          , gddCtxStates :: Map peer (ChainSyncState blk)
gddCtxStates            = Map peer (ChainSyncState blk)
states
          }

    wFingerprint ::
         (GsmState, GDDStateView m blk peer)
      -> Map peer (StrictMaybe (WithOrigin SlotNo), Bool)
    wFingerprint :: (GsmState, GDDStateView m blk peer)
-> Map peer (StrictMaybe (WithOrigin SlotNo), Bool)
wFingerprint (GsmState
gsmState, GDDStateView{Map peer (ChainSyncState blk)
gddCtxStates :: forall (m :: * -> *) blk peer.
GDDStateView m blk peer -> Map peer (ChainSyncState blk)
gddCtxStates :: Map peer (ChainSyncState blk)
gddCtxStates}) = case GsmState
gsmState of
        -- When we are in 'PreSyncing' (HAA not satisfied) or are caught up, we
        -- don't have to run the GDD on changes to the candidate fragments.
        -- (Maybe we want to do it in 'PreSycing'?)
        GsmState
PreSyncing -> Map peer (StrictMaybe (WithOrigin SlotNo), Bool)
forall k a. Map k a
Map.empty
        GsmState
CaughtUp   -> Map peer (StrictMaybe (WithOrigin SlotNo), Bool)
forall k a. Map k a
Map.empty
        -- When syncing, wake up regularly while headers are sent.
        -- Watching csLatestSlot ensures that GDD is woken up when a peer is
        -- sending headers even if they are after the forecast horizon. Note
        -- that there can be some delay between the header being validated and
        -- it becoming visible to GDD. It will be visible only when csLatestSlot
        -- changes again or when csIdling changes, which is guaranteed to happen
        -- eventually.
        GsmState
Syncing    ->
          (ChainSyncState blk -> (StrictMaybe (WithOrigin SlotNo), Bool))
-> Map peer (ChainSyncState blk)
-> Map peer (StrictMaybe (WithOrigin SlotNo), Bool)
forall a b k. (a -> b) -> Map k a -> Map k b
Map.map (\ChainSyncState blk
css -> (ChainSyncState blk -> StrictMaybe (WithOrigin SlotNo)
forall blk. ChainSyncState blk -> StrictMaybe (WithOrigin SlotNo)
csLatestSlot ChainSyncState blk
css, ChainSyncState blk -> Bool
forall blk. ChainSyncState blk -> Bool
csIdling ChainSyncState blk
css)) Map peer (ChainSyncState blk)
gddCtxStates

    wNotify :: (GsmState, GDDStateView m blk peer) -> m ()
    wNotify :: (GsmState, GDDStateView m blk peer) -> m ()
wNotify (GsmState
_gsmState, GDDStateView m blk peer
stateView) = do
        AnchoredFragment (Header blk)
loeFrag <- TopLevelConfig blk
-> Tracer m (TraceGDDEvent peer blk)
-> GDDStateView m blk peer
-> m (AnchoredFragment (Header blk))
forall (m :: * -> *) blk peer.
(IOLike m, Ord peer, LedgerSupportsProtocol blk,
 HasHardForkHistory blk) =>
TopLevelConfig blk
-> Tracer m (TraceGDDEvent peer blk)
-> GDDStateView m blk peer
-> m (AnchoredFragment (Header blk))
evaluateGDD TopLevelConfig blk
cfg Tracer m (TraceGDDEvent peer blk)
tracer GDDStateView m blk peer
stateView
        AnchoredFragment (Header blk)
oldLoEFrag <- STM m (AnchoredFragment (Header blk))
-> m (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 (AnchoredFragment (Header blk))
 -> m (AnchoredFragment (Header blk)))
-> STM m (AnchoredFragment (Header blk))
-> m (AnchoredFragment (Header blk))
forall a b. (a -> b) -> a -> b
$ StrictTVar m (AnchoredFragment (Header blk))
-> AnchoredFragment (Header blk)
-> STM m (AnchoredFragment (Header blk))
forall (m :: * -> *) a.
MonadSTM m =>
StrictTVar m a -> a -> STM m a
swapTVar StrictTVar m (AnchoredFragment (Header blk))
varLoEFrag AnchoredFragment (Header blk)
loeFrag
        -- The chain selection only depends on the LoE tip, so there
        -- is no point in retriggering it if the LoE tip hasn't changed.
        Bool -> m () -> m ()
forall (f :: * -> *). Applicative f => Bool -> f () -> f ()
when (AnchoredFragment (Header blk) -> ChainHash (Header blk)
forall block.
HasHeader block =>
AnchoredFragment block -> ChainHash block
AF.headHash AnchoredFragment (Header blk)
oldLoEFrag ChainHash (Header blk) -> ChainHash (Header blk) -> Bool
forall a. Eq a => a -> a -> Bool
/= AnchoredFragment (Header blk) -> ChainHash (Header blk)
forall block.
HasHeader block =>
AnchoredFragment block -> ChainHash block
AF.headHash AnchoredFragment (Header blk)
loeFrag) (m () -> m ()) -> m () -> m ()
forall a b. (a -> b) -> a -> b
$
          m (ChainSelectionPromise m) -> m ()
forall (f :: * -> *) a. Functor f => f a -> f ()
void (m (ChainSelectionPromise m) -> m ())
-> m (ChainSelectionPromise m) -> m ()
forall a b. (a -> b) -> a -> b
$ ChainDB m blk -> m (ChainSelectionPromise m)
forall (m :: * -> *) blk.
ChainDB m blk -> m (ChainSelectionPromise m)
ChainDB.triggerChainSelectionAsync ChainDB m blk
chainDb

-- | Pure snapshot of the dynamic data the GDD operates on.
data GDDStateView m blk peer = GDDStateView {
    -- | The current chain selection
    forall (m :: * -> *) blk peer.
GDDStateView m blk peer -> AnchoredFragment (Header blk)
gddCtxCurChain          :: AnchoredFragment (Header blk)
    -- | The current ledger state
  , forall (m :: * -> *) blk peer.
GDDStateView m blk peer -> ExtLedgerState blk
gddCtxImmutableLedgerSt :: ExtLedgerState blk
    -- | Callbacks to disconnect from peers
  , forall (m :: * -> *) blk peer.
GDDStateView m blk peer -> Map peer (m ())
gddCtxKillActions       :: Map peer (m ())
  , forall (m :: * -> *) blk peer.
GDDStateView m blk peer -> Map peer (ChainSyncState blk)
gddCtxStates            :: Map peer (ChainSyncState blk)
  }

-- | Disconnect peers that lose density comparisons and recompute the LoE fragment.
--
-- Disconnecting peers causes candidate fragments to be removed, which causes
-- the GDD governor to reevaluate GDD over and over until no more peers are
-- disconnected.
--
-- Yields the new LoE fragment.
--
evaluateGDD ::
     forall m blk peer.
     ( IOLike m
     , Ord peer
     , LedgerSupportsProtocol blk
     , HasHardForkHistory blk
     )
  => TopLevelConfig blk
  -> Tracer m (TraceGDDEvent peer blk)
  -> GDDStateView m blk peer
  -> m (AnchoredFragment (Header blk))
evaluateGDD :: forall (m :: * -> *) blk peer.
(IOLike m, Ord peer, LedgerSupportsProtocol blk,
 HasHardForkHistory blk) =>
TopLevelConfig blk
-> Tracer m (TraceGDDEvent peer blk)
-> GDDStateView m blk peer
-> m (AnchoredFragment (Header blk))
evaluateGDD TopLevelConfig blk
cfg Tracer m (TraceGDDEvent peer blk)
tracer GDDStateView m blk peer
stateView = do
    let GDDStateView {
            gddCtxCurChain :: forall (m :: * -> *) blk peer.
GDDStateView m blk peer -> AnchoredFragment (Header blk)
gddCtxCurChain          = AnchoredFragment (Header blk)
curChain
          , gddCtxImmutableLedgerSt :: forall (m :: * -> *) blk peer.
GDDStateView m blk peer -> ExtLedgerState blk
gddCtxImmutableLedgerSt = ExtLedgerState blk
immutableLedgerSt
          , gddCtxKillActions :: forall (m :: * -> *) blk peer.
GDDStateView m blk peer -> Map peer (m ())
gddCtxKillActions       = Map peer (m ())
killActions
          , gddCtxStates :: forall (m :: * -> *) blk peer.
GDDStateView m blk peer -> Map peer (ChainSyncState blk)
gddCtxStates            = Map peer (ChainSyncState blk)
states
          } = GDDStateView m blk peer
stateView

        (AnchoredFragment (Header blk)
loeFrag, [(peer, AnchoredFragment (Header blk))]
candidateSuffixes) =
          AnchoredFragment (Header blk)
-> [(peer, AnchoredFragment (Header blk))]
-> (AnchoredFragment (Header blk),
    [(peer, AnchoredFragment (Header blk))])
forall blk peer.
GetHeader blk =>
AnchoredFragment (Header blk)
-> [(peer, AnchoredFragment (Header blk))]
-> (AnchoredFragment (Header blk),
    [(peer, AnchoredFragment (Header blk))])
sharedCandidatePrefix AnchoredFragment (Header blk)
curChain [(peer, AnchoredFragment (Header blk))]
candidates
        candidates :: [(peer, AnchoredFragment (Header blk))]
candidates = Map peer (AnchoredFragment (Header blk))
-> [(peer, AnchoredFragment (Header blk))]
forall k a. Map k a -> [(k, a)]
Map.toList (ChainSyncState blk -> AnchoredFragment (Header blk)
forall blk. ChainSyncState blk -> AnchoredFragment (Header blk)
csCandidate (ChainSyncState blk -> AnchoredFragment (Header blk))
-> Map peer (ChainSyncState blk)
-> Map peer (AnchoredFragment (Header blk))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Map peer (ChainSyncState blk)
states)

        msgen :: Maybe GenesisWindow
        -- This could also use 'runWithCachedSummary' if deemed desirable.
        msgen :: Maybe GenesisWindow
msgen = Either PastHorizonException GenesisWindow -> Maybe GenesisWindow
forall a b. Either a b -> Maybe b
eitherToMaybe (Either PastHorizonException GenesisWindow -> Maybe GenesisWindow)
-> Either PastHorizonException GenesisWindow -> Maybe GenesisWindow
forall a b. (a -> b) -> a -> b
$ Qry GenesisWindow
-> Summary (HardForkIndices blk)
-> Either PastHorizonException GenesisWindow
forall a (xs :: [*]).
HasCallStack =>
Qry a -> Summary xs -> Either PastHorizonException a
runQuery Qry GenesisWindow
qry Summary (HardForkIndices blk)
summary
          where
            -- We use the Genesis window for the first slot /after/ the common
            -- intersection. In particular, when the intersection is the last
            -- slot of an era, we will use the Genesis window of the next era,
            -- as all slots in the Genesis window reside in that next era.
            slot :: SlotNo
slot    = WithOrigin SlotNo -> SlotNo
forall t. (Bounded t, Enum t) => WithOrigin t -> t
succWithOrigin (WithOrigin SlotNo -> SlotNo) -> WithOrigin SlotNo -> SlotNo
forall a b. (a -> b) -> a -> b
$ AnchoredFragment (Header blk) -> WithOrigin SlotNo
forall block.
HasHeader block =>
AnchoredFragment block -> WithOrigin SlotNo
AF.headSlot AnchoredFragment (Header blk)
loeFrag
            qry :: Qry GenesisWindow
qry     = (forall (f :: * -> *). Expr f GenesisWindow) -> Qry GenesisWindow
forall a. (forall (f :: * -> *). Expr f a) -> Qry a
qryFromExpr ((forall (f :: * -> *). Expr f GenesisWindow) -> Qry GenesisWindow)
-> (forall (f :: * -> *). Expr f GenesisWindow)
-> Qry GenesisWindow
forall a b. (a -> b) -> a -> b
$ SlotNo -> Expr f GenesisWindow
forall (f :: * -> *). SlotNo -> Expr f GenesisWindow
slotToGenesisWindow SlotNo
slot
            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)
                -- Due to the cross-chain lemma (Property 17.3 in the Consensus
                -- report) one could also use the ledger state at the tip of our
                -- selection here (in which case this should never return
                -- 'Nothing'), but this is subtle and maybe not desirable.
                --
                -- In any case, the immutable ledger state will also
                -- /eventually/ catch up to the LoE tip, so @msgen@ won't be
                -- 'Nothing' forever.
                (ExtLedgerState blk -> LedgerState blk
forall blk. ExtLedgerState blk -> LedgerState blk
ledgerState ExtLedgerState blk
immutableLedgerSt)

    Maybe GenesisWindow -> (GenesisWindow -> m ()) -> m ()
forall (f :: * -> *) a.
Applicative f =>
Maybe a -> (a -> f ()) -> f ()
whenJust Maybe GenesisWindow
msgen ((GenesisWindow -> m ()) -> m ())
-> (GenesisWindow -> m ()) -> m ()
forall a b. (a -> b) -> a -> b
$ \GenesisWindow
sgen -> do
      let
        ([peer]
losingPeers, [(peer, DensityBounds blk)]
bounds) =
          GenesisWindow
-> SecurityParam
-> Map peer (ChainSyncState blk)
-> [(peer, AnchoredFragment (Header blk))]
-> AnchoredFragment (Header blk)
-> ([peer], [(peer, DensityBounds blk)])
forall peer blk.
(Ord peer, LedgerSupportsProtocol blk) =>
GenesisWindow
-> SecurityParam
-> Map peer (ChainSyncState blk)
-> [(peer, AnchoredFragment (Header blk))]
-> AnchoredFragment (Header blk)
-> ([peer], [(peer, DensityBounds blk)])
densityDisconnect GenesisWindow
sgen (TopLevelConfig blk -> SecurityParam
forall blk.
ConsensusProtocol (BlockProtocol blk) =>
TopLevelConfig blk -> SecurityParam
configSecurityParam TopLevelConfig blk
cfg) Map peer (ChainSyncState blk)
states [(peer, AnchoredFragment (Header blk))]
candidateSuffixes AnchoredFragment (Header blk)
loeFrag
        loeHead :: Anchor (Header blk)
loeHead = AnchoredFragment (Header blk) -> Anchor (Header blk)
forall v a b. Anchorable v a b => AnchoredSeq v a b -> a
AF.headAnchor AnchoredFragment (Header blk)
loeFrag

      Tracer m (TraceGDDEvent peer blk) -> TraceGDDEvent peer blk -> m ()
forall (m :: * -> *) a. Tracer m a -> a -> m ()
traceWith Tracer m (TraceGDDEvent peer blk)
tracer (TraceGDDEvent peer blk -> m ()) -> TraceGDDEvent peer blk -> m ()
forall a b. (a -> b) -> a -> b
$ GDDDebugInfo peer blk -> TraceGDDEvent peer blk
forall peer blk. GDDDebugInfo peer blk -> TraceGDDEvent peer blk
TraceGDDDebug
        GDDDebugInfo {GenesisWindow
sgen :: GenesisWindow
sgen :: GenesisWindow
sgen, AnchoredFragment (Header blk)
curChain :: AnchoredFragment (Header blk)
curChain :: AnchoredFragment (Header blk)
curChain, [(peer, DensityBounds blk)]
bounds :: [(peer, DensityBounds blk)]
bounds :: [(peer, DensityBounds blk)]
bounds, [(peer, AnchoredFragment (Header blk))]
candidates :: [(peer, AnchoredFragment (Header blk))]
candidates :: [(peer, AnchoredFragment (Header blk))]
candidates, [(peer, AnchoredFragment (Header blk))]
candidateSuffixes :: [(peer, AnchoredFragment (Header blk))]
candidateSuffixes :: [(peer, AnchoredFragment (Header blk))]
candidateSuffixes, [peer]
losingPeers :: [peer]
losingPeers :: [peer]
losingPeers, Anchor (Header blk)
loeHead :: Anchor (Header blk)
loeHead :: Anchor (Header blk)
loeHead}

      Maybe (NonEmpty peer) -> (NonEmpty peer -> m ()) -> m ()
forall (f :: * -> *) a.
Applicative f =>
Maybe a -> (a -> f ()) -> f ()
whenJust ([peer] -> Maybe (NonEmpty peer)
forall a. [a] -> Maybe (NonEmpty a)
NE.nonEmpty [peer]
losingPeers) ((NonEmpty peer -> m ()) -> m ())
-> (NonEmpty peer -> m ()) -> m ()
forall a b. (a -> b) -> a -> b
$ \NonEmpty peer
losingPeersNE -> do
        NonEmpty peer -> (peer -> m ()) -> m ()
forall (t :: * -> *) (f :: * -> *) a b.
(Foldable t, Applicative f) =>
t a -> (a -> f b) -> f ()
for_ NonEmpty peer
losingPeersNE ((peer -> m ()) -> m ()) -> (peer -> m ()) -> m ()
forall a b. (a -> b) -> a -> b
$ \peer
peer -> Map peer (m ())
killActions Map peer (m ()) -> peer -> m ()
forall k a. Ord k => Map k a -> k -> a
Map.! peer
peer
        Tracer m (TraceGDDEvent peer blk) -> TraceGDDEvent peer blk -> m ()
forall (m :: * -> *) a. Tracer m a -> a -> m ()
traceWith Tracer m (TraceGDDEvent peer blk)
tracer (TraceGDDEvent peer blk -> m ()) -> TraceGDDEvent peer blk -> m ()
forall a b. (a -> b) -> a -> b
$ NonEmpty peer -> TraceGDDEvent peer blk
forall peer blk. NonEmpty peer -> TraceGDDEvent peer blk
TraceGDDDisconnected NonEmpty peer
losingPeersNE

    AnchoredFragment (Header blk) -> m (AnchoredFragment (Header blk))
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure AnchoredFragment (Header blk)
loeFrag

-- | Compute the fragment @loeFrag@ between the immutable tip and the
-- earliest intersection between @curChain@ and any of the @candidates@.
--
-- The immutable tip is the anchor of @curChain@.
--
-- The function also yields the suffixes of the intersection of @loeFrag@ with
-- every candidate fragment.
sharedCandidatePrefix ::
  GetHeader blk =>
  AnchoredFragment (Header blk) ->
  [(peer, AnchoredFragment (Header blk))] ->
  (AnchoredFragment (Header blk), [(peer, AnchoredFragment (Header blk))])
sharedCandidatePrefix :: forall blk peer.
GetHeader blk =>
AnchoredFragment (Header blk)
-> [(peer, AnchoredFragment (Header blk))]
-> (AnchoredFragment (Header blk),
    [(peer, AnchoredFragment (Header blk))])
sharedCandidatePrefix AnchoredFragment (Header blk)
curChain [(peer, AnchoredFragment (Header blk))]
candidates =
  (Compose [] ((,) peer) (AnchoredFragment (Header blk))
 -> [(peer, AnchoredFragment (Header blk))])
-> (AnchoredFragment (Header blk),
    Compose [] ((,) peer) (AnchoredFragment (Header blk)))
-> (AnchoredFragment (Header blk),
    [(peer, AnchoredFragment (Header blk))])
forall b c a. (b -> c) -> (a, b) -> (a, c)
forall (p :: * -> * -> *) b c a.
Bifunctor p =>
(b -> c) -> p a b -> p a c
second Compose [] ((,) peer) (AnchoredFragment (Header blk))
-> [(peer, AnchoredFragment (Header blk))]
forall {k1} {k2} (f :: k1 -> *) (g :: k2 -> k1) (a :: k2).
Compose f g a -> f (g a)
getCompose ((AnchoredFragment (Header blk),
  Compose [] ((,) peer) (AnchoredFragment (Header blk)))
 -> (AnchoredFragment (Header blk),
     [(peer, AnchoredFragment (Header blk))]))
-> (AnchoredFragment (Header blk),
    Compose [] ((,) peer) (AnchoredFragment (Header blk)))
-> (AnchoredFragment (Header blk),
    [(peer, AnchoredFragment (Header blk))])
forall a b. (a -> b) -> a -> b
$
  Anchor (Header blk)
-> Compose [] ((,) peer) (AnchoredFragment (Header blk))
-> (AnchoredFragment (Header blk),
    Compose [] ((,) peer) (AnchoredFragment (Header blk)))
forall (f :: * -> *) blk.
(Functor f, Foldable f, HasHeader blk) =>
Anchor blk
-> f (AnchoredFragment blk)
-> (AnchoredFragment blk, f (AnchoredFragment blk))
stripCommonPrefix (AnchoredFragment (Header blk) -> Anchor (Header blk)
forall v a b. AnchoredSeq v a b -> a
AF.anchor AnchoredFragment (Header blk)
curChain) (Compose [] ((,) peer) (AnchoredFragment (Header blk))
 -> (AnchoredFragment (Header blk),
     Compose [] ((,) peer) (AnchoredFragment (Header blk))))
-> Compose [] ((,) peer) (AnchoredFragment (Header blk))
-> (AnchoredFragment (Header blk),
    Compose [] ((,) peer) (AnchoredFragment (Header blk)))
forall a b. (a -> b) -> a -> b
$
  [(peer, AnchoredFragment (Header blk))]
-> Compose [] ((,) peer) (AnchoredFragment (Header blk))
forall {k} {k1} (f :: k -> *) (g :: k1 -> k) (a :: k1).
f (g a) -> Compose f g a
Compose [(peer, AnchoredFragment (Header blk))]
immutableTipSuffixes
  where
    immutableTip :: Point (Header blk)
immutableTip = AnchoredFragment (Header blk) -> Point (Header blk)
forall block. AnchoredFragment block -> Point block
AF.anchorPoint AnchoredFragment (Header blk)
curChain

    splitAfterImmutableTip :: (peer, AnchoredFragment (Header blk))
-> Maybe (peer, AnchoredFragment (Header blk))
splitAfterImmutableTip (peer
peer, AnchoredFragment (Header blk)
frag) =
      (,) peer
peer (AnchoredFragment (Header blk)
 -> (peer, AnchoredFragment (Header blk)))
-> ((AnchoredFragment (Header blk), AnchoredFragment (Header blk))
    -> AnchoredFragment (Header blk))
-> (AnchoredFragment (Header blk), AnchoredFragment (Header blk))
-> (peer, AnchoredFragment (Header blk))
forall b c a. (b -> c) -> (a -> b) -> a -> c
. (AnchoredFragment (Header blk), AnchoredFragment (Header blk))
-> AnchoredFragment (Header blk)
forall a b. (a, b) -> b
snd ((AnchoredFragment (Header blk), AnchoredFragment (Header blk))
 -> (peer, AnchoredFragment (Header blk)))
-> Maybe
     (AnchoredFragment (Header blk), AnchoredFragment (Header blk))
-> Maybe (peer, AnchoredFragment (Header blk))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> 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)
frag Point (Header blk)
immutableTip

    immutableTipSuffixes :: [(peer, AnchoredFragment (Header blk))]
immutableTipSuffixes =
      -- If a ChainSync client's candidate forks off before the
      -- immutable tip, then this transaction is currently winning an
      -- innocuous race versus the thread that will fatally raise
      -- 'InvalidIntersection' within that ChainSync client, so it's
      -- sound to pre-emptively discard their candidate from this
      -- 'Map' via 'mapMaybe'.
      ((peer, AnchoredFragment (Header blk))
 -> Maybe (peer, AnchoredFragment (Header blk)))
-> [(peer, AnchoredFragment (Header blk))]
-> [(peer, AnchoredFragment (Header blk))]
forall a b. (a -> Maybe b) -> [a] -> [b]
mapMaybe (peer, AnchoredFragment (Header blk))
-> Maybe (peer, AnchoredFragment (Header blk))
splitAfterImmutableTip [(peer, AnchoredFragment (Header blk))]
candidates

data DensityBounds blk =
  DensityBounds {
    forall blk. DensityBounds blk -> AnchoredFragment (Header blk)
clippedFragment :: AnchoredFragment (Header blk),
    forall blk. DensityBounds blk -> Bool
offersMoreThanK :: Bool,
    forall blk. DensityBounds blk -> Word64
lowerBound      :: Word64,
    forall blk. DensityBounds blk -> Word64
upperBound      :: Word64,
    forall blk. DensityBounds blk -> Bool
hasBlockAfter   :: Bool,
    forall blk. DensityBounds blk -> WithOrigin SlotNo
latestSlot      :: WithOrigin SlotNo,
    forall blk. DensityBounds blk -> Bool
idling          :: Bool
  }

deriving stock instance (Show (Header blk), GetHeader blk) => Show (DensityBounds blk)

-- | @densityDisconnect genWin k states candidateSuffixes loeFrag@
-- yields the list of peers which are known to lose the density comparison with
-- any other peer, when looking at the genesis window after @loeFrag@.
--
-- The peers are taken from the keys of @candidateSuffixes@.
--
-- @candidateSuffixes@ tells for every peer what is the fragment that the peer
-- proposes to use after @loeFrag@.
--
-- @states@ contains further information for every peer, such as the last
-- ChainSync instruction the peer sent, and whether the peer is idling (i.e. it
-- sent @MsgAwaitReply@).
--
-- @loeFrag@ is the fragment anchored at the immutable tip and ending in the
-- LoE tip.
--
-- ChainSync jumping depends on this function to disconnect either of any two
-- peers that offer different chains and provided a header in the last slot of
-- the genesis window or later. Either of them should be disconnected, even if
-- both of them are serving adversarial chains. See
-- "Ouroboros.Consensus.MiniProtocol.ChainSync.Client.Jumping" for more details.
--
densityDisconnect ::
     ( Ord peer
     , LedgerSupportsProtocol blk
     )
  => GenesisWindow
  -> SecurityParam
  -> Map peer (ChainSyncState blk)
  -> [(peer, AnchoredFragment (Header blk))]
  -> AnchoredFragment (Header blk)
  -> ([peer], [(peer, DensityBounds blk)])
densityDisconnect :: forall peer blk.
(Ord peer, LedgerSupportsProtocol blk) =>
GenesisWindow
-> SecurityParam
-> Map peer (ChainSyncState blk)
-> [(peer, AnchoredFragment (Header blk))]
-> AnchoredFragment (Header blk)
-> ([peer], [(peer, DensityBounds blk)])
densityDisconnect (GenesisWindow Word64
sgen) (SecurityParam Word64
k) Map peer (ChainSyncState blk)
states [(peer, AnchoredFragment (Header blk))]
candidateSuffixes AnchoredFragment (Header blk)
loeFrag =
  ([peer]
losingPeers, [(peer, DensityBounds blk)]
densityBounds)
  where
    densityBounds :: [(peer, DensityBounds blk)]
densityBounds = do
      (peer
peer, AnchoredFragment (Header blk)
candidateSuffix) <- [(peer, AnchoredFragment (Header blk))]
candidateSuffixes
      let (AnchoredFragment (Header blk)
clippedFragment, AnchoredFragment (Header blk)
_) =
            SlotNo
-> AnchoredFragment (Header blk)
-> (AnchoredFragment (Header blk), AnchoredFragment (Header blk))
forall block.
HasHeader block =>
SlotNo
-> AnchoredFragment block
-> (AnchoredFragment block, AnchoredFragment block)
AF.splitAtSlot SlotNo
firstSlotAfterGenesisWindow AnchoredFragment (Header blk)
candidateSuffix
      ChainSyncState blk
state <- Maybe (ChainSyncState blk) -> [ChainSyncState blk]
forall a. Maybe a -> [a]
maybeToList (Map peer (ChainSyncState blk)
states Map peer (ChainSyncState blk) -> peer -> Maybe (ChainSyncState blk)
forall k a. Ord k => Map k a -> k -> Maybe a
Map.!? peer
peer)
      -- Skip peers that haven't sent any headers yet.
      -- They should be disconnected by timeouts instead.
      WithOrigin SlotNo
latestSlot <- StrictMaybe (WithOrigin SlotNo) -> [WithOrigin SlotNo]
forall a. StrictMaybe a -> [a]
forall (t :: * -> *) a. Foldable t => t a -> [a]
toList (ChainSyncState blk -> StrictMaybe (WithOrigin SlotNo)
forall blk. ChainSyncState blk -> StrictMaybe (WithOrigin SlotNo)
csLatestSlot ChainSyncState blk
state)
      let idling :: Bool
idling = ChainSyncState blk -> Bool
forall blk. ChainSyncState blk -> Bool
csIdling ChainSyncState blk
state

          -- Is there a block after the end of the Genesis window?
          hasBlockAfter :: Bool
hasBlockAfter =
               WithOrigin SlotNo -> WithOrigin SlotNo -> WithOrigin SlotNo
forall a. Ord a => a -> a -> a
max (AnchoredFragment (Header blk) -> WithOrigin SlotNo
forall block.
HasHeader block =>
AnchoredFragment block -> WithOrigin SlotNo
AF.headSlot AnchoredFragment (Header blk)
candidateSuffix) WithOrigin SlotNo
latestSlot
            WithOrigin SlotNo -> WithOrigin SlotNo -> Bool
forall a. Ord a => a -> a -> Bool
>= SlotNo -> WithOrigin SlotNo
forall t. t -> WithOrigin t
NotOrigin SlotNo
firstSlotAfterGenesisWindow

          -- If the slot of the latest header we know of is _after_ the end of
          -- the Genesis window (either because the candidate fragment extends
          -- beyond it or because we are waiting to validate a header beyond the
          -- forecast horizon that we already received), there can be no headers
          -- in between and 'potentialSlots' is 0.
          potentialSlots :: Word64
potentialSlots =
            if Bool
hasBlockAfter then Word64
0
            else Word64
unknownTrailingSlots

          -- Number of trailing slots in the genesis window that could have
          -- headers which haven't been sent yet
          unknownTrailingSlots :: Word64
unknownTrailingSlots = SlotNo -> Word64
unSlotNo (SlotNo -> Word64) -> SlotNo -> Word64
forall a b. (a -> b) -> a -> b
$
            -- cannot underflow as the fragment is clipped to the genesis window
            SlotNo
firstSlotAfterGenesisWindow SlotNo -> SlotNo -> SlotNo
forall a. Num a => a -> a -> a
- WithOrigin SlotNo -> SlotNo
forall t. (Bounded t, Enum t) => WithOrigin t -> t
succWithOrigin (AnchoredFragment (Header blk) -> WithOrigin SlotNo
forall block.
HasHeader block =>
AnchoredFragment block -> WithOrigin SlotNo
AF.headSlot AnchoredFragment (Header blk)
clippedFragment)

          -- The number of blocks within the Genesis window we know with certainty
          lowerBound :: Word64
lowerBound = Int -> Word64
forall a b. (Integral a, Num b) => a -> b
fromIntegral (Int -> Word64) -> Int -> Word64
forall a b. (a -> b) -> a -> b
$ AnchoredFragment (Header blk) -> Int
forall v a b. Anchorable v a b => AnchoredSeq v a b -> Int
AF.length AnchoredFragment (Header blk)
clippedFragment

          upperBound :: Word64
upperBound = Word64
lowerBound Word64 -> Word64 -> Word64
forall a. Num a => a -> a -> a
+ Word64
potentialSlots

          -- The number of blocks we know to be on the candidate chain after
          -- the intersection, not limited to the Genesis window.
          totalBlockCount :: Word64
totalBlockCount = 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)
candidateSuffix)

          -- Does the peer have more than k known blocks in _total_ after the intersection?
          -- If not, it is not qualified to compete by density (yet).
          offersMoreThanK :: Bool
offersMoreThanK = Word64
totalBlockCount Word64 -> Word64 -> Bool
forall a. Ord a => a -> a -> Bool
> Word64
k

      (peer, DensityBounds blk) -> [(peer, DensityBounds blk)]
forall a. a -> [a]
forall (f :: * -> *) a. Applicative f => a -> f a
pure (peer
peer, DensityBounds {AnchoredFragment (Header blk)
clippedFragment :: AnchoredFragment (Header blk)
clippedFragment :: AnchoredFragment (Header blk)
clippedFragment, Bool
offersMoreThanK :: Bool
offersMoreThanK :: Bool
offersMoreThanK, Word64
lowerBound :: Word64
lowerBound :: Word64
lowerBound, Word64
upperBound :: Word64
upperBound :: Word64
upperBound, Bool
hasBlockAfter :: Bool
hasBlockAfter :: Bool
hasBlockAfter, WithOrigin SlotNo
latestSlot :: WithOrigin SlotNo
latestSlot :: WithOrigin SlotNo
latestSlot, Bool
idling :: Bool
idling :: Bool
idling})

    losingPeers :: [peer]
losingPeers = [peer] -> [peer]
forall a. Ord a => [a] -> [a]
nubOrd ([peer] -> [peer]) -> [peer] -> [peer]
forall a b. (a -> b) -> a -> b
$ [(peer, DensityBounds blk)]
densityBounds [(peer, DensityBounds blk)]
-> ((peer, DensityBounds blk) -> [peer]) -> [peer]
forall a b. [a] -> (a -> [b]) -> [b]
forall (m :: * -> *) a b. Monad m => m a -> (a -> m b) -> m b
>>= \
      (peer
peer0 , DensityBounds { clippedFragment :: forall blk. DensityBounds blk -> AnchoredFragment (Header blk)
clippedFragment = AnchoredFragment (Header blk)
frag0
                             , lowerBound :: forall blk. DensityBounds blk -> Word64
lowerBound = Word64
lb0
                             , upperBound :: forall blk. DensityBounds blk -> Word64
upperBound = Word64
ub0
                             , hasBlockAfter :: forall blk. DensityBounds blk -> Bool
hasBlockAfter = Bool
hasBlockAfter0
                             , idling :: forall blk. DensityBounds blk -> Bool
idling = Bool
idling0
                             }) ->
      -- If the density is 0, the peer should be disconnected. This affects
      -- ChainSync jumping, where genesis windows with no headers prevent jumps
      -- from happening.
      if Word64
ub0 Word64 -> Word64 -> Bool
forall a. Eq a => a -> a -> Bool
== Word64
0 then peer -> [peer]
forall a. a -> [a]
forall (f :: * -> *) a. Applicative f => a -> f a
pure peer
peer0 else do
      (peer
_peer1, DensityBounds {clippedFragment :: forall blk. DensityBounds blk -> AnchoredFragment (Header blk)
clippedFragment = AnchoredFragment (Header blk)
frag1, Bool
offersMoreThanK :: forall blk. DensityBounds blk -> Bool
offersMoreThanK :: Bool
offersMoreThanK, lowerBound :: forall blk. DensityBounds blk -> Word64
lowerBound = Word64
lb1 }) <-
        [(peer, DensityBounds blk)]
densityBounds
      -- Don't disconnect peer0 if it sent no headers after the intersection yet
      -- and it is not idling.
      --
      -- See Note [Chain disagreement]
      --
      -- Note: hasBlockAfter0 is False if frag0 is empty and ub0>0.
      -- But we leave it here as a reminder that we care about it.
      Bool -> [()]
forall (f :: * -> *). Alternative f => Bool -> f ()
guard (Bool -> [()]) -> Bool -> [()]
forall a b. (a -> b) -> a -> b
$ Bool
idling0 Bool -> Bool -> Bool
|| Bool -> Bool
not (AnchoredFragment (Header blk) -> Bool
forall v a b. AnchoredSeq v a b -> Bool
AF.null AnchoredFragment (Header blk)
frag0) Bool -> Bool -> Bool
|| Bool
hasBlockAfter0
      -- ensure that the two peer fragments don't share any
      -- headers after the LoE
      Bool -> [()]
forall (f :: * -> *). Alternative f => Bool -> f ()
guard (Bool -> [()]) -> Bool -> [()]
forall a b. (a -> b) -> a -> b
$ AnchoredFragment (Header blk) -> Point (Header blk)
forall block.
HasHeader block =>
AnchoredFragment block -> Point block
AF.lastPoint AnchoredFragment (Header blk)
frag0 Point (Header blk) -> Point (Header blk) -> Bool
forall a. Eq a => a -> a -> Bool
/= AnchoredFragment (Header blk) -> Point (Header blk)
forall block.
HasHeader block =>
AnchoredFragment block -> Point block
AF.lastPoint AnchoredFragment (Header blk)
frag1
      -- peer1 offers more than k blocks or peer0 has sent all headers in the
      -- genesis window after the intersection (idling or not)
      --
      -- Checking for offersMoreThanK is important to avoid disconnecting
      -- competing honest peers when the syncing node is nearly caught up.
      Bool -> [()]
forall (f :: * -> *). Alternative f => Bool -> f ()
guard (Bool -> [()]) -> Bool -> [()]
forall a b. (a -> b) -> a -> b
$ Bool
offersMoreThanK Bool -> Bool -> Bool
|| Word64
lb0 Word64 -> Word64 -> Bool
forall a. Eq a => a -> a -> Bool
== Word64
ub0
      -- peer1 has the same or better density than peer0
      -- If peer0 is idling, we assume no more headers will be sent.
      --
      -- Having the same density is enough to disconnect peer0, as the honest
      -- chain is expected to have a strictly higher density than all of the
      -- other chains.
      --
      -- This matters to ChainSync jumping, where adversarial dynamo and
      -- objector could offer chains of equal density.
      Bool -> [()]
forall (f :: * -> *). Alternative f => Bool -> f ()
guard (Bool -> [()]) -> Bool -> [()]
forall a b. (a -> b) -> a -> b
$ Word64
lb1 Word64 -> Word64 -> Bool
forall a. Ord a => a -> a -> Bool
>= (if Bool
idling0 then Word64
lb0 else Word64
ub0)

      -- We disconnect peer0 if there is at least another peer peer1 with a
      -- chain which is at least as good, and peer0 is either idling or there is
      -- no extension to peer0's chain that can make it better than peer1's, and
      -- peer1's has more than k headers or peer0 has sent all its headers in
      -- the genesis window anchored at the intersection.
      --
      -- A chain is "as good as another" if it has at least as many headers in
      -- the genesis window anchored at the intersection.
      peer -> [peer]
forall a. a -> [a]
forall (f :: * -> *) a. Applicative f => a -> f a
pure peer
peer0

    loeIntersectionSlot :: WithOrigin SlotNo
loeIntersectionSlot = AnchoredFragment (Header blk) -> WithOrigin SlotNo
forall block.
HasHeader block =>
AnchoredFragment block -> WithOrigin SlotNo
AF.headSlot AnchoredFragment (Header blk)
loeFrag

    firstSlotAfterGenesisWindow :: SlotNo
firstSlotAfterGenesisWindow =
        WithOrigin SlotNo -> SlotNo
forall t. (Bounded t, Enum t) => WithOrigin t -> t
succWithOrigin WithOrigin SlotNo
loeIntersectionSlot SlotNo -> SlotNo -> SlotNo
forall a. Num a => a -> a -> a
+ Word64 -> SlotNo
SlotNo Word64
sgen

-- Note [Chain disagreement]
-- ~~~~~~~~~~~~~~~~~~~~~~~~~
--
-- Imagine two peers serving the following chain:
--
-- > k: 1
-- > sgen: 2
-- >
-- >   0 1 2
-- > G---1-2
--
-- Say peer1 sent no headers yet and peer2 sent 2 headers.
-- The intersection of both is G, the density of peer2's chain is 2,
-- while the upperbound of the density of peer1 is also 2.
--
-- For GDD to disconnect peer1 safely, it is essential that both chains
-- disagree after the intersection.
--
-- To know if the chains will dissagree we defer disconnecting peer1
-- until it declares to have no more headers, or until it sends one header
-- after the intersection. If both chains agree on the next header after
-- the intersection, we don't disconnect peer1 either.

data GDDDebugInfo peer blk =
  GDDDebugInfo {
    forall peer blk.
GDDDebugInfo peer blk -> [(peer, DensityBounds blk)]
bounds            :: [(peer, DensityBounds blk)],
    forall peer blk.
GDDDebugInfo peer blk -> AnchoredFragment (Header blk)
curChain          :: AnchoredFragment (Header blk),
    forall peer blk.
GDDDebugInfo peer blk -> [(peer, AnchoredFragment (Header blk))]
candidates        :: [(peer, AnchoredFragment (Header blk))],
    forall peer blk.
GDDDebugInfo peer blk -> [(peer, AnchoredFragment (Header blk))]
candidateSuffixes :: [(peer, AnchoredFragment (Header blk))],
    forall peer blk. GDDDebugInfo peer blk -> [peer]
losingPeers       :: [peer],
    forall peer blk. GDDDebugInfo peer blk -> Anchor (Header blk)
loeHead           :: AF.Anchor (Header blk),
    forall peer blk. GDDDebugInfo peer blk -> GenesisWindow
sgen              :: GenesisWindow
  }

deriving stock instance
  ( GetHeader blk, Show (Header blk), Show peer
  ) => Show (GDDDebugInfo peer blk)

data TraceGDDEvent peer blk =
    -- | The GDD disconnected from the given peers due to insufficient density.
    TraceGDDDisconnected (NonEmpty peer)
  |
    TraceGDDDebug (GDDDebugInfo peer blk)

deriving stock instance
  ( GetHeader blk, Show (Header blk), Show peer
  ) => Show (TraceGDDEvent peer blk)