From a39cd950cec5903461ab7efbc03b054fad343e15 Mon Sep 17 00:00:00 2001 From: Nicolas Frisby Date: Thu, 30 Nov 2023 08:30:37 -0800 Subject: [PATCH] consensus-test: include near-future headers in ChainSync client test This now exercises the real ChainSync InFutureCheck. --- ouroboros-consensus/ouroboros-consensus.cabal | 2 + .../Ouroboros/Consensus/Util/Time.hs | 15 +- .../MiniProtocol/ChainSync/Client.hs | 379 ++++++++++++++++-- 3 files changed, 356 insertions(+), 40 deletions(-) diff --git a/ouroboros-consensus/ouroboros-consensus.cabal b/ouroboros-consensus/ouroboros-consensus.cabal index 253582952e..34f809d940 100644 --- a/ouroboros-consensus/ouroboros-consensus.cabal +++ b/ouroboros-consensus/ouroboros-consensus.cabal @@ -484,6 +484,7 @@ test-suite consensus-test build-depends: , async , base + , base-deriving-via , cardano-binary , cardano-crypto-class , cardano-slotting @@ -505,6 +506,7 @@ test-suite consensus-test , ouroboros-network-protocols:{ouroboros-network-protocols, testlib} , QuickCheck , quickcheck-state-machine + , quiet , random , serialise , si-timers diff --git a/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Util/Time.hs b/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Util/Time.hs index 6fce425df9..b43a7f99e7 100644 --- a/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Util/Time.hs +++ b/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Util/Time.hs @@ -1,11 +1,22 @@ module Ouroboros.Consensus.Util.Time ( - -- Conversions - nominalDelay + multipleNominalDelay + , nominalDelay , secondsToNominalDiffTime ) where import Data.Time (DiffTime, NominalDiffTime) +{------------------------------------------------------------------------------- + Operations +-------------------------------------------------------------------------------} + +-- | Multiply a 'NominalDiffTime' by an integer +-- +-- The right conversions to use are somewhat tricky. The key fact is that +-- 'fromIntegral' interprets its argument as seconds. +multipleNominalDelay :: Integral a => NominalDiffTime -> a -> NominalDiffTime +multipleNominalDelay dur i = dur * fromIntegral i + {------------------------------------------------------------------------------- Conversions -------------------------------------------------------------------------------} diff --git a/ouroboros-consensus/test/consensus-test/Test/Consensus/MiniProtocol/ChainSync/Client.hs b/ouroboros-consensus/test/consensus-test/Test/Consensus/MiniProtocol/ChainSync/Client.hs index 2528049d9e..3a45492bc3 100644 --- a/ouroboros-consensus/test/consensus-test/Test/Consensus/MiniProtocol/ChainSync/Client.hs +++ b/ouroboros-consensus/test/consensus-test/Test/Consensus/MiniProtocol/ChainSync/Client.hs @@ -1,4 +1,8 @@ {-# LANGUAGE BangPatterns #-} +{-# LANGUAGE DeriveFunctor #-} +{-# LANGUAGE DeriveGeneric #-} +{-# LANGUAGE DerivingStrategies #-} +{-# LANGUAGE DerivingVia #-} {-# LANGUAGE FlexibleContexts #-} {-# LANGUAGE GADTs #-} {-# LANGUAGE LambdaCase #-} @@ -6,6 +10,7 @@ {-# LANGUAGE RankNTypes #-} {-# LANGUAGE ScopedTypeVariables #-} {-# LANGUAGE TypeApplications #-} + -- | Tests for the chain sync client. -- -- The chain sync client is a stateful component that tracks the chain of an @@ -22,23 +27,53 @@ -- them to be thrown based on the mock state changes (exceptions such as -- "fork is deep", "up-stream node asked for an invalid rollback", etc.). -- +-- The client's (simulated) wall-clock matters in this test because the +-- ChainSync client has special handling for headers that arrive before the +-- wall-clock reaches the onset of the header's claimed slot, which is +-- inevitable even with only honest peers due to non-global clocks +-- drifting/etc. This test advances time in a way that is unrealistic but does +-- allow for some headers to arrive early (but not so early that the client +-- disconnects from the server). +-- +-- The approach to the clocks is as follows. A logical clock drives the whole +-- test; it ticks along the naturals. Each tick causes the local and upstream +-- chains to update and that's the primary content of the whole test. However, +-- the /first/ thing that happens at the start of each logical tick is the +-- client's simulated wall-clock advances (via a single 'threadDelay' call) to +-- the onset of the greatest slot involved in any of that logical tick's +-- server-side chain updates /less/ the randomly-chosen local clock skew. Thus, +-- if the greatest header involved in some logical tick is part of an upstream +-- chain update, then it will arrive as a future header (but only near-future, +-- never far-future). (Client-side updates are also handled, but slightly +-- differently; see the code comments.) Finally, recall that the @io-sim@ layer +-- means those delays happen nearly instantaneously with respect to the real +-- world wall-clock. module Test.Consensus.MiniProtocol.ChainSync.Client (tests) where import Cardano.Crypto.DSIGN.Mock -import Control.Monad (unless, void, when) +import Cardano.Slotting.Slot (WithOrigin (..)) +import Control.Monad (forM_, unless, void, when) import Control.Monad.Class.MonadThrow (Handler (..), catches) +import Control.Monad.Class.MonadTime (MonadTime, getCurrentTime) import Control.Monad.IOSim (runSimOrThrow) import Control.Tracer (contramap, contramapM, nullTracer) +import Data.DerivingVia (InstantiatedAt (InstantiatedAt)) import Data.List (intercalate) +import qualified Data.Map.Merge.Strict as Map import qualified Data.Map.Strict as Map import Data.Maybe (isJust) +import Data.Semigroup (Max (Max), getMax) import qualified Data.Set as Set +import Data.Time (diffUTCTime) import Data.Typeable +import GHC.Generics (Generic) import Network.TypedProtocol.Channel import Network.TypedProtocol.Driver.Simple import Ouroboros.Consensus.Block import Ouroboros.Consensus.BlockchainTime import Ouroboros.Consensus.Config +import Ouroboros.Consensus.Fragment.InFuture (ClockSkew, + clockSkewInSeconds, unClockSkew) import qualified Ouroboros.Consensus.HardFork.History as HardFork import Ouroboros.Consensus.HeaderStateHistory (HeaderStateHistory (..)) @@ -46,8 +81,9 @@ import qualified Ouroboros.Consensus.HeaderStateHistory as HeaderStateHistory import Ouroboros.Consensus.Ledger.Abstract import Ouroboros.Consensus.Ledger.Extended hiding (ledgerState) import Ouroboros.Consensus.MiniProtocol.ChainSync.Client -import Ouroboros.Consensus.MiniProtocol.ChainSync.Client.InFutureCheck as InFutureCheck -import Ouroboros.Consensus.Node.NetworkProtocolVersion (NodeToNodeVersion) +import qualified Ouroboros.Consensus.MiniProtocol.ChainSync.Client.InFutureCheck as InFutureCheck +import Ouroboros.Consensus.Node.NetworkProtocolVersion + (NodeToNodeVersion) import Ouroboros.Consensus.Node.ProtocolInfo import Ouroboros.Consensus.NodeId import Ouroboros.Consensus.Protocol.BFT @@ -59,6 +95,8 @@ import Ouroboros.Consensus.Util.IOLike import Ouroboros.Consensus.Util.ResourceRegistry import Ouroboros.Consensus.Util.STM (Fingerprint (..), WithFingerprint (..)) +import Ouroboros.Consensus.Util.Time (multipleNominalDelay, + nominalDelay) import Ouroboros.Network.AnchoredFragment (AnchoredFragment) import qualified Ouroboros.Network.AnchoredFragment as AF import Ouroboros.Network.Block (getTipPoint) @@ -75,6 +113,7 @@ import Ouroboros.Network.Protocol.ChainSync.PipelineDecision (pipelineDecisionLowHighMark) import Ouroboros.Network.Protocol.ChainSync.Server import Ouroboros.Network.Protocol.ChainSync.Type (ChainSync) +import Quiet (Quiet (..)) import Test.QuickCheck import Test.Tasty import Test.Tasty.QuickCheck @@ -103,13 +142,16 @@ tests = testGroup "ChainSyncClient" -------------------------------------------------------------------------------} prop_chainSync :: ChainSyncClientSetup -> Property -prop_chainSync ChainSyncClientSetup { +prop_chainSync testSetup@ChainSyncClientSetup { securityParam , clientUpdates , serverUpdates , startTick , invalidBlocks + , clientSlowBy } = + tabulate "TickArrivalTimeStats" [show (tickArrivalTimeStats traceEvents)] $ + counterexample (prettyChainSyncClientSetup testSetup) $ counterexample ("Client chain: " <> ppChain finalClientChain <> "\n" <> "Server chain: " <> ppChain finalServerChain <> "\n" <> @@ -119,23 +161,25 @@ prop_chainSync ChainSyncClientSetup { -- it, but not the other way around: we don't check whether a situation -- has occured where an exception should have been thrown, but wasn't. case mbResult of - Just (Right (ForkTooDeep intersection _ _)) -> + Just (ClientFinished (ForkTooDeep intersection _ _)) -> label "ForkTooDeep" $ counterexample ("ForkTooDeep intersection: " <> ppPoint intersection) $ not (withinFragmentBounds intersection clientFragment) - Just (Right (NoMoreIntersection (Our ourTip) (Their theirTip))) -> + Just (ClientFinished (NoMoreIntersection (Our ourTip) (Their theirTip))) -> label "NoMoreIntersection" $ counterexample ("NoMoreIntersection ourHead: " <> ppPoint (getTipPoint ourTip) <> ", theirHead: " <> ppPoint (getTipPoint theirTip)) $ not (clientFragment `forksWithinK` syncedFragment) - Just (Right (RolledBackPastIntersection intersection _ _)) -> + Just (ClientFinished (RolledBackPastIntersection intersection _ _)) -> label "RolledBackPastIntersection" $ counterexample ("RolledBackPastIntersection intersection: " <> ppPoint intersection) $ not (withinFragmentBounds intersection syncedFragment) - Just (Right result) -> + Just (ClientFinished result) -> counterexample ("Terminated with result: " ++ show result) False - Just (Left ex) -> + Just (ClientThrew ex) -> counterexample ("Exception: " ++ displayException ex) False + Just (ClientSelectedFutureTip ft) -> + counterexample ("Client selected future tip: " ++ show ft) False Nothing -> counterexample "Synced fragment not a suffix of the server chain" (syncedFragment `isSuffixOf` finalServerChain) .&&. @@ -154,6 +198,7 @@ prop_chainSync ChainSyncClientSetup { , traceEvents } = runSimOrThrow $ runChainSync + (slotLengthTenthsToClockSkew clientSlowBy) securityParam clientUpdates serverUpdates @@ -222,7 +267,7 @@ newtype InvalidBlocks = InvalidBlocks { getInvalidBlocks :: Schedule TestHash } deriving (Show) -type TraceEvent = (Tick, Either +type TraceEvent = (Tick, RelativeTime, Either (TraceChainSyncClientEvent TestBlock) (TraceSendRecv (ChainSync (Header TestBlock) (Point TestBlock) (Tip TestBlock)))) @@ -230,7 +275,7 @@ data ChainSyncOutcome = ChainSyncOutcome { finalClientChain :: Chain TestBlock , finalServerChain :: Chain TestBlock , syncedFragment :: AnchoredFragment TestBlock - , mbResult :: Maybe (Either ChainSyncClientException ChainSyncClientResult) + , mbResult :: Maybe ChainSyncClientTestResult , traceEvents :: [TraceEvent] } @@ -258,18 +303,42 @@ data ChainSyncOutcome = ChainSyncOutcome { -- Note that updates that are scheduled before the time at which we start -- syncing help generate different chains to start syncing from. runChainSync - :: forall m. IOLike m - => SecurityParam + :: forall m. (IOLike m, MonadTime m) + => ClockSkew + -> SecurityParam -> ClientUpdates -> ServerUpdates -> InvalidBlocks -> Tick -- ^ Start chain syncing at this time -> m ChainSyncOutcome -runChainSync securityParam (ClientUpdates clientUpdates) +runChainSync skew securityParam (ClientUpdates clientUpdates) (ServerUpdates serverUpdates) (InvalidBlocks invalidBlocks) startSyncingAt = withRegistry $ \registry -> do + clientSystemTime <- do + initialIoSimClockValue <- getCurrentTime + pure SystemTime { + systemTimeWait = pure () + , systemTimeCurrent = do + now <- getCurrentTime + -- Subtracting the initial @io-sim@ wall clock to create this + -- 'RelativeTime' causes the test to behave as if the local + -- node and the peer were invoked when the "true" wall clock + -- (which the server's clock happens to equal) is at exactly + -- the onset of Slot 0. + pure $ RelativeTime $ + (now `diffUTCTime` initialIoSimClockValue) + - + unClockSkew skew + } + let _ = clientSystemTime :: SystemTime m + varCurrentLogicalTick <- uncheckedNewTVarM (Tick 0) + let clockUpdates :: Schedule NewMaxSlot + clockUpdates = + mkClockUpdates + (ClientUpdates clientUpdates) + (ServerUpdates serverUpdates) -- Set up the client varCandidates <- uncheckedNewTVarM Map.empty @@ -284,10 +353,11 @@ runChainSync securityParam (ClientUpdates clientUpdates) (tracer, getTrace) <- do (tracer', getTrace) <- recordingTracerTVar - let foo ev = do - now <- readTVarIO varCurrentLogicalTick - pure (now, ev) - pure (contramapM foo tracer', getTrace) + let pairWithNow ev = do + logicalNow <- readTVarIO varCurrentLogicalTick + now <- systemTimeCurrent clientSystemTime + pure (logicalNow, now, ev) + pure (contramapM pairWithNow tracer', getTrace) let chainSyncTracer = contramap Left tracer protocolTracer = contramap Right tracer @@ -319,13 +389,11 @@ runChainSync securityParam (ClientUpdates clientUpdates) pure $ WithFingerprint isInvalidBlock fp } - headerInFutureCheck :: InFutureCheck.HeaderInFutureCheck m blk - headerInFutureCheck = InFutureCheck.HeaderInFutureCheck - { InFutureCheck.handleHeaderArrival = \_judgment -> pure Nothing - , InFutureCheck.judgeHeaderArrival = \_cfg _lst _arrival -> pure () - , InFutureCheck.proxyArrival = Proxy - , InFutureCheck.recordHeaderArrival = \_hdr -> pure () - } + headerInFutureCheck :: InFutureCheck.HeaderInFutureCheck m TestBlock + headerInFutureCheck = + InFutureCheck.realHeaderInFutureCheck skew clientSystemTime + -- Note that this tests passes in the exact difference between the + -- client's and server's clock as the tolerable clock skew. client :: StrictTVar m (AnchoredFragment (Header TestBlock)) -> Consensus ChainSyncClientPipelined @@ -347,6 +415,21 @@ runChainSync securityParam (ClientUpdates clientUpdates) (Tip TestBlock) m () server = chainSyncServerExample () varChainProducerState getHeader + let advanceWallClockForTick :: Tick -> m () + advanceWallClockForTick tick = do + doTick clockUpdates tick $ \case + [newMaxSlot] -> do + let target = case newMaxSlot of + NewMaxClientSlot slot -> toOnset slot + NewMaxServerSlot slot -> toSkewedOnset slot + + NewMaxClientAndServerSlot cslot sslot -> + toOnset cslot `max` toSkewedOnset sslot + now <- systemTimeCurrent clientSystemTime + threadDelay $ nominalDelay $ target `diffRelTime` now + + _ -> error "impossible! bad mkClockUpdates" + -- Do scheduled updates of the client and server chains let updateChainsDuringTick :: Tick -> m () updateChainsDuringTick tick = do @@ -360,6 +443,9 @@ runChainSync securityParam (ClientUpdates clientUpdates) whenJust (Map.lookup tick (getSchedule invalidBlocks)) $ atomically . modifyTVar varKnownInvalid . Set.union . Set.fromList + -- TODO interleave the client and server chain update + -- applications in a more interesting way? + -- Client doTick clientUpdates tick $ \chainUpdates -> atomically $ modifyTVar varClientState $ updateClientState chainUpdates @@ -398,21 +484,55 @@ runChainSync securityParam (ClientUpdates clientUpdates) result <- runPipelinedPeer protocolTracer codecChainSyncId clientChannel $ chainSyncClientPeerPipelined $ client varCandidate - atomically $ writeTVar varClientResult (Just (Right result)) + atomically $ writeTVar varClientResult (Just (ClientFinished result)) return () `catchAlsoLinked` \ex -> do - atomically $ writeTVar varClientResult (Just (Left ex)) + atomically $ writeTVar varClientResult (Just (ClientThrew ex)) -- Rethrow, but it will be ignored anyway. throwIO ex void $ forkLinkedThread registry "ChainSyncServer" $ runPeer nullTracer codecChainSyncId serverChannel (chainSyncServerPeer server) + -- If the candidate's tip's slot's onset is ahead of the local wall-clock + -- (which is skewed by 'clientSlowBy'), then the ChainSync client + -- mishandled a block from the future. + let checkTipTime :: m () + checkTipTime = do + now <- systemTimeCurrent clientSystemTime + candidates <- atomically $ + readTVar varCandidates >>= traverse readTVar + forM_ candidates $ \candidate -> do + let p = castPoint $ AF.headPoint candidate :: Point TestBlock + case pointSlot p of + Origin -> pure () + At slot -> when (now < toOnset slot) $ do + atomically $ writeTVar varClientResult $ Just + $ ClientSelectedFutureTip $ FutureTip { + ftNow = now + , ftPoint = (toOnset slot, p) + } + do let loop tick = do + -- first update the clocks + advanceWallClockForTick tick + atomically $ writeTVar varCurrentLogicalTick tick + + -- then do the messages updateChainsDuringTick tick when (tick == startSyncingAt) $ initiateChainSync - when (tick < finalTick) $ loop (tick + 1) + + -- check the invariants before advancing the clock again + -- + -- This is not a perfect check, since the server's chain may have + -- violated the invariant ephemerally (ie due to a subsequent + -- rollback during the same logical tick). However, other + -- QuickCheck seeds/counterexamples should trigger such a bug in + -- a non-ephemeral way. + checkTipTime + + when (tick < finalTick) $ loop (tick + 1) loop (Tick 1) -- This delay seems enough to let all threads finish their final work. @@ -437,6 +557,12 @@ runChainSync securityParam (ClientUpdates clientUpdates) where k = maxRollbacks securityParam + toSkewedOnset :: SlotNo -> RelativeTime + toSkewedOnset slot = + let RelativeTime onset = toOnset slot + in + RelativeTime $ onset - unClockSkew skew + doTick :: Schedule a -> Tick -> ([a] -> m ()) -> m () doTick sched tick kont = whenJust (Map.lookup tick (getSchedule sched)) kont @@ -478,6 +604,24 @@ runChainSync securityParam (ClientUpdates clientUpdates) , Handler $ \(ExceptionInLinkedThread _ ex) -> throwIO ex `catch` handler ] +-- | See 'ClientSelectedFutureTip' +data FutureTip = FutureTip { + ftNow :: RelativeTime + -- ^ when the header was selected prematurely + , ftPoint :: (RelativeTime, Point TestBlock) + -- ^ point of the header that was selected prematurely, and the + -- 'RelativeTime' of its slot's onset + } + deriving (Show) + +data ChainSyncClientTestResult = + ClientFinished !ChainSyncClientResult + -- ^ This is only a property failure if the result was unjustified. + | ClientSelectedFutureTip !FutureTip + -- ^ This is always a property failure. + | ClientThrew !ChainSyncClientException + -- ^ This is only a property failure if the exception was unjustified. + updateClientState :: [ChainUpdate] -> Chain TestBlock -> Chain TestBlock updateClientState chainUpdates chain = case Chain.applyChainUpdates (toChainUpdates chainUpdates) chain of @@ -540,7 +684,24 @@ slotLength :: SlotLength slotLength = slotLengthFromSec $ toEnum slotLengthInSeconds slotLengthInSeconds :: Int -slotLengthInSeconds = 20 +slotLengthInSeconds = 10 + +-- | The onset of the slot +toOnset :: SlotNo -> RelativeTime +toOnset slot = RelativeTime $ + multipleNominalDelay + (getSlotLength slotLength) + (unSlotNo slot) + +-- | Tenths of a slot length +-- +-- This adds some fractionality to the test without over-complicating it. +newtype SlotLengthTenths = SlotLengthTenths Int + deriving (Show) + +slotLengthTenthsToClockSkew :: SlotLengthTenths -> ClockSkew +slotLengthTenthsToClockSkew (SlotLengthTenths tenths) = + clockSkewInSeconds $ (toEnum slotLengthInSeconds * toEnum tenths) / 10 -- | Bundle dependent arguments for test generation data ChainSyncClientSetup = ChainSyncClientSetup @@ -553,7 +714,12 @@ data ChainSyncClientSetup = ChainSyncClientSetup -- ^ Depends on 'clientUpdates' and 'serverUpdates' , invalidBlocks :: InvalidBlocks -- ^ Blocks that are discovered to be invalid. + , clientSlowBy :: SlotLengthTenths + -- ^ The server's clock minus the client's clock. + -- + -- This is also passed to the code-under-test as the tolerable clock skew. } + deriving (Show) instance Arbitrary ChainSyncClientSetup where arbitrary = do @@ -576,17 +742,21 @@ instance Arbitrary ChainSyncClientSetup where ] invalidBlocks <- InvalidBlocks <$> (genSchedule =<< shuffle trapBlocks) + clientSlowBy <- SlotLengthTenths <$> choose (0, 50) + return ChainSyncClientSetup { securityParam , clientUpdates , serverUpdates , startTick , invalidBlocks + , clientSlowBy } shrink cscs@ChainSyncClientSetup { clientUpdates , serverUpdates , startTick + , clientSlowBy } = -- We don't shrink 'securityParam' because the updates depend on it @@ -621,18 +791,21 @@ instance Arbitrary ChainSyncClientSetup where , lastTick (getServerUpdates serverUpdates) - 1 ] , startTick' <- [1..min startTick maxStartTick] + ] <> + [ cscs { clientSlowBy = SlotLengthTenths y } + | let SlotLengthTenths x = clientSlowBy + , y <- shrink x ] -instance Show ChainSyncClientSetup where - show ChainSyncClientSetup { - securityParam - , clientUpdates - , serverUpdates - , startTick - , invalidBlocks - } = unlines +prettyChainSyncClientSetup :: ChainSyncClientSetup -> String +prettyChainSyncClientSetup testSetup = + unlines [ "ChainSyncClientSetup:" , "securityParam: " <> show (maxRollbacks securityParam) + , "clientSlowBy: " <> show (unClockSkew skew) + , "--" + , "clockUpdates:" + , condense (mkClockUpdates clientUpdates serverUpdates) <> "--" , "clientUpdates:" , condense (getClientUpdates clientUpdates) <> "--" , "serverUpdates:" @@ -641,6 +814,19 @@ instance Show ChainSyncClientSetup where , "invalidBlocks: " , condense (getInvalidBlocks invalidBlocks) ] + where + -- if you add a field to this pattern to avoid warnings, add it below too + ChainSyncClientSetup _ _ _ _ _ _dummy = testSetup + ChainSyncClientSetup { + securityParam + , clientSlowBy + , clientUpdates + , serverUpdates + , startTick + , invalidBlocks + } = testSetup + + skew = slotLengthTenthsToClockSkew clientSlowBy -- | Remove client updates that happen at a tick after the tick in which the -- last server updates happened. @@ -674,6 +860,64 @@ genUpdateSchedule genUpdateSchedule updateBehavior securityParam = genChainUpdates updateBehavior securityParam 10 >>= genSchedule +data NewMaxSlot = + NewMaxClientSlot SlotNo + -- ^ the client's chain reaches a new greatest slot + | NewMaxServerSlot SlotNo + -- ^ the server's chain reaches a new greatest slot + | NewMaxClientAndServerSlot SlotNo SlotNo + -- ^ both the client and the server's chain reach a new greatest slot, + -- respectively + deriving (Show) + +instance Condense NewMaxSlot where + condense = \case + NewMaxClientSlot slot -> "c" <> condense slot <> "|s_" + NewMaxServerSlot slot -> "c_|s" <> condense slot + + NewMaxClientAndServerSlot cslot sslot -> + "c" <> condense cslot <> "|s" <> condense sslot + +-- | The schedule of when the the client and server chains reach a new greatest +-- slot, respectively. +-- +-- The resulting schedule has exactly one entry per tick in the map (ie no +-- simultaneity). Moreover, it's monotonic within the client and within the +-- server, but not necessarily in their union. +-- +-- We need to track them separately because the client selecting a block in a +-- slot implies the local clock has reached surpassed that onset, whereas the +-- server doing so does not. +mkClockUpdates :: ClientUpdates -> ServerUpdates -> Schedule NewMaxSlot +mkClockUpdates = \(ClientUpdates cupds) (ServerUpdates supds) -> + Schedule + $ Map.map ((:[])) + $ Map.merge + (Map.mapMissing $ \_ -> NewMaxClientSlot) + (Map.mapMissing $ \_ -> NewMaxServerSlot) + (Map.zipWithMatched $ \_ -> NewMaxClientAndServerSlot) + (newMaxes cupds) + (newMaxes supds) + where + newMaxes :: Schedule ChainUpdate -> Map.Map Tick SlotNo + newMaxes = + makeMonotonic + . Map.mapMaybe (fmap getMax . foldMap maxSlot) + . getSchedule + + maxSlot :: ChainUpdate -> Maybe (Max SlotNo) + maxSlot = foldMap (Just . Max . blockSlot) . \case + AddBlock b -> [b] + SwitchFork _ bs -> bs + + makeMonotonic :: (Eq k, Ord v) => Map.Map k v -> Map.Map k v + makeMonotonic mp = Map.fromAscList $ case Map.toAscList mp of + [] -> [] + (k0, x) : xs -> (k0, x) : go x xs + go acc = \case + [] -> [] + (k, x) : xs -> if x > acc then (k, x) : go x xs else go acc xs + {------------------------------------------------------------------------------- Pretty-printing -------------------------------------------------------------------------------} @@ -695,6 +939,65 @@ ppBlocks :: Point TestBlock -> [TestBlock] -> String ppBlocks a bs = ppPoint a <> " ] " <> intercalate " :> " (map ppBlock bs) ppTraceEvent :: TraceEvent -> String -ppTraceEvent (Tick n, ev) = show n <> " | " <> case ev of +ppTraceEvent (Tick n, RelativeTime t, ev) = show (n, t) <> " | " <> case ev of Left cl -> "Client: " <> show cl Right pt -> "Protocol: " <> show pt + +{------------------------------------------------------------------------------- + Classifying examples +-------------------------------------------------------------------------------} + +data TickArrivalTimeStats a = OnlyNotEarly_SomeEarly { + onlyNotEarlyTATS :: !a + -- ^ Logical ticks in which some headers are arriving but none are from the + -- future + , someEarlyTATS :: !a + -- ^ Logical ticks in which some headers are arriving from the future + } + deriving (Functor, Generic) + deriving (Show) via (Quiet (TickArrivalTimeStats a)) + deriving (Monoid, Semigroup) via + (InstantiatedAt Generic (TickArrivalTimeStats a)) + +data ZOM = Zero | One | Many + deriving (Show) + +sizeZOM :: Set.Set a -> ZOM +sizeZOM x = case Set.size x of + 0 -> Zero + 1 -> One + _ -> Many -- NB negatives are impossible + +tickArrivalTimeStats :: [TraceEvent] -> TickArrivalTimeStats ZOM +tickArrivalTimeStats events = + fmap sizeZOM $ + OnlyNotEarly_SomeEarly { + onlyNotEarlyTATS = onlyNotEarly `Set.difference` someEarly + , someEarlyTATS = someEarly + } + where + -- if you add a field to this pattern to avoid warnings, add it below too + OnlyNotEarly_SomeEarly _ _dummy = tickArrivalTimes events + OnlyNotEarly_SomeEarly { + onlyNotEarlyTATS = onlyNotEarly + , someEarlyTATS = someEarly + } = tickArrivalTimes events + +-- | WARNING 'onlyNotEarlyTATS' is instead merely @someNotEarlyTATS@ in this +-- codomain: it might overlap with the 'someEarlyTATs' field +tickArrivalTimes :: [TraceEvent] -> TickArrivalTimeStats (Set.Set Tick) +tickArrivalTimes = foldMap $ \case + (n, now, Left (TraceDownloadedHeader hdr)) -> + let onset = toOnset (blockSlot hdr) + thisTick = Set.singleton n + in + if now < onset + then OnlyNotEarly_SomeEarly { + onlyNotEarlyTATS = Set.empty + , someEarlyTATS = thisTick + } + else OnlyNotEarly_SomeEarly { + onlyNotEarlyTATS = thisTick + , someEarlyTATS = Set.empty + } + _ -> mempty