{-# 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           Cardano.Ledger.BaseTypes (unNonZero)
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 (maybeToList)
import           Data.Maybe.Strict (StrictMaybe)
import           Data.Typeable (Typeable)
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.HeaderValidation (HeaderWithTime (..))
import           Ouroboros.Consensus.Ledger.Basics (EmptyMK)
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
  -> DiffTime -- ^ How often to evaluate GDD. 0 means as soon as possible.
              -- Otherwise, no faster than once every T seconds, where T is
              -- the provided value.
  -> 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 (HeaderWithTime 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
-> DiffTime
-> STM m GsmState
-> STM m (Map peer (ChainSyncClientHandle m blk))
-> StrictTVar m (AnchoredFragment (HeaderWithTime 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 DiffTime
rateLimit STM m GsmState
getGsmState STM m (Map peer (ChainSyncClientHandle m blk))
getHandles StrictTVar m (AnchoredFragment (HeaderWithTime 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
        curChain          <- ChainDB m blk -> STM m (AnchoredFragment (HeaderWithTime blk))
forall (m :: * -> *) blk.
ChainDB m blk -> STM m (AnchoredFragment (HeaderWithTime blk))
ChainDB.getCurrentChainWithTime ChainDB m blk
chainDb
        immutableLedgerSt <- ChainDB.getImmutableLedger chainDb
        handles           <- getHandles
        states            <- traverse (readTVar . cschState) handles
        pure GDDStateView {
            gddCtxCurChain          = curChain
          , gddCtxImmutableLedgerSt = immutableLedgerSt
          , gddCtxKillActions       = Map.map cschGDDKill handles
          , gddCtxStates            = 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
        t0 <- m Time
forall (m :: * -> *). MonadMonotonicTime m => m Time
getMonotonicTime
        loeFrag <- evaluateGDD cfg tracer stateView
        oldLoEFrag <- atomically $ swapTVar varLoEFrag 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.
        when (AF.headHash oldLoEFrag /= AF.headHash loeFrag) $
          void $ ChainDB.triggerChainSelectionAsync chainDb
        tf <- getMonotonicTime
        -- We limit the rate at which GDD is evaluated, otherwise it would
        -- be called every time a new header is validated.
        threadDelay $ rateLimit - diffTime tf t0

-- | 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 (HeaderWithTime blk)
gddCtxCurChain          :: AnchoredFragment (HeaderWithTime blk)
    -- | The current ledger state
  , forall (m :: * -> *) blk peer.
GDDStateView m blk peer -> ExtLedgerState blk EmptyMK
gddCtxImmutableLedgerSt :: ExtLedgerState blk EmptyMK
    -- | 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 (HeaderWithTime 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 (HeaderWithTime 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 (HeaderWithTime blk)
gddCtxCurChain          = AnchoredFragment (HeaderWithTime blk)
curChain
          , gddCtxImmutableLedgerSt :: forall (m :: * -> *) blk peer.
GDDStateView m blk peer -> ExtLedgerState blk EmptyMK
gddCtxImmutableLedgerSt = ExtLedgerState blk EmptyMK
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 (HeaderWithTime blk)
loeFrag, [(peer, AnchoredFragment (HeaderWithTime blk))]
candidateSuffixes) =
          AnchoredFragment (HeaderWithTime blk)
-> [(peer, AnchoredFragment (HeaderWithTime blk))]
-> (AnchoredFragment (HeaderWithTime blk),
    [(peer, AnchoredFragment (HeaderWithTime blk))])
forall blk peer.
(GetHeader blk, Typeable blk) =>
AnchoredFragment (HeaderWithTime blk)
-> [(peer, AnchoredFragment (HeaderWithTime blk))]
-> (AnchoredFragment (HeaderWithTime blk),
    [(peer, AnchoredFragment (HeaderWithTime blk))])
sharedCandidatePrefix AnchoredFragment (HeaderWithTime blk)
curChain [(peer, AnchoredFragment (HeaderWithTime blk))]
candidates
        candidates :: [(peer, AnchoredFragment (HeaderWithTime blk))]
candidates = Map peer (AnchoredFragment (HeaderWithTime blk))
-> [(peer, AnchoredFragment (HeaderWithTime blk))]
forall k a. Map k a -> [(k, a)]
Map.toList (ChainSyncState blk -> AnchoredFragment (HeaderWithTime blk)
forall blk.
ChainSyncState blk -> AnchoredFragment (HeaderWithTime blk)
csCandidate (ChainSyncState blk -> AnchoredFragment (HeaderWithTime blk))
-> Map peer (ChainSyncState blk)
-> Map peer (AnchoredFragment (HeaderWithTime 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 (HeaderWithTime blk) -> WithOrigin SlotNo
forall block.
HasHeader block =>
AnchoredFragment block -> WithOrigin SlotNo
AF.headSlot AnchoredFragment (HeaderWithTime 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 EmptyMK -> Summary (HardForkIndices blk)
forall blk (mk :: MapKind).
HasHardForkHistory blk =>
LedgerConfig blk
-> LedgerState blk mk -> Summary (HardForkIndices blk)
forall (mk :: MapKind).
LedgerConfig blk
-> LedgerState blk mk -> Summary (HardForkIndices blk)
hardForkSummary
                (TopLevelConfig blk -> LedgerConfig blk
forall blk. TopLevelConfig blk -> LedgerConfig blk
configLedger TopLevelConfig blk
cfg)
                -- 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 EmptyMK -> LedgerState blk EmptyMK
forall blk (mk :: MapKind).
ExtLedgerState blk mk -> LedgerState blk mk
ledgerState ExtLedgerState blk EmptyMK
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 (HeaderWithTime blk))]
-> AnchoredFragment (HeaderWithTime blk)
-> ([peer], [(peer, DensityBounds blk)])
forall peer blk.
(Ord peer, LedgerSupportsProtocol blk) =>
GenesisWindow
-> SecurityParam
-> Map peer (ChainSyncState blk)
-> [(peer, AnchoredFragment (HeaderWithTime blk))]
-> AnchoredFragment (HeaderWithTime 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 (HeaderWithTime blk))]
candidateSuffixes AnchoredFragment (HeaderWithTime blk)
loeFrag
        loeHead :: Anchor (Header blk)
loeHead = Anchor (HeaderWithTime blk) -> Anchor (Header blk)
forall a b. (HeaderHash a ~ HeaderHash b) => Anchor a -> Anchor b
AF.castAnchor (Anchor (HeaderWithTime blk) -> Anchor (Header blk))
-> Anchor (HeaderWithTime blk) -> Anchor (Header blk)
forall a b. (a -> b) -> a -> b
$ AnchoredFragment (HeaderWithTime blk)
-> Anchor (HeaderWithTime blk)
forall v a b. Anchorable v a b => AnchoredSeq v a b -> a
AF.headAnchor AnchoredFragment (HeaderWithTime blk)
loeFrag

        dropTimes :: [(a, AnchoredFragment (HeaderWithTime blk))]
-> [(a, AnchoredFragment (Header blk))]
dropTimes = ((a, AnchoredFragment (HeaderWithTime blk))
 -> (a, AnchoredFragment (Header blk)))
-> [(a, AnchoredFragment (HeaderWithTime blk))]
-> [(a, AnchoredFragment (Header blk))]
forall a b. (a -> b) -> [a] -> [b]
map ((AnchoredFragment (HeaderWithTime blk)
 -> AnchoredFragment (Header blk))
-> (a, AnchoredFragment (HeaderWithTime blk))
-> (a, AnchoredFragment (Header blk))
forall b c a. (b -> c) -> (a, b) -> (a, c)
forall (p :: MapKind) b c a.
Bifunctor p =>
(b -> c) -> p a b -> p a c
second ((HeaderWithTime blk -> Header blk)
-> AnchoredFragment (HeaderWithTime blk)
-> AnchoredFragment (Header blk)
forall block2 block1.
(HasHeader block2, HeaderHash block1 ~ HeaderHash block2) =>
(block1 -> block2)
-> AnchoredFragment block1 -> AnchoredFragment block2
AF.mapAnchoredFragment HeaderWithTime blk -> Header blk
forall blk. HeaderWithTime blk -> Header blk
hwtHeader))

      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 peer blk -> TraceGDDEvent peer blk)
-> GDDDebugInfo peer blk -> TraceGDDEvent peer blk
forall a b. (a -> b) -> a -> b
$ GDDDebugInfo
        { GenesisWindow
sgen :: GenesisWindow
sgen :: GenesisWindow
sgen
        , AnchoredFragment (HeaderWithTime blk)
curChain :: AnchoredFragment (HeaderWithTime blk)
curChain :: AnchoredFragment (HeaderWithTime blk)
curChain
        , [(peer, DensityBounds blk)]
bounds :: [(peer, DensityBounds blk)]
bounds :: [(peer, DensityBounds blk)]
bounds
        , candidates :: [(peer, AnchoredFragment (Header blk))]
candidates        = [(peer, AnchoredFragment (HeaderWithTime blk))]
-> [(peer, AnchoredFragment (Header blk))]
forall {a}.
[(a, AnchoredFragment (HeaderWithTime blk))]
-> [(a, AnchoredFragment (Header blk))]
dropTimes [(peer, AnchoredFragment (HeaderWithTime blk))]
candidates
        , candidateSuffixes :: [(peer, AnchoredFragment (Header blk))]
candidateSuffixes = [(peer, AnchoredFragment (HeaderWithTime blk))]
-> [(peer, AnchoredFragment (Header blk))]
forall {a}.
[(a, AnchoredFragment (HeaderWithTime blk))]
-> [(a, AnchoredFragment (Header blk))]
dropTimes [(peer, AnchoredFragment (HeaderWithTime 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 (HeaderWithTime blk)
-> m (AnchoredFragment (HeaderWithTime blk))
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure AnchoredFragment (HeaderWithTime 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, Typeable blk) =>
  AnchoredFragment (HeaderWithTime blk) ->
  [(peer, AnchoredFragment (HeaderWithTime blk))] ->
  (AnchoredFragment (HeaderWithTime blk), [(peer, AnchoredFragment (HeaderWithTime blk))])
sharedCandidatePrefix :: forall blk peer.
(GetHeader blk, Typeable blk) =>
AnchoredFragment (HeaderWithTime blk)
-> [(peer, AnchoredFragment (HeaderWithTime blk))]
-> (AnchoredFragment (HeaderWithTime blk),
    [(peer, AnchoredFragment (HeaderWithTime blk))])
sharedCandidatePrefix AnchoredFragment (HeaderWithTime blk)
curChain [(peer, AnchoredFragment (HeaderWithTime blk))]
candidates =
  (Compose [] ((,) peer) (AnchoredFragment (HeaderWithTime blk))
 -> [(peer, AnchoredFragment (HeaderWithTime blk))])
-> (AnchoredFragment (HeaderWithTime blk),
    Compose [] ((,) peer) (AnchoredFragment (HeaderWithTime blk)))
-> (AnchoredFragment (HeaderWithTime blk),
    [(peer, AnchoredFragment (HeaderWithTime blk))])
forall b c a. (b -> c) -> (a, b) -> (a, c)
forall (p :: MapKind) b c a.
Bifunctor p =>
(b -> c) -> p a b -> p a c
second Compose [] ((,) peer) (AnchoredFragment (HeaderWithTime blk))
-> [(peer, AnchoredFragment (HeaderWithTime blk))]
forall {k1} {k2} (f :: k1 -> *) (g :: k2 -> k1) (a :: k2).
Compose f g a -> f (g a)
getCompose ((AnchoredFragment (HeaderWithTime blk),
  Compose [] ((,) peer) (AnchoredFragment (HeaderWithTime blk)))
 -> (AnchoredFragment (HeaderWithTime blk),
     [(peer, AnchoredFragment (HeaderWithTime blk))]))
-> (AnchoredFragment (HeaderWithTime blk),
    Compose [] ((,) peer) (AnchoredFragment (HeaderWithTime blk)))
-> (AnchoredFragment (HeaderWithTime blk),
    [(peer, AnchoredFragment (HeaderWithTime blk))])
forall a b. (a -> b) -> a -> b
$
  Anchor (HeaderWithTime blk)
-> Compose [] ((,) peer) (AnchoredFragment (HeaderWithTime blk))
-> (AnchoredFragment (HeaderWithTime blk),
    Compose [] ((,) peer) (AnchoredFragment (HeaderWithTime blk)))
forall (f :: * -> *) blk.
(Functor f, Foldable f, HasHeader blk) =>
Anchor blk
-> f (AnchoredFragment blk)
-> (AnchoredFragment blk, f (AnchoredFragment blk))
stripCommonPrefix (Anchor (HeaderWithTime blk) -> Anchor (HeaderWithTime blk)
forall a b. (HeaderHash a ~ HeaderHash b) => Anchor a -> Anchor b
AF.castAnchor (Anchor (HeaderWithTime blk) -> Anchor (HeaderWithTime blk))
-> Anchor (HeaderWithTime blk) -> Anchor (HeaderWithTime blk)
forall a b. (a -> b) -> a -> b
$ AnchoredFragment (HeaderWithTime blk)
-> Anchor (HeaderWithTime blk)
forall v a b. AnchoredSeq v a b -> a
AF.anchor AnchoredFragment (HeaderWithTime blk)
curChain) (Compose [] ((,) peer) (AnchoredFragment (HeaderWithTime blk))
 -> (AnchoredFragment (HeaderWithTime blk),
     Compose [] ((,) peer) (AnchoredFragment (HeaderWithTime blk))))
-> Compose [] ((,) peer) (AnchoredFragment (HeaderWithTime blk))
-> (AnchoredFragment (HeaderWithTime blk),
    Compose [] ((,) peer) (AnchoredFragment (HeaderWithTime blk)))
forall a b. (a -> b) -> a -> b
$
  [(peer, AnchoredFragment (HeaderWithTime blk))]
-> Compose [] ((,) peer) (AnchoredFragment (HeaderWithTime blk))
forall {k} {k1} (f :: k -> *) (g :: k1 -> k) (a :: k1).
f (g a) -> Compose f g a
Compose [(peer, AnchoredFragment (HeaderWithTime blk))]
immutableTipSuffixes
  where
    immutableTip :: Point (HeaderWithTime blk)
immutableTip = AnchoredFragment (HeaderWithTime blk) -> Point (HeaderWithTime blk)
forall block. AnchoredFragment block -> Point block
AF.anchorPoint AnchoredFragment (HeaderWithTime blk)
curChain

    splitAfterImmutableTip :: (peer, AnchoredFragment (HeaderWithTime blk))
-> (peer, AnchoredFragment (HeaderWithTime blk))
splitAfterImmutableTip (peer
peer, AnchoredFragment (HeaderWithTime blk)
frag) =
      case AnchoredFragment (HeaderWithTime blk)
-> Point (HeaderWithTime blk)
-> Maybe
     (AnchoredFragment (HeaderWithTime blk),
      AnchoredFragment (HeaderWithTime blk))
forall block1 block2.
(HasHeader block1, HeaderHash block1 ~ HeaderHash block2) =>
AnchoredFragment block1
-> Point block2
-> Maybe (AnchoredFragment block1, AnchoredFragment block1)
AF.splitAfterPoint AnchoredFragment (HeaderWithTime blk)
frag Point (HeaderWithTime blk)
immutableTip of
        -- When there is no intersection, we assume the candidate fragment is
        -- empty and anchored at the immutable tip.
        -- See Note [CSJ truncates the candidate fragments].
        Maybe
  (AnchoredFragment (HeaderWithTime blk),
   AnchoredFragment (HeaderWithTime blk))
Nothing          -> (peer
peer, Int
-> AnchoredFragment (HeaderWithTime blk)
-> AnchoredFragment (HeaderWithTime blk)
forall v a b.
Anchorable v a b =>
Int -> AnchoredSeq v a b -> AnchoredSeq v a b
AF.takeOldest Int
0 AnchoredFragment (HeaderWithTime blk)
curChain)
        Just (AnchoredFragment (HeaderWithTime blk)
_, AnchoredFragment (HeaderWithTime blk)
suffix) -> (peer
peer, AnchoredFragment (HeaderWithTime blk)
suffix)

    immutableTipSuffixes :: [(peer, AnchoredFragment (HeaderWithTime blk))]
immutableTipSuffixes =
      ((peer, AnchoredFragment (HeaderWithTime blk))
 -> (peer, AnchoredFragment (HeaderWithTime blk)))
-> [(peer, AnchoredFragment (HeaderWithTime blk))]
-> [(peer, AnchoredFragment (HeaderWithTime blk))]
forall a b. (a -> b) -> [a] -> [b]
map (peer, AnchoredFragment (HeaderWithTime blk))
-> (peer, AnchoredFragment (HeaderWithTime blk))
splitAfterImmutableTip [(peer, AnchoredFragment (HeaderWithTime blk))]
candidates

-- Note [CSJ truncates the candidate fragments]
-- ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
--
-- Before CSJ, only rollback could cause truncation of a candidate fragment.
-- Truncation is a serious business to GDD because the LoE might have allowed
-- the selection to advance, based on the tips of the candidate fragments.
--
-- Truncating a candidate fragment risks moving the LoE back, which could be
-- earlier than the anchor of the latest selection. When rollbacks where the
-- only mechanism to truncate, it was fine to ignore candidate fragments that
-- don't intersect with the current selection. This could only happen if the
-- peer is rolling back more than k blocks, which is dishonest behavior.
--
-- With CSJ, however, the candidate fragments can recede without a rollback.
-- A former objector might be asked to jump back when it becomes a jumper again.
-- The jump point might still be a descendent of the immutable tip. But by the
-- time the jump is accepted, the immutable tip might have advanced, and the
-- candidate fragment of the otherwise honest peer might be ignored by GDD.
--
-- Therefore, at the moment, when there is no intersection with the current
-- selection, the GDD assumes that the candidate fragment is empty and anchored
-- at the immutable tip. It is the job of the ChainSync client to update the
-- candidate fragment so it intersects with the selection or to disconnect the
-- peer if no such fragment can be established.
--

data DensityBounds blk =
  DensityBounds {
    forall blk.
DensityBounds blk -> AnchoredFragment (HeaderWithTime blk)
clippedFragment :: AnchoredFragment (HeaderWithTime 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 (HeaderWithTime blk))]
  -> AnchoredFragment (HeaderWithTime blk)
  -> ([peer], [(peer, DensityBounds blk)])
densityDisconnect :: forall peer blk.
(Ord peer, LedgerSupportsProtocol blk) =>
GenesisWindow
-> SecurityParam
-> Map peer (ChainSyncState blk)
-> [(peer, AnchoredFragment (HeaderWithTime blk))]
-> AnchoredFragment (HeaderWithTime blk)
-> ([peer], [(peer, DensityBounds blk)])
densityDisconnect (GenesisWindow Word64
sgen) (SecurityParam NonZero Word64
k) Map peer (ChainSyncState blk)
states [(peer, AnchoredFragment (HeaderWithTime blk))]
candidateSuffixes AnchoredFragment (HeaderWithTime blk)
loeFrag =
  ([peer]
losingPeers, [(peer, DensityBounds blk)]
densityBounds)
  where
    densityBounds :: [(peer, DensityBounds blk)]
densityBounds = do
      (peer, candidateSuffix) <- [(peer, AnchoredFragment (HeaderWithTime blk))]
candidateSuffixes
      let (clippedFragment, _) =
            AF.splitAtSlot firstSlotAfterGenesisWindow candidateSuffix
      state <- maybeToList (states Map.!? peer)
      -- Skip peers that haven't sent any headers yet.
      -- They should be disconnected by timeouts instead.
      latestSlot <- toList (csLatestSlot state)
      let 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 =
               WithOrigin SlotNo -> WithOrigin SlotNo -> WithOrigin SlotNo
forall a. Ord a => a -> a -> a
max (AnchoredFragment (HeaderWithTime blk) -> WithOrigin SlotNo
forall block.
HasHeader block =>
AnchoredFragment block -> WithOrigin SlotNo
AF.headSlot AnchoredFragment (HeaderWithTime 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 =
            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 = 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 (HeaderWithTime blk) -> WithOrigin SlotNo
forall block.
HasHeader block =>
AnchoredFragment block -> WithOrigin SlotNo
AF.headSlot AnchoredFragment (HeaderWithTime blk)
clippedFragment)

          -- The number of blocks within the Genesis window we know with certainty
          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 (HeaderWithTime blk) -> Int
forall v a b. Anchorable v a b => AnchoredSeq v a b -> Int
AF.length AnchoredFragment (HeaderWithTime blk)
clippedFragment

          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 = Int -> Word64
forall a b. (Integral a, Num b) => a -> b
fromIntegral (AnchoredFragment (HeaderWithTime blk) -> Int
forall v a b. Anchorable v a b => AnchoredSeq v a b -> Int
AF.length AnchoredFragment (HeaderWithTime 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 = Word64
totalBlockCount Word64 -> Word64 -> Bool
forall a. Ord a => a -> a -> Bool
> NonZero Word64 -> Word64
forall a. NonZero a -> a
unNonZero NonZero Word64
k

      pure (peer, DensityBounds {clippedFragment, offersMoreThanK, lowerBound, upperBound, hasBlockAfter, latestSlot, 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 (HeaderWithTime blk)
clippedFragment = AnchoredFragment (HeaderWithTime 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
                             }) -> do
      (_peer1, DensityBounds {clippedFragment = frag1, offersMoreThanK, lowerBound = 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]
      --
      guard $ idling0 || not (AF.null frag0) || hasBlockAfter0
      -- ensure that the two peer fragments don't share any
      -- headers after the LoE
      guard $ AF.lastPoint frag0 /= AF.lastPoint 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.
      guard $ offersMoreThanK || lb0 == 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.
      guard $ lb1 >= (if idling0 then lb0 else 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.
      pure peer0

    loeIntersectionSlot :: WithOrigin SlotNo
loeIntersectionSlot = AnchoredFragment (HeaderWithTime blk) -> WithOrigin SlotNo
forall block.
HasHeader block =>
AnchoredFragment block -> WithOrigin SlotNo
AF.headSlot AnchoredFragment (HeaderWithTime 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 (HeaderWithTime blk)
curChain          :: AnchoredFragment (HeaderWithTime 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)