From f4a1ca0ccd438bf5084a6fee188285ea35476756 Mon Sep 17 00:00:00 2001 From: Javier Sagredo Date: Tue, 14 Oct 2025 16:47:01 +0200 Subject: [PATCH 01/10] Fix resource registry handle in V2 Forker handles --- cabal.project | 8 +++ .../Consensus/Storage/LedgerDB/V2/LSM.hs | 39 ++++++----- .../Consensus/Storage/LedgerDB/V2.hs | 43 ++++++------ .../Consensus/Storage/LedgerDB/V2/Forker.hs | 65 ++++++++++--------- .../Consensus/Storage/LedgerDB/V2/InMemory.hs | 11 ++-- .../Storage/LedgerDB/V2/LedgerSeq.hs | 16 +++-- 6 files changed, 102 insertions(+), 80 deletions(-) diff --git a/cabal.project b/cabal.project index 9c97a6c5ba..a5e14620b3 100644 --- a/cabal.project +++ b/cabal.project @@ -50,6 +50,14 @@ if impl (ghc >= 9.10) -- https://github.com/phadej/regression-simple/pull/14 , regression-simple:base +source-repository-package + type: git + location: https://github.com/IntersectMBO/io-classes-extra + tag: bdef43a0660c7af2a0c1f4021228ff4e9ac5af71 + --sha256: sha256-1if7Gt1dDx2Ryj9iXTbb6SlLA+nOLs6R3pH6TUAkM1o= + subdir: + resource-registry + source-repository-package type: git location: https://github.com/IntersectMBO/cardano-ledger diff --git a/ouroboros-consensus/src/ouroboros-consensus-lsm/Ouroboros/Consensus/Storage/LedgerDB/V2/LSM.hs b/ouroboros-consensus/src/ouroboros-consensus-lsm/Ouroboros/Consensus/Storage/LedgerDB/V2/LSM.hs index fb32c422bd..deb682d5b8 100644 --- a/ouroboros-consensus/src/ouroboros-consensus-lsm/Ouroboros/Consensus/Storage/LedgerDB/V2/LSM.hs +++ b/ouroboros-consensus/src/ouroboros-consensus-lsm/Ouroboros/Consensus/Storage/LedgerDB/V2/LSM.hs @@ -64,6 +64,7 @@ import Data.Void import Database.LSMTree (Salt, Session, Table) import qualified Database.LSMTree as LSM import GHC.Generics +import GHC.Stack (HasCallStack) import NoThunks.Class import Ouroboros.Consensus.Block import Ouroboros.Consensus.Ledger.Abstract @@ -167,21 +168,22 @@ newLSMLedgerTablesHandle :: , IndexedMemPack (l EmptyMK) (TxOut l) ) => Tracer m LedgerDBV2Trace -> - ResourceRegistry m -> (ResourceKey m, UTxOTable m) -> m (LedgerTablesHandle m l) -newLSMLedgerTablesHandle tracer rr (resKey, t) = do +newLSMLedgerTablesHandle tracer (origResKey, t) = do traceWith tracer TraceLedgerTablesHandleCreate + tv <- newTVarIO origResKey pure LedgerTablesHandle - { close = implClose resKey - , duplicate = implDuplicate rr t tracer + { close = implClose tv + , duplicate = \rr -> implDuplicate rr t tracer , read = implRead t , readRange = implReadRange t , readAll = implReadAll t , pushDiffs = implPushDiffs t , takeHandleSnapshot = implTakeHandleSnapshot t , tablesSize = pure Nothing + , transfer = atomically . writeTVar tv } {-# INLINE implClose #-} @@ -192,8 +194,11 @@ newLSMLedgerTablesHandle tracer rr (resKey, t) = do {-# INLINE implPushDiffs #-} {-# INLINE implTakeHandleSnapshot #-} -implClose :: IOLike m => ResourceKey m -> m () -implClose = Monad.void . release +implClose :: (HasCallStack, IOLike m) => StrictTVar m (ResourceKey m) -> Bool -> m () +implClose tv False = + Monad.void $ release =<< readTVarIO tv +implClose tv True = do + Monad.void $ unsafeRelease =<< readTVarIO tv implDuplicate :: ( IOLike m @@ -203,9 +208,9 @@ implDuplicate :: ResourceRegistry m -> UTxOTable m -> Tracer m LedgerDBV2Trace -> - m (LedgerTablesHandle m l) + m (ResourceKey m, LedgerTablesHandle m l) implDuplicate rr t tracer = do - table <- + (rk, table) <- allocate rr (\_ -> LSM.duplicate t) @@ -213,7 +218,7 @@ implDuplicate rr t tracer = do traceWith tracer TraceLedgerTablesHandleClose LSM.closeTable t' ) - newLSMLedgerTablesHandle tracer rr table + (rk,) <$> newLSMLedgerTablesHandle tracer (rk, table) implRead :: forall m l. @@ -461,7 +466,7 @@ loadSnapshot tracer rr ccfg fs session ds = case pointToWithOriginRealPoint (castPoint (getTip extLedgerSt)) of Origin -> throwE InitFailureGenesis NotOrigin pt -> do - values <- + (rk, values) <- lift $ allocate rr @@ -481,7 +486,7 @@ loadSnapshot tracer rr ccfg fs session ds = $ InitFailureRead ReadSnapshotDataCorruption (,pt) - <$> lift (empty extLedgerSt values (newLSMLedgerTablesHandle tracer rr)) + <$> lift (empty extLedgerSt (rk, values) (newLSMLedgerTablesHandle tracer)) -- | Create the initial LSM table from values, which should happen only at -- Genesis. @@ -495,18 +500,16 @@ tableFromValuesMK :: LedgerTables l ValuesMK -> m (ResourceKey m, UTxOTable m) tableFromValuesMK tracer rr session st (LedgerTables (ValuesMK values)) = do - res@(_, table) <- + (rk, table) <- allocate rr - ( \_ -> - LSM.newTableWith (LSM.defaultTableConfig{LSM.confFencePointerIndex = LSM.OrdinaryIndex}) session - ) + (\_ -> LSM.newTable session) ( \tb -> do traceWith tracer TraceLedgerTablesHandleClose LSM.closeTable tb ) mapM_ (go table) $ chunks 1000 $ Map.toList values - pure res + pure (rk, table) where go table items = LSM.inserts table $ @@ -600,7 +603,7 @@ instance newHandleFromValues trcr reg res st = do table <- tableFromValuesMK trcr reg (sessionResource res) (forgetLedgerTables st) (ltprj st) - newLSMLedgerTablesHandle trcr reg table + newLSMLedgerTablesHandle trcr table snapshotManager _ res = Ouroboros.Consensus.Storage.LedgerDB.V2.LSM.snapshotManager (sessionResource res) @@ -731,7 +734,7 @@ mkLSMYieldArgs fp snapName mkFS mkGen _ reg = do (LSM.SnapshotLabel $ T.pack "UTxO table") ) LSM.closeTable - YieldLSM 1000 <$> newLSMLedgerTablesHandle nullTracer reg tb + YieldLSM 1000 <$> newLSMLedgerTablesHandle nullTracer tb -- | Create Sink arguments for LSM mkLSMSinkArgs :: diff --git a/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Storage/LedgerDB/V2.hs b/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Storage/LedgerDB/V2.hs index d0e7bb1c9d..5e607a3423 100644 --- a/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Storage/LedgerDB/V2.hs +++ b/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Storage/LedgerDB/V2.hs @@ -111,6 +111,7 @@ mkInitDb args getBlock snapManager getVolatileSuffix res = do , ldbCfg = lgrConfig , ldbHasFS = lgrHasFS , ldbResolveBlock = getBlock + , ldbRegistry = lgrRegistry , ldbQueryBatchSize = lgrQueryBatchSize , ldbOpenHandlesLock = lock , ldbGetVolatileSuffix = getVolatileSuffix @@ -185,7 +186,7 @@ mkInternals h snapManager = let selectWhereTo = case whereTo of TakeAtImmutableTip -> anchorHandle TakeAtVolatileTip -> currentHandle - withStateRef env (MkSolo . selectWhereTo) $ \(MkSolo (st, _)) -> + withStateRef env (MkSolo . selectWhereTo) $ \(MkSolo (_, st)) -> Monad.void $ takeSnapshot snapManager @@ -350,7 +351,7 @@ implTryTakeSnapshot :: implTryTakeSnapshot snapManager env mTime nrBlocks = if onDiskShouldTakeSnapshot (ldbSnapshotPolicy env) (uncurry (flip diffTime) <$> mTime) nrBlocks then do - withStateRef env (MkSolo . anchorHandle) $ \(MkSolo (st, _)) -> + withStateRef env (MkSolo . anchorHandle) $ \(MkSolo (_, st)) -> Monad.void $ takeSnapshot snapManager @@ -428,6 +429,7 @@ data LedgerDBEnv m l blk = LedgerDBEnv , ldbHasFS :: !(SomeHasFS m) , ldbResolveBlock :: !(ResolveBlock m blk) , ldbQueryBatchSize :: !QueryBatchSize + , ldbRegistry :: !(ResourceRegistry m) , ldbOpenHandlesLock :: !(RAWLock m ()) -- ^ While holding a read lock (at least), all handles in the 'ldbSeq' are -- guaranteed to be open. During this time, the handle can be duplicated and @@ -562,13 +564,13 @@ getStateRef :: LedgerDBEnv m l blk -> ResourceRegistry m -> (LedgerSeq m l -> t (StateRef m l)) -> - m (t (StateRef m l, ResourceKey m)) + m (t (ResourceKey m, StateRef m l)) getStateRef ldbEnv reg project = RAWLock.withReadAccess (ldbOpenHandlesLock ldbEnv) $ \() -> do tst <- project <$> atomically (getVolatileLedgerSeq ldbEnv) for tst $ \st -> do - (resKey, tables') <- allocate reg (\_ -> duplicate $ tables st) close - pure (st{tables = tables'}, resKey) + (key, tables') <- duplicate (tables st) reg + pure (key, st{tables = tables'}) -- | Like 'StateRef', but takes care of closing the handle when the given action -- returns or errors. @@ -576,7 +578,7 @@ withStateRef :: (IOLike m, Traversable t, GetTip l) => LedgerDBEnv m l blk -> (LedgerSeq m l -> t (StateRef m l)) -> - (t (StateRef m l, ResourceKey m) -> m a) -> + (t (ResourceKey m, StateRef m l) -> m a) -> m a withStateRef ldbEnv project f = withRegistry $ \reg -> getStateRef ldbEnv reg project >>= f @@ -591,7 +593,7 @@ acquireAtTarget :: LedgerDBEnv m l blk -> Either Word64 (Target (Point blk)) -> ResourceRegistry m -> - m (Either GetForkerError (StateRef m l, ResourceKey m)) + m (Either GetForkerError (ResourceKey m, StateRef m l)) acquireAtTarget ldbEnv target reg = getStateRef ldbEnv reg $ \l -> case target of Right VolatileTip -> pure $ currentHandle l @@ -646,12 +648,11 @@ newForkerByRollback h rr n = getEnv h $ \ldbEnv -> closeForkerEnv :: IOLike m => ForkerEnv m l blk -> m () -closeForkerEnv ForkerEnv{foeResourcesToRelease = (lock, key, toRelease)} = - RAWLock.withWriteAccess lock $ - const $ do - Monad.join $ atomically (swapTVar toRelease (pure ())) - _ <- release key - pure ((), ()) +closeForkerEnv ForkerEnv{foeLedgerDbLock, foeCleanup, foeInitialHandleKey} = + RAWLock.withWriteAccess foeLedgerDbLock $ \() -> do + Monad.void $ release foeInitialHandleKey + Monad.join $ readTVarIO foeCleanup + pure ((), ()) getForkerEnv :: forall m l blk r. @@ -739,26 +740,24 @@ newForker :: LedgerDBHandle m l blk -> LedgerDBEnv m l blk -> ResourceRegistry m -> - (StateRef m l, ResourceKey m) -> + (ResourceKey m, StateRef m l) -> m (Forker m l blk) -newForker h ldbEnv rr (st, rk) = do +newForker h ldbEnv rr (rk, st) = do forkerKey <- atomically $ stateTVar (ldbNextForkerKey ldbEnv) $ \r -> (r, r + 1) let tr = LedgerDBForkerEvent . TraceForkerEventWithKey forkerKey >$< ldbTracer ldbEnv traceWith tr ForkerOpen lseqVar <- newTVarIO . LedgerSeq . AS.Empty $ st - -- The closing action that we allocate in the TVar from the start is not - -- strictly necessary if the caller uses a short-lived registry like the ones - -- in Chain selection or the forging loop. Just in case the user passes a - -- long-lived registry, we store such closing action to make sure the handle - -- is closed even under @forkerClose@ if the registry outlives the forker. - (k, toRelease) <- allocate rr (\_ -> newTVarIO (Monad.void (release rk))) (Monad.join . readTVarIO) + foeCleanup <- newTVarIO $ pure () let forkerEnv = ForkerEnv { foeLedgerSeq = lseqVar + , foeLedgerDbRegistry = ldbRegistry ldbEnv + , foeResourceRegistry = rr , foeSwitchVar = ldbSeq ldbEnv , foeTracer = tr - , foeResourcesToRelease = (ldbOpenHandlesLock ldbEnv, k, toRelease) , foeInitialHandleKey = rk + , foeCleanup + , foeLedgerDbLock = ldbOpenHandlesLock ldbEnv } atomically $ modifyTVar (ldbForkers ldbEnv) $ Map.insert forkerKey forkerEnv pure $ diff --git a/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Storage/LedgerDB/V2/Forker.hs b/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Storage/LedgerDB/V2/Forker.hs index b252ae5da2..e4c2f4bc81 100644 --- a/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Storage/LedgerDB/V2/Forker.hs +++ b/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Storage/LedgerDB/V2/Forker.hs @@ -19,8 +19,7 @@ module Ouroboros.Consensus.Storage.LedgerDB.V2.Forker , module Ouroboros.Consensus.Storage.LedgerDB.Forker ) where -import qualified Control.Monad as Monad -import Control.RAWLock hiding (read) +import Control.RAWLock (RAWLock) import Control.ResourceRegistry import Control.Tracer import Data.Maybe (fromMaybe) @@ -49,13 +48,22 @@ data ForkerEnv m l blk = ForkerEnv -- ^ Local version of the LedgerSeq , foeSwitchVar :: !(StrictTVar m (LedgerSeq m l)) -- ^ This TVar is the same as the LedgerDB one + , foeLedgerDbRegistry :: !(ResourceRegistry m) + -- ^ The registry in the LedgerDB to move handles to in case we commit the + -- forker. , foeTracer :: !(Tracer m TraceForkerEvent) -- ^ Config - , foeResourcesToRelease :: !(RAWLock m (), ResourceKey m, StrictTVar m (m ())) - -- ^ Release the resources + , foeResourceRegistry :: !(ResourceRegistry m) + -- ^ The registry local to the forker , foeInitialHandleKey :: !(ResourceKey m) -- ^ Resource key for the initial handle to ensure it is released. See -- comments in 'implForkerCommit'. + , foeCleanup :: !(StrictTVar m (m ())) + -- ^ An action to run on cleanup. If the forker was not committed this will be + -- the trivial action. Otherwise it will move the required handles to the + -- LedgerDB and release the discarded ones. + , foeLedgerDbLock :: !(RAWLock m ()) + -- ^ 'ldbOpenHandlesLock'. } deriving Generic @@ -127,17 +135,15 @@ implForkerPush env newState = do st = forgetLedgerTables newState bracketOnError - (duplicate (tables $ currentHandle lseq)) - close - ( \newtbs -> do + (duplicate (tables $ currentHandle lseq) (foeResourceRegistry env)) + (release . fst) + ( \(_, newtbs) -> do pushDiffs newtbs st0 newState let lseq' = extend (StateRef st newtbs) lseq traceWith (foeTracer env) ForkerPushEnd - atomically $ do - writeTVar (foeLedgerSeq env) lseq' - modifyTVar ((\(_, _, r) -> r) $ foeResourcesToRelease env) (>> close newtbs) + atomically $ writeTVar (foeLedgerSeq env) lseq' ) implForkerCommit :: @@ -148,42 +154,39 @@ implForkerCommit env = do LedgerSeq lseq <- readTVar foeLedgerSeq let intersectionSlot = getTipSlot $ state $ AS.anchor lseq let predicate = (== getTipHash (state (AS.anchor lseq))) . getTipHash . state - closeDiscarded <- do + transfer <- stateTVar foeSwitchVar ( \(LedgerSeq olddb) -> fromMaybe theImpossible $ do -- Split the selection at the intersection point. The snd component will -- have to be closed. - (olddb', toClose) <- AS.splitAfterMeasure intersectionSlot (either predicate predicate) olddb + (toKeepBase, toCloseLdb) <- AS.splitAfterMeasure intersectionSlot (either predicate predicate) olddb + (toCloseForker, toKeepTip) <- + AS.splitAfterMeasure intersectionSlot (either predicate predicate) lseq -- Join the prefix of the selection with the sequence in the forker - newdb <- AS.join (const $ const True) olddb' lseq - let closeDiscarded = do + newdb <- AS.join (const $ const True) toKeepBase toKeepTip + let transferCommitted = do -- Do /not/ close the anchor of @toClose@, as that is also the -- tip of @olddb'@ which will be used in @newdb@. - case toClose of + case toCloseLdb of AS.Empty _ -> pure () _ AS.:< closeOld' -> closeLedgerSeq (LedgerSeq closeOld') - -- Finally, close the anchor of @lseq@ (which is a duplicate of - -- the head of @olddb'@). To close this handle, we have to - -- release the 'foeInitialHandleKey' as that one is registered - -- on the registry used to open the forker. Releasing it will - -- call 'close' on the handle which will call 'release' on the key - -- for the handle. - Monad.void $ release foeInitialHandleKey - pure (closeDiscarded, LedgerSeq newdb) - ) + closeLedgerSeq (LedgerSeq toCloseForker) + + -- All the other remaining handles are transferred to the LedgerDB registry + keys <- ingestRegistry foeResourceRegistry foeLedgerDbRegistry + mapM_ (\(k, v) -> transfer (tables v) k) $ zip keys (AS.toOldestFirst toKeepTip) - -- We are discarding the previous value in the TVar because we had accumulated - -- actions for closing the states pushed to the forker. As we are committing - -- those we have to close the ones discarded in this function and forget about - -- those releasing actions. - writeTVar ((\(_, _, r) -> r) $ foeResourcesToRelease) closeDiscarded + pure (transferCommitted, LedgerSeq newdb) + ) + writeTVar foeCleanup transfer where ForkerEnv { foeLedgerSeq , foeSwitchVar - , foeResourcesToRelease - , foeInitialHandleKey + , foeResourceRegistry + , foeLedgerDbRegistry + , foeCleanup } = env theImpossible = diff --git a/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Storage/LedgerDB/V2/InMemory.hs b/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Storage/LedgerDB/V2/InMemory.hs index be6ec8a080..1d73f5af60 100644 --- a/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Storage/LedgerDB/V2/InMemory.hs +++ b/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Storage/LedgerDB/V2/InMemory.hs @@ -116,6 +116,7 @@ newInMemoryLedgerTablesHandle tracer someFS@(SomeHasFS hasFS) l = do , pushDiffs = implPushDiffs tv , takeHandleSnapshot = implTakeHandleSnapshot tv hasFS , tablesSize = implTablesSize tv + , transfer = const (pure ()) } {-# INLINE implClose #-} @@ -131,8 +132,9 @@ implClose :: IOLike m => Tracer m LedgerDBV2Trace -> StrictTVar m (LedgerTablesHandleState l) -> + Bool -> m () -implClose tracer tv = do +implClose tracer tv _ = do p <- atomically $ swapTVar tv LedgerTablesHandleClosed case p of LedgerTablesHandleOpen{} -> traceWith tracer TraceLedgerTablesHandleClose @@ -147,10 +149,11 @@ implDuplicate :: Tracer m LedgerDBV2Trace -> StrictTVar m (LedgerTablesHandleState l) -> SomeHasFS m -> - m (LedgerTablesHandle m l) -implDuplicate tracer tv someFS = do + ResourceRegistry m -> + m (ResourceKey m, LedgerTablesHandle m l) +implDuplicate tracer tv someFS rr = do hs <- readTVarIO tv - !x <- guardClosed hs $ newInMemoryLedgerTablesHandle tracer someFS + !x <- guardClosed hs $ \v -> allocate rr (\_ -> newInMemoryLedgerTablesHandle tracer someFS v) (const $ pure ()) pure x implRead :: diff --git a/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Storage/LedgerDB/V2/LedgerSeq.hs b/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Storage/LedgerDB/V2/LedgerSeq.hs index 774c35b5e6..8c2240b40b 100644 --- a/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Storage/LedgerDB/V2/LedgerSeq.hs +++ b/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Storage/LedgerDB/V2/LedgerSeq.hs @@ -82,8 +82,12 @@ import Prelude hiding (read) -- | The interface fulfilled by handles on both the InMemory and LSM handles. data LedgerTablesHandle m l = LedgerTablesHandle - { close :: !(m ()) - , duplicate :: !(m (LedgerTablesHandle m l)) + { close :: !(Bool -> m ()) + -- ^ Boolean is whether to force release or not. + , transfer :: !(ResourceKey m -> m ()) + -- ^ Update the closing action in this handle with a new resource key, as the + -- handle has moved to a different registry. + , duplicate :: !(ResourceRegistry m -> m (ResourceKey m, LedgerTablesHandle m l)) -- ^ Create a copy of the handle. -- -- A duplicated handle must provide access to all the data that was there in @@ -219,7 +223,9 @@ empty' st = empty (forgetLedgerTables st) st -- the anchor. closeLedgerSeq :: Monad m => LedgerSeq m l -> m () closeLedgerSeq (LedgerSeq l) = - mapM_ (close . tables) $ AS.anchor l : AS.toOldestFirst l + mapM_ + (\t -> close (tables t) True) + $ AS.anchor l : AS.toOldestFirst l {------------------------------------------------------------------------------- Apply blocks @@ -249,10 +255,10 @@ reapplyBlock :: ResourceRegistry m -> LedgerSeq m l -> m (StateRef m l) -reapplyBlock evs cfg b _rr db = do +reapplyBlock evs cfg b rr db = do let ks = getBlockKeySets b StateRef st tbs = currentHandle db - newtbs <- duplicate tbs + (_, newtbs) <- duplicate tbs rr vals <- read newtbs st ks let st' = tickThenReapply evs cfg b (st `withLedgerTables` vals) newst = forgetLedgerTables st' From 8ad79ba817693a6376dd8ee212337e544cade0dd Mon Sep 17 00:00:00 2001 From: Javier Sagredo Date: Thu, 16 Oct 2025 11:30:44 +0200 Subject: [PATCH 02/10] Release LedgerDB handles in the LedgerDB --- .../Consensus/Storage/LedgerDB/V2/LSM.hs | 6 ++---- .../Consensus/Storage/LedgerDB/V2.hs | 10 ++++++++++ .../Consensus/Storage/LedgerDB/V2/Forker.hs | 20 +++++++++++-------- .../Consensus/Storage/LedgerDB/V2/InMemory.hs | 3 +-- .../Storage/LedgerDB/V2/LedgerSeq.hs | 7 ++----- 5 files changed, 27 insertions(+), 19 deletions(-) diff --git a/ouroboros-consensus/src/ouroboros-consensus-lsm/Ouroboros/Consensus/Storage/LedgerDB/V2/LSM.hs b/ouroboros-consensus/src/ouroboros-consensus-lsm/Ouroboros/Consensus/Storage/LedgerDB/V2/LSM.hs index deb682d5b8..659cac9f5a 100644 --- a/ouroboros-consensus/src/ouroboros-consensus-lsm/Ouroboros/Consensus/Storage/LedgerDB/V2/LSM.hs +++ b/ouroboros-consensus/src/ouroboros-consensus-lsm/Ouroboros/Consensus/Storage/LedgerDB/V2/LSM.hs @@ -194,11 +194,9 @@ newLSMLedgerTablesHandle tracer (origResKey, t) = do {-# INLINE implPushDiffs #-} {-# INLINE implTakeHandleSnapshot #-} -implClose :: (HasCallStack, IOLike m) => StrictTVar m (ResourceKey m) -> Bool -> m () -implClose tv False = +implClose :: (HasCallStack, IOLike m) => StrictTVar m (ResourceKey m) -> m () +implClose tv = Monad.void $ release =<< readTVarIO tv -implClose tv True = do - Monad.void $ unsafeRelease =<< readTVarIO tv implDuplicate :: ( IOLike m diff --git a/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Storage/LedgerDB/V2.hs b/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Storage/LedgerDB/V2.hs index 5e607a3423..5013a4a628 100644 --- a/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Storage/LedgerDB/V2.hs +++ b/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Storage/LedgerDB/V2.hs @@ -100,6 +100,7 @@ mkInitDb args getBlock snapManager getVolatileSuffix res = do lock <- RAWLock.new () forkers <- newTVarIO Map.empty nextForkerKey <- newTVarIO (ForkerKey 0) + ldbToClose <- newTVarIO [] let env = LedgerDBEnv { ldbSeq = varDB @@ -116,6 +117,7 @@ mkInitDb args getBlock snapManager getVolatileSuffix res = do , ldbOpenHandlesLock = lock , ldbGetVolatileSuffix = getVolatileSuffix , ldbResourceKeys = SomeResources res + , ldbToClose } h <- LDBHandle <$> newTVarIO (LedgerDBOpen env) pure $ implMkLedgerDb h snapManager @@ -331,6 +333,7 @@ implGarbageCollect env slotNo = do atomically $ modifyTVar (ldbPrevApplied env) $ Set.dropWhileAntitone ((< slotNo) . realPointSlot) + mapM_ closeLedgerSeq =<< readTVarIO (ldbToClose env) -- It is safe to close the handles outside of the locked region, which reduces -- contention. See the docs of 'ldbOpenHandlesLock'. Monad.join $ RAWLock.withWriteAccess (ldbOpenHandlesLock env) $ \() -> do @@ -430,6 +433,12 @@ data LedgerDBEnv m l blk = LedgerDBEnv , ldbResolveBlock :: !(ResolveBlock m blk) , ldbQueryBatchSize :: !QueryBatchSize , ldbRegistry :: !(ResourceRegistry m) + -- ^ The registry of the LedgerDB, to give it to forkers to transfer committed + -- handles to the LedgerDB. + , ldbToClose :: !(StrictTVar m [LedgerSeq m l]) + -- ^ When committing forkers, the discarded part of the LedgerDB will be put + -- in this TVar such that the 'garbageCollect' function will release such + -- resources. , ldbOpenHandlesLock :: !(RAWLock m ()) -- ^ While holding a read lock (at least), all handles in the 'ldbSeq' are -- guaranteed to be open. During this time, the handle can be duplicated and @@ -758,6 +767,7 @@ newForker h ldbEnv rr (rk, st) = do , foeInitialHandleKey = rk , foeCleanup , foeLedgerDbLock = ldbOpenHandlesLock ldbEnv + , foeLedgerDbToClose = ldbToClose ldbEnv } atomically $ modifyTVar (ldbForkers ldbEnv) $ Map.insert forkerKey forkerEnv pure $ diff --git a/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Storage/LedgerDB/V2/Forker.hs b/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Storage/LedgerDB/V2/Forker.hs index e4c2f4bc81..479f007966 100644 --- a/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Storage/LedgerDB/V2/Forker.hs +++ b/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Storage/LedgerDB/V2/Forker.hs @@ -33,6 +33,7 @@ import Ouroboros.Consensus.Storage.LedgerDB.API import Ouroboros.Consensus.Storage.LedgerDB.Args import Ouroboros.Consensus.Storage.LedgerDB.Forker import Ouroboros.Consensus.Storage.LedgerDB.V2.LedgerSeq +import Ouroboros.Consensus.Util (whenJust) import Ouroboros.Consensus.Util.CallStack import Ouroboros.Consensus.Util.IOLike import Ouroboros.Consensus.Util.NormalForm.StrictTVar () @@ -51,6 +52,7 @@ data ForkerEnv m l blk = ForkerEnv , foeLedgerDbRegistry :: !(ResourceRegistry m) -- ^ The registry in the LedgerDB to move handles to in case we commit the -- forker. + , foeLedgerDbToClose :: !(StrictTVar m [LedgerSeq m l]) , foeTracer :: !(Tracer m TraceForkerEvent) -- ^ Config , foeResourceRegistry :: !(ResourceRegistry m) @@ -154,7 +156,7 @@ implForkerCommit env = do LedgerSeq lseq <- readTVar foeLedgerSeq let intersectionSlot = getTipSlot $ state $ AS.anchor lseq let predicate = (== getTipHash (state (AS.anchor lseq))) . getTipHash . state - transfer <- + (transfer, ldbToClose) <- stateTVar foeSwitchVar ( \(LedgerSeq olddb) -> fromMaybe theImpossible $ do @@ -165,20 +167,21 @@ implForkerCommit env = do AS.splitAfterMeasure intersectionSlot (either predicate predicate) lseq -- Join the prefix of the selection with the sequence in the forker newdb <- AS.join (const $ const True) toKeepBase toKeepTip - let transferCommitted = do - -- Do /not/ close the anchor of @toClose@, as that is also the - -- tip of @olddb'@ which will be used in @newdb@. - case toCloseLdb of - AS.Empty _ -> pure () - _ AS.:< closeOld' -> closeLedgerSeq (LedgerSeq closeOld') + -- Do /not/ close the anchor of @toClose@, as that is also the + -- tip of @olddb'@ which will be used in @newdb@. + let ldbToClose = case toCloseLdb of + AS.Empty _ -> Nothing + _ AS.:< closeOld' -> Just (LedgerSeq closeOld') + transferCommitted = do closeLedgerSeq (LedgerSeq toCloseForker) -- All the other remaining handles are transferred to the LedgerDB registry keys <- ingestRegistry foeResourceRegistry foeLedgerDbRegistry mapM_ (\(k, v) -> transfer (tables v) k) $ zip keys (AS.toOldestFirst toKeepTip) - pure (transferCommitted, LedgerSeq newdb) + pure ((transferCommitted, ldbToClose), LedgerSeq newdb) ) + whenJust ldbToClose (modifyTVar foeLedgerDbToClose . (:)) writeTVar foeCleanup transfer where ForkerEnv @@ -187,6 +190,7 @@ implForkerCommit env = do , foeResourceRegistry , foeLedgerDbRegistry , foeCleanup + , foeLedgerDbToClose } = env theImpossible = diff --git a/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Storage/LedgerDB/V2/InMemory.hs b/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Storage/LedgerDB/V2/InMemory.hs index 1d73f5af60..a9049b5ff8 100644 --- a/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Storage/LedgerDB/V2/InMemory.hs +++ b/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Storage/LedgerDB/V2/InMemory.hs @@ -132,9 +132,8 @@ implClose :: IOLike m => Tracer m LedgerDBV2Trace -> StrictTVar m (LedgerTablesHandleState l) -> - Bool -> m () -implClose tracer tv _ = do +implClose tracer tv = do p <- atomically $ swapTVar tv LedgerTablesHandleClosed case p of LedgerTablesHandleOpen{} -> traceWith tracer TraceLedgerTablesHandleClose diff --git a/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Storage/LedgerDB/V2/LedgerSeq.hs b/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Storage/LedgerDB/V2/LedgerSeq.hs index 8c2240b40b..49ce1df73e 100644 --- a/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Storage/LedgerDB/V2/LedgerSeq.hs +++ b/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Storage/LedgerDB/V2/LedgerSeq.hs @@ -82,8 +82,7 @@ import Prelude hiding (read) -- | The interface fulfilled by handles on both the InMemory and LSM handles. data LedgerTablesHandle m l = LedgerTablesHandle - { close :: !(Bool -> m ()) - -- ^ Boolean is whether to force release or not. + { close :: !(m ()) , transfer :: !(ResourceKey m -> m ()) -- ^ Update the closing action in this handle with a new resource key, as the -- handle has moved to a different registry. @@ -223,9 +222,7 @@ empty' st = empty (forgetLedgerTables st) st -- the anchor. closeLedgerSeq :: Monad m => LedgerSeq m l -> m () closeLedgerSeq (LedgerSeq l) = - mapM_ - (\t -> close (tables t) True) - $ AS.anchor l : AS.toOldestFirst l + mapM_ (close . tables) $ AS.anchor l : AS.toOldestFirst l {------------------------------------------------------------------------------- Apply blocks From 8efd19b5744bfce66becd24e247d08e999127c7a Mon Sep 17 00:00:00 2001 From: Javier Sagredo Date: Thu, 16 Oct 2025 17:10:10 +0200 Subject: [PATCH 03/10] Garbage collect when necessary in LedgerDB.StateMachine tests --- .../Ouroboros/Consensus/Storage/LedgerDB/V2/InMemory.hs | 6 +++++- .../Test/Ouroboros/Storage/LedgerDB/StateMachine.hs | 3 +++ 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Storage/LedgerDB/V2/InMemory.hs b/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Storage/LedgerDB/V2/InMemory.hs index a9049b5ff8..ffcfc064fe 100644 --- a/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Storage/LedgerDB/V2/InMemory.hs +++ b/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Storage/LedgerDB/V2/InMemory.hs @@ -152,7 +152,11 @@ implDuplicate :: m (ResourceKey m, LedgerTablesHandle m l) implDuplicate tracer tv someFS rr = do hs <- readTVarIO tv - !x <- guardClosed hs $ \v -> allocate rr (\_ -> newInMemoryLedgerTablesHandle tracer someFS v) (const $ pure ()) + !x <- guardClosed hs $ \v -> + allocate + rr + (\_ -> newInMemoryLedgerTablesHandle tracer someFS v) + close pure x implRead :: diff --git a/ouroboros-consensus/test/storage-test/Test/Ouroboros/Storage/LedgerDB/StateMachine.hs b/ouroboros-consensus/test/storage-test/Test/Ouroboros/Storage/LedgerDB/StateMachine.hs index 1c45dae1be..f1e7d4ac37 100644 --- a/ouroboros-consensus/test/storage-test/Test/Ouroboros/Storage/LedgerDB/StateMachine.hs +++ b/ouroboros-consensus/test/storage-test/Test/Ouroboros/Storage/LedgerDB/StateMachine.hs @@ -614,6 +614,8 @@ instance RunModel Model (StateT Environment IO) where let args = mkArgs secParam salt -- TODO after a drop and restore we restart the db but the session has been closed below where I wrote blahblahblah openLedgerDB (argFlavorArgs args) chainDb (argLedgerDbCfg args) fs rr + lift $ + garbageCollect ldb . fromWithOrigin 0 . pointSlot . getTip =<< atomically (getImmutableTip ldb) put (Environment ldb testInternals chainDb mkArgs fs getNumOpenHandles cleanup rr) pure $ pure () perform _ WipeLedgerDB _ = do @@ -642,6 +644,7 @@ instance RunModel Model (StateT Environment IO) where (reverse (map blockRealPoint blks) ++) . drop (fromIntegral n) atomically (forkerCommit forker) forkerClose forker + garbageCollect ldb . fromWithOrigin 0 . pointSlot . getTip =<< atomically (getImmutableTip ldb) pure $ pure () ValidateExceededRollBack{} -> pure $ Left ErrorValidateExceededRollback ValidateLedgerError (AnnLedgerError forker _ err) -> forkerClose forker >> error ("Unexpected ledger error" <> show err) From 2b491bd2fec99ec5b8b4fcbab2462b8e37b0db2e Mon Sep 17 00:00:00 2001 From: Javier Sagredo Date: Thu, 16 Oct 2025 17:10:14 +0200 Subject: [PATCH 04/10] Use new resource-registry which cancels threads before closing registries --- cabal.project | 10 +--------- flake.lock | 12 ++++++------ .../ouroboros-consensus-diffusion.cabal | 2 +- .../Test/ThreadNet/Network.hs | 2 +- ouroboros-consensus/ouroboros-consensus.cabal | 4 ++-- .../Consensus/Storage/LedgerDB/V2/Forker.hs | 2 +- 6 files changed, 12 insertions(+), 20 deletions(-) diff --git a/cabal.project b/cabal.project index a5e14620b3..b983c81b56 100644 --- a/cabal.project +++ b/cabal.project @@ -14,7 +14,7 @@ repository cardano-haskell-packages -- update either of these. index-state: -- Bump this if you need newer packages from Hackage - , hackage.haskell.org 2025-09-26T20:57:57Z + , hackage.haskell.org 2025-10-23T13:39:53Z -- Bump this if you need newer packages from CHaP , cardano-haskell-packages 2025-10-01T14:54:25Z @@ -50,14 +50,6 @@ if impl (ghc >= 9.10) -- https://github.com/phadej/regression-simple/pull/14 , regression-simple:base -source-repository-package - type: git - location: https://github.com/IntersectMBO/io-classes-extra - tag: bdef43a0660c7af2a0c1f4021228ff4e9ac5af71 - --sha256: sha256-1if7Gt1dDx2Ryj9iXTbb6SlLA+nOLs6R3pH6TUAkM1o= - subdir: - resource-registry - source-repository-package type: git location: https://github.com/IntersectMBO/cardano-ledger diff --git a/flake.lock b/flake.lock index acef080678..65bf9573ef 100644 --- a/flake.lock +++ b/flake.lock @@ -3,11 +3,11 @@ "CHaP": { "flake": false, "locked": { - "lastModified": 1759339316, - "narHash": "sha256-SW/K9yfhNLNCDAl2ZC8ol0w8X+AwyLin0XOvnn50468=", + "lastModified": 1761315163, + "narHash": "sha256-h+JPIMflNAOpY3XhZNcS5sUAOyO06499uWATj2j6P5Q=", "owner": "intersectmbo", "repo": "cardano-haskell-packages", - "rev": "aa50d6dffede91c8fdfcef94c71641a00214522a", + "rev": "131bcd51c4869b191e8c3afbb9f3fd326cd6e5e1", "type": "github" }, "original": { @@ -270,11 +270,11 @@ "hackageNix": { "flake": false, "locked": { - "lastModified": 1759314141, - "narHash": "sha256-eioqBX8q8H9lIj6bcG7JhwfN7Kg+OdpV1lkz7GS+/GI=", + "lastModified": 1761733768, + "narHash": "sha256-49u0SAbp4J9MqqbtKeTo+mVFuwA1CDb/ssc+AZtjgiI=", "owner": "input-output-hk", "repo": "hackage.nix", - "rev": "a58ee1f20b5db10c09beffc6d18505b6a253b84a", + "rev": "35e8a092207e6d5d921efffc59ec0a3c1b450d48", "type": "github" }, "original": { diff --git a/ouroboros-consensus-diffusion/ouroboros-consensus-diffusion.cabal b/ouroboros-consensus-diffusion/ouroboros-consensus-diffusion.cabal index f1c5b42fcd..917d1145f7 100644 --- a/ouroboros-consensus-diffusion/ouroboros-consensus-diffusion.cabal +++ b/ouroboros-consensus-diffusion/ouroboros-consensus-diffusion.cabal @@ -95,7 +95,7 @@ library ouroboros-network-framework ^>=0.19, ouroboros-network-protocols ^>=0.15, random, - resource-registry ^>=0.1, + resource-registry ^>=0.2, safe-wild-cards ^>=1.0, serialise ^>=0.2, text, diff --git a/ouroboros-consensus-diffusion/src/unstable-diffusion-testlib/Test/ThreadNet/Network.hs b/ouroboros-consensus-diffusion/src/unstable-diffusion-testlib/Test/ThreadNet/Network.hs index 6df036c539..8a58fe3b03 100644 --- a/ouroboros-consensus-diffusion/src/unstable-diffusion-testlib/Test/ThreadNet/Network.hs +++ b/ouroboros-consensus-diffusion/src/unstable-diffusion-testlib/Test/ThreadNet/Network.hs @@ -1158,7 +1158,7 @@ runThreadNetwork mempool txs0 - void $ allocate registry (\_ -> pure threadCrucialTxs) cancelThread + void $ allocateThread registry (\_ -> pure threadCrucialTxs) forkTxProducer coreNodeId diff --git a/ouroboros-consensus/ouroboros-consensus.cabal b/ouroboros-consensus/ouroboros-consensus.cabal index 5f2cd98720..8774d1c049 100644 --- a/ouroboros-consensus/ouroboros-consensus.cabal +++ b/ouroboros-consensus/ouroboros-consensus.cabal @@ -344,7 +344,7 @@ library psqueues ^>=0.2.3, quiet ^>=0.2, rawlock ^>=0.1.1, - resource-registry ^>=0.1, + resource-registry ^>=0.2, semialign >=1.1, serialise ^>=0.2, singletons, @@ -393,7 +393,7 @@ library ouroboros-consensus-lsm ouroboros-consensus, primitive, random, - resource-registry ^>=0.1, + resource-registry ^>=0.2, serialise ^>=0.2, streaming, text, diff --git a/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Storage/LedgerDB/V2/Forker.hs b/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Storage/LedgerDB/V2/Forker.hs index 479f007966..a1b9e77c49 100644 --- a/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Storage/LedgerDB/V2/Forker.hs +++ b/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Storage/LedgerDB/V2/Forker.hs @@ -176,7 +176,7 @@ implForkerCommit env = do closeLedgerSeq (LedgerSeq toCloseForker) -- All the other remaining handles are transferred to the LedgerDB registry - keys <- ingestRegistry foeResourceRegistry foeLedgerDbRegistry + keys <- transferRegistry foeResourceRegistry foeLedgerDbRegistry mapM_ (\(k, v) -> transfer (tables v) k) $ zip keys (AS.toOldestFirst toKeepTip) pure ((transferCommitted, ldbToClose), LedgerSeq newdb) From ec18c080a9c8b5e276a3a64d2041be45dd89fcd5 Mon Sep 17 00:00:00 2001 From: Javier Sagredo Date: Wed, 29 Oct 2025 11:32:41 +0100 Subject: [PATCH 05/10] Don't allocate the mempool sync thread in the top level registry --- ...ier.sagredo_fix_mempool_dangling_forker.md | 25 +++++++++++++++++++ .../Ouroboros/Consensus/Mempool/Init.hs | 13 +++------- 2 files changed, 28 insertions(+), 10 deletions(-) create mode 100644 ouroboros-consensus/changelog.d/20251029_113308_javier.sagredo_fix_mempool_dangling_forker.md diff --git a/ouroboros-consensus/changelog.d/20251029_113308_javier.sagredo_fix_mempool_dangling_forker.md b/ouroboros-consensus/changelog.d/20251029_113308_javier.sagredo_fix_mempool_dangling_forker.md new file mode 100644 index 0000000000..1c07c90a4a --- /dev/null +++ b/ouroboros-consensus/changelog.d/20251029_113308_javier.sagredo_fix_mempool_dangling_forker.md @@ -0,0 +1,25 @@ + + +### Patch + +- The Mempool sync thread was allocated in the top level registry in order to + ensure it would be cancelled before the mempool registry was shutting + down. This was solved in `resource-registry-0.2.0.0`. + + + diff --git a/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Mempool/Init.hs b/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Mempool/Init.hs index 06708c4317..5132133fe4 100644 --- a/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Mempool/Init.hs +++ b/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Mempool/Init.hs @@ -41,7 +41,7 @@ openMempool :: m (Mempool m blk) openMempool topLevelRegistry ledger cfg capacityOverride tracer = do env <- initMempoolEnv ledger cfg capacityOverride tracer topLevelRegistry - forkSyncStateOnTipPointChange topLevelRegistry env + forkSyncStateOnTipPointChange env return $ mkMempool env -- | Spawn a thread which syncs the 'Mempool' state whenever the 'LedgerState' @@ -53,11 +53,10 @@ forkSyncStateOnTipPointChange :: , HasTxId (GenTx blk) , ValidateEnvelope blk ) => - ResourceRegistry m -> MempoolEnv m blk -> m () -forkSyncStateOnTipPointChange topLevelRegistry menv = do - w <- +forkSyncStateOnTipPointChange menv = + void $ forkLinkedWatcher (mpEnvRegistry menv) "Mempool.syncStateOnTipPointChange" @@ -67,12 +66,6 @@ forkSyncStateOnTipPointChange topLevelRegistry menv = do , wNotify = action , wReader = getCurrentTip } - - -- With this allocation on the top level registry, we make sure that we first - -- stop the watcher thread before closing the mempool registry, as otherwise - -- we would run into a race condition (the thread might try to re-sync and - -- allocate a forker on the mempool registry which would be closing down). - void $ allocate topLevelRegistry (\_ -> pure w) cancelThread where action :: MempoolLedgerDBView m blk -> m () action _a = From cdfed5cf6b7310b5acb25449a294f444253a7d6e Mon Sep 17 00:00:00 2001 From: Javier Sagredo Date: Thu, 23 Oct 2025 15:27:02 +0200 Subject: [PATCH 06/10] Add changelogs --- ...49_javier.sagredo_fix_resource_registry.md | 23 +++++++++++++++++++ ...34_javier.sagredo_fix_resource_registry.md | 22 ++++++++++++++++++ 2 files changed, 45 insertions(+) create mode 100644 ouroboros-consensus-diffusion/changelog.d/20251023_152349_javier.sagredo_fix_resource_registry.md create mode 100644 ouroboros-consensus/changelog.d/20251023_152234_javier.sagredo_fix_resource_registry.md diff --git a/ouroboros-consensus-diffusion/changelog.d/20251023_152349_javier.sagredo_fix_resource_registry.md b/ouroboros-consensus-diffusion/changelog.d/20251023_152349_javier.sagredo_fix_resource_registry.md new file mode 100644 index 0000000000..e39a3215bf --- /dev/null +++ b/ouroboros-consensus-diffusion/changelog.d/20251023_152349_javier.sagredo_fix_resource_registry.md @@ -0,0 +1,23 @@ + + +### Patch + +- Bump to `resource-registry ^>= 0.2`. + + + diff --git a/ouroboros-consensus/changelog.d/20251023_152234_javier.sagredo_fix_resource_registry.md b/ouroboros-consensus/changelog.d/20251023_152234_javier.sagredo_fix_resource_registry.md new file mode 100644 index 0000000000..ba6e9f7049 --- /dev/null +++ b/ouroboros-consensus/changelog.d/20251023_152234_javier.sagredo_fix_resource_registry.md @@ -0,0 +1,22 @@ + + +### Patch + +- Bump to `resource-registry ^>= 0.2`. + +### Non-Breaking + +- Committing a forker will move the handles to the registry of the LedgerDB. The + discarded fork will be queued to be released by the `garbageCollect` logic. + + From 16b8139e51ee905369c58295e6ef4e46e0274857 Mon Sep 17 00:00:00 2001 From: Javier Sagredo Date: Tue, 14 Oct 2025 17:01:04 +0200 Subject: [PATCH 07/10] Expose traces for node --- .../Consensus/Storage/LedgerDB/V1/BackingStore/Impl/LMDB.hs | 3 ++- .../Ouroboros/Consensus/Storage/LedgerDB/V2/LSM.hs | 4 +++- .../Ouroboros/Consensus/Storage/LedgerDB/V1/BackingStore.hs | 5 +++-- .../Storage/LedgerDB/V1/BackingStore/Impl/InMemory.hs | 2 +- .../Ouroboros/Consensus/Storage/LedgerDB/V2/Backend.hs | 6 ++++-- .../Ouroboros/Consensus/Storage/LedgerDB/V2/InMemory.hs | 3 ++- 6 files changed, 15 insertions(+), 8 deletions(-) diff --git a/ouroboros-consensus/src/ouroboros-consensus-lmdb/Ouroboros/Consensus/Storage/LedgerDB/V1/BackingStore/Impl/LMDB.hs b/ouroboros-consensus/src/ouroboros-consensus-lmdb/Ouroboros/Consensus/Storage/LedgerDB/V1/BackingStore/Impl/LMDB.hs index 1a0044476d..068c3cce61 100644 --- a/ouroboros-consensus/src/ouroboros-consensus-lmdb/Ouroboros/Consensus/Storage/LedgerDB/V1/BackingStore/Impl/LMDB.hs +++ b/ouroboros-consensus/src/ouroboros-consensus-lmdb/Ouroboros/Consensus/Storage/LedgerDB/V1/BackingStore/Impl/LMDB.hs @@ -22,6 +22,7 @@ module Ouroboros.Consensus.Storage.LedgerDB.V1.BackingStore.Impl.LMDB LMDB , Backend (..) , Args (LMDBBackingStoreArgs) + , Trace (OnDiskBackingStoreInitialise, OnDiskBackingStoreTrace) , LMDBLimits (LMDBLimits, lmdbMapSize, lmdbMaxDatabases, lmdbMaxReaders) , mkLMDBArgs @@ -832,7 +833,7 @@ instance where data Args m LMDB = LMDBBackingStoreArgs FilePath LMDBLimits (Dict.Dict MonadIOPrim m) - data Trace m LMDB + data Trace LMDB = OnDiskBackingStoreInitialise LMDB.Limits | OnDiskBackingStoreTrace BackingStoreTrace deriving (Eq, Show) diff --git a/ouroboros-consensus/src/ouroboros-consensus-lsm/Ouroboros/Consensus/Storage/LedgerDB/V2/LSM.hs b/ouroboros-consensus/src/ouroboros-consensus-lsm/Ouroboros/Consensus/Storage/LedgerDB/V2/LSM.hs index 659cac9f5a..4f0c97c333 100644 --- a/ouroboros-consensus/src/ouroboros-consensus-lsm/Ouroboros/Consensus/Storage/LedgerDB/V2/LSM.hs +++ b/ouroboros-consensus/src/ouroboros-consensus-lsm/Ouroboros/Consensus/Storage/LedgerDB/V2/LSM.hs @@ -25,6 +25,8 @@ module Ouroboros.Consensus.Storage.LedgerDB.V2.LSM LSM , Backend (..) , Args (LSMArgs) + , Trace (LSMTreeTrace) + , LSM.LSMTreeTrace (..) , mkLSMArgs , stdMkBlockIOFS @@ -571,7 +573,7 @@ instance } deriving Generic - data Trace m LSM + data Trace LSM = LSMTreeTrace !LSM.LSMTreeTrace deriving Show diff --git a/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Storage/LedgerDB/V1/BackingStore.hs b/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Storage/LedgerDB/V1/BackingStore.hs index 360c23e96c..91042007b3 100644 --- a/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Storage/LedgerDB/V1/BackingStore.hs +++ b/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Storage/LedgerDB/V1/BackingStore.hs @@ -32,6 +32,7 @@ module Ouroboros.Consensus.Storage.LedgerDB.V1.BackingStore import Cardano.Slotting.Slot import Control.Tracer import Data.Proxy +import Data.Typeable import Ouroboros.Consensus.Ledger.Basics import Ouroboros.Consensus.Storage.LedgerDB.Snapshots import Ouroboros.Consensus.Storage.LedgerDB.V1.BackingStore.API @@ -67,7 +68,7 @@ data SomeBackendArgs m l where SomeBackendArgs :: Backend m backend l => Args m backend -> SomeBackendArgs m l data SomeBackendTrace where - SomeBackendTrace :: Show (Trace m backend) => Trace m backend -> SomeBackendTrace + SomeBackendTrace :: (Show (Trace backend), Typeable backend) => Trace backend -> SomeBackendTrace instance Show SomeBackendTrace where show (SomeBackendTrace tr) = show tr @@ -75,7 +76,7 @@ instance Show SomeBackendTrace where class Backend m backend l where data Args m backend - data Trace m backend + data Trace backend isRightBackendForSnapshot :: Proxy l -> diff --git a/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Storage/LedgerDB/V1/BackingStore/Impl/InMemory.hs b/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Storage/LedgerDB/V1/BackingStore/Impl/InMemory.hs index f20100eae8..eb7e3964dc 100644 --- a/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Storage/LedgerDB/V1/BackingStore/Impl/InMemory.hs +++ b/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Storage/LedgerDB/V1/BackingStore/Impl/InMemory.hs @@ -360,7 +360,7 @@ instance Backend m Mem l where data Args m Mem = InMemArgs - data Trace m Mem + data Trace Mem = InMemoryBackingStoreInitialise | InMemoryBackingStoreTrace BackingStoreTrace deriving (Eq, Show) diff --git a/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Storage/LedgerDB/V2/Backend.hs b/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Storage/LedgerDB/V2/Backend.hs index 7ff562ee48..409dd021b4 100644 --- a/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Storage/LedgerDB/V2/Backend.hs +++ b/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Storage/LedgerDB/V2/Backend.hs @@ -23,6 +23,7 @@ import Control.Monad.Except import Control.ResourceRegistry import Control.Tracer import Data.Proxy +import Data.Typeable import NoThunks.Class import Ouroboros.Consensus.Block import Ouroboros.Consensus.Ledger.Abstract @@ -41,7 +42,7 @@ class NoThunks (Resources m backend) => Backend m backend blk where data Resources m backend -- | A trace dependent on the particular backend. - data Trace m backend + data Trace backend -- | Transform 'Args' into 'Resources', with some context made up of -- 'LedgerDbArgs'. @@ -89,7 +90,8 @@ class NoThunks (Resources m backend) => Backend m backend blk where -------------------------------------------------------------------------------} data SomeBackendTrace where - SomeBackendTrace :: Show (Trace m backend) => Trace m backend -> SomeBackendTrace + SomeBackendTrace :: + (Show (Trace backend), Typeable backend) => Trace backend -> SomeBackendTrace instance Show SomeBackendTrace where show (SomeBackendTrace tr) = show tr diff --git a/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Storage/LedgerDB/V2/InMemory.hs b/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Storage/LedgerDB/V2/InMemory.hs index ffcfc064fe..906937b457 100644 --- a/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Storage/LedgerDB/V2/InMemory.hs +++ b/ouroboros-consensus/src/ouroboros-consensus/Ouroboros/Consensus/Storage/LedgerDB/V2/InMemory.hs @@ -18,6 +18,7 @@ module Ouroboros.Consensus.Storage.LedgerDB.V2.InMemory ( Backend (..) , Args (InMemArgs) + , Trace (NoTrace) , Mem , YieldArgs (YieldInMemory) , SinkArgs (SinkInMemory) @@ -363,7 +364,7 @@ instance data Args m Mem = InMemArgs newtype Resources m Mem = Resources (SomeHasFS m) deriving newtype NoThunks - newtype Trace m Mem = NoTrace Void + newtype Trace Mem = NoTrace Void deriving newtype Show mkResources _ _ _ _ = pure . Resources From 5cf2ef2dc6449e3b9e7e5759d1f7c91cafe3c0aa Mon Sep 17 00:00:00 2001 From: Javier Sagredo Date: Tue, 14 Oct 2025 17:01:19 +0200 Subject: [PATCH 08/10] `srnLedgerDbBackendArgs` gets and returns a StdGen --- .../Ouroboros/Consensus/Node.hs | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/ouroboros-consensus-diffusion/src/ouroboros-consensus-diffusion/Ouroboros/Consensus/Node.hs b/ouroboros-consensus-diffusion/src/ouroboros-consensus-diffusion/Ouroboros/Consensus/Node.hs index 5bd6b825f0..44123d6f3a 100644 --- a/ouroboros-consensus-diffusion/src/ouroboros-consensus-diffusion/Ouroboros/Consensus/Node.hs +++ b/ouroboros-consensus-diffusion/src/ouroboros-consensus-diffusion/Ouroboros/Consensus/Node.hs @@ -43,6 +43,8 @@ module Ouroboros.Consensus.Node , LowLevelRunNodeArgs (..) , MempoolCapacityBytesOverride (..) , NodeDatabasePaths (..) + , immutableDbPath + , nonImmutableDbPath , NodeKernel (..) , NodeKernelArgs (..) , ProtocolInfo (..) @@ -376,7 +378,10 @@ data , -- Ad hoc values to replace default ChainDB configurations srnSnapshotPolicyArgs :: SnapshotPolicyArgs , srnQueryBatchSize :: QueryBatchSize - , srnLedgerDbBackendArgs :: LedgerDbBackendArgs m blk + , srnLedgerDbBackendArgs :: (StdGen -> (LedgerDbBackendArgs m blk, StdGen)) + -- ^ The 'StdGen' will be used to initialize the salt for the LSM backend. It + -- is expected that it is the same 'StdGen' that is passed elsewhere in + -- Consensus, i.e. 'llrnRng'. } {------------------------------------------------------------------------------- @@ -1005,7 +1010,7 @@ stdLowLevelRunNodeArgsIO } $(SafeWildCards.fields 'StdRunNodeArgs) = do llrnBfcSalt <- stdBfcSaltIO - llrnRng <- newStdGen + (ldbBackendArgs, llrnRng) <- srnLedgerDbBackendArgs <$> newStdGen pure LowLevelRunNodeArgs { llrnBfcSalt @@ -1050,7 +1055,7 @@ stdLowLevelRunNodeArgsIO InFutureCheck.defaultClockSkew , llrnPublicPeerSelectionStateVar = Diffusion.dcPublicPeerSelectionVar srnDiffusionConfiguration - , llrnLdbFlavorArgs = srnLedgerDbBackendArgs + , llrnLdbFlavorArgs = ldbBackendArgs } where networkMagic :: NetworkMagic From bf57a704fc7f570d5b45222dfb06329af4a41699 Mon Sep 17 00:00:00 2001 From: Javier Sagredo Date: Thu, 16 Oct 2025 11:40:27 +0200 Subject: [PATCH 09/10] Create lsm path on startup --- .../Ouroboros/Consensus/Storage/LedgerDB/V2/LSM.hs | 1 + 1 file changed, 1 insertion(+) diff --git a/ouroboros-consensus/src/ouroboros-consensus-lsm/Ouroboros/Consensus/Storage/LedgerDB/V2/LSM.hs b/ouroboros-consensus/src/ouroboros-consensus-lsm/Ouroboros/Consensus/Storage/LedgerDB/V2/LSM.hs index 4f0c97c333..cc44c81736 100644 --- a/ouroboros-consensus/src/ouroboros-consensus-lsm/Ouroboros/Consensus/Storage/LedgerDB/V2/LSM.hs +++ b/ouroboros-consensus/src/ouroboros-consensus-lsm/Ouroboros/Consensus/Storage/LedgerDB/V2/LSM.hs @@ -579,6 +579,7 @@ instance mkResources _ trcr (LSMArgs path salt mkFS) reg _ = do (rk1, SomeHasFSAndBlockIO fs blockio) <- mkFS reg + createDirectoryIfMissing fs True path session <- allocate reg From f12ecde125c1fb905e21e3f9e6a660fbe4d3b387 Mon Sep 17 00:00:00 2001 From: Javier Sagredo Date: Thu, 23 Oct 2025 15:47:22 +0200 Subject: [PATCH 10/10] Add changelogs --- ...3_154349_javier.sagredo_expose_for_node.md | 23 +++++++++++++++++++ ...3_154344_javier.sagredo_expose_for_node.md | 18 +++++++++++++++ 2 files changed, 41 insertions(+) create mode 100644 ouroboros-consensus-diffusion/changelog.d/20251023_154349_javier.sagredo_expose_for_node.md create mode 100644 ouroboros-consensus/changelog.d/20251023_154344_javier.sagredo_expose_for_node.md diff --git a/ouroboros-consensus-diffusion/changelog.d/20251023_154349_javier.sagredo_expose_for_node.md b/ouroboros-consensus-diffusion/changelog.d/20251023_154349_javier.sagredo_expose_for_node.md new file mode 100644 index 0000000000..a0b5dd2938 --- /dev/null +++ b/ouroboros-consensus-diffusion/changelog.d/20251023_154349_javier.sagredo_expose_for_node.md @@ -0,0 +1,23 @@ + + + + + +### Breaking + +- `srnLedgerDbBackendArgs` now receives and returns a `StdGen` argument. diff --git a/ouroboros-consensus/changelog.d/20251023_154344_javier.sagredo_expose_for_node.md b/ouroboros-consensus/changelog.d/20251023_154344_javier.sagredo_expose_for_node.md new file mode 100644 index 0000000000..784b5da589 --- /dev/null +++ b/ouroboros-consensus/changelog.d/20251023_154344_javier.sagredo_expose_for_node.md @@ -0,0 +1,18 @@ + + +### Patch + +- LSM-trees database directory is now created on startup. + +### Non-Breaking + +- Expose `Ouroboros.Consensus.Storage.LedgerDB.(V1.BackingStore|V2).Backend(Trace)` constructors. + +### Breaking + +- `Ouroboros.Consensus.Storage.LedgerDB.(V1.BackingStore|V2).Backend(Trace)` no longer depends on the running monad `m`.