{-# LANGUAGE ExistentialQuantification #-}
{-# LANGUAGE FlexibleContexts #-}
{-# LANGUAGE LambdaCase #-}
{-# LANGUAGE NamedFieldPuns #-}
{-# LANGUAGE PatternSynonyms #-}
{-# LANGUAGE RankNTypes #-}
{-# LANGUAGE ScopedTypeVariables #-}
{-# LANGUAGE StandaloneDeriving #-}
{-# LANGUAGE TypeApplications #-}
{-# LANGUAGE UndecidableInstances #-}

module Test.ThreadNet.General (
    PropGeneralArgs (..)
  , calcFinalIntersectionDepth
  , prop_general
  , prop_general_semisync
  , prop_inSync
  , runTestNetwork
    -- * TestConfig
  , TestConfig (..)
  , TestConfigB (..)
  , TestConfigMB (..)
  , truncateNodeJoinPlan
  , truncateNodeRestarts
  , truncateNodeTopology
    -- * Expected CannotForge
  , noExpectedCannotForges
    -- * Re-exports
  , ForgeEbbEnv (..)
  , TestOutput (..)
  , noCalcMessageDelay
  , plainTestNodeInitialization
  ) where

import           Control.Exception (assert)
import           Control.Monad (guard)
import           Control.Monad.IOSim (runSimOrThrow, setCurrentTime)
import           Control.Tracer (nullTracer)
import qualified Data.Map.Strict as Map
import           Data.Set (Set)
import qualified Data.Set as Set
import           Data.Word (Word64)
import           GHC.Stack (HasCallStack)
import           Ouroboros.Consensus.Block
import qualified Ouroboros.Consensus.Block.Abstract as BA
import qualified Ouroboros.Consensus.BlockchainTime as BTime
import           Ouroboros.Consensus.Config.SecurityParam
import           Ouroboros.Consensus.Ledger.Extended (ExtValidationError)
import           Ouroboros.Consensus.Node.NetworkProtocolVersion
import           Ouroboros.Consensus.Node.ProtocolInfo
import           Ouroboros.Consensus.Node.Run
import           Ouroboros.Consensus.NodeId
import           Ouroboros.Consensus.Protocol.Abstract (LedgerView)
import           Ouroboros.Consensus.Protocol.LeaderSchedule
import qualified Ouroboros.Consensus.Storage.ChainDB as ChainDB
import           Ouroboros.Consensus.TypeFamilyWrappers
import           Ouroboros.Consensus.Util.Condense
import           Ouroboros.Consensus.Util.Enclose (pattern FallingEdge)
import           Ouroboros.Consensus.Util.IOLike
import           Ouroboros.Consensus.Util.Orphans ()
import           Ouroboros.Consensus.Util.RedundantConstraints
import qualified Ouroboros.Network.Mock.Chain as MockChain
import qualified System.FS.Sim.MockFS as Mock
import           System.FS.Sim.MockFS (MockFS)
import           Test.QuickCheck
import           Test.ThreadNet.Network
import           Test.ThreadNet.TxGen
import           Test.ThreadNet.Util
import           Test.ThreadNet.Util.NodeJoinPlan
import           Test.ThreadNet.Util.NodeRestarts
import           Test.ThreadNet.Util.NodeTopology
import           Test.ThreadNet.Util.Seed
import           Test.Util.HardFork.Future (Future)
import           Test.Util.Orphans.Arbitrary ()
import           Test.Util.Orphans.IOLike ()
import           Test.Util.Orphans.NoThunks ()
import           Test.Util.QuickCheck
import           Test.Util.Range
import           Test.Util.Shrink (andId, dropId)
import           Test.Util.Slots (NumSlots (..))
import           Test.Util.Time (dawnOfTime)

{-------------------------------------------------------------------------------
  Configuring tests
-------------------------------------------------------------------------------}

-- | Test configuration that does not depend on the block
--
-- The primary motivation for separating this type from 'TestConfigB' and
-- 'TestConfigMB' is so that the instance @'Arbitrary' 'TestConfig'@ can be
-- reused by multiple tests using different @blk@s: /as/ /of/ /yet/, no block
-- (each of which realizes a ledger-protocol combination) influences the
-- validity of these data.
data TestConfig = TestConfig
  { TestConfig -> Seed
initSeed     :: Seed
  , TestConfig -> NodeTopology
nodeTopology :: NodeTopology
  , TestConfig -> NumCoreNodes
numCoreNodes :: NumCoreNodes
  , TestConfig -> NumSlots
numSlots     :: NumSlots
    -- ^ TODO generate in function of @k@
  }
  deriving (Int -> TestConfig -> ShowS
[TestConfig] -> ShowS
TestConfig -> String
(Int -> TestConfig -> ShowS)
-> (TestConfig -> String)
-> ([TestConfig] -> ShowS)
-> Show TestConfig
forall a.
(Int -> a -> ShowS) -> (a -> String) -> ([a] -> ShowS) -> Show a
$cshowsPrec :: Int -> TestConfig -> ShowS
showsPrec :: Int -> TestConfig -> ShowS
$cshow :: TestConfig -> String
show :: TestConfig -> String
$cshowList :: [TestConfig] -> ShowS
showList :: [TestConfig] -> ShowS
Show)

truncateNodeJoinPlan ::
    NodeJoinPlan -> NumCoreNodes -> (NumSlots, NumSlots) -> NodeJoinPlan
truncateNodeJoinPlan :: NodeJoinPlan
-> NumCoreNodes -> (NumSlots, NumSlots) -> NodeJoinPlan
truncateNodeJoinPlan
  (NodeJoinPlan Map CoreNodeId SlotNo
m) (NumCoreNodes Word64
n') (NumSlots Word64
t, NumSlots Word64
t') =
    Map CoreNodeId SlotNo -> NodeJoinPlan
NodeJoinPlan (Map CoreNodeId SlotNo -> NodeJoinPlan)
-> Map CoreNodeId SlotNo -> NodeJoinPlan
forall a b. (a -> b) -> a -> b
$
    -- scale by t' / t
    (SlotNo -> SlotNo)
-> Map CoreNodeId SlotNo -> Map CoreNodeId SlotNo
forall a b k. (a -> b) -> Map k a -> Map k b
Map.map (\(SlotNo Word64
i) -> Word64 -> SlotNo
SlotNo (Word64 -> SlotNo) -> Word64 -> SlotNo
forall a b. (a -> b) -> a -> b
$ (Word64
i Word64 -> Word64 -> Word64
forall a. Num a => a -> a -> a
* Word64
t') Word64 -> Word64 -> Word64
forall a. Integral a => a -> a -> a
`div` Word64
t) (Map CoreNodeId SlotNo -> Map CoreNodeId SlotNo)
-> Map CoreNodeId SlotNo -> Map CoreNodeId SlotNo
forall a b. (a -> b) -> a -> b
$
    -- discard discarded nodes
    (CoreNodeId -> SlotNo -> Bool)
-> Map CoreNodeId SlotNo -> Map CoreNodeId SlotNo
forall k a. (k -> a -> Bool) -> Map k a -> Map k a
Map.filterWithKey (\(CoreNodeId Word64
nid) SlotNo
_ -> Word64
nid Word64 -> Word64 -> Bool
forall a. Ord a => a -> a -> Bool
< Word64
n') (Map CoreNodeId SlotNo -> Map CoreNodeId SlotNo)
-> Map CoreNodeId SlotNo -> Map CoreNodeId SlotNo
forall a b. (a -> b) -> a -> b
$
    Map CoreNodeId SlotNo
m

truncateNodeTopology :: NodeTopology -> NumCoreNodes -> NodeTopology
truncateNodeTopology :: NodeTopology -> NumCoreNodes -> NodeTopology
truncateNodeTopology (NodeTopology Map CoreNodeId (Set CoreNodeId)
m) (NumCoreNodes Word64
n') =
    Map CoreNodeId (Set CoreNodeId) -> NodeTopology
NodeTopology (Map CoreNodeId (Set CoreNodeId) -> NodeTopology)
-> Map CoreNodeId (Set CoreNodeId) -> NodeTopology
forall a b. (a -> b) -> a -> b
$ (CoreNodeId -> Set CoreNodeId -> Bool)
-> Map CoreNodeId (Set CoreNodeId)
-> Map CoreNodeId (Set CoreNodeId)
forall k a. (k -> a -> Bool) -> Map k a -> Map k a
Map.filterWithKey (\(CoreNodeId Word64
i) Set CoreNodeId
_ -> Word64
i Word64 -> Word64 -> Bool
forall a. Ord a => a -> a -> Bool
< Word64
n') Map CoreNodeId (Set CoreNodeId)
m

truncateNodeRestarts :: NodeRestarts -> NumSlots -> NodeRestarts
truncateNodeRestarts :: NodeRestarts -> NumSlots -> NodeRestarts
truncateNodeRestarts (NodeRestarts Map SlotNo (Map CoreNodeId NodeRestart)
m) (NumSlots Word64
t) =
    Map SlotNo (Map CoreNodeId NodeRestart) -> NodeRestarts
NodeRestarts (Map SlotNo (Map CoreNodeId NodeRestart) -> NodeRestarts)
-> Map SlotNo (Map CoreNodeId NodeRestart) -> NodeRestarts
forall a b. (a -> b) -> a -> b
$ (SlotNo -> Map CoreNodeId NodeRestart -> Bool)
-> Map SlotNo (Map CoreNodeId NodeRestart)
-> Map SlotNo (Map CoreNodeId NodeRestart)
forall k a. (k -> a -> Bool) -> Map k a -> Map k a
Map.filterWithKey (\(SlotNo Word64
s) Map CoreNodeId NodeRestart
_ -> Word64
s Word64 -> Word64 -> Bool
forall a. Ord a => a -> a -> Bool
< Word64
t) Map SlotNo (Map CoreNodeId NodeRestart)
m

instance Arbitrary TestConfig where
  arbitrary :: Gen TestConfig
arbitrary = do
      Seed
initSeed     <- Gen Seed
forall a. Arbitrary a => Gen a
arbitrary

      NumCoreNodes
numCoreNodes <- Gen NumCoreNodes
forall a. Arbitrary a => Gen a
arbitrary
      NodeTopology
nodeTopology <- HasCallStack => NumCoreNodes -> Gen NodeTopology
NumCoreNodes -> Gen NodeTopology
genNodeTopology NumCoreNodes
numCoreNodes

      NumSlots
numSlots     <- Gen NumSlots
forall a. Arbitrary a => Gen a
arbitrary
      TestConfig -> Gen TestConfig
forall a. a -> Gen a
forall (f :: * -> *) a. Applicative f => a -> f a
pure TestConfig
        { Seed
initSeed :: Seed
initSeed :: Seed
initSeed
        , NodeTopology
nodeTopology :: NodeTopology
nodeTopology :: NodeTopology
nodeTopology
        , NumCoreNodes
numCoreNodes :: NumCoreNodes
numCoreNodes :: NumCoreNodes
numCoreNodes
        , NumSlots
numSlots :: NumSlots
numSlots :: NumSlots
numSlots
        }

  shrink :: TestConfig -> [TestConfig]
shrink TestConfig
    { Seed
initSeed :: TestConfig -> Seed
initSeed :: Seed
initSeed
    , NodeTopology
nodeTopology :: TestConfig -> NodeTopology
nodeTopology :: NodeTopology
nodeTopology
    , NumCoreNodes
numCoreNodes :: TestConfig -> NumCoreNodes
numCoreNodes :: NumCoreNodes
numCoreNodes
    , NumSlots
numSlots :: TestConfig -> NumSlots
numSlots :: NumSlots
numSlots
    } =
      [TestConfig] -> [TestConfig]
forall a. [a] -> [a]
dropId ([TestConfig] -> [TestConfig]) -> [TestConfig] -> [TestConfig]
forall a b. (a -> b) -> a -> b
$
      [ TestConfig
          { Seed
initSeed :: Seed
initSeed :: Seed
initSeed
          , nodeTopology :: NodeTopology
nodeTopology = NodeTopology
top'
          , numCoreNodes :: NumCoreNodes
numCoreNodes = NumCoreNodes
n'
          , numSlots :: NumSlots
numSlots     = NumSlots
t'
          }
      | NumCoreNodes
n'             <- (NumCoreNodes -> [NumCoreNodes]) -> NumCoreNodes -> [NumCoreNodes]
forall a. (a -> [a]) -> a -> [a]
andId NumCoreNodes -> [NumCoreNodes]
forall a. Arbitrary a => a -> [a]
shrink NumCoreNodes
numCoreNodes
      , NumSlots
t'             <- (NumSlots -> [NumSlots]) -> NumSlots -> [NumSlots]
forall a. (a -> [a]) -> a -> [a]
andId NumSlots -> [NumSlots]
forall a. Arbitrary a => a -> [a]
shrink NumSlots
numSlots
      , let adjustedTop :: NodeTopology
adjustedTop = NodeTopology -> NumCoreNodes -> NodeTopology
truncateNodeTopology NodeTopology
nodeTopology NumCoreNodes
n'
      , NodeTopology
top'           <- (NodeTopology -> [NodeTopology]) -> NodeTopology -> [NodeTopology]
forall a. (a -> [a]) -> a -> [a]
andId NodeTopology -> [NodeTopology]
shrinkNodeTopology NodeTopology
adjustedTop
      ]

{-------------------------------------------------------------------------------
  Configuring tests for a specific block type
-------------------------------------------------------------------------------}

-- | Test configuration that depends on the block (incl the ledger and\/or
-- protocol) but not on the monad
--
-- Some fields do not explicitly involve the @blk@ type, but their semantics
-- (at least their validity) does depend on the semantics of the underlying
-- ledger and\/or protocol. For example, 'nodeJoinPlan' is here instead of in
-- 'TestConfig' because different blocks can withstand different degrees of
-- absence/lateness. And 'epochSize' is here because eg the Byron ledger
-- assumes a fixed epoch size of @10k@. And so on.
data TestConfigB blk = TestConfigB
  { forall blk. TestConfigB blk -> Maybe (ForgeEbbEnv blk)
forgeEbbEnv  :: Maybe (ForgeEbbEnv blk)
  , forall blk. TestConfigB blk -> Future
future       :: Future
  , forall blk. TestConfigB blk -> CalcMessageDelay blk
messageDelay :: CalcMessageDelay blk
  , forall blk. TestConfigB blk -> NodeJoinPlan
nodeJoinPlan :: NodeJoinPlan
  , forall blk. TestConfigB blk -> NodeRestarts
nodeRestarts :: NodeRestarts
  , forall blk. TestConfigB blk -> TxGenExtra blk
txGenExtra   :: TxGenExtra blk
  , forall blk.
TestConfigB blk -> (NodeToNodeVersion, BlockNodeToNodeVersion blk)
version      :: (NodeToNodeVersion, BlockNodeToNodeVersion blk)
  }

deriving instance (Show (TxGenExtra blk), Show (BlockNodeToNodeVersion blk))
                => Show (TestConfigB blk)

-- | Test configuration that depends on the block and the monad
--
-- The primary motivation for separating this type from 'TestConfigB' is so
-- that 'TestConfigB' can occur in contexts (such as in 'PropGeneralArgs') for
-- which the @m@ parameter is irrelevant and hence unknown.
data TestConfigMB m blk = TestConfigMB
  { forall (m :: * -> *) blk.
TestConfigMB m blk -> CoreNodeId -> TestNodeInitialization m blk
nodeInfo :: CoreNodeId -> TestNodeInitialization m blk
  , forall (m :: * -> *) blk.
TestConfigMB m blk -> Maybe (m (RekeyM m blk))
mkRekeyM :: Maybe (m (RekeyM m blk))
    -- ^ 'runTestNetwork' immediately runs this action once in order to
    -- initialize an 'RekeyM' value that it then reuses throughout the test
  }

{-------------------------------------------------------------------------------
   Running tests
-------------------------------------------------------------------------------}

-- | Thin wrapper around 'runThreadNetwork'
--
runTestNetwork ::
  forall blk.
     ( RunNode blk
     , TxGen blk
     , TracingConstraints blk
     , HasCallStack
     )
  => TestConfig
  -> TestConfigB blk
  -> (forall m. IOLike m => TestConfigMB m blk)
  -> TestOutput blk
runTestNetwork :: forall blk.
(RunNode blk, TxGen blk, TracingConstraints blk, HasCallStack) =>
TestConfig
-> TestConfigB blk
-> (forall (m :: * -> *). IOLike m => TestConfigMB m blk)
-> TestOutput blk
runTestNetwork TestConfig
  { NumCoreNodes
numCoreNodes :: TestConfig -> NumCoreNodes
numCoreNodes :: NumCoreNodes
numCoreNodes
  , NumSlots
numSlots :: TestConfig -> NumSlots
numSlots :: NumSlots
numSlots
  , NodeTopology
nodeTopology :: TestConfig -> NodeTopology
nodeTopology :: NodeTopology
nodeTopology
  , Seed
initSeed :: TestConfig -> Seed
initSeed :: Seed
initSeed
  } TestConfigB
  { Maybe (ForgeEbbEnv blk)
forgeEbbEnv :: forall blk. TestConfigB blk -> Maybe (ForgeEbbEnv blk)
forgeEbbEnv :: Maybe (ForgeEbbEnv blk)
forgeEbbEnv
  , Future
future :: forall blk. TestConfigB blk -> Future
future :: Future
future
  , CalcMessageDelay blk
messageDelay :: forall blk. TestConfigB blk -> CalcMessageDelay blk
messageDelay :: CalcMessageDelay blk
messageDelay
  , NodeJoinPlan
nodeJoinPlan :: forall blk. TestConfigB blk -> NodeJoinPlan
nodeJoinPlan :: NodeJoinPlan
nodeJoinPlan
  , NodeRestarts
nodeRestarts :: forall blk. TestConfigB blk -> NodeRestarts
nodeRestarts :: NodeRestarts
nodeRestarts
  , TxGenExtra blk
txGenExtra :: forall blk. TestConfigB blk -> TxGenExtra blk
txGenExtra :: TxGenExtra blk
txGenExtra
  , version :: forall blk.
TestConfigB blk -> (NodeToNodeVersion, BlockNodeToNodeVersion blk)
version = (NodeToNodeVersion
networkVersion, BlockNodeToNodeVersion blk
blockVersion)
  }
    forall (m :: * -> *). IOLike m => TestConfigMB m blk
mkTestConfigMB
  = (forall s. IOSim s (TestOutput blk)) -> TestOutput blk
forall a. (forall s. IOSim s a) -> a
runSimOrThrow ((forall s. IOSim s (TestOutput blk)) -> TestOutput blk)
-> (forall s. IOSim s (TestOutput blk)) -> TestOutput blk
forall a b. (a -> b) -> a -> b
$ do
    UTCTime -> IOSim s ()
forall s. UTCTime -> IOSim s ()
setCurrentTime UTCTime
dawnOfTime
    let TestConfigMB
          { CoreNodeId -> TestNodeInitialization (IOSim s) blk
nodeInfo :: forall (m :: * -> *) blk.
TestConfigMB m blk -> CoreNodeId -> TestNodeInitialization m blk
nodeInfo :: CoreNodeId -> TestNodeInitialization (IOSim s) blk
nodeInfo
          , Maybe (IOSim s (RekeyM (IOSim s) blk))
mkRekeyM :: forall (m :: * -> *) blk.
TestConfigMB m blk -> Maybe (m (RekeyM m blk))
mkRekeyM :: Maybe (IOSim s (RekeyM (IOSim s) blk))
mkRekeyM
          } = TestConfigMB (IOSim s) blk
forall (m :: * -> *). IOLike m => TestConfigMB m blk
mkTestConfigMB
    let systemTime :: SystemTime (IOSim s)
systemTime =
            SystemStart
-> Tracer (IOSim s) (TraceBlockchainTimeEvent UTCTime)
-> SystemTime (IOSim s)
forall (m :: * -> *).
(MonadTime m, MonadDelay m) =>
SystemStart
-> Tracer m (TraceBlockchainTimeEvent UTCTime) -> SystemTime m
BTime.defaultSystemTime
              (UTCTime -> SystemStart
BTime.SystemStart UTCTime
dawnOfTime)
              Tracer (IOSim s) (TraceBlockchainTimeEvent UTCTime)
forall (m :: * -> *) a. Applicative m => Tracer m a
nullTracer
    SystemTime (IOSim s)
-> ThreadNetworkArgs (IOSim s) blk -> IOSim s (TestOutput blk)
forall (m :: * -> *) blk.
(IOLike m, MonadTime m, MonadTimer m, RunNode blk, TxGen blk,
 TracingConstraints blk, HasCallStack) =>
SystemTime m -> ThreadNetworkArgs m blk -> m (TestOutput blk)
runThreadNetwork SystemTime (IOSim s)
systemTime ThreadNetworkArgs
      { tnaForgeEbbEnv :: Maybe (ForgeEbbEnv blk)
tnaForgeEbbEnv  = Maybe (ForgeEbbEnv blk)
forgeEbbEnv
      , tnaFuture :: Future
tnaFuture       = Future
future
      , tnaJoinPlan :: NodeJoinPlan
tnaJoinPlan     = NodeJoinPlan
nodeJoinPlan
      , tnaMessageDelay :: CalcMessageDelay blk
tnaMessageDelay = CalcMessageDelay blk
messageDelay
      , tnaNodeInfo :: CoreNodeId -> TestNodeInitialization (IOSim s) blk
tnaNodeInfo     = CoreNodeId -> TestNodeInitialization (IOSim s) blk
nodeInfo
      , tnaNumCoreNodes :: NumCoreNodes
tnaNumCoreNodes = NumCoreNodes
numCoreNodes
      , tnaNumSlots :: NumSlots
tnaNumSlots     = NumSlots
numSlots
      , tnaSeed :: Seed
tnaSeed         = Seed
initSeed
      , tnaMkRekeyM :: Maybe (IOSim s (RekeyM (IOSim s) blk))
tnaMkRekeyM     = Maybe (IOSim s (RekeyM (IOSim s) blk))
mkRekeyM
      , tnaRestarts :: NodeRestarts
tnaRestarts     = NodeRestarts
nodeRestarts
      , tnaTopology :: NodeTopology
tnaTopology     = NodeTopology
nodeTopology
      , tnaTxGenExtra :: TxGenExtra blk
tnaTxGenExtra   = TxGenExtra blk
txGenExtra
      , tnaVersion :: NodeToNodeVersion
tnaVersion      = NodeToNodeVersion
networkVersion
      , tnaBlockVersion :: BlockNodeToNodeVersion blk
tnaBlockVersion = BlockNodeToNodeVersion blk
blockVersion
      }

{-------------------------------------------------------------------------------
  Test properties
-------------------------------------------------------------------------------}

-- | Data about a node rejecting a block as invalid
--
data BlockRejection blk = BlockRejection
  { forall blk. BlockRejection blk -> HeaderHash blk
brBlockHash :: !(HeaderHash blk)
  , forall blk. BlockRejection blk -> SlotNo
brBlockSlot :: !SlotNo
  , forall blk. BlockRejection blk -> ExtValidationError blk
brReason    :: !(ExtValidationError blk)
  , forall blk. BlockRejection blk -> NodeId
brRejector  :: !NodeId
  }
  deriving (Int -> BlockRejection blk -> ShowS
[BlockRejection blk] -> ShowS
BlockRejection blk -> String
(Int -> BlockRejection blk -> ShowS)
-> (BlockRejection blk -> String)
-> ([BlockRejection blk] -> ShowS)
-> Show (BlockRejection blk)
forall blk.
LedgerSupportsProtocol blk =>
Int -> BlockRejection blk -> ShowS
forall blk.
LedgerSupportsProtocol blk =>
[BlockRejection blk] -> ShowS
forall blk.
LedgerSupportsProtocol blk =>
BlockRejection blk -> String
forall a.
(Int -> a -> ShowS) -> (a -> String) -> ([a] -> ShowS) -> Show a
$cshowsPrec :: forall blk.
LedgerSupportsProtocol blk =>
Int -> BlockRejection blk -> ShowS
showsPrec :: Int -> BlockRejection blk -> ShowS
$cshow :: forall blk.
LedgerSupportsProtocol blk =>
BlockRejection blk -> String
show :: BlockRejection blk -> String
$cshowList :: forall blk.
LedgerSupportsProtocol blk =>
[BlockRejection blk] -> ShowS
showList :: [BlockRejection blk] -> ShowS
Show)

data PropGeneralArgs blk = PropGeneralArgs
  { forall blk. PropGeneralArgs blk -> blk -> Property
pgaBlockProperty       :: blk -> Property
    -- ^ test if the block is as expected
    --
    -- For example, it may fail if the block includes transactions that should
    -- have expired before/when the block was forged.
    --
  , forall blk. PropGeneralArgs blk -> blk -> Word64
pgaCountTxs            :: blk -> Word64
    -- ^ the number of transactions in the block
    --
  , forall blk.
PropGeneralArgs blk
-> SlotNo -> NodeId -> WrapCannotForge blk -> Bool
pgaExpectedCannotForge :: SlotNo -> NodeId -> WrapCannotForge blk -> Bool
    -- ^ whether this 'CannotForge' was expected
    --
  , forall blk. PropGeneralArgs blk -> BlockNo
pgaFirstBlockNo        :: BlockNo
    -- ^ the block number of the first proper block on the chain
    --
    -- At time of writing this comment... For example, this is 1 for Byron
    -- tests and 0 for mock tests. The epoch boundary block (EBB) in slot 0
    -- specifies itself as having block number 0, which implies the genesis
    -- block is block number 0, and so the first proper block is number 1. For
    -- the mock tests, the first proper block is block number 0.
    --
  , forall blk. PropGeneralArgs blk -> Maybe NumBlocks
pgaFixedMaxForkLength  :: Maybe NumBlocks
    -- ^ the maximum length of a unique suffix among the final chains
    --
    -- If not provided, it will be crudely estimated. For example, this
    -- estimation is known to be incorrect for PBFT; it does not anticipate
    -- 'Ouroboros.Consensus.Protocol.PBFT.PBftExceededSignThreshold'.
    --
  , forall blk. PropGeneralArgs blk -> Maybe LeaderSchedule
pgaFixedSchedule       :: Maybe LeaderSchedule
    -- ^ the leader schedule of the nodes
    --
    -- If not provided, it will be recovered from the nodes' 'Tracer' data.
    --
  , forall blk. PropGeneralArgs blk -> SecurityParam
pgaSecurityParam       :: SecurityParam
  , forall blk. PropGeneralArgs blk -> TestConfig
pgaTestConfig          :: TestConfig
  , forall blk. PropGeneralArgs blk -> TestConfigB blk
pgaTestConfigB         :: TestConfigB blk
  }

-- | Expect no 'CannotForge's
noExpectedCannotForges :: SlotNo -> NodeId -> WrapCannotForge blk -> Bool
noExpectedCannotForges :: forall blk. SlotNo -> NodeId -> WrapCannotForge blk -> Bool
noExpectedCannotForges SlotNo
_ NodeId
_ WrapCannotForge blk
_ = Bool
False

-- | The properties always required
--
-- Assumes: /Synchrony/ ie (long) chains diffuse to all connected nodes before
-- the onset of the next slot.
--
-- Includes:
--
-- * The competitive chains at the end of the simulation respect the expected
--   bound on fork length
--
-- * The nodes do not leak file handles
--
-- * Blocks are exchanged without unexpected delays.
--
-- * The nodes' chains grow without unexpected delays.
--
-- * No nodes are unduly unable to lead (see 'pgaExpectedCannotForge').

-- * No blocks are rejected as invalid.
--
-- Those properties are currently checked under several assumptions. If the
-- nodes violate any of these assumptions, the tests will fail. The following
-- are the most primary assumptions.
--
-- Late Join Assumption: TODO! Only Ouroboros Genesis is designed to have nodes
-- join with "empty" (or even "stale") chains, so we do not strictly have a
-- spec for how the net should handle such a join under general circumstances.
-- (Note that we have not implemented Genesis yet.) For now, we make stop-gap
-- assumptions.
--
--   * Stop-Gap Assumption 0: Nodes join at the onset of a slot.
--
--   * Stop-Gap Assumption 1: A node joins with the "empty" chain (eg this
--     effectively includes the epoch 0 EBB for Byron).
--
--   * Stop-Gap Assumption 2: If a node leads a slot, then, at the slot onset,
--     its block production thread wins the race against the ChainSync and
--     BlockFetch client threads.
--
--   * Stop-Gap Assumption 3: If the old nodes (ie those that had joined in
--     previous slots) have a chain of greater than k blocks, then their
--     ChainSync client will throw ForkTooDeep :: ChainSyncClientException when
--     a new node joins. If any have such a chain, they all do, by the
--     Synchrony Assumption (see below). Thus all MiniProtocol clients on all
--     old nodes that share an edge with the new node in the topology will
--     disconnect from the new node until *exactly* the onset of the next slot.
--
-- Synchrony Assumption: Every (longest) chain diffuses to all other nodes
-- before the onset of the next slot.
--
--   * Corollary: Every node should have already selected one of the longest
--     chains in the net by the onset of each slot.
--
--   * Exception to the Corollary: A node cannot select such a chain if every
--     path between it and every recent leader is temporarily shutdown due to a
--     non-fatal MiniProtocol exception (such as ForkTooDeep discussed above).
--
-- Joining Leader Assumption: By Stop-Gap Assumption 2, a node that leads the
-- slot in which it joins forges a block atop the "empty" chain (ie origin or
-- else the epoch 0 EBB). Such a unary chain will be immediately discarded
-- unless there is no longer chain in the net.
--
--   * Remark: Under the current assumptions, that is the only way the selected
--     chains in the (P)BFT tests can diverge: if a node joins a net in which
--     the longest chain has only 1 block then it will introduce another
--     longest chain. As soon as any node forges a chain with 2 blocks, all
--     existing nodes will select that chain and will remain in consensus
--     thereafter. Once that happens, a node that joins in a later slot will
--     initially have the empty chain and might forge and briefly select its
--     own 1 block chain, but it also will have selected the single longest
--     chain by the end of the slot.
--
--   * Remark: Thus the (P)BFT tests only ever rollback to the "empty" chain,
--     and this will rollback at most 1 non-EBB block.
--
-- Online Assumption: Nodes are always online. In particular, a node always
-- forges a block at the onset of a slot it leads.
--
--   * Clarification: The test suite currently anticipates only two situations
--     in which a node will be unable to forge a valid block:
--     PBftExceededSignThreshold and PBftNotGenesisDelegate. If the node fails
--     to forge for any other reason, the tests will fail.
--
--   * Remark: Recall that a node leading the slot it joins usually wastes its
--     leadership by immediately forging a chain with 1 non-EBB block.
--
-- The above assumptions provide the (P)BFT tests enough information to predict
-- the net's behavior in each slot, except for identifiable cases in which that
-- behavior ultimately depends on how a node happens to select one among
-- multiple longest chains; see the PBFT reference simulator
-- "Test.ThreadNet.Ref.PBFT".
--
-- Specific tests make additional assumptions, eg the @Byron@ tests make
-- assumptions about delegation certificates, update proposals, etc.
prop_general ::
  forall blk.
     ( Condense blk
     , Condense (HeaderHash blk)
     , Eq blk
     , RunNode blk
     )
  => PropGeneralArgs blk
  -> TestOutput blk
  -> Property
prop_general :: forall blk.
(Condense blk, Condense (HeaderHash blk), Eq blk, RunNode blk) =>
PropGeneralArgs blk -> TestOutput blk -> Property
prop_general = Synchronicity -> PropGeneralArgs blk -> TestOutput blk -> Property
forall blk.
(Condense blk, Condense (HeaderHash blk), Eq blk, RunNode blk) =>
Synchronicity -> PropGeneralArgs blk -> TestOutput blk -> Property
prop_general_internal Synchronicity
Sync

-- | /Synchrony/ or /Semi-synchrony/
--
-- /Synchrony/ is characterized by every (relevant) message arriving during the
-- same slot in which it was sent. The Ouroboros research papers instead
-- characterize /semi-synchrony/ by a constant @Δ@ that bounds the number of
-- slots that it takes for any (relevant) message to arrive under " nominal "
-- circumstances (ie undersea cables have not been cut). Synchrony corresponds
-- to @Δ=1@ (ie " before the next slot ").
--
-- The net strictly cannot know @Δ@, but it can strive towards some value as an
-- objective eg.
data Synchronicity = SemiSync | Sync

-- | Like 'prop_general' but instead assuming /semi-synchrony/
--
-- For now, this simply disables a few 'Property's that depend on synchrony.
prop_general_semisync ::
  forall blk.
     ( Condense blk
     , Condense (HeaderHash blk)
     , Eq blk
     , RunNode blk
     )
  => PropGeneralArgs blk
  -> TestOutput blk
  -> Property
prop_general_semisync :: forall blk.
(Condense blk, Condense (HeaderHash blk), Eq blk, RunNode blk) =>
PropGeneralArgs blk -> TestOutput blk -> Property
prop_general_semisync = Synchronicity -> PropGeneralArgs blk -> TestOutput blk -> Property
forall blk.
(Condense blk, Condense (HeaderHash blk), Eq blk, RunNode blk) =>
Synchronicity -> PropGeneralArgs blk -> TestOutput blk -> Property
prop_general_internal Synchronicity
SemiSync

prop_general_internal ::
  forall blk.
     ( Condense blk
     , Condense (HeaderHash blk)
     , Eq blk
     , RunNode blk
     )
  => Synchronicity
  -> PropGeneralArgs blk
  -> TestOutput blk
  -> Property
prop_general_internal :: forall blk.
(Condense blk, Condense (HeaderHash blk), Eq blk, RunNode blk) =>
Synchronicity -> PropGeneralArgs blk -> TestOutput blk -> Property
prop_general_internal Synchronicity
syncity PropGeneralArgs blk
pga TestOutput blk
testOutput =
    String -> Property -> Property
forall prop. Testable prop => String -> prop -> Property
counterexample (String
"nodeChains: " String -> ShowS
forall a. Semigroup a => a -> a -> a
<> String
nodeChainsString) (Property -> Property) -> Property -> Property
forall a b. (a -> b) -> a -> b
$
    String -> Property -> Property
forall prop. Testable prop => String -> prop -> Property
counterexample (String
"nodeJoinPlan: " String -> ShowS
forall a. Semigroup a => a -> a -> a
<> NodeJoinPlan -> String
forall a. Condense a => a -> String
condense NodeJoinPlan
nodeJoinPlan) (Property -> Property) -> Property -> Property
forall a b. (a -> b) -> a -> b
$
    String -> Property -> Property
forall prop. Testable prop => String -> prop -> Property
counterexample (String
"nodeRestarts: " String -> ShowS
forall a. Semigroup a => a -> a -> a
<> NodeRestarts -> String
forall a. Condense a => a -> String
condense NodeRestarts
nodeRestarts) (Property -> Property) -> Property -> Property
forall a b. (a -> b) -> a -> b
$
    String -> Property -> Property
forall prop. Testable prop => String -> prop -> Property
counterexample (String
"nodeTopology: " String -> ShowS
forall a. Semigroup a => a -> a -> a
<> NodeTopology -> String
forall a. Condense a => a -> String
condense NodeTopology
nodeTopology) (Property -> Property) -> Property -> Property
forall a b. (a -> b) -> a -> b
$
    String -> Property -> Property
forall prop. Testable prop => String -> prop -> Property
counterexample (String
"slot-node-tipBlockNo: " String -> ShowS
forall a. Semigroup a => a -> a -> a
<> [(SlotNo, [(NodeId, WithOrigin BlockNo)])] -> String
forall a. Condense a => a -> String
condense [(SlotNo, [(NodeId, WithOrigin BlockNo)])]
tipBlockNos) (Property -> Property) -> Property -> Property
forall a b. (a -> b) -> a -> b
$
    String -> Property -> Property
forall prop. Testable prop => String -> prop -> Property
counterexample (String
"mbSchedule: " String -> ShowS
forall a. Semigroup a => a -> a -> a
<> Maybe LeaderSchedule -> String
forall a. Condense a => a -> String
condense Maybe LeaderSchedule
mbSchedule) (Property -> Property) -> Property -> Property
forall a b. (a -> b) -> a -> b
$
    String -> Property -> Property
forall prop. Testable prop => String -> prop -> Property
counterexample (String
"growth schedule: " String -> ShowS
forall a. Semigroup a => a -> a -> a
<> LeaderSchedule -> String
forall a. Condense a => a -> String
condense LeaderSchedule
growthSchedule) (Property -> Property) -> Property -> Property
forall a b. (a -> b) -> a -> b
$
    String -> Property -> Property
forall prop. Testable prop => String -> prop -> Property
counterexample (String
"actual leader schedule: " String -> ShowS
forall a. Semigroup a => a -> a -> a
<> LeaderSchedule -> String
forall a. Condense a => a -> String
condense LeaderSchedule
actualLeaderSchedule) (Property -> Property) -> Property -> Property
forall a b. (a -> b) -> a -> b
$
    String -> Property -> Property
forall prop. Testable prop => String -> prop -> Property
counterexample (String
"consensus expected: " String -> ShowS
forall a. Semigroup a => a -> a -> a
<> Bool -> String
forall a. Show a => a -> String
show Bool
isConsensusExpected) (Property -> Property) -> Property -> Property
forall a b. (a -> b) -> a -> b
$
    String -> Property -> Property
forall prop. Testable prop => String -> prop -> Property
counterexample (String
"maxForkLength: " String -> ShowS
forall a. Semigroup a => a -> a -> a
<> Word64 -> String
forall a. Show a => a -> String
show Word64
maxForkLength) (Property -> Property) -> Property -> Property
forall a b. (a -> b) -> a -> b
$
    String -> [String] -> Property -> Property
tabulateSync String
"consensus expected" [Bool -> String
forall a. Show a => a -> String
show Bool
isConsensusExpected] (Property -> Property) -> Property -> Property
forall a b. (a -> b) -> a -> b
$
    String -> [String] -> Property -> Property
forall prop.
Testable prop =>
String -> [String] -> prop -> Property
tabulate String
"k" [Word64 -> String
forall a. Show a => a -> String
show (SecurityParam -> Word64
maxRollbacks SecurityParam
k)] (Property -> Property) -> Property -> Property
forall a b. (a -> b) -> a -> b
$
    String -> [String] -> Property -> Property
forall prop.
Testable prop =>
String -> [String] -> prop -> Property
tabulate (String
"shortestLength (k = " String -> ShowS
forall a. Semigroup a => a -> a -> a
<> Word64 -> String
forall a. Show a => a -> String
show (SecurityParam -> Word64
maxRollbacks SecurityParam
k) String -> ShowS
forall a. Semigroup a => a -> a -> a
<> String
")")
      [RangeK -> String
forall a. Show a => a -> String
show (SecurityParam -> Natural -> RangeK
forall a. Integral a => SecurityParam -> a -> RangeK
rangeK SecurityParam
k (Map NodeId (Chain blk) -> Natural
forall b. Map NodeId (Chain b) -> Natural
shortestLength Map NodeId (Chain blk)
nodeChains))] (Property -> Property) -> Property -> Property
forall a b. (a -> b) -> a -> b
$
    String -> [String] -> Property -> Property
forall prop.
Testable prop =>
String -> [String] -> prop -> Property
tabulate String
"floor(4 * lastJoinSlot / numSlots)" [Maybe Word64 -> String
forall a. Show a => a -> String
show Maybe Word64
lastJoinSlot] (Property -> Property) -> Property -> Property
forall a b. (a -> b) -> a -> b
$
    String -> [String] -> Property -> Property
forall prop.
Testable prop =>
String -> [String] -> prop -> Property
tabulate String
"minimumDegreeNodeTopology" [Maybe Int -> String
forall a. Show a => a -> String
show (NodeTopology -> Maybe Int
minimumDegreeNodeTopology NodeTopology
nodeTopology)] (Property -> Property) -> Property -> Property
forall a b. (a -> b) -> a -> b
$
    String -> [String] -> Property -> Property
forall prop.
Testable prop =>
String -> [String] -> prop -> Property
tabulate String
"involves >=1 re-delegation" [Bool -> String
forall a. Show a => a -> String
show Bool
hasNodeRekey] (Property -> Property) -> Property -> Property
forall a b. (a -> b) -> a -> b
$
    String -> [String] -> Property -> Property
forall prop.
Testable prop =>
String -> [String] -> prop -> Property
tabulate String
"average #txs/block" [Range Double -> String
forall a. Show a => a -> String
show (Double -> Range Double
forall n. (Ord n, Show n, Num n) => n -> Range n
range Double
averageNumTxs)] (Property -> Property) -> Property -> Property
forall a b. (a -> b) -> a -> b
$
    String -> [String] -> Property -> Property
forall prop.
Testable prop =>
String -> [String] -> prop -> Property
tabulate String
"updates" [[String] -> String
unlines (String
"" String -> [String] -> [String]
forall a. a -> [a] -> [a]
: ((NodeId, [LedgerUpdate blk]) -> String)
-> [(NodeId, [LedgerUpdate blk])] -> [String]
forall a b. (a -> b) -> [a] -> [b]
map (\(NodeId, [LedgerUpdate blk])
x -> String
"  " String -> ShowS
forall a. Semigroup a => a -> a -> a
<> (NodeId, [LedgerUpdate blk]) -> String
forall a. Condense a => a -> String
condense (NodeId, [LedgerUpdate blk])
x) (Map NodeId [LedgerUpdate blk] -> [(NodeId, [LedgerUpdate blk])]
forall k a. Map k a -> [(k, a)]
Map.toList Map NodeId [LedgerUpdate blk]
nodeUpdates))] (Property -> Property) -> Property -> Property
forall a b. (a -> b) -> a -> b
$
    Property
prop_no_BlockRejections Property -> Property -> Property
forall prop1 prop2.
(Testable prop1, Testable prop2) =>
prop1 -> prop2 -> Property
.&&.
    Property
prop_no_unexpected_CannotForges Property -> Property -> Property
forall prop1 prop2.
(Testable prop1, Testable prop2) =>
prop1 -> prop2 -> Property
.&&.
    Property
prop_no_invalid_blocks Property -> Property -> Property
forall prop1 prop2.
(Testable prop1, Testable prop2) =>
prop1 -> prop2 -> Property
.&&.
    Property
prop_pipelining Property -> Property -> Property
forall prop1 prop2.
(Testable prop1, Testable prop2) =>
prop1 -> prop2 -> Property
.&&.
    Property -> Property
propSync
      ( Word64 -> [Chain blk] -> Property
forall b.
(HasHeader b, Condense (HeaderHash b), Eq b) =>
Word64 -> [Chain b] -> Property
prop_all_common_prefix Word64
maxForkLength (Map NodeId (Chain blk) -> [Chain blk]
forall k a. Map k a -> [a]
Map.elems Map NodeId (Chain blk)
nodeChains) Property -> Property -> Property
forall prop1 prop2.
(Testable prop1, Testable prop2) =>
prop1 -> prop2 -> Property
.&&.
        Property
prop_all_growth Property -> Property -> Property
forall prop1 prop2.
(Testable prop1, Testable prop2) =>
prop1 -> prop2 -> Property
.&&.
        Property
HasCallStack => Property
prop_no_unexpected_message_delays
      ) Property -> Property -> Property
forall prop1 prop2.
(Testable prop1, Testable prop2) =>
prop1 -> prop2 -> Property
.&&.
    [Property] -> Property
forall prop. Testable prop => [prop] -> Property
conjoin
      [ NodeId -> NodeDBs MockFS -> Property
fileHandleLeakCheck NodeId
nid NodeDBs MockFS
nodeDBs
      | (NodeId
nid, NodeDBs MockFS
nodeDBs) <- Map NodeId (NodeDBs MockFS) -> [(NodeId, NodeDBs MockFS)]
forall k a. Map k a -> [(k, a)]
Map.toList Map NodeId (NodeDBs MockFS)
nodeOutputDBs ]
  where
    tabulateSync :: String -> [String] -> Property -> Property
tabulateSync  = case Synchronicity
syncity of
        Synchronicity
Sync     -> String -> [String] -> Property -> Property
forall prop.
Testable prop =>
String -> [String] -> prop -> Property
tabulate
        Synchronicity
SemiSync -> \String
_ [String]
_ -> Property -> Property
forall a. a -> a
id
    propSync :: Property -> Property
propSync Property
prop = case Synchronicity
syncity of
        Synchronicity
Sync     -> Property
prop
        Synchronicity
SemiSync -> Bool -> Property
forall prop. Testable prop => prop -> Property
property Bool
True

    ()
_ = Proxy (Show (LedgerView (BlockProtocol blk))) -> ()
forall (c :: Constraint) (proxy :: Constraint -> *).
c =>
proxy c -> ()
keepRedundantConstraint (forall {k} (t :: k). Proxy t
forall (t :: Constraint). Proxy t
Proxy @(Show (LedgerView (BlockProtocol blk))))

    PropGeneralArgs
      { pgaBlockProperty :: forall blk. PropGeneralArgs blk -> blk -> Property
pgaBlockProperty       = blk -> Property
prop_valid_block
      , pgaCountTxs :: forall blk. PropGeneralArgs blk -> blk -> Word64
pgaCountTxs            = blk -> Word64
countTxs
      , pgaExpectedCannotForge :: forall blk.
PropGeneralArgs blk
-> SlotNo -> NodeId -> WrapCannotForge blk -> Bool
pgaExpectedCannotForge = SlotNo -> NodeId -> WrapCannotForge blk -> Bool
expectedCannotForge
      , pgaFirstBlockNo :: forall blk. PropGeneralArgs blk -> BlockNo
pgaFirstBlockNo        = BlockNo
firstBlockNo
      , pgaFixedMaxForkLength :: forall blk. PropGeneralArgs blk -> Maybe NumBlocks
pgaFixedMaxForkLength  = Maybe NumBlocks
mbMaxForkLength
      , pgaFixedSchedule :: forall blk. PropGeneralArgs blk -> Maybe LeaderSchedule
pgaFixedSchedule       = Maybe LeaderSchedule
mbSchedule
      , pgaSecurityParam :: forall blk. PropGeneralArgs blk -> SecurityParam
pgaSecurityParam       = SecurityParam
k
      , TestConfig
pgaTestConfig :: forall blk. PropGeneralArgs blk -> TestConfig
pgaTestConfig :: TestConfig
pgaTestConfig
      , TestConfigB blk
pgaTestConfigB :: forall blk. PropGeneralArgs blk -> TestConfigB blk
pgaTestConfigB :: TestConfigB blk
pgaTestConfigB
      } = PropGeneralArgs blk
pga
    TestConfig
      { NumSlots
numSlots :: TestConfig -> NumSlots
numSlots :: NumSlots
numSlots
      , NodeTopology
nodeTopology :: TestConfig -> NodeTopology
nodeTopology :: NodeTopology
nodeTopology
      } = TestConfig
pgaTestConfig
    TestConfigB
      { NodeJoinPlan
nodeJoinPlan :: forall blk. TestConfigB blk -> NodeJoinPlan
nodeJoinPlan :: NodeJoinPlan
nodeJoinPlan
      , NodeRestarts
nodeRestarts :: forall blk. TestConfigB blk -> NodeRestarts
nodeRestarts :: NodeRestarts
nodeRestarts
      } = TestConfigB blk
pgaTestConfigB
    TestOutput
      { Map NodeId (NodeOutput blk)
testOutputNodes :: Map NodeId (NodeOutput blk)
testOutputNodes :: forall blk. TestOutput blk -> Map NodeId (NodeOutput blk)
testOutputNodes
      , Map SlotNo (Map NodeId (WithOrigin BlockNo))
testOutputTipBlockNos :: Map SlotNo (Map NodeId (WithOrigin BlockNo))
testOutputTipBlockNos :: forall blk.
TestOutput blk -> Map SlotNo (Map NodeId (WithOrigin BlockNo))
testOutputTipBlockNos
      } = TestOutput blk
testOutput

    prop_no_BlockRejections :: Property
prop_no_BlockRejections =
        String -> Bool -> Property
forall prop. Testable prop => String -> prop -> Property
counterexample String
msg (Bool -> Property) -> Bool -> Property
forall a b. (a -> b) -> a -> b
$
        [BlockRejection blk] -> Bool
forall a. [a] -> Bool
forall (t :: * -> *) a. Foldable t => t a -> Bool
null [BlockRejection blk]
brs
      where
        msg :: String
msg =
            String
"There were unexpected block rejections: " String -> ShowS
forall a. Semigroup a => a -> a -> a
<>
            [String] -> String
unlines ((BlockRejection blk -> String) -> [BlockRejection blk] -> [String]
forall a b. (a -> b) -> [a] -> [b]
map BlockRejection blk -> String
forall a. Show a => a -> String
show [BlockRejection blk]
brs)
        brs :: [BlockRejection blk]
brs =
            [ BlockRejection
                { brBlockHash :: HeaderHash blk
brBlockHash = HeaderHash blk
h
                , brBlockSlot :: SlotNo
brBlockSlot = SlotNo
s
                , brRejector :: NodeId
brRejector  = NodeId
nid
                , brReason :: ExtValidationError blk
brReason    = ExtValidationError blk
err
                }
            | (NodeId
nid, NodeOutput blk
no) <- Map NodeId (NodeOutput blk) -> [(NodeId, NodeOutput blk)]
forall k a. Map k a -> [(k, a)]
Map.toList Map NodeId (NodeOutput blk)
testOutputNodes
            , let NodeOutput{Map (RealPoint blk) [ExtValidationError blk]
nodeOutputInvalids :: Map (RealPoint blk) [ExtValidationError blk]
nodeOutputInvalids :: forall blk.
NodeOutput blk -> Map (RealPoint blk) [ExtValidationError blk]
nodeOutputInvalids} = NodeOutput blk
no
            , (RealPoint SlotNo
s HeaderHash blk
h, [ExtValidationError blk]
errs) <- Map (RealPoint blk) [ExtValidationError blk]
-> [(RealPoint blk, [ExtValidationError blk])]
forall k a. Map k a -> [(k, a)]
Map.toList Map (RealPoint blk) [ExtValidationError blk]
nodeOutputInvalids
            , ExtValidationError blk
err                   <- [ExtValidationError blk]
errs
            ]

    prop_no_unexpected_CannotForges :: Property
prop_no_unexpected_CannotForges =
        String -> Bool -> Property
forall prop. Testable prop => String -> prop -> Property
counterexample String
msg (Bool -> Property) -> Bool -> Property
forall a b. (a -> b) -> a -> b
$
        Map SlotNo [CannotForge blk] -> Bool
forall k a. Map k a -> Bool
Map.null Map SlotNo [CannotForge blk]
cls
      where
        msg :: String
msg = String
"There were unexpected CannotForges: " String -> ShowS
forall a. Semigroup a => a -> a -> a
<> Map SlotNo [CannotForge blk] -> String
forall a. Show a => a -> String
show Map SlotNo [CannotForge blk]
cls
        cls :: Map SlotNo [CannotForge blk]
cls =
            ([CannotForge blk] -> [CannotForge blk] -> [CannotForge blk])
-> [Map SlotNo [CannotForge blk]] -> Map SlotNo [CannotForge blk]
forall (f :: * -> *) k a.
(Foldable f, Ord k) =>
(a -> a -> a) -> f (Map k a) -> Map k a
Map.unionsWith [CannotForge blk] -> [CannotForge blk] -> [CannotForge blk]
forall a. [a] -> [a] -> [a]
(++) ([Map SlotNo [CannotForge blk]] -> Map SlotNo [CannotForge blk])
-> [Map SlotNo [CannotForge blk]] -> Map SlotNo [CannotForge blk]
forall a b. (a -> b) -> a -> b
$
            [ ([CannotForge blk] -> Bool)
-> Map SlotNo [CannotForge blk] -> Map SlotNo [CannotForge blk]
forall a k. (a -> Bool) -> Map k a -> Map k a
Map.filter (Bool -> Bool
not (Bool -> Bool)
-> ([CannotForge blk] -> Bool) -> [CannotForge blk] -> Bool
forall b c a. (b -> c) -> (a -> b) -> a -> c
. [CannotForge blk] -> Bool
forall a. [a] -> Bool
forall (t :: * -> *) a. Foldable t => t a -> Bool
null) (Map SlotNo [CannotForge blk] -> Map SlotNo [CannotForge blk])
-> Map SlotNo [CannotForge blk] -> Map SlotNo [CannotForge blk]
forall a b. (a -> b) -> a -> b
$
              (SlotNo -> [CannotForge blk] -> [CannotForge blk])
-> Map SlotNo [CannotForge blk] -> Map SlotNo [CannotForge blk]
forall k a b. (k -> a -> b) -> Map k a -> Map k b
Map.mapWithKey (\SlotNo
s -> (CannotForge blk -> Bool) -> [CannotForge blk] -> [CannotForge blk]
forall a. (a -> Bool) -> [a] -> [a]
filter (Bool -> Bool
not (Bool -> Bool)
-> (CannotForge blk -> Bool) -> CannotForge blk -> Bool
forall b c a. (b -> c) -> (a -> b) -> a -> c
. SlotNo -> NodeId -> CannotForge blk -> Bool
ok SlotNo
s NodeId
nid)) (Map SlotNo [CannotForge blk] -> Map SlotNo [CannotForge blk])
-> Map SlotNo [CannotForge blk] -> Map SlotNo [CannotForge blk]
forall a b. (a -> b) -> a -> b
$
              Map SlotNo [CannotForge blk]
nodeOutputCannotForges
            | (NodeId
nid, NodeOutput blk
no) <- Map NodeId (NodeOutput blk) -> [(NodeId, NodeOutput blk)]
forall k a. Map k a -> [(k, a)]
Map.toList Map NodeId (NodeOutput blk)
testOutputNodes
            , let NodeOutput{Map SlotNo [CannotForge blk]
nodeOutputCannotForges :: Map SlotNo [CannotForge blk]
nodeOutputCannotForges :: forall blk. NodeOutput blk -> Map SlotNo [CannotForge blk]
nodeOutputCannotForges} = NodeOutput blk
no
            ]
        ok :: SlotNo -> NodeId -> CannotForge blk -> Bool
ok SlotNo
s NodeId
nid CannotForge blk
cl =
            SlotNo -> NodeId -> WrapCannotForge blk -> Bool
expectedCannotForge SlotNo
s NodeId
nid (CannotForge blk -> WrapCannotForge blk
forall blk. CannotForge blk -> WrapCannotForge blk
WrapCannotForge CannotForge blk
cl)

    schedule :: LeaderSchedule
schedule = case Maybe LeaderSchedule
mbSchedule of
        Maybe LeaderSchedule
Nothing    -> LeaderSchedule
actualLeaderSchedule
        Just LeaderSchedule
sched -> LeaderSchedule
sched

    NumBlocks Word64
maxForkLength = case Maybe NumBlocks
mbMaxForkLength of
      Maybe NumBlocks
Nothing -> SecurityParam -> NodeJoinPlan -> LeaderSchedule -> NumBlocks
determineForkLength SecurityParam
k NodeJoinPlan
nodeJoinPlan LeaderSchedule
schedule
      Just NumBlocks
fl -> NumBlocks
fl

    -- build a leader schedule which includes every node that forged unless:
    --
    -- * the node rejected its own new block (eg 'PBftExceededSignThreshold')
    --
    actualLeaderSchedule :: LeaderSchedule
    actualLeaderSchedule :: LeaderSchedule
actualLeaderSchedule =
        (LeaderSchedule -> LeaderSchedule -> LeaderSchedule)
-> LeaderSchedule -> [LeaderSchedule] -> LeaderSchedule
forall b a. (b -> a -> b) -> b -> [a] -> b
forall (t :: * -> *) b a.
Foldable t =>
(b -> a -> b) -> b -> t a -> b
foldl LeaderSchedule -> LeaderSchedule -> LeaderSchedule
forall a. Semigroup a => a -> a -> a
(<>) (NumSlots -> LeaderSchedule
emptyLeaderSchedule NumSlots
numSlots) ([LeaderSchedule] -> LeaderSchedule)
-> [LeaderSchedule] -> LeaderSchedule
forall a b. (a -> b) -> a -> b
$
        [ let NodeOutput
                { Map SlotNo blk
nodeOutputForges :: Map SlotNo blk
nodeOutputForges :: forall blk. NodeOutput blk -> Map SlotNo blk
nodeOutputForges
                , Map (RealPoint blk) [ExtValidationError blk]
nodeOutputInvalids :: forall blk.
NodeOutput blk -> Map (RealPoint blk) [ExtValidationError blk]
nodeOutputInvalids :: Map (RealPoint blk) [ExtValidationError blk]
nodeOutputInvalids
                } = NodeOutput blk
no
          in
          Map SlotNo [CoreNodeId] -> LeaderSchedule
LeaderSchedule (Map SlotNo [CoreNodeId] -> LeaderSchedule)
-> Map SlotNo [CoreNodeId] -> LeaderSchedule
forall a b. (a -> b) -> a -> b
$
          (SlotNo -> blk -> Maybe [CoreNodeId])
-> Map SlotNo blk -> Map SlotNo [CoreNodeId]
forall k a b. (k -> a -> Maybe b) -> Map k a -> Map k b
Map.mapMaybeWithKey
              (NodeId
-> Set (RealPoint blk) -> SlotNo -> blk -> Maybe [CoreNodeId]
actuallyLead NodeId
cid (Map (RealPoint blk) [ExtValidationError blk] -> Set (RealPoint blk)
forall k a. Map k a -> Set k
Map.keysSet Map (RealPoint blk) [ExtValidationError blk]
nodeOutputInvalids))
              Map SlotNo blk
nodeOutputForges
        | (NodeId
cid, NodeOutput blk
no) <- Map NodeId (NodeOutput blk) -> [(NodeId, NodeOutput blk)]
forall k a. Map k a -> [(k, a)]
Map.toList Map NodeId (NodeOutput blk)
testOutputNodes
        ]
      where
        actuallyLead ::
             NodeId
          -> Set (RealPoint blk)
          -> SlotNo
          -> blk
          -> Maybe [CoreNodeId]
        actuallyLead :: NodeId
-> Set (RealPoint blk) -> SlotNo -> blk -> Maybe [CoreNodeId]
actuallyLead NodeId
nid Set (RealPoint blk)
invalids SlotNo
s blk
b = do
            CoreNodeId
cid <- case NodeId
nid of
                CoreId CoreNodeId
i  -> CoreNodeId -> Maybe CoreNodeId
forall a. a -> Maybe a
Just CoreNodeId
i
                RelayId Word64
_ -> Maybe CoreNodeId
forall a. Maybe a
Nothing

            let j :: SlotNo
j = HasCallStack => NodeJoinPlan -> NodeId -> SlotNo
NodeJoinPlan -> NodeId -> SlotNo
nodeIdJoinSlot NodeJoinPlan
nodeJoinPlan NodeId
nid
            Bool -> Maybe ()
forall (f :: * -> *). Alternative f => Bool -> f ()
guard (Bool -> Maybe ()) -> Bool -> Maybe ()
forall a b. (a -> b) -> a -> b
$ SlotNo
j SlotNo -> SlotNo -> Bool
forall a. Ord a => a -> a -> Bool
<= SlotNo
s

            Bool -> Maybe ()
forall (f :: * -> *). Alternative f => Bool -> f ()
guard (Bool -> Maybe ()) -> Bool -> Maybe ()
forall a b. (a -> b) -> a -> b
$ Bool -> Bool
not (Bool -> Bool) -> Bool -> Bool
forall a b. (a -> b) -> a -> b
$ RealPoint blk -> Set (RealPoint blk) -> Bool
forall a. Ord a => a -> Set a -> Bool
Set.member (blk -> RealPoint blk
forall blk. HasHeader blk => blk -> RealPoint blk
blockRealPoint blk
b) Set (RealPoint blk)
invalids

            [CoreNodeId] -> Maybe [CoreNodeId]
forall a. a -> Maybe a
forall (f :: * -> *) a. Applicative f => a -> f a
pure [CoreNodeId
cid]

    -- Refine 'actualLeaderSchedule' to also ignore a leader if:
    --
    -- * the node just joined in this slot (unless it's the earliest slot in
    --   which any nodes joined)
    --
    growthSchedule :: LeaderSchedule
    growthSchedule :: LeaderSchedule
growthSchedule =
        Map SlotNo [CoreNodeId] -> LeaderSchedule
LeaderSchedule (Map SlotNo [CoreNodeId] -> LeaderSchedule)
-> Map SlotNo [CoreNodeId] -> LeaderSchedule
forall a b. (a -> b) -> a -> b
$ (SlotNo -> [CoreNodeId] -> [CoreNodeId])
-> Map SlotNo [CoreNodeId] -> Map SlotNo [CoreNodeId]
forall k a b. (k -> a -> b) -> Map k a -> Map k b
Map.mapWithKey (\SlotNo
s -> (CoreNodeId -> Bool) -> [CoreNodeId] -> [CoreNodeId]
forall a. (a -> Bool) -> [a] -> [a]
filter (SlotNo -> CoreNodeId -> Bool
keep SlotNo
s)) Map SlotNo [CoreNodeId]
mlead
      where
        LeaderSchedule Map SlotNo [CoreNodeId]
mlead = LeaderSchedule
actualLeaderSchedule

        keep :: SlotNo -> CoreNodeId -> Bool
keep SlotNo
s CoreNodeId
cid =
             SlotNo -> Bool
isFirstJoinSlot SlotNo
s
          Bool -> Bool -> Bool
|| HasCallStack => NodeJoinPlan -> CoreNodeId -> SlotNo
NodeJoinPlan -> CoreNodeId -> SlotNo
coreNodeIdJoinSlot NodeJoinPlan
nodeJoinPlan CoreNodeId
cid SlotNo -> SlotNo -> Bool
forall a. Ord a => a -> a -> Bool
< SlotNo
s

        isFirstJoinSlot :: SlotNo -> Bool
isFirstJoinSlot SlotNo
s =
            SlotNo -> Maybe SlotNo
forall a. a -> Maybe a
Just SlotNo
s Maybe SlotNo -> Maybe SlotNo -> Bool
forall a. Eq a => a -> a -> Bool
== ((CoreNodeId, SlotNo) -> SlotNo
forall a b. (a, b) -> b
snd ((CoreNodeId, SlotNo) -> SlotNo)
-> Maybe (CoreNodeId, SlotNo) -> Maybe SlotNo
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Map CoreNodeId SlotNo -> Maybe (CoreNodeId, SlotNo)
forall k a. Map k a -> Maybe (k, a)
Map.lookupMin Map CoreNodeId SlotNo
mjoin)
          where
            NodeJoinPlan Map CoreNodeId SlotNo
mjoin = NodeJoinPlan
nodeJoinPlan

    nodeChains :: Map NodeId (Chain blk)
nodeChains    = NodeOutput blk -> Chain blk
forall blk. NodeOutput blk -> Chain blk
nodeOutputFinalChain (NodeOutput blk -> Chain blk)
-> Map NodeId (NodeOutput blk) -> Map NodeId (Chain blk)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Map NodeId (NodeOutput blk)
testOutputNodes
    nodeOutputDBs :: Map NodeId (NodeDBs MockFS)
nodeOutputDBs = NodeOutput blk -> NodeDBs MockFS
forall blk. NodeOutput blk -> NodeDBs MockFS
nodeOutputNodeDBs    (NodeOutput blk -> NodeDBs MockFS)
-> Map NodeId (NodeOutput blk) -> Map NodeId (NodeDBs MockFS)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Map NodeId (NodeOutput blk)
testOutputNodes
    nodeUpdates :: Map NodeId [LedgerUpdate blk]
nodeUpdates   = NodeOutput blk -> [LedgerUpdate blk]
forall blk. NodeOutput blk -> [LedgerUpdate blk]
nodeOutputUpdates    (NodeOutput blk -> [LedgerUpdate blk])
-> Map NodeId (NodeOutput blk) -> Map NodeId [LedgerUpdate blk]
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Map NodeId (NodeOutput blk)
testOutputNodes

    nodeChainsString :: String
    nodeChainsString :: String
nodeChainsString =
        [String] -> String
unlines ([String] -> String) -> [String] -> String
forall a b. (a -> b) -> a -> b
$ (String
"" String -> [String] -> [String]
forall a. a -> [a] -> [a]
:) ([String] -> [String]) -> [String] -> [String]
forall a b. (a -> b) -> a -> b
$
        ((NodeId, Tip blk) -> String) -> [(NodeId, Tip blk)] -> [String]
forall a b. (a -> b) -> [a] -> [b]
map (\(NodeId, Tip blk)
x -> String
"  " String -> ShowS
forall a. Semigroup a => a -> a -> a
<> (NodeId, Tip blk) -> String
forall a. Condense a => a -> String
condense (NodeId, Tip blk)
x) ([(NodeId, Tip blk)] -> [String])
-> [(NodeId, Tip blk)] -> [String]
forall a b. (a -> b) -> a -> b
$
        Map NodeId (Tip blk) -> [(NodeId, Tip blk)]
forall k a. Map k a -> [(k, a)]
Map.toList (Map NodeId (Tip blk) -> [(NodeId, Tip blk)])
-> Map NodeId (Tip blk) -> [(NodeId, Tip blk)]
forall a b. (a -> b) -> a -> b
$ (Chain blk -> Tip blk)
-> Map NodeId (Chain blk) -> Map NodeId (Tip blk)
forall a b. (a -> b) -> Map NodeId a -> Map NodeId b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap Chain blk -> Tip blk
forall block. HasHeader block => Chain block -> Tip block
MockChain.headTip Map NodeId (Chain blk)
nodeChains

    isConsensusExpected :: Bool
    isConsensusExpected :: Bool
isConsensusExpected = SecurityParam -> NodeJoinPlan -> LeaderSchedule -> Bool
consensusExpected SecurityParam
k NodeJoinPlan
nodeJoinPlan LeaderSchedule
schedule

    fileHandleLeakCheck :: NodeId -> NodeDBs MockFS -> Property
    fileHandleLeakCheck :: NodeId -> NodeDBs MockFS -> Property
fileHandleLeakCheck NodeId
nid NodeDBs MockFS
nodeDBs = [Property] -> Property
forall prop. Testable prop => [prop] -> Property
conjoin
        [ String -> MockFS -> Property
checkLeak String
"ImmutableDB" (MockFS -> Property) -> MockFS -> Property
forall a b. (a -> b) -> a -> b
$ NodeDBs MockFS -> MockFS
forall db. NodeDBs db -> db
nodeDBsImm NodeDBs MockFS
nodeDBs
        , String -> MockFS -> Property
checkLeak String
"VolatileDB"  (MockFS -> Property) -> MockFS -> Property
forall a b. (a -> b) -> a -> b
$ NodeDBs MockFS -> MockFS
forall db. NodeDBs db -> db
nodeDBsVol NodeDBs MockFS
nodeDBs
        , String -> MockFS -> Property
checkLeak String
"LedgerDB"    (MockFS -> Property) -> MockFS -> Property
forall a b. (a -> b) -> a -> b
$ NodeDBs MockFS -> MockFS
forall db. NodeDBs db -> db
nodeDBsLgr NodeDBs MockFS
nodeDBs
        ]
      where
        checkLeak :: String -> MockFS -> Property
checkLeak String
dbName MockFS
fs = String -> Property -> Property
forall prop. Testable prop => String -> prop -> Property
counterexample
          (String
"Node " String -> ShowS
forall a. Semigroup a => a -> a -> a
<> NodeId -> String
forall a. Show a => a -> String
show NodeId
nid String -> ShowS
forall a. Semigroup a => a -> a -> a
<> String
"'s " String -> ShowS
forall a. Semigroup a => a -> a -> a
<> String
dbName String -> ShowS
forall a. Semigroup a => a -> a -> a
<> String
" is leaking file handles")
          (MockFS -> Int
Mock.numOpenHandles MockFS
fs Int -> Int -> Property
forall a. (Eq a, Show a) => a -> a -> Property
=== Int
0)

    -- in which quarter of the simulation does the last node join?
    lastJoinSlot :: Maybe Word64
    lastJoinSlot :: Maybe Word64
lastJoinSlot =
        ((SlotNo, Map CoreNodeId SlotNo) -> Word64)
-> Maybe (SlotNo, Map CoreNodeId SlotNo) -> Maybe Word64
forall a b. (a -> b) -> Maybe a -> Maybe b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap (\(SlotNo Word64
i, Map CoreNodeId SlotNo
_) -> (Word64
4 Word64 -> Word64 -> Word64
forall a. Num a => a -> a -> a
* Word64
i) Word64 -> Word64 -> Word64
forall a. Integral a => a -> a -> a
`div` Word64
t) (Maybe (SlotNo, Map CoreNodeId SlotNo) -> Maybe Word64)
-> Maybe (SlotNo, Map CoreNodeId SlotNo) -> Maybe Word64
forall a b. (a -> b) -> a -> b
$
        Map CoreNodeId SlotNo -> Maybe (SlotNo, Map CoreNodeId SlotNo)
forall k a. Map k a -> Maybe (a, Map k a)
Map.maxView Map CoreNodeId SlotNo
m
      where
        NumSlots Word64
t = NumSlots
numSlots
        NodeJoinPlan Map CoreNodeId SlotNo
m = NodeJoinPlan
nodeJoinPlan

    -- check for Chain Growth violations if there are no Common Prefix
    -- violations
    --
    -- We consider all possible non-empty intervals, so the interval span
    -- @s@ varies but is always at least 1. We compute a different /speed
    -- coefficient/ @τ@ for each interval under the assumption that there are
    -- no message delays (ie @Δ = 0@). This is essentially a count of the
    -- active slots for that interval in the refined @growthSchedule@.
    --
    -- The paper <https://eprint.iacr.org/2017/573/20171115:00183> defines
    -- Common Growth as follows.
    --
    -- * Chain Growth (CG); with parameters τ ∈ (0, 1], s ∈ N. Consider the
    --   chains C1, C2 possessed by two honest parties at the onset of two
    --   slots sl1, sl2 with sl2 at least s slots ahead of sl1. Then it holds
    --   that len(C2) − len(C1) ≥ τs. We call τ the speed coefficient.
    prop_all_growth :: Property
prop_all_growth =
        Bool
isConsensusExpected Bool -> Property -> Property
forall prop. Testable prop => Bool -> prop -> Property
`implies`
            [Property] -> Property
forall prop. Testable prop => [prop] -> Property
conjoin
                [ (SlotNo, WithOrigin BlockNo)
-> (SlotNo, WithOrigin BlockNo) -> Property
prop_growth (SlotNo
s1, WithOrigin BlockNo
max1) (SlotNo
s2, WithOrigin BlockNo
min2)
                | ((SlotNo
s1, WithOrigin BlockNo
_, WithOrigin BlockNo
max1), (SlotNo
s2, WithOrigin BlockNo
min2, WithOrigin BlockNo
_)) <- [(SlotNo, WithOrigin BlockNo, WithOrigin BlockNo)]
-> [((SlotNo, WithOrigin BlockNo, WithOrigin BlockNo),
     (SlotNo, WithOrigin BlockNo, WithOrigin BlockNo))]
forall a. [a] -> [(a, a)]
orderedPairs [(SlotNo, WithOrigin BlockNo, WithOrigin BlockNo)]
extrema
                ]
      where
        -- all pairs @(x, y)@ where @x@ precedes @y@ in the given list
        orderedPairs :: [a] -> [(a, a)]
        orderedPairs :: forall a. [a] -> [(a, a)]
orderedPairs = \case
            []   -> []
            a
x:[a]
ys -> (a -> [(a, a)] -> [(a, a)]) -> [(a, a)] -> [a] -> [(a, a)]
forall a b. (a -> b -> b) -> b -> [a] -> b
forall (t :: * -> *) a b.
Foldable t =>
(a -> b -> b) -> b -> t a -> b
foldr ((:) ((a, a) -> [(a, a)] -> [(a, a)])
-> (a -> (a, a)) -> a -> [(a, a)] -> [(a, a)]
forall b c a. (b -> c) -> (a -> b) -> a -> c
. (,) a
x) ([a] -> [(a, a)]
forall a. [a] -> [(a, a)]
orderedPairs [a]
ys) [a]
ys

        prop_growth :: (SlotNo, WithOrigin BlockNo)
                    -> (SlotNo, WithOrigin BlockNo)
                    -> Property
        prop_growth :: (SlotNo, WithOrigin BlockNo)
-> (SlotNo, WithOrigin BlockNo) -> Property
prop_growth (SlotNo
s1, WithOrigin BlockNo
b1) (SlotNo
s2, WithOrigin BlockNo
b2) =
            String -> Property -> Property
forall prop. Testable prop => String -> prop -> Property
counterexample ((SlotNo, SlotNo, WithOrigin BlockNo, WithOrigin BlockNo, Int)
-> String
forall a. Condense a => a -> String
condense (SlotNo
s1, SlotNo
s2, WithOrigin BlockNo
b1, WithOrigin BlockNo
b2, Int
numActiveSlots)) (Property -> Property) -> Property -> Property
forall a b. (a -> b) -> a -> b
$
            Property
nonNegativeGrowth Property -> Property -> Property
forall prop1 prop2.
(Testable prop1, Testable prop2) =>
prop1 -> prop2 -> Property
.&&.
            Property
sufficientGrowth
          where
            nonNegativeGrowth :: Property
nonNegativeGrowth =
                String -> Property -> Property
forall prop. Testable prop => String -> prop -> Property
counterexample String
"negative chain growth" (Property -> Property) -> Property -> Property
forall a b. (a -> b) -> a -> b
$
                    Bool -> Property
forall prop. Testable prop => prop -> Property
property (WithOrigin BlockNo
b2 WithOrigin BlockNo -> WithOrigin BlockNo -> Bool
forall a. Ord a => a -> a -> Bool
>= WithOrigin BlockNo
b1)

            sufficientGrowth :: Property
sufficientGrowth =
                String -> Property -> Property
forall prop. Testable prop => String -> prop -> Property
counterexample String
"insufficient chain growth" (Property -> Property) -> Property -> Property
forall a b. (a -> b) -> a -> b
$
                    Bool -> Property
forall prop. Testable prop => prop -> Property
property (Word64
d Word64 -> Word64 -> Bool
forall a. Ord a => a -> a -> Bool
>= Int -> Word64
forall a. Enum a => Int -> a
toEnum Int
numActiveSlots)

            BlockNo Word64
d = case (WithOrigin BlockNo
b1, WithOrigin BlockNo
b2) of
                          (NotOrigin BlockNo
b1', NotOrigin BlockNo
b2') -> BlockNo
b2' BlockNo -> BlockNo -> BlockNo
forall a. Num a => a -> a -> a
- BlockNo
b1'
                          (WithOrigin BlockNo
Origin,        NotOrigin BlockNo
b2') -> BlockNo
b2' BlockNo -> BlockNo -> BlockNo
forall a. Num a => a -> a -> a
+ BlockNo
1
                          (WithOrigin BlockNo
Origin,        WithOrigin BlockNo
Origin)        -> BlockNo
0
                          (NotOrigin BlockNo
_,   WithOrigin BlockNo
Origin)        -> String -> BlockNo
forall a. HasCallStack => String -> a
error String
"prop_growth: negative growth"
            numActiveSlots :: Int
numActiveSlots =
                Map SlotNo [CoreNodeId] -> Int
forall k a. Map k a -> Int
Map.size (Map SlotNo [CoreNodeId] -> Int) -> Map SlotNo [CoreNodeId] -> Int
forall a b. (a -> b) -> a -> b
$
                ((SlotNo -> [CoreNodeId] -> Bool)
 -> Map SlotNo [CoreNodeId] -> Map SlotNo [CoreNodeId])
-> Map SlotNo [CoreNodeId]
-> (SlotNo -> [CoreNodeId] -> Bool)
-> Map SlotNo [CoreNodeId]
forall a b c. (a -> b -> c) -> b -> a -> c
flip (SlotNo -> [CoreNodeId] -> Bool)
-> Map SlotNo [CoreNodeId] -> Map SlotNo [CoreNodeId]
forall k a. (k -> a -> Bool) -> Map k a -> Map k a
Map.filterWithKey (LeaderSchedule -> Map SlotNo [CoreNodeId]
getLeaderSchedule LeaderSchedule
growthSchedule) ((SlotNo -> [CoreNodeId] -> Bool) -> Map SlotNo [CoreNodeId])
-> (SlotNo -> [CoreNodeId] -> Bool) -> Map SlotNo [CoreNodeId]
forall a b. (a -> b) -> a -> b
$
                \SlotNo
slot [CoreNodeId]
ls -> SlotNo
s1 SlotNo -> SlotNo -> Bool
forall a. Ord a => a -> a -> Bool
<= SlotNo
slot Bool -> Bool -> Bool
&& SlotNo
slot SlotNo -> SlotNo -> Bool
forall a. Ord a => a -> a -> Bool
< SlotNo
s2 Bool -> Bool -> Bool
&& (Bool -> Bool
not (Bool -> Bool) -> ([CoreNodeId] -> Bool) -> [CoreNodeId] -> Bool
forall b c a. (b -> c) -> (a -> b) -> a -> c
. [CoreNodeId] -> Bool
forall a. [a] -> Bool
forall (t :: * -> *) a. Foldable t => t a -> Bool
null) [CoreNodeId]
ls

        -- @(s, min, max)@ the minimum and maximum block number of the tip of a
        -- chain at the onset of slot @s@.
        extrema :: [(SlotNo, WithOrigin BlockNo, WithOrigin BlockNo)]
        extrema :: [(SlotNo, WithOrigin BlockNo, WithOrigin BlockNo)]
extrema =
            [ case ((NodeId, WithOrigin BlockNo) -> WithOrigin BlockNo)
-> [(NodeId, WithOrigin BlockNo)] -> [WithOrigin BlockNo]
forall a b. (a -> b) -> [a] -> [b]
map (NodeId, WithOrigin BlockNo) -> WithOrigin BlockNo
forall a b. (a, b) -> b
snd [(NodeId, WithOrigin BlockNo)]
bnos' of
                  [] -> (SlotNo
slot, WithOrigin BlockNo
forall t. WithOrigin t
Origin, WithOrigin BlockNo
forall t. WithOrigin t
Origin)
                  [WithOrigin BlockNo]
o  -> (SlotNo
slot, [WithOrigin BlockNo] -> WithOrigin BlockNo
forall a. Ord a => [a] -> a
forall (t :: * -> *) a. (Foldable t, Ord a) => t a -> a
minimum [WithOrigin BlockNo]
o, [WithOrigin BlockNo] -> WithOrigin BlockNo
forall a. Ord a => [a] -> a
forall (t :: * -> *) a. (Foldable t, Ord a) => t a -> a
maximum [WithOrigin BlockNo]
o)
            | (SlotNo
slot, [(NodeId, WithOrigin BlockNo)]
bnos) <- [(SlotNo, [(NodeId, WithOrigin BlockNo)])]
tipBlockNos
            , let bnos' :: [(NodeId, WithOrigin BlockNo)]
bnos' = ((NodeId, WithOrigin BlockNo) -> Bool)
-> [(NodeId, WithOrigin BlockNo)] -> [(NodeId, WithOrigin BlockNo)]
forall a. (a -> Bool) -> [a] -> [a]
filter (SlotNo -> NodeId -> Bool
joinedBefore SlotNo
slot (NodeId -> Bool)
-> ((NodeId, WithOrigin BlockNo) -> NodeId)
-> (NodeId, WithOrigin BlockNo)
-> Bool
forall b c a. (b -> c) -> (a -> b) -> a -> c
. (NodeId, WithOrigin BlockNo) -> NodeId
forall a b. (a, b) -> a
fst) [(NodeId, WithOrigin BlockNo)]
bnos
            ]

        joinedBefore :: SlotNo -> NodeId -> Bool
joinedBefore SlotNo
slot NodeId
nid = HasCallStack => NodeJoinPlan -> NodeId -> SlotNo
NodeJoinPlan -> NodeId -> SlotNo
nodeIdJoinSlot NodeJoinPlan
nodeJoinPlan NodeId
nid SlotNo -> SlotNo -> Bool
forall a. Ord a => a -> a -> Bool
< SlotNo
slot

    -- swizzled 'testOutputTipBlockNos'
    tipBlockNos :: [(SlotNo, [(NodeId, WithOrigin BlockNo)])]
    tipBlockNos :: [(SlotNo, [(NodeId, WithOrigin BlockNo)])]
tipBlockNos =
        Map SlotNo [(NodeId, WithOrigin BlockNo)]
-> [(SlotNo, [(NodeId, WithOrigin BlockNo)])]
forall k a. Map k a -> [(k, a)]
Map.toAscList (Map SlotNo [(NodeId, WithOrigin BlockNo)]
 -> [(SlotNo, [(NodeId, WithOrigin BlockNo)])])
-> Map SlotNo [(NodeId, WithOrigin BlockNo)]
-> [(SlotNo, [(NodeId, WithOrigin BlockNo)])]
forall a b. (a -> b) -> a -> b
$
        (Map NodeId (WithOrigin BlockNo) -> [(NodeId, WithOrigin BlockNo)])
-> Map SlotNo (Map NodeId (WithOrigin BlockNo))
-> Map SlotNo [(NodeId, WithOrigin BlockNo)]
forall a b. (a -> b) -> Map SlotNo a -> Map SlotNo b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap Map NodeId (WithOrigin BlockNo) -> [(NodeId, WithOrigin BlockNo)]
forall k a. Map k a -> [(k, a)]
Map.toAscList (Map SlotNo (Map NodeId (WithOrigin BlockNo))
 -> Map SlotNo [(NodeId, WithOrigin BlockNo)])
-> Map SlotNo (Map NodeId (WithOrigin BlockNo))
-> Map SlotNo [(NodeId, WithOrigin BlockNo)]
forall a b. (a -> b) -> a -> b
$
        Map SlotNo (Map NodeId (WithOrigin BlockNo))
testOutputTipBlockNos

    -- In the paper <https://eprint.iacr.org/2017/573/20171115:00183>, a
    -- /message/ carries a chain from one party to another. When a party forges
    -- a block, it \"diffuses\" the chain with that block as its head by
    -- sending a message to each other party (actually, to itself too, but
    -- that's ultimately redundant). The adversary is able to delay each
    -- message differently, so some parties may receive it before others do.
    -- Once a party receives a message, the party can consider that chain for
    -- selection.
    --
    -- In the implementation, on the other hand, our messages are varied and
    -- much more granular than a whole chain. We therefore observe a delay
    -- analogous to the paper's /message/ /delay/ by comparing the slot in
    -- which a block is added to each node's ChainDB against the slot in which
    -- that block was forged.
    --
    -- Since our mock network currently introduces only negligible latency
    -- compared to the slot duration, we generally expect all messages to have
    -- no delay: they should arrive to all nodes during the same slot in which
    -- they were forged. However, some delays are expected, due to nodes
    -- joining late and also due to the practicality of the ChainSync and
    -- BlockFetch policies, which try to avoid /unnecessary/ header/block
    -- fetches. See the relevant comments below.
    --
    -- NOTE: This current property does not check for interminable message
    -- delay: i.e. for blocks that were never added to some ChainDBs. It only
    -- checks the slot difference once a message does arrive. This seems
    -- acceptable: if there are no Common Prefix or Chain Growth violations,
    -- then each message must have either arrived or ultimately been
    -- irrelevant.
    --
    prop_no_unexpected_message_delays :: HasCallStack => Property
    prop_no_unexpected_message_delays :: HasCallStack => Property
prop_no_unexpected_message_delays =
        [Property] -> Property
forall prop. Testable prop => [prop] -> Property
conjoin ([Property] -> Property) -> [Property] -> Property
forall a b. (a -> b) -> a -> b
$
        [ case RealPoint blk
p of
              RealPoint SlotNo
sendSlot HeaderHash blk
hsh ->
                  NodeId -> SlotNo -> SlotNo -> HeaderHash blk -> BlockNo -> Property
forall {p}.
Show p =>
NodeId -> SlotNo -> SlotNo -> p -> BlockNo -> Property
prop1 NodeId
nid SlotNo
recvSlot SlotNo
sendSlot HeaderHash blk
hsh BlockNo
bno
        | (NodeId
nid, Map SlotNo (Set (RealPoint blk, BlockNo))
m)          <- Map NodeId (Map SlotNo (Set (RealPoint blk, BlockNo)))
-> [(NodeId, Map SlotNo (Set (RealPoint blk, BlockNo)))]
forall k a. Map k a -> [(k, a)]
Map.toList Map NodeId (Map SlotNo (Set (RealPoint blk, BlockNo)))
adds
        , (SlotNo
recvSlot, Set (RealPoint blk, BlockNo)
pbnos) <- Map SlotNo (Set (RealPoint blk, BlockNo))
-> [(SlotNo, Set (RealPoint blk, BlockNo))]
forall k a. Map k a -> [(k, a)]
Map.toList Map SlotNo (Set (RealPoint blk, BlockNo))
m
        , (RealPoint blk
p, BlockNo
bno)          <- Set (RealPoint blk, BlockNo) -> [(RealPoint blk, BlockNo)]
forall a. Set a -> [a]
Set.toList Set (RealPoint blk, BlockNo)
pbnos
        ]
      where
        -- INVARIANT: these AddBlock events are *not* for EBBs
        adds :: Map NodeId (Map SlotNo (Set (RealPoint blk, BlockNo)))
adds = NodeOutput blk -> Map SlotNo (Set (RealPoint blk, BlockNo))
forall blk.
NodeOutput blk -> Map SlotNo (Set (RealPoint blk, BlockNo))
nodeOutputAdds (NodeOutput blk -> Map SlotNo (Set (RealPoint blk, BlockNo)))
-> Map NodeId (NodeOutput blk)
-> Map NodeId (Map SlotNo (Set (RealPoint blk, BlockNo)))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Map NodeId (NodeOutput blk)
testOutputNodes

        prop1 :: NodeId -> SlotNo -> SlotNo -> p -> BlockNo -> Property
prop1 NodeId
nid SlotNo
recvSlot SlotNo
sendSlot p
hsh BlockNo
bno =
            String -> Bool -> Property
forall prop. Testable prop => String -> prop -> Property
counterexample String
msg (Bool -> Property) -> Bool -> Property
forall a b. (a -> b) -> a -> b
$
            Bool
delayOK Bool -> Bool -> Bool
|| Bool
noDelay
          where
            msg :: String
msg =
                String
"Unexpected message delay " String -> ShowS
forall a. Semigroup a => a -> a -> a
<>
                String
"(" String -> ShowS
forall a. Semigroup a => a -> a -> a
<> String
"recipient: " String -> ShowS
forall a. Semigroup a => a -> a -> a
<> NodeId -> String
forall a. Condense a => a -> String
condense NodeId
nid String -> ShowS
forall a. Semigroup a => a -> a -> a
<>
                String
"," String -> ShowS
forall a. Semigroup a => a -> a -> a
<> String
"expected receive slot: "
                    String -> ShowS
forall a. Semigroup a => a -> a -> a
<> SlotNo -> String
forall a. Condense a => a -> String
condense SlotNo
firstPossibleReception String -> ShowS
forall a. Semigroup a => a -> a -> a
<>
                String
"," String -> ShowS
forall a. Semigroup a => a -> a -> a
<> String
"actual receive slot: " String -> ShowS
forall a. Semigroup a => a -> a -> a
<> SlotNo -> String
forall a. Condense a => a -> String
condense SlotNo
recvSlot String -> ShowS
forall a. Semigroup a => a -> a -> a
<>
                String
"," String -> ShowS
forall a. Semigroup a => a -> a -> a
<> String
"blockHash: " String -> ShowS
forall a. Semigroup a => a -> a -> a
<> p -> String
forall a. Show a => a -> String
show p
hsh String -> ShowS
forall a. Semigroup a => a -> a -> a
<>
                String
"," String -> ShowS
forall a. Semigroup a => a -> a -> a
<> String
"blockNo: " String -> ShowS
forall a. Semigroup a => a -> a -> a
<> Word64 -> String
forall a. Condense a => a -> String
condense (BlockNo -> Word64
unBlockNo BlockNo
bno) String -> ShowS
forall a. Semigroup a => a -> a -> a
<>
                String
")"

            -- a node cannot receive a block until both exist
            firstPossibleReception :: SlotNo
firstPossibleReception =
                HasCallStack => NodeJoinPlan -> NodeId -> SlotNo
NodeJoinPlan -> NodeId -> SlotNo
nodeIdJoinSlot NodeJoinPlan
nodeJoinPlan NodeId
nid SlotNo -> SlotNo -> SlotNo
forall a. Ord a => a -> a -> a
`max` SlotNo
sendSlot

            noDelay :: Bool
noDelay = SlotNo
recvSlot SlotNo -> SlotNo -> Bool
forall a. Eq a => a -> a -> Bool
== SlotNo
firstPossibleReception

            delayOK :: Bool
delayOK = Bool
delayOK1 Bool -> Bool -> Bool
|| Bool
delayOK2

            -- When a node leads in the same slot in which it joins the
            -- network, it immediately forges a single block on top of Genesis;
            -- this block then prevents it from fetching the network's current
            -- chain if that also consists of just one block.
            --
            -- NOTE This predicate is more general than that specific scenario,
            -- but we don't anticipate it wholly masking any interesting cases.
            delayOK1 :: Bool
delayOK1 = BlockNo
firstBlockNo BlockNo -> BlockNo -> Bool
forall a. Eq a => a -> a -> Bool
== BlockNo
bno

            -- When a slot has multiple leaders, each node chooses one of the
            -- mutually-exclusive forged blocks and won't fetch any of the
            -- others until it's later compelled to switch to a chain
            -- containing one of them
            --
            -- TODO This predicate is more general than that specific scenario,
            -- and should be tightened accordingly. We currently anticipate
            -- that Issues #229 and #230 will handle that.
            delayOK2 :: Bool
delayOK2 = case SlotNo -> Map SlotNo [CoreNodeId] -> Maybe [CoreNodeId]
forall k a. Ord k => k -> Map k a -> Maybe a
Map.lookup SlotNo
sendSlot Map SlotNo [CoreNodeId]
sched of
                Just (CoreNodeId
_:CoreNodeId
_:[CoreNodeId]
_) -> Bool
True
                Maybe [CoreNodeId]
_            -> Bool
False
              where
                LeaderSchedule Map SlotNo [CoreNodeId]
sched = LeaderSchedule
actualLeaderSchedule

    hasNodeRekey :: Bool
    hasNodeRekey :: Bool
hasNodeRekey =
        NodeRestart
NodeRekey NodeRestart -> Set NodeRestart -> Bool
forall a. Ord a => a -> Set a -> Bool
`Set.member` ((Map CoreNodeId NodeRestart -> Set NodeRestart)
-> Map SlotNo (Map CoreNodeId NodeRestart) -> Set NodeRestart
forall m a. Monoid m => (a -> m) -> Map SlotNo a -> m
forall (t :: * -> *) m a.
(Foldable t, Monoid m) =>
(a -> m) -> t a -> m
foldMap ((Map CoreNodeId NodeRestart -> Set NodeRestart)
 -> Map SlotNo (Map CoreNodeId NodeRestart) -> Set NodeRestart)
-> ((NodeRestart -> Set NodeRestart)
    -> Map CoreNodeId NodeRestart -> Set NodeRestart)
-> (NodeRestart -> Set NodeRestart)
-> Map SlotNo (Map CoreNodeId NodeRestart)
-> Set NodeRestart
forall b c a. (b -> c) -> (a -> b) -> a -> c
. (NodeRestart -> Set NodeRestart)
-> Map CoreNodeId NodeRestart -> Set NodeRestart
forall m a. Monoid m => (a -> m) -> Map CoreNodeId a -> m
forall (t :: * -> *) m a.
(Foldable t, Monoid m) =>
(a -> m) -> t a -> m
foldMap) NodeRestart -> Set NodeRestart
forall a. a -> Set a
Set.singleton Map SlotNo (Map CoreNodeId NodeRestart)
m
      where
        NodeRestarts Map SlotNo (Map CoreNodeId NodeRestart)
m = NodeRestarts
nodeRestarts

    -- Average number of txs/block
    averageNumTxs :: Double
    averageNumTxs :: Double
averageNumTxs =
          [Double] -> Double
average
        ([Double] -> Double)
-> ([Chain blk] -> [Double]) -> [Chain blk] -> Double
forall b c a. (b -> c) -> (a -> b) -> a -> c
. (blk -> Double) -> [blk] -> [Double]
forall a b. (a -> b) -> [a] -> [b]
map (Word64 -> Double
forall a b. (Integral a, Num b) => a -> b
fromIntegral (Word64 -> Double) -> (blk -> Word64) -> blk -> Double
forall b c a. (b -> c) -> (a -> b) -> a -> c
. blk -> Word64
countTxs)
        ([blk] -> [Double])
-> ([Chain blk] -> [blk]) -> [Chain blk] -> [Double]
forall b c a. (b -> c) -> (a -> b) -> a -> c
. (Chain blk -> [blk]) -> [Chain blk] -> [blk]
forall (t :: * -> *) a b. Foldable t => (a -> [b]) -> t a -> [b]
concatMap Chain blk -> [blk]
forall block. Chain block -> [block]
MockChain.toOldestFirst
        ([Chain blk] -> Double) -> [Chain blk] -> Double
forall a b. (a -> b) -> a -> b
$ Map NodeId (Chain blk) -> [Chain blk]
forall k a. Map k a -> [a]
Map.elems Map NodeId (Chain blk)
nodeChains
      where
        average :: [Double] -> Double
        average :: [Double] -> Double
average [] = Double
0
        average [Double]
xs = [Double] -> Double
forall a. Num a => [a] -> a
forall (t :: * -> *) a. (Foldable t, Num a) => t a -> a
sum [Double]
xs Double -> Double -> Double
forall a. Fractional a => a -> a -> a
/ Int -> Double
forall a b. (Integral a, Num b) => a -> b
fromIntegral ([Double] -> Int
forall a. [a] -> Int
forall (t :: * -> *) a. Foldable t => t a -> Int
length [Double]
xs)

    -- The 'prop_valid_block' argument could, for example, check for no expired
    -- transactions.
    prop_no_invalid_blocks :: Property
    prop_no_invalid_blocks :: Property
prop_no_invalid_blocks = [Property] -> Property
forall prop. Testable prop => [prop] -> Property
conjoin ([Property] -> Property) -> [Property] -> Property
forall a b. (a -> b) -> a -> b
$
        [ String -> Property -> Property
forall prop. Testable prop => String -> prop -> Property
counterexample
            (String
"In slot " String -> ShowS
forall a. Semigroup a => a -> a -> a
<> SlotNo -> String
forall a. Condense a => a -> String
condense SlotNo
s String -> ShowS
forall a. Semigroup a => a -> a -> a
<> String
", node " String -> ShowS
forall a. Semigroup a => a -> a -> a
<> NodeId -> String
forall a. Condense a => a -> String
condense NodeId
nid) (Property -> Property) -> Property -> Property
forall a b. (a -> b) -> a -> b
$
          String -> Property -> Property
forall prop. Testable prop => String -> prop -> Property
counterexample (String
"forged an invalid block " String -> ShowS
forall a. Semigroup a => a -> a -> a
<> blk -> String
forall a. Condense a => a -> String
condense blk
blk) (Property -> Property) -> Property -> Property
forall a b. (a -> b) -> a -> b
$
          blk -> Property
prop_valid_block blk
blk
        | (NodeId
nid, NodeOutput{Map SlotNo blk
nodeOutputForges :: forall blk. NodeOutput blk -> Map SlotNo blk
nodeOutputForges :: Map SlotNo blk
nodeOutputForges}) <- Map NodeId (NodeOutput blk) -> [(NodeId, NodeOutput blk)]
forall k a. Map k a -> [(k, a)]
Map.toList Map NodeId (NodeOutput blk)
testOutputNodes
          -- checking all forged blocks, even if they were never or only
          -- temporarily selected.
        , (SlotNo
s, blk
blk) <- Map SlotNo blk -> [(SlotNo, blk)]
forall k a. Map k a -> [(k, a)]
Map.toAscList Map SlotNo blk
nodeOutputForges
        ]

    -- Check that all self-issued blocks are pipelined.
    prop_pipelining :: Property
    prop_pipelining :: Property
prop_pipelining = [Property] -> Property
forall prop. Testable prop => [prop] -> Property
conjoin
        [ String -> Property -> Property
forall prop. Testable prop => String -> prop -> Property
counterexample (String
"Node " String -> ShowS
forall a. Semigroup a => a -> a -> a
<> NodeId -> String
forall a. Condense a => a -> String
condense NodeId
nid String -> ShowS
forall a. Semigroup a => a -> a -> a
<> String
" did not pipeline") (Property -> Property) -> Property -> Property
forall a b. (a -> b) -> a -> b
$
          String -> Property -> Property
forall prop. Testable prop => String -> prop -> Property
counterexample (String
"some of its blocks forged as the sole slot leader:") (Property -> Property) -> Property -> Property
forall a b. (a -> b) -> a -> b
$
          String -> Bool -> Property
forall prop. Testable prop => String -> prop -> Property
counterexample (Set (Point blk) -> String
forall a. Condense a => a -> String
condense Set (Point blk)
forgedButNotPipelined) (Bool -> Property) -> Bool -> Property
forall a b. (a -> b) -> a -> b
$
          Set (Point blk) -> Bool
forall a. Set a -> Bool
Set.null Set (Point blk)
forgedButNotPipelined
        | (NodeId
nid, NodeOutput
            { Map SlotNo blk
nodeOutputForges :: forall blk. NodeOutput blk -> Map SlotNo blk
nodeOutputForges :: Map SlotNo blk
nodeOutputForges
            , [TracePipeliningEvent blk]
nodePipeliningEvents :: [TracePipeliningEvent blk]
nodePipeliningEvents :: forall blk. NodeOutput blk -> [TracePipeliningEvent blk]
nodePipeliningEvents
            }) <- Map NodeId (NodeOutput blk) -> [(NodeId, NodeOutput blk)]
forall k a. Map k a -> [(k, a)]
Map.toList Map NodeId (NodeOutput blk)
testOutputNodes
        , CoreId CoreNodeId
cnid <- [NodeId
nid]
        , let tentativePoints :: Set (Point blk)
tentativePoints = [Point blk] -> Set (Point blk)
forall a. Ord a => [a] -> Set a
Set.fromList
                [ Header blk -> Point blk
forall blk. HasHeader (Header blk) => Header blk -> Point blk
headerPoint Header blk
hdr
                | ChainDB.SetTentativeHeader Header blk
hdr Enclosing' ()
FallingEdge <- [TracePipeliningEvent blk]
nodePipeliningEvents
                ]
              forgedAsSoleLeaderPoints :: Set (Point blk)
forgedAsSoleLeaderPoints = [Point blk] -> Set (Point blk)
forall a. Ord a => [a] -> Set a
Set.fromList ([Point blk] -> Set (Point blk)) -> [Point blk] -> Set (Point blk)
forall a b. (a -> b) -> a -> b
$
                [ blk -> Point blk
forall block. HasHeader block => block -> Point block
blockPoint blk
blk
                | blk
blk <- Map SlotNo blk -> [blk]
forall k a. Map k a -> [a]
Map.elems Map SlotNo blk
nodeOutputForges
                , let s :: SlotNo
s = blk -> SlotNo
forall b. HasHeader b => b -> SlotNo
blockSlot blk
blk
                      NodeRestarts Map SlotNo (Map CoreNodeId NodeRestart)
nrs = NodeRestarts
nodeRestarts
                , LeaderSchedule -> Map SlotNo [CoreNodeId]
getLeaderSchedule LeaderSchedule
actualLeaderSchedule Map SlotNo [CoreNodeId] -> SlotNo -> [CoreNodeId]
forall k a. Ord k => Map k a -> k -> a
Map.! SlotNo
s [CoreNodeId] -> [CoreNodeId] -> Bool
forall a. Eq a => a -> a -> Bool
== [CoreNodeId
cnid]
                  -- When the node is restarted while it is a slot
                  -- leader, this property is often not satisfied in
                  -- the Byron ThreadNet tests. As diffusion
                  -- pipelining is concerned with up-to-date,
                  -- long-running nodes, we ignore this edge case.
                , CoreNodeId
cnid CoreNodeId -> Map CoreNodeId NodeRestart -> Bool
forall k a. Ord k => k -> Map k a -> Bool
`Map.notMember` Map CoreNodeId NodeRestart
-> SlotNo
-> Map SlotNo (Map CoreNodeId NodeRestart)
-> Map CoreNodeId NodeRestart
forall k a. Ord k => a -> k -> Map k a -> a
Map.findWithDefault Map CoreNodeId NodeRestart
forall a. Monoid a => a
mempty SlotNo
s Map SlotNo (Map CoreNodeId NodeRestart)
nrs
                ]
              forgedButNotPipelined :: Set (Point blk)
forgedButNotPipelined =
                Set (Point blk)
forgedAsSoleLeaderPoints Set (Point blk) -> Set (Point blk) -> Set (Point blk)
forall a. Ord a => Set a -> Set a -> Set a
Set.\\ Set (Point blk)
tentativePoints
        ]

{-------------------------------------------------------------------------------
  Final chains properties
-------------------------------------------------------------------------------}

-- | What was the most number of /signed/ blocks needed to be dropped from a
-- final chain in order to reach the final chains' common prefix?
--
-- NOTE: This count excludes EBBs.
calcFinalIntersectionDepth :: forall blk. (BA.HasHeader blk)
                           => PropGeneralArgs blk
                           -> TestOutput blk
                           -> NumBlocks
calcFinalIntersectionDepth :: forall blk.
HasHeader blk =>
PropGeneralArgs blk -> TestOutput blk -> NumBlocks
calcFinalIntersectionDepth PropGeneralArgs blk
pga TestOutput blk
testOutput =
    Word64 -> NumBlocks
NumBlocks (Word64 -> NumBlocks) -> Word64 -> NumBlocks
forall a b. (a -> b) -> a -> b
$ BlockNo -> Word64
unBlockNo (BlockNo -> Word64) -> BlockNo -> Word64
forall a b. (a -> b) -> a -> b
$
    case (Chain blk -> WithOrigin BlockNo
forall block. HasHeader block => Chain block -> WithOrigin BlockNo
MockChain.headBlockNo Chain blk
commonPrefix, WithOrigin BlockNo
maxLength) of
      (WithOrigin BlockNo
BA.Origin,       WithOrigin BlockNo
BA.Origin)      -> BlockNo
0
      (WithOrigin BlockNo
BA.Origin,       BA.NotOrigin BlockNo
b) -> BlockNo
1 BlockNo -> BlockNo -> BlockNo
forall a. Num a => a -> a -> a
+ BlockNo
b BlockNo -> BlockNo -> BlockNo
forall a. Num a => a -> a -> a
- BlockNo
pgaFirstBlockNo
      (BA.NotOrigin{},  WithOrigin BlockNo
BA.Origin)      -> String -> BlockNo
forall a. HasCallStack => String -> a
error String
"impossible"
      (BA.NotOrigin BlockNo
cp, BA.NotOrigin BlockNo
b) ->
          Bool -> BlockNo -> BlockNo
forall a. HasCallStack => Bool -> a -> a
assert (BlockNo
b BlockNo -> BlockNo -> Bool
forall a. Ord a => a -> a -> Bool
>= BlockNo
cp) (BlockNo -> BlockNo) -> BlockNo -> BlockNo
forall a b. (a -> b) -> a -> b
$   -- guaranteed by the foldl below
          BlockNo
b BlockNo -> BlockNo -> BlockNo
forall a. Num a => a -> a -> a
- BlockNo
cp
  where
    PropGeneralArgs{BlockNo
pgaFirstBlockNo :: forall blk. PropGeneralArgs blk -> BlockNo
pgaFirstBlockNo :: BlockNo
pgaFirstBlockNo} = PropGeneralArgs blk
pga
    TestOutput{Map NodeId (NodeOutput blk)
testOutputNodes :: forall blk. TestOutput blk -> Map NodeId (NodeOutput blk)
testOutputNodes :: Map NodeId (NodeOutput blk)
testOutputNodes}      = TestOutput blk
testOutput

    -- length of longest chain
    maxLength    :: BA.WithOrigin BlockNo
    -- the common prefix
    commonPrefix :: MockChain.Chain blk
    (WithOrigin BlockNo
maxLength, Chain blk
commonPrefix) =
        case ((NodeId, NodeOutput blk) -> (WithOrigin BlockNo, Chain blk))
-> [(NodeId, NodeOutput blk)] -> [(WithOrigin BlockNo, Chain blk)]
forall a b. (a -> b) -> [a] -> [b]
map (NodeId, NodeOutput blk) -> (WithOrigin BlockNo, Chain blk)
forall {blk} {a}.
HasHeader blk =>
(a, NodeOutput blk) -> (WithOrigin BlockNo, Chain blk)
prj ([(NodeId, NodeOutput blk)] -> [(WithOrigin BlockNo, Chain blk)])
-> [(NodeId, NodeOutput blk)] -> [(WithOrigin BlockNo, Chain blk)]
forall a b. (a -> b) -> a -> b
$ Map NodeId (NodeOutput blk) -> [(NodeId, NodeOutput blk)]
forall k a. Map k a -> [(k, a)]
Map.toList Map NodeId (NodeOutput blk)
testOutputNodes of
          []   -> (WithOrigin BlockNo
forall t. WithOrigin t
BA.Origin, Chain blk
forall block. Chain block
MockChain.Genesis)
          (WithOrigin BlockNo, Chain blk)
x:[(WithOrigin BlockNo, Chain blk)]
xs -> ((WithOrigin BlockNo, Chain blk)
 -> (WithOrigin BlockNo, Chain blk)
 -> (WithOrigin BlockNo, Chain blk))
-> (WithOrigin BlockNo, Chain blk)
-> [(WithOrigin BlockNo, Chain blk)]
-> (WithOrigin BlockNo, Chain blk)
forall b a. (b -> a -> b) -> b -> [a] -> b
forall (t :: * -> *) b a.
Foldable t =>
(b -> a -> b) -> b -> t a -> b
foldl (WithOrigin BlockNo, Chain blk)
-> (WithOrigin BlockNo, Chain blk)
-> (WithOrigin BlockNo, Chain blk)
forall {b} {a}.
(HasHeader b, Ord a) =>
(a, Chain b) -> (a, Chain b) -> (a, Chain b)
combine (WithOrigin BlockNo, Chain blk)
x [(WithOrigin BlockNo, Chain blk)]
xs
      where
        prj :: (a, NodeOutput blk) -> (WithOrigin BlockNo, Chain blk)
prj (a
_nid, NodeOutput{Chain blk
nodeOutputFinalChain :: forall blk. NodeOutput blk -> Chain blk
nodeOutputFinalChain :: Chain blk
nodeOutputFinalChain}) = (WithOrigin BlockNo
d, Chain blk
c)
          where
            d :: WithOrigin BlockNo
d = Chain blk -> WithOrigin BlockNo
forall block. HasHeader block => Chain block -> WithOrigin BlockNo
MockChain.headBlockNo Chain blk
nodeOutputFinalChain
            c :: Chain blk
c = Chain blk
nodeOutputFinalChain

        combine :: (a, Chain b) -> (a, Chain b) -> (a, Chain b)
combine (a
dl, Chain b
cl) (a
dr, Chain b
cr) = (a -> a -> a
forall a. Ord a => a -> a -> a
max a
dl a
dr, Chain b -> Chain b -> Chain b
forall b. HasHeader b => Chain b -> Chain b -> Chain b
chainCommonPrefix Chain b
cl Chain b
cr)

-- | All final chains have the same block number
prop_inSync :: forall blk. (BA.HasHeader blk)
            => TestOutput blk -> Property
prop_inSync :: forall blk. HasHeader blk => TestOutput blk -> Property
prop_inSync TestOutput blk
testOutput =
    String -> Property -> Property
forall prop. Testable prop => String -> prop -> Property
counterexample ([WithOrigin BlockNo] -> String
forall a. Show a => a -> String
show [WithOrigin BlockNo]
lengths) (Property -> Property) -> Property -> Property
forall a b. (a -> b) -> a -> b
$
    String -> Property -> Property
forall prop. Testable prop => String -> prop -> Property
counterexample String
"the nodes' final chains have different block numbers" (Property -> Property) -> Property -> Property
forall a b. (a -> b) -> a -> b
$
    Bool -> Property
forall prop. Testable prop => prop -> Property
property (Bool -> Property) -> Bool -> Property
forall a b. (a -> b) -> a -> b
$
    case [WithOrigin BlockNo]
lengths of
      []   -> Bool
False
      WithOrigin BlockNo
l:[WithOrigin BlockNo]
ls -> (WithOrigin BlockNo -> Bool) -> [WithOrigin BlockNo] -> Bool
forall (t :: * -> *) a. Foldable t => (a -> Bool) -> t a -> Bool
all (WithOrigin BlockNo -> WithOrigin BlockNo -> Bool
forall a. Eq a => a -> a -> Bool
== WithOrigin BlockNo
l) [WithOrigin BlockNo]
ls
  where
    TestOutput{Map NodeId (NodeOutput blk)
testOutputNodes :: forall blk. TestOutput blk -> Map NodeId (NodeOutput blk)
testOutputNodes :: Map NodeId (NodeOutput blk)
testOutputNodes} = TestOutput blk
testOutput

    -- the length of each final chain
    lengths :: [BA.WithOrigin BlockNo]
    lengths :: [WithOrigin BlockNo]
lengths =
        [ Chain blk -> WithOrigin BlockNo
forall block. HasHeader block => Chain block -> WithOrigin BlockNo
MockChain.headBlockNo Chain blk
nodeOutputFinalChain
        | (NodeId
_nid, NodeOutput blk
no) <- Map NodeId (NodeOutput blk) -> [(NodeId, NodeOutput blk)]
forall k a. Map k a -> [(k, a)]
Map.toList Map NodeId (NodeOutput blk)
testOutputNodes
        , let NodeOutput{Chain blk
nodeOutputFinalChain :: forall blk. NodeOutput blk -> Chain blk
nodeOutputFinalChain :: Chain blk
nodeOutputFinalChain} = NodeOutput blk
no
        ]