diff --git a/ouroboros-consensus-cardano/src/tools/Cardano/Tools/DBAnalyser/Run.hs b/ouroboros-consensus-cardano/src/tools/Cardano/Tools/DBAnalyser/Run.hs index 5bb5e3d3eb..c3afab2123 100644 --- a/ouroboros-consensus-cardano/src/tools/Cardano/Tools/DBAnalyser/Run.hs +++ b/ouroboros-consensus-cardano/src/tools/Cardano/Tools/DBAnalyser/Run.hs @@ -10,6 +10,7 @@ import Cardano.Tools.DBAnalyser.HasAnalysis import Cardano.Tools.DBAnalyser.Types import Codec.CBOR.Decoding (Decoder) import Codec.Serialise (Serialise (decode)) +import Control.Concurrent.Class.MonadMVar.Strict.NoThunks import Control.Monad.Except (runExceptT) import Control.Tracer (Tracer (..), nullTracer) import qualified Debug.Trace as Debug @@ -52,7 +53,7 @@ analyse :: -> IO (Maybe AnalysisResult) analyse DBAnalyserConfig{analysis, confLimit, dbDir, selectDB, validation, verbose} args = withRegistry $ \registry -> do - lock <- newSVar () + lock <- newMVar () chainDBTracer <- mkTracer lock verbose analysisTracer <- mkTracer lock True ProtocolInfo { pInfoInitLedger = genesisLedger, pInfoConfig = cfg } <- @@ -128,7 +129,7 @@ analyse DBAnalyserConfig{analysis, confLimit, dbDir, selectDB, validation, verbo hPutStrLn stderr $ concat ["[", show diff, "] ", show ev] hFlush stderr where - withLock = bracket_ (takeSVar lock) (putSVar lock ()) + withLock = bracket_ (takeMVar lock) (putMVar lock ()) immValidationPolicy = case (analysis, validation) of (_, Just ValidateAllBlocks) -> ImmutableDB.ValidateAllChunks diff --git a/ouroboros-consensus-cardano/src/tools/Cardano/Tools/DBTruncater/Run.hs b/ouroboros-consensus-cardano/src/tools/Cardano/Tools/DBTruncater/Run.hs index 5b0068e198..500a8836a9 100644 --- a/ouroboros-consensus-cardano/src/tools/Cardano/Tools/DBTruncater/Run.hs +++ b/ouroboros-consensus-cardano/src/tools/Cardano/Tools/DBTruncater/Run.hs @@ -9,6 +9,7 @@ module Cardano.Tools.DBTruncater.Run (truncate) where import Cardano.Slotting.Slot (WithOrigin (..)) import Cardano.Tools.DBAnalyser.HasAnalysis import Cardano.Tools.DBTruncater.Types +import Control.Concurrent.Class.MonadMVar.Strict.NoThunks import Control.Monad import Control.Tracer import Data.Functor.Identity @@ -103,15 +104,15 @@ findNewTip target iter = IteratorResult item -> do if acceptable item then go (Just item) else pure acc -mkLock :: MonadSTM m => m (StrictSVar m ()) -mkLock = newSVar () +mkLock :: MonadMVar m => m (StrictMVar m ()) +mkLock = newMVar () -mkTracer :: Show a => StrictSVar IO () -> Bool -> IO (Tracer IO a) +mkTracer :: Show a => StrictMVar IO () -> Bool -> IO (Tracer IO a) mkTracer _ False = pure mempty mkTracer lock True = do startTime <- getMonotonicTime pure $ Tracer $ \ev -> do - bracket_ (takeSVar lock) (putSVar lock ()) $ do + bracket_ (takeMVar lock) (putMVar lock ()) $ do traceTime <- getMonotonicTime let diff = diffTime traceTime startTime hPutStrLn stderr $ concat ["[", show diff, "] ", show ev] diff --git a/ouroboros-consensus-protocol/src/ouroboros-consensus-protocol/Ouroboros/Consensus/Protocol/Ledger/HotKey.hs b/ouroboros-consensus-protocol/src/ouroboros-consensus-protocol/Ouroboros/Consensus/Protocol/Ledger/HotKey.hs index 9df8d6c8fe..a18e2c850e 100644 --- a/ouroboros-consensus-protocol/src/ouroboros-consensus-protocol/Ouroboros/Consensus/Protocol/Ledger/HotKey.hs +++ b/ouroboros-consensus-protocol/src/ouroboros-consensus-protocol/Ouroboros/Consensus/Protocol/Ledger/HotKey.hs @@ -28,6 +28,7 @@ import qualified Cardano.Crypto.KES as Relative (Period) import Cardano.Ledger.Crypto (Crypto) import qualified Cardano.Ledger.Keys as SL import qualified Cardano.Protocol.TPraos.OCert as Absolute (KESPeriod (..)) +import Control.Concurrent.Class.MonadMVar.Strict.NoThunks import Data.Word (Word64) import GHC.Generics (Generic) import GHC.Stack (HasCallStack) @@ -173,13 +174,13 @@ mkHotKey :: -> Word64 -- ^ Max KES evolutions -> m (HotKey c m) mkHotKey initKey startPeriod@(Absolute.KESPeriod start) maxKESEvolutions = do - varKESState <- newSVar initKESState + varKESState <- newMVar initKESState return HotKey { evolve = evolveKey varKESState - , getInfo = kesStateInfo <$> readSVar varKESState - , isPoisoned = kesKeyIsPoisoned . kesStateKey <$> readSVar varKESState + , getInfo = kesStateInfo <$> readMVar varKESState + , isPoisoned = kesKeyIsPoisoned . kesStateKey <$> readMVar varKESState , sign_ = \toSign -> do - KESState { kesStateInfo, kesStateKey } <- readSVar varKESState + KESState { kesStateInfo, kesStateKey } <- readMVar varKESState case kesStateKey of KESKeyPoisoned -> error "trying to sign with a poisoned key" KESKey key -> do @@ -217,8 +218,8 @@ mkHotKey initKey startPeriod@(Absolute.KESPeriod start) maxKESEvolutions = do -- When the key is poisoned, we always return 'UpdateFailed'. evolveKey :: forall m c. (Crypto c, IOLike m) - => StrictSVar m (KESState c) -> Absolute.KESPeriod -> m KESEvolutionInfo -evolveKey varKESState targetPeriod = modifySVar varKESState $ \kesState -> do + => StrictMVar m (KESState c) -> Absolute.KESPeriod -> m KESEvolutionInfo +evolveKey varKESState targetPeriod = modifyMVar varKESState $ \kesState -> do let info = kesStateInfo kesState -- We mask the evolution process because if we got interrupted after -- calling 'forgetSignKeyKES', which destructively updates the current diff --git a/ouroboros-consensus/ouroboros-consensus.cabal b/ouroboros-consensus/ouroboros-consensus.cabal index 600100dfab..3d6a21bb4d 100644 --- a/ouroboros-consensus/ouroboros-consensus.cabal +++ b/ouroboros-consensus/ouroboros-consensus.cabal @@ -539,6 +539,7 @@ test-suite consensus-test , random , serialise , si-timers + , strict-mvar , tasty , tasty-hunit , tasty-quickcheck diff --git a/ouroboros-consensus/src/consensus-testlib/Test/Util/LogicalClock.hs b/ouroboros-consensus/src/consensus-testlib/Test/Util/LogicalClock.hs index e964617248..2423df9642 100644 --- a/ouroboros-consensus/src/consensus-testlib/Test/Util/LogicalClock.hs +++ b/ouroboros-consensus/src/consensus-testlib/Test/Util/LogicalClock.hs @@ -24,6 +24,7 @@ module Test.Util.LogicalClock ( , tickTracer ) where +import Control.Concurrent.Class.MonadMVar.Strict.NoThunks import Control.Monad import Control.Tracer (Tracer, contramapM) import Data.Time (NominalDiffTime) @@ -152,7 +153,7 @@ newWithDelay :: (IOLike m, HasCallStack) -> m (LogicalClock m) newWithDelay registry (NumTicks numTicks) tickLen = do current <- newTVarIO 0 - done <- newEmptySVar () + done <- newEmptyMVar _thread <- forkThread registry "ticker" $ do -- Tick 0 is the first tick, so increment @numTicks - 1@ times replicateM_ (fromIntegral numTicks - 1) $ do @@ -163,11 +164,11 @@ newWithDelay registry (NumTicks numTicks) tickLen = do -- Give tests that need to do some final processing on the last -- tick a chance to do that before we indicate completion. threadDelay (nominalDelay tickLen) - putSVar done () + putMVar done () return LogicalClock { getCurrentTick = Tick <$> readTVar current - , waitUntilDone = readSVar done + , waitUntilDone = readMVar done , mockSystemTime = BTime.SystemTime { BTime.systemTimeCurrent = do tick <- atomically $ readTVar current diff --git a/ouroboros-consensus/src/consensus-testlib/Test/Util/Orphans/NoThunks.hs b/ouroboros-consensus/src/consensus-testlib/Test/Util/Orphans/NoThunks.hs index 49b78353ed..99e8be11e2 100644 --- a/ouroboros-consensus/src/consensus-testlib/Test/Util/Orphans/NoThunks.hs +++ b/ouroboros-consensus/src/consensus-testlib/Test/Util/Orphans/NoThunks.hs @@ -1,9 +1,13 @@ -{-# LANGUAGE FlexibleInstances #-} -{-# LANGUAGE NamedFieldPuns #-} +{-# LANGUAGE FlexibleInstances #-} +{-# LANGUAGE NamedFieldPuns #-} +-- TODO: remove ScopedTypeVariables +{-# LANGUAGE InstanceSigs #-} +{-# LANGUAGE ScopedTypeVariables #-} {-# OPTIONS_GHC -Wno-orphans #-} module Test.Util.Orphans.NoThunks () where +import Control.Concurrent.Class.MonadMVar.Strict.NoThunks import Control.Monad.IOSim import Control.Monad.ST.Lazy import Control.Monad.ST.Unsafe (unsafeSTToIO) @@ -16,3 +20,10 @@ instance NoThunks a => NoThunks (StrictSVar (IOSim s) a) where wNoThunks ctxt StrictSVar { tvar } = do a <- unsafeSTToIO $ lazyToStrictST $ inspectTVar (Proxy :: Proxy (IOSim s)) tvar noThunks ctxt a + +-- TODO: we need to be able to inspect the value inside the mvar a la MonadInspectSTM. +instance NoThunks a => NoThunks (StrictMVar (IOSim s) a) where + showTypeOf _ = "StrictMVar IOSim" + wNoThunks ctxt _v = do + a <- undefined :: IO a -- TODO + noThunks ctxt a diff --git a/ouroboros-consensus/src/mock-block/Ouroboros/Consensus/Mock/Node/Praos.hs b/ouroboros-consensus/src/mock-block/Ouroboros/Consensus/Mock/Node/Praos.hs index 4d51426f22..75d90fa62b 100644 --- a/ouroboros-consensus/src/mock-block/Ouroboros/Consensus/Mock/Node/Praos.hs +++ b/ouroboros-consensus/src/mock-block/Ouroboros/Consensus/Mock/Node/Praos.hs @@ -11,6 +11,7 @@ module Ouroboros.Consensus.Mock.Node.Praos ( import Cardano.Crypto.KES import Cardano.Crypto.VRF +import Control.Concurrent.Class.MonadMVar.Strict.NoThunks import Data.Bifunctor (second) import Data.Map.Strict (Map) import qualified Data.Map.Strict as Map @@ -96,16 +97,17 @@ praosBlockForging :: -> HotKey PraosMockCrypto -> m (BlockForging m MockPraosBlock) praosBlockForging cid initHotKey = do - varHotKey <- newSVar initHotKey + varHotKey <- newMVar initHotKey return $ BlockForging { forgeLabel = "praosBlockForging" , canBeLeader = cid - , updateForgeState = \_ sno _ -> updateSVar varHotKey $ - second forgeStateUpdateInfoFromUpdateInfo + , updateForgeState = \_ sno _ -> modifyMVar varHotKey $ + pure + . second forgeStateUpdateInfoFromUpdateInfo . evolveKey sno , checkCanForge = \_ _ _ _ _ -> return () , forgeBlock = \cfg bno sno tickedLedgerSt txs isLeader -> do - hotKey <- readSVar varHotKey + hotKey <- readMVar varHotKey return $ forgeSimple (forgePraosExt hotKey) diff --git a/ouroboros-consensus/src/mock-block/Ouroboros/Consensus/Mock/Protocol/Praos.hs b/ouroboros-consensus/src/mock-block/Ouroboros/Consensus/Mock/Protocol/Praos.hs index 60c6c984a5..df8ebedfd8 100644 --- a/ouroboros-consensus/src/mock-block/Ouroboros/Consensus/Mock/Protocol/Praos.hs +++ b/ouroboros-consensus/src/mock-block/Ouroboros/Consensus/Mock/Protocol/Praos.hs @@ -213,7 +213,7 @@ deriving instance PraosCrypto c => Show (HotKey c) newtype HotKeyEvolutionError = HotKeyEvolutionError Period deriving (Show) --- | To be used in conjunction with, e.g., 'updateSVar'. +-- | To be used in conjunction with, e.g., 'modifyMVar'. -- -- NOTE: when the key's period is after the target period, we shouldn't use -- it, but we currently do. In real TPraos we check this in diff --git a/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Storage/ChainDB/Impl.hs b/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Storage/ChainDB/Impl.hs index 0cf59045ed..becdf4e84f 100644 --- a/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Storage/ChainDB/Impl.hs +++ b/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Storage/ChainDB/Impl.hs @@ -35,6 +35,7 @@ module Ouroboros.Consensus.Storage.ChainDB.Impl ( , openDBInternal ) where +import Control.Concurrent.Class.MonadMVar.Strict.NoThunks import Control.Monad (when) import Control.Monad.Trans.Class (lift) import Control.Tracer @@ -170,7 +171,7 @@ openDBInternal args launchBgTasks = runWithTempRegistry $ do varFollowers <- newTVarIO Map.empty varNextIteratorKey <- newTVarIO (IteratorKey 0) varNextFollowerKey <- newTVarIO (FollowerKey 0) - varCopyLock <- newSVar () + varCopyLock <- newMVar () varKillBgThreads <- newTVarIO $ return () blocksToAdd <- newBlocksToAdd (Args.cdbBlocksToAddSize args) diff --git a/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Storage/ChainDB/Impl/Background.hs b/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Storage/ChainDB/Impl/Background.hs index 27d6801f47..fa2dc770e7 100644 --- a/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Storage/ChainDB/Impl/Background.hs +++ b/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Storage/ChainDB/Impl/Background.hs @@ -37,6 +37,7 @@ module Ouroboros.Consensus.Storage.ChainDB.Impl.Background ( , addBlockRunner ) where +import Control.Concurrent.Class.MonadMVar.Strict.NoThunks import Control.Exception (assert) import Control.Monad (forM_, forever, void) import Control.Tracer @@ -197,8 +198,8 @@ copyToImmutableDB CDB{..} = withCopyLock $ do withCopyLock :: forall a. HasCallStack => m a -> m a withCopyLock = bracket_ - (fmap mustBeUnlocked $ tryTakeSVar cdbCopyLock) - (putSVar cdbCopyLock ()) + (fmap mustBeUnlocked $ tryTakeMVar cdbCopyLock) + (putMVar cdbCopyLock ()) mustBeUnlocked :: forall b. HasCallStack => Maybe b -> b mustBeUnlocked = fromMaybe diff --git a/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Storage/ChainDB/Impl/Types.hs b/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Storage/ChainDB/Impl/Types.hs index 5f2c029f4e..57272ac594 100644 --- a/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Storage/ChainDB/Impl/Types.hs +++ b/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Storage/ChainDB/Impl/Types.hs @@ -60,6 +60,7 @@ module Ouroboros.Consensus.Storage.ChainDB.Impl.Types ( , TraceValidationEvent (..) ) where +import Control.Concurrent.Class.MonadMVar.Strict.NoThunks import Control.Tracer import Data.Map.Strict (Map) import Data.Maybe.Strict (StrictMaybe (..)) @@ -225,7 +226,7 @@ data ChainDbEnv m blk = CDB -- not when hashes are garbage-collected from the map. , cdbNextIteratorKey :: !(StrictTVar m IteratorKey) , cdbNextFollowerKey :: !(StrictTVar m FollowerKey) - , cdbCopyLock :: !(StrictSVar m ()) + , cdbCopyLock :: !(StrictMVar m ()) -- ^ Lock used to ensure that 'copyToImmutableDB' is not executed more than -- once concurrently. -- diff --git a/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Storage/ImmutableDB/Impl/Index/Cache.hs b/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Storage/ImmutableDB/Impl/Index/Cache.hs index 722a2d8014..af57d2892f 100644 --- a/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Storage/ImmutableDB/Impl/Index/Cache.hs +++ b/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Storage/ImmutableDB/Impl/Index/Cache.hs @@ -12,6 +12,9 @@ {-# LANGUAGE ScopedTypeVariables #-} {-# LANGUAGE TypeApplications #-} +-- TODO: remove +{-# OPTIONS_GHC -Wno-redundant-constraints #-} + module Ouroboros.Consensus.Storage.ImmutableDB.Impl.Index.Cache ( -- * Environment CacheConfig (..) @@ -35,6 +38,7 @@ module Ouroboros.Consensus.Storage.ImmutableDB.Impl.Index.Cache ( ) where import Cardano.Prelude (forceElemsToWHNF) +import Control.Concurrent.Class.MonadMVar.Strict.NoThunks import Control.Exception (assert) import Control.Monad (forM, forM_, forever, mplus, unless, void, when) import Control.Monad.Except (throwError) @@ -74,7 +78,6 @@ import Ouroboros.Consensus.Storage.ImmutableDB.Impl.Util import Ouroboros.Consensus.Util (takeUntil, whenJust) import Ouroboros.Consensus.Util.CallStack import Ouroboros.Consensus.Util.IOLike -import qualified Ouroboros.Consensus.Util.MonadSTM.StrictSVar as Strict import Ouroboros.Consensus.Util.ResourceRegistry import System.FS.API (HasFS (..), withFile) import System.FS.API.Types (AllowExisting (..), Handle, @@ -356,9 +359,9 @@ data CacheEnv m blk h = CacheEnv { hasFS :: HasFS m h , registry :: ResourceRegistry m , tracer :: Tracer m TraceCacheEvent - , cacheVar :: StrictSVar m (Cached blk) + , cacheVar :: StrictMVar m (Cached blk) , cacheConfig :: CacheConfig - , bgThreadVar :: StrictSVar m (Maybe (Thread m Void)) + , bgThreadVar :: StrictMVar m (Maybe (Thread m Void)) -- ^ Nothing if no thread running , chunkInfo :: ChunkInfo } @@ -387,9 +390,9 @@ newEnv hasFS registry tracer cacheConfig chunkInfo chunk = do currentChunkInfo <- loadCurrentChunkInfo hasFS chunkInfo chunk cacheVar <- newMVarWithInvariants $ emptyCached chunk currentChunkInfo - bgThreadVar <- newSVar Nothing + bgThreadVar <- newMVar Nothing let cacheEnv = CacheEnv {..} - mask_ $ modifySVar_ bgThreadVar $ \_mustBeNothing -> do + mask_ $ modifyMVar_ bgThreadVar $ \_mustBeNothing -> do !bgThread <- forkLinkedThread registry "ImmutableDB.expireUnusedChunks" $ expireUnusedChunks cacheEnv return $ Just bgThread @@ -400,7 +403,7 @@ newEnv hasFS registry tracer cacheConfig chunkInfo chunk = do -- When checking invariants, check both our invariants and for thunks. -- Note that this is only done when the corresponding flag is enabled. newMVarWithInvariants = - Strict.newSVarWithInvariant $ \cached -> + newMVarWithInvariant $ \cached -> checkInvariants pastChunksToCache cached `mplus` (show <$> unsafeNoThunks cached) @@ -420,7 +423,7 @@ expireUnusedChunks expireUnusedChunks CacheEnv { cacheVar, cacheConfig, tracer } = forever $ do now <- getMonotonicTime - mbTraceMsg <- updateSVar cacheVar $ garbageCollect now + mbTraceMsg <- modifyMVar cacheVar $ pure . garbageCollect now mapM_ (traceWith tracer) mbTraceMsg threadDelay expireUnusedAfter where @@ -578,25 +581,25 @@ getChunkInfo -> m (Either (CurrentChunkInfo blk) (PastChunkInfo blk)) getChunkInfo cacheEnv chunk = do lastUsed <- LastUsed <$> getMonotonicTime - -- Make sure we don't leave an empty SVar in case of an exception. - mbCacheHit <- bracketOnError (takeSVar cacheVar) (tryPutSVar cacheVar) $ + -- Make sure we don't leave an empty MVar in case of an exception. + mbCacheHit <- bracketOnError (takeMVar cacheVar) (tryPutMVar cacheVar) $ \cached@Cached { currentChunk, currentChunkInfo, nbPastChunks } -> if | chunk == currentChunk -> do -- Cache hit for the current chunk - putSVar cacheVar cached + putMVar cacheVar cached traceWith tracer $ TraceCurrentChunkHit chunk nbPastChunks return $ Just $ Left currentChunkInfo | Just (pastChunkInfo, cached') <- lookupPastChunkInfo chunk lastUsed cached -> do -- Cache hit for an chunk in the past - putSVar cacheVar cached' + putMVar cacheVar cached' traceWith tracer $ TracePastChunkHit chunk nbPastChunks return $ Just $ Right pastChunkInfo | otherwise -> do -- Cache miss for an chunk in the past. We don't want to hold on to - -- the 'cacheVar' SVar, blocking all other access to the cace, while + -- the 'cacheVar' MVar, blocking all other access to the cace, while -- we're reading things from disk, so put it back now and update the -- cache afterwards. - putSVar cacheVar cached + putMVar cacheVar cached traceWith tracer $ TracePastChunkMiss chunk nbPastChunks return Nothing case mbCacheHit of @@ -607,7 +610,8 @@ getChunkInfo cacheEnv chunk = do -- Loading the chunk might have taken some time, so obtain the time -- again. lastUsed' <- LastUsed <$> getMonotonicTime - mbEvicted <- updateSVar cacheVar $ + mbEvicted <- modifyMVar cacheVar $ + pure . evictIfNecessary pastChunksToCache . addPastChunkInfo chunk lastUsed' pastChunkInfo whenJust mbEvicted $ \evicted -> @@ -627,7 +631,7 @@ getChunkInfo cacheEnv chunk = do -- This operation is idempotent. close :: IOLike m => CacheEnv m blk h -> m () close CacheEnv { bgThreadVar } = - mask_ $ modifySVar_ bgThreadVar $ \mbBgThread -> do + mask_ $ modifyMVar_ bgThreadVar $ \mbBgThread -> do mapM_ cancelThread mbBgThread return Nothing @@ -643,8 +647,8 @@ restart -> m () restart cacheEnv chunk = do currentChunkInfo <- loadCurrentChunkInfo hasFS chunkInfo chunk - void $ swapSVar cacheVar $ emptyCached chunk currentChunkInfo - mask_ $ modifySVar_ bgThreadVar $ \mbBgThread -> + void $ swapMVar cacheVar $ emptyCached chunk currentChunkInfo + mask_ $ modifyMVar_ bgThreadVar $ \mbBgThread -> case mbBgThread of Just _ -> throwIO $ userError "background thread still running" Nothing -> do @@ -747,7 +751,8 @@ openPrimaryIndex cacheEnv chunk allowExisting = do newCurrentChunkInfo <- case allowExisting of MustBeNew -> return $ emptyCurrentChunkInfo chunk AllowExisting -> loadCurrentChunkInfo hasFS chunkInfo chunk - mbEvicted <- updateSVar cacheVar $ + mbEvicted <- modifyMVar cacheVar $ + pure . evictIfNecessary pastChunksToCache . openChunk chunk lastUsed newCurrentChunkInfo whenJust mbEvicted $ \evicted -> @@ -767,7 +772,7 @@ appendOffsets -> m () appendOffsets CacheEnv { hasFS, cacheVar } pHnd offsets = do Primary.appendOffsets hasFS pHnd offsets - updateSVar_ cacheVar addCurrentChunkOffsets + modifyMVar_ cacheVar $ pure . addCurrentChunkOffsets where -- Lenses would be nice here addCurrentChunkOffsets :: Cached blk -> Cached blk @@ -861,7 +866,7 @@ appendEntry -> m Word64 appendEntry CacheEnv { hasFS, cacheVar } chunk sHnd entry = do nbBytes <- Secondary.appendEntry hasFS sHnd (withoutBlockSize entry) - updateSVar_ cacheVar addCurrentChunkEntry + modifyMVar_ cacheVar $ pure . addCurrentChunkEntry return nbBytes where -- Lenses would be nice here diff --git a/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Util/EarlyExit.hs b/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Util/EarlyExit.hs index 43d04fe1ef..7dee437ee9 100644 --- a/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Util/EarlyExit.hs +++ b/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Util/EarlyExit.hs @@ -20,6 +20,7 @@ module Ouroboros.Consensus.Util.EarlyExit ( import Control.Applicative import Control.Concurrent.Class.MonadMVar +import Control.Concurrent.Class.MonadMVar.Strict.NoThunks (StrictMVar) import Control.Monad import Control.Monad.Class.MonadAsync import Control.Monad.Class.MonadEventlog @@ -283,6 +284,7 @@ instance MonadEventlog m => MonadEventlog (WithEarlyExit m) where instance ( IOLike m , forall a. NoThunks (StrictTVar (WithEarlyExit m) a) , forall a. NoThunks (StrictSVar (WithEarlyExit m) a) + , forall a. NoThunks (StrictMVar (WithEarlyExit m) a) -- The simulator does not currently support @MonadCatch (STM m)@, -- making this @IOLike@ instance applicable to @IO@ only. Once that -- missing @MonadCatch@ instance is added, @IOLike@ should require diff --git a/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Util/IOLike.hs b/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Util/IOLike.hs index 7b712853cc..2a0993cdb5 100644 --- a/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Util/IOLike.hs +++ b/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Util/IOLike.hs @@ -44,7 +44,7 @@ module Ouroboros.Consensus.Util.IOLike ( import Cardano.Crypto.KES (KESAlgorithm, SignKeyKES) import qualified Cardano.Crypto.KES as KES import Control.Applicative (Alternative) -import Control.Concurrent.Class.MonadMVar +import Control.Concurrent.Class.MonadMVar.Strict.NoThunks import Control.Monad.Class.MonadAsync import Control.Monad.Class.MonadEventlog import Control.Monad.Class.MonadFork @@ -78,6 +78,7 @@ class ( MonadAsync m , forall a. NoThunks (m a) , forall a. NoThunks a => NoThunks (StrictTVar m a) , forall a. NoThunks a => NoThunks (StrictSVar m a) + , forall a. NoThunks a => NoThunks (StrictMVar m a) ) => IOLike m where -- | Securely forget a KES signing key. -- diff --git a/ouroboros-consensus/test/consensus-test/Test/Consensus/BlockchainTime/Simple.hs b/ouroboros-consensus/test/consensus-test/Test/Consensus/BlockchainTime/Simple.hs index 81bf7257bf..254b10f6a3 100644 --- a/ouroboros-consensus/test/consensus-test/Test/Consensus/BlockchainTime/Simple.hs +++ b/ouroboros-consensus/test/consensus-test/Test/Consensus/BlockchainTime/Simple.hs @@ -37,7 +37,8 @@ module Test.Consensus.BlockchainTime.Simple (tests) where import Control.Applicative (Alternative (..)) -import Control.Concurrent.Class.MonadMVar (MonadMVar) +import Control.Concurrent.Class.MonadMVar.Strict.NoThunks (MonadMVar, + StrictMVar) import qualified Control.Monad.Class.MonadSTM.Internal as LazySTM import Control.Monad.Class.MonadTime import qualified Control.Monad.Class.MonadTimer as MonadTimer @@ -379,6 +380,9 @@ deriving via AllowThunk (StrictTVar (OverrideDelay s) a) deriving via AllowThunk (StrictSVar (OverrideDelay s) a) instance NoThunks (StrictSVar (OverrideDelay s) a) +deriving via AllowThunk (StrictMVar (OverrideDelay s) a) + instance NoThunks (StrictMVar (OverrideDelay s) a) + instance MonadTimer.MonadDelay (OverrideDelay (IOSim s)) where threadDelay d = OverrideDelay $ ReaderT $ \schedule -> do -- Do the original delay. This is important, because otherwise this diff --git a/ouroboros-consensus/test/consensus-test/Test/Consensus/ResourceRegistry.hs b/ouroboros-consensus/test/consensus-test/Test/Consensus/ResourceRegistry.hs index 6276951e2d..d72a15f7d5 100644 --- a/ouroboros-consensus/test/consensus-test/Test/Consensus/ResourceRegistry.hs +++ b/ouroboros-consensus/test/consensus-test/Test/Consensus/ResourceRegistry.hs @@ -30,6 +30,7 @@ -- module Test.Consensus.ResourceRegistry (tests) where +import Control.Concurrent.Class.MonadMVar.Strict import Control.Monad.Class.MonadTimer.SI import Control.Monad.Except import Data.Foldable (toList) @@ -285,14 +286,14 @@ data ThreadInstr m :: Type -> Type where -- | Raise an exception in the thread ThreadCrash :: ThreadInstr m () --- | Instruction along with an SVar for the result -data QueuedInstr m = forall a. QueuedInstr (ThreadInstr m a) (StrictSVar m a) +-- | Instruction along with an MVar for the result +data QueuedInstr m = forall a. QueuedInstr (ThreadInstr m a) (StrictMVar m a) runInThread :: IOLike m => TestThread m -> ThreadInstr m a -> m a runInThread TestThread{..} instr = do - result <- uncheckedNewEmptySVar (error "no result yet") + result <- newEmptyMVar atomically $ writeTQueue threadComms (QueuedInstr instr result) - takeSVar result + takeMVar result instance (IOLike m) => Show (TestThread m) where show TestThread{..} = "" @@ -312,7 +313,7 @@ newThread :: forall m. IOLike m -> m (TestThread m) newThread alive parentReg = \shouldLink -> do comms <- atomically $ newTQueue - spawned <- uncheckedNewEmptySVar (error "no thread spawned yet") + spawned <- newEmptyMVar thread <- forkThread parentReg "newThread" $ withRegistry $ \childReg -> @@ -330,15 +331,15 @@ newThread alive parentReg = \shouldLink -> do -- Make sure to register thread before starting it atomically $ modifyTVar alive (testThread:) - putSVar spawned testThread + putMVar spawned testThread return testThread where threadBody :: ResourceRegistry m - -> StrictSVar m (TestThread m) + -> StrictMVar m (TestThread m) -> TQueue m (QueuedInstr m) -> m () threadBody childReg spawned comms = do - us <- readSVar spawned + us <- readMVar spawned loop us `finally` (atomically $ modifyTVar alive (delete us)) where loop :: TestThread m -> m () @@ -347,12 +348,12 @@ newThread alive parentReg = \shouldLink -> do case instr of ThreadFork linked -> do child <- newThread alive childReg (const us <$> linked) - putSVar result child + putMVar result child loop us ThreadTerminate -> do - putSVar result () + putMVar result () ThreadCrash -> do - putSVar result () + putMVar result () error "crashing" runIO :: forall m. (IOLike m, MonadTimer m) diff --git a/ouroboros-consensus/test/storage-test/Test/Ouroboros/Storage/ChainDB/StateMachine.hs b/ouroboros-consensus/test/storage-test/Test/Ouroboros/Storage/ChainDB/StateMachine.hs index 99df483531..6457e3ca56 100644 --- a/ouroboros-consensus/test/storage-test/Test/Ouroboros/Storage/ChainDB/StateMachine.hs +++ b/ouroboros-consensus/test/storage-test/Test/Ouroboros/Storage/ChainDB/StateMachine.hs @@ -80,6 +80,7 @@ module Test.Ouroboros.Storage.ChainDB.StateMachine ( ) where import Codec.Serialise (Serialise) +import Control.Concurrent.Class.MonadMVar.Strict.NoThunks import Control.Monad (replicateM, void) import Control.Tracer import Data.Bifoldable @@ -352,7 +353,7 @@ data ChainDBState m blk = ChainDBState -- | Environment to run commands against the real ChainDB implementation. data ChainDBEnv m blk = ChainDBEnv { - varDB :: StrictSVar m (ChainDBState m blk) + varDB :: StrictMVar m (ChainDBState m blk) , registry :: ResourceRegistry m , varCurSlot :: StrictTVar m SlotNo , varNextId :: StrictTVar m Id @@ -376,7 +377,7 @@ reopen => ChainDBEnv m blk -> m () reopen ChainDBEnv { varDB, args } = do chainDBState <- open args - void $ swapSVar varDB chainDBState + void $ swapMVar varDB chainDBState close :: IOLike m => ChainDBState m blk -> m () close ChainDBState { chainDB, addBlockAsync } = do @@ -389,7 +390,7 @@ run :: forall m blk. -> Cmd blk (TestIterator m blk) (TestFollower m blk) -> m (Success blk (TestIterator m blk) (TestFollower m blk)) run env@ChainDBEnv { varDB, .. } cmd = - readSVar varDB >>= \st@ChainDBState { chainDB = ChainDB{..}, internal } -> case cmd of + readMVar varDB >>= \st@ChainDBState { chainDB = ChainDB{..}, internal } -> case cmd of AddBlock blk -> Point <$> (advanceAndAdd st (blockSlot blk) blk) AddFutureBlock blk s -> Point <$> (advanceAndAdd st s blk) GetCurrentChain -> Chain <$> atomically getCurrentChain @@ -433,7 +434,7 @@ run env@ChainDBEnv { varDB, .. } cmd = close st atomically $ writeTVar varVolatileDbFs Mock.empty reopen env - ChainDB { getTipPoint } <- chainDB <$> readSVar varDB + ChainDB { getTipPoint } <- chainDB <$> readMVar varDB atomically getTipPoint giveWithEq :: a -> m (WithEq a) @@ -1541,11 +1542,11 @@ runCmdsLockstep maxClockSkew (SmallChunkInfo chunkInfo) cmds = maxClockSkew varCurSlot (hist, model, res, trace) <- bracket - (open args >>= newSVar) + (open args >>= newMVar) -- Note: we might be closing a different ChainDB than the one we -- opened, as we can reopen it the ChainDB, swapping the ChainDB in -- the MVar. - (\varDB -> readSVar varDB >>= close) + (\varDB -> readMVar varDB >>= close) $ \varDB -> do let env = ChainDBEnv diff --git a/ouroboros-consensus/test/storage-test/Test/Ouroboros/Storage/ChainDB/Unit.hs b/ouroboros-consensus/test/storage-test/Test/Ouroboros/Storage/ChainDB/Unit.hs index 406dc0e8e9..1570c13fcb 100644 --- a/ouroboros-consensus/test/storage-test/Test/Ouroboros/Storage/ChainDB/Unit.hs +++ b/ouroboros-consensus/test/storage-test/Test/Ouroboros/Storage/ChainDB/Unit.hs @@ -14,6 +14,7 @@ module Test.Ouroboros.Storage.ChainDB.Unit (tests) where import Cardano.Slotting.Slot (WithOrigin (..)) +import Control.Concurrent.Class.MonadMVar.Strict.NoThunks import Control.Monad.Except import Control.Monad.Reader import Control.Monad.State @@ -401,7 +402,7 @@ withTestChainDbEnv topLevelConfig chunkInfo extLedgerState cont nodeDbs <- emptyNodeDBs (tracer, getTrace) <- recordingTracerTVar let args = chainDbArgs threadRegistry nodeDbs tracer - varDB <- open args >>= newSVar + varDB <- open args >>= newMVar let env = ChainDBEnv { varDB , registry = iteratorRegistry @@ -413,7 +414,7 @@ withTestChainDbEnv topLevelConfig chunkInfo extLedgerState cont pure (env, getTrace) closeChainDbEnv (env, _) = do - readSVar (varDB env) >>= close + readMVar (varDB env) >>= close closeRegistry (registry env) closeRegistry (cdbRegistry $ args env) @@ -437,20 +438,20 @@ instance IOLike m => SupportsUnitTest (SystemM blk m) where addBlock blk = do env <- ask SystemM $ lift $ lift $ do - api <- chainDB <$> readSVar (varDB env) + api <- chainDB <$> readMVar (varDB env) void $ API.addBlock api API.noPunishment blk pure blk persistBlks shouldGarbageCollect = do env <- ask SystemM $ lift $ lift $ do - internal <- internal <$> readSVar (varDB env) + internal <- internal <$> readMVar (varDB env) SM.persistBlks shouldGarbageCollect internal newFollower = do env <- ask SystemM $ lift $ lift $ do - api <- chainDB <$> readSVar (varDB env) + api <- chainDB <$> readMVar (varDB env) API.newFollower api (registry env) API.SelectedChain allComponents followerInstruction = SystemM . lift . lift . fmap Right @@ -462,7 +463,7 @@ instance IOLike m => SupportsUnitTest (SystemM blk m) where stream from to = do env <- ask SystemM $ lift $ lift $ fmap Right $ do - api <- chainDB <$> readSVar (varDB env) + api <- chainDB <$> readMVar (varDB env) API.stream api (registry env) allComponents from to iteratorNext iterator = SystemM $ lift $ lift (API.iteratorNext iterator) diff --git a/ouroboros-consensus/test/storage-test/Test/Ouroboros/Storage/ImmutableDB/StateMachine.hs b/ouroboros-consensus/test/storage-test/Test/Ouroboros/Storage/ImmutableDB/StateMachine.hs index 424abf2836..99a28ff7b9 100644 --- a/ouroboros-consensus/test/storage-test/Test/Ouroboros/Storage/ImmutableDB/StateMachine.hs +++ b/ouroboros-consensus/test/storage-test/Test/Ouroboros/Storage/ImmutableDB/StateMachine.hs @@ -46,6 +46,7 @@ module Test.Ouroboros.Storage.ImmutableDB.StateMachine ( , tests ) where +import Control.Concurrent.Class.MonadMVar.Strict.NoThunks import Control.Monad (forM_, void) import Data.Bifunctor (first) import Data.ByteString.Lazy (ByteString) @@ -216,7 +217,7 @@ open args = do reopen :: ImmutableDBEnv -> ValidationPolicy -> IO () reopen ImmutableDBEnv { varDB, args } valPol = do immutableDbState <- open args { immValidationPolicy = valPol } - void $ swapSVar varDB immutableDbState + void $ swapMVar varDB immutableDbState -- | Run the command against the given database. run :: @@ -233,7 +234,7 @@ run env@ImmutableDBEnv { , immHasFS = SomeHasFS hasFS } } cmd = - readSVar varDB >>= \ImmutableDBState { db, internal } -> case cmd of + readMVar varDB >>= \ImmutableDBState { db, internal } -> case cmd of GetTip -> ImmTip <$> atomically (getTip db) GetBlockComponent pt -> ErAllComponents <$> getBlockComponent db allComponents pt AppendBlock blk -> Unit <$> appendBlock db blk @@ -799,15 +800,15 @@ data ImmutableDBEnv = ImmutableDBEnv { -- During truncation we might need to delete a file that is still opened -- by an iterator. As this is not allowed by the MockFS implementation, we -- first close all open iterators in these cases. - , varDB :: StrictSVar IO ImmutableDBState + , varDB :: StrictMVar IO ImmutableDBState , args :: ImmutableDbArgs Identity IO TestBlock } getImmutableDB :: ImmutableDBEnv -> IO (ImmutableDB IO TestBlock) -getImmutableDB = fmap db . readSVar . varDB +getImmutableDB = fmap db . readMVar . varDB getInternal :: ImmutableDBEnv -> IO (ImmutableDB.Internal IO TestBlock) -getInternal = fmap internal . readSVar . varDB +getInternal = fmap internal . readMVar . varDB semantics :: ImmutableDBEnv @@ -1208,11 +1209,11 @@ test cacheConfig chunkInfo cmds = do } (hist, model, res, trace) <- bracket - (open args >>= newSVar) + (open args >>= newMVar) -- Note: we might be closing a different ImmutableDB than the one we -- opened, as we can reopen it the ImmutableDB, swapping the -- ImmutableDB in the MVar. - (\varDB -> readSVar varDB >>= closeDB . db) + (\varDB -> readMVar varDB >>= closeDB . db) $ \varDB -> do let env = ImmutableDBEnv { varErrors diff --git a/ouroboros-consensus/test/storage-test/Test/Ouroboros/Storage/VolatileDB/StateMachine.hs b/ouroboros-consensus/test/storage-test/Test/Ouroboros/Storage/VolatileDB/StateMachine.hs index 827328bce7..4cbbe02b62 100644 --- a/ouroboros-consensus/test/storage-test/Test/Ouroboros/Storage/VolatileDB/StateMachine.hs +++ b/ouroboros-consensus/test/storage-test/Test/Ouroboros/Storage/VolatileDB/StateMachine.hs @@ -32,6 +32,7 @@ module Test.Ouroboros.Storage.VolatileDB.StateMachine ( , tests ) where +import Control.Concurrent.Class.MonadMVar.Strict.NoThunks import Control.Monad (forM_, void) import Data.Bifunctor (first) import Data.ByteString.Lazy (ByteString) @@ -480,7 +481,7 @@ shrinkCmd _ cmd = case cmd of -- | Environment to run commands against the real VolatileDB implementation. data VolatileDBEnv = VolatileDBEnv { varErrors :: StrictTVar IO Errors - , varDB :: StrictSVar IO (VolatileDB IO Block) + , varDB :: StrictMVar IO (VolatileDB IO Block) , args :: VolatileDbArgs Identity IO Block } @@ -490,7 +491,7 @@ data VolatileDBEnv = VolatileDBEnv reopenDB :: VolatileDBEnv -> IO () reopenDB VolatileDBEnv { varDB, args } = do db <- openDB args runWithTempRegistry - void $ swapSVar varDB db + void $ swapMVar varDB db semanticsImpl :: VolatileDBEnv -> At CmdErr Concrete -> IO (At Resp Concrete) semanticsImpl env@VolatileDBEnv { varDB, varErrors } (At (CmdErr cmd mbErrors)) = @@ -501,7 +502,7 @@ semanticsImpl env@VolatileDBEnv { varDB, varErrors } (At (CmdErr cmd mbErrors)) tryVolatileDB (Proxy @Block) (runDB env cmd) -- As all operations on the VolatileDB are idempotent, close (not -- idempotent by default!), reopen it, and run the command again. - readSVar varDB >>= idemPotentCloseDB + readMVar varDB >>= idemPotentCloseDB reopenDB env tryVolatileDB (Proxy @Block) (runDB env cmd) @@ -517,7 +518,7 @@ idemPotentCloseDB db = isClosedDBError _ = Nothing runDB :: HasCallStack => VolatileDBEnv -> Cmd -> IO Success -runDB env@VolatileDBEnv { varDB, args = VolatileDbArgs { volHasFS = SomeHasFS hasFS } } cmd = readSVar varDB >>= \db -> case cmd of +runDB env@VolatileDBEnv { varDB, args = VolatileDbArgs { volHasFS = SomeHasFS hasFS } } cmd = readMVar varDB >>= \db -> case cmd of GetBlockComponent hash -> MbAllComponents <$> getBlockComponent db allComponents hash PutBlock blk -> Unit <$> putBlock db blk FilterByPredecessor hashes -> Successors . (<$> hashes) <$> atomically (filterByPredecessor db) @@ -526,7 +527,7 @@ runDB env@VolatileDBEnv { varDB, args = VolatileDbArgs { volHasFS = SomeHasFS ha GetMaxSlotNo -> MaxSlot <$> atomically (getMaxSlotNo db) Close -> Unit <$> closeDB db ReOpen -> do - readSVar varDB >>= idemPotentCloseDB + readMVar varDB >>= idemPotentCloseDB Unit <$> reopenDB env Corruption corrs -> withClosedDB $ @@ -538,7 +539,7 @@ runDB env@VolatileDBEnv { varDB, args = VolatileDbArgs { volHasFS = SomeHasFS ha where withClosedDB :: IO () -> IO Success withClosedDB action = do - readSVar varDB >>= closeDB + readMVar varDB >>= closeDB action reopenDB env return $ Unit () @@ -593,11 +594,11 @@ test cmds = do } (hist, res, trace) <- bracket - (openDB args runWithTempRegistry >>= newSVar) + (openDB args runWithTempRegistry >>= newMVar) -- Note: we might be closing a different VolatileDB than the one we -- opened, as we can reopen it the VolatileDB, swapping the VolatileDB -- in the MVar. - (\varDB -> readSVar varDB >>= closeDB) + (\varDB -> readMVar varDB >>= closeDB) $ \varDB -> do let env = VolatileDBEnv { varErrors, varDB, args } sm' = sm env dbm