{-# LANGUAGE LambdaCase #-}
{-# LANGUAGE NamedFieldPuns #-}
{-# LANGUAGE ScopedTypeVariables #-}
module Test.Consensus.MiniProtocol.ChainSync.CSJ (tests) where
import qualified Control.Concurrent.Class.MonadSTM.Strict.TVar as TVar
import Control.Monad (void)
import Control.Monad.Class.MonadTimer (MonadTimer)
import Control.Monad.IOSim (runSim)
import Control.ResourceRegistry
import Control.Tracer (nullTracer)
import Data.Typeable
import Network.TypedProtocol.Channel
import Network.TypedProtocol.Driver.Simple
import Ouroboros.Consensus.Block
import Ouroboros.Consensus.BlockchainTime
import Ouroboros.Consensus.Config
import qualified Ouroboros.Consensus.HeaderStateHistory as HeaderStateHistory
import Ouroboros.Consensus.Ledger.Tables.Utils (forgetLedgerTables)
import Ouroboros.Consensus.MiniProtocol.ChainSync.Client
( CSJConfig (..)
, CSJEnabledConfig (..)
, ChainDbView (..)
, ChainSyncLoPBucketConfig (..)
, ChainSyncStateView (..)
, ConfigEnv (..)
, Consensus
, DynamicEnv (..)
, bracketChainSyncClient
, chainSyncClient
, newChainSyncClientHandleCollection
)
import qualified Ouroboros.Consensus.MiniProtocol.ChainSync.Client.HistoricityCheck as HistoricityCheck
import qualified Ouroboros.Consensus.MiniProtocol.ChainSync.Client.InFutureCheck as InFutureCheck
import qualified Ouroboros.Consensus.Node.GsmState as GSM
import Ouroboros.Consensus.Node.NetworkProtocolVersion
( NodeToNodeVersion
)
import Ouroboros.Consensus.NodeId
import Ouroboros.Consensus.Util.IOLike
import Ouroboros.Consensus.Util.STM
( Fingerprint (..)
, WithFingerprint (..)
)
import qualified Ouroboros.Network.AnchoredFragment as AF
import Ouroboros.Network.ControlMessage (ControlMessage (..))
import qualified Ouroboros.Network.Mock.Chain as MockChain
import Ouroboros.Network.Protocol.ChainSync.ClientPipelined
import Ouroboros.Network.Protocol.ChainSync.Codec (codecChainSyncId)
import Ouroboros.Network.Protocol.ChainSync.Examples
import Ouroboros.Network.Protocol.ChainSync.PipelineDecision
( pipelineDecisionLowHighMark
)
import Ouroboros.Network.Protocol.ChainSync.Server
import Test.QuickCheck
import Test.Tasty
import Test.Tasty.QuickCheck
import Test.Util.Orphans.Arbitrary ()
import Test.Util.Orphans.IOLike ()
import Test.Util.TestBlock
tests :: TestTree
tests :: TestTree
tests =
String -> [TestTree] -> TestTree
testGroup
String
"Narrow CSJ"
[ String -> (TestSetup -> Property) -> TestTree
forall a. Testable a => String -> a -> TestTree
testProperty String
"CaughtUp" TestSetup -> Property
prop_CaughtUpCsj
]
data TestSetup
=
TestSetup
deriving (ReadPrec [TestSetup]
ReadPrec TestSetup
Int -> ReadS TestSetup
ReadS [TestSetup]
(Int -> ReadS TestSetup)
-> ReadS [TestSetup]
-> ReadPrec TestSetup
-> ReadPrec [TestSetup]
-> Read TestSetup
forall a.
(Int -> ReadS a)
-> ReadS [a] -> ReadPrec a -> ReadPrec [a] -> Read a
$creadsPrec :: Int -> ReadS TestSetup
readsPrec :: Int -> ReadS TestSetup
$creadList :: ReadS [TestSetup]
readList :: ReadS [TestSetup]
$creadPrec :: ReadPrec TestSetup
readPrec :: ReadPrec TestSetup
$creadListPrec :: ReadPrec [TestSetup]
readListPrec :: ReadPrec [TestSetup]
Read, Int -> TestSetup -> ShowS
[TestSetup] -> ShowS
TestSetup -> String
(Int -> TestSetup -> ShowS)
-> (TestSetup -> String)
-> ([TestSetup] -> ShowS)
-> Show TestSetup
forall a.
(Int -> a -> ShowS) -> (a -> String) -> ([a] -> ShowS) -> Show a
$cshowsPrec :: Int -> TestSetup -> ShowS
showsPrec :: Int -> TestSetup -> ShowS
$cshow :: TestSetup -> String
show :: TestSetup -> String
$cshowList :: [TestSetup] -> ShowS
showList :: [TestSetup] -> ShowS
Show)
instance Arbitrary TestSetup where
arbitrary :: Gen TestSetup
arbitrary = TestSetup -> Gen TestSetup
forall a. a -> Gen a
forall (f :: * -> *) a. Applicative f => a -> f a
pure TestSetup
TestSetup
prop_CaughtUpCsj :: TestSetup -> Property
prop_CaughtUpCsj :: TestSetup -> Property
prop_CaughtUpCsj TestSetup
testSetup =
case (forall s.
IOSim
s (WhetherReceivedMsgNextRequest, WhetherReceivedMsgNextRequest))
-> Either
Failure
(WhetherReceivedMsgNextRequest, WhetherReceivedMsgNextRequest)
forall a. (forall s. IOSim s a) -> Either Failure a
runSim ((forall s.
IOSim
s (WhetherReceivedMsgNextRequest, WhetherReceivedMsgNextRequest))
-> Either
Failure
(WhetherReceivedMsgNextRequest, WhetherReceivedMsgNextRequest))
-> (forall s.
IOSim
s (WhetherReceivedMsgNextRequest, WhetherReceivedMsgNextRequest))
-> Either
Failure
(WhetherReceivedMsgNextRequest, WhetherReceivedMsgNextRequest)
forall a b. (a -> b) -> a -> b
$ TestSetup
-> IOSim
s (WhetherReceivedMsgNextRequest, WhetherReceivedMsgNextRequest)
forall (m :: * -> *).
(IOLike m, MonadTimer m) =>
TestSetup
-> m (WhetherReceivedMsgNextRequest, WhetherReceivedMsgNextRequest)
runTest TestSetup
testSetup of
Left Failure
exn ->
String -> Property -> Property
forall prop. Testable prop => String -> prop -> Property
counterexample (String
"`runTest' threw an exception: " String -> ShowS
forall a. Semigroup a => a -> a -> a
<> Failure -> String
forall a. Show a => a -> String
show Failure
exn) (Property -> Property) -> Property -> Property
forall a b. (a -> b) -> a -> b
$
Bool -> Property
forall prop. Testable prop => prop -> Property
property Bool
False
Right (WhetherReceivedMsgNextRequest, WhetherReceivedMsgNextRequest)
results ->
String -> Property -> Property
forall prop. Testable prop => String -> prop -> Property
counterexample String
"At least one peer did not receive MsgRequestNext" (Property -> Property) -> Property -> Property
forall a b. (a -> b) -> a -> b
$
(WhetherReceivedMsgNextRequest, WhetherReceivedMsgNextRequest)
results (WhetherReceivedMsgNextRequest, WhetherReceivedMsgNextRequest)
-> (WhetherReceivedMsgNextRequest, WhetherReceivedMsgNextRequest)
-> Property
forall a. (Eq a, Show a) => a -> a -> Property
=== (WhetherReceivedMsgNextRequest
HasReceived, WhetherReceivedMsgNextRequest
HasReceived)
data WhetherReceivedMsgNextRequest
= HasNotYetReceived
| HasReceived
deriving (WhetherReceivedMsgNextRequest
-> WhetherReceivedMsgNextRequest -> Bool
(WhetherReceivedMsgNextRequest
-> WhetherReceivedMsgNextRequest -> Bool)
-> (WhetherReceivedMsgNextRequest
-> WhetherReceivedMsgNextRequest -> Bool)
-> Eq WhetherReceivedMsgNextRequest
forall a. (a -> a -> Bool) -> (a -> a -> Bool) -> Eq a
$c== :: WhetherReceivedMsgNextRequest
-> WhetherReceivedMsgNextRequest -> Bool
== :: WhetherReceivedMsgNextRequest
-> WhetherReceivedMsgNextRequest -> Bool
$c/= :: WhetherReceivedMsgNextRequest
-> WhetherReceivedMsgNextRequest -> Bool
/= :: WhetherReceivedMsgNextRequest
-> WhetherReceivedMsgNextRequest -> Bool
Eq, Int -> WhetherReceivedMsgNextRequest -> ShowS
[WhetherReceivedMsgNextRequest] -> ShowS
WhetherReceivedMsgNextRequest -> String
(Int -> WhetherReceivedMsgNextRequest -> ShowS)
-> (WhetherReceivedMsgNextRequest -> String)
-> ([WhetherReceivedMsgNextRequest] -> ShowS)
-> Show WhetherReceivedMsgNextRequest
forall a.
(Int -> a -> ShowS) -> (a -> String) -> ([a] -> ShowS) -> Show a
$cshowsPrec :: Int -> WhetherReceivedMsgNextRequest -> ShowS
showsPrec :: Int -> WhetherReceivedMsgNextRequest -> ShowS
$cshow :: WhetherReceivedMsgNextRequest -> String
show :: WhetherReceivedMsgNextRequest -> String
$cshowList :: [WhetherReceivedMsgNextRequest] -> ShowS
showList :: [WhetherReceivedMsgNextRequest] -> ShowS
Show)
runTest ::
forall m.
( IOLike m
, MonadTimer m
) =>
TestSetup ->
m (WhetherReceivedMsgNextRequest, WhetherReceivedMsgNextRequest)
runTest :: forall (m :: * -> *).
(IOLike m, MonadTimer m) =>
TestSetup
-> m (WhetherReceivedMsgNextRequest, WhetherReceivedMsgNextRequest)
runTest TestSetup
TestSetup = (ResourceRegistry m
-> m (WhetherReceivedMsgNextRequest,
WhetherReceivedMsgNextRequest))
-> m (WhetherReceivedMsgNextRequest, WhetherReceivedMsgNextRequest)
forall (m :: * -> *) a.
(MonadSTM m, MonadMask m, MonadThread m, HasCallStack) =>
(ResourceRegistry m -> m a) -> m a
withRegistry ((ResourceRegistry m
-> m (WhetherReceivedMsgNextRequest,
WhetherReceivedMsgNextRequest))
-> m (WhetherReceivedMsgNextRequest,
WhetherReceivedMsgNextRequest))
-> (ResourceRegistry m
-> m (WhetherReceivedMsgNextRequest,
WhetherReceivedMsgNextRequest))
-> m (WhetherReceivedMsgNextRequest, WhetherReceivedMsgNextRequest)
forall a b. (a -> b) -> a -> b
$ \ResourceRegistry m
registry -> do
varHandles <- STM m (ChainSyncClientHandleCollection CoreNodeId m TestBlock)
-> m (ChainSyncClientHandleCollection CoreNodeId m TestBlock)
forall a. HasCallStack => STM m a -> m a
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
STM m a -> m a
atomically STM m (ChainSyncClientHandleCollection CoreNodeId m TestBlock)
forall peer (m :: * -> *) blk.
(Ord peer, IOLike m, LedgerSupportsProtocol blk, NoThunks peer) =>
STM m (ChainSyncClientHandleCollection peer m blk)
newChainSyncClientHandleCollection
let chainDbView :: ChainDbView m TestBlock
chainDbView =
ChainDbView
{ getCurrentChain :: STM m (AnchoredFragment (Header TestBlock))
getCurrentChain = AnchoredFragment (Header TestBlock)
-> STM m (AnchoredFragment (Header TestBlock))
forall a. a -> STM m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (AnchoredFragment (Header TestBlock)
-> STM m (AnchoredFragment (Header TestBlock)))
-> AnchoredFragment (Header TestBlock)
-> STM m (AnchoredFragment (Header TestBlock))
forall a b. (a -> b) -> a -> b
$ Anchor (Header TestBlock) -> AnchoredFragment (Header TestBlock)
forall v a b. Anchorable v a b => a -> AnchoredSeq v a b
AF.Empty Anchor (Header TestBlock)
forall block. Anchor block
AF.AnchorGenesis
, getHeaderStateHistory :: STM m (HeaderStateHistory TestBlock)
getHeaderStateHistory =
HeaderStateHistory TestBlock
-> STM m (HeaderStateHistory TestBlock)
forall a. a -> STM m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (HeaderStateHistory TestBlock
-> STM m (HeaderStateHistory TestBlock))
-> HeaderStateHistory TestBlock
-> STM m (HeaderStateHistory TestBlock)
forall a b. (a -> b) -> a -> b
$
TopLevelConfig TestBlock
-> ExtLedgerState TestBlock ValuesMK
-> Chain TestBlock
-> HeaderStateHistory TestBlock
forall blk.
(ApplyBlock (ExtLedgerState blk) blk, HasHardForkHistory blk,
HasAnnTip blk) =>
TopLevelConfig blk
-> ExtLedgerState blk ValuesMK
-> Chain blk
-> HeaderStateHistory blk
HeaderStateHistory.fromChain
TopLevelConfig TestBlock
topLevelCfg
ExtLedgerState TestBlock ValuesMK
testInitExtLedger
Chain TestBlock
forall block. Chain block
MockChain.Genesis
, getPastLedger :: Point TestBlock -> STM m (Maybe (ExtLedgerState TestBlock EmptyMK))
getPastLedger =
Maybe (ExtLedgerState TestBlock EmptyMK)
-> STM m (Maybe (ExtLedgerState TestBlock EmptyMK))
forall a. a -> STM m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (Maybe (ExtLedgerState TestBlock EmptyMK)
-> STM m (Maybe (ExtLedgerState TestBlock EmptyMK)))
-> (Point TestBlock -> Maybe (ExtLedgerState TestBlock EmptyMK))
-> Point TestBlock
-> STM m (Maybe (ExtLedgerState TestBlock EmptyMK))
forall b c a. (b -> c) -> (a -> b) -> a -> c
. \case
Point TestBlock
GenesisPoint -> ExtLedgerState TestBlock EmptyMK
-> Maybe (ExtLedgerState TestBlock EmptyMK)
forall a. a -> Maybe a
Just (ExtLedgerState TestBlock EmptyMK
-> Maybe (ExtLedgerState TestBlock EmptyMK))
-> ExtLedgerState TestBlock EmptyMK
-> Maybe (ExtLedgerState TestBlock EmptyMK)
forall a b. (a -> b) -> a -> b
$ ExtLedgerState TestBlock ValuesMK
-> ExtLedgerState TestBlock EmptyMK
forall (l :: LedgerStateKind) (mk :: MapKind).
HasLedgerTables l =>
l mk -> l EmptyMK
forgetLedgerTables ExtLedgerState TestBlock ValuesMK
testInitExtLedger
BlockPoint{} -> Maybe (ExtLedgerState TestBlock EmptyMK)
forall a. Maybe a
Nothing
, getIsInvalidBlock :: STM
m
(WithFingerprint
(HeaderHash TestBlock -> Maybe (ExtValidationError TestBlock)))
getIsInvalidBlock =
WithFingerprint
(HeaderHash TestBlock -> Maybe (ExtValidationError TestBlock))
-> STM
m
(WithFingerprint
(HeaderHash TestBlock -> Maybe (ExtValidationError TestBlock)))
forall a. a -> STM m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (WithFingerprint
(HeaderHash TestBlock -> Maybe (ExtValidationError TestBlock))
-> STM
m
(WithFingerprint
(HeaderHash TestBlock -> Maybe (ExtValidationError TestBlock))))
-> WithFingerprint
(HeaderHash TestBlock -> Maybe (ExtValidationError TestBlock))
-> STM
m
(WithFingerprint
(HeaderHash TestBlock -> Maybe (ExtValidationError TestBlock)))
forall a b. (a -> b) -> a -> b
$ (TestHash -> Maybe (ExtValidationError TestBlock))
-> Fingerprint
-> WithFingerprint
(TestHash -> Maybe (ExtValidationError TestBlock))
forall a. a -> Fingerprint -> WithFingerprint a
WithFingerprint (\TestHash
_hash -> Maybe (ExtValidationError TestBlock)
forall a. Maybe a
Nothing) (Word64 -> Fingerprint
Fingerprint Word64
0)
}
version :: NodeToNodeVersion
version = NodeToNodeVersion
forall a. Bounded a => a
maxBound
lopBucketConfig :: ChainSyncLoPBucketConfig
lopBucketConfig = ChainSyncLoPBucketConfig
ChainSyncLoPBucketDisabled
csjConfig :: CSJEnabledConfig
csjConfig = CSJEnabledConfig{csjcJumpSize :: SlotNo
csjcJumpSize = Word64 -> SlotNo
SlotNo Word64
10000}
diffusionPipelining :: DiffusionPipeliningSupport
diffusionPipelining = DiffusionPipeliningSupport
DiffusionPipeliningOn
headerInFutureCheck ::
InFutureCheck.SomeHeaderInFutureCheck m TestBlock
headerInFutureCheck =
HeaderInFutureCheck m TestBlock () ()
-> SomeHeaderInFutureCheck m TestBlock
forall (m :: * -> *) blk arrival judgment.
HeaderInFutureCheck m blk arrival judgment
-> SomeHeaderInFutureCheck m blk
InFutureCheck.SomeHeaderInFutureCheck (HeaderInFutureCheck m TestBlock () ()
-> SomeHeaderInFutureCheck m TestBlock)
-> HeaderInFutureCheck m TestBlock () ()
-> SomeHeaderInFutureCheck m TestBlock
forall a b. (a -> b) -> a -> b
$
InFutureCheck.HeaderInFutureCheck
{ proxyArrival :: Proxy ()
InFutureCheck.proxyArrival = Proxy ()
forall {k} (t :: k). Proxy t
Proxy :: Proxy ()
, recordHeaderArrival :: Header TestBlock -> m ()
InFutureCheck.recordHeaderArrival = \Header TestBlock
_ -> () -> m ()
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure ()
, judgeHeaderArrival :: LedgerConfig TestBlock
-> LedgerState TestBlock EmptyMK
-> ()
-> Except PastHorizonException ()
InFutureCheck.judgeHeaderArrival =
\LedgerConfig TestBlock
_lcfg LedgerState TestBlock EmptyMK
_lstate () -> () -> Except PastHorizonException ()
forall a. a -> ExceptT PastHorizonException Identity a
forall (f :: * -> *) a. Applicative f => a -> f a
pure ()
, handleHeaderArrival :: () -> m (Except HeaderArrivalException RelativeTime)
InFutureCheck.handleHeaderArrival =
\() -> Except HeaderArrivalException RelativeTime
-> m (Except HeaderArrivalException RelativeTime)
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (Except HeaderArrivalException RelativeTime
-> m (Except HeaderArrivalException RelativeTime))
-> Except HeaderArrivalException RelativeTime
-> m (Except HeaderArrivalException RelativeTime)
forall a b. (a -> b) -> a -> b
$ RelativeTime -> Except HeaderArrivalException RelativeTime
forall a. a -> ExceptT HeaderArrivalException Identity a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (RelativeTime -> Except HeaderArrivalException RelativeTime)
-> RelativeTime -> Except HeaderArrivalException RelativeTime
forall a b. (a -> b) -> a -> b
$ NominalDiffTime -> RelativeTime
RelativeTime NominalDiffTime
0
}
mkClient ::
ChainSyncStateView m TestBlock ->
Consensus ChainSyncClientPipelined TestBlock m
mkClient ChainSyncStateView m TestBlock
csv =
let ChainSyncStateView
{ AnchoredFragment (HeaderWithTime TestBlock) -> STM m ()
csvSetCandidate :: AnchoredFragment (HeaderWithTime TestBlock) -> STM m ()
csvSetCandidate :: forall (m :: * -> *) blk.
ChainSyncStateView m blk
-> AnchoredFragment (HeaderWithTime blk) -> STM m ()
csvSetCandidate
, WithOrigin SlotNo -> STM m ()
csvSetLatestSlot :: WithOrigin SlotNo -> STM m ()
csvSetLatestSlot :: forall (m :: * -> *) blk.
ChainSyncStateView m blk -> WithOrigin SlotNo -> STM m ()
csvSetLatestSlot
, Idling m
csvIdling :: Idling m
csvIdling :: forall (m :: * -> *) blk. ChainSyncStateView m blk -> Idling m
csvIdling
, LoPBucket m
csvLoPBucket :: LoPBucket m
csvLoPBucket :: forall (m :: * -> *) blk. ChainSyncStateView m blk -> LoPBucket m
csvLoPBucket
, Jumping m TestBlock
csvJumping :: Jumping m TestBlock
csvJumping :: forall (m :: * -> *) blk. ChainSyncStateView m blk -> Jumping m blk
csvJumping
} = ChainSyncStateView m TestBlock
csv
in ConfigEnv m TestBlock
-> DynamicEnv m TestBlock
-> Consensus ChainSyncClientPipelined TestBlock m
forall (m :: * -> *) blk.
(IOLike m, LedgerSupportsProtocol blk) =>
ConfigEnv m blk
-> DynamicEnv m blk -> Consensus ChainSyncClientPipelined blk m
chainSyncClient
ConfigEnv
{ ChainDbView m TestBlock
chainDbView :: ChainDbView m TestBlock
chainDbView :: ChainDbView m TestBlock
chainDbView
, cfg :: TopLevelConfig TestBlock
cfg = TopLevelConfig TestBlock
topLevelCfg
, tracer :: Tracer m (TraceChainSyncClientEvent TestBlock)
tracer = Tracer m (TraceChainSyncClientEvent TestBlock)
forall (m :: * -> *) a. Applicative m => Tracer m a
nullTracer
, someHeaderInFutureCheck :: SomeHeaderInFutureCheck m TestBlock
someHeaderInFutureCheck = SomeHeaderInFutureCheck m TestBlock
headerInFutureCheck
, historicityCheck :: HistoricityCheck m TestBlock
historicityCheck = HistoricityCheck m TestBlock
forall (m :: * -> *) blk. Applicative m => HistoricityCheck m blk
HistoricityCheck.noCheck
, mkPipelineDecision0 :: MkPipelineDecision
mkPipelineDecision0 = Word16 -> Word16 -> MkPipelineDecision
pipelineDecisionLowHighMark Word16
10 Word16
20
, getDiffusionPipeliningSupport :: DiffusionPipeliningSupport
getDiffusionPipeliningSupport = DiffusionPipeliningSupport
diffusionPipelining
}
DynamicEnv
{ NodeToNodeVersion
version :: NodeToNodeVersion
version :: NodeToNodeVersion
version
, controlMessageSTM :: ControlMessageSTM m
controlMessageSTM = ControlMessage -> ControlMessageSTM m
forall a. a -> STM m a
forall (m :: * -> *) a. Monad m => a -> m a
return ControlMessage
Continue
, headerMetricsTracer :: HeaderMetricsTracer m
headerMetricsTracer = HeaderMetricsTracer m
forall (m :: * -> *) a. Applicative m => Tracer m a
nullTracer
, setCandidate :: AnchoredFragment (HeaderWithTime TestBlock) -> STM m ()
setCandidate = AnchoredFragment (HeaderWithTime TestBlock) -> STM m ()
csvSetCandidate
, idling :: Idling m
idling = Idling m
csvIdling
, loPBucket :: LoPBucket m
loPBucket = LoPBucket m
csvLoPBucket
, setLatestSlot :: WithOrigin SlotNo -> STM m ()
setLatestSlot = WithOrigin SlotNo -> STM m ()
csvSetLatestSlot
, jumping :: Jumping m TestBlock
jumping = Jumping m TestBlock
csvJumping
}
bracketedClient ::
CoreNodeId ->
(Consensus ChainSyncClientPipelined TestBlock m -> m a) ->
m a
bracketedClient CoreNodeId
peer Consensus ChainSyncClientPipelined TestBlock m -> m a
k =
Tracer m (TraceChainSyncClientEvent TestBlock)
-> Tracer m (TraceEventCsj CoreNodeId TestBlock)
-> ChainDbView m TestBlock
-> ChainSyncClientHandleCollection CoreNodeId m TestBlock
-> STM m GsmState
-> CoreNodeId
-> NodeToNodeVersion
-> ChainSyncLoPBucketConfig
-> CSJConfig
-> DiffusionPipeliningSupport
-> (ChainSyncStateView m TestBlock -> m a)
-> m a
forall (m :: * -> *) peer blk a.
(IOLike m, Ord peer, LedgerSupportsProtocol blk, MonadTimer m) =>
Tracer m (TraceChainSyncClientEvent blk)
-> Tracer m (TraceEventCsj peer blk)
-> ChainDbView m blk
-> ChainSyncClientHandleCollection peer m blk
-> STM m GsmState
-> peer
-> NodeToNodeVersion
-> ChainSyncLoPBucketConfig
-> CSJConfig
-> DiffusionPipeliningSupport
-> (ChainSyncStateView m blk -> m a)
-> m a
bracketChainSyncClient
Tracer m (TraceChainSyncClientEvent TestBlock)
forall (m :: * -> *) a. Applicative m => Tracer m a
nullTracer
Tracer m (TraceEventCsj CoreNodeId TestBlock)
forall (m :: * -> *) a. Applicative m => Tracer m a
nullTracer
ChainDbView m TestBlock
chainDbView
ChainSyncClientHandleCollection CoreNodeId m TestBlock
varHandles
(GsmState -> STM m GsmState
forall a. a -> STM m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure GsmState
GSM.CaughtUp)
CoreNodeId
peer
NodeToNodeVersion
version
ChainSyncLoPBucketConfig
lopBucketConfig
(CSJEnabledConfig -> CSJConfig
CSJEnabled CSJEnabledConfig
csjConfig)
DiffusionPipeliningSupport
diffusionPipelining
(Consensus ChainSyncClientPipelined TestBlock m -> m a
k (Consensus ChainSyncClientPipelined TestBlock m -> m a)
-> (ChainSyncStateView m TestBlock
-> Consensus ChainSyncClientPipelined TestBlock m)
-> ChainSyncStateView m TestBlock
-> m a
forall b c a. (b -> c) -> (a -> b) -> a -> c
. ChainSyncStateView m TestBlock
-> Consensus ChainSyncClientPipelined TestBlock m
mkClient)
spawnConnection ::
CoreNodeId ->
m (TVar.StrictTVar m WhetherReceivedMsgNextRequest)
spawnConnection CoreNodeId
peer = do
var <- WhetherReceivedMsgNextRequest
-> m (StrictTVar m WhetherReceivedMsgNextRequest)
forall (m :: * -> *) a. MonadSTM m => a -> m (StrictTVar m a)
TVar.newTVarIO WhetherReceivedMsgNextRequest
HasNotYetReceived
(clientChannel, serverChannel) <- createConnectedChannels
void $ forkLinkedThread registry ("client " <> show peer) $ do
bracketedClient peer $ \Consensus ChainSyncClientPipelined TestBlock m
client -> do
Tracer
m
(TraceSendRecv
(ChainSync (Header TestBlock) (Point TestBlock) (Tip TestBlock)))
-> Codec
(ChainSync (Header TestBlock) (Point TestBlock) (Tip TestBlock))
CodecFailure
m
(AnyMessage
(ChainSync (Header TestBlock) (Point TestBlock) (Tip TestBlock)))
-> Channel
m
(AnyMessage
(ChainSync (Header TestBlock) (Point TestBlock) (Tip TestBlock)))
-> PeerPipelined
(ChainSync (Header TestBlock) (Point TestBlock) (Tip TestBlock))
'AsClient
'StIdle
m
ChainSyncClientResult
-> m (ChainSyncClientResult,
Maybe
(AnyMessage
(ChainSync (Header TestBlock) (Point TestBlock) (Tip TestBlock))))
forall ps (st :: ps) (pr :: PeerRole) failure bytes (m :: * -> *)
a.
(MonadAsync m, MonadThrow m, Exception failure) =>
Tracer m (TraceSendRecv ps)
-> Codec ps failure m bytes
-> Channel m bytes
-> PeerPipelined ps pr st m a
-> m (a, Maybe bytes)
runPipelinedPeer
Tracer
m
(TraceSendRecv
(ChainSync (Header TestBlock) (Point TestBlock) (Tip TestBlock)))
forall (m :: * -> *) a. Applicative m => Tracer m a
nullTracer
Codec
(ChainSync (Header TestBlock) (Point TestBlock) (Tip TestBlock))
CodecFailure
m
(AnyMessage
(ChainSync (Header TestBlock) (Point TestBlock) (Tip TestBlock)))
forall header point tip (m :: * -> *).
Monad m =>
Codec
(ChainSync header point tip)
CodecFailure
m
(AnyMessage (ChainSync header point tip))
codecChainSyncId
Channel
m
(AnyMessage
(ChainSync (Header TestBlock) (Point TestBlock) (Tip TestBlock)))
clientChannel
(Consensus ChainSyncClientPipelined TestBlock m
-> PeerPipelined
(ChainSync (Header TestBlock) (Point TestBlock) (Tip TestBlock))
'AsClient
'StIdle
m
ChainSyncClientResult
forall header point tip (m :: * -> *) a.
Monad m =>
ChainSyncClientPipelined header point tip m a
-> ClientPipelined (ChainSync header point tip) 'StIdle m a
chainSyncClientPeerPipelined Consensus ChainSyncClientPipelined TestBlock m
client)
void
$ forkLinkedThread registry ("server " <> show peer)
$ do
runPeer
nullTracer
codecChainSyncId
serverChannel
$ chainSyncServerPeer
$ server
$ atomically (TVar.writeTVar var HasReceived)
pure var
var1 <- spawnConnection $ CoreNodeId 1
var2 <- spawnConnection $ CoreNodeId 2
threadDelay testDuration
atomically $ (,) <$> TVar.readTVar var1 <*> TVar.readTVar var2
testDuration :: Num a => a
testDuration :: forall a. Num a => a
testDuration = a
100
server ::
IOLike m =>
m () ->
ChainSyncServer (Header TestBlock) (Point TestBlock) (Tip TestBlock) m ()
server :: forall (m :: * -> *).
IOLike m =>
m ()
-> ChainSyncServer
(Header TestBlock) (Point TestBlock) (Tip TestBlock) m ()
server m ()
onFirstMsgRequestNext =
ChainSyncServer
(Header TestBlock) (Point TestBlock) (Tip TestBlock) m ()
forall {header} {block} {a}.
ChainSyncServer header (Point block) (Tip TestBlock) m a
go
where
dummyTip :: Tip TestBlock
dummyTip = SlotNo -> HeaderHash TestBlock -> BlockNo -> Tip TestBlock
forall {k} (b :: k). SlotNo -> HeaderHash b -> BlockNo -> Tip b
Tip (Word64 -> SlotNo
SlotNo Word64
1000) ([Word64] -> TestHash
testHashFromList [Word64
0]) (Word64 -> BlockNo
BlockNo Word64
1000)
go :: ChainSyncServer header (Point block) (Tip TestBlock) m a
go =
m (ServerStIdle header (Point block) (Tip TestBlock) m a)
-> ChainSyncServer header (Point block) (Tip TestBlock) m a
forall header point tip (m :: * -> *) a.
m (ServerStIdle header point tip m a)
-> ChainSyncServer header point tip m a
ChainSyncServer (m (ServerStIdle header (Point block) (Tip TestBlock) m a)
-> ChainSyncServer header (Point block) (Tip TestBlock) m a)
-> m (ServerStIdle header (Point block) (Tip TestBlock) m a)
-> ChainSyncServer header (Point block) (Tip TestBlock) m a
forall a b. (a -> b) -> a -> b
$
ServerStIdle header (Point block) (Tip TestBlock) m a
-> m (ServerStIdle header (Point block) (Tip TestBlock) m a)
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (ServerStIdle header (Point block) (Tip TestBlock) m a
-> m (ServerStIdle header (Point block) (Tip TestBlock) m a))
-> ServerStIdle header (Point block) (Tip TestBlock) m a
-> m (ServerStIdle header (Point block) (Tip TestBlock) m a)
forall a b. (a -> b) -> a -> b
$
ServerStIdle
{ recvMsgRequestNext :: m (Either
(ServerStNext header (Point block) (Tip TestBlock) m a)
(m (ServerStNext header (Point block) (Tip TestBlock) m a)))
recvMsgRequestNext = do
m ()
onFirstMsgRequestNext
DiffTime -> m ()
forall (m :: * -> *). MonadDelay m => DiffTime -> m ()
threadDelay (DiffTime -> m ()) -> DiffTime -> m ()
forall a b. (a -> b) -> a -> b
$ DiffTime
forall a. Num a => a
testDuration DiffTime -> DiffTime -> DiffTime
forall a. Num a => a -> a -> a
+ DiffTime
1
TestException
-> m (Either
(ServerStNext header (Point block) (Tip TestBlock) m a)
(m (ServerStNext header (Point block) (Tip TestBlock) m a)))
forall e a. Exception e => e -> m a
forall (m :: * -> *) e a. (MonadThrow m, Exception e) => e -> m a
throwIO TestException
Exhausted
, recvMsgFindIntersect :: [Point block]
-> m (ServerStIntersect header (Point block) (Tip TestBlock) m a)
recvMsgFindIntersect = \[Point block]
_points ->
ServerStIntersect header (Point block) (Tip TestBlock) m a
-> m (ServerStIntersect header (Point block) (Tip TestBlock) m a)
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (ServerStIntersect header (Point block) (Tip TestBlock) m a
-> m (ServerStIntersect header (Point block) (Tip TestBlock) m a))
-> ServerStIntersect header (Point block) (Tip TestBlock) m a
-> m (ServerStIntersect header (Point block) (Tip TestBlock) m a)
forall a b. (a -> b) -> a -> b
$ Point block
-> Tip TestBlock
-> ChainSyncServer header (Point block) (Tip TestBlock) m a
-> ServerStIntersect header (Point block) (Tip TestBlock) m a
forall point tip header (m :: * -> *) a.
point
-> tip
-> ChainSyncServer header point tip m a
-> ServerStIntersect header point tip m a
SendMsgIntersectFound Point block
forall {k} (block :: k). Point block
GenesisPoint Tip TestBlock
dummyTip ChainSyncServer header (Point block) (Tip TestBlock) m a
go
, recvMsgDoneClient :: m a
recvMsgDoneClient = TestException -> m a
forall e a. Exception e => e -> m a
forall (m :: * -> *) e a. (MonadThrow m, Exception e) => e -> m a
throwIO TestException
UnexpectedTermination
}
data TestException
=
Exhausted
|
UnexpectedTermination
deriving (TestException -> TestException -> Bool
(TestException -> TestException -> Bool)
-> (TestException -> TestException -> Bool) -> Eq TestException
forall a. (a -> a -> Bool) -> (a -> a -> Bool) -> Eq a
$c== :: TestException -> TestException -> Bool
== :: TestException -> TestException -> Bool
$c/= :: TestException -> TestException -> Bool
/= :: TestException -> TestException -> Bool
Eq, Int -> TestException -> ShowS
[TestException] -> ShowS
TestException -> String
(Int -> TestException -> ShowS)
-> (TestException -> String)
-> ([TestException] -> ShowS)
-> Show TestException
forall a.
(Int -> a -> ShowS) -> (a -> String) -> ([a] -> ShowS) -> Show a
$cshowsPrec :: Int -> TestException -> ShowS
showsPrec :: Int -> TestException -> ShowS
$cshow :: TestException -> String
show :: TestException -> String
$cshowList :: [TestException] -> ShowS
showList :: [TestException] -> ShowS
Show)
instance Exception TestException
topLevelCfg :: TopLevelConfig TestBlock
topLevelCfg :: TopLevelConfig TestBlock
topLevelCfg = TopLevelConfig TestBlock
singleNodeTestConfig