Skip to content

Commit

Permalink
p2p-governor: test asynchronous demotiions
Browse files Browse the repository at this point in the history
This is only enabled in the re-enabled `prop_governor_nolivelock`.
Running the test throws an exception (failed assertion), this is a bug
fixed in the next commit.

If a peer during asynchronous demotion is demoted to 'ColdPeer', we
interpret this as an error and throw an exception.  This corresponds to
a real scenario.  This is important because otherwise the governor will
return and finish the transition without running its error handlers
which bring the governor to the right state.

The above corresponds to the property of 'PeerStateActions': on failures
the peer state changes to 'PeerCold'.
  • Loading branch information
coot committed Jan 28, 2021
1 parent f6e430c commit fef4bcb
Showing 1 changed file with 168 additions and 36 deletions.
204 changes: 168 additions & 36 deletions ouroboros-network/test/Ouroboros/Network/PeerSelection/Test.hs
@@ -1,5 +1,7 @@
{-# LANGUAGE BangPatterns #-}
{-# LANGUAGE DeriveTraversable #-}
{-# LANGUAGE DerivingVia #-}
{-# LANGUAGE DerivingStrategies #-}
{-# LANGUAGE FlexibleContexts #-}
{-# LANGUAGE GeneralisedNewtypeDeriving #-}
{-# LANGUAGE NamedFieldPuns #-}
Expand All @@ -14,6 +16,7 @@ module Ouroboros.Network.PeerSelection.Test (tests) where
import qualified Data.ByteString.Char8 as BS
import Data.Dynamic (fromDynamic)
import Data.Function (on)
import Data.Functor (($>))
import Data.Graph (Graph)
import qualified Data.Graph as Graph
import Data.List (groupBy, nub)
Expand All @@ -28,10 +31,11 @@ import qualified Data.Tree as Tree
import Data.Typeable (Typeable)
import Data.Void (Void)

import Control.Exception (throw)
import Control.Exception (Exception, throw)
import Control.Monad.Class.MonadAsync
import Control.Monad.Class.MonadSTM
import Control.Monad.Class.MonadTime
import Control.Monad.Class.MonadThrow
import qualified Control.Monad.Fail as Fail
import Control.Tracer (Tracer (..), contramap, traceWith)

Expand Down Expand Up @@ -67,7 +71,7 @@ tests =
, localOption (QuickCheckMaxSize 30) $
testProperty "shrink for GovernorMockEnvironment" prop_shrink_GovernorMockEnvironment
]
-- , testProperty "governor no livelock" _prop_governor_nolivelock
, testProperty "governor no livelock" prop_governor_nolivelock
, testProperty "governor gossip reachable in 1hr" prop_governor_gossip_1hr
, testProperty "governor connection status" prop_governor_connstatus
]
Expand Down Expand Up @@ -105,16 +109,19 @@ newtype PeerAddr = PeerAddr Int

data PeerConn m = PeerConn !PeerAddr !(TVar m PeerStatus)

instance Show (PeerConn m) where
show (PeerConn peeraddr _) = "PeerConn " ++ show peeraddr

-- | The peer graph is the graph of all the peers in the mock p2p network, in
-- traditional adjacency representation.
--
newtype PeerGraph = PeerGraph [(PeerAddr, [PeerAddr], PeerInfo)]
deriving (Eq, Show)

-- | For now the information associated with each node is just the gossip
-- script.
-- script and connection script.
--
type PeerInfo = GossipScript
type PeerInfo = GovernorScripts

-- | The gossip script is the script we interpret to provide answers to gossip
-- requests that the governor makes. After each gossip request to a peer we
Expand All @@ -135,6 +142,72 @@ type GossipScript = Script (Maybe ([PeerAddr], GossipTime))
data GossipTime = GossipTimeQuick | GossipTimeSlow | GossipTimeTimeout
deriving (Eq, Show)


data AsyncDemotion = ToWarm
| ToCold
| Noop
deriving (Eq, Show)

instance Arbitrary AsyncDemotion where
arbitrary = frequency [ (2, pure ToWarm)
, (2, pure ToCold)
, (6, pure Noop)
]
shrink ToWarm = [ToCold, Noop]
shrink ToCold = [Noop]
shrink Noop = []


-- | Connection script is the script which provides asynchronous demotions
-- either to cold or warm peer.
--
type ConnectionScript = TimedScript AsyncDemotion


data GovernorScripts = GovernorScripts {
gossipScript :: GossipScript,
connectionScript :: ConnectionScript
}
deriving (Eq, Show)

instance Arbitrary GovernorScripts where
arbitrary = GovernorScripts
<$> arbitrary
<*> (fixConnectionScript <$> arbitrary)
shrink GovernorScripts { gossipScript, connectionScript } =
[ GovernorScripts gossipScript' connectionScript
| gossipScript' <- shrink gossipScript
]
++
[ GovernorScripts gossipScript (fixConnectionScript connectionScript')
| connectionScript' <- shrink connectionScript
]

-- | We ensure that eventually the connection script will allow to connect to
-- a given peer. This simplifies test conditions.
--
fixConnectionScript :: ConnectionScript -> ConnectionScript
fixConnectionScript (Script script) =
case NonEmpty.last script of
(Noop, _) -> Script script
_ -> Script $ script <> ((Noop, NoDelay) :| [])

-- | 'GovernorMockEnvironment' which does not do any asynchronous demotions.
--
newtype GovernorMockEnvironmentWithoutAsyncDemotion =
GovernorMockEnvironmentWAD GovernorMockEnvironment
deriving Show

instance Arbitrary GovernorMockEnvironmentWithoutAsyncDemotion where
arbitrary = GovernorMockEnvironmentWAD . fixGraph <$> arbitrary
where
fixGraph g@GovernorMockEnvironment { peerGraph = PeerGraph peerGraph } =
g { peerGraph = PeerGraph (map fixNode peerGraph) }
fixNode (addr, addrs, peerInfo) =
(addr, addrs, peerInfo { connectionScript = Script ((Noop, ShortDelay) :| []) })
shrink (GovernorMockEnvironmentWAD env) = map GovernorMockEnvironmentWAD (shrink env)


-- | A pick script is used to interpret the 'policyPickKnownPeersForGossip' and
-- the 'policyPickColdPeersToForget'. It selects elements from the given
-- choices by their index (modulo the number of choices). This representation
Expand Down Expand Up @@ -165,7 +238,7 @@ validPeerGraph g@(PeerGraph adjacency) =
and [ edgesSet `Set.isSubsetOf` allpeersSet &&
gossipSet `Set.isSubsetOf` edgesSet
| let allpeersSet = allPeers g
, (_, outedges, Script script) <- adjacency
, (_, outedges, GovernorScripts { gossipScript = Script script }) <- adjacency
, let edgesSet = Set.fromList outedges
gossipSet = Set.fromList
[ x | Just (xs, _) <- NonEmpty.toList script
Expand Down Expand Up @@ -199,7 +272,8 @@ runGovernorInMockEnvironment mockEnv =
data TraceMockEnv = TraceEnvPeersStatus (Map PeerAddr PeerStatus)
deriving Show

mockPeerSelectionActions :: (MonadAsync m, MonadTimer m, Fail.MonadFail m)
mockPeerSelectionActions :: (MonadAsync m, MonadTimer m, Fail.MonadFail m,
MonadThrow (STM m))
=> Tracer m TraceMockEnv
-> GovernorMockEnvironment
-> m (PeerSelectionActions PeerAddr (PeerConn m) m)
Expand All @@ -208,20 +282,32 @@ mockPeerSelectionActions tracer
peerGraph = PeerGraph adjacency,
targets
} = do
gossipScripts <- Map.fromList <$>
sequence [ (,) addr <$> initScript gossip
| (addr, _, gossip) <- adjacency ]
scripts <- Map.fromList <$>
sequence [ (\a b -> (addr, (a, b)))
<$> initScript gossipScript
<*> initScript connectionScript
| (addr, _, GovernorScripts { gossipScript, connectionScript }) <- adjacency ]
targetsVar <- playTimedScript targets
peerConns <- newTVarIO Map.empty
return $ mockPeerSelectionActions'
tracer env
gossipScripts targetsVar peerConns
scripts targetsVar peerConns


data TransitionError
= ActivationError
| DeactivationError
deriving (Show, Typeable)

instance Exception TransitionError where


mockPeerSelectionActions' :: forall m.
(MonadSTM m, MonadTimer m, Fail.MonadFail m)
(MonadAsync m, MonadSTM m, MonadTimer m, Fail.MonadFail m,
MonadThrow (STM m))
=> Tracer m TraceMockEnv
-> GovernorMockEnvironment
-> Map PeerAddr (TVar m GossipScript)
-> Map PeerAddr (TVar m GossipScript, TVar m ConnectionScript)
-> TVar m PeerSelectionTargets
-> TVar m (Map PeerAddr (TVar m PeerStatus))
-> PeerSelectionActions PeerAddr (PeerConn m) m
Expand All @@ -230,7 +316,7 @@ mockPeerSelectionActions' tracer
localRootPeers,
publicRootPeers
}
gossipScripts
scripts
targetsVar
connsVar =
PeerSelectionActions {
Expand All @@ -248,8 +334,8 @@ mockPeerSelectionActions' tracer
}
where
requestPeerGossip addr = do
let Just script = Map.lookup addr gossipScripts
mgossip <- stepScript script
let Just (gossipScript, _) = Map.lookup addr scripts
mgossip <- stepScript gossipScript
case mgossip of
Nothing -> fail "no peers"
Just (peeraddrs, time) -> do
Expand All @@ -259,14 +345,47 @@ mockPeerSelectionActions' tracer
establishPeerConnection :: PeerAddr -> m (PeerConn m)
establishPeerConnection peeraddr = do
threadDelay 1
(conn, snapshot) <- atomically $ do
(conn@(PeerConn _ v), snapshot) <- atomically $ do
conn <- newTVar PeerWarm
conns <- readTVar connsVar
let !conns' = Map.insert peeraddr conn conns
writeTVar connsVar conns'
snapshot <- traverse readTVar conns'
return (PeerConn peeraddr conn, snapshot)
traceWith tracer (TraceEnvPeersStatus snapshot)
let Just (_, connectScript) = Map.lookup peeraddr scripts
_ <- async $
-- monitoring loop which does asynchronous demotions. It will terminate
-- as soon as either of the events:
--
-- * the script returns 'Noop'
-- * peer demoted to 'PeerCold'
--
let loop = do
(demotion, delay) <- stepScript connectScript
let interpretScriptDelay NoDelay = 1
interpretScriptDelay ShortDelay = 60
interpretScriptDelay LongDelay = 600
done <-
case demotion of
Noop -> return True
ToWarm -> do
threadDelay (interpretScriptDelay delay)
atomically $ do
s <- readTVar v
case s of
PeerHot -> writeTVar v PeerWarm
$> False
_ -> return (PeerCold == s)
ToCold -> do
threadDelay (interpretScriptDelay delay)
atomically $ writeTVar v PeerCold
$> True

if done
then return ()
else loop
in loop
return conn

activatePeerConnection :: PeerConn m -> m ()
Expand All @@ -278,7 +397,15 @@ mockPeerSelectionActions' tracer
PeerHot -> error "activatePeerConnection of hot peer"
PeerWarm -> writeTVar conn PeerHot
--TODO: check it's just a race condition and not just wrong:
PeerCold -> return ()
--
-- We throw 'ActivationError' for the following reason:
-- 'PeerCold' can be set by the monitoring loop started by
-- 'establishedPeerConnection' above. However if that happens we
-- want to signal the governor that the warm -> hot transition
-- errored. Otherwise 'jobPromoteWarmPeer' will try to update the
-- state as if the transition went fine which will violate
-- 'invariantPeerSelectionState'.
PeerCold -> throwIO ActivationError
conns <- readTVar connsVar
traverse readTVar conns
traceWith tracer (TraceEnvPeersStatus snapshot)
Expand All @@ -291,7 +418,9 @@ mockPeerSelectionActions' tracer
PeerHot -> writeTVar conn PeerWarm
--TODO: check it's just a race condition and not just wrong:
PeerWarm -> return ()
PeerCold -> return ()
-- See the note in 'activatePeerConnection' why we throw an exception
-- here.
PeerCold -> throwIO DeactivationError
conns <- readTVar connsVar
traverse readTVar conns
traceWith tracer (TraceEnvPeersStatus snapshot)
Expand Down Expand Up @@ -416,8 +545,8 @@ pickMapKeys m ns =
-- This uses static targets and root peers.
--
-- TODO: Reenable this testcase.
_prop_governor_nolivelock :: GovernorMockEnvironment -> Property
_prop_governor_nolivelock env =
prop_governor_nolivelock :: GovernorMockEnvironment -> Property
prop_governor_nolivelock env =
let trace = takeFirstNHours 24 .
selectGovernorEvents .
selectPeerSelectionTraceEvents $
Expand Down Expand Up @@ -499,13 +628,13 @@ makesAdequateProgress n adequate ts =
-- must find all the reachable ones, or if the target for the number of known
-- peers to find is too low then it should at least find the target number.
--
prop_governor_gossip_1hr :: GovernorMockEnvironment -> Property
prop_governor_gossip_1hr env@GovernorMockEnvironment{
prop_governor_gossip_1hr :: GovernorMockEnvironmentWithoutAsyncDemotion -> Property
prop_governor_gossip_1hr (GovernorMockEnvironmentWAD env@GovernorMockEnvironment{
peerGraph,
localRootPeers,
publicRootPeers,
targets
} =
}) =
let trace = selectPeerSelectionTraceEvents $
runGovernorInMockEnvironment env {
targets = singletonScript (targets', NoDelay)
Expand Down Expand Up @@ -562,8 +691,8 @@ prop_governor_gossip_1hr env@GovernorMockEnvironment{
-- | Check the governor's view of connection status does not lag behind reality
-- by too much.
--
prop_governor_connstatus :: GovernorMockEnvironment -> Bool
prop_governor_connstatus env =
prop_governor_connstatus :: GovernorMockEnvironmentWithoutAsyncDemotion -> Bool
prop_governor_connstatus (GovernorMockEnvironmentWAD env) =
let trace = takeFirstNHours 1
. selectPeerSelectionTraceEvents $
runGovernorInMockEnvironment env
Expand Down Expand Up @@ -674,8 +803,8 @@ firstGossipGraph :: PeerGraph
firstGossipGraph (PeerGraph adjacency) =
simpleGraphRep $
Graph.graphFromEdges
[ ((), node, gossipScriptEdges gossip)
| (node, _edges, gossip) <- adjacency ]
[ ((), node, gossipScriptEdges gossipScript)
| (node, _edges, GovernorScripts { gossipScript }) <- adjacency ]
where
gossipScriptEdges :: GossipScript -> [PeerAddr]
gossipScriptEdges (Script (script :| _)) =
Expand Down Expand Up @@ -788,7 +917,9 @@ instance Arbitrary PeerGraph where
let adjacency = Map.fromListWith (<>)
[ (from, Set.singleton (PeerAddr to))
| (from, to) <- edges ]
graph <- sequence [ do node <- arbitraryGossipScript outedges
graph <- sequence [ do gossipScript <- arbitraryGossipScript outedges
connectionScript <- fixConnectionScript <$> arbitrary
let node = GovernorScripts { gossipScript, connectionScript }
return (PeerAddr n, outedges, node)
| n <- [0..numNodes-1]
, let outedges = maybe [] Set.toList
Expand Down Expand Up @@ -826,11 +957,15 @@ arbitraryGossipScript peers =
prunePeerGraphEdges :: [(PeerAddr, [PeerAddr], PeerInfo)]
-> [(PeerAddr, [PeerAddr], PeerInfo)]
prunePeerGraphEdges graph =
[ (nodeaddr, edges', Script script')
[ (nodeaddr, edges', node)
| let nodes = Set.fromList [ nodeaddr | (nodeaddr, _, _) <- graph ]
, (nodeaddr, edges, Script script) <- graph
, (nodeaddr, edges, GovernorScripts { gossipScript = Script gossip, connectionScript }) <- graph
, let edges' = pruneEdgeList nodes edges
script' = pruneGossipScript (Set.fromList edges') script
gossip' = pruneGossipScript (Set.fromList edges') gossip
node = GovernorScripts {
gossipScript = Script gossip',
connectionScript
}
]
where
pruneEdgeList :: Set PeerAddr -> [PeerAddr] -> [PeerAddr]
Expand Down Expand Up @@ -899,17 +1034,14 @@ instance Arbitrary PeerSelectionTargets where

newtype Script a = Script (NonEmpty a)
deriving (Eq, Show, Functor, Foldable, Traversable)
deriving Arbitrary via NonEmpty a

singletonScript :: a -> Script a
singletonScript x = (Script (x :| []))

scriptHead :: Script a -> a
scriptHead (Script (x :| _)) = x

instance Arbitrary a => Arbitrary (Script a) where
arbitrary = Script <$> arbitrary
shrink (Script xs) = map Script (shrink xs)

arbitraryShortScriptOf :: Gen a -> Gen (Script a)
arbitraryShortScriptOf a =
sized $ \sz ->
Expand All @@ -932,7 +1064,7 @@ stepScriptSTM scriptVar = do
type TimedScript a = Script (a, ScriptDelay)

data ScriptDelay = NoDelay | ShortDelay | LongDelay
deriving (Show)
deriving (Eq, Show)

instance Arbitrary ScriptDelay where
arbitrary = frequency [ (1, pure NoDelay)
Expand Down

0 comments on commit fef4bcb

Please sign in to comment.