diff --git a/src/Database/LSMTree/Internal.hs b/src/Database/LSMTree/Internal.hs index 49a7c1900..3d020b714 100644 --- a/src/Database/LSMTree/Internal.hs +++ b/src/Database/LSMTree/Internal.hs @@ -1088,8 +1088,9 @@ createSnapshot resolve snap label tableType t = do traceWith (tableTracer t) $ TraceSnapshot snap withOpenTable t $ \thEnv -> withTempRegistry $ \reg -> do -- TODO: use the temp registry for all side effects - let hfs = tableHasFS thEnv - let hbio = tableHasBlockIO thEnv + let hfs = tableHasFS thEnv + hbio = tableHasBlockIO thEnv + uc = tableSessionUniqCounter thEnv -- Guard that the snapshot does not exist already let snapDir = Paths.namedSnapshotDir (tableSessionRoot thEnv) snap @@ -1105,10 +1106,11 @@ createSnapshot resolve snap label tableType t = do -- Get the table content. content <- RW.withReadAccess (tableContent thEnv) pure - -- Write the write buffer. - snapWriteBuffer <- uniqueToRunNumber <$> incrUniqCounter (tableSessionUniqCounter thEnv) - let wbPaths = Paths.WriteBufferFsPaths (Paths.getNamedSnapshotDir snapDir) snapWriteBuffer - WBW.writeWriteBuffer hfs hbio wbPaths (tableWriteBuffer content) (tableWriteBufferBlobs content) + -- Snapshot the write buffer. + let activeDir = Paths.activeDir (tableSessionRoot thEnv) + let wb = tableWriteBuffer content + let wbb = tableWriteBufferBlobs content + snapWriteBufferNumber <- Paths.writeBufferNumber <$> snapshotWriteBuffer hfs hbio uc activeDir snapDir wb wbb -- Convert to snapshot format snapLevels <- toSnapLevels (tableLevels content) @@ -1116,7 +1118,7 @@ createSnapshot resolve snap label tableType t = do -- Hard link runs into the named snapshot directory snapLevels' <- snapshotRuns reg snapDir snapLevels - let snapMetaData = SnapshotMetaData label tableType (tableConfig t) snapWriteBuffer snapLevels' + let snapMetaData = SnapshotMetaData label tableType (tableConfig t) snapWriteBufferNumber snapLevels' SnapshotMetaDataFile contentPath = Paths.snapshotMetaDataFile snapDir SnapshotMetaDataChecksumFile checksumPath = Paths.snapshotMetaDataChecksumFile snapDir writeFileSnapshotMetaData hfs contentPath checksumPath snapMetaData @@ -1145,6 +1147,7 @@ openSnapshot sesh label tableType override snap resolve = do withTempRegistry $ \reg -> do let hfs = sessionHasFS seshEnv hbio = sessionHasBlockIO seshEnv + uc = sessionUniqCounter seshEnv -- Guard that the snapshot exists let snapDir = Paths.namedSnapshotDir (sessionRoot seshEnv) snap @@ -1167,22 +1170,20 @@ openSnapshot sesh label tableType override snap resolve = do let conf' = applyOverride override conf am <- newArenaManager - blobpath <- Paths.tableBlobPath (sessionRoot seshEnv) <$> - incrUniqCounter (sessionUniqCounter seshEnv) - tableWriteBufferBlobs <- allocateTemp reg (WBB.new hfs blobpath) - releaseRef - let actDir = Paths.activeDir (sessionRoot seshEnv) + let activeDir = Paths.activeDir (sessionRoot seshEnv) -- Read write buffer - let wbPaths = Paths.WriteBufferFsPaths (Paths.getNamedSnapshotDir snapDir) snapWriteBuffer - tableWriteBuffer <- readWriteBuffer hfs hbio resolve tableWriteBufferBlobs wbPaths + activeWriteBufferNumber <- uniqueToRunNumber <$> incrUniqCounter uc + let activeWriteBufferPaths = Paths.WriteBufferFsPaths (Paths.getActiveDir activeDir) activeWriteBufferNumber + let snapWriteBufferPaths = Paths.WriteBufferFsPaths (Paths.getNamedSnapshotDir snapDir) snapWriteBuffer + (tableWriteBuffer, tableWriteBufferBlobs) <- openWriteBuffer reg resolve hfs hbio snapWriteBufferPaths activeWriteBufferPaths -- Hard link runs into the active directory, - snapLevels' <- openRuns reg hfs hbio conf (sessionUniqCounter seshEnv) snapDir actDir snapLevels + snapLevels' <- openRuns reg hfs hbio conf (sessionUniqCounter seshEnv) snapDir activeDir snapLevels -- Convert from the snapshot format, restoring merge progress in the process - tableLevels <- fromSnapLevels reg hfs hbio conf (sessionUniqCounter seshEnv) resolve actDir snapLevels' + tableLevels <- fromSnapLevels reg hfs hbio conf (sessionUniqCounter seshEnv) resolve activeDir snapLevels' tableCache <- mkLevelsCache reg tableLevels newWith reg sesh seshEnv conf' am $! TableContent { diff --git a/src/Database/LSMTree/Internal/BlobFile.hs b/src/Database/LSMTree/Internal/BlobFile.hs index 2ccccf599..280c2eacd 100644 --- a/src/Database/LSMTree/Internal/BlobFile.hs +++ b/src/Database/LSMTree/Internal/BlobFile.hs @@ -20,6 +20,7 @@ import Database.LSMTree.Internal.Serialise (SerialisedBlob (..)) import qualified System.FS.API as FS import System.FS.API (HasFS) import qualified System.FS.BlockIO.API as FS +import System.FS.CallStack (HasCallStack) -- | A handle to a file containing blobs. -- @@ -51,9 +52,10 @@ instance NFData BlobSpan where -- | Open the given file to make a 'BlobFile'. The finaliser will close and -- delete the file. -{-# SPECIALISE openBlobFile :: HasFS IO h -> FS.FsPath -> FS.OpenMode -> IO (Ref (BlobFile IO h)) #-} +{-# SPECIALISE openBlobFile :: HasCallStack => HasFS IO h -> FS.FsPath -> FS.OpenMode -> IO (Ref (BlobFile IO h)) #-} openBlobFile :: PrimMonad m + => HasCallStack => HasFS m h -> FS.FsPath -> FS.OpenMode diff --git a/src/Database/LSMTree/Internal/Snapshot.hs b/src/Database/LSMTree/Internal/Snapshot.hs index c2c393246..19f73d234 100644 --- a/src/Database/LSMTree/Internal/Snapshot.hs +++ b/src/Database/LSMTree/Internal/Snapshot.hs @@ -12,6 +12,9 @@ module Database.LSMTree.Internal.Snapshot ( , SpentCredits (..) -- * Conversion to levels snapshot format , toSnapLevels + -- * Write buffer + , snapshotWriteBuffer + , openWriteBuffer -- * Runs , snapshotRuns , openRuns @@ -42,7 +45,7 @@ import qualified Database.LSMTree.Internal.Merge as Merge import Database.LSMTree.Internal.MergeSchedule import Database.LSMTree.Internal.Paths (ActiveDir (..), NamedSnapshotDir (..), RunFsPaths (..), pathsForRunFiles, - runChecksumsPath) + runChecksumsPath, WriteBufferFsPaths (..), writeBufferKOpsPath, writeBufferBlobPath, writeBufferChecksumsPath) import Database.LSMTree.Internal.Run (Run) import qualified Database.LSMTree.Internal.Run as Run import Database.LSMTree.Internal.RunNumber @@ -52,6 +55,11 @@ import qualified System.FS.API as FS import System.FS.API (HasFS) import qualified System.FS.API.Lazy as FS import System.FS.BlockIO.API (HasBlockIO) +import Database.LSMTree.Internal.WriteBuffer (WriteBuffer) +import Database.LSMTree.Internal.WriteBufferBlobs (WriteBufferBlobs) +import qualified Database.LSMTree.Internal.WriteBufferBlobs as WBB +import qualified Database.LSMTree.Internal.WriteBufferWriter as WBW +import qualified Database.LSMTree.Internal.WriteBufferReader as WBR {------------------------------------------------------------------------------- Snapshot metadata @@ -186,6 +194,72 @@ toSnapMergingRunState (OngoingMerge rs (SpentCreditsVar spentCreditsVar) m) = do spentCredits <- readPrimVar spentCreditsVar pure (SnapOngoingMerge rs (SpentCredits spentCredits) (Merge.mergeLevel m)) +{------------------------------------------------------------------------------- + Write Buffer +-------------------------------------------------------------------------------} + +{-# SPECIALISE + snapshotWriteBuffer :: + HasFS IO h + -> HasBlockIO IO h + -> UniqCounter IO + -> ActiveDir + -> NamedSnapshotDir + -> WriteBuffer + -> Ref (WriteBufferBlobs IO h) + -> IO WriteBufferFsPaths + #-} +snapshotWriteBuffer :: + (MonadMVar m, MonadSTM m, MonadST m, MonadThrow m) + => HasFS m h + -> HasBlockIO m h + -> UniqCounter m + -> ActiveDir + -> NamedSnapshotDir + -> WriteBuffer + -> Ref (WriteBufferBlobs m h) + -> m WriteBufferFsPaths +snapshotWriteBuffer hfs hbio uc activeDir snapDir wb wbb = do + -- Write the write buffer and write buffer blobs to the active directory. + activeWriteBufferNumber <- uniqueToRunNumber <$> incrUniqCounter uc + let activeWriteBufferPaths = WriteBufferFsPaths (getActiveDir activeDir) activeWriteBufferNumber + WBW.writeWriteBuffer hfs hbio activeWriteBufferPaths wb wbb + -- Hard link the write buffer and write buffer blobs to the snapshot directory. + snapWriteBufferNumber <- uniqueToRunNumber <$> incrUniqCounter uc + let snapWriteBufferPaths = WriteBufferFsPaths (getNamedSnapshotDir snapDir) snapWriteBufferNumber + hardLink hfs hbio (writeBufferKOpsPath activeWriteBufferPaths) (writeBufferKOpsPath snapWriteBufferPaths) + hardLink hfs hbio (writeBufferBlobPath activeWriteBufferPaths) (writeBufferBlobPath snapWriteBufferPaths) + hardLink hfs hbio (writeBufferChecksumsPath activeWriteBufferPaths) (writeBufferChecksumsPath snapWriteBufferPaths) + pure snapWriteBufferPaths + +{-# SPECIALISE + openWriteBuffer :: + TempRegistry IO + -> ResolveSerialisedValue + -> HasFS IO h + -> HasBlockIO IO h + -> WriteBufferFsPaths + -> WriteBufferFsPaths + -> IO (WriteBuffer, Ref (WriteBufferBlobs IO h)) + #-} +openWriteBuffer :: + (MonadMVar m, MonadMask m, MonadSTM m, MonadST m) + => TempRegistry m + -> ResolveSerialisedValue + -> HasFS m h + -> HasBlockIO m h + -> WriteBufferFsPaths + -> WriteBufferFsPaths + -> m (WriteBuffer, Ref (WriteBufferBlobs m h)) +openWriteBuffer reg resolve hfs hbio snapWriteBufferPaths activeWriteBufferPaths = do + -- Hard link the write buffer keyops and checksum files to the snapshot directory. + hardLink hfs hbio (writeBufferKOpsPath snapWriteBufferPaths) (writeBufferKOpsPath activeWriteBufferPaths) + hardLink hfs hbio (writeBufferChecksumsPath snapWriteBufferPaths) (writeBufferChecksumsPath activeWriteBufferPaths) + -- Copy the write buffer blobs file to the snapshot directory. + copyFile hfs hbio (writeBufferBlobPath snapWriteBufferPaths) (writeBufferBlobPath activeWriteBufferPaths) + -- Read write buffer + WBR.readWriteBuffer reg resolve hfs hbio activeWriteBufferPaths + {------------------------------------------------------------------------------- Runs -------------------------------------------------------------------------------} @@ -396,3 +470,22 @@ hardLink hfs _hbio sourcePath targetPath = -- not, it is only a temporary placeholder hack. bs <- FS.hGetAll hfs sourceHandle void $ FS.hPutAll hfs targetHandle bs + + +{------------------------------------------------------------------------------- + Copy file +-------------------------------------------------------------------------------} + +{-# SPECIALISE copyFile :: + HasFS IO h + -> HasBlockIO IO h + -> FS.FsPath + -> FS.FsPath + -> IO () #-} +-- | @'copyFile' hfs hbio source target@ copies the @source@ path to the @target@ path. +copyFile :: MonadThrow m => HasFS m h -> HasBlockIO m h -> FS.FsPath -> FS.FsPath -> m () +copyFile hfs _hbio sourcePath targetPath = + FS.withFile hfs sourcePath FS.ReadMode $ \sourceHandle -> + FS.withFile hfs targetPath (FS.WriteMode FS.MustBeNew) $ \targetHandle -> do + bs <- FS.hGetAll hfs sourceHandle + void $ FS.hPutAll hfs targetHandle bs diff --git a/src/Database/LSMTree/Internal/WriteBufferBlobs.hs b/src/Database/LSMTree/Internal/WriteBufferBlobs.hs index d97802038..5166ef5cd 100644 --- a/src/Database/LSMTree/Internal/WriteBufferBlobs.hs +++ b/src/Database/LSMTree/Internal/WriteBufferBlobs.hs @@ -25,6 +25,7 @@ module Database.LSMTree.Internal.WriteBufferBlobs ( WriteBufferBlobs (..), new, + open, addBlob, mkRawBlobRef, mkWeakBlobRef, @@ -117,14 +118,24 @@ instance RefCounted (WriteBufferBlobs m h) where getRefCounter = writeBufRefCounter {-# SPECIALISE new :: HasFS IO h -> FS.FsPath -> IO (Ref (WriteBufferBlobs IO h)) #-} -new :: (PrimMonad m, MonadMask m) - => HasFS m h - -> FS.FsPath - -> m (Ref (WriteBufferBlobs m h)) -new fs blobFileName = do +new :: + (PrimMonad m, MonadMask m) + => HasFS m h + -> FS.FsPath + -> m (Ref (WriteBufferBlobs m h)) +new fs blobFileName = open fs blobFileName FS.MustBeNew + +{-# SPECIALISE new :: HasFS IO h -> FS.FsPath -> IO (Ref (WriteBufferBlobs IO h)) #-} +open :: + (PrimMonad m, MonadMask m) + => HasFS m h + -> FS.FsPath + -> FS.AllowExisting + -> m (Ref (WriteBufferBlobs m h)) +open fs blobFileName blobFileAllowExisting = do -- Must use read/write mode because we write blobs when adding, but -- we can also be asked to retrieve blobs at any time. - blobFile <- openBlobFile fs blobFileName (FS.ReadWriteMode FS.MustBeNew) + blobFile <- openBlobFile fs blobFileName (FS.ReadWriteMode blobFileAllowExisting) blobFilePointer <- newFilePointer newRef (releaseRef blobFile) $ \writeBufRefCounter -> WriteBufferBlobs { diff --git a/src/Database/LSMTree/Internal/WriteBufferReader.hs b/src/Database/LSMTree/Internal/WriteBufferReader.hs index 121415a44..24a97d114 100644 --- a/src/Database/LSMTree/Internal/WriteBufferReader.hs +++ b/src/Database/LSMTree/Internal/WriteBufferReader.hs @@ -4,23 +4,21 @@ module Database.LSMTree.Internal.WriteBufferReader ( readWriteBuffer ) where +import Control.Concurrent.Class.MonadMVar.Strict import Control.Monad.Class.MonadST (MonadST (..)) import Control.Monad.Class.MonadSTM (MonadSTM (..)) import Control.Monad.Class.MonadThrow (MonadCatch (..), MonadMask, MonadThrow (..)) import Control.Monad.Primitive (PrimMonad (..)) import Control.RefCount (Ref, releaseRef) +import Control.TempRegistry import Data.Primitive.MutVar (MutVar, newMutVar, readMutVar, writeMutVar) import Data.Primitive.PrimVar -import qualified Data.Vector as V import Data.Word (Word16) -import Database.LSMTree.Internal.BlobFile (BlobFile, openBlobFile) -import Database.LSMTree.Internal.BlobRef (RawBlobRef (..), - mkRawBlobRef, readRawBlobRef) +import Database.LSMTree.Internal.BlobRef (RawBlobRef (..)) import qualified Database.LSMTree.Internal.Entry as E import Database.LSMTree.Internal.Lookup (ResolveSerialisedValue) -import Database.LSMTree.Internal.MergeSchedule (addWriteBufferEntries) import Database.LSMTree.Internal.Paths import Database.LSMTree.Internal.RawPage import Database.LSMTree.Internal.RunReader (Entry (..), Result (..), @@ -30,42 +28,40 @@ import Database.LSMTree.Internal.Serialise (SerialisedValue) import Database.LSMTree.Internal.WriteBuffer (WriteBuffer) import qualified Database.LSMTree.Internal.WriteBuffer as WB import Database.LSMTree.Internal.WriteBufferBlobs (WriteBufferBlobs) +import qualified Database.LSMTree.Internal.WriteBufferBlobs as WBB import qualified System.FS.API as FS import System.FS.API (HasFS) import qualified System.FS.BlockIO.API as FS import System.FS.BlockIO.API (HasBlockIO) import qualified Database.LSMTree.Internal.Paths as Paths +import Control.Monad (void) {-# SPECIALISE readWriteBuffer :: - HasFS IO h - -> HasBlockIO IO h + TempRegistry IO -> ResolveSerialisedValue - -> Ref (WriteBufferBlobs IO h) + -> HasFS IO h + -> HasBlockIO IO h -> WriteBufferFsPaths - -> IO WriteBuffer + -> IO (WriteBuffer, Ref (WriteBufferBlobs IO h)) #-} readWriteBuffer :: - (MonadMask m, MonadSTM m, MonadST m) - => HasFS m h - -> HasBlockIO m h + (MonadMVar m, MonadMask m, MonadSTM m, MonadST m) + => TempRegistry m -> ResolveSerialisedValue - -> Ref (WriteBufferBlobs m h) + -> HasFS m h + -> HasBlockIO m h -> WriteBufferFsPaths - -> m WriteBuffer -readWriteBuffer hfs hbio f wbb fsPaths = - bracket (new hfs hbio fsPaths) close $ \reader -> do - let readEntry = - E.traverseBlobRef (readRawBlobRef hfs) . toFullEntry - let readEntries = next reader >>= \case - Empty -> pure [] - ReadEntry key entry -> - readEntry entry >>= \entry' -> - ((key, entry') :) <$> readEntries - es <- V.fromList <$> readEntries - -- TODO: We cannot derive the number of entries from the in-memory index. - let maxn = E.NumEntries $ V.length es - fst <$> addWriteBufferEntries hfs f wbb maxn WB.empty es + -> m (WriteBuffer, Ref (WriteBufferBlobs m h)) +readWriteBuffer reg resolve hfs hbio wbPaths = + bracket (new reg hfs hbio wbPaths) close $ \reader@WriteBufferReader{..} -> + (,) <$> readEntries reader <*> pure readerWriteBufferBlobs + where + readEntries reader = readEntriesAcc WB.empty + where + readEntriesAcc acc = next reader >>= \case + Empty -> pure acc + ReadEntry key entry -> readEntriesAcc $ WB.addEntry resolve key (rawBlobRefSpan <$> toFullEntry entry) acc -- | Allows reading the k\/ops of a run incrementally, using its own read-only -- file handle and in-memory cache of the current disk page. @@ -74,34 +70,36 @@ readWriteBuffer hfs hbio f wbb fsPaths = data WriteBufferReader m h = WriteBufferReader { -- | The disk page currently being read. If it is 'Nothing', the reader -- is considered closed. - readerCurrentPage :: !(MutVar (PrimState m) (Maybe RawPage)) + readerCurrentPage :: !(MutVar (PrimState m) (Maybe RawPage)) -- | The index of the entry to be returned by the next call to 'next'. - , readerCurrentEntryNo :: !(PrimVar (PrimState m) Word16) - , readerKOpsHandle :: !(FS.Handle h) - , readerBlobFile :: !(Ref (BlobFile m h)) - , readerHasFS :: !(HasFS m h) - , readerHasBlockIO :: !(HasBlockIO m h) + , readerCurrentEntryNo :: !(PrimVar (PrimState m) Word16) + , readerKOpsHandle :: !(FS.Handle h) + , readerWriteBufferBlobs :: !(Ref (WriteBufferBlobs m h)) + , readerHasFS :: !(HasFS m h) + , readerHasBlockIO :: !(HasBlockIO m h) } {-# SPECIALISE new :: - HasFS IO h + TempRegistry IO + -> HasFS IO h -> HasBlockIO IO h -> WriteBufferFsPaths -> IO (WriteBufferReader IO h) #-} -- | See 'Database.LSMTree.Internal.RunReader.new'. new :: forall m h. - (MonadCatch m, MonadSTM m, MonadST m) - => HasFS m h + (MonadMVar m, MonadSTM m, MonadST m, MonadMask m) + => TempRegistry m + -> HasFS m h -> HasBlockIO m h -> WriteBufferFsPaths -> m (WriteBufferReader m h) -new readerHasFS readerHasBlockIO fsPaths = do +new reg readerHasFS readerHasBlockIO fsPaths = do (readerKOpsHandle :: FS.Handle h) <- FS.hOpen readerHasFS (Paths.writeBufferKOpsPath fsPaths) FS.ReadMode -- Double the file readahead window (only applies to this file descriptor) FS.hAdviseAll readerHasBlockIO readerKOpsHandle FS.AdviceSequential - readerBlobFile <- openBlobFile readerHasFS (Paths.writeBufferBlobPath fsPaths) FS.ReadMode + readerWriteBufferBlobs <- allocateTemp reg (WBB.open readerHasFS (Paths.writeBufferBlobPath fsPaths) FS.AllowExisting) releaseRef -- Load first page from disk, if it exists. readerCurrentEntryNo <- newPrimVar (0 :: Word16) firstPage <- readDiskPage readerHasFS readerKOpsHandle @@ -143,14 +141,14 @@ next WriteBufferReader {..} = do IndexEntry key entry -> do modifyPrimVar readerCurrentEntryNo (+1) let entry' :: E.Entry SerialisedValue (RawBlobRef m h) - entry' = fmap (mkRawBlobRef readerBlobFile) entry + entry' = fmap (WBB.mkRawBlobRef readerWriteBufferBlobs) entry let rawEntry = Entry entry' return (ReadEntry key rawEntry) IndexEntryOverflow key entry lenSuffix -> do -- TODO: we know that we need the next page, could already load? modifyPrimVar readerCurrentEntryNo (+1) let entry' :: E.Entry SerialisedValue (RawBlobRef m h) - entry' = fmap (mkRawBlobRef readerBlobFile) entry + entry' = fmap (WBB.mkRawBlobRef readerWriteBufferBlobs) entry overflowPages <- readOverflowPages readerHasFS readerKOpsHandle lenSuffix let rawEntry = mkEntryOverflow entry' page lenSuffix overflowPages return (ReadEntry key rawEntry) @@ -160,11 +158,10 @@ next WriteBufferReader {..} = do WriteBufferReader IO h -> IO () #-} --- | See 'Database.LSMTree.Internal.RunReader.close'. close :: - (MonadMask m, PrimMonad m) + (PrimMonad m) => WriteBufferReader m h -> m () close WriteBufferReader{..} = do FS.hClose readerHasFS readerKOpsHandle - releaseRef readerBlobFile + diff --git a/test/Test/Database/LSMTree/Internal/Run.hs b/test/Test/Database/LSMTree/Internal/Run.hs index 3589bad98..5438fbb9e 100644 --- a/test/Test/Database/LSMTree/Internal/Run.hs +++ b/test/Test/Database/LSMTree/Internal/Run.hs @@ -238,22 +238,21 @@ prop_WriteAndOpenWriteBuffer :: -> RunData KeyForIndexCompact SerialisedValue SerialisedBlob -> IO Property prop_WriteAndOpenWriteBuffer hfs hbio rd = do - -- Serialise run data as write buffer: - let srd = serialiseRunData rd - let inPaths = WrapRunFsPaths $ simplePath 1111 - let f (SerialisedValue x) (SerialisedValue y) = SerialisedValue (x <> y) - withRunDataAsWriteBuffer hfs f inPaths srd $ \wb wbb -> do - -- Write write buffer to disk: - let wbPaths = WrapRunFsPaths $ simplePath 1312 - withSerialisedWriteBuffer hfs hbio wbPaths wb wbb $ do - -- Read write buffer from disk: - let outPaths = WrapRunFsPaths $ simplePath 2222 - let outBlobPath = Paths.writeBufferBlobPath outPaths - bracket (WBB.new hfs outBlobPath) releaseRef $ \wbb' -> do - wb' <- readWriteBuffer hfs hbio f wbb' wbPaths + withTempRegistry $ \reg -> do + -- Serialise run data as write buffer: + let srd = serialiseRunData rd + let inPaths = WrapRunFsPaths $ simplePath 1111 + let resolve (SerialisedValue x) (SerialisedValue y) = SerialisedValue (x <> y) + withRunDataAsWriteBuffer hfs resolve inPaths srd $ \wb wbb -> do + -- Write write buffer to disk: + let wbPaths = WrapRunFsPaths $ simplePath 1312 + withSerialisedWriteBuffer hfs hbio wbPaths wb wbb $ do + -- Read write buffer from disk: + (wb', wbb') <- readWriteBuffer reg resolve hfs hbio wbPaths assertEqual "k/ops" wb wb' - -- TODO: return a proper Property instead of using assertEqual etc. - return (property True) + releaseRef wbb' + -- TODO: return a proper Property instead of using assertEqual etc. + return (property True) -- | Writing run data to the disk via 'writeWriteBuffer' gives the same key/ops -- and blob files as when written out as a run.