From af0beabee697eec45409e69177b99183428c4c67 Mon Sep 17 00:00:00 2001 From: Duncan Coutts Date: Tue, 24 Sep 2024 13:12:46 +0100 Subject: [PATCH] Refactor: push write buffer lookups into lookupsIO Previously lookupsIO dealt only with lookups in runs, and combining results from the write buffer was done in D.L.Internal. Originally that was ok because combining the write buffer results was simple, but it has grown more complex with the blobs feature being properly enabled. So now we pass it down into lookupsIO, specifically into intraPageLookups where all the results are combined. It works nicely here, since per-key results are accumulated into a mutable vector, so the only change is to initialise that array with the write buffer lookups. All the lookup results from runs are then accumulated on top. This should also mildly reduce allocations. In addition, instead of D.L.Internal.lookups getting passed in a function for converting into the final result, just return the intermediate representation and have the final conversion done in the API wrappers. This approach is more friendly for inlining and specialisation (so the deserialisation can be specialised at the call site). The benchmarks are updated but use an empty write buffer. Plausibly we might want some micro benchmarks for lookupIO et al that do make use of the write buffer too. --- bench/macro/lsm-tree-bench-lookups.hs | 28 ++++-- .../Bench/Database/LSMTree/Internal/Lookup.hs | 30 ++++-- src/Database/LSMTree/Internal.hs | 50 ++++------ src/Database/LSMTree/Internal/Lookup.hs | 49 ++++++++-- .../LSMTree/Internal/WriteBufferBlobs.hs | 5 + src/Database/LSMTree/Monoidal.hs | 18 ++-- src/Database/LSMTree/Normal.hs | 20 ++-- test/Test/Database/LSMTree/Internal.hs | 4 +- test/Test/Database/LSMTree/Internal/Lookup.hs | 95 +++++++++++++------ 9 files changed, 190 insertions(+), 109 deletions(-) diff --git a/bench/macro/lsm-tree-bench-lookups.hs b/bench/macro/lsm-tree-bench-lookups.hs index 35a82cd8c..70b1496c3 100644 --- a/bench/macro/lsm-tree-bench-lookups.hs +++ b/bench/macro/lsm-tree-bench-lookups.hs @@ -35,6 +35,8 @@ import qualified Database.LSMTree.Internal.RunBuilder as RunBuilder import Database.LSMTree.Internal.RunNumber import Database.LSMTree.Internal.Serialise (SerialisedKey, serialiseKey, serialiseValue) +import qualified Database.LSMTree.Internal.WriteBuffer as WB +import qualified Database.LSMTree.Internal.WriteBufferBlobs as WBB import Debug.Trace (traceMarkerIO) import GHC.Stats import Numeric @@ -197,8 +199,15 @@ benchmarks !caching = withFS $ \hfs hbio -> do _blookupsIO <- benchmark "benchLookupsIO" "Calculate batches of keys, and perform disk lookups for each batch. This is roughly doing the same as benchPrepLookups, but also performing the disk I/O and resolving values. Net time/allocation is the result of subtracting the cost of benchGenKeyBatches." - (benchLookupsIO hbio arenaManager benchmarkResolveSerialisedValue runs blooms indexes handles keyRng0) benchmarkNumLookups + (\n -> do + let wb_unused = WB.empty + wbblobs_unused <- WBB.new hfs (FS.mkFsPath []) + benchLookupsIO hbio arenaManager benchmarkResolveSerialisedValue + wb_unused wbblobs_unused runs blooms indexes handles + keyRng0 n) + benchmarkNumLookups bgenKeyBatches + --TODO: consider adding benchmarks that also use the write buffer traceMarkerIO "Cleaning up" putStrLn "Cleaning up" @@ -454,6 +463,8 @@ benchLookupsIO :: FS.HasBlockIO IO h -> ArenaManager RealWorld -> ResolveSerialisedValue + -> WB.WriteBuffer + -> WBB.WriteBufferBlobs IO h -> V.Vector (Run IO (FS.Handle h)) -> V.Vector (Bloom SerialisedKey) -> V.Vector IndexCompact @@ -461,12 +472,15 @@ benchLookupsIO :: -> StdGen -> Int -> IO () -benchLookupsIO !hbio !arenaManager !resolve !rs !bs !ics !hs !keyRng !n - | n <= 0 = pure () - | otherwise = do - let (!ks, !keyRng') = genLookupBatch keyRng benchmarkGenBatchSize - !_ <- lookupsIO hbio arenaManager resolve rs bs ics hs ks - benchLookupsIO hbio arenaManager resolve rs bs ics hs keyRng' (n-benchmarkGenBatchSize) +benchLookupsIO !hbio !arenaManager !resolve !wb !wbblobs !rs !bs !ics !hs = + go + where + go !keyRng !n + | n <= 0 = pure () + | otherwise = do + let (!ks, !keyRng') = genLookupBatch keyRng benchmarkGenBatchSize + !_ <- lookupsIO hbio arenaManager resolve wb wbblobs rs bs ics hs ks + go keyRng' (n-benchmarkGenBatchSize) {------------------------------------------------------------------------------- Utilities diff --git a/bench/micro/Bench/Database/LSMTree/Internal/Lookup.hs b/bench/micro/Bench/Database/LSMTree/Internal/Lookup.hs index 9b100b744..c3bce1ab0 100644 --- a/bench/micro/Bench/Database/LSMTree/Internal/Lookup.hs +++ b/bench/micro/Bench/Database/LSMTree/Internal/Lookup.hs @@ -82,7 +82,7 @@ benchmarks = bgroup "Bench.Database.LSMTree.Internal.Lookup" [ benchLookups :: Config -> Benchmark benchLookups conf@Config{name} = - withEnv $ \ ~(_dir, arenaManager, _hasFS, hasBlockIO, rs, ks) -> + withEnv $ \ ~(_dir, arenaManager, hasFS, hasBlockIO, rs, ks) -> env ( pure ( V.map Run.runFilter rs , V.map Run.runIndex rs , V.map Run.runKOpsFile rs @@ -122,21 +122,31 @@ benchLookups conf@Config{name} = -- only compute WHNF. , bench "Perform intra-page lookups" $ perRunEnvWithCleanup - ( newArena arenaManager >>= \arena -> - stToIO (prepLookups arena blooms indexes kopsFiles ks) >>= \(rkixs, ioops) -> - FS.submitIO hasBlockIO ioops >>= \ioress -> - pure (rkixs, ioops, ioress, arena) + ( do arena <- newArena arenaManager + (rkixs, ioops) <- stToIO (prepLookups arena blooms indexes kopsFiles ks) + ioress <- FS.submitIO hasBlockIO ioops + wbblobs <- WBB.new hasFS (FS.mkFsPath []) + pure (rkixs, ioops, ioress, arena, wbblobs) ) - (\(_, _, _, arena) -> closeArena arenaManager arena) $ \ ~(rkixs, ioops, ioress, _) -> do - !_ <- intraPageLookups resolveV rs ks rkixs ioops ioress - pure () + (\(_, _, _, arena, wbblobs) -> do + closeArena arenaManager arena + WBB.removeReference wbblobs) + (\ ~(rkixs, ioops, ioress, _, wbblobs_unused) -> do + !_ <- intraPageLookups resolveV WB.empty wbblobs_unused + rs ks rkixs ioops ioress + pure ()) -- The whole shebang: lookup preparation, doing the IO, and then -- performing intra-page-lookups. Again, we evaluate the result to -- WHNF because it is the same result that intraPageLookups produces -- (see above). - , bench "Lookups in IO" $ - whnfAppIO (\ks' -> lookupsIO hasBlockIO arenaManager resolveV rs blooms indexes kopsFiles ks') ks + , let wb_unused = WB.empty in + env (WBB.new hasFS (FS.mkFsPath [])) $ \wbblobs_unused -> + bench "Lookups in IO" $ + whnfAppIO (\ks' -> lookupsIO hasBlockIO arenaManager resolveV + wb_unused wbblobs_unused + rs blooms indexes kopsFiles ks') ks ] + --TODO: consider adding benchmarks that also use the write buffer where withEnv = envWithCleanup (lookupsInBatchesEnv conf) diff --git a/src/Database/LSMTree/Internal.hs b/src/Database/LSMTree/Internal.hs index cf4a84a19..9ec977f71 100644 --- a/src/Database/LSMTree/Internal.hs +++ b/src/Database/LSMTree/Internal.hs @@ -664,46 +664,30 @@ close th = do pure tc pure TableHandleClosed -{-# SPECIALISE lookups :: ResolveSerialisedValue -> V.Vector SerialisedKey -> TableHandle IO h -> (Maybe (Entry SerialisedValue (WeakBlobRef IO (Handle h))) -> lookupResult) -> IO (V.Vector lookupResult) #-} +{-# SPECIALISE lookups :: ResolveSerialisedValue -> V.Vector SerialisedKey -> TableHandle IO h -> IO (V.Vector (Maybe (Entry SerialisedValue (WeakBlobRef IO (Handle h))))) #-} -- | See 'Database.LSMTree.Normal.lookups'. lookups :: m ~ IO -- TODO: replace by @io-classes@ constraints for IO simulation. => ResolveSerialisedValue -> V.Vector SerialisedKey -> TableHandle m h - -> (Maybe (Entry SerialisedValue (WeakBlobRef m (Handle h))) -> lookupResult) - -- ^ How to map from an entry to a lookup result. - -> m (V.Vector lookupResult) -lookups resolve ks th fromEntry = do + -> m (V.Vector (Maybe (Entry SerialisedValue (WeakBlobRef m (Handle h))))) +lookups resolve ks th = do traceWith (tableTracer th) $ TraceLookups (V.length ks) - withOpenTable th $ \thEnv -> do - let arenaManager = tableHandleArenaManager th - RW.withReadAccess (tableContent thEnv) $ \tableContent -> do - let !cache = tableCache tableContent - ioRes <- - lookupsIO - (tableHasBlockIO thEnv) - arenaManager - resolve - (cachedRuns cache) - (cachedFilters cache) - (cachedIndexes cache) - (cachedKOpsFiles cache) - ks - --TODO: this bit is all a bit of a mess, not well factored - --TODO: incorporate write buffer lookups into the lookupsIO - --TODO: reduce allocations involved with converting BlobSpan to BlobRef - -- and Entry to the lookup result. Try one single conversion rather - -- than multiple steps that each allocate. - let !wb = tableWriteBuffer tableContent - !wbblobs = tableWriteBufferBlobs tableContent - toBlobRef <- WBB.mkBlobRef wbblobs - let wbLookup = fmap (fmap (WeakBlobRef . toBlobRef)) - . WB.lookup wb - pure $! - V.zipWithStrict - (\k1 e2 -> fromEntry $ Entry.combineMaybe resolve (wbLookup k1) e2) - ks ioRes + withOpenTable th $ \thEnv -> + RW.withReadAccess (tableContent thEnv) $ \tableContent -> + let !cache = tableCache tableContent in + lookupsIO + (tableHasBlockIO thEnv) + (tableHandleArenaManager th) + resolve + (tableWriteBuffer tableContent) + (tableWriteBufferBlobs tableContent) + (cachedRuns cache) + (cachedFilters cache) + (cachedIndexes cache) + (cachedKOpsFiles cache) + ks {-# SPECIALISE rangeLookup :: ResolveSerialisedValue -> Range SerialisedKey -> TableHandle IO h -> (SerialisedKey -> SerialisedValue -> Maybe (WeakBlobRef IO (Handle h)) -> res) -> IO (V.Vector res) #-} -- | See 'Database.LSMTree.Normal.rangeLookup'. diff --git a/src/Database/LSMTree/Internal/Lookup.hs b/src/Database/LSMTree/Internal/Lookup.hs index f9807c6f7..59ea36184 100644 --- a/src/Database/LSMTree/Internal/Lookup.hs +++ b/src/Database/LSMTree/Internal/Lookup.hs @@ -51,6 +51,8 @@ import Database.LSMTree.Internal.RawPage import Database.LSMTree.Internal.Run (Run, mkBlobRefForRun) import Database.LSMTree.Internal.Serialise import qualified Database.LSMTree.Internal.Vector as V +import qualified Database.LSMTree.Internal.WriteBuffer as WB +import qualified Database.LSMTree.Internal.WriteBufferBlobs as WBB import System.FS.API (BufferOffset (..), Handle) import System.FS.BlockIO.API @@ -217,6 +219,8 @@ data ByteCountDiscrepancy = ByteCountDiscrepancy { HasBlockIO IO h -> ArenaManager RealWorld -> ResolveSerialisedValue + -> WB.WriteBuffer + -> WBB.WriteBufferBlobs IO h -> V.Vector (Run IO (Handle h)) -> V.Vector (Bloom SerialisedKey) -> V.Vector IndexCompact @@ -235,16 +239,20 @@ lookupsIO :: => HasBlockIO m h -> ArenaManager (PrimState m) -> ResolveSerialisedValue + -> WB.WriteBuffer + -> WBB.WriteBufferBlobs m h -> V.Vector (Run m (Handle h)) -- ^ Runs @rs@ -> V.Vector (Bloom SerialisedKey) -- ^ The bloom filters inside @rs@ -> V.Vector IndexCompact -- ^ The indexes inside @rs@ -> V.Vector (Handle h) -- ^ The file handles to the key\/value files inside @rs@ -> V.Vector SerialisedKey -> m (V.Vector (Maybe (Entry SerialisedValue (WeakBlobRef m (Handle h))))) -lookupsIO !hbio !mgr !resolveV !rs !blooms !indexes !kopsFiles !ks = assert precondition $ withArena mgr $ \arena -> do - (rkixs, ioops) <- Class.stToIO $ prepLookups arena blooms indexes kopsFiles ks - ioress <- submitIO hbio ioops - intraPageLookups resolveV rs ks rkixs ioops ioress +lookupsIO !hbio !mgr !resolveV !wb !wbblobs !rs !blooms !indexes !kopsFiles !ks = + assert precondition $ + withArena mgr $ \arena -> do + (rkixs, ioops) <- Class.stToIO $ prepLookups arena blooms indexes kopsFiles ks + ioress <- submitIO hbio ioops + intraPageLookups resolveV wb wbblobs rs ks rkixs ioops ioress where -- we check only that the lengths match, because checking the contents is -- too expensive. @@ -256,6 +264,8 @@ lookupsIO !hbio !mgr !resolveV !rs !blooms !indexes !kopsFiles !ks = assert prec {-# SPECIALIZE intraPageLookups :: ResolveSerialisedValue + -> WB.WriteBuffer + -> WBB.WriteBufferBlobs IO h -> V.Vector (Run IO (Handle h)) -> V.Vector SerialisedKey -> VU.Vector (RunIx, KeyIx) @@ -263,22 +273,45 @@ lookupsIO !hbio !mgr !resolveV !rs !blooms !indexes !kopsFiles !ks = assert prec -> VU.Vector IOResult -> IO (V.Vector (Maybe (Entry SerialisedValue (WeakBlobRef IO (Handle h))))) #-} --- | Intra-page lookups. +-- | Intra-page lookups, and combining lookup results from multiple runs and +-- the write buffer. -- -- This function assumes that @rkixs@ is ordered such that newer runs are -- handled first. The order matters for resolving cases where we find the same -- key in multiple runs. +-- intraPageLookups :: forall m h. (PrimMonad m, MonadThrow m) => ResolveSerialisedValue + -> WB.WriteBuffer + -> WBB.WriteBufferBlobs m h -> V.Vector (Run m (Handle h)) -> V.Vector SerialisedKey -> VU.Vector (RunIx, KeyIx) -> V.Vector (IOOp (PrimState m) h) -> VU.Vector IOResult -> m (V.Vector (Maybe (Entry SerialisedValue (WeakBlobRef m (Handle h))))) -intraPageLookups !resolveV !rs !ks !rkixs !ioops !ioress = do - res <- VM.replicate (V.length ks) Nothing +intraPageLookups !resolveV !wb !wbblobs !rs !ks !rkixs !ioops !ioress = do + -- We accumulate results into the 'res' vector. When there are several + -- lookup hits for the same key then we combine the results. The combining + -- operator is associative but not commutative, so we must do this in the + -- right order. We start with the write buffer lookup results and then go + -- through the run lookup results in rkixs, which must be ordered by run. + -- + -- TODO: reassess the representation of the result vector to try to reduce + -- intermediate allocations. For example use a less convenient + -- representation with several vectors (e.g. separate blob info) and + -- convert to the final convenient representation in a single pass near + -- the surface API so that all the conversions can be done in one pass + -- without intermediate allocations. + -- + toBlobRef <- WBB.mkBlobRef wbblobs + res <- VM.generateM (V.length ks) $ \ki -> + case WB.lookup wb (V.unsafeIndex ks ki) of + Nothing -> pure Nothing + Just e -> pure $! Just $! fmap (WeakBlobRef . toBlobRef) e + -- TODO: ^^ we should be able to avoid this allocation by + -- combining the conversion with other later conversions. loop res 0 V.unsafeFreeze res where @@ -307,6 +340,8 @@ intraPageLookups !resolveV !rs !ks !rkixs !ioops !ioress = do LookupEntry e -> do let e' = bimap copySerialisedValue (WeakBlobRef . mkBlobRefForRun r) e + -- TODO: ^^ we should be able to avoid this allocation by + -- combining the conversion with other later conversions. V.unsafeInsertWithMStrict res (combine resolveV) kix e' -- Laziness ensures that we only compute the appending of the prefix -- and suffix when the result is needed. We do not use 'force' here, diff --git a/src/Database/LSMTree/Internal/WriteBufferBlobs.hs b/src/Database/LSMTree/Internal/WriteBufferBlobs.hs index 8519b37e9..c78327f10 100644 --- a/src/Database/LSMTree/Internal/WriteBufferBlobs.hs +++ b/src/Database/LSMTree/Internal/WriteBufferBlobs.hs @@ -1,4 +1,5 @@ {-# OPTIONS_GHC -Wno-partial-fields #-} +{- HLINT ignore "Use record patterns" -} -- | An on-disk store for blobs for the write buffer. -- @@ -33,6 +34,7 @@ module Database.LSMTree.Internal.WriteBufferBlobs ( FilePointer (..) ) where +import Control.DeepSeq (NFData (..)) import Control.Monad.Class.MonadThrow import Control.Monad.Primitive (PrimMonad, PrimState) import qualified Control.RefCount as RC @@ -123,6 +125,9 @@ data BlobFileState m h = , blobFilePointer :: !(FilePointer m) } +instance NFData (WriteBufferBlobs m h) where + rnf (WriteBufferBlobs _ b c) = rnf b `seq` rnf c + new :: PrimMonad m => HasFS m h -> FS.FsPath diff --git a/src/Database/LSMTree/Monoidal.hs b/src/Database/LSMTree/Monoidal.hs index c2762178b..ce99eb081 100644 --- a/src/Database/LSMTree/Monoidal.hs +++ b/src/Database/LSMTree/Monoidal.hs @@ -208,6 +208,7 @@ close (Internal.MonoidalTable th) = Internal.close th -------------------------------------------------------------------------------} {-# SPECIALISE lookups :: (SerialiseKey k, SerialiseValue v, ResolveValue v) => V.Vector k -> TableHandle IO k v -> IO (V.Vector (LookupResult v)) #-} +{-# INLINEABLE lookups #-} -- | Perform a batch of lookups. -- -- Lookups can be performed concurrently from multiple Haskell threads. @@ -217,21 +218,18 @@ lookups :: -> TableHandle m k v -> m (V.Vector (LookupResult v)) lookups ks (Internal.MonoidalTable th) = - V.mapStrict (fmap Internal.deserialiseValue) <$!> + V.map toLookupResult <$> Internal.lookups (resolve @v Proxy) (V.map Internal.serialiseKey ks) th - toMonoidalLookupResult - -toMonoidalLookupResult :: Maybe (Entry.Entry v b) -> LookupResult v -toMonoidalLookupResult = \case - Just e -> case e of - Entry.Insert v -> Found v - Entry.InsertWithBlob _ _ -> error "toMonoidalLookupResult: InsertWithBlob unexpected" - Entry.Mupdate v -> Found v + where + toLookupResult (Just e) = case e of + Entry.Insert v -> Found (Internal.deserialiseValue v) + Entry.InsertWithBlob _ _ -> error "Monoidal.lookups: unexpected InsertWithBlob" + Entry.Mupdate v -> Found (Internal.deserialiseValue v) Entry.Delete -> NotFound - Nothing -> NotFound + toLookupResult Nothing = NotFound {-# SPECIALISE rangeLookup :: (SerialiseKey k, SerialiseValue v, ResolveValue v) => Range k -> TableHandle IO k v -> IO (V.Vector (QueryResult k v)) #-} -- | Perform a range lookup. diff --git a/src/Database/LSMTree/Normal.hs b/src/Database/LSMTree/Normal.hs index 874d37a1a..bff506b2d 100644 --- a/src/Database/LSMTree/Normal.hs +++ b/src/Database/LSMTree/Normal.hs @@ -280,6 +280,7 @@ close (Internal.NormalTable th) = Internal.close th -------------------------------------------------------------------------------} {-# SPECIALISE lookups :: (SerialiseKey k, SerialiseValue v) => V.Vector k -> TableHandle IO k v blob -> IO (V.Vector (LookupResult v (BlobRef IO blob))) #-} +{-# INLINEABLE lookups #-} -- | Perform a batch of lookups. -- -- Lookups can be performed concurrently from multiple Haskell threads. @@ -289,17 +290,16 @@ lookups :: -> TableHandle m k v blob -> m (V.Vector (LookupResult v (BlobRef m blob))) lookups ks (Internal.NormalTable th) = - V.mapStrict (bimap Internal.deserialiseValue BlobRef) <$!> - Internal.lookups const (V.map Internal.serialiseKey ks) th toNormalLookupResult - -toNormalLookupResult :: Maybe (Entry.Entry v b) -> LookupResult v b -toNormalLookupResult = \case - Just e -> case e of - Entry.Insert v -> Found v - Entry.InsertWithBlob v br -> FoundWithBlob v br - Entry.Mupdate _ -> error "toNormalLookupResult: Mupdate unexpected" + V.map toLookupResult <$> + Internal.lookups const (V.map Internal.serialiseKey ks) th + where + toLookupResult (Just e) = case e of + Entry.Insert v -> Found (Internal.deserialiseValue v) + Entry.InsertWithBlob v br -> FoundWithBlob (Internal.deserialiseValue v) + (BlobRef br) + Entry.Mupdate _ -> error "Normal.lookups: unexpected Mupdate" Entry.Delete -> NotFound - Nothing -> NotFound + toLookupResult Nothing = NotFound {-# SPECIALISE rangeLookup :: (SerialiseKey k, SerialiseValue v) => Range k -> TableHandle IO k v blob -> IO (V.Vector (QueryResult k v (BlobRef IO blob))) #-} -- | Perform a range lookup. diff --git a/test/Test/Database/LSMTree/Internal.hs b/test/Test/Database/LSMTree/Internal.hs index 4b3b18223..9a70b8c9b 100644 --- a/test/Test/Database/LSMTree/Internal.hs +++ b/test/Test/Database/LSMTree/Internal.hs @@ -171,8 +171,8 @@ prop_interimOpenTable dat = ioProperty $ let snap = fromMaybe (error "invalid name") $ mkSnapshotName "snap" numRunsSnapped <- snapshot const snap "someLabel" th th' <- open sesh "someLabel" configNoOverride snap - lhs <- lookups const ks th id - rhs <- lookups const ks th' id + lhs <- lookups const ks th + rhs <- lookups const ks th' close th close th' -- TODO: checking lookups is a simple check, but we could have stronger diff --git a/test/Test/Database/LSMTree/Internal/Lookup.hs b/test/Test/Database/LSMTree/Internal/Lookup.hs index 7fd4297fd..707070963 100644 --- a/test/Test/Database/LSMTree/Internal/Lookup.hs +++ b/test/Test/Database/LSMTree/Internal/Lookup.hs @@ -43,7 +43,7 @@ import Data.Word import Database.LSMTree.Extras import Database.LSMTree.Extras.Generators import Database.LSMTree.Internal.BlobRef (BlobSpan) -import Database.LSMTree.Internal.Entry +import Database.LSMTree.Internal.Entry as Entry import Database.LSMTree.Internal.IndexCompact as Index import Database.LSMTree.Internal.Lookup import Database.LSMTree.Internal.Paths (RunFsPaths (..)) @@ -55,6 +55,8 @@ import Database.LSMTree.Internal.RunAcc as Run import Database.LSMTree.Internal.RunNumber import Database.LSMTree.Internal.Serialise import Database.LSMTree.Internal.Serialise.Class +import qualified Database.LSMTree.Internal.WriteBuffer as WB +import qualified Database.LSMTree.Internal.WriteBufferBlobs as WBB import GHC.Generics import qualified System.FS.API as FS import System.FS.API (Handle (..), mkFsPath) @@ -288,39 +290,72 @@ prop_inMemRunLookupAndConstruction dat = prop_roundtripFromWriteBufferLookupIO :: SmallList (InMemLookupData SerialisedKey SerialisedValue SerialisedBlob) -> Property -prop_roundtripFromWriteBufferLookupIO dats = - ioProperty $ withTempIOHasBlockIO "prop_roundtripFromWriteBufferLookupIO" $ \hasFS hasBlockIO -> do - (runs, wbs) <- mkRuns hasFS hasBlockIO - let wbAll = Map.unionsWith (combine resolveV) wbs +prop_roundtripFromWriteBufferLookupIO (SmallList dats) = + ioProperty $ + withTempIOHasBlockIO "prop_roundtripFromWriteBufferLookupIO" $ \hfs hbio -> + withRuns hfs hbio dats $ \wb wbblobs runs -> do + let model :: Map SerialisedKey (Entry SerialisedValue SerialisedBlob) + model = Map.unionsWith (Entry.combine resolveV) (map runData dats) + keys = V.fromList [ k | InMemLookupData{lookups} <- dats + , k <- lookups ] + modelres = V.map (\k -> Map.lookup k model) keys arenaManager <- newArenaManager - real <- lookupsIO - hasBlockIO - arenaManager - resolveV - runs - (V.map Run.runFilter runs) - (V.map Run.runIndex runs) - (V.map Run.runKOpsFile runs) - lookupss - let model = V.map (\k -> Map.lookup k wbAll) lookupss - V.mapM_ Run.removeReference runs - FS.close hasBlockIO - -- TODO: we don't compare blobs, because we haven't implemented blob - -- retrieval yet. - - pure $ opaqueifyBlobs model === opaqueifyBlobs real + realres <- + lookupsIO + hbio + arenaManager + resolveV + wb wbblobs + runs + (V.map Run.runFilter runs) + (V.map Run.runIndex runs) + (V.map Run.runKOpsFile runs) + keys + -- TODO: compare blobs: we can do this now we implemented blob retrieval. + pure $ opaqueifyBlobs modelres === opaqueifyBlobs realres where - mkRuns hasFS hasBlockIO = - first V.fromList . unzip <$> - sequence - [ (,wb) <$> mkRunFromSerialisedKOps hasFS hasBlockIO fsPaths wb - | (i, dat) <- zip [0..] (getSmallList dats) - , let wb = runData dat - fsPaths = RunFsPaths (FS.mkFsPath []) (RunNumber i) - ] - lookupss = V.fromList $ concatMap lookups dats resolveV = \(SerialisedValue v1) (SerialisedValue v2) -> SerialisedValue (v1 <> v2) +-- | Given a bunch of 'InMemLookupData', prepare the data into the form needed +-- for 'lookupsIO': a write buffer (and blobs) and a vector of on-disk runs. +-- Also passes the model and the keys to look up to the inner action. +-- +withRuns :: FS.HasFS IO h + -> FS.HasBlockIO IO h + -> [InMemLookupData SerialisedKey SerialisedValue SerialisedBlob] + -> ( WB.WriteBuffer + -> WBB.WriteBufferBlobs IO h + -> V.Vector (Run.Run IO (Handle h)) + -> IO a) + -> IO a +withRuns hfs _ [] action = + bracket + (WBB.new hfs (FS.mkFsPath ["wbblobs"])) + WBB.removeReference + (\wbblobs -> action WB.empty wbblobs V.empty) + +withRuns hfs hbio (wbdat:rundats) action = + bracket + (do wbblobs <- WBB.new hfs (FS.mkFsPath ["wbblobs"]) + wbkops <- traverse (traverse (WBB.addBlob hfs wbblobs)) + (runData wbdat) + let wb = WB.fromMap wbkops + runs <- + V.fromList <$> + sequence + [ mkRunFromSerialisedKOps hfs hbio fsPaths runData + | (i, InMemLookupData{runData}) <- zip [1..] rundats + , let fsPaths = RunFsPaths (FS.mkFsPath []) (RunNumber i) + ] + return (wb, wbblobs, runs)) + + (\(_wb, wbblobs, runs) -> do + V.mapM_ Run.removeReference runs + WBB.removeReference wbblobs) + + (\(wb, wbblobs, runs) -> + action wb wbblobs runs) + opaqueifyBlobs :: V.Vector (Maybe (Entry v b)) -> V.Vector (Maybe (Entry v Opaque)) opaqueifyBlobs = fmap (fmap (fmap Opaque))