diff --git a/bench/macro/lsm-tree-bench-lookups.hs b/bench/macro/lsm-tree-bench-lookups.hs index c8b6adff3..7246157c6 100644 --- a/bench/macro/lsm-tree-bench-lookups.hs +++ b/bench/macro/lsm-tree-bench-lookups.hs @@ -3,12 +3,10 @@ module Main ( main ) where import Control.DeepSeq -import Control.Exception (assert) +import Control.Exception (assert, evaluate) import Control.Monad -import Control.Monad.Class.MonadST import Control.Monad.Primitive import Control.Monad.ST.Strict (ST, runST) -import Data.Arena (ArenaManager, newArenaManager, withArena) import Data.Bits ((.&.)) import Data.BloomFilter (Bloom) import qualified Data.BloomFilter as Bloom @@ -26,6 +24,7 @@ import Database.LSMTree.Internal.Entry (Entry (Insert), NumEntries (..)) import Database.LSMTree.Internal.IndexCompact (IndexCompact) import Database.LSMTree.Internal.Lookup +import Database.LSMTree.Internal.PageAlloc import Database.LSMTree.Internal.Paths (RunFsPaths (RunFsPaths)) import Database.LSMTree.Internal.Run (Run) import qualified Database.LSMTree.Internal.Run as Run @@ -136,7 +135,7 @@ benchmarks !caching = withFS $ \hfs hbio -> do #ifdef NO_IGNORE_ASSERTS putStrLn "BENCHMARKING A BUILD WITH -fno-ignore-asserts" #endif - arenaManager <- newArenaManager + pagealloc <- newPageAlloc enabled <- getRTSStatsEnabled when (not enabled) $ fail "Need RTS +T statistics enabled" let runSizes = lsmStyleRuns benchmarkSizeBase @@ -197,17 +196,17 @@ benchmarks !caching = withFS $ \hfs hbio -> do _bindexSearches <- benchmark "benchIndexSearches" "Calculate batches of keys, perform bloom queries for each batch, and perform index searches for positively queried keys in each batch. Net time/allocation is the result of subtracting the cost of benchGenKeyBatches and benchBloomQueries." - (benchIndexSearches arenaManager blooms indexes handles keyRng0) benchmarkNumLookups + (benchIndexSearches pagealloc blooms indexes handles keyRng0) benchmarkNumLookups (x1 + x2, y1 + y2) _bprepLookups <- benchmark "benchPrepLookups" "Calculate batches of keys, and prepare lookups for each batch. This is roughly doing the same amount of work as benchIndexSearches. Net time/allocation is the result of subtracting the cost of benchGenKeyBatches." - (benchPrepLookups arenaManager blooms indexes handles keyRng0) benchmarkNumLookups + (benchPrepLookups pagealloc blooms indexes handles keyRng0) benchmarkNumLookups bgenKeyBatches _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 + (benchLookupsIO hbio pagealloc benchmarkResolveSerialisedValue runs blooms indexes handles keyRng0) benchmarkNumLookups bgenKeyBatches traceMarkerIO "Cleaning up" @@ -303,8 +302,8 @@ withFS :: (FS.HasFS IO FS.HandleIO -> FS.HasBlockIO IO FS.HandleIO -> IO a) -> IO a withFS action = do - let hfs = FS.ioHasFS (FS.MountPoint "") - exists <- FS.doesDirectoryExist hfs (FS.mkFsPath ["_bench_lookups"]) + let hfs = FS.ioHasFS (FS.MountPoint "_bench_lookups") + exists <- FS.doesDirectoryExist hfs (FS.mkFsPath [""]) unless exists $ error ("_bench_lookups directory does not exist") FS.withIOHasBlockIO hfs FS.defaultIOCtxParams $ \hbio -> action hfs hbio @@ -427,43 +426,46 @@ benchBloomQueries !bs !keyRng !n -- | This gives us the combined cost of calculating batches of keys, performing -- bloom queries for each batch, and performing index searches for each batch. benchIndexSearches - :: ArenaManager RealWorld + :: PageAlloc RealWorld -> V.Vector (Bloom SerialisedKey) -> V.Vector IndexCompact -> V.Vector (FS.Handle h) -> StdGen -> Int -> IO () -benchIndexSearches !arenaManager !bs !ics !hs !keyRng !n +benchIndexSearches !pagealloc !bs !ics !hs !keyRng !n | n <= 0 = pure () | otherwise = do let (!ks, !keyRng') = genLookupBatch keyRng benchmarkGenBatchSize !rkixs = bloomQueriesDefault bs ks - !_ioops <- withArena arenaManager $ \arena -> stToIO $ indexSearches arena ics hs ks rkixs - benchIndexSearches arenaManager bs ics hs keyRng' (n-benchmarkGenBatchSize) + withPages pagealloc (VU.length rkixs) $ \pages -> do + _ <- evaluate (indexSearches ics hs ks pages rkixs) + return () + benchIndexSearches pagealloc bs ics hs keyRng' (n-benchmarkGenBatchSize) -- | This gives us the combined cost of calculating batches of keys, and -- preparing lookups for each batch. benchPrepLookups - :: ArenaManager RealWorld + :: PageAlloc RealWorld -> V.Vector (Bloom SerialisedKey) -> V.Vector IndexCompact -> V.Vector (FS.Handle h) -> StdGen -> Int -> IO () -benchPrepLookups !arenaManager !bs !ics !hs !keyRng !n +benchPrepLookups !pagealloc !bs !ics !hs !keyRng !n | n <= 0 = pure () | otherwise = do let (!ks, !keyRng') = genLookupBatch keyRng benchmarkGenBatchSize - (!_rkixs, !_ioops) <- withArena arenaManager $ \arena -> stToIO $ prepLookups arena bs ics hs ks - benchPrepLookups arenaManager bs ics hs keyRng' (n-benchmarkGenBatchSize) + withPreparedLookups pagealloc bs ics hs ks $ \rkixs ioops -> + void $ evaluate rkixs >> evaluate ioops + benchPrepLookups pagealloc bs ics hs keyRng' (n-benchmarkGenBatchSize) -- | This gives us the combined cost of calculating batches of keys, and -- performing disk lookups for each batch. benchLookupsIO :: FS.HasBlockIO IO h - -> ArenaManager RealWorld + -> PageAlloc RealWorld -> ResolveSerialisedValue -> V.Vector (Run (FS.Handle h)) -> V.Vector (Bloom SerialisedKey) @@ -472,12 +474,12 @@ benchLookupsIO :: -> StdGen -> Int -> IO () -benchLookupsIO !hbio !arenaManager !resolve !rs !bs !ics !hs !keyRng !n +benchLookupsIO !hbio !pagealloc !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) + !_ <- lookupsIO hbio pagealloc resolve rs bs ics hs ks + benchLookupsIO hbio pagealloc resolve rs bs ics hs 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 e5b9077cd..be5d2b814 100644 --- a/bench/micro/Bench/Database/LSMTree/Internal/Lookup.hs +++ b/bench/micro/Bench/Database/LSMTree/Internal/Lookup.hs @@ -2,13 +2,10 @@ module Bench.Database.LSMTree.Internal.Lookup (benchmarks) where -import Control.Exception (assert) +import Control.Exception (assert, evaluate) import Control.Monad -import Control.Monad.ST.Strict (stToIO) import Criterion.Main (Benchmark, bench, bgroup, env, envWithCleanup, - perRunEnv, perRunEnvWithCleanup, whnf, whnfAppIO) -import Data.Arena (ArenaManager, closeArena, newArena, - newArenaManager, withArena) + perRunEnvWithCleanup, whnf, whnfAppIO) import Data.Bifunctor (Bifunctor (..)) import qualified Data.List as List import Data.Map.Strict (Map) @@ -21,7 +18,10 @@ import Database.LSMTree.Extras.Random (frequency, import Database.LSMTree.Extras.UTxO import Database.LSMTree.Internal.Entry (Entry (..), NumEntries (..)) import Database.LSMTree.Internal.Lookup (bloomQueriesDefault, - indexSearches, intraPageLookups, lookupsIO, prepLookups) + cleanupPreparedLookups, indexSearches, intraPageLookups, + lookupsIO, unmanagedAllocatePagesForIndexSearches, + unmanagedPreparedLookups, withPreparedLookups) +import Database.LSMTree.Internal.PageAlloc import Database.LSMTree.Internal.Paths (RunFsPaths (..)) import Database.LSMTree.Internal.Run (Run) import qualified Database.LSMTree.Internal.Run as Run @@ -77,7 +77,7 @@ benchmarks = bgroup "Bench.Database.LSMTree.Internal.Lookup" [ benchLookups :: Config -> Benchmark benchLookups conf@Config{name} = - withEnv $ \ ~(_dir, arenaManager, _hasFS, hasBlockIO, rs, ks) -> + withEnv $ \ ~(_dir, pagealloc, _hasFS, hasBlockIO, rs, ks) -> env ( pure ( V.map Run.runFilter rs , V.map Run.runIndex rs , V.map Run.runKOpsFile rs @@ -90,14 +90,22 @@ benchLookups conf@Config{name} = whnf (\ks' -> bloomQueriesDefault blooms ks') ks -- The compact index is only searched for (true and false) positive -- lookup keys. We use whnf here because the result is - , env (pure $ bloomQueriesDefault blooms ks) $ \rkixs -> - bench "Compact index search" $ - whnfAppIO (\ks' -> withArena arenaManager $ \arena -> stToIO $ indexSearches arena indexes kopsFiles ks' rkixs) ks + , env (do let !rkixs = bloomQueriesDefault blooms ks + pages <- unmanagedAllocatePagesForIndexSearches pagealloc rkixs + return (rkixs, pages)) + (\ ~(rkixs, pages) -> + bench "Compact index search" $ + whnf (\ks' -> indexSearches indexes kopsFiles + ks' pages rkixs) ks) -- prepLookups combines bloom filter querying and index searching. -- The implementation forces the results to WHNF, so we use -- whnfAppIO here instead of nfAppIO. , bench "Lookup preparation in memory" $ - whnfAppIO (\ks' -> withArena arenaManager $ \arena -> stToIO $ prepLookups arena blooms indexes kopsFiles ks') ks + whnfAppIO (\ks' -> withPreparedLookups + pagealloc blooms indexes + kopsFiles ks' $ \rkixs ioops -> + void $ evaluate rkixs >> evaluate ioops + ) ks -- Submit the IOOps we get from prepLookups to HasBlockIO. We use -- perRunEnv because IOOps contain mutable buffers, so we want fresh -- ones for each run of the benchmark. We manually evaluate the @@ -105,9 +113,10 @@ benchLookups conf@Config{name} = , bench "Submit IOOps" $ -- TODO: here arena is destroyed too soon -- but it should be fine for non-debug code - perRunEnv (withArena arenaManager $ \arena -> stToIO $ prepLookups arena blooms indexes kopsFiles ks) $ \ ~(_rkixs, ioops) -> do - !_ioress <- FS.submitIO hasBlockIO ioops - pure () + perRunEnvWithCleanup + (unmanagedPreparedLookups pagealloc blooms indexes kopsFiles ks) + (cleanupPreparedLookups pagealloc) + (\ ~(_, ioops, _) -> void $ evaluate =<< FS.submitIO hasBlockIO ioops) -- When IO result have been collected, intra-page lookups searches -- through the raw bytes (representing a disk page) for the lookup -- key. Again, we use perRunEnv here because IOOps contain mutable @@ -117,20 +126,21 @@ 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) - ) - (\(_, _, _, arena) -> closeArena arenaManager arena) $ \ ~(rkixs, ioops, ioress, _) -> do - !_ <- intraPageLookups resolveV rs ks rkixs ioops ioress - pure () + (do (rkixs, ioops, pages) <- + unmanagedPreparedLookups pagealloc blooms + indexes kopsFiles ks + ioress <- FS.submitIO hasBlockIO ioops + return ((rkixs, ioops, pages), ioress)) + (cleanupPreparedLookups pagealloc . fst) + (\ ~((rkixs, ioops, _pages), ioress) -> + void $ evaluate =<< intraPageLookups resolveV rs ks + rkixs ioops ioress) -- 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 + whnfAppIO (\ks' -> lookupsIO hasBlockIO pagealloc resolveV rs blooms indexes kopsFiles ks') ks ] where withEnv = envWithCleanup @@ -162,14 +172,14 @@ data Config = Config { lookupsInBatchesEnv :: Config -> IO ( FilePath -- ^ Temporary directory - , ArenaManager RealWorld + , PageAlloc RealWorld , FS.HasFS IO FS.HandleIO , FS.HasBlockIO IO FS.HandleIO , V.Vector (Run (FS.Handle FS.HandleIO)) , V.Vector SerialisedKey ) lookupsInBatchesEnv Config {..} = do - arenaManager <- newArenaManager + pagealloc <- newPageAlloc sysTmpDir <- getCanonicalTemporaryDirectory benchTmpDir <- createTempDirectory sysTmpDir "lookupsInBatchesEnv" (storedKeys, lookupKeys) <- lookupsEnv (mkStdGen 17) nentries npos nneg @@ -184,7 +194,7 @@ lookupsInBatchesEnv Config {..} = do assert (npagesReal * 42 <= nentriesReal) $ pure () assert (npagesReal * 43 >= nentriesReal) $ pure () pure ( benchTmpDir - , arenaManager + , pagealloc , hasFS , hasBlockIO , V.singleton r @@ -193,14 +203,14 @@ lookupsInBatchesEnv Config {..} = do lookupsInBatchesCleanup :: ( FilePath -- ^ Temporary directory - , ArenaManager RealWorld + , PageAlloc RealWorld , FS.HasFS IO FS.HandleIO , FS.HasBlockIO IO FS.HandleIO , V.Vector (Run (FS.Handle FS.HandleIO)) , V.Vector SerialisedKey ) -> IO () -lookupsInBatchesCleanup (tmpDir, _arenaManager, hasFS, hasBlockIO, rs, _) = do +lookupsInBatchesCleanup (tmpDir, _pagealloc, hasFS, hasBlockIO, rs, _) = do FS.close hasBlockIO forM_ rs $ Run.removeReference hasFS removeDirectoryRecursive tmpDir diff --git a/cabal.project b/cabal.project index cf078342c..2b6e4e405 100644 --- a/cabal.project +++ b/cabal.project @@ -30,7 +30,7 @@ benchmarks: True constraints: bloomfilter <0 -- comment me if you are benchmarking -import: cabal.project.debug +--import: cabal.project.debug -- comment me if you don't have liburing installed -- diff --git a/lsm-tree.cabal b/lsm-tree.cabal index ac7f0d084..d99b18481 100644 --- a/lsm-tree.cabal +++ b/lsm-tree.cabal @@ -51,7 +51,6 @@ library hs-source-dirs: src exposed-modules: Control.Concurrent.Class.MonadSTM.RWVar - Data.Arena Data.Map.Range Database.LSMTree.Common Database.LSMTree.Internal @@ -73,6 +72,7 @@ library Database.LSMTree.Internal.Normal Database.LSMTree.Internal.PageAcc Database.LSMTree.Internal.PageAcc1 + Database.LSMTree.Internal.PageAlloc Database.LSMTree.Internal.Paths Database.LSMTree.Internal.Primitive Database.LSMTree.Internal.Range @@ -233,7 +233,6 @@ test-suite lsm-tree-test Database.LSMTree.ModelIO.Normal Database.LSMTree.ModelIO.Session Test.Control.Concurrent.Class.MonadSTM.RWVar - Test.Data.Arena Test.Database.LSMTree.Class.Monoidal Test.Database.LSMTree.Class.Normal Test.Database.LSMTree.Generators @@ -247,6 +246,7 @@ test-suite lsm-tree-test Test.Database.LSMTree.Internal.Monkey Test.Database.LSMTree.Internal.PageAcc Test.Database.LSMTree.Internal.PageAcc1 + Test.Database.LSMTree.Internal.PageAlloc Test.Database.LSMTree.Internal.RawOverflowPage Test.Database.LSMTree.Internal.RawPage Test.Database.LSMTree.Internal.Run @@ -394,7 +394,6 @@ benchmark lsm-tree-bench-lookups , bytestring , deepseq , fs-api - , io-classes , lsm-tree , lsm-tree:blockio-api , lsm-tree:bloomfilter diff --git a/src/Data/Arena.hs b/src/Data/Arena.hs deleted file mode 100644 index 1826ef38e..000000000 --- a/src/Data/Arena.hs +++ /dev/null @@ -1,185 +0,0 @@ -{-# LANGUAGE CPP #-} -{-# LANGUAGE LambdaCase #-} -{-# LANGUAGE RecordWildCards #-} -module Data.Arena ( - ArenaManager, - newArenaManager, - Arena, - Size, - Offset, - Alignment, - withArena, - newArena, - closeArena, - - allocateFromArena, - -- * Test helpers - withUnmanagedArena, -) where - -import Control.DeepSeq (NFData (..)) -import Control.Exception (assert) -import Control.Monad.Primitive -import Data.Bits (complement, popCount, (.&.)) -import Data.Primitive.ByteArray -import Data.Primitive.MutVar -import Data.Primitive.MVar -import Data.Primitive.PrimVar - -#ifdef NO_IGNORE_ASSERTS -import Data.Word (Word8) -#endif - -data ArenaManager s = ArenaManager (MutVar s [Arena s]) - -newArenaManager :: PrimMonad m => m (ArenaManager (PrimState m)) -newArenaManager = do - m <- newMutVar [] - return $ ArenaManager m - --- | For use in bencmark environments -instance NFData (ArenaManager s) where - rnf (ArenaManager !_) = () - -data Arena s = Arena - { curr :: !(MVar s (Block s)) -- current block, also acts as a lock - , free :: !(MutVar s [Block s]) - , full :: !(MutVar s [Block s]) - } - -data Block s = Block !(PrimVar s Int) !(MutableByteArray s) - -instance NFData (Arena s) where - rnf (Arena !_ !_ !_) = () - -type Size = Int -type Offset = Int -type Alignment = Int - -blockSize :: Int -blockSize = 0x100000 - -newBlock :: PrimMonad m => m (Block (PrimState m)) -newBlock = do - off <- newPrimVar 0 - mba <- newAlignedPinnedByteArray blockSize 4096 - return (Block off mba) - -withArena :: PrimMonad m => ArenaManager (PrimState m) -> (Arena (PrimState m) -> m a) -> m a -withArena am f = do - a <- newArena am - x <- f a - closeArena am a - pure x - -newArena :: PrimMonad m => ArenaManager (PrimState m) -> m (Arena (PrimState m)) -newArena (ArenaManager arenas) = do - marena <- atomicModifyMutVar' arenas $ \case - [] -> ([], Nothing) - (x:xs) -> (xs, Just x) - - case marena of - Just arena -> return arena - Nothing -> do - curr <- newBlock >>= newMVar - free <- newMutVar [] - full <- newMutVar [] - return Arena {..} - -closeArena :: PrimMonad m => ArenaManager (PrimState m) -> Arena (PrimState m) -> m () -closeArena (ArenaManager arenas) arena = do - scrambleArena arena - - -- reset the arena to clear state - resetArena arena - - atomicModifyMutVar' arenas $ \xs -> (arena : xs, ()) - - - -scrambleArena :: PrimMonad m => Arena (PrimState m) -> m () -#ifndef NO_IGNORE_ASSERTS -scrambleArena _ = return () -#else -scrambleArena Arena {..} = do - readMVar curr >>= scrambleBlock - readMutVar full >>= mapM_ scrambleBlock - readMutVar free >>= mapM_ scrambleBlock - -scrambleBlock :: PrimMonad m => Block (PrimState m) -> m () -scrambleBlock (Block _ mba) = do - size <- getSizeofMutableByteArray mba - setByteArray mba 0 size (0x77 :: Word8) -#endif - --- | Reset arena, i.e. return used blocks to free list. -resetArena :: PrimMonad m => Arena (PrimState m) -> m () -resetArena Arena {..} = do - Block off mba <- takeMVar curr - - -- reset current block - writePrimVar off 0 - - -- move full block to free blocks. - -- block's offset will be reset in 'newBlockWithFree' - full' <- atomicModifyMutVar' full $ \xs -> ([], xs) - atomicModifyMutVar' free $ \xs -> (full' <> xs, ()) - - putMVar curr $! Block off mba - --- | Create unmanaged arena. --- --- Never use this in non-tests code. -withUnmanagedArena :: PrimMonad m => (Arena (PrimState m) -> m a) -> m a -withUnmanagedArena k = do - mgr <- newArenaManager - withArena mgr k - --- | Allocate a slice of mutable byte array from the arena. -allocateFromArena :: PrimMonad m => Arena (PrimState m)-> Size -> Alignment -> m (Offset, MutableByteArray (PrimState m)) -allocateFromArena !arena !size !alignment = - assert (popCount alignment == 1) $ -- powers of 2 - assert (size <= blockSize) $ -- not too large allocations - allocateFromArena' arena size alignment - --- TODO!? this is not async exception safe -allocateFromArena' :: PrimMonad m => Arena (PrimState m)-> Size -> Alignment -> m (Offset, MutableByteArray (PrimState m)) -allocateFromArena' arena@Arena { .. } !size !alignment = do - -- take current block, lock the arena - curr'@(Block off mba) <- takeMVar curr - - off' <- readPrimVar off - let !ali = alignment - 1 - let !off'' = (off' + ali) .&. complement ali -- ceil towards next alignment - let !end = off'' + size - if end <= blockSize - then do - -- fits into current block: - -- * update offset - writePrimVar off end - -- * release lock - putMVar curr curr' - -- * return data - return (off'', mba) - - else do - -- doesn't fit into current block: - -- * move current block into full - atomicModifyMutVar' full (\xs -> (curr' : xs, ())) - -- * allocate new block - new <- newBlockWithFree free - -- * set new block as current, release the lock - putMVar curr new - -- * go again - allocateFromArena' arena size alignment - --- | Allocate new block, possibly taking it from a free list -newBlockWithFree :: PrimMonad m => MutVar (PrimState m) [Block (PrimState m)] -> m (Block (PrimState m)) -newBlockWithFree free = do - free' <- readMutVar free - case free' of - [] -> newBlock - x@(Block off _):xs -> do - writePrimVar off 0 - writeMutVar free xs - return x diff --git a/src/Database/LSMTree/Internal.hs b/src/Database/LSMTree/Internal.hs index 442fb5010..888ab2351 100644 --- a/src/Database/LSMTree/Internal.hs +++ b/src/Database/LSMTree/Internal.hs @@ -59,7 +59,6 @@ import Control.Monad (unless, void, when) import Control.Monad.Class.MonadThrow import Control.Monad.Primitive (PrimState (..)) import Control.Monad.ST.Strict (ST, runST) -import Data.Arena (ArenaManager, newArenaManager) import Data.Bifunctor (Bifunctor (..)) import Data.BloomFilter (Bloom) import qualified Data.ByteString.Char8 as BSC @@ -82,6 +81,7 @@ import Database.LSMTree.Internal.Lookup (ByteCountDiscrepancy, import Database.LSMTree.Internal.Managed import qualified Database.LSMTree.Internal.Merge as Merge import qualified Database.LSMTree.Internal.Normal as Normal +import Database.LSMTree.Internal.PageAlloc (PageAlloc, newPageAlloc) import Database.LSMTree.Internal.Paths (RunFsPaths (..), SessionRoot (..), SnapshotName) import qualified Database.LSMTree.Internal.Paths as Paths @@ -360,13 +360,13 @@ closeSession Session{sessionState} = -- -- For more information, see 'Database.LSMTree.Normal.TableHandle'. data TableHandle m h = TableHandle { - tableConfig :: !TableConfig + tableConfig :: !TableConfig -- | The primary purpose of this 'RWVar' is to ensure consistent views of -- the open-/closedness of a table when multiple threads require access to -- the table's fields (see 'withOpenTable'). We use more fine-grained -- synchronisation for various mutable parts of an open table. - , tableHandleState :: !(RWVar m (TableHandleState m h)) - , tableHandleArenaManager :: !(ArenaManager (PrimState m)) + , tableHandleState :: !(RWVar m (TableHandleState m h)) + , tableHandlePageAlloc :: !(PageAlloc (PrimState m)) } -- | A table handle may assume that its corresponding session is still open as @@ -575,8 +575,8 @@ newWithLevels sesh seshEnv conf !levels = do , tableId = tableId , tableContent = contentVar } - arenaManager <- newArenaManager - let !th = TableHandle conf tableVar arenaManager + pagealloc <- newPageAlloc + let !th = TableHandle conf tableVar pagealloc -- Track the current table modifyMVar_ (sessionOpenTables seshEnv) $ pure . Map.insert tableId th pure $! th @@ -610,7 +610,6 @@ lookups :: -- 'toNormalLookupResult' or 'toMonoidalLookupResult'. -> m (V.Vector lookupResult) lookups ks th fromEntry = withOpenTable th $ \thEnv -> do - let arenaManager = tableHandleArenaManager th let resolve = resolveMupsert (tableConfig th) RW.withReadAccess (tableContent thEnv) $ \tableContent -> do let !wb = tableWriteBuffer tableContent @@ -618,7 +617,7 @@ lookups ks th fromEntry = withOpenTable th $ \thEnv -> do ioRes <- lookupsIO (tableHasBlockIO thEnv) - arenaManager + (tableHandlePageAlloc th) resolve (cachedRuns cache) (cachedFilters cache) diff --git a/src/Database/LSMTree/Internal/Lookup.hs b/src/Database/LSMTree/Internal/Lookup.hs index 1d9294acf..77fd57b53 100644 --- a/src/Database/LSMTree/Internal/Lookup.hs +++ b/src/Database/LSMTree/Internal/Lookup.hs @@ -14,20 +14,22 @@ module Database.LSMTree.Internal.Lookup ( -- * Internal: exposed for tests and benchmarks , RunIx , KeyIx - , prepLookups + , withPreparedLookups + , unmanagedPreparedLookups + , cleanupPreparedLookups , bloomQueries , bloomQueriesDefault , indexSearches + , withPagesForIndexSearches + , unmanagedAllocatePagesForIndexSearches , intraPageLookups ) where import Control.Exception (Exception, assert) import Control.Monad -import Control.Monad.Class.MonadST as Class import Control.Monad.Class.MonadThrow (MonadThrow (..)) import Control.Monad.Primitive import Control.Monad.ST.Strict -import Data.Arena (Arena, ArenaManager, allocateFromArena, withArena) import Data.Bifunctor import Data.BloomFilter (Bloom) import qualified Data.BloomFilter as Bloom @@ -44,6 +46,7 @@ import Database.LSMTree.Internal.Entry import Database.LSMTree.Internal.IndexCompact (IndexCompact, PageSpan (..)) import qualified Database.LSMTree.Internal.IndexCompact as Index +import Database.LSMTree.Internal.PageAlloc import Database.LSMTree.Internal.RawBytes (RawBytes (..)) import qualified Database.LSMTree.Internal.RawBytes as RB import Database.LSMTree.Internal.RawPage @@ -53,21 +56,65 @@ import qualified Database.LSMTree.Internal.Vector as V import System.FS.API (BufferOffset (..), Handle) import System.FS.BlockIO.API + +{-# INLINE withPreparedLookups #-} -- | Prepare disk lookups by doing bloom filter queries, index searches and --- creating 'IOOp's. The result is a vector of 'IOOp's and a vector of indexes, --- both of which are the same length. The indexes record the run and key --- associated with each 'IOOp'. -prepLookups :: - Arena s +-- creating 'IOOp's. The inner action receives a vector of 'IOOp's and a vector +-- of indexes, both of which are the same length. The indexes record the run +-- and key associated with each 'IOOp'. +-- +-- The \"with resource\" style is used because interally it needs to use +-- resources from the 'PageAlloc'. If the \"with\" style cannot be used, use +-- the direct style with the combination of 'unmanagedPreparedLookups' and +-- 'cleanupPreparedLookups'. +withPreparedLookups + :: PrimMonad m + => PageAlloc (PrimState m) -> V.Vector (Bloom SerialisedKey) -> V.Vector IndexCompact -> V.Vector (Handle h) -> V.Vector SerialisedKey - -> ST s (VU.Vector (RunIx, KeyIx), V.Vector (IOOp s h)) -prepLookups arena blooms indexes kopsFiles ks = do - let !rkixs = bloomQueriesDefault blooms ks - !ioops <- indexSearches arena indexes kopsFiles ks rkixs - pure (rkixs, ioops) + -> (VU.Vector (RunIx, KeyIx) -> V.Vector (IOOp (PrimState m) h) -> m a) + -> m a +withPreparedLookups pagealloc blooms indexes kopsFiles ks f = + let !rkixs = bloomQueriesDefault blooms ks in + withPagesForIndexSearches pagealloc rkixs $ \pages -> + let !ioops = indexSearches indexes kopsFiles ks pages rkixs + in f rkixs ioops + +-- | Like 'withPreparedLookups', but in a direct style which requires matching +-- resource cleanup using 'cleanupPreparedLookups'. +-- +-- Use 'withPreparedLookups' instead if possible. This direct style is sometimes +-- required in tests and benchmarks. +unmanagedPreparedLookups + :: PrimMonad m + => PageAlloc (PrimState m) + -> V.Vector (Bloom SerialisedKey) + -> V.Vector IndexCompact + -> V.Vector (Handle h) + -> V.Vector SerialisedKey + -> m ( VU.Vector (RunIx, KeyIx) + , V.Vector (IOOp (PrimState m) h) + , Pages (PrimState m) + ) +unmanagedPreparedLookups pagealloc blooms indexes kopsFiles ks = do + let !rkixs = bloomQueriesDefault blooms ks + pages <- unmanagedAllocatePages pagealloc (VU.length rkixs) + let !ioops = indexSearches indexes kopsFiles ks pages rkixs + return (rkixs, ioops, pages) + +-- | The matching resource cleanup for 'unmanagedPreparedLookups'. +cleanupPreparedLookups + :: PrimMonad m + => PageAlloc (PrimState m) + -> ( VU.Vector (RunIx, KeyIx) + , V.Vector (IOOp (PrimState m) h) + , Pages (PrimState m) + ) + -> m () +cleanupPreparedLookups pagealloc (_rkixs, _ioops, pages) = + freePages pagealloc pages type KeyIx = Int type RunIx = Int @@ -148,32 +195,49 @@ bloomQueries !blooms !ks !resN -- @VU.Vector (RunIx, KeyIx)@ argument, because index searching always returns a -- positive search result. indexSearches - :: Arena s - -> V.Vector IndexCompact + :: V.Vector IndexCompact -> V.Vector (Handle h) -> V.Vector SerialisedKey + -> Pages s -- ^ See 'allocatePagesForIndexSearches' -> VU.Vector (RunIx, KeyIx) -- ^ Result of 'bloomQueries' - -> ST s (V.Vector (IOOp s h)) -indexSearches !arena !indexes !kopsFiles !ks !rkixs = V.generateM n $ \i -> do - let (!rix, !kix) = rkixs `VU.unsafeIndex` i - !c = indexes `V.unsafeIndex` rix - !h = kopsFiles `V.unsafeIndex` rix - !k = ks `V.unsafeIndex` kix - !pspan = Index.search k c - !size = Index.pageSpanSize pspan - -- The current allocation strategy is to allocate a new pinned - -- byte array for each 'IOOp'. One optimisation we are planning to - -- do is to use a cache of re-usable buffers, in which case we - -- decrease the GC load. TODO: re-usable buffers. - (!off, !buf) <- allocateFromArena arena (size * 4096) 4096 - pure $! IOOpRead - h - (fromIntegral $ Index.unPageNo (pageSpanStart pspan) * 4096) - buf - (fromIntegral off) - (fromIntegral $ size * 4096) - where - !n = VU.length rkixs + -> V.Vector (IOOp s h) +indexSearches !indexes !kopsFiles !ks !pages !rkixs = + V.generateStrict (VU.length rkixs) $ \i -> + let (!rix,!kix) = rkixs `VU.unsafeIndex` i + !c = indexes `V.unsafeIndex` rix + !h = kopsFiles `V.unsafeIndex` rix + !k = ks `V.unsafeIndex` kix + !pspan = Index.search k c + !size = Index.pageSpanSize pspan + (!buf,!off) = unsafeIndexPages pages i + in IOOpRead + h + (fromIntegral $ Index.unPageNo (pageSpanStart pspan) * 4096) + buf + (fromIntegral off) + (fromIntegral $ size * 4096) + + +-- | Helper for 'indexSearches' to allocatePages the required 'Pages'. +withPagesForIndexSearches + :: PrimMonad m + => PageAlloc (PrimState m) + -> VU.Vector (RunIx, KeyIx) -- ^ Result of 'bloomQueries' + -> (Pages (PrimState m) -> m a) + -> m a +withPagesForIndexSearches pagealloc rkixs f = + withPages pagealloc (VU.length rkixs) f +{-# INLINE withPagesForIndexSearches #-} + +-- | Helper for 'indexSearches' to allocatePages the required 'Pages'. +unmanagedAllocatePagesForIndexSearches + :: PrimMonad m + => PageAlloc (PrimState m) + -> VU.Vector (RunIx, KeyIx) -- ^ Result of 'bloomQueries' + -> m (Pages (PrimState m)) +unmanagedAllocatePagesForIndexSearches pagealloc rkixs = + unmanagedAllocatePages pagealloc (VU.length rkixs) +{-# INLINE unmanagedAllocatePagesForIndexSearches #-} {- Note [Batched lookups, buffer strategy and restrictions] @@ -214,7 +278,7 @@ data ByteCountDiscrepancy = ByteCountDiscrepancy { {-# SPECIALIZE lookupsIO :: HasBlockIO IO h - -> ArenaManager RealWorld + -> PageAlloc RealWorld -> ResolveSerialisedValue -> V.Vector (Run (Handle h)) -> V.Vector (Bloom SerialisedKey) @@ -230,9 +294,9 @@ data ByteCountDiscrepancy = ByteCountDiscrepancy { -- PRECONDITION: the vectors of bloom filters, indexes and file handles -- should pointwise match with the vectors of runs. lookupsIO :: - forall m h. (PrimMonad m, MonadThrow m, MonadST m) + forall m h. (PrimMonad m, MonadThrow m) => HasBlockIO m h - -> ArenaManager (PrimState m) + -> PageAlloc (PrimState m) -> ResolveSerialisedValue -> V.Vector (Run (Handle h)) -- ^ Runs @rs@ -> V.Vector (Bloom SerialisedKey) -- ^ The bloom filters inside @rs@ @@ -240,10 +304,11 @@ lookupsIO :: -> V.Vector (Handle h) -- ^ The file handles to the key\/value files inside @rs@ -> V.Vector SerialisedKey -> m (V.Vector (Maybe (Entry SerialisedValue (BlobRef (Run (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 !pagealloc !resolveV !rs !blooms !indexes !kopsFiles !ks = + assert precondition $ + withPreparedLookups pagealloc blooms indexes kopsFiles ks $ \rkixs ioops -> do + ioress <- submitIO hbio ioops + intraPageLookups resolveV rs ks rkixs ioops ioress where -- we check only that the lengths match, because checking the contents is -- too expensive. diff --git a/src/Database/LSMTree/Internal/PageAlloc.hs b/src/Database/LSMTree/Internal/PageAlloc.hs new file mode 100644 index 000000000..94ef258e6 --- /dev/null +++ b/src/Database/LSMTree/Internal/PageAlloc.hs @@ -0,0 +1,157 @@ +{-# LANGUAGE CPP #-} +{-# LANGUAGE ScopedTypeVariables #-} +module Database.LSMTree.Internal.PageAlloc ( + PageAlloc, + newPageAlloc, + withPages, + Pages, + unsafeIndexPages, + -- * Lower level + unmanagedAllocatePages, + freePages, +) where + +import Data.Bits +import Data.Primitive.ByteArray +import Data.Primitive.MutVar +import Data.Primitive.SmallArray + +import Control.DeepSeq (NFData (..)) +import Control.Exception (assert) +import Control.Monad.Primitive (PrimMonad, PrimState, unsafeSTToPrim) +import Control.Monad.ST + + +newtype PageAlloc s = PageAlloc (MutVar s [Slab s]) + +-- | A slab is a cache of 16 contiguous 4k pages, aligned to 4k. +type Slab s = MutableByteArray s + +-- | A set of allocated pages, as returned by 'withPages'. Use 'unsafeIndexPages' +-- to index individual pages. +newtype Pages s = Pages (SmallArray (Slab s)) + deriving newtype NFData + +-- | For use in bencmark environments +instance NFData (PageAlloc s) where + rnf _ = () + +newPageAlloc :: PrimMonad m => m (PageAlloc (PrimState m)) +newPageAlloc = PageAlloc <$> newMutVar [] + +{-# INLINE withPages #-} +withPages :: PrimMonad m + => PageAlloc (PrimState m) + -> Int + -> (Pages (PrimState m) -> m a) + -> m a +withPages pa !n f = do + -- no need for bracket to guarantee freeing, it'll all get GC'd. + ps <- unmanagedAllocatePages pa n + x <- f ps + freePages pa ps + return x + +{-# INLINE unsafeIndexPages #-} +-- | The index must be within the number of pages requested from 'withPages'. +unsafeIndexPages :: Pages s -> Int -> (MutableByteArray s, Int) +unsafeIndexPages (Pages slabs) n = + assert (slab_i >= 0 && slab_i < sizeofSmallArray slabs) + (indexSmallArray slabs slab_i, page_i) + where + slab_i = n `unsafeShiftR` 4 + page_i = (n .&. 0xf) `unsafeShiftL` 12 -- byte offset of 4k page within slab + +{-# SPECIALIZE + unmanagedAllocatePages :: PageAlloc RealWorld + -> Int + -> IO (Pages RealWorld) + #-} +-- | Where it is not possible to use 'withPages', use this to allocate pages +-- but this must be matched by exactly one use of 'freePages'. +unmanagedAllocatePages :: PrimMonad m + => PageAlloc (PrimState m) + -> Int + -> m (Pages (PrimState m)) +unmanagedAllocatePages (PageAlloc slabcache) !npages = do + let !nslabs = (npages + 0xf) `unsafeShiftR` 4 + slabs <- atomicModifyMutVar' slabcache (takeSlabs nslabs) + let !pages = assemblePages nslabs slabs + return pages + +takeSlabs :: Int -> [a] -> ([a], [a]) +takeSlabs = go [] + where + go acc 0 ss = (ss, acc) -- (result, state') + go acc _ [] = (acc, []) + go acc !n (s:ss) = go (s:acc) (n-1) ss + +assemblePages :: forall s. Int -> [Slab s] -> Pages s +assemblePages !nslabs slabs0 = + Pages $ + createSmallArray nslabs undefined $ \slabarr -> + useCachedSlabs slabarr 0 nslabs slabs0 + where + useCachedSlabs :: forall s'. + SmallMutableArray s' (Slab s) + -> Int -> Int + -> [Slab s] + -> ST s' () + useCachedSlabs !slabarr !i !n (slab:slabs) = + assert (n > 0 && i >= 0 && i < nslabs) $ do + writeSmallArray slabarr i slab + useCachedSlabs slabarr (i+1) (n-1) slabs + + useCachedSlabs !slabs !i !n [] = + allocFreshSlabs slabs i n + + allocFreshSlabs :: forall s'. + SmallMutableArray s' (Slab s) + -> Int -> Int + -> ST s' () + allocFreshSlabs !_ !_ 0 = + return () + + allocFreshSlabs slabarr !i !n = do + -- Use unsafeSTToPrim to separate the s and s' here: + -- The s is the outer tag, usually RealWorld + -- The s' is the inner tag for createSmallArray above. + slab <- unsafeSTToPrim $ newAlignedPinnedByteArray 0x10000 0x1000 + writeSmallArray slabarr i slab + allocFreshSlabs slabarr (i+1) (n-1) + +{-# SPECIALIZE + freePages :: PageAlloc RealWorld + -> Pages RealWorld + -> IO () + #-} +-- | For use with unmanagedAllocatePages'. Note that this is /not/ idempotent. +-- Freeing the same pages more than once will lead to corruption of the page +-- allocator state and bugs that are hard to diagnose. +freePages :: PrimMonad m + => PageAlloc (PrimState m) + -> Pages (PrimState m) + -> m () +freePages (PageAlloc slabcache) pgs = do +#ifdef NO_IGNORE_ASSERTS + scramblePages pgs +#endif + atomicModifyMutVar' slabcache (putSlabs pgs) + --TODO: in NO_IGNORE_ASSERTS mode, verify that the same pages have not + -- been added to the slabcache already. + +putSlabs :: Pages s -> [Slab s] -> ([Slab s], ()) +putSlabs = \(Pages slabarr) slabs -> + go slabarr 0 (sizeofSmallArray slabarr) slabs + where + go _ _ 0 slabs = (slabs, ()) + go slabarr i n slabs = let !slab = indexSmallArray slabarr i + in go slabarr (i+1) (n-1) (slab:slabs) + +#ifdef NO_IGNORE_ASSERTS +-- | Scramble the allocated bytearrays, they shouldn't be in use anymore! +scramblePages (Pages slabs) = + forM_ slabs $ \slab -> do + size <- getSizeofMutableByteArray slab + setByteArray slab 0 size (0x77 :: Word8) +#endif diff --git a/src/Database/LSMTree/Internal/Vector.hs b/src/Database/LSMTree/Internal/Vector.hs index 3e65f0e32..260149846 100644 --- a/src/Database/LSMTree/Internal/Vector.hs +++ b/src/Database/LSMTree/Internal/Vector.hs @@ -4,6 +4,7 @@ module Database.LSMTree.Internal.Vector ( mkPrimVector, noRetainedExtraMemory, + generateStrict, mapStrict, mapMStrict, imapMStrict, @@ -39,6 +40,10 @@ noRetainedExtraMemory (VP.Vector off len ba) = where sizeof = I# (sizeOfType# (Proxy @a)) +{-# INLINE generateStrict #-} +generateStrict :: Int -> (Int -> a) -> V.Vector a +generateStrict n f = runST (V.generateM n (\i -> pure $! f i)) + {-# INLINE mapStrict #-} -- | /( O(n) /) Like 'V.map', but strict in the produced elements of type @b@. mapStrict :: forall a b. (a -> b) -> V.Vector a -> V.Vector b diff --git a/test/Main.hs b/test/Main.hs index 87aac0dcc..7a73e7d0b 100644 --- a/test/Main.hs +++ b/test/Main.hs @@ -3,7 +3,6 @@ module Main (main) where import qualified Test.Control.Concurrent.Class.MonadSTM.RWVar -import qualified Test.Data.Arena import qualified Test.Database.LSMTree.Class.Monoidal import qualified Test.Database.LSMTree.Class.Normal import qualified Test.Database.LSMTree.Generators @@ -16,6 +15,7 @@ import qualified Test.Database.LSMTree.Internal.Merge import qualified Test.Database.LSMTree.Internal.Monkey import qualified Test.Database.LSMTree.Internal.PageAcc import qualified Test.Database.LSMTree.Internal.PageAcc1 +import qualified Test.Database.LSMTree.Internal.PageAlloc import qualified Test.Database.LSMTree.Internal.RawOverflowPage import qualified Test.Database.LSMTree.Internal.RawPage import qualified Test.Database.LSMTree.Internal.Run @@ -45,6 +45,7 @@ main = defaultMain $ testGroup "lsm-tree" , Test.Database.LSMTree.Internal.Monkey.tests , Test.Database.LSMTree.Internal.PageAcc.tests , Test.Database.LSMTree.Internal.PageAcc1.tests + , Test.Database.LSMTree.Internal.PageAlloc.tests , Test.Database.LSMTree.Internal.RawPage.tests , Test.Database.LSMTree.Internal.RawOverflowPage.tests , Test.Database.LSMTree.Internal.Run.tests @@ -59,5 +60,4 @@ main = defaultMain $ testGroup "lsm-tree" , Test.Database.LSMTree.Model.Monoidal.tests , Test.Database.LSMTree.Normal.StateMachine.tests , Test.System.Posix.Fcntl.NoCache.tests - , Test.Data.Arena.tests ] diff --git a/test/Test/Database/LSMTree/Internal/Lookup.hs b/test/Test/Database/LSMTree/Internal/Lookup.hs index 7e786d147..bab1ca576 100644 --- a/test/Test/Database/LSMTree/Internal/Lookup.hs +++ b/test/Test/Database/LSMTree/Internal/Lookup.hs @@ -23,7 +23,6 @@ module Test.Database.LSMTree.Internal.Lookup ( import Control.DeepSeq import Control.Exception import Control.Monad.ST.Strict -import Data.Arena (newArenaManager, withUnmanagedArena) import Data.Bifunctor import Data.BloomFilter (Bloom) import qualified Data.BloomFilter as Bloom @@ -46,6 +45,7 @@ import Database.LSMTree.Internal.BlobRef (BlobSpan) import Database.LSMTree.Internal.Entry import Database.LSMTree.Internal.IndexCompact as Index import Database.LSMTree.Internal.Lookup +import Database.LSMTree.Internal.PageAlloc import Database.LSMTree.Internal.Paths (RunFsPaths (..)) import qualified Database.LSMTree.Internal.RawBytes as RB import Database.LSMTree.Internal.RawOverflowPage @@ -148,10 +148,12 @@ prop_indexSearchesModel dats = runs = getSmallList $ fmap (mkTestRun . runData) dats lookupss = concatMap lookups $ getSmallList dats - real rkixs = runST $ withUnmanagedArena $ \arena -> do + real rkixs = runST $ do let rs = V.fromList (fmap runWithHandle runs) ks = V.fromList lookupss - res <- indexSearches arena (V.map thrd3 rs) (V.map fst3 rs) ks rkixs + pagealloc <- newPageAlloc + pages <- unmanagedAllocatePagesForIndexSearches pagealloc rkixs + let res = indexSearches (V.map thrd3 rs) (V.map fst3 rs) ks pages rkixs pure $ V.map ioopPageSpan res model rkixs = V.fromList $ indexSearchesModel (fmap thrd3 runs) lookupss $ rkixs @@ -173,14 +175,17 @@ prop_prepLookupsModel dats = real === model where runs = getSmallList $ fmap (mkTestRun . runData) dats lookupss = concatMap lookups $ getSmallList dats - real = runST $ withUnmanagedArena $ \arena -> do + real, model :: (VU.Vector (RunIx, KeyIx), V.Vector PageSpan) + real = runST $ do + pagealloc <- newPageAlloc let rs = V.fromList (fmap runWithHandle runs) ks = V.fromList lookupss - (kixs, ioops) <- prepLookups - arena - (V.map snd3 rs) - (V.map thrd3 rs) - (V.map fst3 rs) ks + (kixs, ioops, _pages) <- + unmanagedPreparedLookups + pagealloc + (V.map snd3 rs) + (V.map thrd3 rs) + (V.map fst3 rs) ks pure $ (kixs, V.map ioopPageSpan ioops) model = bimap VU.fromList V.fromList $ prepLookupsModel (fmap (\x -> (snd3 x, thrd3 x)) runs) lookupss @@ -214,14 +219,16 @@ prop_inMemRunLookupAndConstruction dat = run = mkTestRun runData keys = V.fromList lookups -- prepLookups says that a key /could be/ in the given page - keysMaybeInRun = runST $ withUnmanagedArena $ \arena -> do - (kixs, ioops) <- let r = V.singleton (runWithHandle run) - in prepLookups - arena - (V.map snd3 r) - (V.map thrd3 r) - (V.map fst3 r) - keys + keysMaybeInRun = runST $ do + pagealloc <- newPageAlloc + (kixs, ioops, _pages) <- + let r = V.singleton (runWithHandle run) in + unmanagedPreparedLookups + pagealloc + (V.map snd3 r) + (V.map thrd3 r) + (V.map fst3 r) + keys let ks = V.map (V.fromList lookups V.!) (V.convert $ snd $ VU.unzip kixs) pss = V.map (handleRaw . ioopHandle) ioops pspans = V.map (ioopPageSpan) ioops @@ -291,10 +298,10 @@ prop_roundtripFromWriteBufferLookupIO dats = ioProperty $ withTempIOHasBlockIO "prop_roundtripFromWriteBufferLookupIO" $ \hasFS hasBlockIO -> do (runs, wbs) <- mkRuns hasFS hasBlockIO let wbAll = WB.fromMap $ Map.unionsWith (combine resolveV) (fmap WB.toMap wbs) - arenaManager <- newArenaManager + pagealloc <- newPageAlloc real <- lookupsIO hasBlockIO - arenaManager + pagealloc resolveV runs (V.map Run.runFilter runs) diff --git a/test/Test/Data/Arena.hs b/test/Test/Database/LSMTree/Internal/PageAlloc.hs similarity index 61% rename from test/Test/Data/Arena.hs rename to test/Test/Database/LSMTree/Internal/PageAlloc.hs index 633ea2ba2..2d50f66a6 100644 --- a/test/Test/Data/Arena.hs +++ b/test/Test/Database/LSMTree/Internal/PageAlloc.hs @@ -1,29 +1,34 @@ {-# LANGUAGE CPP #-} -module Test.Data.Arena ( +module Test.Database.LSMTree.Internal.PageAlloc ( tests, ) where import Control.Monad.ST (runST) -import Data.Arena import Data.Primitive.ByteArray import Data.Word (Word8) import GHC.Exts (toList) import Test.Tasty (TestTree, testGroup) import Test.Tasty.HUnit (testCaseSteps, (@?=)) +import Database.LSMTree.Internal.PageAlloc + tests :: TestTree -tests = testGroup "Test.Data.Arena" +tests = testGroup "Test.Database.LSMTree.Internal.PageAlloc" [ testCaseSteps "safe" $ \_info -> do - let !ba = runST $ withUnmanagedArena $ \arena -> do - (off', mba) <- allocateFromArena arena 32 8 + let !ba = runST $ do + pagealloc <- newPageAlloc + pages <- unmanagedAllocatePages pagealloc 1 + let (mba, off') = unsafeIndexPages pages 0 setByteArray mba off' 32 (1 :: Word8) freezeByteArray mba off' 32 toList ba @?= replicate 32 (1 :: Word8) , testCaseSteps "unsafe" $ \_info -> do - let !(off, ba) = runST $ withUnmanagedArena $ \arena -> do - (off', mba) <- allocateFromArena arena 32 8 + let !(off, ba) = runST $ do + pagealloc <- newPageAlloc + pages <- unmanagedAllocatePages pagealloc 1 + let (mba, off') = unsafeIndexPages pages 0 setByteArray mba off' 32 (1 :: Word8) ba' <- unsafeFreezeByteArray mba return (off', ba')