{-# LANGUAGE BlockArguments #-}
{-# LANGUAGE DataKinds #-}
{-# LANGUAGE DerivingStrategies #-}
{-# LANGUAGE NamedFieldPuns #-}
{-# LANGUAGE ScopedTypeVariables #-}
{-# LANGUAGE TypeApplications #-}

-- | This is a regression test for
-- <https://github.com/IntersectMBO/ouroboros-consensus/issues/1503>.
--
-- Concretely, consider @k = 1@ (security parameter), and a syncing Genesis
-- enabled.
--
-- Now consider the following block tree:
--
-- > G :> A >: C
-- >   :> B
--
-- Suppose that we have two peers, Peer 1 and Peer 2:
--
--  * Peer 1 first sends A, then C, then rolls back to A, and then idles.
--
--  * Peer 2 sends B and then idles.
--
-- In any possible interleaving (tested using IOSimPOR), the node should in the
-- end be caught-up and have selected C as it is the best chain.
--
-- To (somewhat) simplify the test setup boilerplate, we do not actually run
-- ChainSync and BlockFetch, but rather simulate their behavior by modifying the
-- ChainSync client state (eg candidate fragments) as well as adding blocks to
-- the ChainDB.
module Test.Consensus.Genesis.Tests.LoE.CaughtUp (tests) where

import Cardano.Ledger.BaseTypes (knownNonZeroBounded)
import Control.Monad (join)
import Control.Monad.Class.MonadTest (MonadTest (..))
import qualified Control.Monad.Class.MonadTimer.SI as SI
import Control.Monad.IOSim (exploreSimTrace, traceResult)
import Control.ResourceRegistry
import Control.Tracer (nullTracer)
import Data.Function (on)
import Data.Functor (void)
import Ouroboros.Consensus.Block
import Ouroboros.Consensus.Config
import Ouroboros.Consensus.Genesis.Governor (gddWatcher)
import Ouroboros.Consensus.HeaderValidation (HeaderWithTime)
import Ouroboros.Consensus.MiniProtocol.ChainSync.Client
  ( ChainSyncClientHandle (..)
  , ChainSyncClientHandleCollection (..)
  , ChainSyncState (..)
  , newChainSyncClientHandleCollection
  )
import Ouroboros.Consensus.MiniProtocol.ChainSync.Client.State
  ( ChainSyncJumpingState (..)
  , DisengagedInitState (..)
  )
import qualified Ouroboros.Consensus.Node.GSM as GSM
import Ouroboros.Consensus.Node.Genesis (setGetLoEFragment)
import Ouroboros.Consensus.Node.GsmState
import Ouroboros.Consensus.NodeId
import qualified Ouroboros.Consensus.Storage.ChainDB as ChainDB
import Ouroboros.Consensus.Storage.ChainDB.API (ChainDB)
import qualified Ouroboros.Consensus.Storage.ChainDB.API.Types.InvalidBlockPunishment as Punishment
import qualified Ouroboros.Consensus.Storage.ChainDB.Impl as ChainDB.Impl
import qualified Ouroboros.Consensus.Storage.ChainDB.Impl.Args as ChainDB
import Ouroboros.Consensus.Util.AnchoredFragment
  ( preferAnchoredCandidate
  )
import Ouroboros.Consensus.Util.IOLike
import Ouroboros.Consensus.Util.STM (forkLinkedWatcher)
import Ouroboros.Network.AnchoredFragment (AnchoredFragment)
import qualified Ouroboros.Network.AnchoredFragment as AF
import Test.QuickCheck
import Test.Tasty
import Test.Tasty.QuickCheck
import Test.Util.ChainDB
import Test.Util.Header
import Test.Util.Orphans.IOLike ()
import Test.Util.TestBlock

tests :: TestTree
tests :: TestTree
tests = String -> Property -> TestTree
forall a. Testable a => String -> a -> TestTree
testProperty String
"Select best chain when CaughtUp" Property
prop_test

prop_test :: Property
prop_test :: Property
prop_test =
  (ExplorationOptions -> ExplorationOptions)
-> (forall s. IOSim s Property)
-> (Maybe (SimTrace Property) -> SimTrace Property -> Property)
-> Property
forall a test.
Testable test =>
(ExplorationOptions -> ExplorationOptions)
-> (forall s. IOSim s a)
-> (Maybe (SimTrace a) -> SimTrace a -> test)
-> Property
exploreSimTrace ExplorationOptions -> ExplorationOptions
forall a. a -> a
id (IOSim s ()
forall (m :: * -> *). MonadTest m => m ()
exploreRaces IOSim s () -> IOSim s Property -> IOSim s Property
forall a b. IOSim s a -> IOSim s b -> IOSim s b
forall (f :: * -> *) a b. Applicative f => f a -> f b -> f b
*> IOSim s Property
forall (m :: * -> *). (IOLike m, MonadTimer m) => m Property
run) \Maybe (SimTrace Property)
_ SimTrace Property
tr ->
    case Bool -> SimTrace Property -> Either Failure Property
forall a. Bool -> SimTrace a -> Either Failure a
traceResult Bool
False SimTrace Property
tr of
      Right Property
prop -> Property
prop
      Left Failure
e -> String -> Bool -> Property
forall prop. Testable prop => String -> prop -> Property
counterexample (String
"Failure: " String -> String -> String
forall a. Semigroup a => a -> a -> a
<> Failure -> String
forall a. Show a => a -> String
show Failure
e) Bool
False

run :: forall m. (IOLike m, SI.MonadTimer m) => m Property
run :: forall (m :: * -> *). (IOLike m, MonadTimer m) => m Property
run = (ResourceRegistry m -> m Property) -> m Property
forall (m :: * -> *) a.
(MonadSTM m, MonadMask m, MonadThread m, HasCallStack) =>
(ResourceRegistry m -> m a) -> m a
withRegistry \ResourceRegistry m
registry -> do
  -- Setup
  varGsmState <- GsmState -> m (StrictTVar m GsmState)
forall (m :: * -> *) a.
(HasCallStack, MonadSTM m, NoThunks a) =>
a -> m (StrictTVar m a)
newTVarIO GsmState
PreSyncing
  varLoEFragment <- newTVarIO $ AF.Empty AF.AnchorGenesis
  varGetLoEFragment <-
    newTVarIO $
      pure $
        ChainDB.LoEEnabled $
          AF.Empty AF.AnchorGenesis
  setGetLoEFragment
    (readTVar varGsmState)
    (readTVar varLoEFragment)
    varGetLoEFragment

  chainDB <- openChainDB registry (join $ readTVarIO varGetLoEFragment)
  let addBlk = ChainDB m TestBlock
-> InvalidBlockPunishment m -> TestBlock -> m ()
forall (m :: * -> *) blk.
IOLike m =>
ChainDB m blk -> InvalidBlockPunishment m -> blk -> m ()
ChainDB.addBlock_ ChainDB m TestBlock
chainDB InvalidBlockPunishment m
forall (m :: * -> *). Applicative m => InvalidBlockPunishment m
Punishment.noPunishment

  chainSyncHandles <- atomically newChainSyncClientHandleCollection

  _ <-
    forkLinkedThread registry "GSM" $
      GSM.enterPreSyncing $
        mkGsmEntryPoints
          chainSyncHandles
          chainDB
          (atomically . writeTVar varGsmState)

  forkGDD
    registry
    chainSyncHandles
    chainDB
    (readTVar varGsmState)
    varLoEFragment

  -- Make sure that the ChainDB background thread, the GSM and the GDD are
  -- running (any positive amount should do).
  threadDelay 1

  -- Simulate receiving A, B, C and C being rolled back. In the real system,
  -- this would happen via ChainSync and BlockFetch.

  _ <- forkLinkedThread registry "Peer1" $ do
    -- First, let Peer1 connect, serving block A (without idling).
    let initialFrag =
          TopLevelConfig TestBlock
-> AnchoredFragment (Header TestBlock)
-> AnchoredSeq
     (WithOrigin SlotNo)
     (Anchor (HeaderWithTime TestBlock))
     (HeaderWithTime TestBlock)
forall blk.
(HasHeader (Header blk), Typeable blk, ImmutableEraParams blk) =>
TopLevelConfig blk
-> AnchoredFragment (Header blk)
-> AnchoredFragment (HeaderWithTime blk)
attachSlotTimeToFragment TopLevelConfig TestBlock
cfg (AnchoredFragment (Header TestBlock)
 -> AnchoredSeq
      (WithOrigin SlotNo)
      (Anchor (HeaderWithTime TestBlock))
      (HeaderWithTime TestBlock))
-> AnchoredFragment (Header TestBlock)
-> AnchoredSeq
     (WithOrigin SlotNo)
     (Anchor (HeaderWithTime TestBlock))
     (HeaderWithTime TestBlock)
forall a b. (a -> b) -> a -> b
$
            Anchor (Header TestBlock) -> AnchoredFragment (Header TestBlock)
forall v a b. Anchorable v a b => a -> AnchoredSeq v a b
AF.Empty Anchor (Header TestBlock)
forall block. Anchor block
AF.AnchorGenesis AnchoredFragment (Header TestBlock)
-> Header TestBlock -> AnchoredFragment (Header TestBlock)
forall v a b.
Anchorable v a b =>
AnchoredSeq v a b -> b -> AnchoredSeq v a b
AF.:> TestBlock -> Header TestBlock
forall blk. GetHeader blk => blk -> Header blk
getHeader TestBlock
blkA
    hdl <- atomically $ mkTestChainSyncClientHandle initialFrag
    atomically $ cschcAddHandle chainSyncHandles peer1 hdl
    addBlk blkA

    -- Then, send C.
    atomically $ modifyTVar (cschState hdl) $ \ChainSyncState TestBlock
s ->
      ChainSyncState
        { csCandidate :: AnchoredSeq
  (WithOrigin SlotNo)
  (Anchor (HeaderWithTime TestBlock))
  (HeaderWithTime TestBlock)
csCandidate = ChainSyncState TestBlock
-> AnchoredSeq
     (WithOrigin SlotNo)
     (Anchor (HeaderWithTime TestBlock))
     (HeaderWithTime TestBlock)
forall blk.
ChainSyncState blk -> AnchoredFragment (HeaderWithTime blk)
csCandidate ChainSyncState TestBlock
s AnchoredSeq
  (WithOrigin SlotNo)
  (Anchor (HeaderWithTime TestBlock))
  (HeaderWithTime TestBlock)
-> HeaderWithTime TestBlock
-> AnchoredSeq
     (WithOrigin SlotNo)
     (Anchor (HeaderWithTime TestBlock))
     (HeaderWithTime TestBlock)
forall v a b.
Anchorable v a b =>
AnchoredSeq v a b -> b -> AnchoredSeq v a b
AF.:> TopLevelConfig TestBlock
-> Header TestBlock -> HeaderWithTime TestBlock
forall blk.
(HasHeader (Header blk), ImmutableEraParams blk) =>
TopLevelConfig blk -> Header blk -> HeaderWithTime blk
attachSlotTime TopLevelConfig TestBlock
cfg (TestBlock -> Header TestBlock
forall blk. GetHeader blk => blk -> Header blk
getHeader TestBlock
blkC)
        , csLatestSlot :: StrictMaybe (WithOrigin SlotNo)
csLatestSlot = WithOrigin SlotNo -> StrictMaybe (WithOrigin SlotNo)
forall a. a -> StrictMaybe a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (WithOrigin SlotNo -> StrictMaybe (WithOrigin SlotNo))
-> WithOrigin SlotNo -> StrictMaybe (WithOrigin SlotNo)
forall a b. (a -> b) -> a -> b
$ SlotNo -> WithOrigin SlotNo
forall t. t -> WithOrigin t
NotOrigin (SlotNo -> WithOrigin SlotNo) -> SlotNo -> WithOrigin SlotNo
forall a b. (a -> b) -> a -> b
$ TestBlock -> SlotNo
forall b. HasHeader b => b -> SlotNo
blockSlot TestBlock
blkC
        , csIdling :: Bool
csIdling = ChainSyncState TestBlock -> Bool
forall blk. ChainSyncState blk -> Bool
csIdling ChainSyncState TestBlock
s
        }
    addBlk blkC

    -- Finally, roll back to the initial fragment and idle.
    atomically $ modifyTVar (cschState hdl) $ \ChainSyncState TestBlock
_s ->
      ChainSyncState
        { csCandidate :: AnchoredSeq
  (WithOrigin SlotNo)
  (Anchor (HeaderWithTime TestBlock))
  (HeaderWithTime TestBlock)
csCandidate = AnchoredSeq
  (WithOrigin SlotNo)
  (Anchor (HeaderWithTime TestBlock))
  (HeaderWithTime TestBlock)
initialFrag
        , csLatestSlot :: StrictMaybe (WithOrigin SlotNo)
csLatestSlot = WithOrigin SlotNo -> StrictMaybe (WithOrigin SlotNo)
forall a. a -> StrictMaybe a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (WithOrigin SlotNo -> StrictMaybe (WithOrigin SlotNo))
-> WithOrigin SlotNo -> StrictMaybe (WithOrigin SlotNo)
forall a b. (a -> b) -> a -> b
$ AnchoredSeq
  (WithOrigin SlotNo)
  (Anchor (HeaderWithTime TestBlock))
  (HeaderWithTime TestBlock)
-> WithOrigin SlotNo
forall block.
HasHeader block =>
AnchoredFragment block -> WithOrigin SlotNo
AF.headSlot AnchoredSeq
  (WithOrigin SlotNo)
  (Anchor (HeaderWithTime TestBlock))
  (HeaderWithTime TestBlock)
initialFrag
        , csIdling :: Bool
csIdling = Bool
True
        }

  _ <- forkLinkedThread registry "Peer2" $ do
    -- Let Peer2 connect and send B.
    hdl <-
      atomically $
        mkTestChainSyncClientHandle $
          attachSlotTimeToFragment cfg $
            AF.Empty AF.AnchorGenesis AF.:> getHeader blkB
    atomically $ cschcAddHandle chainSyncHandles peer2 hdl
    addBlk blkB

    -- Finally, idle.
    atomically $ modifyTVar (cschState hdl) $ \ChainSyncState TestBlock
s ->
      ChainSyncState
        { csCandidate :: AnchoredSeq
  (WithOrigin SlotNo)
  (Anchor (HeaderWithTime TestBlock))
  (HeaderWithTime TestBlock)
csCandidate = ChainSyncState TestBlock
-> AnchoredSeq
     (WithOrigin SlotNo)
     (Anchor (HeaderWithTime TestBlock))
     (HeaderWithTime TestBlock)
forall blk.
ChainSyncState blk -> AnchoredFragment (HeaderWithTime blk)
csCandidate ChainSyncState TestBlock
s
        , csLatestSlot :: StrictMaybe (WithOrigin SlotNo)
csLatestSlot = ChainSyncState TestBlock -> StrictMaybe (WithOrigin SlotNo)
forall blk. ChainSyncState blk -> StrictMaybe (WithOrigin SlotNo)
csLatestSlot ChainSyncState TestBlock
s
        , csIdling :: Bool
csIdling = Bool
True
        }

  -- Give time to process the new blocks (any positive amount should do).
  threadDelay 1

  gsmState <- atomically $ readTVar varGsmState
  tipPt <- atomically $ AF.headPoint <$> ChainDB.getCurrentChain chainDB
  pure $
    conjoin
      [ gsmState === CaughtUp
      , counterexample ("Selection tip is not C") $
          castPoint tipPt === blockPoint blkC
      ]
 where
  peer1, peer2 :: CoreNodeId
  peer1 :: CoreNodeId
peer1 = Word64 -> CoreNodeId
CoreNodeId Word64
1
  peer2 :: CoreNodeId
peer2 = Word64 -> CoreNodeId
CoreNodeId Word64
2

  blkA, blkB, blkC :: TestBlock
  blkA :: TestBlock
blkA = Word64 -> TestBlock
firstBlock Word64
1
  blkB :: TestBlock
blkB = Word64 -> TestBlock
firstBlock Word64
2
  blkC :: TestBlock
blkC = TestBlock -> TestBlock
successorBlock TestBlock
blkA

{-------------------------------------------------------------------------------
  Boilerplate for setting up the various test components
-------------------------------------------------------------------------------}

cfg :: TopLevelConfig TestBlock
cfg :: TopLevelConfig TestBlock
cfg =
  CodecConfig TestBlock
-> StorageConfig TestBlock
-> SecurityParam
-> GenesisWindow
-> TopLevelConfig TestBlock
forall ptype.
CodecConfig (TestBlockWith ptype)
-> StorageConfig (TestBlockWith ptype)
-> SecurityParam
-> GenesisWindow
-> TopLevelConfig (TestBlockWith ptype)
singleNodeTestConfigWith
    CodecConfig TestBlock
TestBlockCodecConfig
    StorageConfig TestBlock
TestBlockStorageConfig
    -- To make the test as simple as possible (otherwise, "saturating" the LoE
    -- requires more blocks).
    (NonZero Word64 -> SecurityParam
SecurityParam (NonZero Word64 -> SecurityParam)
-> NonZero Word64 -> SecurityParam
forall a b. (a -> b) -> a -> b
$ forall (n :: Natural) a.
(KnownNat n, 1 <= n, WithinBounds n a, Num a) =>
NonZero a
knownNonZeroBounded @1)
    -- large Genesis window to avoid disconnecting any peers
    (Word64 -> GenesisWindow
GenesisWindow Word64
20)

mkTestChainSyncClientHandle ::
  forall m.
  IOLike m =>
  AnchoredFragment (HeaderWithTime TestBlock) ->
  STM m (ChainSyncClientHandle m TestBlock)
mkTestChainSyncClientHandle :: forall (m :: * -> *).
IOLike m =>
AnchoredSeq
  (WithOrigin SlotNo)
  (Anchor (HeaderWithTime TestBlock))
  (HeaderWithTime TestBlock)
-> STM m (ChainSyncClientHandle m TestBlock)
mkTestChainSyncClientHandle AnchoredSeq
  (WithOrigin SlotNo)
  (Anchor (HeaderWithTime TestBlock))
  (HeaderWithTime TestBlock)
frag = do
  varState <-
    ChainSyncState TestBlock
-> STM m (StrictTVar m (ChainSyncState TestBlock))
forall (m :: * -> *) a.
(HasCallStack, MonadSTM m, NoThunks a) =>
a -> STM m (StrictTVar m a)
newTVar
      ChainSyncState
        { csCandidate :: AnchoredSeq
  (WithOrigin SlotNo)
  (Anchor (HeaderWithTime TestBlock))
  (HeaderWithTime TestBlock)
csCandidate = AnchoredSeq
  (WithOrigin SlotNo)
  (Anchor (HeaderWithTime TestBlock))
  (HeaderWithTime TestBlock)
frag
        , csIdling :: Bool
csIdling = Bool
False
        , csLatestSlot :: StrictMaybe (WithOrigin SlotNo)
csLatestSlot = WithOrigin SlotNo -> StrictMaybe (WithOrigin SlotNo)
forall a. a -> StrictMaybe a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (WithOrigin SlotNo -> StrictMaybe (WithOrigin SlotNo))
-> WithOrigin SlotNo -> StrictMaybe (WithOrigin SlotNo)
forall a b. (a -> b) -> a -> b
$ AnchoredSeq
  (WithOrigin SlotNo)
  (Anchor (HeaderWithTime TestBlock))
  (HeaderWithTime TestBlock)
-> WithOrigin SlotNo
forall block.
HasHeader block =>
AnchoredFragment block -> WithOrigin SlotNo
AF.headSlot AnchoredSeq
  (WithOrigin SlotNo)
  (Anchor (HeaderWithTime TestBlock))
  (HeaderWithTime TestBlock)
frag
        }
  varJumping <- newTVar $ Disengaged DisengagedDone
  varJumpInfo <- newTVar Nothing
  pure
    ChainSyncClientHandle
      { cschState = varState
      , -- Irrelevant for this test (as we don't actually run ChainSync).
        cschOnGsmStateChanged = \GsmState
_gsmState Time
_curTime -> () -> STM m ()
forall a. a -> STM m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure ()
      , cschGDDKill = pure ()
      , cschJumping = varJumping
      , cschJumpInfo = varJumpInfo
      }

openChainDB ::
  forall m.
  IOLike m =>
  ResourceRegistry m ->
  ChainDB.GetLoEFragment m TestBlock ->
  m (ChainDB m TestBlock)
openChainDB :: forall (m :: * -> *).
IOLike m =>
ResourceRegistry m
-> GetLoEFragment m TestBlock -> m (ChainDB m TestBlock)
openChainDB ResourceRegistry m
registry GetLoEFragment m TestBlock
getLoEFragment = do
  chainDbArgs <- do
    mcdbNodeDBs <- m (NodeDBs (StrictTMVar m MockFS))
forall (m :: * -> *).
MonadSTM m =>
m (NodeDBs (StrictTMVar m MockFS))
emptyNodeDBs
    let mcdbTopLevelConfig = TopLevelConfig TestBlock
cfg
        configureLoE ChainDbArgs f m TestBlock
a =
          ChainDbArgs f m TestBlock
a
            { ChainDB.cdbsArgs =
                (ChainDB.cdbsArgs a){ChainDB.cdbsLoE = getLoEFragment}
            }
    pure $
      configureLoE $
        fromMinimalChainDbArgs
          MinimalChainDbArgs
            { mcdbChunkInfo = mkTestChunkInfo mcdbTopLevelConfig
            , mcdbInitLedger = testInitExtLedger
            , mcdbRegistry = registry
            , mcdbTopLevelConfig
            , mcdbNodeDBs
            }
  (_, (chainDB, ChainDB.Impl.Internal{ChainDB.Impl.intAddBlockRunner})) <-
    allocate
      registry
      (\ResourceId
_ -> ChainDbArgs Identity m TestBlock
-> Bool -> m (ChainDB m TestBlock, Internal m TestBlock)
forall (m :: * -> *) blk.
(IOLike m, LedgerSupportsProtocol blk,
 BlockSupportsDiffusionPipelining blk, InspectLedger blk,
 HasHardForkHistory blk, ConvertRawHash blk,
 SerialiseDiskConstraints blk, HasCallStack,
 LedgerSupportsLedgerDB blk) =>
Complete ChainDbArgs m blk
-> Bool -> m (ChainDB m blk, Internal m blk)
ChainDB.Impl.openDBInternal ChainDbArgs Identity m TestBlock
chainDbArgs Bool
False)
      (ChainDB.closeDB . fst)
  _ <- forkLinkedThread registry "AddBlockRunner" intAddBlockRunner
  pure chainDB

mkGsmEntryPoints ::
  forall m.
  (IOLike m, SI.MonadTimer m) =>
  ChainSyncClientHandleCollection CoreNodeId m TestBlock ->
  ChainDB m TestBlock ->
  (GsmState -> m ()) ->
  GSM.GsmEntryPoints m
mkGsmEntryPoints :: forall (m :: * -> *).
(IOLike m, MonadTimer m) =>
ChainSyncClientHandleCollection CoreNodeId m TestBlock
-> ChainDB m TestBlock -> (GsmState -> m ()) -> GsmEntryPoints m
mkGsmEntryPoints ChainSyncClientHandleCollection CoreNodeId m TestBlock
varChainSyncHandles ChainDB m TestBlock
chainDB GsmState -> m ()
writeGsmState =
  (AnchoredFragment (Header TestBlock)
 -> AnchoredFragment (Header TestBlock),
 Tracer m (TraceGsmEvent (AnchoredFragment (Header TestBlock))))
-> GsmView
     m
     CoreNodeId
     (AnchoredFragment (Header TestBlock))
     (ChainSyncState TestBlock)
-> GsmEntryPoints m
forall (m :: * -> *) upstreamPeer selection tracedSelection
       candidate.
(MonadDelay m, MonadTimer m) =>
(selection -> tracedSelection,
 Tracer m (TraceGsmEvent tracedSelection))
-> GsmView m upstreamPeer selection candidate -> GsmEntryPoints m
GSM.realGsmEntryPoints
    (AnchoredFragment (Header TestBlock)
-> AnchoredFragment (Header TestBlock)
forall a. a -> a
id, Tracer m (TraceGsmEvent (AnchoredFragment (Header TestBlock)))
forall (m :: * -> *) a. Applicative m => Tracer m a
nullTracer)
    GSM.GsmView
      { AnchoredFragment (Header TestBlock)
-> ChainSyncState TestBlock -> CandidateVersusSelection
forall {h :: * -> *}.
(HeaderHash (h TestBlock) ~ TestHash, HasHeader (h TestBlock),
 GetHeader1 h, Typeable h) =>
AnchoredFragment (h TestBlock)
-> ChainSyncState TestBlock -> CandidateVersusSelection
candidateOverSelection :: forall {h :: * -> *}.
(HeaderHash (h TestBlock) ~ TestHash, HasHeader (h TestBlock),
 GetHeader1 h, Typeable h) =>
AnchoredFragment (h TestBlock)
-> ChainSyncState TestBlock -> CandidateVersusSelection
candidateOverSelection :: AnchoredFragment (Header TestBlock)
-> ChainSyncState TestBlock -> CandidateVersusSelection
GSM.candidateOverSelection
      , peerIsIdle :: ChainSyncState TestBlock -> Bool
GSM.peerIsIdle = ChainSyncState TestBlock -> Bool
forall blk. ChainSyncState blk -> Bool
csIdling
      , equivalent :: AnchoredFragment (Header TestBlock)
-> AnchoredFragment (Header TestBlock) -> Bool
GSM.equivalent = Point (Header TestBlock) -> Point (Header TestBlock) -> Bool
forall a. Eq a => a -> a -> Bool
(==) (Point (Header TestBlock) -> Point (Header TestBlock) -> Bool)
-> (AnchoredFragment (Header TestBlock)
    -> Point (Header TestBlock))
-> AnchoredFragment (Header TestBlock)
-> AnchoredFragment (Header TestBlock)
-> Bool
forall b c a. (b -> b -> c) -> (a -> b) -> a -> a -> c
`on` AnchoredFragment (Header TestBlock) -> Point (Header TestBlock)
forall block.
HasHeader block =>
AnchoredFragment block -> Point block
AF.headPoint
      , getChainSyncStates :: STM m (Map CoreNodeId (StrictTVar m (ChainSyncState TestBlock)))
GSM.getChainSyncStates = (ChainSyncClientHandle m TestBlock
 -> StrictTVar m (ChainSyncState TestBlock))
-> Map CoreNodeId (ChainSyncClientHandle m TestBlock)
-> Map CoreNodeId (StrictTVar m (ChainSyncState TestBlock))
forall a b. (a -> b) -> Map CoreNodeId a -> Map CoreNodeId b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap ChainSyncClientHandle m TestBlock
-> StrictTVar m (ChainSyncState TestBlock)
forall (m :: * -> *) blk.
ChainSyncClientHandle m blk -> StrictTVar m (ChainSyncState blk)
cschState (Map CoreNodeId (ChainSyncClientHandle m TestBlock)
 -> Map CoreNodeId (StrictTVar m (ChainSyncState TestBlock)))
-> STM m (Map CoreNodeId (ChainSyncClientHandle m TestBlock))
-> STM m (Map CoreNodeId (StrictTVar m (ChainSyncState TestBlock)))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> ChainSyncClientHandleCollection CoreNodeId m TestBlock
-> STM m (Map CoreNodeId (ChainSyncClientHandle m TestBlock))
forall peer (m :: * -> *) blk.
ChainSyncClientHandleCollection peer m blk
-> STM m (Map peer (ChainSyncClientHandle m blk))
cschcMap ChainSyncClientHandleCollection CoreNodeId m TestBlock
varChainSyncHandles
      , getCurrentSelection :: STM m (AnchoredFragment (Header TestBlock))
GSM.getCurrentSelection = ChainDB m TestBlock -> STM m (AnchoredFragment (Header TestBlock))
forall (m :: * -> *) blk.
ChainDB m blk -> STM m (AnchoredFragment (Header blk))
ChainDB.getCurrentChain ChainDB m TestBlock
chainDB
      , -- Make sure that we stay in CaughtUp for the duration of the test once we
        -- have entered it.
        minCaughtUpDuration :: NominalDiffTime
GSM.minCaughtUpDuration = NominalDiffTime
10 -- seconds
      , GsmState -> m ()
writeGsmState :: GsmState -> m ()
writeGsmState :: GsmState -> m ()
GSM.writeGsmState
      , -- Not interesting for this test.
        antiThunderingHerd :: Maybe StdGen
GSM.antiThunderingHerd = Maybe StdGen
forall a. Maybe a
Nothing
      , setCaughtUpPersistentMark :: Bool -> m ()
GSM.setCaughtUpPersistentMark = \Bool
_ -> () -> m ()
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure ()
      , durationUntilTooOld :: Maybe (AnchoredFragment (Header TestBlock) -> m DurationFromNow)
GSM.durationUntilTooOld = Maybe (AnchoredFragment (Header TestBlock) -> m DurationFromNow)
forall a. Maybe a
Nothing
      , isHaaSatisfied :: STM m Bool
GSM.isHaaSatisfied = Bool -> STM m Bool
forall a. a -> STM m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure Bool
True
      }
 where
  candidateOverSelection :: AnchoredFragment (h TestBlock)
-> ChainSyncState TestBlock -> CandidateVersusSelection
candidateOverSelection AnchoredFragment (h TestBlock)
selection ChainSyncState TestBlock
candidateState =
    case AnchoredFragment (h TestBlock)
-> AnchoredSeq
     (WithOrigin SlotNo)
     (Anchor (HeaderWithTime TestBlock))
     (HeaderWithTime TestBlock)
-> Maybe (Point (h TestBlock))
forall block1 block2.
(HasHeader block1, HasHeader block2,
 HeaderHash block1 ~ HeaderHash block2) =>
AnchoredFragment block1
-> AnchoredFragment block2 -> Maybe (Point block1)
AF.intersectionPoint AnchoredFragment (h TestBlock)
selection AnchoredSeq
  (WithOrigin SlotNo)
  (Anchor (HeaderWithTime TestBlock))
  (HeaderWithTime TestBlock)
candFrag of
      Maybe (Point (h TestBlock))
Nothing -> CandidateVersusSelection
GSM.CandidateDoesNotIntersect
      Just{} ->
        -- precondition requires intersection
        Bool -> CandidateVersusSelection
GSM.WhetherCandidateIsBetter (Bool -> CandidateVersusSelection)
-> Bool -> CandidateVersusSelection
forall a b. (a -> b) -> a -> b
$
          BlockConfig TestBlock
-> AnchoredFragment (h TestBlock)
-> AnchoredSeq
     (WithOrigin SlotNo)
     (Anchor (HeaderWithTime TestBlock))
     (HeaderWithTime TestBlock)
-> Bool
forall blk (h :: * -> *) (h' :: * -> *).
(BlockSupportsProtocol blk, HasCallStack, GetHeader1 h,
 GetHeader1 h', HeaderHash (h blk) ~ HeaderHash (h' blk),
 HasHeader (h blk), HasHeader (h' blk)) =>
BlockConfig blk
-> AnchoredFragment (h blk) -> AnchoredFragment (h' blk) -> Bool
preferAnchoredCandidate (TopLevelConfig TestBlock -> BlockConfig TestBlock
forall blk. TopLevelConfig blk -> BlockConfig blk
configBlock TopLevelConfig TestBlock
cfg) AnchoredFragment (h TestBlock)
selection AnchoredSeq
  (WithOrigin SlotNo)
  (Anchor (HeaderWithTime TestBlock))
  (HeaderWithTime TestBlock)
candFrag
   where
    candFrag :: AnchoredSeq
  (WithOrigin SlotNo)
  (Anchor (HeaderWithTime TestBlock))
  (HeaderWithTime TestBlock)
candFrag = ChainSyncState TestBlock
-> AnchoredSeq
     (WithOrigin SlotNo)
     (Anchor (HeaderWithTime TestBlock))
     (HeaderWithTime TestBlock)
forall blk.
ChainSyncState blk -> AnchoredFragment (HeaderWithTime blk)
csCandidate ChainSyncState TestBlock
candidateState

forkGDD ::
  forall m.
  IOLike m =>
  ResourceRegistry m ->
  ChainSyncClientHandleCollection CoreNodeId m TestBlock ->
  ChainDB m TestBlock ->
  STM m GsmState ->
  StrictTVar m (AnchoredFragment (HeaderWithTime TestBlock)) ->
  m ()
forkGDD :: forall (m :: * -> *).
IOLike m =>
ResourceRegistry m
-> ChainSyncClientHandleCollection CoreNodeId m TestBlock
-> ChainDB m TestBlock
-> STM m GsmState
-> StrictTVar
     m
     (AnchoredSeq
        (WithOrigin SlotNo)
        (Anchor (HeaderWithTime TestBlock))
        (HeaderWithTime TestBlock))
-> m ()
forkGDD ResourceRegistry m
registry ChainSyncClientHandleCollection CoreNodeId m TestBlock
varChainSyncHandles ChainDB m TestBlock
chainDB STM m GsmState
getGsmState StrictTVar
  m
  (AnchoredSeq
     (WithOrigin SlotNo)
     (Anchor (HeaderWithTime TestBlock))
     (HeaderWithTime TestBlock))
varLoEFrag =
  m (Thread m Void) -> m ()
forall (f :: * -> *) a. Functor f => f a -> f ()
void (m (Thread m Void) -> m ()) -> m (Thread m Void) -> m ()
forall a b. (a -> b) -> a -> b
$
    ResourceRegistry m
-> String
-> Watcher
     m
     (GDDTrigger (GDDStateView m TestBlock CoreNodeId))
     (GDDTrigger
        (Map CoreNodeId (StrictMaybe (WithOrigin SlotNo), Bool)))
-> m (Thread m Void)
forall (m :: * -> *) a fp.
(IOLike m, Eq fp, HasCallStack) =>
ResourceRegistry m -> String -> Watcher m a fp -> m (Thread m Void)
forkLinkedWatcher ResourceRegistry m
registry String
"GDD" (Watcher
   m
   (GDDTrigger (GDDStateView m TestBlock CoreNodeId))
   (GDDTrigger
      (Map CoreNodeId (StrictMaybe (WithOrigin SlotNo), Bool)))
 -> m (Thread m Void))
-> Watcher
     m
     (GDDTrigger (GDDStateView m TestBlock CoreNodeId))
     (GDDTrigger
        (Map CoreNodeId (StrictMaybe (WithOrigin SlotNo), Bool)))
-> m (Thread m Void)
forall a b. (a -> b) -> a -> b
$
      TopLevelConfig TestBlock
-> Tracer m (TraceGDDEvent CoreNodeId TestBlock)
-> ChainDB m TestBlock
-> DiffTime
-> STM m GsmState
-> STM m (Map CoreNodeId (ChainSyncClientHandle m TestBlock))
-> StrictTVar
     m
     (AnchoredSeq
        (WithOrigin SlotNo)
        (Anchor (HeaderWithTime TestBlock))
        (HeaderWithTime TestBlock))
-> Watcher
     m
     (GDDTrigger (GDDStateView m TestBlock CoreNodeId))
     (GDDTrigger
        (Map CoreNodeId (StrictMaybe (WithOrigin SlotNo), Bool)))
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
     (GDDTrigger (GDDStateView m blk peer))
     (GDDTrigger (Map peer (StrictMaybe (WithOrigin SlotNo), Bool)))
gddWatcher
        TopLevelConfig TestBlock
cfg
        Tracer m (TraceGDDEvent CoreNodeId TestBlock)
forall (m :: * -> *) a. Applicative m => Tracer m a
nullTracer
        ChainDB m TestBlock
chainDB
        (DiffTime
0 :: DiffTime) -- no rate limiting
        STM m GsmState
getGsmState
        (ChainSyncClientHandleCollection CoreNodeId m TestBlock
-> STM m (Map CoreNodeId (ChainSyncClientHandle m TestBlock))
forall peer (m :: * -> *) blk.
ChainSyncClientHandleCollection peer m blk
-> STM m (Map peer (ChainSyncClientHandle m blk))
cschcMap ChainSyncClientHandleCollection CoreNodeId m TestBlock
varChainSyncHandles)
        StrictTVar
  m
  (AnchoredSeq
     (WithOrigin SlotNo)
     (Anchor (HeaderWithTime TestBlock))
     (HeaderWithTime TestBlock))
varLoEFrag