{-# LANGUAGE BangPatterns #-}
{-# LANGUAGE DeriveGeneric #-}
{-# LANGUAGE FlexibleContexts #-}
{-# LANGUAGE GADTs #-}
{-# LANGUAGE NamedFieldPuns #-}
{-# LANGUAGE ScopedTypeVariables #-}
{-# LANGUAGE TupleSections #-}
{-# LANGUAGE TypeOperators #-}
module Test.Ouroboros.Network.TxSubmission (tests) where
import Prelude hiding (seq)
import NoThunks.Class (NoThunks)
import Control.Concurrent.Class.MonadSTM
import Control.Exception (SomeException (..))
import Control.Monad.Class.MonadAsync
import Control.Monad.Class.MonadFork
import Control.Monad.Class.MonadSay
import Control.Monad.Class.MonadST
import Control.Monad.Class.MonadThrow
import Control.Monad.Class.MonadTime.SI
import Control.Monad.Class.MonadTimer.SI
import Control.Monad.IOSim hiding (SimResult)
import Control.Tracer (Tracer (..), contramap, nullTracer, showTracing,
traceWith)
import Codec.CBOR.Decoding qualified as CBOR
import Codec.CBOR.Encoding qualified as CBOR
import Codec.CBOR.Read qualified as CBOR
import Data.ByteString.Lazy (ByteString)
import Data.ByteString.Lazy qualified as BSL
import Data.Foldable as Foldable (find, foldl', toList)
import Data.Function (on)
import Data.List (intercalate, nubBy)
import Data.Maybe (fromMaybe, isJust)
import Data.Sequence (Seq)
import Data.Sequence qualified as Seq
import Data.Set qualified as Set
import Data.Word (Word16)
import GHC.Generics (Generic)
import Network.TypedProtocol.Codec
import Ouroboros.Network.Channel
import Ouroboros.Network.ControlMessage (ControlMessage (..), ControlMessageSTM)
import Ouroboros.Network.Driver
import Ouroboros.Network.NodeToNode (NodeToNodeVersion (..))
import Ouroboros.Network.Protocol.TxSubmission2.Client
import Ouroboros.Network.Protocol.TxSubmission2.Codec
import Ouroboros.Network.Protocol.TxSubmission2.Server
import Ouroboros.Network.Protocol.TxSubmission2.Type
import Ouroboros.Network.TxSubmission.Inbound
import Ouroboros.Network.TxSubmission.Mempool.Reader
import Ouroboros.Network.TxSubmission.Outbound
import Ouroboros.Network.Util.ShowProxy
import Ouroboros.Network.Testing.Utils
import Test.QuickCheck
import Test.Tasty (TestTree, testGroup)
import Test.Tasty.QuickCheck (testProperty)
import Text.Printf
tests :: TestTree
tests :: TestTree
tests = TestName -> [TestTree] -> TestTree
testGroup TestName
"TxSubmission"
[ TestName
-> (Positive Word16
-> NonEmptyList (Tx Int)
-> Maybe (Positive SmallDelay)
-> Property)
-> TestTree
forall a. Testable a => TestName -> a -> TestTree
testProperty TestName
"txSubmission" Positive Word16
-> NonEmptyList (Tx Int) -> Maybe (Positive SmallDelay) -> Property
prop_txSubmission
, TestName -> Property -> TestTree
forall a. Testable a => TestName -> a -> TestTree
testProperty TestName
"x" Property
prop_x
]
data Tx txid = Tx {
forall txid. Tx txid -> txid
getTxId :: txid,
forall txid. Tx txid -> SizeInBytes
getTxSize :: !SizeInBytes,
forall txid. Tx txid -> Bool
getTxValid :: Bool
}
deriving (Tx txid -> Tx txid -> Bool
(Tx txid -> Tx txid -> Bool)
-> (Tx txid -> Tx txid -> Bool) -> Eq (Tx txid)
forall txid. Eq txid => Tx txid -> Tx txid -> Bool
forall a. (a -> a -> Bool) -> (a -> a -> Bool) -> Eq a
$c== :: forall txid. Eq txid => Tx txid -> Tx txid -> Bool
== :: Tx txid -> Tx txid -> Bool
$c/= :: forall txid. Eq txid => Tx txid -> Tx txid -> Bool
/= :: Tx txid -> Tx txid -> Bool
Eq, Int -> Tx txid -> ShowS
[Tx txid] -> ShowS
Tx txid -> TestName
(Int -> Tx txid -> ShowS)
-> (Tx txid -> TestName) -> ([Tx txid] -> ShowS) -> Show (Tx txid)
forall txid. Show txid => Int -> Tx txid -> ShowS
forall txid. Show txid => [Tx txid] -> ShowS
forall txid. Show txid => Tx txid -> TestName
forall a.
(Int -> a -> ShowS) -> (a -> TestName) -> ([a] -> ShowS) -> Show a
$cshowsPrec :: forall txid. Show txid => Int -> Tx txid -> ShowS
showsPrec :: Int -> Tx txid -> ShowS
$cshow :: forall txid. Show txid => Tx txid -> TestName
show :: Tx txid -> TestName
$cshowList :: forall txid. Show txid => [Tx txid] -> ShowS
showList :: [Tx txid] -> ShowS
Show, (forall x. Tx txid -> Rep (Tx txid) x)
-> (forall x. Rep (Tx txid) x -> Tx txid) -> Generic (Tx txid)
forall x. Rep (Tx txid) x -> Tx txid
forall x. Tx txid -> Rep (Tx txid) x
forall a.
(forall x. a -> Rep a x) -> (forall x. Rep a x -> a) -> Generic a
forall txid x. Rep (Tx txid) x -> Tx txid
forall txid x. Tx txid -> Rep (Tx txid) x
$cfrom :: forall txid x. Tx txid -> Rep (Tx txid) x
from :: forall x. Tx txid -> Rep (Tx txid) x
$cto :: forall txid x. Rep (Tx txid) x -> Tx txid
to :: forall x. Rep (Tx txid) x -> Tx txid
Generic)
instance NoThunks txid => NoThunks (Tx txid)
instance ShowProxy txid => ShowProxy (Tx txid) where
showProxy :: Proxy (Tx txid) -> TestName
showProxy Proxy (Tx txid)
_ = TestName
"Tx " TestName -> ShowS
forall a. [a] -> [a] -> [a]
++ Proxy txid -> TestName
forall {k} (p :: k). ShowProxy p => Proxy p -> TestName
showProxy (Proxy txid
forall {k} (t :: k). Proxy t
Proxy :: Proxy txid)
instance Arbitrary txid => Arbitrary (Tx txid) where
arbitrary :: Gen (Tx txid)
arbitrary =
txid -> SizeInBytes -> Bool -> Tx txid
forall txid. txid -> SizeInBytes -> Bool -> Tx txid
Tx (txid -> SizeInBytes -> Bool -> Tx txid)
-> Gen txid -> Gen (SizeInBytes -> Bool -> Tx txid)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Gen txid
forall a. Arbitrary a => Gen a
arbitrary
Gen (SizeInBytes -> Bool -> Tx txid)
-> Gen SizeInBytes -> Gen (Bool -> Tx txid)
forall a b. Gen (a -> b) -> Gen a -> Gen b
forall (f :: * -> *) a b. Applicative f => f (a -> b) -> f a -> f b
<*> (SizeInBytes, SizeInBytes) -> Gen SizeInBytes
forall a. Enum a => (a, a) -> Gen a
chooseEnum (SizeInBytes
0, SizeInBytes
maxTxSize)
Gen (Bool -> Tx txid) -> Gen Bool -> Gen (Tx txid)
forall a b. Gen (a -> b) -> Gen a -> Gen b
forall (f :: * -> *) a b. Applicative f => f (a -> b) -> f a -> f b
<*> [(Int, Gen Bool)] -> Gen Bool
forall a. [(Int, Gen a)] -> Gen a
frequency [ (Int
3, Bool -> Gen Bool
forall a. a -> Gen a
forall (f :: * -> *) a. Applicative f => a -> f a
pure Bool
True)
, (Int
1, Bool -> Gen Bool
forall a. a -> Gen a
forall (f :: * -> *) a. Applicative f => a -> f a
pure Bool
False)
]
maxTxSize :: SizeInBytes
maxTxSize :: SizeInBytes
maxTxSize = SizeInBytes
65536
newtype Mempool m txid = Mempool (TVar m (Seq (Tx txid)))
emptyMempool :: MonadSTM m => m (Mempool m txid)
emptyMempool :: forall (m :: * -> *) txid. MonadSTM m => m (Mempool m txid)
emptyMempool = TVar m (Seq (Tx txid)) -> Mempool m txid
forall (m :: * -> *) txid. TVar m (Seq (Tx txid)) -> Mempool m txid
Mempool (TVar m (Seq (Tx txid)) -> Mempool m txid)
-> m (TVar m (Seq (Tx txid))) -> m (Mempool m txid)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Seq (Tx txid) -> m (TVar m (Seq (Tx txid)))
forall a. a -> m (TVar m a)
forall (m :: * -> *) a. MonadSTM m => a -> m (TVar m a)
newTVarIO Seq (Tx txid)
forall a. Seq a
Seq.empty
newMempool :: MonadSTM m
=> [Tx txid]
-> m (Mempool m txid)
newMempool :: forall (m :: * -> *) txid.
MonadSTM m =>
[Tx txid] -> m (Mempool m txid)
newMempool = (TVar m (Seq (Tx txid)) -> Mempool m txid)
-> m (TVar m (Seq (Tx txid))) -> m (Mempool m txid)
forall a b. (a -> b) -> m a -> m b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap TVar m (Seq (Tx txid)) -> Mempool m txid
forall (m :: * -> *) txid. TVar m (Seq (Tx txid)) -> Mempool m txid
Mempool
(m (TVar m (Seq (Tx txid))) -> m (Mempool m txid))
-> ([Tx txid] -> m (TVar m (Seq (Tx txid))))
-> [Tx txid]
-> m (Mempool m txid)
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Seq (Tx txid) -> m (TVar m (Seq (Tx txid)))
forall a. a -> m (TVar m a)
forall (m :: * -> *) a. MonadSTM m => a -> m (TVar m a)
newTVarIO
(Seq (Tx txid) -> m (TVar m (Seq (Tx txid))))
-> ([Tx txid] -> Seq (Tx txid))
-> [Tx txid]
-> m (TVar m (Seq (Tx txid)))
forall b c a. (b -> c) -> (a -> b) -> a -> c
. [Tx txid] -> Seq (Tx txid)
forall a. [a] -> Seq a
Seq.fromList
readMempool :: MonadSTM m => Mempool m txid -> m [Tx txid]
readMempool :: forall (m :: * -> *) txid.
MonadSTM m =>
Mempool m txid -> m [Tx txid]
readMempool (Mempool TVar m (Seq (Tx txid))
mempool) = Seq (Tx txid) -> [Tx txid]
forall a. Seq a -> [a]
forall (t :: * -> *) a. Foldable t => t a -> [a]
toList (Seq (Tx txid) -> [Tx txid]) -> m (Seq (Tx txid)) -> m [Tx txid]
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> TVar m (Seq (Tx txid)) -> m (Seq (Tx txid))
forall a. TVar m a -> m a
forall (m :: * -> *) a. MonadSTM m => TVar m a -> m a
readTVarIO TVar m (Seq (Tx txid))
mempool
getMempoolReader :: forall txid m.
( MonadSTM m
, Eq txid
)
=> Mempool m txid
-> TxSubmissionMempoolReader txid (Tx txid) Int m
getMempoolReader :: forall txid (m :: * -> *).
(MonadSTM m, Eq txid) =>
Mempool m txid -> TxSubmissionMempoolReader txid (Tx txid) Int m
getMempoolReader (Mempool TVar m (Seq (Tx txid))
mempool) =
TxSubmissionMempoolReader { STM m (MempoolSnapshot txid (Tx txid) Int)
mempoolGetSnapshot :: STM m (MempoolSnapshot txid (Tx txid) Int)
mempoolGetSnapshot :: STM m (MempoolSnapshot txid (Tx txid) Int)
mempoolGetSnapshot, mempoolZeroIdx :: Int
mempoolZeroIdx = Int
0 }
where
mempoolGetSnapshot :: STM m (MempoolSnapshot txid (Tx txid) Int)
mempoolGetSnapshot :: STM m (MempoolSnapshot txid (Tx txid) Int)
mempoolGetSnapshot = Seq (Tx txid) -> MempoolSnapshot txid (Tx txid) Int
getSnapshot (Seq (Tx txid) -> MempoolSnapshot txid (Tx txid) Int)
-> STM m (Seq (Tx txid))
-> STM m (MempoolSnapshot txid (Tx txid) Int)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> TVar m (Seq (Tx txid)) -> STM m (Seq (Tx txid))
forall a. TVar m a -> STM m a
forall (m :: * -> *) a. MonadSTM m => TVar m a -> STM m a
readTVar TVar m (Seq (Tx txid))
mempool
getSnapshot :: Seq (Tx txid)
-> MempoolSnapshot txid (Tx txid) Int
getSnapshot :: Seq (Tx txid) -> MempoolSnapshot txid (Tx txid) Int
getSnapshot Seq (Tx txid)
seq =
MempoolSnapshot {
mempoolTxIdsAfter :: Int -> [(txid, Int, SizeInBytes)]
mempoolTxIdsAfter =
\Int
idx -> (Int -> Tx txid -> (txid, Int, SizeInBytes))
-> [Int] -> [Tx txid] -> [(txid, Int, SizeInBytes)]
forall a b c. (a -> b -> c) -> [a] -> [b] -> [c]
zipWith Int -> Tx txid -> (txid, Int, SizeInBytes)
f [Int
idx Int -> Int -> Int
forall a. Num a => a -> a -> a
+ Int
1 ..] (Seq (Tx txid) -> [Tx txid]
forall a. Seq a -> [a]
forall (t :: * -> *) a. Foldable t => t a -> [a]
toList (Seq (Tx txid) -> [Tx txid]) -> Seq (Tx txid) -> [Tx txid]
forall a b. (a -> b) -> a -> b
$ Int -> Seq (Tx txid) -> Seq (Tx txid)
forall a. Int -> Seq a -> Seq a
Seq.drop Int
idx Seq (Tx txid)
seq),
mempoolLookupTx :: Int -> Maybe (Tx txid)
mempoolLookupTx = (Int -> Seq (Tx txid) -> Maybe (Tx txid))
-> Seq (Tx txid) -> Int -> Maybe (Tx txid)
forall a b c. (a -> b -> c) -> b -> a -> c
flip Int -> Seq (Tx txid) -> Maybe (Tx txid)
forall a. Int -> Seq a -> Maybe a
Seq.lookup Seq (Tx txid)
seq (Int -> Maybe (Tx txid)) -> (Int -> Int) -> Int -> Maybe (Tx txid)
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Int -> Int
forall a. Enum a => a -> a
pred,
mempoolHasTx :: txid -> Bool
mempoolHasTx = \txid
txid -> Maybe (Tx txid) -> Bool
forall a. Maybe a -> Bool
isJust (Maybe (Tx txid) -> Bool) -> Maybe (Tx txid) -> Bool
forall a b. (a -> b) -> a -> b
$ (Tx txid -> Bool) -> Seq (Tx txid) -> Maybe (Tx txid)
forall (t :: * -> *) a. Foldable t => (a -> Bool) -> t a -> Maybe a
find (\Tx txid
tx -> Tx txid -> txid
forall txid. Tx txid -> txid
getTxId Tx txid
tx txid -> txid -> Bool
forall a. Eq a => a -> a -> Bool
== txid
txid) Seq (Tx txid)
seq
}
f :: Int -> Tx txid -> (txid, Int, SizeInBytes)
f :: Int -> Tx txid -> (txid, Int, SizeInBytes)
f Int
idx Tx {txid
getTxId :: forall txid. Tx txid -> txid
getTxId :: txid
getTxId, SizeInBytes
getTxSize :: forall txid. Tx txid -> SizeInBytes
getTxSize :: SizeInBytes
getTxSize} = (txid
getTxId, Int
idx, SizeInBytes
getTxSize)
getMempoolWriter :: forall txid m.
( MonadSTM m
, Ord txid
)
=> Mempool m txid
-> TxSubmissionMempoolWriter txid (Tx txid) Int m
getMempoolWriter :: forall txid (m :: * -> *).
(MonadSTM m, Ord txid) =>
Mempool m txid -> TxSubmissionMempoolWriter txid (Tx txid) Int m
getMempoolWriter (Mempool TVar m (Seq (Tx txid))
mempool) =
TxSubmissionMempoolWriter {
txId :: Tx txid -> txid
txId = Tx txid -> txid
forall txid. Tx txid -> txid
getTxId,
mempoolAddTxs :: [Tx txid] -> m [txid]
mempoolAddTxs = \[Tx txid]
txs -> do
STM m [txid] -> m [txid]
forall a. HasCallStack => STM m a -> m a
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
STM m a -> m a
atomically (STM m [txid] -> m [txid]) -> STM m [txid] -> m [txid]
forall a b. (a -> b) -> a -> b
$ do
mempoolTxs <- TVar m (Seq (Tx txid)) -> STM m (Seq (Tx txid))
forall a. TVar m a -> STM m a
forall (m :: * -> *) a. MonadSTM m => TVar m a -> STM m a
readTVar TVar m (Seq (Tx txid))
mempool
let currentIds = [txid] -> Set txid
forall a. Ord a => [a] -> Set a
Set.fromList ((Tx txid -> txid) -> [Tx txid] -> [txid]
forall a b. (a -> b) -> [a] -> [b]
map Tx txid -> txid
forall txid. Tx txid -> txid
getTxId (Seq (Tx txid) -> [Tx txid]
forall a. Seq a -> [a]
forall (t :: * -> *) a. Foldable t => t a -> [a]
toList Seq (Tx txid)
mempoolTxs))
validTxs = (Tx txid -> Tx txid -> Bool) -> [Tx txid] -> [Tx txid]
forall a. (a -> a -> Bool) -> [a] -> [a]
nubBy ((txid -> txid -> Bool)
-> (Tx txid -> txid) -> Tx txid -> Tx txid -> Bool
forall b c a. (b -> b -> c) -> (a -> b) -> a -> a -> c
on txid -> txid -> Bool
forall a. Eq a => a -> a -> Bool
(==) Tx txid -> txid
forall txid. Tx txid -> txid
getTxId)
([Tx txid] -> [Tx txid]) -> [Tx txid] -> [Tx txid]
forall a b. (a -> b) -> a -> b
$ (Tx txid -> Bool) -> [Tx txid] -> [Tx txid]
forall a. (a -> Bool) -> [a] -> [a]
filter
(\Tx { txid
getTxId :: forall txid. Tx txid -> txid
getTxId :: txid
getTxId, Bool
getTxValid :: forall txid. Tx txid -> Bool
getTxValid :: Bool
getTxValid } ->
Bool
getTxValid
Bool -> Bool -> Bool
&& txid
getTxId txid -> Set txid -> Bool
forall a. Ord a => a -> Set a -> Bool
`Set.notMember` Set txid
currentIds)
([Tx txid] -> [Tx txid]) -> [Tx txid] -> [Tx txid]
forall a b. (a -> b) -> a -> b
$ [Tx txid]
txs
mempoolTxs' = (Seq (Tx txid) -> Tx txid -> Seq (Tx txid))
-> Seq (Tx txid) -> [Tx txid] -> Seq (Tx txid)
forall b a. (b -> a -> b) -> b -> [a] -> b
forall (t :: * -> *) b a.
Foldable t =>
(b -> a -> b) -> b -> t a -> b
Foldable.foldl' Seq (Tx txid) -> Tx txid -> Seq (Tx txid)
forall a. Seq a -> a -> Seq a
(Seq.|>) Seq (Tx txid)
mempoolTxs [Tx txid]
validTxs
writeTVar mempool mempoolTxs'
return (map getTxId validTxs)
}
txSubmissionCodec2 :: MonadST m
=> Codec (TxSubmission2 Int (Tx Int))
CBOR.DeserialiseFailure m ByteString
txSubmissionCodec2 :: forall (m :: * -> *).
MonadST m =>
Codec (TxSubmission2 Int (Tx Int)) DeserialiseFailure m ByteString
txSubmissionCodec2 =
(Int -> Encoding)
-> (forall s. Decoder s Int)
-> (Tx Int -> Encoding)
-> (forall s. Decoder s (Tx Int))
-> Codec
(TxSubmission2 Int (Tx Int)) DeserialiseFailure m ByteString
forall txid tx (m :: * -> *).
MonadST m =>
(txid -> Encoding)
-> (forall s. Decoder s txid)
-> (tx -> Encoding)
-> (forall s. Decoder s tx)
-> Codec (TxSubmission2 txid tx) DeserialiseFailure m ByteString
codecTxSubmission2 Int -> Encoding
CBOR.encodeInt Decoder s Int
forall s. Decoder s Int
CBOR.decodeInt
Tx Int -> Encoding
encodeTx Decoder s (Tx Int)
forall s. Decoder s (Tx Int)
decodeTx
where
encodeTx :: Tx Int -> Encoding
encodeTx Tx {Int
getTxId :: forall txid. Tx txid -> txid
getTxId :: Int
getTxId, SizeInBytes
getTxSize :: forall txid. Tx txid -> SizeInBytes
getTxSize :: SizeInBytes
getTxSize, Bool
getTxValid :: forall txid. Tx txid -> Bool
getTxValid :: Bool
getTxValid} =
Word -> Encoding
CBOR.encodeListLen Word
3
Encoding -> Encoding -> Encoding
forall a. Semigroup a => a -> a -> a
<> Int -> Encoding
CBOR.encodeInt Int
getTxId
Encoding -> Encoding -> Encoding
forall a. Semigroup a => a -> a -> a
<> Word32 -> Encoding
CBOR.encodeWord32 (SizeInBytes -> Word32
getSizeInBytes SizeInBytes
getTxSize)
Encoding -> Encoding -> Encoding
forall a. Semigroup a => a -> a -> a
<> Bool -> Encoding
CBOR.encodeBool Bool
getTxValid
decodeTx :: Decoder s (Tx Int)
decodeTx = do
_ <- Decoder s Int
forall s. Decoder s Int
CBOR.decodeListLen
Tx <$> CBOR.decodeInt
<*> (SizeInBytes <$> CBOR.decodeWord32)
<*> CBOR.decodeBool
txSubmissionSimulation
:: forall m txid.
( MonadAsync m
, MonadDelay m
, MonadFork m
, MonadLabelledSTM m
, MonadMask m
, MonadSay m
, MonadST m
, MonadTimer m
, MonadThrow (STM m)
, Ord txid
, ShowProxy txid
, NoThunks (Tx txid)
, txid ~ Int
)
=> NumTxIdsToAck
-> [Tx txid]
-> ControlMessageSTM m
-> Maybe DiffTime
-> Maybe DiffTime
-> m ([Tx txid], [Tx txid])
txSubmissionSimulation :: forall (m :: * -> *) txid.
(MonadAsync m, MonadDelay m, MonadFork m, MonadLabelledSTM m,
MonadMask m, MonadSay m, MonadST m, MonadTimer m,
MonadThrow (STM m), Ord txid, ShowProxy txid, NoThunks (Tx txid),
txid ~ Int) =>
NumTxIdsToAck
-> [Tx txid]
-> ControlMessageSTM m
-> Maybe DiffTime
-> Maybe DiffTime
-> m ([Tx txid], [Tx txid])
txSubmissionSimulation NumTxIdsToAck
maxUnacked [Tx txid]
outboundTxs
ControlMessageSTM m
controlMessageSTM
Maybe DiffTime
inboundDelay Maybe DiffTime
outboundDelay = do
inboundMempool <- m (Mempool m txid)
forall (m :: * -> *) txid. MonadSTM m => m (Mempool m txid)
emptyMempool
outboundMempool <- newMempool outboundTxs
(outboundChannel, inboundChannel) <- createConnectedBufferedChannels
(fromIntegral maxUnacked)
outboundAsync <-
async $ runPeerWithLimits
(("OUTBOUND",) `contramap` verboseTracer)
txSubmissionCodec2
(byteLimitsTxSubmission2 (fromIntegral . BSL.length))
timeLimitsTxSubmission2
(maybe id delayChannel outboundDelay outboundChannel)
(txSubmissionClientPeer (outboundPeer outboundMempool))
inboundAsync <-
async $ runPipelinedPeerWithLimits
(("INBOUND",) `contramap` verboseTracer)
txSubmissionCodec2
(byteLimitsTxSubmission2 (fromIntegral . BSL.length))
timeLimitsTxSubmission2
(maybe id delayChannel inboundDelay inboundChannel)
(txSubmissionServerPeerPipelined (inboundPeer inboundMempool))
_ <- waitAnyCancel [ outboundAsync, inboundAsync ]
inmp <- readMempool inboundMempool
outmp <- readMempool outboundMempool
return (inmp, outmp)
where
outboundPeer :: Mempool m txid -> TxSubmissionClient txid (Tx txid) m ()
outboundPeer :: Mempool m txid -> TxSubmissionClient txid (Tx txid) m ()
outboundPeer Mempool m txid
outboundMempool =
Tracer m (TraceTxSubmissionOutbound txid (Tx txid))
-> NumTxIdsToAck
-> TxSubmissionMempoolReader txid (Tx txid) Int m
-> NodeToNodeVersion
-> ControlMessageSTM m
-> TxSubmissionClient txid (Tx txid) m ()
forall txid tx idx (m :: * -> *).
(Ord txid, Ord idx, MonadSTM m, MonadThrow m) =>
Tracer m (TraceTxSubmissionOutbound txid tx)
-> NumTxIdsToAck
-> TxSubmissionMempoolReader txid tx idx m
-> NodeToNodeVersion
-> ControlMessageSTM m
-> TxSubmissionClient txid tx m ()
txSubmissionOutbound
Tracer m (TraceTxSubmissionOutbound txid (Tx txid))
forall (m :: * -> *) a. Applicative m => Tracer m a
nullTracer
NumTxIdsToAck
maxUnacked
(Mempool m txid -> TxSubmissionMempoolReader txid (Tx txid) Int m
forall txid (m :: * -> *).
(MonadSTM m, Eq txid) =>
Mempool m txid -> TxSubmissionMempoolReader txid (Tx txid) Int m
getMempoolReader Mempool m txid
outboundMempool)
(NodeToNodeVersion
forall a. Bounded a => a
maxBound :: NodeToNodeVersion)
ControlMessageSTM m
controlMessageSTM
inboundPeer :: Mempool m txid -> TxSubmissionServerPipelined txid (Tx txid) m ()
inboundPeer :: Mempool m txid -> TxSubmissionServerPipelined txid (Tx txid) m ()
inboundPeer Mempool m txid
inboundMempool =
Tracer m (TraceTxSubmissionInbound txid (Tx txid))
-> NumTxIdsToAck
-> TxSubmissionMempoolReader txid (Tx txid) Int m
-> TxSubmissionMempoolWriter txid (Tx txid) Int m
-> NodeToNodeVersion
-> TxSubmissionServerPipelined txid (Tx txid) m ()
forall txid tx idx (m :: * -> *).
(Ord txid, NoThunks txid, NoThunks tx, MonadSTM m, MonadThrow m,
MonadDelay m) =>
Tracer m (TraceTxSubmissionInbound txid tx)
-> NumTxIdsToAck
-> TxSubmissionMempoolReader txid tx idx m
-> TxSubmissionMempoolWriter txid tx idx m
-> NodeToNodeVersion
-> TxSubmissionServerPipelined txid tx m ()
txSubmissionInbound
Tracer m (TraceTxSubmissionInbound txid (Tx txid))
forall (m :: * -> *) a. Applicative m => Tracer m a
nullTracer
NumTxIdsToAck
maxUnacked
(Mempool m txid -> TxSubmissionMempoolReader txid (Tx txid) Int m
forall txid (m :: * -> *).
(MonadSTM m, Eq txid) =>
Mempool m txid -> TxSubmissionMempoolReader txid (Tx txid) Int m
getMempoolReader Mempool m txid
inboundMempool)
(Mempool m txid -> TxSubmissionMempoolWriter txid (Tx txid) Int m
forall txid (m :: * -> *).
(MonadSTM m, Ord txid) =>
Mempool m txid -> TxSubmissionMempoolWriter txid (Tx txid) Int m
getMempoolWriter Mempool m txid
inboundMempool)
(NodeToNodeVersion
forall a. Bounded a => a
maxBound :: NodeToNodeVersion)
newtype LargeNonEmptyList a = LargeNonEmpty { forall a. LargeNonEmptyList a -> [a]
getLargeNonEmpty :: [a] }
deriving Int -> LargeNonEmptyList a -> ShowS
[LargeNonEmptyList a] -> ShowS
LargeNonEmptyList a -> TestName
(Int -> LargeNonEmptyList a -> ShowS)
-> (LargeNonEmptyList a -> TestName)
-> ([LargeNonEmptyList a] -> ShowS)
-> Show (LargeNonEmptyList a)
forall a. Show a => Int -> LargeNonEmptyList a -> ShowS
forall a. Show a => [LargeNonEmptyList a] -> ShowS
forall a. Show a => LargeNonEmptyList a -> TestName
forall a.
(Int -> a -> ShowS) -> (a -> TestName) -> ([a] -> ShowS) -> Show a
$cshowsPrec :: forall a. Show a => Int -> LargeNonEmptyList a -> ShowS
showsPrec :: Int -> LargeNonEmptyList a -> ShowS
$cshow :: forall a. Show a => LargeNonEmptyList a -> TestName
show :: LargeNonEmptyList a -> TestName
$cshowList :: forall a. Show a => [LargeNonEmptyList a] -> ShowS
showList :: [LargeNonEmptyList a] -> ShowS
Show
instance Arbitrary a => Arbitrary (LargeNonEmptyList a) where
arbitrary :: Gen (LargeNonEmptyList a)
arbitrary =
[a] -> LargeNonEmptyList a
forall a. [a] -> LargeNonEmptyList a
LargeNonEmpty ([a] -> LargeNonEmptyList a)
-> Gen [a] -> Gen (LargeNonEmptyList a)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Gen [a] -> ([a] -> Bool) -> Gen [a]
forall a. Gen a -> (a -> Bool) -> Gen a
suchThat (Int -> Gen [a] -> Gen [a]
forall a. Int -> Gen a -> Gen a
resize Int
500 (Gen a -> Gen [a]
forall a. Gen a -> Gen [a]
listOf Gen a
forall a. Arbitrary a => Gen a
arbitrary)) ((Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
>Int
25) (Int -> Bool) -> ([a] -> Int) -> [a] -> Bool
forall b c a. (b -> c) -> (a -> b) -> a -> c
. [a] -> Int
forall a. [a] -> Int
forall (t :: * -> *) a. Foldable t => t a -> Int
length)
prop_txSubmission :: Positive Word16
-> NonEmptyList (Tx Int)
-> Maybe (Positive SmallDelay)
-> Property
prop_txSubmission :: Positive Word16
-> NonEmptyList (Tx Int) -> Maybe (Positive SmallDelay) -> Property
prop_txSubmission (Positive Word16
maxUnacked) (NonEmpty [Tx Int]
outboundTxs) Maybe (Positive SmallDelay)
delay =
let mbDelayTime :: Maybe DiffTime
mbDelayTime = SmallDelay -> DiffTime
getSmallDelay (SmallDelay -> DiffTime)
-> (Positive SmallDelay -> SmallDelay)
-> Positive SmallDelay
-> DiffTime
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Positive SmallDelay -> SmallDelay
forall a. Positive a -> a
getPositive (Positive SmallDelay -> DiffTime)
-> Maybe (Positive SmallDelay) -> Maybe DiffTime
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Maybe (Positive SmallDelay)
delay
tr :: SimTrace ([Tx Int], [Tx Int])
tr = ((forall s. IOSim s ([Tx Int], [Tx Int]))
-> SimTrace ([Tx Int], [Tx Int])
forall a. (forall s. IOSim s a) -> SimTrace a
runSimTrace ((forall s. IOSim s ([Tx Int], [Tx Int]))
-> SimTrace ([Tx Int], [Tx Int]))
-> (forall s. IOSim s ([Tx Int], [Tx Int]))
-> SimTrace ([Tx Int], [Tx Int])
forall a b. (a -> b) -> a -> b
$ do
controlMessageVar <- ControlMessage -> IOSim s (TVar (IOSim s) ControlMessage)
forall a. a -> IOSim s (TVar (IOSim s) a)
forall (m :: * -> *) a. MonadSTM m => a -> m (TVar m a)
newTVarIO ControlMessage
Continue
_ <-
async $ do
threadDelay
(fromMaybe 1 mbDelayTime
* realToFrac (length outboundTxs `div` 4))
atomically (writeTVar controlMessageVar Terminate)
txSubmissionSimulation
(NumTxIdsToAck maxUnacked) outboundTxs
(readTVar controlMessageVar)
mbDelayTime mbDelayTime
) in
IO Property -> Property
forall prop. Testable prop => IO prop -> Property
ioProperty (IO Property -> Property) -> IO Property -> Property
forall a b. (a -> b) -> a -> b
$ do
tr' <- SimTrace ([Tx Int], [Tx Int])
-> IO (SimResult ([Tx Int], [Tx Int]))
forall a. SimTrace a -> IO (SimResult a)
evaluateTrace SimTrace ([Tx Int], [Tx Int])
tr
case tr' of
SimException SomeException
e Context
trace -> do
Property -> IO Property
forall a. a -> IO a
forall (m :: * -> *) a. Monad m => a -> m a
return (Property -> IO Property) -> Property -> IO Property
forall a b. (a -> b) -> a -> b
$ TestName -> Bool -> Property
forall prop. Testable prop => TestName -> prop -> Property
counterexample (TestName -> Context -> TestName
forall a. [a] -> [[a]] -> [a]
intercalate TestName
"\n" (Context -> TestName) -> Context -> TestName
forall a b. (a -> b) -> a -> b
$ SomeException -> TestName
forall a. Show a => a -> TestName
show SomeException
e TestName -> Context -> Context
forall a. a -> [a] -> [a]
: Context
trace) Bool
False
SimDeadLock Context
trace -> do
Property -> IO Property
forall a. a -> IO a
forall (m :: * -> *) a. Monad m => a -> m a
return (Property -> IO Property) -> Property -> IO Property
forall a b. (a -> b) -> a -> b
$ TestName -> Bool -> Property
forall prop. Testable prop => TestName -> prop -> Property
counterexample (TestName -> Context -> TestName
forall a. [a] -> [[a]] -> [a]
intercalate TestName
"\n" (Context -> TestName) -> Context -> TestName
forall a b. (a -> b) -> a -> b
$ TestName
"Deadlock" TestName -> Context -> Context
forall a. a -> [a] -> [a]
: Context
trace) Bool
False
SimReturn ([Tx Int]
inmp, [Tx Int]
outmp) Context
_trace -> do
let outUniqueTxIds :: [Tx Int]
outUniqueTxIds = (Tx Int -> Tx Int -> Bool) -> [Tx Int] -> [Tx Int]
forall a. (a -> a -> Bool) -> [a] -> [a]
nubBy ((Int -> Int -> Bool) -> (Tx Int -> Int) -> Tx Int -> Tx Int -> Bool
forall b c a. (b -> b -> c) -> (a -> b) -> a -> a -> c
on Int -> Int -> Bool
forall a. Eq a => a -> a -> Bool
(==) Tx Int -> Int
forall txid. Tx txid -> txid
getTxId) [Tx Int]
outmp
outValidTxs :: [Tx Int]
outValidTxs = (Tx Int -> Bool) -> [Tx Int] -> [Tx Int]
forall a. (a -> Bool) -> [a] -> [a]
filter Tx Int -> Bool
forall txid. Tx txid -> Bool
getTxValid [Tx Int]
outmp
case ([Tx Int] -> Int
forall a. [a] -> Int
forall (t :: * -> *) a. Foldable t => t a -> Int
length [Tx Int]
outUniqueTxIds Int -> Int -> Bool
forall a. Eq a => a -> a -> Bool
== [Tx Int] -> Int
forall a. [a] -> Int
forall (t :: * -> *) a. Foldable t => t a -> Int
length [Tx Int]
outmp, [Tx Int] -> Int
forall a. [a] -> Int
forall (t :: * -> *) a. Foldable t => t a -> Int
length [Tx Int]
outValidTxs Int -> Int -> Bool
forall a. Eq a => a -> a -> Bool
== [Tx Int] -> Int
forall a. [a] -> Int
forall (t :: * -> *) a. Foldable t => t a -> Int
length [Tx Int]
outmp) of
(Bool
True, Bool
True) ->
Property -> IO Property
forall a. a -> IO a
forall (m :: * -> *) a. Monad m => a -> m a
return (Property -> IO Property) -> Property -> IO Property
forall a b. (a -> b) -> a -> b
$ [Tx Int]
inmp [Tx Int] -> [Tx Int] -> Property
forall a. (Eq a, Show a) => a -> a -> Property
=== Int -> [Tx Int] -> [Tx Int]
forall a. Int -> [a] -> [a]
take ([Tx Int] -> Int
forall a. [a] -> Int
forall (t :: * -> *) a. Foldable t => t a -> Int
length [Tx Int]
inmp) [Tx Int]
outValidTxs
(Bool
True, Bool
False) ->
Property -> IO Property
forall a. a -> IO a
forall (m :: * -> *) a. Monad m => a -> m a
return (Property -> IO Property) -> Property -> IO Property
forall a b. (a -> b) -> a -> b
$ [Tx Int]
inmp [Tx Int] -> [Tx Int] -> Property
forall a. (Eq a, Show a) => a -> a -> Property
=== Int -> [Tx Int] -> [Tx Int]
forall a. Int -> [a] -> [a]
take ([Tx Int] -> Int
forall a. [a] -> Int
forall (t :: * -> *) a. Foldable t => t a -> Int
length [Tx Int]
inmp) [Tx Int]
outValidTxs
(Bool
False, Bool
True) ->
Property -> IO Property
forall a. a -> IO a
forall (m :: * -> *) a. Monad m => a -> m a
return (Property -> IO Property) -> Property -> IO Property
forall a b. (a -> b) -> a -> b
$ (Tx Int -> Int) -> [Tx Int] -> [Int]
forall a b. (a -> b) -> [a] -> [b]
map Tx Int -> Int
forall txid. Tx txid -> txid
getTxId [Tx Int]
inmp [Int] -> [Int] -> Property
forall a. (Eq a, Show a) => a -> a -> Property
=== Int -> [Int] -> [Int]
forall a. Int -> [a] -> [a]
take ([Tx Int] -> Int
forall a. [a] -> Int
forall (t :: * -> *) a. Foldable t => t a -> Int
length [Tx Int]
inmp) ((Tx Int -> Int) -> [Tx Int] -> [Int]
forall a b. (a -> b) -> [a] -> [b]
map Tx Int -> Int
forall txid. Tx txid -> txid
getTxId ([Tx Int] -> [Int]) -> [Tx Int] -> [Int]
forall a b. (a -> b) -> a -> b
$
(Tx Int -> Bool) -> [Tx Int] -> [Tx Int]
forall a. (a -> Bool) -> [a] -> [a]
filter Tx Int -> Bool
forall txid. Tx txid -> Bool
getTxValid [Tx Int]
outUniqueTxIds)
(Bool
False, Bool
False)
-> Property -> IO Property
forall a. a -> IO a
forall (m :: * -> *) a. Monad m => a -> m a
return (Property -> IO Property) -> Property -> IO Property
forall a b. (a -> b) -> a -> b
$ Bool -> Property
forall prop. Testable prop => prop -> Property
property Bool
True
prop_x :: Property
prop_x :: Property
prop_x = Positive Word16
-> NonEmptyList (Tx Int) -> Maybe (Positive SmallDelay) -> Property
prop_txSubmission
Positive {getPositive :: Word16
getPositive = Word16
3}
NonEmpty {getNonEmpty :: [Tx Int]
getNonEmpty = [Tx {getTxId :: Int
getTxId = -Int
83, getTxSize :: SizeInBytes
getTxSize = Word32 -> SizeInBytes
SizeInBytes Word32
62352, getTxValid :: Bool
getTxValid = Bool
True},Tx {getTxId :: Int
getTxId = Int
66, getTxSize :: SizeInBytes
getTxSize = Word32 -> SizeInBytes
SizeInBytes Word32
37084, getTxValid :: Bool
getTxValid = Bool
True},Tx {getTxId :: Int
getTxId = Int
55, getTxSize :: SizeInBytes
getTxSize = Word32 -> SizeInBytes
SizeInBytes Word32
54825, getTxValid :: Bool
getTxValid = Bool
False},Tx {getTxId :: Int
getTxId = -Int
94, getTxSize :: SizeInBytes
getTxSize = Word32 -> SizeInBytes
SizeInBytes Word32
54298, getTxValid :: Bool
getTxValid = Bool
True},Tx {getTxId :: Int
getTxId = -Int
83, getTxSize :: SizeInBytes
getTxSize = Word32 -> SizeInBytes
SizeInBytes Word32
30932, getTxValid :: Bool
getTxValid = Bool
True},Tx {getTxId :: Int
getTxId = Int
33, getTxSize :: SizeInBytes
getTxSize = Word32 -> SizeInBytes
SizeInBytes Word32
40377, getTxValid :: Bool
getTxValid = Bool
True},Tx {getTxId :: Int
getTxId = Int
87, getTxSize :: SizeInBytes
getTxSize = Word32 -> SizeInBytes
SizeInBytes Word32
42883, getTxValid :: Bool
getTxValid = Bool
False},Tx {getTxId :: Int
getTxId = -Int
87, getTxSize :: SizeInBytes
getTxSize = Word32 -> SizeInBytes
SizeInBytes Word32
21529, getTxValid :: Bool
getTxValid = Bool
True},Tx {getTxId :: Int
getTxId = Int
85, getTxSize :: SizeInBytes
getTxSize = Word32 -> SizeInBytes
SizeInBytes Word32
15222, getTxValid :: Bool
getTxValid = Bool
True},Tx {getTxId :: Int
getTxId = -Int
13, getTxSize :: SizeInBytes
getTxSize = Word32 -> SizeInBytes
SizeInBytes Word32
529, getTxValid :: Bool
getTxValid = Bool
True},Tx {getTxId :: Int
getTxId = -Int
21, getTxSize :: SizeInBytes
getTxSize = Word32 -> SizeInBytes
SizeInBytes Word32
14755, getTxValid :: Bool
getTxValid = Bool
True},Tx {getTxId :: Int
getTxId = Int
37, getTxSize :: SizeInBytes
getTxSize = Word32 -> SizeInBytes
SizeInBytes Word32
3921, getTxValid :: Bool
getTxValid = Bool
True},Tx {getTxId :: Int
getTxId = -Int
44, getTxSize :: SizeInBytes
getTxSize = Word32 -> SizeInBytes
SizeInBytes Word32
42390, getTxValid :: Bool
getTxValid = Bool
True},Tx {getTxId :: Int
getTxId = Int
47, getTxSize :: SizeInBytes
getTxSize = Word32 -> SizeInBytes
SizeInBytes Word32
27061, getTxValid :: Bool
getTxValid = Bool
False},Tx {getTxId :: Int
getTxId = Int
64, getTxSize :: SizeInBytes
getTxSize = Word32 -> SizeInBytes
SizeInBytes Word32
8540, getTxValid :: Bool
getTxValid = Bool
True},Tx {getTxId :: Int
getTxId = -Int
85, getTxSize :: SizeInBytes
getTxSize = Word32 -> SizeInBytes
SizeInBytes Word32
15138, getTxValid :: Bool
getTxValid = Bool
False},Tx {getTxId :: Int
getTxId = -Int
23, getTxSize :: SizeInBytes
getTxSize = Word32 -> SizeInBytes
SizeInBytes Word32
16317, getTxValid :: Bool
getTxValid = Bool
False},Tx {getTxId :: Int
getTxId = -Int
35, getTxSize :: SizeInBytes
getTxSize = Word32 -> SizeInBytes
SizeInBytes Word32
4372, getTxValid :: Bool
getTxValid = Bool
True},Tx {getTxId :: Int
getTxId = -Int
11, getTxSize :: SizeInBytes
getTxSize = Word32 -> SizeInBytes
SizeInBytes Word32
13524, getTxValid :: Bool
getTxValid = Bool
True},Tx {getTxId :: Int
getTxId = Int
98, getTxSize :: SizeInBytes
getTxSize = Word32 -> SizeInBytes
SizeInBytes Word32
62024, getTxValid :: Bool
getTxValid = Bool
True},Tx {getTxId :: Int
getTxId = -Int
42, getTxSize :: SizeInBytes
getTxSize = Word32 -> SizeInBytes
SizeInBytes Word32
63227, getTxValid :: Bool
getTxValid = Bool
False},Tx {getTxId :: Int
getTxId = Int
74, getTxSize :: SizeInBytes
getTxSize = Word32 -> SizeInBytes
SizeInBytes Word32
31476, getTxValid :: Bool
getTxValid = Bool
True},Tx {getTxId :: Int
getTxId = Int
72, getTxSize :: SizeInBytes
getTxSize = Word32 -> SizeInBytes
SizeInBytes Word32
42959, getTxValid :: Bool
getTxValid = Bool
True},Tx {getTxId :: Int
getTxId = Int
72, getTxSize :: SizeInBytes
getTxSize = Word32 -> SizeInBytes
SizeInBytes Word32
53084, getTxValid :: Bool
getTxValid = Bool
True},Tx {getTxId :: Int
getTxId = Int
6, getTxSize :: SizeInBytes
getTxSize = Word32 -> SizeInBytes
SizeInBytes Word32
5013, getTxValid :: Bool
getTxValid = Bool
True},Tx {getTxId :: Int
getTxId = -Int
62, getTxSize :: SizeInBytes
getTxSize = Word32 -> SizeInBytes
SizeInBytes Word32
52590, getTxValid :: Bool
getTxValid = Bool
True},Tx {getTxId :: Int
getTxId = -Int
18, getTxSize :: SizeInBytes
getTxSize = Word32 -> SizeInBytes
SizeInBytes Word32
59325, getTxValid :: Bool
getTxValid = Bool
False},Tx {getTxId :: Int
getTxId = Int
70, getTxSize :: SizeInBytes
getTxSize = Word32 -> SizeInBytes
SizeInBytes Word32
40956, getTxValid :: Bool
getTxValid = Bool
True},Tx {getTxId :: Int
getTxId = -Int
82, getTxSize :: SizeInBytes
getTxSize = Word32 -> SizeInBytes
SizeInBytes Word32
33213, getTxValid :: Bool
getTxValid = Bool
True},Tx {getTxId :: Int
getTxId = -Int
73, getTxSize :: SizeInBytes
getTxSize = Word32 -> SizeInBytes
SizeInBytes Word32
31026, getTxValid :: Bool
getTxValid = Bool
True},Tx {getTxId :: Int
getTxId = -Int
4, getTxSize :: SizeInBytes
getTxSize = Word32 -> SizeInBytes
SizeInBytes Word32
19421, getTxValid :: Bool
getTxValid = Bool
True},Tx {getTxId :: Int
getTxId = Int
68, getTxSize :: SizeInBytes
getTxSize = Word32 -> SizeInBytes
SizeInBytes Word32
37501, getTxValid :: Bool
getTxValid = Bool
False},Tx {getTxId :: Int
getTxId = Int
47, getTxSize :: SizeInBytes
getTxSize = Word32 -> SizeInBytes
SizeInBytes Word32
25707, getTxValid :: Bool
getTxValid = Bool
False},Tx {getTxId :: Int
getTxId = -Int
99, getTxSize :: SizeInBytes
getTxSize = Word32 -> SizeInBytes
SizeInBytes Word32
58538, getTxValid :: Bool
getTxValid = Bool
False},Tx {getTxId :: Int
getTxId = Int
86, getTxSize :: SizeInBytes
getTxSize = Word32 -> SizeInBytes
SizeInBytes Word32
63432, getTxValid :: Bool
getTxValid = Bool
False},Tx {getTxId :: Int
getTxId = -Int
73, getTxSize :: SizeInBytes
getTxSize = Word32 -> SizeInBytes
SizeInBytes Word32
32185, getTxValid :: Bool
getTxValid = Bool
True},Tx {getTxId :: Int
getTxId = Int
52, getTxSize :: SizeInBytes
getTxSize = Word32 -> SizeInBytes
SizeInBytes Word32
55174, getTxValid :: Bool
getTxValid = Bool
False},Tx {getTxId :: Int
getTxId = Int
52, getTxSize :: SizeInBytes
getTxSize = Word32 -> SizeInBytes
SizeInBytes Word32
20715, getTxValid :: Bool
getTxValid = Bool
False},Tx {getTxId :: Int
getTxId = -Int
21, getTxSize :: SizeInBytes
getTxSize = Word32 -> SizeInBytes
SizeInBytes Word32
37063, getTxValid :: Bool
getTxValid = Bool
False},Tx {getTxId :: Int
getTxId = Int
15, getTxSize :: SizeInBytes
getTxSize = Word32 -> SizeInBytes
SizeInBytes Word32
63172, getTxValid :: Bool
getTxValid = Bool
True},Tx {getTxId :: Int
getTxId = -Int
26, getTxSize :: SizeInBytes
getTxSize = Word32 -> SizeInBytes
SizeInBytes Word32
51314, getTxValid :: Bool
getTxValid = Bool
True},Tx {getTxId :: Int
getTxId = Int
19, getTxSize :: SizeInBytes
getTxSize = Word32 -> SizeInBytes
SizeInBytes Word32
5042, getTxValid :: Bool
getTxValid = Bool
True},Tx {getTxId :: Int
getTxId = Int
36, getTxSize :: SizeInBytes
getTxSize = Word32 -> SizeInBytes
SizeInBytes Word32
40532, getTxValid :: Bool
getTxValid = Bool
True},Tx {getTxId :: Int
getTxId = -Int
30, getTxSize :: SizeInBytes
getTxSize = Word32 -> SizeInBytes
SizeInBytes Word32
18812, getTxValid :: Bool
getTxValid = Bool
True},Tx {getTxId :: Int
getTxId = Int
22, getTxSize :: SizeInBytes
getTxSize = Word32 -> SizeInBytes
SizeInBytes Word32
61634, getTxValid :: Bool
getTxValid = Bool
True},Tx {getTxId :: Int
getTxId = Int
89, getTxSize :: SizeInBytes
getTxSize = Word32 -> SizeInBytes
SizeInBytes Word32
44309, getTxValid :: Bool
getTxValid = Bool
True},Tx {getTxId :: Int
getTxId = -Int
98, getTxSize :: SizeInBytes
getTxSize = Word32 -> SizeInBytes
SizeInBytes Word32
61700, getTxValid :: Bool
getTxValid = Bool
True},Tx {getTxId :: Int
getTxId = -Int
17, getTxSize :: SizeInBytes
getTxSize = Word32 -> SizeInBytes
SizeInBytes Word32
46606, getTxValid :: Bool
getTxValid = Bool
True},Tx {getTxId :: Int
getTxId = -Int
37, getTxSize :: SizeInBytes
getTxSize = Word32 -> SizeInBytes
SizeInBytes Word32
25004, getTxValid :: Bool
getTxValid = Bool
False},Tx {getTxId :: Int
getTxId = -Int
53, getTxSize :: SizeInBytes
getTxSize = Word32 -> SizeInBytes
SizeInBytes Word32
51991, getTxValid :: Bool
getTxValid = Bool
False},Tx {getTxId :: Int
getTxId = -Int
88, getTxSize :: SizeInBytes
getTxSize = Word32 -> SizeInBytes
SizeInBytes Word32
17941, getTxValid :: Bool
getTxValid = Bool
True},Tx {getTxId :: Int
getTxId = Int
24, getTxSize :: SizeInBytes
getTxSize = Word32 -> SizeInBytes
SizeInBytes Word32
19866, getTxValid :: Bool
getTxValid = Bool
True},Tx {getTxId :: Int
getTxId = -Int
99, getTxSize :: SizeInBytes
getTxSize = Word32 -> SizeInBytes
SizeInBytes Word32
52082, getTxValid :: Bool
getTxValid = Bool
True},Tx {getTxId :: Int
getTxId = Int
50, getTxSize :: SizeInBytes
getTxSize = Word32 -> SizeInBytes
SizeInBytes Word32
48715, getTxValid :: Bool
getTxValid = Bool
True},Tx {getTxId :: Int
getTxId = -Int
8, getTxSize :: SizeInBytes
getTxSize = Word32 -> SizeInBytes
SizeInBytes Word32
24522, getTxValid :: Bool
getTxValid = Bool
True},Tx {getTxId :: Int
getTxId = Int
92, getTxSize :: SizeInBytes
getTxSize = Word32 -> SizeInBytes
SizeInBytes Word32
53516, getTxValid :: Bool
getTxValid = Bool
True},Tx {getTxId :: Int
getTxId = Int
59, getTxSize :: SizeInBytes
getTxSize = Word32 -> SizeInBytes
SizeInBytes Word32
16151, getTxValid :: Bool
getTxValid = Bool
False},Tx {getTxId :: Int
getTxId = -Int
85, getTxSize :: SizeInBytes
getTxSize = Word32 -> SizeInBytes
SizeInBytes Word32
57386, getTxValid :: Bool
getTxValid = Bool
True},Tx {getTxId :: Int
getTxId = Int
23, getTxSize :: SizeInBytes
getTxSize = Word32 -> SizeInBytes
SizeInBytes Word32
36444, getTxValid :: Bool
getTxValid = Bool
False},Tx {getTxId :: Int
getTxId = -Int
59, getTxSize :: SizeInBytes
getTxSize = Word32 -> SizeInBytes
SizeInBytes Word32
63727, getTxValid :: Bool
getTxValid = Bool
False},Tx {getTxId :: Int
getTxId = -Int
59, getTxSize :: SizeInBytes
getTxSize = Word32 -> SizeInBytes
SizeInBytes Word32
12656, getTxValid :: Bool
getTxValid = Bool
True},Tx {getTxId :: Int
getTxId = Int
13, getTxSize :: SizeInBytes
getTxSize = Word32 -> SizeInBytes
SizeInBytes Word32
19160, getTxValid :: Bool
getTxValid = Bool
False},Tx {getTxId :: Int
getTxId = -Int
35, getTxSize :: SizeInBytes
getTxSize = Word32 -> SizeInBytes
SizeInBytes Word32
1681, getTxValid :: Bool
getTxValid = Bool
True},Tx {getTxId :: Int
getTxId = -Int
13, getTxSize :: SizeInBytes
getTxSize = Word32 -> SizeInBytes
SizeInBytes Word32
46705, getTxValid :: Bool
getTxValid = Bool
False}]}
(Positive SmallDelay -> Maybe (Positive SmallDelay)
forall a. a -> Maybe a
Just (Positive {getPositive :: SmallDelay
getPositive = SmallDelay {getSmallDelay :: DiffTime
getSmallDelay = DiffTime
4.3}}))
data SimResult a = SimReturn a [String]
| SimException SomeException [String]
| SimDeadLock [String]
evaluateTrace :: SimTrace a -> IO (SimResult a)
evaluateTrace :: forall a. SimTrace a -> IO (SimResult a)
evaluateTrace = Context -> SimTrace a -> IO (SimResult a)
forall {m :: * -> *} {a}.
(MonadCatch m, MonadEvaluate m) =>
Context -> SimTrace a -> m (SimResult a)
go []
where
go :: Context -> SimTrace a -> m (SimResult a)
go Context
as SimTrace a
tr = do
r <- m (SimTrace a) -> m (Either SomeException (SimTrace a))
forall e a. Exception e => m a -> m (Either e a)
forall (m :: * -> *) e a.
(MonadCatch m, Exception e) =>
m a -> m (Either e a)
try (SimTrace a -> m (SimTrace a)
forall a. a -> m a
forall (m :: * -> *) a. MonadEvaluate m => a -> m a
evaluate SimTrace a
tr)
case r of
Right (SimTrace Time
_ IOSimThreadId
_ Maybe TestName
_ (EventSay TestName
s) SimTrace a
tr') -> Context -> SimTrace a -> m (SimResult a)
go (TestName
s TestName -> Context -> Context
forall a. a -> [a] -> [a]
: Context
as) SimTrace a
tr'
Right (SimTrace Time
_ IOSimThreadId
_ Maybe TestName
_ SimEventType
_ SimTrace a
tr' ) -> Context -> SimTrace a -> m (SimResult a)
go Context
as SimTrace a
tr'
Right (SimPORTrace Time
_ IOSimThreadId
_ Int
_ Maybe TestName
_ (EventSay TestName
s) SimTrace a
tr') -> Context -> SimTrace a -> m (SimResult a)
go (TestName
s TestName -> Context -> Context
forall a. a -> [a] -> [a]
: Context
as) SimTrace a
tr'
Right (SimPORTrace Time
_ IOSimThreadId
_ Int
_ Maybe TestName
_ SimEventType
_ SimTrace a
tr' ) -> Context -> SimTrace a -> m (SimResult a)
go Context
as SimTrace a
tr'
Right (TraceMainReturn Time
_ Labelled IOSimThreadId
_ a
a [Labelled IOSimThreadId]
_) -> SimResult a -> m (SimResult a)
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (SimResult a -> m (SimResult a)) -> SimResult a -> m (SimResult a)
forall a b. (a -> b) -> a -> b
$ a -> Context -> SimResult a
forall a. a -> Context -> SimResult a
SimReturn a
a (Context -> Context
forall a. [a] -> [a]
reverse Context
as)
Right (TraceMainException Time
_ Labelled IOSimThreadId
_ SomeException
e [Labelled IOSimThreadId]
_) -> SimResult a -> m (SimResult a)
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (SimResult a -> m (SimResult a)) -> SimResult a -> m (SimResult a)
forall a b. (a -> b) -> a -> b
$ SomeException -> Context -> SimResult a
forall a. SomeException -> Context -> SimResult a
SimException SomeException
e (Context -> Context
forall a. [a] -> [a]
reverse Context
as)
Right (TraceDeadlock Time
_ [Labelled IOSimThreadId]
_) -> SimResult a -> m (SimResult a)
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (SimResult a -> m (SimResult a)) -> SimResult a -> m (SimResult a)
forall a b. (a -> b) -> a -> b
$ Context -> SimResult a
forall a. Context -> SimResult a
SimDeadLock (Context -> Context
forall a. [a] -> [a]
reverse Context
as)
Right SimTrace a
TraceLoop -> TestName -> m (SimResult a)
forall a. HasCallStack => TestName -> a
error TestName
"IOSimPOR step time limit exceeded"
Right (TraceInternalError TestName
e) -> TestName -> m (SimResult a)
forall a. HasCallStack => TestName -> a
error (TestName
"IOSim: " TestName -> ShowS
forall a. [a] -> [a] -> [a]
++ TestName
e)
Left (SomeException e
e) -> SimResult a -> m (SimResult a)
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (SimResult a -> m (SimResult a)) -> SimResult a -> m (SimResult a)
forall a b. (a -> b) -> a -> b
$ SomeException -> Context -> SimResult a
forall a. SomeException -> Context -> SimResult a
SimException (e -> SomeException
forall e. (Exception e, HasExceptionContext) => e -> SomeException
SomeException e
e) (Context -> Context
forall a. [a] -> [a]
reverse Context
as)
data WithThreadAndTime a = WithThreadAndTime {
forall a. WithThreadAndTime a -> Time
wtatOccuredAt :: !Time
, forall a. WithThreadAndTime a -> TestName
wtatWithinThread :: !String
, forall a. WithThreadAndTime a -> a
wtatEvent :: !a
}
instance (Show a) => Show (WithThreadAndTime a) where
show :: WithThreadAndTime a -> TestName
show WithThreadAndTime {Time
wtatOccuredAt :: forall a. WithThreadAndTime a -> Time
wtatOccuredAt :: Time
wtatOccuredAt, TestName
wtatWithinThread :: forall a. WithThreadAndTime a -> TestName
wtatWithinThread :: TestName
wtatWithinThread, a
wtatEvent :: forall a. WithThreadAndTime a -> a
wtatEvent :: a
wtatEvent} =
TestName -> TestName -> TestName -> ShowS
forall r. PrintfType r => TestName -> r
printf TestName
"%s: %s: %s" (Time -> TestName
forall a. Show a => a -> TestName
show Time
wtatOccuredAt) (ShowS
forall a. Show a => a -> TestName
show TestName
wtatWithinThread) (a -> TestName
forall a. Show a => a -> TestName
show a
wtatEvent)
verboseTracer :: forall a m.
( MonadAsync m
, MonadSay m
, MonadMonotonicTime m
, Show a
)
=> Tracer m a
verboseTracer :: forall a (m :: * -> *).
(MonadAsync m, MonadSay m, MonadMonotonicTime m, Show a) =>
Tracer m a
verboseTracer = Tracer m (WithThreadAndTime a) -> Tracer m a
forall a (m :: * -> *).
(MonadAsync m, MonadMonotonicTime m) =>
Tracer m (WithThreadAndTime a) -> Tracer m a
threadAndTimeTracer (Tracer m (WithThreadAndTime a) -> Tracer m a)
-> Tracer m (WithThreadAndTime a) -> Tracer m a
forall a b. (a -> b) -> a -> b
$ Tracer m TestName -> Tracer m (WithThreadAndTime a)
forall a (m :: * -> *). Show a => Tracer m TestName -> Tracer m a
showTracing (Tracer m TestName -> Tracer m (WithThreadAndTime a))
-> Tracer m TestName -> Tracer m (WithThreadAndTime a)
forall a b. (a -> b) -> a -> b
$ (TestName -> m ()) -> Tracer m TestName
forall (m :: * -> *) a. (a -> m ()) -> Tracer m a
Tracer TestName -> m ()
forall (m :: * -> *). MonadSay m => TestName -> m ()
say
threadAndTimeTracer :: forall a m.
( MonadAsync m
, MonadMonotonicTime m
)
=> Tracer m (WithThreadAndTime a) -> Tracer m a
threadAndTimeTracer :: forall a (m :: * -> *).
(MonadAsync m, MonadMonotonicTime m) =>
Tracer m (WithThreadAndTime a) -> Tracer m a
threadAndTimeTracer Tracer m (WithThreadAndTime a)
tr = (a -> m ()) -> Tracer m a
forall (m :: * -> *) a. (a -> m ()) -> Tracer m a
Tracer ((a -> m ()) -> Tracer m a) -> (a -> m ()) -> Tracer m a
forall a b. (a -> b) -> a -> b
$ \a
s -> do
!now <- m Time
forall (m :: * -> *). MonadMonotonicTime m => m Time
getMonotonicTime
!tid <- myThreadId
traceWith tr $ WithThreadAndTime now (show tid) s