{- |
Module      : Database.LSMTree
Copyright   : (c) 2023, Input Output Global, Inc. (IOG)
              (c) 2023-2025, INTERSECT
License     : Apache-2.0
Stability   : experimental
Portability : portable
-}
module Database.LSMTree (
  -- * Usage Notes
  -- $usage_notes

  -- ** Real and Simulated IO
  -- $real_and_simulated_io
  IOLike,

  -- * Examples
  -- $setup

  -- * Sessions
  Session,
  withSession,
  withSessionIO,
  openSession,
  openSessionIO,
  closeSession,

  -- * Tables
  Table,
  withTable,
  withTableWith,
  newTable,
  newTableWith,
  closeTable,

  -- ** Table Lookups #table_lookups#
  member,
  members,
  LookupResult (..),
  getValue,
  getBlob,
  lookup,
  lookups,
  Entry (..),
  rangeLookup,

  -- ** Table Updates #table_updates#
  insert,
  inserts,
  upsert,
  upserts,
  delete,
  deletes,
  Update (..),
  update,
  updates,

  -- ** Table Duplication #table_duplication#
  withDuplicate,
  duplicate,

  -- ** Table Unions #table_unions#
  withUnion,
  withUnions,
  union,
  unions,
  withIncrementalUnion,
  withIncrementalUnions,
  incrementalUnion,
  incrementalUnions,
  remainingUnionDebt,
  supplyUnionCredits,

  -- * Blob References #blob_references#
  BlobRef,
  retrieveBlob,
  retrieveBlobs,

  -- * Cursors #cursor#
  Cursor,
  withCursor,
  withCursorAtOffset,
  newCursor,
  newCursorAtOffset,
  closeCursor,
  next,
  take,
  takeWhile,

  -- * Snapshots #snapshots#
  saveSnapshot,
  withTableFromSnapshot,
  withTableFromSnapshotWith,
  openTableFromSnapshot,
  openTableFromSnapshotWith,
  doesSnapshotExist,
  deleteSnapshot,
  listSnapshots,
  SnapshotName,
  isValidSnapshotName,
  toSnapshotName,
  SnapshotLabel (..),

  -- * Table Configuration #table_configuration#
  TableConfig (
    confMergePolicy,
    confSizeRatio,
    confWriteBufferAlloc,
    confBloomFilterAlloc,
    confFencePointerIndex,
    confDiskCachePolicy,
    confMergeSchedule
  ),
  defaultTableConfig,
  MergePolicy (LazyLevelling),
  MergeSchedule (..),
  SizeRatio (Four),
  WriteBufferAlloc (AllocNumEntries),
  BloomFilterAlloc (AllocFixed, AllocRequestFPR),
  FencePointerIndexType (OrdinaryIndex, CompactIndex),
  DiskCachePolicy (..),

  -- ** Table Configuration Overrides #table_configuration_overrides#
  OverrideDiskCachePolicy (..),

  -- * Ranges #ranges#
  Range (..),

  -- * Union Credit and Debt
  UnionCredits (..),
  UnionDebt (..),

  -- * Key\/Value Serialisation #key_value_serialisation#
  RawBytes (RawBytes),
  SerialiseKey (serialiseKey, deserialiseKey),
  SerialiseKeyOrderPreserving,
  SerialiseValue (serialiseValue, deserialiseValue),

  -- ** Key\/Value Serialisation Property Tests #key_value_serialisation_property_tests#
  serialiseKeyIdentity,
  serialiseKeyIdentityUpToSlicing,
  serialiseKeyPreservesOrdering,
  serialiseKeyMinimalSize,
  serialiseValueIdentity,
  serialiseValueIdentityUpToSlicing,
  packSlice,

  -- * Monoidal Value Resolution #monoidal_value_resolution#
  ResolveValue (..),
  ResolveViaSemigroup (..),
  ResolveAsFirst (..),

  -- ** Monoidal Value Resolution Property Tests #monoidal_value_resolution_property_tests#
  resolveCompatibility,
  resolveValidOutput,
  resolveAssociativity,

  -- * Errors #errors#
  SessionDirDoesNotExistError (..),
  SessionDirLockedError (..),
  SessionDirCorruptedError (..),
  SessionClosedError (..),
  TableClosedError (..),
  TableCorruptedError (..),
  TableTooLargeError (..),
  TableUnionNotCompatibleError (..),
  SnapshotExistsError (..),
  SnapshotDoesNotExistError (..),
  SnapshotCorruptedError (..),
  SnapshotNotCompatibleError (..),
  BlobRefInvalidError (..),
  CursorClosedError (..),
  InvalidSnapshotNameError (..),

  -- * Traces #traces#
  Tracer,
  LSMTreeTrace (..),
  TableTrace (..),
  CursorTrace (..),
  MergeTrace (..),
  CursorId (..),
  TableId (..),
  AtLevel (..),
  LevelNo (..),
  NumEntries (..),
  RunNumber (..),
  MergePolicyForLevel (..),
  LevelMergeType (..),
  RunParams (..),
  RunDataCaching (..),
  IndexType (..),
) where

import           Control.Concurrent.Class.MonadMVar.Strict (MonadMVar)
import           Control.Concurrent.Class.MonadSTM (MonadSTM (STM))
import           Control.DeepSeq (NFData (..))
import           Control.Exception.Base (assert)
import           Control.Monad.Class.MonadAsync (MonadAsync)
import           Control.Monad.Class.MonadST (MonadST)
import           Control.Monad.Class.MonadThrow (MonadCatch (..), MonadMask,
                     MonadThrow (..))
import           Control.Monad.Primitive (PrimMonad)
import           Control.Tracer (Tracer)
import           Data.Bifunctor (Bifunctor (..))
import           Data.Coerce (coerce)
import           Data.Kind (Constraint, Type)
import           Data.List.NonEmpty (NonEmpty (..))
import           Data.Maybe (fromMaybe, isJust)
import           Data.Typeable (Proxy (..), Typeable, eqT, type (:~:) (Refl),
                     typeRep)
import           Data.Vector (Vector)
import qualified Data.Vector as V
import qualified Database.LSMTree.Internal.BlobRef as Internal
import           Database.LSMTree.Internal.Config
                     (BloomFilterAlloc (AllocFixed, AllocRequestFPR),
                     DiskCachePolicy (..), FencePointerIndexType (..),
                     LevelNo (..), MergePolicy (..), MergeSchedule (..),
                     SizeRatio (..), TableConfig (..), WriteBufferAlloc (..),
                     defaultTableConfig, serialiseKeyMinimalSize)
import           Database.LSMTree.Internal.Config.Override
                     (OverrideDiskCachePolicy (..))
import           Database.LSMTree.Internal.Entry (NumEntries (..))
import qualified Database.LSMTree.Internal.Entry as Entry
import           Database.LSMTree.Internal.Merge (LevelMergeType (..))
import           Database.LSMTree.Internal.MergeSchedule (AtLevel (..),
                     MergePolicyForLevel (..), MergeTrace (..))
import           Database.LSMTree.Internal.Paths (SnapshotName,
                     isValidSnapshotName, toSnapshotName)
import           Database.LSMTree.Internal.Range (Range (..))
import           Database.LSMTree.Internal.RawBytes (RawBytes (..))
import           Database.LSMTree.Internal.RunBuilder (IndexType (..),
                     RunDataCaching (..), RunParams (..))
import           Database.LSMTree.Internal.RunNumber (CursorId (..),
                     RunNumber (..), TableId (..))
import qualified Database.LSMTree.Internal.Serialise as Internal
import           Database.LSMTree.Internal.Serialise.Class (SerialiseKey (..),
                     SerialiseKeyOrderPreserving, SerialiseValue (..),
                     packSlice, serialiseKeyIdentity,
                     serialiseKeyIdentityUpToSlicing,
                     serialiseKeyPreservesOrdering, serialiseValueIdentity,
                     serialiseValueIdentityUpToSlicing)
import           Database.LSMTree.Internal.Snapshot (SnapshotLabel (..))
import           Database.LSMTree.Internal.Types (BlobRef (..), Cursor (..),
                     ResolveAsFirst (..), ResolveValue (..),
                     ResolveViaSemigroup (..), Session (..), Table (..),
                     resolveAssociativity, resolveCompatibility,
                     resolveValidOutput)
import           Database.LSMTree.Internal.Unsafe (BlobRefInvalidError (..),
                     CursorClosedError (..), CursorTrace,
                     InvalidSnapshotNameError (..), LSMTreeTrace (..),
                     ResolveSerialisedValue, SessionClosedError (..),
                     SessionDirCorruptedError (..),
                     SessionDirDoesNotExistError (..),
                     SessionDirLockedError (..), SnapshotCorruptedError (..),
                     SnapshotDoesNotExistError (..), SnapshotExistsError (..),
                     SnapshotNotCompatibleError (..), TableClosedError (..),
                     TableCorruptedError (..), TableTooLargeError (..),
                     TableTrace, TableUnionNotCompatibleError (..),
                     UnionCredits (..), UnionDebt (..))
import qualified Database.LSMTree.Internal.Unsafe as Internal
import           Prelude hiding (lookup, take, takeWhile)
import           System.FS.API (FsPath, HasFS (..), MountPoint (..), mkFsPath)
import           System.FS.BlockIO.API (HasBlockIO (..), defaultIOCtxParams)
import           System.FS.BlockIO.IO (ioHasBlockIO, withIOHasBlockIO)
import           System.FS.IO (HandleIO, ioHasFS)

--------------------------------------------------------------------------------
-- Usage Notes
--------------------------------------------------------------------------------

{- $usage_notes
This section focuses on the differences between the full API as defined in this module and the simple API as defined in "Database.LSMTree.Simple".
It assumes that the reader is familiar with [Usage Notes for the simple API]("Database.LSMTree.Simple#g:usage_notes"), which discusses crucial topics such as [Resource Management]("Database.LSMTree.Simple#g:resource_management"), [Concurrency]("Database.LSMTree.Simple#g:concurrency"), [ACID properties]("Database.LSMTree.Simple#g:acid"), and [Sharing]("Database.LSMTree.Simple#g:sharing").
-}

{- $real_and_simulated_io
-}

type IOLike :: (Type -> Type) -> Constraint
type IOLike m =
  ( MonadAsync m
  , MonadMVar m
  , MonadThrow m
  , MonadThrow (STM m)
  , MonadCatch m
  , MonadMask m
  , PrimMonad m
  , MonadST m
  )

--------------------------------------------------------------------------------
-- Example
--------------------------------------------------------------------------------

{- $setup

The examples in this module use the preamble described in this section, which does three things:

1.  It imports this module qualified, as intended, as well as any other relevant modules.
2.  It defines types for keys, values, and BLOBs.
3.  It defines a helper function that runs examples with access to an open session and fresh table.

=== Importing "Database.LSMTree"

This module is intended to be imported qualified, to avoid name clashes with Prelude functions.

>>> import           Database.LSMTree (BlobRef, Cursor, RawBytes, ResolveValue (..), SerialiseKey (..), SerialiseValue (..), Session, Table)
>>> import qualified Database.LSMTree as LSMT

=== Defining key, value, and BLOB types

The examples in this module use the types @Key@, @Value@, and @Blob@ for keys, values and BLOBs.

>>> import Data.ByteString (ByteString)
>>> import Data.ByteString.Short (ShortByteString)
>>> import Data.Proxy (Proxy)
>>> import Data.String (IsString)
>>> import Data.Word (Word64)

The type @Key@ is a newtype wrapper around 'Data.Word.Word64'.
The required instance of 'SerialiseKey' is derived by @GeneralisedNewtypeDeriving@ from the preexisting instance for 'Data.Word.Word64'.

>>> :{
newtype Key = Key Word64
  deriving stock (Eq, Ord, Show)
  deriving newtype (Num, SerialiseKey)
:}

The type @Value@ is a newtype wrapper around 'Data.ByteString.Short.ShortByteString'.
The required instance of 'SerialiseValue' is derived by @GeneralisedNewtypeDeriving@ from the preexisting instance for 'Data.ByteString.Short.ShortByteString'.

>>> :{
newtype Value = Value ShortByteString
  deriving stock (Eq, Show)
  deriving newtype (IsString, SerialiseValue)
:}

The type @Value@ has an instance of @ResolveValue@ which appends the new value to the old value separated by a space.
It is sufficient to define either 'resolve' or 'resolveSerialised',
as each can be defined in terms of the other and 'serialiseValue'\/'deserialiseValue'.
For optimal performance, you should /always/ define 'resolveSerialised' manually.

__NOTE__:
The /first/ argument of 'resolve' and 'resolveSerialised' is the /new/ value and the /second/ argument is the /old/ value.

>>> :{
instance ResolveValue Value where
  resolve :: Value -> Value -> Value
  resolve (Value new) (Value old) = Value (new <> " " <> old)
  resolveSerialised :: Proxy Value -> RawBytes -> RawBytes -> RawBytes
  resolveSerialised _ new old = new <> " " <> old
:}

The type @Blob@ is a newtype wrapper around 'Data.ByteString.ByteString',
The required instance of 'SerialiseValue' is derived by @GeneralisedNewtypeDeriving@ from the preexisting instance for 'Data.ByteString.ByteString'.

>>> :{
newtype Blob = Blob ByteString
  deriving stock (Eq, Show)
  deriving newtype (IsString, SerialiseValue)
:}

=== Defining a helper function to run examples

The examples in this module are wrapped in a call to @runExample@,
which creates a temporary session directory and
runs the example with access to an open 'Session' and a fresh 'Table'.

>>> import           Control.Exception (bracket, bracket_)
>>> import           Data.Foldable (traverse_)
>>> import qualified System.Directory as Dir
>>> import           System.FilePath ((</>))
>>> import           System.Process (getCurrentPid)
>>> :{
runExample :: (Session IO -> Table IO Key Value Blob -> IO a) -> IO a
runExample action = do
  tmpDir <- Dir.getTemporaryDirectory
  pid <- getCurrentPid
  let sessionDir = tmpDir </> "doctest_Database_LSMTree" </> show pid
  let createSessionDir = Dir.createDirectoryIfMissing True sessionDir
  let removeSessionDir = Dir.removeDirectoryRecursive sessionDir
  bracket_ createSessionDir removeSessionDir $ do
    LSMT.withSessionIO mempty sessionDir $ \session -> do
      LSMT.withTable session $ \table ->
        action session table
:}
-}

--------------------------------------------------------------------------------
-- Sessions
--------------------------------------------------------------------------------

-- NOTE: 'Session' is defined in 'Database.LSMTree.Internal.Types'

{- |
Run an action with access to a session opened from a session directory.

If the session directory is empty, a new session is created.
Otherwise, the session directory is opened as an existing session.

If there are no open tables or cursors when the session terminates, then the disk I\/O complexity of this operation is \(O(1)\).
Otherwise, 'closeTable' is called for each open table and 'closeCursor' is called for each open cursor.
Consequently, the worst-case disk I\/O complexity of this operation depends on the merge policy of the open tables in the session.
The following assumes all tables in the session have the same merge policy:

['LazyLevelling']:
  \(O(o \: T \log_T \frac{n}{B})\).

The variable \(o\) refers to the number of open tables and cursors in the session.

This function is exception-safe for both synchronous and asynchronous exceptions.

It is recommended to use this function instead of 'openSession' and 'closeSession'.

Throws the following exceptions:

['SessionDirDoesNotExistError']:
    If the session directory does not exist.
['SessionDirLockedError']:
    If the session directory is locked by another process.
['SessionDirCorruptedError']:
    If the session directory is malformed.
-}
{-# SPECIALISE
  withSession ::
    Tracer IO LSMTreeTrace ->
    HasFS IO HandleIO ->
    HasBlockIO IO HandleIO ->
    FsPath ->
    (Session IO -> IO a) ->
    IO a
  #-}
withSession ::
  forall m h a.
  (IOLike m, Typeable h) =>
  Tracer m LSMTreeTrace ->
  HasFS m h ->
  HasBlockIO m h ->
  -- | The session directory.
  FsPath ->
  (Session m -> m a) ->
  m a
withSession :: forall (m :: * -> *) h a.
(IOLike m, Typeable h) =>
Tracer m LSMTreeTrace
-> HasFS m h
-> HasBlockIO m h
-> FsPath
-> (Session m -> m a)
-> m a
withSession Tracer m LSMTreeTrace
tracer HasFS m h
hasFS HasBlockIO m h
hasBlockIO FsPath
sessionDir Session m -> m a
action = do
  Tracer m LSMTreeTrace
-> HasFS m h
-> HasBlockIO m h
-> FsPath
-> (Session m h -> m a)
-> m a
forall (m :: * -> *) h a.
(MonadMask m, MonadSTM m, MonadMVar m, PrimMonad m) =>
Tracer m LSMTreeTrace
-> HasFS m h
-> HasBlockIO m h
-> FsPath
-> (Session m h -> m a)
-> m a
Internal.withSession Tracer m LSMTreeTrace
tracer HasFS m h
hasFS HasBlockIO m h
hasBlockIO FsPath
sessionDir (Session m -> m a
action (Session m -> m a)
-> (Session m h -> Session m) -> Session m h -> m a
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Session m h -> Session m
forall (m :: * -> *) h. Typeable h => Session m h -> Session m
Session)

-- | Variant of 'withSession' that is specialised to 'IO' using the real filesystem.
withSessionIO ::
  Tracer IO LSMTreeTrace ->
  FilePath ->
  (Session IO -> IO a) ->
  IO a
withSessionIO :: forall a.
Tracer IO LSMTreeTrace -> FilePath -> (Session IO -> IO a) -> IO a
withSessionIO Tracer IO LSMTreeTrace
tracer FilePath
sessionDir Session IO -> IO a
action = do
  let mountPoint :: MountPoint
mountPoint = FilePath -> MountPoint
MountPoint FilePath
sessionDir
  let sessionDirFsPath :: FsPath
sessionDirFsPath = [FilePath] -> FsPath
mkFsPath []
  let hasFS :: HasFS IO HandleIO
hasFS = MountPoint -> HasFS IO HandleIO
forall (m :: * -> *).
(MonadIO m, PrimState IO ~ PrimState m) =>
MountPoint -> HasFS m HandleIO
ioHasFS MountPoint
mountPoint
  HasFS IO HandleIO
-> IOCtxParams -> (HasBlockIO IO HandleIO -> IO a) -> IO a
forall a.
HasFS IO HandleIO
-> IOCtxParams -> (HasBlockIO IO HandleIO -> IO a) -> IO a
withIOHasBlockIO HasFS IO HandleIO
hasFS IOCtxParams
defaultIOCtxParams ((HasBlockIO IO HandleIO -> IO a) -> IO a)
-> (HasBlockIO IO HandleIO -> IO a) -> IO a
forall a b. (a -> b) -> a -> b
$ \HasBlockIO IO HandleIO
hasBlockIO ->
    Tracer IO LSMTreeTrace
-> HasFS IO HandleIO
-> HasBlockIO IO HandleIO
-> FsPath
-> (Session IO -> IO a)
-> IO a
forall (m :: * -> *) h a.
(IOLike m, Typeable h) =>
Tracer m LSMTreeTrace
-> HasFS m h
-> HasBlockIO m h
-> FsPath
-> (Session m -> m a)
-> m a
withSession Tracer IO LSMTreeTrace
tracer HasFS IO HandleIO
hasFS HasBlockIO IO HandleIO
hasBlockIO FsPath
sessionDirFsPath Session IO -> IO a
action

{- |
Open a session from a session directory.

If the session directory is empty, a new session is created.
Otherwise, the session directory is opened as an existing session.

The worst-case disk I\/O complexity of this operation is \(O(1)\).

__Warning:__ Sessions hold open resources and must be closed using 'closeSession'.

Throws the following exceptions:

['SessionDirDoesNotExistError']:
    If the session directory does not exist.
['SessionDirLockedError']:
    If the session directory is locked by another process.
['SessionDirCorruptedError']:
    If the session directory is malformed.
-}
{-# SPECIALISE
  openSession ::
    Tracer IO LSMTreeTrace ->
    HasFS IO HandleIO ->
    HasBlockIO IO HandleIO ->
    -- \| The session directory.
    FsPath ->
    IO (Session IO)
  #-}
openSession ::
  forall m h.
  (IOLike m, Typeable h) =>
  Tracer m LSMTreeTrace ->
  HasFS m h ->
  HasBlockIO m h ->
  -- | The session directory.
  FsPath ->
  m (Session m)
openSession :: forall (m :: * -> *) h.
(IOLike m, Typeable h) =>
Tracer m LSMTreeTrace
-> HasFS m h -> HasBlockIO m h -> FsPath -> m (Session m)
openSession Tracer m LSMTreeTrace
tracer HasFS m h
hasFS HasBlockIO m h
hasBlockIO FsPath
sessionDir =
  Session m h -> Session m
forall (m :: * -> *) h. Typeable h => Session m h -> Session m
Session (Session m h -> Session m) -> m (Session m h) -> m (Session m)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Tracer m LSMTreeTrace
-> HasFS m h -> HasBlockIO m h -> FsPath -> m (Session m h)
forall (m :: * -> *) h.
(MonadSTM m, MonadMVar m, PrimMonad m, MonadMask m) =>
Tracer m LSMTreeTrace
-> HasFS m h -> HasBlockIO m h -> FsPath -> m (Session m h)
Internal.openSession Tracer m LSMTreeTrace
tracer HasFS m h
hasFS HasBlockIO m h
hasBlockIO FsPath
sessionDir

-- | Variant of 'openSession' that is specialised to 'IO' using the real filesystem.
openSessionIO ::
  Tracer IO LSMTreeTrace ->
  -- \| The session directory.
  FilePath ->
  IO (Session IO)
openSessionIO :: Tracer IO LSMTreeTrace -> FilePath -> IO (Session IO)
openSessionIO Tracer IO LSMTreeTrace
tracer FilePath
sessionDir = do
  let mountPoint :: MountPoint
mountPoint = FilePath -> MountPoint
MountPoint FilePath
sessionDir
  let sessionDirFsPath :: FsPath
sessionDirFsPath = [FilePath] -> FsPath
mkFsPath []
  let hasFS :: HasFS IO HandleIO
hasFS = MountPoint -> HasFS IO HandleIO
forall (m :: * -> *).
(MonadIO m, PrimState IO ~ PrimState m) =>
MountPoint -> HasFS m HandleIO
ioHasFS MountPoint
mountPoint
  let acquireHasBlockIO :: IO (HasBlockIO IO HandleIO)
acquireHasBlockIO = HasFS IO HandleIO -> IOCtxParams -> IO (HasBlockIO IO HandleIO)
ioHasBlockIO HasFS IO HandleIO
hasFS IOCtxParams
defaultIOCtxParams
  let releaseHasBlockIO :: HasBlockIO m h -> m ()
releaseHasBlockIO HasBlockIO{HasCallStack => m ()
close :: HasCallStack => m ()
close :: forall (m :: * -> *) h. HasBlockIO m h -> HasCallStack => m ()
close} = m ()
HasCallStack => m ()
close
  IO (HasBlockIO IO HandleIO)
-> (HasBlockIO IO HandleIO -> IO ())
-> (HasBlockIO IO HandleIO -> IO (Session IO))
-> IO (Session IO)
forall a b c. IO a -> (a -> IO b) -> (a -> IO c) -> IO c
forall (m :: * -> *) a b c.
MonadCatch m =>
m a -> (a -> m b) -> (a -> m c) -> m c
bracketOnError IO (HasBlockIO IO HandleIO)
acquireHasBlockIO HasBlockIO IO HandleIO -> IO ()
forall {m :: * -> *} {h}. HasBlockIO m h -> m ()
releaseHasBlockIO ((HasBlockIO IO HandleIO -> IO (Session IO)) -> IO (Session IO))
-> (HasBlockIO IO HandleIO -> IO (Session IO)) -> IO (Session IO)
forall a b. (a -> b) -> a -> b
$ \HasBlockIO IO HandleIO
hasBlockIO ->
    Tracer IO LSMTreeTrace
-> HasFS IO HandleIO
-> HasBlockIO IO HandleIO
-> FsPath
-> IO (Session IO)
forall (m :: * -> *) h.
(IOLike m, Typeable h) =>
Tracer m LSMTreeTrace
-> HasFS m h -> HasBlockIO m h -> FsPath -> m (Session m)
openSession Tracer IO LSMTreeTrace
tracer HasFS IO HandleIO
hasFS HasBlockIO IO HandleIO
hasBlockIO FsPath
sessionDirFsPath

{- |
Close a session.

If there are no open tables or cursors in the session, then the disk I\/O complexity of this operation is \(O(1)\).
Otherwise, 'closeTable' is called for each open table and 'closeCursor' is called for each open cursor.
Consequently, the worst-case disk I\/O complexity of this operation depends on the merge policy of the tables in the session.
The following assumes all tables in the session have the same merge policy:

['LazyLevelling']:
  \(O(o \: T \log_T \frac{n}{B})\).

The variable \(o\) refers to the number of open tables and cursors in the session.

Closing is idempotent, i.e., closing a closed session does nothing.
All other operations on a closed session will throw an exception.
-}
{-# SPECIALISE
  closeSession ::
    Session IO ->
    IO ()
  #-}
closeSession ::
  forall m.
  (IOLike m) =>
  Session m ->
  m ()
closeSession :: forall (m :: * -> *). IOLike m => Session m -> m ()
closeSession (Session Session m h
session) =
  Session m h -> m ()
forall (m :: * -> *) h.
(MonadMask m, MonadSTM m, MonadMVar m, PrimMonad m) =>
Session m h -> m ()
Internal.closeSession Session m h
session

--------------------------------------------------------------------------------
-- Tables
--------------------------------------------------------------------------------

-- NOTE: 'Table' is defined in 'Database.LSMTree.Internal.Types'

{- |
Run an action with access to an empty table.

The worst-case disk I\/O complexity of this operation depends on the merge policy of the table:

['LazyLevelling']:
    \(O(T \log_T \frac{n}{B})\).

This function is exception-safe for both synchronous and asynchronous exceptions.

It is recommended to use this function instead of 'newTable' and 'closeTable'.

Throws the following exceptions:

['SessionClosedError']:
    If the session is closed.
-}
{-# SPECIALISE
  withTable ::
    Session IO ->
    (Table IO k v b -> IO a) ->
    IO a
  #-}
withTable ::
  forall m k v b a.
  (IOLike m) =>
  Session m ->
  (Table m k v b -> m a) ->
  m a
withTable :: forall (m :: * -> *) k v b a.
IOLike m =>
Session m -> (Table m k v b -> m a) -> m a
withTable Session m
session =
  TableConfig -> Session m -> (Table m k v b -> m a) -> m a
forall (m :: * -> *) k v b a.
IOLike m =>
TableConfig -> Session m -> (Table m k v b -> m a) -> m a
withTableWith TableConfig
defaultTableConfig Session m
session

-- | Variant of 'withTable' that accepts [table configuration](#g:table_configuration).
{-# SPECIALISE
  withTableWith ::
    TableConfig ->
    Session IO ->
    (Table IO k v b -> IO a) ->
    IO a
  #-}
withTableWith ::
  forall m k v b a.
  (IOLike m) =>
  TableConfig ->
  Session m ->
  (Table m k v b -> m a) ->
  m a
withTableWith :: forall (m :: * -> *) k v b a.
IOLike m =>
TableConfig -> Session m -> (Table m k v b -> m a) -> m a
withTableWith TableConfig
tableConfig (Session Session m h
session) Table m k v b -> m a
action =
  Session m h -> TableConfig -> (Table m h -> m a) -> m a
forall (m :: * -> *) h a.
(MonadMask m, MonadSTM m, MonadMVar m, PrimMonad m) =>
Session m h -> TableConfig -> (Table m h -> m a) -> m a
Internal.withTable Session m h
session TableConfig
tableConfig (Table m k v b -> m a
action (Table m k v b -> m a)
-> (Table m h -> Table m k v b) -> Table m h -> m a
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Table m h -> Table m k v b
forall (m :: * -> *) k v b h.
Typeable h =>
Table m h -> Table m k v b
Table)

{- |
Create an empty table.

The worst-case disk I\/O complexity of this operation is \(O(1)\).

__Warning:__ Tables hold open resources and must be closed using 'closeTable'.

Throws the following exceptions:

['SessionClosedError']:
    If the session is closed.
-}
{-# SPECIALISE
  newTable ::
    Session IO ->
    IO (Table IO k v b)
  #-}
newTable ::
  forall m k v b.
  (IOLike m) =>
  Session m ->
  m (Table m k v b)
newTable :: forall (m :: * -> *) k v b.
IOLike m =>
Session m -> m (Table m k v b)
newTable Session m
session =
  TableConfig -> Session m -> m (Table m k v b)
forall (m :: * -> *) k v b.
IOLike m =>
TableConfig -> Session m -> m (Table m k v b)
newTableWith TableConfig
defaultTableConfig Session m
session

{- |
Variant of 'newTable' that accepts [table configuration](#g:table_configuration).
-}
{-# SPECIALISE
  newTableWith ::
    TableConfig ->
    Session IO ->
    IO (Table IO k v b)
  #-}
newTableWith ::
  forall m k v b.
  (IOLike m) =>
  TableConfig ->
  Session m ->
  m (Table m k v b)
newTableWith :: forall (m :: * -> *) k v b.
IOLike m =>
TableConfig -> Session m -> m (Table m k v b)
newTableWith TableConfig
tableConfig (Session Session m h
session) =
  Table m h -> Table m k v b
forall (m :: * -> *) k v b h.
Typeable h =>
Table m h -> Table m k v b
Table (Table m h -> Table m k v b) -> m (Table m h) -> m (Table m k v b)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Session m h -> TableConfig -> m (Table m h)
forall (m :: * -> *) h.
(MonadSTM m, MonadMVar m, PrimMonad m, MonadMask m) =>
Session m h -> TableConfig -> m (Table m h)
Internal.new Session m h
session TableConfig
tableConfig

{- |
Close a table.

The worst-case disk I\/O complexity of this operation depends on the merge policy of the table:

['LazyLevelling']:
    \(O(T \log_T \frac{n}{B})\).

Closing is idempotent, i.e., closing a closed table does nothing.
All other operations on a closed table will throw an exception.

__Warning:__ Tables are ephemeral. Once you close a table, its data is lost forever. To persist tables, use [snapshots](#g:snapshots).
-}
{-# SPECIALISE
  closeTable ::
    Table IO k v b ->
    IO ()
  #-}
closeTable ::
  forall m k v b.
  (IOLike m) =>
  Table m k v b ->
  m ()
closeTable :: forall (m :: * -> *) k v b. IOLike m => Table m k v b -> m ()
closeTable (Table Table m h
table) =
  Table m h -> m ()
forall (m :: * -> *) h.
(MonadMask m, MonadSTM m, MonadMVar m, PrimMonad m) =>
Table m h -> m ()
Internal.close Table m h
table

--------------------------------------------------------------------------------
-- Lookups
--------------------------------------------------------------------------------

{- |
Check if the key is a member of the table.

>>> :{
runExample $ \session table -> do
  LSMT.insert table 0 "Hello" Nothing
  LSMT.insert table 1 "World" Nothing
  print =<< LSMT.member table 0
:}
True

The worst-case disk I\/O complexity of this operation depends on the merge policy of the table:

['LazyLevelling']:
    \(O(T \log_T \frac{n}{B})\).

Membership tests can be performed concurrently from multiple Haskell threads.

Throws the following exceptions:

['SessionClosedError']:
    If the session is closed.
['TableClosedError']:
    If the table is closed.
['TableCorruptedError']:
    If the table data is corrupted.
-}
{-# SPECIALISE
  member ::
    (SerialiseKey k, SerialiseValue v, ResolveValue v) =>
    Table IO k v b ->
    k ->
    IO Bool
  #-}
member ::
  forall m k v b.
  (IOLike m) =>
  (SerialiseKey k, SerialiseValue v, ResolveValue v) =>
  Table m k v b ->
  k ->
  m Bool
member :: forall (m :: * -> *) k v b.
(IOLike m, SerialiseKey k, SerialiseValue v, ResolveValue v) =>
Table m k v b -> k -> m Bool
member =
  -- Technically, this does not need the 'SerialiseValue' constraint.
  ((LookupResult v (BlobRef m b) -> Bool)
-> m (LookupResult v (BlobRef m b)) -> m Bool
forall a b. (a -> b) -> m a -> m b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap (Maybe v -> Bool
forall a. Maybe a -> Bool
isJust (Maybe v -> Bool)
-> (LookupResult v (BlobRef m b) -> Maybe v)
-> LookupResult v (BlobRef m b)
-> Bool
forall b c a. (b -> c) -> (a -> b) -> a -> c
. LookupResult v (BlobRef m b) -> Maybe v
forall v b. LookupResult v b -> Maybe v
getValue) .) ((k -> m (LookupResult v (BlobRef m b))) -> k -> m Bool)
-> (Table m k v b -> k -> m (LookupResult v (BlobRef m b)))
-> Table m k v b
-> k
-> m Bool
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Table m k v b -> k -> m (LookupResult v (BlobRef m b))
forall (m :: * -> *) k v b.
(IOLike m, SerialiseKey k, SerialiseValue v, ResolveValue v) =>
Table m k v b -> k -> m (LookupResult v (BlobRef m b))
lookup

{- |
Variant of 'member' for batch membership tests.
The batch of keys corresponds in-order to the batch of results.

The worst-case disk I\/O complexity of this operation depends on the merge policy of the table:

['LazyLevelling']:
    \(O(b \: T \log_T \frac{n}{B})\).

The variable \(b\) refers to the length of the input vector.

The following property holds in the absence of races:

prop> members table keys = traverse (member table) keys
-}
{-# SPECIALISE
  members ::
    (SerialiseKey k, SerialiseValue v, ResolveValue v) =>
    Table IO k v b ->
    Vector k ->
    IO (Vector Bool)
  #-}
members ::
  forall m k v b.
  (IOLike m) =>
  (SerialiseKey k, SerialiseValue v, ResolveValue v) =>
  Table m k v b ->
  Vector k ->
  m (Vector Bool)
members :: forall (m :: * -> *) k v b.
(IOLike m, SerialiseKey k, SerialiseValue v, ResolveValue v) =>
Table m k v b -> Vector k -> m (Vector Bool)
members =
  -- Technically, this does not need the 'SerialiseValue' constraint.
  ((Vector (LookupResult v (BlobRef m b)) -> Vector Bool)
-> m (Vector (LookupResult v (BlobRef m b))) -> m (Vector Bool)
forall a b. (a -> b) -> m a -> m b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap ((LookupResult v (BlobRef m b) -> Bool)
-> Vector (LookupResult v (BlobRef m b)) -> Vector Bool
forall a b. (a -> b) -> Vector a -> Vector b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap (Maybe v -> Bool
forall a. Maybe a -> Bool
isJust (Maybe v -> Bool)
-> (LookupResult v (BlobRef m b) -> Maybe v)
-> LookupResult v (BlobRef m b)
-> Bool
forall b c a. (b -> c) -> (a -> b) -> a -> c
. LookupResult v (BlobRef m b) -> Maybe v
forall v b. LookupResult v b -> Maybe v
getValue)) .) ((Vector k -> m (Vector (LookupResult v (BlobRef m b))))
 -> Vector k -> m (Vector Bool))
-> (Table m k v b
    -> Vector k -> m (Vector (LookupResult v (BlobRef m b))))
-> Table m k v b
-> Vector k
-> m (Vector Bool)
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Table m k v b
-> Vector k -> m (Vector (LookupResult v (BlobRef m b)))
forall (m :: * -> *) k v b.
(IOLike m, SerialiseKey k, SerialiseValue v, ResolveValue v) =>
Table m k v b
-> Vector k -> m (Vector (LookupResult v (BlobRef m b)))
lookups

data LookupResult v b
  = NotFound
  | Found !v
  | FoundWithBlob !v !b
  deriving stock (LookupResult v b -> LookupResult v b -> Bool
(LookupResult v b -> LookupResult v b -> Bool)
-> (LookupResult v b -> LookupResult v b -> Bool)
-> Eq (LookupResult v b)
forall a. (a -> a -> Bool) -> (a -> a -> Bool) -> Eq a
forall v b.
(Eq v, Eq b) =>
LookupResult v b -> LookupResult v b -> Bool
$c== :: forall v b.
(Eq v, Eq b) =>
LookupResult v b -> LookupResult v b -> Bool
== :: LookupResult v b -> LookupResult v b -> Bool
$c/= :: forall v b.
(Eq v, Eq b) =>
LookupResult v b -> LookupResult v b -> Bool
/= :: LookupResult v b -> LookupResult v b -> Bool
Eq, Int -> LookupResult v b -> ShowS
[LookupResult v b] -> ShowS
LookupResult v b -> FilePath
(Int -> LookupResult v b -> ShowS)
-> (LookupResult v b -> FilePath)
-> ([LookupResult v b] -> ShowS)
-> Show (LookupResult v b)
forall a.
(Int -> a -> ShowS) -> (a -> FilePath) -> ([a] -> ShowS) -> Show a
forall v b. (Show v, Show b) => Int -> LookupResult v b -> ShowS
forall v b. (Show v, Show b) => [LookupResult v b] -> ShowS
forall v b. (Show v, Show b) => LookupResult v b -> FilePath
$cshowsPrec :: forall v b. (Show v, Show b) => Int -> LookupResult v b -> ShowS
showsPrec :: Int -> LookupResult v b -> ShowS
$cshow :: forall v b. (Show v, Show b) => LookupResult v b -> FilePath
show :: LookupResult v b -> FilePath
$cshowList :: forall v b. (Show v, Show b) => [LookupResult v b] -> ShowS
showList :: [LookupResult v b] -> ShowS
Show, (forall a b. (a -> b) -> LookupResult v a -> LookupResult v b)
-> (forall a b. a -> LookupResult v b -> LookupResult v a)
-> Functor (LookupResult v)
forall a b. a -> LookupResult v b -> LookupResult v a
forall a b. (a -> b) -> LookupResult v a -> LookupResult v b
forall v a b. a -> LookupResult v b -> LookupResult v a
forall v a b. (a -> b) -> LookupResult v a -> LookupResult v b
forall (f :: * -> *).
(forall a b. (a -> b) -> f a -> f b)
-> (forall a b. a -> f b -> f a) -> Functor f
$cfmap :: forall v a b. (a -> b) -> LookupResult v a -> LookupResult v b
fmap :: forall a b. (a -> b) -> LookupResult v a -> LookupResult v b
$c<$ :: forall v a b. a -> LookupResult v b -> LookupResult v a
<$ :: forall a b. a -> LookupResult v b -> LookupResult v a
Functor, (forall m. Monoid m => LookupResult v m -> m)
-> (forall m a. Monoid m => (a -> m) -> LookupResult v a -> m)
-> (forall m a. Monoid m => (a -> m) -> LookupResult v a -> m)
-> (forall a b. (a -> b -> b) -> b -> LookupResult v a -> b)
-> (forall a b. (a -> b -> b) -> b -> LookupResult v a -> b)
-> (forall b a. (b -> a -> b) -> b -> LookupResult v a -> b)
-> (forall b a. (b -> a -> b) -> b -> LookupResult v a -> b)
-> (forall a. (a -> a -> a) -> LookupResult v a -> a)
-> (forall a. (a -> a -> a) -> LookupResult v a -> a)
-> (forall a. LookupResult v a -> [a])
-> (forall a. LookupResult v a -> Bool)
-> (forall a. LookupResult v a -> Int)
-> (forall a. Eq a => a -> LookupResult v a -> Bool)
-> (forall a. Ord a => LookupResult v a -> a)
-> (forall a. Ord a => LookupResult v a -> a)
-> (forall a. Num a => LookupResult v a -> a)
-> (forall a. Num a => LookupResult v a -> a)
-> Foldable (LookupResult v)
forall a. Eq a => a -> LookupResult v a -> Bool
forall a. Num a => LookupResult v a -> a
forall a. Ord a => LookupResult v a -> a
forall m. Monoid m => LookupResult v m -> m
forall a. LookupResult v a -> Bool
forall a. LookupResult v a -> Int
forall a. LookupResult v a -> [a]
forall a. (a -> a -> a) -> LookupResult v a -> a
forall v a. Eq a => a -> LookupResult v a -> Bool
forall v a. Num a => LookupResult v a -> a
forall v a. Ord a => LookupResult v a -> a
forall m a. Monoid m => (a -> m) -> LookupResult v a -> m
forall v m. Monoid m => LookupResult v m -> m
forall v a. LookupResult v a -> Bool
forall v a. LookupResult v a -> Int
forall v a. LookupResult v a -> [a]
forall b a. (b -> a -> b) -> b -> LookupResult v a -> b
forall a b. (a -> b -> b) -> b -> LookupResult v a -> b
forall v a. (a -> a -> a) -> LookupResult v a -> a
forall v m a. Monoid m => (a -> m) -> LookupResult v a -> m
forall v b a. (b -> a -> b) -> b -> LookupResult v a -> b
forall v a b. (a -> b -> b) -> b -> LookupResult v a -> b
forall (t :: * -> *).
(forall m. Monoid m => t m -> m)
-> (forall m a. Monoid m => (a -> m) -> t a -> m)
-> (forall m a. Monoid m => (a -> m) -> t a -> m)
-> (forall a b. (a -> b -> b) -> b -> t a -> b)
-> (forall a b. (a -> b -> b) -> b -> t a -> b)
-> (forall b a. (b -> a -> b) -> b -> t a -> b)
-> (forall b a. (b -> a -> b) -> b -> t a -> b)
-> (forall a. (a -> a -> a) -> t a -> a)
-> (forall a. (a -> a -> a) -> t a -> a)
-> (forall a. t a -> [a])
-> (forall a. t a -> Bool)
-> (forall a. t a -> Int)
-> (forall a. Eq a => a -> t a -> Bool)
-> (forall a. Ord a => t a -> a)
-> (forall a. Ord a => t a -> a)
-> (forall a. Num a => t a -> a)
-> (forall a. Num a => t a -> a)
-> Foldable t
$cfold :: forall v m. Monoid m => LookupResult v m -> m
fold :: forall m. Monoid m => LookupResult v m -> m
$cfoldMap :: forall v m a. Monoid m => (a -> m) -> LookupResult v a -> m
foldMap :: forall m a. Monoid m => (a -> m) -> LookupResult v a -> m
$cfoldMap' :: forall v m a. Monoid m => (a -> m) -> LookupResult v a -> m
foldMap' :: forall m a. Monoid m => (a -> m) -> LookupResult v a -> m
$cfoldr :: forall v a b. (a -> b -> b) -> b -> LookupResult v a -> b
foldr :: forall a b. (a -> b -> b) -> b -> LookupResult v a -> b
$cfoldr' :: forall v a b. (a -> b -> b) -> b -> LookupResult v a -> b
foldr' :: forall a b. (a -> b -> b) -> b -> LookupResult v a -> b
$cfoldl :: forall v b a. (b -> a -> b) -> b -> LookupResult v a -> b
foldl :: forall b a. (b -> a -> b) -> b -> LookupResult v a -> b
$cfoldl' :: forall v b a. (b -> a -> b) -> b -> LookupResult v a -> b
foldl' :: forall b a. (b -> a -> b) -> b -> LookupResult v a -> b
$cfoldr1 :: forall v a. (a -> a -> a) -> LookupResult v a -> a
foldr1 :: forall a. (a -> a -> a) -> LookupResult v a -> a
$cfoldl1 :: forall v a. (a -> a -> a) -> LookupResult v a -> a
foldl1 :: forall a. (a -> a -> a) -> LookupResult v a -> a
$ctoList :: forall v a. LookupResult v a -> [a]
toList :: forall a. LookupResult v a -> [a]
$cnull :: forall v a. LookupResult v a -> Bool
null :: forall a. LookupResult v a -> Bool
$clength :: forall v a. LookupResult v a -> Int
length :: forall a. LookupResult v a -> Int
$celem :: forall v a. Eq a => a -> LookupResult v a -> Bool
elem :: forall a. Eq a => a -> LookupResult v a -> Bool
$cmaximum :: forall v a. Ord a => LookupResult v a -> a
maximum :: forall a. Ord a => LookupResult v a -> a
$cminimum :: forall v a. Ord a => LookupResult v a -> a
minimum :: forall a. Ord a => LookupResult v a -> a
$csum :: forall v a. Num a => LookupResult v a -> a
sum :: forall a. Num a => LookupResult v a -> a
$cproduct :: forall v a. Num a => LookupResult v a -> a
product :: forall a. Num a => LookupResult v a -> a
Foldable, Functor (LookupResult v)
Foldable (LookupResult v)
(Functor (LookupResult v), Foldable (LookupResult v)) =>
(forall (f :: * -> *) a b.
 Applicative f =>
 (a -> f b) -> LookupResult v a -> f (LookupResult v b))
-> (forall (f :: * -> *) a.
    Applicative f =>
    LookupResult v (f a) -> f (LookupResult v a))
-> (forall (m :: * -> *) a b.
    Monad m =>
    (a -> m b) -> LookupResult v a -> m (LookupResult v b))
-> (forall (m :: * -> *) a.
    Monad m =>
    LookupResult v (m a) -> m (LookupResult v a))
-> Traversable (LookupResult v)
forall v. Functor (LookupResult v)
forall v. Foldable (LookupResult v)
forall v (m :: * -> *) a.
Monad m =>
LookupResult v (m a) -> m (LookupResult v a)
forall v (f :: * -> *) a.
Applicative f =>
LookupResult v (f a) -> f (LookupResult v a)
forall v (m :: * -> *) a b.
Monad m =>
(a -> m b) -> LookupResult v a -> m (LookupResult v b)
forall v (f :: * -> *) a b.
Applicative f =>
(a -> f b) -> LookupResult v a -> f (LookupResult v b)
forall (t :: * -> *).
(Functor t, Foldable t) =>
(forall (f :: * -> *) a b.
 Applicative f =>
 (a -> f b) -> t a -> f (t b))
-> (forall (f :: * -> *) a. Applicative f => t (f a) -> f (t a))
-> (forall (m :: * -> *) a b.
    Monad m =>
    (a -> m b) -> t a -> m (t b))
-> (forall (m :: * -> *) a. Monad m => t (m a) -> m (t a))
-> Traversable t
forall (m :: * -> *) a.
Monad m =>
LookupResult v (m a) -> m (LookupResult v a)
forall (f :: * -> *) a.
Applicative f =>
LookupResult v (f a) -> f (LookupResult v a)
forall (m :: * -> *) a b.
Monad m =>
(a -> m b) -> LookupResult v a -> m (LookupResult v b)
forall (f :: * -> *) a b.
Applicative f =>
(a -> f b) -> LookupResult v a -> f (LookupResult v b)
$ctraverse :: forall v (f :: * -> *) a b.
Applicative f =>
(a -> f b) -> LookupResult v a -> f (LookupResult v b)
traverse :: forall (f :: * -> *) a b.
Applicative f =>
(a -> f b) -> LookupResult v a -> f (LookupResult v b)
$csequenceA :: forall v (f :: * -> *) a.
Applicative f =>
LookupResult v (f a) -> f (LookupResult v a)
sequenceA :: forall (f :: * -> *) a.
Applicative f =>
LookupResult v (f a) -> f (LookupResult v a)
$cmapM :: forall v (m :: * -> *) a b.
Monad m =>
(a -> m b) -> LookupResult v a -> m (LookupResult v b)
mapM :: forall (m :: * -> *) a b.
Monad m =>
(a -> m b) -> LookupResult v a -> m (LookupResult v b)
$csequence :: forall v (m :: * -> *) a.
Monad m =>
LookupResult v (m a) -> m (LookupResult v a)
sequence :: forall (m :: * -> *) a.
Monad m =>
LookupResult v (m a) -> m (LookupResult v a)
Traversable)

{- |
Get the field of type @v@ from a @'LookupResult' v b@, if any.
-}
getValue :: LookupResult v b -> Maybe v
getValue :: forall v b. LookupResult v b -> Maybe v
getValue = \case
  LookupResult v b
NotFound -> Maybe v
forall a. Maybe a
Nothing
  Found !v
v -> v -> Maybe v
forall a. a -> Maybe a
Just v
v
  FoundWithBlob !v
v !b
_b -> v -> Maybe v
forall a. a -> Maybe a
Just v
v

{- |
Get the field of type @b@ from a @'LookupResult' v b@, if any.

The following property holds:

prop> isJust (getBlob result) <= isJust (getValue result)
-}
getBlob :: LookupResult v b -> Maybe b
getBlob :: forall v b. LookupResult v b -> Maybe b
getBlob = \case
  LookupResult v b
NotFound -> Maybe b
forall a. Maybe a
Nothing
  Found !v
_v -> Maybe b
forall a. Maybe a
Nothing
  FoundWithBlob !v
_v !b
b -> b -> Maybe b
forall a. a -> Maybe a
Just b
b

instance (NFData v, NFData b) => NFData (LookupResult v b) where
  rnf :: LookupResult v b -> ()
  rnf :: LookupResult v b -> ()
rnf = \case
    LookupResult v b
NotFound -> ()
    Found v
v -> v -> ()
forall a. NFData a => a -> ()
rnf v
v
    FoundWithBlob v
v b
b -> v -> ()
forall a. NFData a => a -> ()
rnf v
v () -> () -> ()
forall a b. a -> b -> b
`seq` b -> ()
forall a. NFData a => a -> ()
rnf b
b

instance Bifunctor LookupResult where
  bimap :: (v -> v') -> (b -> b') -> LookupResult v b -> LookupResult v' b'
  bimap :: forall a b c d.
(a -> b) -> (c -> d) -> LookupResult a c -> LookupResult b d
bimap v -> v'
f b -> b'
g = \case
    LookupResult v b
NotFound -> LookupResult v' b'
forall v b. LookupResult v b
NotFound
    Found v
v -> v' -> LookupResult v' b'
forall v b. v -> LookupResult v b
Found (v -> v'
f v
v)
    FoundWithBlob v
v b
b -> v' -> b' -> LookupResult v' b'
forall v b. v -> b -> LookupResult v b
FoundWithBlob (v -> v'
f v
v) (b -> b'
g b
b)

{- |
Look up the value associated with a key.

>>> :{
runExample $ \session table -> do
  LSMT.insert table 0 "Hello" Nothing
  LSMT.insert table 1 "World" Nothing
  print =<< LSMT.lookup table 0
:}
Found (Value "Hello")

If the key is not associated with any value, 'lookup' returns 'NotFound'.

>>> :{
runExample $ \session table -> do
  LSMT.lookup table 0
:}
NotFound

If the key has an associated BLOB, the result contains a 'BlobRef'.
The full BLOB can be retrieved by passing that 'BlobRef' to 'retrieveBlob'.

>>> :{
runExample $ \session table -> do
  LSMT.insert table 0 "Hello" (Just "World")
  print
    =<< traverse (LSMT.retrieveBlob session)
    =<< LSMT.lookup table 0
:}
FoundWithBlob (Value "Hello") (Blob "World")

The worst-case disk I\/O complexity of this operation depends on the merge policy of the table:

['LazyLevelling']:
    \(O(T \log_T \frac{n}{B})\).

Lookups can be performed concurrently from multiple Haskell threads.

Throws the following exceptions:

['SessionClosedError']:
    If the session is closed.
['TableClosedError']:
    If the table is closed.
['TableCorruptedError']:
    If the table data is corrupted.
-}
{-# SPECIALISE
  lookup ::
    (SerialiseKey k, SerialiseValue v, ResolveValue v) =>
    Table IO k v b ->
    k ->
    IO (LookupResult v (BlobRef IO b))
  #-}
lookup ::
  forall m k v b.
  (IOLike m) =>
  (SerialiseKey k, SerialiseValue v, ResolveValue v) =>
  Table m k v b ->
  k ->
  m (LookupResult v (BlobRef m b))
lookup :: forall (m :: * -> *) k v b.
(IOLike m, SerialiseKey k, SerialiseValue v, ResolveValue v) =>
Table m k v b -> k -> m (LookupResult v (BlobRef m b))
lookup Table m k v b
table k
k = do
  Vector (LookupResult v (BlobRef m b))
mvs <- Table m k v b
-> Vector k -> m (Vector (LookupResult v (BlobRef m b)))
forall (m :: * -> *) k v b.
(IOLike m, SerialiseKey k, SerialiseValue v, ResolveValue v) =>
Table m k v b
-> Vector k -> m (Vector (LookupResult v (BlobRef m b)))
lookups Table m k v b
table (k -> Vector k
forall a. a -> Vector a
V.singleton k
k)
  let mmv :: Maybe (LookupResult v (BlobRef m b))
mmv = (LookupResult v (BlobRef m b),
 Vector (LookupResult v (BlobRef m b)))
-> LookupResult v (BlobRef m b)
forall a b. (a, b) -> a
fst ((LookupResult v (BlobRef m b),
  Vector (LookupResult v (BlobRef m b)))
 -> LookupResult v (BlobRef m b))
-> Maybe
     (LookupResult v (BlobRef m b),
      Vector (LookupResult v (BlobRef m b)))
-> Maybe (LookupResult v (BlobRef m b))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Vector (LookupResult v (BlobRef m b))
-> Maybe
     (LookupResult v (BlobRef m b),
      Vector (LookupResult v (BlobRef m b)))
forall a. Vector a -> Maybe (a, Vector a)
V.uncons Vector (LookupResult v (BlobRef m b))
mvs
  LookupResult v (BlobRef m b) -> m (LookupResult v (BlobRef m b))
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (LookupResult v (BlobRef m b) -> m (LookupResult v (BlobRef m b)))
-> LookupResult v (BlobRef m b) -> m (LookupResult v (BlobRef m b))
forall a b. (a -> b) -> a -> b
$ LookupResult v (BlobRef m b)
-> Maybe (LookupResult v (BlobRef m b))
-> LookupResult v (BlobRef m b)
forall a. a -> Maybe a -> a
fromMaybe LookupResult v (BlobRef m b)
forall v b. LookupResult v b
NotFound Maybe (LookupResult v (BlobRef m b))
mmv

{- |
Variant of 'lookup' for batch lookups.
The batch of keys corresponds in-order to the batch of results.

The worst-case disk I\/O complexity of this operation depends on the merge policy of the table:

['LazyLevelling']:
    \(O(b \: T \log_T \frac{n}{B})\).

The variable \(b\) refers to the length of the input vector.

The following property holds in the absence of races:

prop> lookups table keys = traverse (lookup table) keys
-}
{-# SPECIALISE
  lookups ::
    (SerialiseKey k, SerialiseValue v, ResolveValue v) =>
    Table IO k v b ->
    Vector k ->
    IO (Vector (LookupResult v (BlobRef IO b)))
  #-}
lookups ::
  forall m k v b.
  (IOLike m) =>
  (SerialiseKey k, SerialiseValue v, ResolveValue v) =>
  Table m k v b ->
  Vector k ->
  m (Vector (LookupResult v (BlobRef m b)))
lookups :: forall (m :: * -> *) k v b.
(IOLike m, SerialiseKey k, SerialiseValue v, ResolveValue v) =>
Table m k v b
-> Vector k -> m (Vector (LookupResult v (BlobRef m b)))
lookups (Table Table m h
table :: Table m k v b) Vector k
keys = do
  Vector (Maybe (Entry SerialisedValue (WeakBlobRef m h)))
maybeEntries <- ResolveSerialisedValue
-> Vector SerialisedKey
-> Table m h
-> m (Vector (Maybe (Entry SerialisedValue (WeakBlobRef m h))))
forall (m :: * -> *) h.
(MonadAsync m, MonadMask m, MonadMVar m, MonadST m) =>
ResolveSerialisedValue
-> Vector SerialisedKey
-> Table m h
-> m (Vector (Maybe (Entry SerialisedValue (WeakBlobRef m h))))
Internal.lookups (Proxy v -> ResolveSerialisedValue
forall v. ResolveValue v => Proxy v -> ResolveSerialisedValue
_getResolveSerialisedValue (forall t. Proxy t
forall {k} (t :: k). Proxy t
Proxy @v)) ((k -> SerialisedKey) -> Vector k -> Vector SerialisedKey
forall a b. (a -> b) -> Vector a -> Vector b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap k -> SerialisedKey
forall k. SerialiseKey k => k -> SerialisedKey
Internal.serialiseKey Vector k
keys) Table m h
table
  Vector (LookupResult v (BlobRef m b))
-> m (Vector (LookupResult v (BlobRef m b)))
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (Vector (LookupResult v (BlobRef m b))
 -> m (Vector (LookupResult v (BlobRef m b))))
-> Vector (LookupResult v (BlobRef m b))
-> m (Vector (LookupResult v (BlobRef m b)))
forall a b. (a -> b) -> a -> b
$ LookupResult v (BlobRef m b)
-> (Entry SerialisedValue (WeakBlobRef m h)
    -> LookupResult v (BlobRef m b))
-> Maybe (Entry SerialisedValue (WeakBlobRef m h))
-> LookupResult v (BlobRef m b)
forall b a. b -> (a -> b) -> Maybe a -> b
maybe LookupResult v (BlobRef m b)
forall v b. LookupResult v b
NotFound Entry SerialisedValue (WeakBlobRef m h)
-> LookupResult v (BlobRef m b)
forall {m :: * -> *} {b}.
Entry SerialisedValue (WeakBlobRef m h)
-> LookupResult v (BlobRef m b)
entryToLookupResult (Maybe (Entry SerialisedValue (WeakBlobRef m h))
 -> LookupResult v (BlobRef m b))
-> Vector (Maybe (Entry SerialisedValue (WeakBlobRef m h)))
-> Vector (LookupResult v (BlobRef m b))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Vector (Maybe (Entry SerialisedValue (WeakBlobRef m h)))
maybeEntries
 where
  entryToLookupResult :: Entry SerialisedValue (WeakBlobRef m h)
-> LookupResult v (BlobRef m b)
entryToLookupResult = \case
    Entry.Insert !SerialisedValue
v -> v -> LookupResult v (BlobRef m b)
forall v b. v -> LookupResult v b
Found (SerialisedValue -> v
forall v. SerialiseValue v => SerialisedValue -> v
Internal.deserialiseValue SerialisedValue
v)
    Entry.InsertWithBlob !SerialisedValue
v !WeakBlobRef m h
b -> v -> BlobRef m b -> LookupResult v (BlobRef m b)
forall v b. v -> b -> LookupResult v b
FoundWithBlob (SerialisedValue -> v
forall v. SerialiseValue v => SerialisedValue -> v
Internal.deserialiseValue SerialisedValue
v) (WeakBlobRef m h -> BlobRef m b
forall (m :: * -> *) b h.
Typeable h =>
WeakBlobRef m h -> BlobRef m b
BlobRef WeakBlobRef m h
b)
    Entry.Upsert !SerialisedValue
v -> v -> LookupResult v (BlobRef m b)
forall v b. v -> LookupResult v b
Found (SerialisedValue -> v
forall v. SerialiseValue v => SerialisedValue -> v
Internal.deserialiseValue SerialisedValue
v)
    Entry SerialisedValue (WeakBlobRef m h)
Entry.Delete -> LookupResult v (BlobRef m b)
forall v b. LookupResult v b
NotFound

data Entry k v b
  = Entry !k !v
  | EntryWithBlob !k !v !b
  deriving stock (Entry k v b -> Entry k v b -> Bool
(Entry k v b -> Entry k v b -> Bool)
-> (Entry k v b -> Entry k v b -> Bool) -> Eq (Entry k v b)
forall a. (a -> a -> Bool) -> (a -> a -> Bool) -> Eq a
forall k v b.
(Eq k, Eq v, Eq b) =>
Entry k v b -> Entry k v b -> Bool
$c== :: forall k v b.
(Eq k, Eq v, Eq b) =>
Entry k v b -> Entry k v b -> Bool
== :: Entry k v b -> Entry k v b -> Bool
$c/= :: forall k v b.
(Eq k, Eq v, Eq b) =>
Entry k v b -> Entry k v b -> Bool
/= :: Entry k v b -> Entry k v b -> Bool
Eq, Int -> Entry k v b -> ShowS
[Entry k v b] -> ShowS
Entry k v b -> FilePath
(Int -> Entry k v b -> ShowS)
-> (Entry k v b -> FilePath)
-> ([Entry k v b] -> ShowS)
-> Show (Entry k v b)
forall a.
(Int -> a -> ShowS) -> (a -> FilePath) -> ([a] -> ShowS) -> Show a
forall k v b.
(Show k, Show v, Show b) =>
Int -> Entry k v b -> ShowS
forall k v b. (Show k, Show v, Show b) => [Entry k v b] -> ShowS
forall k v b. (Show k, Show v, Show b) => Entry k v b -> FilePath
$cshowsPrec :: forall k v b.
(Show k, Show v, Show b) =>
Int -> Entry k v b -> ShowS
showsPrec :: Int -> Entry k v b -> ShowS
$cshow :: forall k v b. (Show k, Show v, Show b) => Entry k v b -> FilePath
show :: Entry k v b -> FilePath
$cshowList :: forall k v b. (Show k, Show v, Show b) => [Entry k v b] -> ShowS
showList :: [Entry k v b] -> ShowS
Show, (forall a b. (a -> b) -> Entry k v a -> Entry k v b)
-> (forall a b. a -> Entry k v b -> Entry k v a)
-> Functor (Entry k v)
forall a b. a -> Entry k v b -> Entry k v a
forall a b. (a -> b) -> Entry k v a -> Entry k v b
forall k v a b. a -> Entry k v b -> Entry k v a
forall k v a b. (a -> b) -> Entry k v a -> Entry k v b
forall (f :: * -> *).
(forall a b. (a -> b) -> f a -> f b)
-> (forall a b. a -> f b -> f a) -> Functor f
$cfmap :: forall k v a b. (a -> b) -> Entry k v a -> Entry k v b
fmap :: forall a b. (a -> b) -> Entry k v a -> Entry k v b
$c<$ :: forall k v a b. a -> Entry k v b -> Entry k v a
<$ :: forall a b. a -> Entry k v b -> Entry k v a
Functor, (forall m. Monoid m => Entry k v m -> m)
-> (forall m a. Monoid m => (a -> m) -> Entry k v a -> m)
-> (forall m a. Monoid m => (a -> m) -> Entry k v a -> m)
-> (forall a b. (a -> b -> b) -> b -> Entry k v a -> b)
-> (forall a b. (a -> b -> b) -> b -> Entry k v a -> b)
-> (forall b a. (b -> a -> b) -> b -> Entry k v a -> b)
-> (forall b a. (b -> a -> b) -> b -> Entry k v a -> b)
-> (forall a. (a -> a -> a) -> Entry k v a -> a)
-> (forall a. (a -> a -> a) -> Entry k v a -> a)
-> (forall a. Entry k v a -> [a])
-> (forall a. Entry k v a -> Bool)
-> (forall a. Entry k v a -> Int)
-> (forall a. Eq a => a -> Entry k v a -> Bool)
-> (forall a. Ord a => Entry k v a -> a)
-> (forall a. Ord a => Entry k v a -> a)
-> (forall a. Num a => Entry k v a -> a)
-> (forall a. Num a => Entry k v a -> a)
-> Foldable (Entry k v)
forall a. Eq a => a -> Entry k v a -> Bool
forall a. Num a => Entry k v a -> a
forall a. Ord a => Entry k v a -> a
forall m. Monoid m => Entry k v m -> m
forall a. Entry k v a -> Bool
forall a. Entry k v a -> Int
forall a. Entry k v a -> [a]
forall a. (a -> a -> a) -> Entry k v a -> a
forall m a. Monoid m => (a -> m) -> Entry k v a -> m
forall b a. (b -> a -> b) -> b -> Entry k v a -> b
forall a b. (a -> b -> b) -> b -> Entry k v a -> b
forall k v a. Eq a => a -> Entry k v a -> Bool
forall k v a. Num a => Entry k v a -> a
forall k v a. Ord a => Entry k v a -> a
forall k v m. Monoid m => Entry k v m -> m
forall k v a. Entry k v a -> Bool
forall k v a. Entry k v a -> Int
forall k v a. Entry k v a -> [a]
forall k v a. (a -> a -> a) -> Entry k v a -> a
forall k v m a. Monoid m => (a -> m) -> Entry k v a -> m
forall k v b a. (b -> a -> b) -> b -> Entry k v a -> b
forall k v a b. (a -> b -> b) -> b -> Entry k v a -> b
forall (t :: * -> *).
(forall m. Monoid m => t m -> m)
-> (forall m a. Monoid m => (a -> m) -> t a -> m)
-> (forall m a. Monoid m => (a -> m) -> t a -> m)
-> (forall a b. (a -> b -> b) -> b -> t a -> b)
-> (forall a b. (a -> b -> b) -> b -> t a -> b)
-> (forall b a. (b -> a -> b) -> b -> t a -> b)
-> (forall b a. (b -> a -> b) -> b -> t a -> b)
-> (forall a. (a -> a -> a) -> t a -> a)
-> (forall a. (a -> a -> a) -> t a -> a)
-> (forall a. t a -> [a])
-> (forall a. t a -> Bool)
-> (forall a. t a -> Int)
-> (forall a. Eq a => a -> t a -> Bool)
-> (forall a. Ord a => t a -> a)
-> (forall a. Ord a => t a -> a)
-> (forall a. Num a => t a -> a)
-> (forall a. Num a => t a -> a)
-> Foldable t
$cfold :: forall k v m. Monoid m => Entry k v m -> m
fold :: forall m. Monoid m => Entry k v m -> m
$cfoldMap :: forall k v m a. Monoid m => (a -> m) -> Entry k v a -> m
foldMap :: forall m a. Monoid m => (a -> m) -> Entry k v a -> m
$cfoldMap' :: forall k v m a. Monoid m => (a -> m) -> Entry k v a -> m
foldMap' :: forall m a. Monoid m => (a -> m) -> Entry k v a -> m
$cfoldr :: forall k v a b. (a -> b -> b) -> b -> Entry k v a -> b
foldr :: forall a b. (a -> b -> b) -> b -> Entry k v a -> b
$cfoldr' :: forall k v a b. (a -> b -> b) -> b -> Entry k v a -> b
foldr' :: forall a b. (a -> b -> b) -> b -> Entry k v a -> b
$cfoldl :: forall k v b a. (b -> a -> b) -> b -> Entry k v a -> b
foldl :: forall b a. (b -> a -> b) -> b -> Entry k v a -> b
$cfoldl' :: forall k v b a. (b -> a -> b) -> b -> Entry k v a -> b
foldl' :: forall b a. (b -> a -> b) -> b -> Entry k v a -> b
$cfoldr1 :: forall k v a. (a -> a -> a) -> Entry k v a -> a
foldr1 :: forall a. (a -> a -> a) -> Entry k v a -> a
$cfoldl1 :: forall k v a. (a -> a -> a) -> Entry k v a -> a
foldl1 :: forall a. (a -> a -> a) -> Entry k v a -> a
$ctoList :: forall k v a. Entry k v a -> [a]
toList :: forall a. Entry k v a -> [a]
$cnull :: forall k v a. Entry k v a -> Bool
null :: forall a. Entry k v a -> Bool
$clength :: forall k v a. Entry k v a -> Int
length :: forall a. Entry k v a -> Int
$celem :: forall k v a. Eq a => a -> Entry k v a -> Bool
elem :: forall a. Eq a => a -> Entry k v a -> Bool
$cmaximum :: forall k v a. Ord a => Entry k v a -> a
maximum :: forall a. Ord a => Entry k v a -> a
$cminimum :: forall k v a. Ord a => Entry k v a -> a
minimum :: forall a. Ord a => Entry k v a -> a
$csum :: forall k v a. Num a => Entry k v a -> a
sum :: forall a. Num a => Entry k v a -> a
$cproduct :: forall k v a. Num a => Entry k v a -> a
product :: forall a. Num a => Entry k v a -> a
Foldable, Functor (Entry k v)
Foldable (Entry k v)
(Functor (Entry k v), Foldable (Entry k v)) =>
(forall (f :: * -> *) a b.
 Applicative f =>
 (a -> f b) -> Entry k v a -> f (Entry k v b))
-> (forall (f :: * -> *) a.
    Applicative f =>
    Entry k v (f a) -> f (Entry k v a))
-> (forall (m :: * -> *) a b.
    Monad m =>
    (a -> m b) -> Entry k v a -> m (Entry k v b))
-> (forall (m :: * -> *) a.
    Monad m =>
    Entry k v (m a) -> m (Entry k v a))
-> Traversable (Entry k v)
forall k v. Functor (Entry k v)
forall k v. Foldable (Entry k v)
forall k v (m :: * -> *) a.
Monad m =>
Entry k v (m a) -> m (Entry k v a)
forall k v (f :: * -> *) a.
Applicative f =>
Entry k v (f a) -> f (Entry k v a)
forall k v (m :: * -> *) a b.
Monad m =>
(a -> m b) -> Entry k v a -> m (Entry k v b)
forall k v (f :: * -> *) a b.
Applicative f =>
(a -> f b) -> Entry k v a -> f (Entry k v b)
forall (t :: * -> *).
(Functor t, Foldable t) =>
(forall (f :: * -> *) a b.
 Applicative f =>
 (a -> f b) -> t a -> f (t b))
-> (forall (f :: * -> *) a. Applicative f => t (f a) -> f (t a))
-> (forall (m :: * -> *) a b.
    Monad m =>
    (a -> m b) -> t a -> m (t b))
-> (forall (m :: * -> *) a. Monad m => t (m a) -> m (t a))
-> Traversable t
forall (m :: * -> *) a.
Monad m =>
Entry k v (m a) -> m (Entry k v a)
forall (f :: * -> *) a.
Applicative f =>
Entry k v (f a) -> f (Entry k v a)
forall (m :: * -> *) a b.
Monad m =>
(a -> m b) -> Entry k v a -> m (Entry k v b)
forall (f :: * -> *) a b.
Applicative f =>
(a -> f b) -> Entry k v a -> f (Entry k v b)
$ctraverse :: forall k v (f :: * -> *) a b.
Applicative f =>
(a -> f b) -> Entry k v a -> f (Entry k v b)
traverse :: forall (f :: * -> *) a b.
Applicative f =>
(a -> f b) -> Entry k v a -> f (Entry k v b)
$csequenceA :: forall k v (f :: * -> *) a.
Applicative f =>
Entry k v (f a) -> f (Entry k v a)
sequenceA :: forall (f :: * -> *) a.
Applicative f =>
Entry k v (f a) -> f (Entry k v a)
$cmapM :: forall k v (m :: * -> *) a b.
Monad m =>
(a -> m b) -> Entry k v a -> m (Entry k v b)
mapM :: forall (m :: * -> *) a b.
Monad m =>
(a -> m b) -> Entry k v a -> m (Entry k v b)
$csequence :: forall k v (m :: * -> *) a.
Monad m =>
Entry k v (m a) -> m (Entry k v a)
sequence :: forall (m :: * -> *) a.
Monad m =>
Entry k v (m a) -> m (Entry k v a)
Traversable)

instance (NFData k, NFData v, NFData b) => NFData (Entry k v b) where
  rnf :: Entry k v b -> ()
  rnf :: Entry k v b -> ()
rnf = \case
    Entry k
k v
v -> k -> ()
forall a. NFData a => a -> ()
rnf k
k () -> () -> ()
forall a b. a -> b -> b
`seq` v -> ()
forall a. NFData a => a -> ()
rnf v
v
    EntryWithBlob k
k v
v b
b -> k -> ()
forall a. NFData a => a -> ()
rnf k
k () -> () -> ()
forall a b. a -> b -> b
`seq` v -> ()
forall a. NFData a => a -> ()
rnf v
v () -> () -> ()
forall a b. a -> b -> b
`seq` b -> ()
forall a. NFData a => a -> ()
rnf b
b

instance Bifunctor (Entry k) where
  bimap :: (v -> v') -> (b -> b') -> Entry k v b -> Entry k v' b'
  bimap :: forall a b c d. (a -> b) -> (c -> d) -> Entry k a c -> Entry k b d
bimap v -> v'
f b -> b'
g = \case
    Entry k
k v
v -> k -> v' -> Entry k v' b'
forall k v b. k -> v -> Entry k v b
Entry k
k (v -> v'
f v
v)
    EntryWithBlob k
k v
v b
b -> k -> v' -> b' -> Entry k v' b'
forall k v b. k -> v -> b -> Entry k v b
EntryWithBlob k
k (v -> v'
f v
v) (b -> b'
g b
b)

{- |
Look up a batch of values associated with keys in the given range.

The worst-case disk I\/O complexity of this operation is \(O(T \log_T \frac{n}{B} + \frac{b}{P})\),
where the variable \(b\) refers to the length of the /output/ vector.

Range lookups can be performed concurrently from multiple Haskell threads.

Throws the following exceptions:

['SessionClosedError']:
    If the session is closed.
['TableClosedError']:
    If the table is closed.
['TableCorruptedError']:
    If the table data is corrupted.
-}
{-# SPECIALISE
  rangeLookup ::
    (SerialiseKey k, SerialiseValue v, ResolveValue v) =>
    Table IO k v b ->
    Range k ->
    IO (Vector (Entry k v (BlobRef IO b)))
  #-}
rangeLookup ::
  forall m k v b.
  (IOLike m) =>
  (SerialiseKey k, SerialiseValue v, ResolveValue v) =>
  Table m k v b ->
  Range k ->
  m (Vector (Entry k v (BlobRef m b)))
rangeLookup :: forall (m :: * -> *) k v b.
(IOLike m, SerialiseKey k, SerialiseValue v, ResolveValue v) =>
Table m k v b -> Range k -> m (Vector (Entry k v (BlobRef m b)))
rangeLookup (Table Table m h
table :: Table m k v b) Range k
range =
  ResolveSerialisedValue
-> Range SerialisedKey
-> Table m h
-> (SerialisedKey
    -> SerialisedValue
    -> Maybe (WeakBlobRef m h)
    -> Entry k v (BlobRef m b))
-> m (Vector (Entry k v (BlobRef m b)))
forall (m :: * -> *) h res.
(MonadMask m, MonadMVar m, MonadST m, MonadSTM m) =>
ResolveSerialisedValue
-> Range SerialisedKey
-> Table m h
-> (SerialisedKey
    -> SerialisedValue -> Maybe (WeakBlobRef m h) -> res)
-> m (Vector res)
Internal.rangeLookup (Proxy v -> ResolveSerialisedValue
forall v. ResolveValue v => Proxy v -> ResolveSerialisedValue
_getResolveSerialisedValue (forall t. Proxy t
forall {k} (t :: k). Proxy t
Proxy @v)) (k -> SerialisedKey
forall k. SerialiseKey k => k -> SerialisedKey
Internal.serialiseKey (k -> SerialisedKey) -> Range k -> Range SerialisedKey
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Range k
range) Table m h
table ((SerialisedKey
  -> SerialisedValue
  -> Maybe (WeakBlobRef m h)
  -> Entry k v (BlobRef m b))
 -> m (Vector (Entry k v (BlobRef m b))))
-> (SerialisedKey
    -> SerialisedValue
    -> Maybe (WeakBlobRef m h)
    -> Entry k v (BlobRef m b))
-> m (Vector (Entry k v (BlobRef m b)))
forall a b. (a -> b) -> a -> b
$ \ !SerialisedKey
k !SerialisedValue
v -> \case
    Just !WeakBlobRef m h
b -> k -> v -> BlobRef m b -> Entry k v (BlobRef m b)
forall k v b. k -> v -> b -> Entry k v b
EntryWithBlob (SerialisedKey -> k
forall k. SerialiseKey k => SerialisedKey -> k
Internal.deserialiseKey SerialisedKey
k) (SerialisedValue -> v
forall v. SerialiseValue v => SerialisedValue -> v
Internal.deserialiseValue SerialisedValue
v) (WeakBlobRef m h -> BlobRef m b
forall (m :: * -> *) b h.
Typeable h =>
WeakBlobRef m h -> BlobRef m b
BlobRef WeakBlobRef m h
b)
    Maybe (WeakBlobRef m h)
Nothing -> k -> v -> Entry k v (BlobRef m b)
forall k v b. k -> v -> Entry k v b
Entry (SerialisedKey -> k
forall k. SerialiseKey k => SerialisedKey -> k
Internal.deserialiseKey SerialisedKey
k) (SerialisedValue -> v
forall v. SerialiseValue v => SerialisedValue -> v
Internal.deserialiseValue SerialisedValue
v)

--------------------------------------------------------------------------------
-- Updates
--------------------------------------------------------------------------------

{- |
Insert associates the given value and BLOB with the given key in the table.

>>> :{
runExample $ \session table -> do
  LSMT.insert table 0 "Hello" Nothing
  LSMT.insert table 1 "World" Nothing
  print =<< LSMT.lookup table 0
:}
Found (Value "Hello")

Insert may optionally associate a BLOB value with the given key.

>>> :{
runExample $ \session table -> do
  LSMT.insert table 0 "Hello" (Just "World")
  print
    =<< traverse (retrieveBlob session)
    =<< LSMT.lookup table 0
:}
FoundWithBlob (Value "Hello") (Blob "World")

Insert overwrites any value and BLOB previously associated with the given key,
even if the given BLOB is 'Nothing'.

>>> :{
runExample $ \session table -> do
  LSMT.insert table 0 "Hello" (Just "World")
  LSMT.insert table 0 "Goodbye" Nothing
  print
    =<< traverse (retrieveBlob session)
    =<< LSMT.lookup table 0
:}
Found (Value "Goodbye")

The worst-case disk I\/O complexity of this operation depends on the merge policy and the merge schedule of the table:

['LazyLevelling'\/'Incremental']:
    \(O(\frac{1}{P} \: \log_T \frac{n}{B})\).
['LazyLevelling'\/'OneShot']:
    \(O(\frac{n}{P})\).

Throws the following exceptions:

['SessionClosedError']:
    If the session is closed.
['TableClosedError']:
    If the table is closed.
-}
{-# SPECIALISE
  insert ::
    (SerialiseKey k, SerialiseValue v, ResolveValue v, SerialiseValue b) =>
    Table IO k v b ->
    k ->
    v ->
    Maybe b ->
    IO ()
  #-}
insert ::
  forall m k v b.
  (IOLike m) =>
  (SerialiseKey k, SerialiseValue v, ResolveValue v, SerialiseValue b) =>
  Table m k v b ->
  k ->
  v ->
  Maybe b ->
  m ()
insert :: forall (m :: * -> *) k v b.
(IOLike m, SerialiseKey k, SerialiseValue v, ResolveValue v,
 SerialiseValue b) =>
Table m k v b -> k -> v -> Maybe b -> m ()
insert Table m k v b
table k
k v
v Maybe b
b =
  Table m k v b -> Vector (k, v, Maybe b) -> m ()
forall (m :: * -> *) k v b.
(IOLike m, SerialiseKey k, SerialiseValue v, ResolveValue v,
 SerialiseValue b) =>
Table m k v b -> Vector (k, v, Maybe b) -> m ()
inserts Table m k v b
table ((k, v, Maybe b) -> Vector (k, v, Maybe b)
forall a. a -> Vector a
V.singleton (k
k, v
v, Maybe b
b))

{- |
Variant of 'insert' for batch insertions.

The worst-case disk I\/O complexity of this operation depends on the merge policy and the merge schedule of the table:

['LazyLevelling'\/'Incremental']:
    \(O(b \: \frac{1}{P} \: \log_T \frac{n}{B})\).
['LazyLevelling'\/'OneShot']:
    \(O(\frac{b}{P} \log_T \frac{b}{B} + \frac{n}{P})\).

The variable \(b\) refers to the length of the input vector.

The following property holds in the absence of races:

prop> inserts table entries = traverse_ (uncurry $ insert table) entries
-}
{-# SPECIALISE
  inserts ::
    (SerialiseKey k, SerialiseValue v, ResolveValue v, SerialiseValue b) =>
    Table IO k v b ->
    Vector (k, v, Maybe b) ->
    IO ()
  #-}
inserts ::
  forall m k v b.
  (IOLike m) =>
  (SerialiseKey k, SerialiseValue v, ResolveValue v, SerialiseValue b) =>
  Table m k v b ->
  Vector (k, v, Maybe b) ->
  m ()
inserts :: forall (m :: * -> *) k v b.
(IOLike m, SerialiseKey k, SerialiseValue v, ResolveValue v,
 SerialiseValue b) =>
Table m k v b -> Vector (k, v, Maybe b) -> m ()
inserts Table m k v b
table Vector (k, v, Maybe b)
entries =
  Table m k v b -> Vector (k, Update v b) -> m ()
forall (m :: * -> *) k v b.
(IOLike m, SerialiseKey k, SerialiseValue v, ResolveValue v,
 SerialiseValue b) =>
Table m k v b -> Vector (k, Update v b) -> m ()
updates Table m k v b
table (((k, v, Maybe b) -> (k, Update v b))
-> Vector (k, v, Maybe b) -> Vector (k, Update v b)
forall a b. (a -> b) -> Vector a -> Vector b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap (\(k
k, v
v, Maybe b
mb) -> (k
k, v -> Maybe b -> Update v b
forall v b. v -> Maybe b -> Update v b
Insert v
v Maybe b
mb)) Vector (k, v, Maybe b)
entries)

{- |
If the given key is not a member of the table, 'upsert' associates the given value with the given key in the table.
Otherwise, 'upsert' updates the value associated with the given key by combining it with the given value using 'resolve'.

>>> :{
runExample $ \session table -> do
  LSMT.upsert table 0 "Hello"
  LSMT.upsert table 0 "Goodbye"
  print =<< LSMT.lookup table 0
:}
Found (Value "Goodbye Hello")

__Warning:__
Upsert deletes any BLOB previously associated with the given key.

>>> :{
runExample $ \session table -> do
  LSMT.insert table 0 "Hello" (Just "World")
  LSMT.upsert table 0 "Goodbye"
  print
    =<< traverse (LSMT.retrieveBlob session)
    =<< LSMT.lookup table 0
:}
Found (Value "Goodbye Hello")

The worst-case disk I\/O complexity of this operation depends on the merge policy and the merge schedule of the table:

['LazyLevelling'\/'Incremental']:
    \(O(\frac{1}{P} \: \log_T \frac{n}{B})\).
['LazyLevelling'\/'OneShot']:
    \(O(\frac{n}{P})\).

Throws the following exceptions:

['SessionClosedError']:
    If the session is closed.
['TableClosedError']:
    If the table is closed.

The following property holds in the absence of races:

@
upsert table k v = do
  r <- lookup table k
  let v' = maybe v (resolve v) (getValue r)
  insert table k v' Nothing
@
-}
{-# SPECIALISE
  upsert ::
    (SerialiseKey k, SerialiseValue v, ResolveValue v, SerialiseValue b) =>
    Table IO k v b ->
    k ->
    v ->
    IO ()
  #-}
upsert ::
  forall m k v b.
  (IOLike m) =>
  (SerialiseKey k, SerialiseValue v, ResolveValue v, SerialiseValue b) =>
  Table m k v b ->
  k ->
  v ->
  m ()
upsert :: forall (m :: * -> *) k v b.
(IOLike m, SerialiseKey k, SerialiseValue v, ResolveValue v,
 SerialiseValue b) =>
Table m k v b -> k -> v -> m ()
upsert Table m k v b
table k
k v
v =
  Table m k v b -> Vector (k, v) -> m ()
forall (m :: * -> *) k v b.
(IOLike m, SerialiseKey k, SerialiseValue v, ResolveValue v,
 SerialiseValue b) =>
Table m k v b -> Vector (k, v) -> m ()
upserts Table m k v b
table ((k, v) -> Vector (k, v)
forall a. a -> Vector a
V.singleton (k
k, v
v))

{- |
Variant of 'upsert' for batch insertions.

The worst-case disk I\/O complexity of this operation depends on the merge policy and the merge schedule of the table:

['LazyLevelling'\/'Incremental']:
    \(O(b \: \frac{1}{P} \: \log_T \frac{n}{B})\).
['LazyLevelling'\/'OneShot']:
    \(O(\frac{b}{P} \log_T \frac{b}{B} + \frac{n}{P})\).

The variable \(b\) refers to the length of the input vector.

The following property holds in the absence of races:

prop> upserts table entries = traverse_ (uncurry $ upsert table) entries
-}
{-# SPECIALISE
  upserts ::
    (SerialiseKey k, SerialiseValue v, ResolveValue v, SerialiseValue b) =>
    Table IO k v b ->
    Vector (k, v) ->
    IO ()
  #-}
upserts ::
  forall m k v b.
  (IOLike m) =>
  (SerialiseKey k, SerialiseValue v, ResolveValue v, SerialiseValue b) =>
  Table m k v b ->
  Vector (k, v) ->
  m ()
upserts :: forall (m :: * -> *) k v b.
(IOLike m, SerialiseKey k, SerialiseValue v, ResolveValue v,
 SerialiseValue b) =>
Table m k v b -> Vector (k, v) -> m ()
upserts Table m k v b
table Vector (k, v)
entries =
  Table m k v b -> Vector (k, Update v b) -> m ()
forall (m :: * -> *) k v b.
(IOLike m, SerialiseKey k, SerialiseValue v, ResolveValue v,
 SerialiseValue b) =>
Table m k v b -> Vector (k, Update v b) -> m ()
updates Table m k v b
table ((v -> Update v b) -> (k, v) -> (k, Update v b)
forall b c a. (b -> c) -> (a, b) -> (a, c)
forall (p :: * -> * -> *) b c a.
Bifunctor p =>
(b -> c) -> p a b -> p a c
second v -> Update v b
forall v b. v -> Update v b
Upsert ((k, v) -> (k, Update v b))
-> Vector (k, v) -> Vector (k, Update v b)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Vector (k, v)
entries)

{- |
Delete a key from the table.

>>> :{
runExample $ \session table -> do
  LSMT.insert table 0 "Hello" Nothing
  LSMT.delete table 0
  print =<< LSMT.lookup table 0
:}
NotFound

If the key is not a member of the table, the table is left unchanged.

>>> :{
runExample $ \session table -> do
  LSMT.insert table 0 "Hello" Nothing
  LSMT.delete table 1
  print =<< LSMT.lookup table 0
:}
Found (Value "Hello")

The worst-case disk I\/O complexity of this operation depends on the merge policy and the merge schedule of the table:

['LazyLevelling'\/'Incremental']:
    \(O(\frac{1}{P} \: \log_T \frac{n}{B})\).
['LazyLevelling'\/'OneShot']:
    \(O(\frac{n}{P})\).

Throws the following exceptions:

['SessionClosedError']:
    If the session is closed.
['TableClosedError']:
    If the table is closed.
-}
{-# SPECIALISE
  delete ::
    (SerialiseKey k, SerialiseValue v, ResolveValue v, SerialiseValue b) =>
    Table IO k v b ->
    k ->
    IO ()
  #-}
delete ::
  forall m k v b.
  (IOLike m) =>
  (SerialiseKey k, SerialiseValue v, ResolveValue v, SerialiseValue b) =>
  Table m k v b ->
  k ->
  m ()
delete :: forall (m :: * -> *) k v b.
(IOLike m, SerialiseKey k, SerialiseValue v, ResolveValue v,
 SerialiseValue b) =>
Table m k v b -> k -> m ()
delete Table m k v b
table k
k =
  Table m k v b -> Vector k -> m ()
forall (m :: * -> *) k v b.
(IOLike m, SerialiseKey k, SerialiseValue v, ResolveValue v,
 SerialiseValue b) =>
Table m k v b -> Vector k -> m ()
deletes Table m k v b
table (k -> Vector k
forall a. a -> Vector a
V.singleton k
k)

{- |
Variant of 'delete' for batch deletions.

The worst-case disk I\/O complexity of this operation depends on the merge policy and the merge schedule of the table:

['LazyLevelling'\/'Incremental']:
    \(O(b \: \frac{1}{P} \: \log_T \frac{n}{B})\).
['LazyLevelling'\/'OneShot']:
    \(O(\frac{b}{P} \log_T \frac{b}{B} + \frac{n}{P})\).

The variable \(b\) refers to the length of the input vector.

The following property holds in the absence of races:

prop> deletes table keys = traverse_ (delete table) keys
-}
{-# SPECIALISE
  deletes ::
    (SerialiseKey k, SerialiseValue v, ResolveValue v, SerialiseValue b) =>
    Table IO k v b ->
    Vector k ->
    IO ()
  #-}
deletes ::
  forall m k v b.
  (IOLike m) =>
  (SerialiseKey k, SerialiseValue v, ResolveValue v, SerialiseValue b) =>
  Table m k v b ->
  Vector k ->
  m ()
deletes :: forall (m :: * -> *) k v b.
(IOLike m, SerialiseKey k, SerialiseValue v, ResolveValue v,
 SerialiseValue b) =>
Table m k v b -> Vector k -> m ()
deletes Table m k v b
table Vector k
entries =
  Table m k v b -> Vector (k, Update v b) -> m ()
forall (m :: * -> *) k v b.
(IOLike m, SerialiseKey k, SerialiseValue v, ResolveValue v,
 SerialiseValue b) =>
Table m k v b -> Vector (k, Update v b) -> m ()
updates Table m k v b
table ((k -> (k, Update v b)) -> Vector k -> Vector (k, Update v b)
forall a b. (a -> b) -> Vector a -> Vector b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap (,Update v b
forall v b. Update v b
Delete) Vector k
entries)

type Update :: Type -> Type -> Type
data Update v b
  = Insert !v !(Maybe b)
  | Delete
  | Upsert !v
  deriving stock (Int -> Update v b -> ShowS
[Update v b] -> ShowS
Update v b -> FilePath
(Int -> Update v b -> ShowS)
-> (Update v b -> FilePath)
-> ([Update v b] -> ShowS)
-> Show (Update v b)
forall a.
(Int -> a -> ShowS) -> (a -> FilePath) -> ([a] -> ShowS) -> Show a
forall v b. (Show b, Show v) => Int -> Update v b -> ShowS
forall v b. (Show b, Show v) => [Update v b] -> ShowS
forall v b. (Show b, Show v) => Update v b -> FilePath
$cshowsPrec :: forall v b. (Show b, Show v) => Int -> Update v b -> ShowS
showsPrec :: Int -> Update v b -> ShowS
$cshow :: forall v b. (Show b, Show v) => Update v b -> FilePath
show :: Update v b -> FilePath
$cshowList :: forall v b. (Show b, Show v) => [Update v b] -> ShowS
showList :: [Update v b] -> ShowS
Show, Update v b -> Update v b -> Bool
(Update v b -> Update v b -> Bool)
-> (Update v b -> Update v b -> Bool) -> Eq (Update v b)
forall a. (a -> a -> Bool) -> (a -> a -> Bool) -> Eq a
forall v b. (Eq b, Eq v) => Update v b -> Update v b -> Bool
$c== :: forall v b. (Eq b, Eq v) => Update v b -> Update v b -> Bool
== :: Update v b -> Update v b -> Bool
$c/= :: forall v b. (Eq b, Eq v) => Update v b -> Update v b -> Bool
/= :: Update v b -> Update v b -> Bool
Eq)

instance (NFData v, NFData b) => NFData (Update v b) where
  rnf :: Update v b -> ()
  rnf :: Update v b -> ()
rnf = \case
    Insert v
v Maybe b
mb -> v -> ()
forall a. NFData a => a -> ()
rnf v
v () -> () -> ()
forall a b. a -> b -> b
`seq` Maybe b -> ()
forall a. NFData a => a -> ()
rnf Maybe b
mb
    Update v b
Delete -> ()
    Upsert v
v -> v -> ()
forall a. NFData a => a -> ()
rnf v
v

{- |
Update generalises 'insert', 'delete', and 'upsert'.

The worst-case disk I\/O complexity of this operation depends on the merge policy and the merge schedule of the table:

['LazyLevelling'\/'Incremental']:
    \(O(\frac{1}{P} \: \log_T \frac{n}{B})\).
['LazyLevelling'\/'OneShot']:
    \(O(\frac{n}{P})\).

The following properties hold:

prop> update table k (Insert v mb) = insert table k v mb
prop> update table k Delete = delete table k
prop> update table k (Upsert v) = upsert table k v

Throws the following exceptions:

['SessionClosedError']:
    If the session is closed.
['TableClosedError']:
    If the table is closed.
-}
{-# SPECIALISE
  update ::
    (SerialiseKey k, SerialiseValue v, ResolveValue v, SerialiseValue b) =>
    Table IO k v b ->
    k ->
    Update v b ->
    IO ()
  #-}
update ::
  forall m k v b.
  (IOLike m) =>
  (SerialiseKey k, SerialiseValue v, ResolveValue v, SerialiseValue b) =>
  Table m k v b ->
  k ->
  Update v b ->
  m ()
update :: forall (m :: * -> *) k v b.
(IOLike m, SerialiseKey k, SerialiseValue v, ResolveValue v,
 SerialiseValue b) =>
Table m k v b -> k -> Update v b -> m ()
update Table m k v b
table k
k Update v b
mv =
  Table m k v b -> Vector (k, Update v b) -> m ()
forall (m :: * -> *) k v b.
(IOLike m, SerialiseKey k, SerialiseValue v, ResolveValue v,
 SerialiseValue b) =>
Table m k v b -> Vector (k, Update v b) -> m ()
updates Table m k v b
table ((k, Update v b) -> Vector (k, Update v b)
forall a. a -> Vector a
V.singleton (k
k, Update v b
mv))

{- |
Variant of 'update' for batch updates.

The worst-case disk I\/O complexity of this operation depends on the merge policy and the merge schedule of the table:

['LazyLevelling'\/'Incremental']:
    \(O(b \: \frac{1}{P} \: \log_T \frac{n}{B})\).
['LazyLevelling'\/'OneShot']:
    \(O(\frac{b}{P} \log_T \frac{b}{B} + \frac{n}{P})\).

The variable \(b\) refers to the length of the input vector.

The following property holds in the absence of races:

prop> updates table entries = traverse_ (uncurry $ update table) entries
-}
{-# SPECIALISE
  updates ::
    (SerialiseKey k, SerialiseValue v, ResolveValue v, SerialiseValue b) =>
    Table IO k v b ->
    Vector (k, Update v b) ->
    IO ()
  #-}
updates ::
  forall m k v b.
  (IOLike m) =>
  (SerialiseKey k, SerialiseValue v, ResolveValue v, SerialiseValue b) =>
  Table m k v b ->
  Vector (k, Update v b) ->
  m ()
updates :: forall (m :: * -> *) k v b.
(IOLike m, SerialiseKey k, SerialiseValue v, ResolveValue v,
 SerialiseValue b) =>
Table m k v b -> Vector (k, Update v b) -> m ()
updates (Table Table m h
table :: Table m k v b) Vector (k, Update v b)
entries =
  ResolveSerialisedValue
-> Vector (SerialisedKey, Entry SerialisedValue SerialisedBlob)
-> Table m h
-> m ()
forall (m :: * -> *) h.
(MonadMask m, MonadMVar m, MonadST m, MonadSTM m) =>
ResolveSerialisedValue
-> Vector (SerialisedKey, Entry SerialisedValue SerialisedBlob)
-> Table m h
-> m ()
Internal.updates (Proxy v -> ResolveSerialisedValue
forall v. ResolveValue v => Proxy v -> ResolveSerialisedValue
_getResolveSerialisedValue (forall t. Proxy t
forall {k} (t :: k). Proxy t
Proxy @v)) ((k, Update v b)
-> (SerialisedKey, Entry SerialisedValue SerialisedBlob)
serialiseEntry ((k, Update v b)
 -> (SerialisedKey, Entry SerialisedValue SerialisedBlob))
-> Vector (k, Update v b)
-> Vector (SerialisedKey, Entry SerialisedValue SerialisedBlob)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Vector (k, Update v b)
entries) Table m h
table
 where
  serialiseEntry :: (k, Update v b)
-> (SerialisedKey, Entry SerialisedValue SerialisedBlob)
serialiseEntry (k
k, Update v b
u) = (k -> SerialisedKey
forall k. SerialiseKey k => k -> SerialisedKey
Internal.serialiseKey k
k, Update v b -> Entry SerialisedValue SerialisedBlob
serialiseUpdate Update v b
u)
  serialiseUpdate :: Update v b -> Entry SerialisedValue SerialisedBlob
serialiseUpdate = \case
    Insert v
v (Just b
b) -> SerialisedValue
-> SerialisedBlob -> Entry SerialisedValue SerialisedBlob
forall v b. v -> b -> Entry v b
Entry.InsertWithBlob (v -> SerialisedValue
forall v. SerialiseValue v => v -> SerialisedValue
Internal.serialiseValue v
v) (b -> SerialisedBlob
forall v. SerialiseValue v => v -> SerialisedBlob
Internal.serialiseBlob b
b)
    Insert v
v Maybe b
Nothing -> SerialisedValue -> Entry SerialisedValue SerialisedBlob
forall v b. v -> Entry v b
Entry.Insert (v -> SerialisedValue
forall v. SerialiseValue v => v -> SerialisedValue
Internal.serialiseValue v
v)
    Update v b
Delete -> Entry SerialisedValue SerialisedBlob
forall v b. Entry v b
Entry.Delete
    Upsert v
v -> SerialisedValue -> Entry SerialisedValue SerialisedBlob
forall v b. v -> Entry v b
Entry.Upsert (v -> SerialisedValue
forall v. SerialiseValue v => v -> SerialisedValue
Internal.serialiseValue v
v)

--------------------------------------------------------------------------------
-- Duplication
--------------------------------------------------------------------------------

{- |
Run an action with access to the duplicate of a table.

The duplicate is an independent copy of the given table.
Subsequent updates to the original table do not affect the duplicate, and vice versa.

>>> :{
runExample $ \session table -> do
  LSMT.insert table 0 "Hello" Nothing
  LSMT.withDuplicate table $ \table' -> do
    print =<< LSMT.lookup table' 0
    LSMT.insert table' 0 "Goodbye" Nothing
    print =<< LSMT.lookup table' 0
  LSMT.lookup table 0
  print =<< LSMT.lookup table 0
:}
Found (Value "Hello")
Found (Value "Goodbye")
Found (Value "Hello")

The worst-case disk I\/O complexity of this operation depends on the merge policy of the table:

['LazyLevelling']:
    \(O(T \log_T \frac{n}{B})\).

This function is exception-safe for both synchronous and asynchronous exceptions.

It is recommended to use this function instead of 'duplicate' and 'closeTable'.

Throws the following exceptions:

['SessionClosedError']:
    If the session is closed.
['TableClosedError']:
    If the table is closed.
-}
{-# SPECIALISE
  withDuplicate ::
    Table IO k v b ->
    (Table IO k v b -> IO a) ->
    IO a
  #-}
withDuplicate ::
  forall m k v b a.
  (IOLike m) =>
  Table m k v b ->
  (Table m k v b -> m a) ->
  m a
withDuplicate :: forall (m :: * -> *) k v b a.
IOLike m =>
Table m k v b -> (Table m k v b -> m a) -> m a
withDuplicate Table m k v b
table =
  m (Table m k v b)
-> (Table m k v b -> m ()) -> (Table m k v b -> m a) -> m a
forall a b c. m a -> (a -> m b) -> (a -> m c) -> m c
forall (m :: * -> *) a b c.
MonadThrow m =>
m a -> (a -> m b) -> (a -> m c) -> m c
bracket (Table m k v b -> m (Table m k v b)
forall (m :: * -> *) k v b.
IOLike m =>
Table m k v b -> m (Table m k v b)
duplicate Table m k v b
table) Table m k v b -> m ()
forall (m :: * -> *) k v b. IOLike m => Table m k v b -> m ()
closeTable

{- |
Duplicate a table.

The duplicate is an independent copy of the given table.
Subsequent updates to the original table do not affect the duplicate, and vice versa.

>>> :{
runExample $ \session table -> do
  LSMT.insert table 0 "Hello" Nothing
  bracket (LSMT.duplicate table) LSMT.closeTable $ \table' -> do
    print =<< LSMT.lookup table' 0
    LSMT.insert table' 0 "Goodbye" Nothing
    print =<< LSMT.lookup table' 0
  LSMT.lookup table 0
  print =<< LSMT.lookup table 0
:}
Found (Value "Hello")
Found (Value "Goodbye")
Found (Value "Hello")

The worst-case disk I\/O complexity of this operation is \(O(0)\).

__Warning:__ The duplicate must be independently closed using 'closeTable'.

Throws the following exceptions:

['SessionClosedError']:
    If the session is closed.
['TableClosedError']:
    If the table is closed.
-}
{-# SPECIALISE
  duplicate ::
    Table IO k v b ->
    IO (Table IO k v b)
  #-}
duplicate ::
  forall m k v b.
  (IOLike m) =>
  Table m k v b ->
  m (Table m k v b)
duplicate :: forall (m :: * -> *) k v b.
IOLike m =>
Table m k v b -> m (Table m k v b)
duplicate (Table Table m h
table) =
  Table m h -> Table m k v b
forall (m :: * -> *) k v b h.
Typeable h =>
Table m h -> Table m k v b
Table (Table m h -> Table m k v b) -> m (Table m h) -> m (Table m k v b)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Table m h -> m (Table m h)
forall (m :: * -> *) h.
(MonadMask m, MonadMVar m, MonadST m, MonadSTM m) =>
Table m h -> m (Table m h)
Internal.duplicate Table m h
table

--------------------------------------------------------------------------------
-- Union
--------------------------------------------------------------------------------

{- |
Run an action with access to a table that contains the union of the entries of the given tables.

>>> :{
runExample $ \session table1 -> do
  LSMT.insert table1 0 "Hello" Nothing
  LSMT.withTable session $ \table2 -> do
    LSMT.insert table2 0 "World" Nothing
    LSMT.insert table2 1 "Goodbye" Nothing
    LSMT.withUnion table1 table2 $ \table3 -> do
      print =<< LSMT.lookup table3 0
      print =<< LSMT.lookup table3 1
    print =<< LSMT.lookup table1 0
    print =<< LSMT.lookup table2 0
:}
Found (Value "Hello World")
Found (Value "Goodbye")
Found (Value "Hello")
Found (Value "World")

The worst-case disk I\/O complexity of this operation is \(O(\frac{n}{P})\).

This function is exception-safe for both synchronous and asynchronous exceptions.

It is recommended to use this function instead of 'union' and 'closeTable'.

__Warning:__ Both input tables must be from the same 'Session'.

__Warning:__ This is a relatively expensive operation that may take some time to complete.
See 'withIncrementalUnion' for an incremental alternative.

Throws the following exceptions:

['SessionClosedError']:
    If the session is closed.
['TableClosedError']:
    If the table is closed.
['TableUnionNotCompatibleError']:
    If both tables are not from the same 'Session'.
-}
{-# SPECIALISE
  withUnion ::
    (ResolveValue v) =>
    Table IO k v b ->
    Table IO k v b ->
    (Table IO k v b -> IO a) ->
    IO a
  #-}
withUnion ::
  forall m k v b a.
  (IOLike m) =>
  (ResolveValue v) =>
  Table m k v b ->
  Table m k v b ->
  (Table m k v b -> m a) ->
  m a
withUnion :: forall (m :: * -> *) k v b a.
(IOLike m, ResolveValue v) =>
Table m k v b -> Table m k v b -> (Table m k v b -> m a) -> m a
withUnion Table m k v b
table1 Table m k v b
table2 =
  m (Table m k v b)
-> (Table m k v b -> m ()) -> (Table m k v b -> m a) -> m a
forall a b c. m a -> (a -> m b) -> (a -> m c) -> m c
forall (m :: * -> *) a b c.
MonadThrow m =>
m a -> (a -> m b) -> (a -> m c) -> m c
bracket (Table m k v b
table1 Table m k v b -> Table m k v b -> m (Table m k v b)
forall (m :: * -> *) k v b.
(IOLike m, ResolveValue v) =>
Table m k v b -> Table m k v b -> m (Table m k v b)
`union` Table m k v b
table2) Table m k v b -> m ()
forall (m :: * -> *) k v b. IOLike m => Table m k v b -> m ()
closeTable

{- |
Variant of 'withUnions' that takes any number of tables.
-}
{-# SPECIALISE
  withUnions ::
    (ResolveValue v) =>
    NonEmpty (Table IO k v b) ->
    (Table IO k v b -> IO a) ->
    IO a
  #-}
withUnions ::
  forall m k v b a.
  (IOLike m) =>
  (ResolveValue v) =>
  NonEmpty (Table m k v b) ->
  (Table m k v b -> m a) ->
  m a
withUnions :: forall (m :: * -> *) k v b a.
(IOLike m, ResolveValue v) =>
NonEmpty (Table m k v b) -> (Table m k v b -> m a) -> m a
withUnions NonEmpty (Table m k v b)
tables =
  m (Table m k v b)
-> (Table m k v b -> m ()) -> (Table m k v b -> m a) -> m a
forall a b c. m a -> (a -> m b) -> (a -> m c) -> m c
forall (m :: * -> *) a b c.
MonadThrow m =>
m a -> (a -> m b) -> (a -> m c) -> m c
bracket (NonEmpty (Table m k v b) -> m (Table m k v b)
forall (m :: * -> *) k v b.
(IOLike m, ResolveValue v) =>
NonEmpty (Table m k v b) -> m (Table m k v b)
unions NonEmpty (Table m k v b)
tables) Table m k v b -> m ()
forall (m :: * -> *) k v b. IOLike m => Table m k v b -> m ()
closeTable

{- |
Create a table that contains the union of the entries of the given tables.

If the given key is a member of a single input table, then the same key and value occur in the output table.
Otherwise, the values for duplicate keys are combined using 'resolve' from left to right.
If the 'resolve' function behaves like 'const', then this computes a left-biased union.

>>> :{
runExample $ \session table1 -> do
  LSMT.insert table1 0 "Hello" Nothing
  LSMT.withTable session $ \table2 -> do
    LSMT.insert table2 0 "World" Nothing
    LSMT.insert table2 1 "Goodbye" Nothing
    bracket (LSMT.union table1 table2) LSMT.closeTable $ \table3 -> do
      print =<< LSMT.lookup table3 0
      print =<< LSMT.lookup table3 1
    print =<< LSMT.lookup table1 0
    print =<< LSMT.lookup table2 0
:}
Found (Value "Hello World")
Found (Value "Goodbye")
Found (Value "Hello")
Found (Value "World")

The worst-case disk I\/O complexity of this operation is \(O(\frac{n}{P})\).

__Warning:__ The new table must be independently closed using 'closeTable'.

__Warning:__ Both input tables must be from the same 'Session'.

__Warning:__ This is a relatively expensive operation that may take some time to complete.
See 'incrementalUnion' for an incremental alternative.

Throws the following exceptions:

['SessionClosedError']:
    If the session is closed.
['TableClosedError']:
    If the table is closed.
['TableUnionNotCompatibleError']:
    If both tables are not from the same 'Session'.
-}
{-# SPECIALISE
  union ::
    (ResolveValue v) =>
    Table IO k v b ->
    Table IO k v b ->
    IO (Table IO k v b)
  #-}
union ::
  forall m k v b.
  (IOLike m) =>
  (ResolveValue v) =>
  Table m k v b ->
  Table m k v b ->
  m (Table m k v b)
union :: forall (m :: * -> *) k v b.
(IOLike m, ResolveValue v) =>
Table m k v b -> Table m k v b -> m (Table m k v b)
union Table m k v b
table1 Table m k v b
table2 =
  NonEmpty (Table m k v b) -> m (Table m k v b)
forall (m :: * -> *) k v b.
(IOLike m, ResolveValue v) =>
NonEmpty (Table m k v b) -> m (Table m k v b)
unions (Table m k v b
table1 Table m k v b -> [Table m k v b] -> NonEmpty (Table m k v b)
forall a. a -> [a] -> NonEmpty a
:| Table m k v b
table2 Table m k v b -> [Table m k v b] -> [Table m k v b]
forall a. a -> [a] -> [a]
: [])

{- |
Variant of 'union' that takes any number of tables.
-}
{-# SPECIALISE
  unions ::
    (ResolveValue v) =>
    NonEmpty (Table IO k v b) ->
    IO (Table IO k v b)
  #-}
unions ::
  forall m k v b.
  (IOLike m) =>
  (ResolveValue v) =>
  NonEmpty (Table m k v b) ->
  m (Table m k v b)
unions :: forall (m :: * -> *) k v b.
(IOLike m, ResolveValue v) =>
NonEmpty (Table m k v b) -> m (Table m k v b)
unions NonEmpty (Table m k v b)
tables = do
  m (Table m k v b)
-> (Table m k v b -> m ())
-> (Table m k v b -> m (Table m k v b))
-> m (Table m k v b)
forall a b c. m a -> (a -> m b) -> (a -> m c) -> m c
forall (m :: * -> *) a b c.
MonadCatch m =>
m a -> (a -> m b) -> (a -> m c) -> m c
bracketOnError (NonEmpty (Table m k v b) -> m (Table m k v b)
forall (m :: * -> *) k v b.
IOLike m =>
NonEmpty (Table m k v b) -> m (Table m k v b)
incrementalUnions NonEmpty (Table m k v b)
tables) Table m k v b -> m ()
forall (m :: * -> *) k v b. IOLike m => Table m k v b -> m ()
closeTable ((Table m k v b -> m (Table m k v b)) -> m (Table m k v b))
-> (Table m k v b -> m (Table m k v b)) -> m (Table m k v b)
forall a b. (a -> b) -> a -> b
$ \Table m k v b
table -> do
    UnionDebt Int
debt <- Table m k v b -> m UnionDebt
forall (m :: * -> *) k v b.
IOLike m =>
Table m k v b -> m UnionDebt
remainingUnionDebt Table m k v b
table
    UnionCredits Int
leftovers <- Table m k v b -> UnionCredits -> m UnionCredits
forall (m :: * -> *) k v b.
(IOLike m, ResolveValue v) =>
Table m k v b -> UnionCredits -> m UnionCredits
supplyUnionCredits Table m k v b
table (Int -> UnionCredits
UnionCredits Int
debt)
    Bool -> m () -> m ()
forall a. HasCallStack => Bool -> a -> a
assert (Int
leftovers Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
>= Int
0) (m () -> m ()) -> m () -> m ()
forall a b. (a -> b) -> a -> b
$ () -> m ()
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure ()
    Table m k v b -> m (Table m k v b)
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure Table m k v b
table

{- |
Run an action with access to a table that incrementally computes the union of the given tables.

>>> :{
runExample $ \session table1 -> do
  LSMT.insert table1 0 "Hello" Nothing
  LSMT.withTable session $ \table2 -> do
    LSMT.insert table2 0 "World" Nothing
    LSMT.insert table2 1 "Goodbye" Nothing
    LSMT.withIncrementalUnion table1 table2 $ \table3 -> do
      print =<< LSMT.lookup table3 0
      print =<< LSMT.lookup table3 1
    print =<< LSMT.lookup table1 0
    print =<< LSMT.lookup table2 0
:}
Found (Value "Hello World")
Found (Value "Goodbye")
Found (Value "Hello")
Found (Value "World")

The worst-case disk I\/O complexity of this operation depends on the merge policy of the table:

['LazyLevelling']:
    \(O(T \log_T \frac{n}{B})\).

This function is exception-safe for both synchronous and asynchronous exceptions.

It is recommended to use this function instead of 'incrementalUnion' and 'closeTable'.

The created table has a /union debt/ which represents the amount of computation that remains. See 'remainingUnionDebt'.
The union debt can be paid off by supplying /union credit/ which performs an amount of computation proportional to the amount of union credit. See 'supplyUnionCredits'.
While a table has unresolved union debt, operations may become more expensive by a factor that scales with the number of unresolved unions.

__Warning:__ Both input tables must be from the same 'Session'.

Throws the following exceptions:

['SessionClosedError']:
    If the session is closed.
['TableClosedError']:
    If the table is closed.
['TableUnionNotCompatibleError']:
    If both tables are not from the same 'Session'.
-}
{-# SPECIALISE
  withIncrementalUnion ::
    Table IO k v b ->
    Table IO k v b ->
    (Table IO k v b -> IO a) ->
    IO a
  #-}
withIncrementalUnion ::
  forall m k v b a.
  (IOLike m) =>
  Table m k v b ->
  Table m k v b ->
  (Table m k v b -> m a) ->
  m a
withIncrementalUnion :: forall (m :: * -> *) k v b a.
IOLike m =>
Table m k v b -> Table m k v b -> (Table m k v b -> m a) -> m a
withIncrementalUnion Table m k v b
table1 Table m k v b
table2 =
  m (Table m k v b)
-> (Table m k v b -> m ()) -> (Table m k v b -> m a) -> m a
forall a b c. m a -> (a -> m b) -> (a -> m c) -> m c
forall (m :: * -> *) a b c.
MonadThrow m =>
m a -> (a -> m b) -> (a -> m c) -> m c
bracket (Table m k v b -> Table m k v b -> m (Table m k v b)
forall (m :: * -> *) k v b.
IOLike m =>
Table m k v b -> Table m k v b -> m (Table m k v b)
incrementalUnion Table m k v b
table1 Table m k v b
table2) Table m k v b -> m ()
forall (m :: * -> *) k v b. IOLike m => Table m k v b -> m ()
closeTable

{- |
Variant of 'withIncrementalUnion' that takes any number of tables.

The worst-case disk I\/O complexity of this operation depends on the merge policy of the table:

['LazyLevelling']:
    \(O(T \log_T \frac{n}{B} + b)\).

The variable \(b\) refers to the number of input tables.
-}
{-# SPECIALISE
  withIncrementalUnions ::
    NonEmpty (Table IO k v b) ->
    (Table IO k v b -> IO a) ->
    IO a
  #-}
withIncrementalUnions ::
  forall m k v b a.
  (IOLike m) =>
  NonEmpty (Table m k v b) ->
  (Table m k v b -> m a) ->
  m a
withIncrementalUnions :: forall (m :: * -> *) k v b a.
IOLike m =>
NonEmpty (Table m k v b) -> (Table m k v b -> m a) -> m a
withIncrementalUnions NonEmpty (Table m k v b)
tables =
  m (Table m k v b)
-> (Table m k v b -> m ()) -> (Table m k v b -> m a) -> m a
forall a b c. m a -> (a -> m b) -> (a -> m c) -> m c
forall (m :: * -> *) a b c.
MonadThrow m =>
m a -> (a -> m b) -> (a -> m c) -> m c
bracket (NonEmpty (Table m k v b) -> m (Table m k v b)
forall (m :: * -> *) k v b.
IOLike m =>
NonEmpty (Table m k v b) -> m (Table m k v b)
incrementalUnions NonEmpty (Table m k v b)
tables) Table m k v b -> m ()
forall (m :: * -> *) k v b. IOLike m => Table m k v b -> m ()
closeTable

{- |
Create a table that incrementally computes the union of the given tables.

>>> :{
runExample $ \session table1 -> do
  LSMT.insert table1 0 "Hello" Nothing
  LSMT.withTable session $ \table2 -> do
    LSMT.insert table2 0 "World" Nothing
    LSMT.insert table2 1 "Goodbye" Nothing
    bracket (LSMT.incrementalUnion table1 table2) LSMT.closeTable $ \table3 -> do
      print =<< LSMT.lookup table3 0
      print =<< LSMT.lookup table3 1
    print =<< LSMT.lookup table1 0
    print =<< LSMT.lookup table2 0
:}
Found (Value "Hello World")
Found (Value "Goodbye")
Found (Value "Hello")
Found (Value "World")

The worst-case disk I\/O complexity of this operation is \(O(1)\).

The created table has a /union debt/ which represents the amount of computation that remains. See 'remainingUnionDebt'.
The union debt can be paid off by supplying /union credit/ which performs an amount of computation proportional to the amount of union credit. See 'supplyUnionCredits'.
While a table has unresolved union debt, operations may become more expensive by a factor that scales with the number of unresolved unions.

__Warning:__ The new table must be independently closed using 'closeTable'.

__Warning:__ Both input tables must be from the same 'Session'.

Throws the following exceptions:

['SessionClosedError']:
    If the session is closed.
['TableClosedError']:
    If the table is closed.
['TableUnionNotCompatibleError']:
    If both tables are not from the same 'Session'.
-}
{-# SPECIALISE
  incrementalUnion ::
    Table IO k v b ->
    Table IO k v b ->
    IO (Table IO k v b)
  #-}
incrementalUnion ::
  forall m k v b.
  (IOLike m) =>
  Table m k v b ->
  Table m k v b ->
  m (Table m k v b)
incrementalUnion :: forall (m :: * -> *) k v b.
IOLike m =>
Table m k v b -> Table m k v b -> m (Table m k v b)
incrementalUnion Table m k v b
table1 Table m k v b
table2 = do
  NonEmpty (Table m k v b) -> m (Table m k v b)
forall (m :: * -> *) k v b.
IOLike m =>
NonEmpty (Table m k v b) -> m (Table m k v b)
incrementalUnions (Table m k v b
table1 Table m k v b -> [Table m k v b] -> NonEmpty (Table m k v b)
forall a. a -> [a] -> NonEmpty a
:| Table m k v b
table2 Table m k v b -> [Table m k v b] -> [Table m k v b]
forall a. a -> [a] -> [a]
: [])

{- |
Variant of 'incrementalUnion' for any number of tables.

The worst-case disk I\/O complexity of this operation is \(O(b)\),
where the variable \(b\) refers to the number of input tables.
-}
{-# SPECIALISE
  incrementalUnions ::
    NonEmpty (Table IO k v b) ->
    IO (Table IO k v b)
  #-}
incrementalUnions ::
  forall m k v b.
  (IOLike m) =>
  NonEmpty (Table m k v b) ->
  m (Table m k v b)
incrementalUnions :: forall (m :: * -> *) k v b.
IOLike m =>
NonEmpty (Table m k v b) -> m (Table m k v b)
incrementalUnions tables :: NonEmpty (Table m k v b)
tables@(Table Table m h
_ :| [Table m k v b]
_) =
  NonEmpty (Table m k v b)
-> (forall h.
    Typeable h =>
    NonEmpty (Table m h) -> m (Table m k v b))
-> m (Table m k v b)
forall (m :: * -> *) k v b a.
IOLike m =>
NonEmpty (Table m k v b)
-> (forall h. Typeable h => NonEmpty (Table m h) -> m a) -> m a
_withInternalTables NonEmpty (Table m k v b)
tables ((Table m h -> Table m k v b) -> m (Table m h) -> m (Table m k v b)
forall a b. (a -> b) -> m a -> m b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap Table m h -> Table m k v b
forall (m :: * -> *) k v b h.
Typeable h =>
Table m h -> Table m k v b
Table (m (Table m h) -> m (Table m k v b))
-> (NonEmpty (Table m h) -> m (Table m h))
-> NonEmpty (Table m h)
-> m (Table m k v b)
forall b c a. (b -> c) -> (a -> b) -> a -> c
. NonEmpty (Table m h) -> m (Table m h)
forall (m :: * -> *) h.
(MonadMask m, MonadMVar m, MonadST m, MonadSTM m) =>
NonEmpty (Table m h) -> m (Table m h)
Internal.unions)

-- | Internal helper. Run an action with access to the underlying tables.
{-# SPECIALISE
  _withInternalTables ::
    NonEmpty (Table IO k v b) ->
    (forall h. (Typeable h) => NonEmpty (Internal.Table IO h) -> IO a) ->
    IO a
  #-}
_withInternalTables ::
  forall m k v b a.
  (IOLike m) =>
  NonEmpty (Table m k v b) ->
  (forall h. (Typeable h) => NonEmpty (Internal.Table m h) -> m a) ->
  m a
_withInternalTables :: forall (m :: * -> *) k v b a.
IOLike m =>
NonEmpty (Table m k v b)
-> (forall h. Typeable h => NonEmpty (Table m h) -> m a) -> m a
_withInternalTables (Table (Table m h
table :: Internal.Table m h) :| [Table m k v b]
tables) forall h. Typeable h => NonEmpty (Table m h) -> m a
action =
  NonEmpty (Table m h) -> m a
forall h. Typeable h => NonEmpty (Table m h) -> m a
action (NonEmpty (Table m h) -> m a)
-> ([Table m h] -> NonEmpty (Table m h)) -> [Table m h] -> m a
forall b c a. (b -> c) -> (a -> b) -> a -> c
. (Table m h
table :|) ([Table m h] -> m a) -> m [Table m h] -> m a
forall (m :: * -> *) a b. Monad m => (a -> m b) -> m a -> m b
=<< ((Int, Table m k v b) -> m (Table m h))
-> [(Int, Table m k v b)] -> m [Table m h]
forall (t :: * -> *) (f :: * -> *) a b.
(Traversable t, Applicative f) =>
(a -> f b) -> t a -> f (t b)
forall (f :: * -> *) a b.
Applicative f =>
(a -> f b) -> [a] -> f [b]
traverse (Int, Table m k v b) -> m (Table m h)
assertTableType ([Int] -> [Table m k v b] -> [(Int, Table m k v b)]
forall a b. [a] -> [b] -> [(a, b)]
zip [Int
1 ..] [Table m k v b]
tables)
 where
  assertTableType :: (Int, Table m k v b) -> m (Internal.Table m h)
  assertTableType :: (Int, Table m k v b) -> m (Table m h)
assertTableType (Int
i, Table (Table m h
table' :: Internal.Table m h'))
    | Just h :~: h
Refl <- forall {k} (a :: k) (b :: k).
(Typeable a, Typeable b) =>
Maybe (a :~: b)
forall a b. (Typeable a, Typeable b) => Maybe (a :~: b)
eqT @h @h' = Table m h -> m (Table m h)
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure Table m h
Table m h
table'
    | Bool
otherwise = TableUnionNotCompatibleError -> m (Table m h)
forall e a. Exception e => e -> m a
forall (m :: * -> *) e a. (MonadThrow m, Exception e) => e -> m a
throwIO (TableUnionNotCompatibleError -> m (Table m h))
-> TableUnionNotCompatibleError -> m (Table m h)
forall a b. (a -> b) -> a -> b
$ Int -> TypeRep -> Int -> TypeRep -> TableUnionNotCompatibleError
ErrTableUnionHandleTypeMismatch Int
0 (Proxy h -> TypeRep
forall {k} (proxy :: k -> *) (a :: k).
Typeable a =>
proxy a -> TypeRep
typeRep (Proxy h -> TypeRep) -> Proxy h -> TypeRep
forall a b. (a -> b) -> a -> b
$ forall t. Proxy t
forall {k} (t :: k). Proxy t
Proxy @h) Int
i (Proxy h -> TypeRep
forall {k} (proxy :: k -> *) (a :: k).
Typeable a =>
proxy a -> TypeRep
typeRep (Proxy h -> TypeRep) -> Proxy h -> TypeRep
forall a b. (a -> b) -> a -> b
$ forall t. Proxy t
forall {k} (t :: k). Proxy t
Proxy @h')

{- |
Get an /upper bound/ for the amount of remaining union debt.
This includes the union debt of any table that was part of the union's input.

>>> :{
runExample $ \session table1 -> do
  LSMT.insert table1 0 "Hello" Nothing
  LSMT.withTable session $ \table2 -> do
    LSMT.insert table2 0 "World" Nothing
    LSMT.insert table2 1 "Goodbye" Nothing
    bracket (LSMT.incrementalUnion table1 table2) LSMT.closeTable $ \table3 -> do
      putStrLn . ("UnionDebt: "<>) . show =<< LSMT.remainingUnionDebt table3
:}
UnionDebt: 4

The worst-case disk I\/O complexity of this operation is \(O(0)\).

Throws the following exceptions:

['SessionClosedError']:
    If the session is closed.
['TableClosedError']:
    If the table is closed.
-}
{-# SPECIALISE
  remainingUnionDebt ::
    Table IO k v b ->
    IO UnionDebt
  #-}
remainingUnionDebt ::
  forall m k v b.
  (IOLike m) =>
  Table m k v b ->
  m UnionDebt
remainingUnionDebt :: forall (m :: * -> *) k v b.
IOLike m =>
Table m k v b -> m UnionDebt
remainingUnionDebt (Table Table m h
table) =
  Table m h -> m UnionDebt
forall (m :: * -> *) h.
(MonadSTM m, MonadMVar m, MonadThrow m, PrimMonad m) =>
Table m h -> m UnionDebt
Internal.remainingUnionDebt Table m h
table

{- |
Supply the given amount of union credits.

This reduces the union debt by /at least/ the number of supplied union credits.
It is therefore advisable to query 'remainingUnionDebt' every once in a while to get an upper bound on the current debt.

This function returns any surplus of union credits as /leftover/ credits when a union has finished.
In particular, if the returned number of credits is positive, then the union is finished.

>>> :{
runExample $ \session table1 -> do
  LSMT.insert table1 0 "Hello" Nothing
  LSMT.withTable session $ \table2 -> do
    LSMT.insert table2 0 "World" Nothing
    LSMT.insert table2 1 "Goodbye" Nothing
    bracket (LSMT.incrementalUnion table1 table2) LSMT.closeTable $ \table3 -> do
      putStrLn . ("UnionDebt: "<>) . show =<< LSMT.remainingUnionDebt table3
      putStrLn . ("Leftovers: "<>) . show =<< LSMT.supplyUnionCredits table3 2
      putStrLn . ("UnionDebt: "<>) . show =<< LSMT.remainingUnionDebt table3
      putStrLn . ("Leftovers: "<>) . show =<< LSMT.supplyUnionCredits table3 4
:}
UnionDebt: 4
Leftovers: 0
UnionDebt: 2
Leftovers: 3

__NOTE:__
The 'remainingUnionDebt' functions gets an /upper bound/ for the amount of remaning union debt.
In the example above, the second call to 'remainingUnionDebt' reports @2@, but the union debt is @1@.
Therefore, the second call to 'supplyUnionCredits' returns more leftovers than expected.

The worst-case disk I\/O complexity of this operation is \(O(\frac{b}{P})\),
where the variable \(b\) refers to the amount of credits supplied.

Throws the following exceptions:

['SessionClosedError']:
    If the session is closed.
['TableClosedError']:
    If the table is closed.
-}
{-# SPECIALISE
  supplyUnionCredits ::
    (ResolveValue v) =>
    Table IO k v b ->
    UnionCredits ->
    IO UnionCredits
  #-}
supplyUnionCredits ::
  forall m k v b.
  (IOLike m) =>
  (ResolveValue v) =>
  Table m k v b ->
  UnionCredits ->
  m UnionCredits
supplyUnionCredits :: forall (m :: * -> *) k v b.
(IOLike m, ResolveValue v) =>
Table m k v b -> UnionCredits -> m UnionCredits
supplyUnionCredits (Table Table m h
table :: Table m k v b) UnionCredits
credits =
  ResolveSerialisedValue
-> Table m h -> UnionCredits -> m UnionCredits
forall (m :: * -> *) h.
(MonadST m, MonadSTM m, MonadMVar m, MonadMask m) =>
ResolveSerialisedValue
-> Table m h -> UnionCredits -> m UnionCredits
Internal.supplyUnionCredits (Proxy v -> ResolveSerialisedValue
forall v. ResolveValue v => Proxy v -> ResolveSerialisedValue
_getResolveSerialisedValue (forall t. Proxy t
forall {k} (t :: k). Proxy t
Proxy @v)) Table m h
table UnionCredits
credits

--------------------------------------------------------------------------------
-- Blob References
--------------------------------------------------------------------------------

-- NOTE: 'BlobRef' is defined in 'Database.LSMTree.Internal.Types'

{- |
Retrieve the blob value from a blob reference.

>>> :{
runExample $ \session table -> do
  LSMT.insert table 0 "Hello" (Just "World")
  print
    =<< traverse (LSMT.retrieveBlob session)
    =<< LSMT.lookup table 0
:}
FoundWithBlob (Value "Hello") (Blob "World")

The worst-case disk I\/O complexity of this operation is \(O(1)\).

__Warning:__ A blob reference is /not stable/. Any operation that modifies the table,
cursor, or session that corresponds to a blob reference may cause it to be invalidated.

Throws the following exceptions:

['SessionClosedError']:
    If the session is closed.
['BlobRefInvalidError']:
    If the blob reference has been invalidated.
-}
{-# SPECIALISE
  retrieveBlob ::
    (SerialiseValue b) =>
    Session IO ->
    BlobRef IO b ->
    IO b
  #-}
retrieveBlob ::
  forall m b.
  (IOLike m, SerialiseValue b) =>
  Session m ->
  BlobRef m b ->
  m b
retrieveBlob :: forall (m :: * -> *) b.
(IOLike m, SerialiseValue b) =>
Session m -> BlobRef m b -> m b
retrieveBlob Session m
session BlobRef m b
blobRef = do
  Vector b
blobs <- Session m -> Vector (BlobRef m b) -> m (Vector b)
forall (m :: * -> *) b.
(IOLike m, SerialiseValue b) =>
Session m -> Vector (BlobRef m b) -> m (Vector b)
retrieveBlobs Session m
session (BlobRef m b -> Vector (BlobRef m b)
forall a. a -> Vector a
V.singleton BlobRef m b
blobRef)
  b -> m b
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (b -> m b) -> b -> m b
forall a b. (a -> b) -> a -> b
$ Vector b -> b
forall a. Vector a -> a
V.head Vector b
blobs

{- |
Variant of 'retrieveBlob' for batch retrieval.
The batch of blob references corresponds in-order to the batch of results.

The worst-case disk I\/O complexity of this operation is \(O(b)\),
where the variable \(b\) refers to the length of the input vector.

The following property holds in the absence of races:

prop> retrieveBlobs session blobRefs = traverse (retrieveBlob session) blobRefs
-}
{-# SPECIALISE
  retrieveBlobs ::
    (SerialiseValue b) =>
    Session IO ->
    Vector (BlobRef IO b) ->
    IO (Vector b)
  #-}
retrieveBlobs ::
  forall m b.
  (IOLike m, SerialiseValue b) =>
  Session m ->
  Vector (BlobRef m b) ->
  m (Vector b)
retrieveBlobs :: forall (m :: * -> *) b.
(IOLike m, SerialiseValue b) =>
Session m -> Vector (BlobRef m b) -> m (Vector b)
retrieveBlobs (Session (Session m h
session :: Internal.Session m h)) Vector (BlobRef m b)
blobRefs = do
  let numBlobRefs :: Int
numBlobRefs = Vector (BlobRef m b) -> Int
forall a. Vector a -> Int
V.length Vector (BlobRef m b)
blobRefs
  let blobRefNums :: Vector Int
blobRefNums = Int -> Int -> Vector Int
forall a. Enum a => a -> a -> Vector a
V.enumFromTo Int
0 (Int
numBlobRefs Int -> Int -> Int
forall a. Num a => a -> a -> a
- Int
1)
  Vector (WeakBlobRef m h)
weakBlobRefs <- ((Int, BlobRef m b) -> m (WeakBlobRef m h))
-> Vector (Int, BlobRef m b) -> m (Vector (WeakBlobRef m h))
forall (t :: * -> *) (f :: * -> *) a b.
(Traversable t, Applicative f) =>
(a -> f b) -> t a -> f (t b)
forall (f :: * -> *) a b.
Applicative f =>
(a -> f b) -> Vector a -> f (Vector b)
traverse (Int, BlobRef m b) -> m (WeakBlobRef m h)
assertBlobRefHandleType (Vector Int -> Vector (BlobRef m b) -> Vector (Int, BlobRef m b)
forall a b. Vector a -> Vector b -> Vector (a, b)
V.zip Vector Int
blobRefNums Vector (BlobRef m b)
blobRefs)
  Vector SerialisedBlob
serialisedBlobs <- Session m h
-> Vector (WeakBlobRef m h) -> m (Vector SerialisedBlob)
forall (m :: * -> *) h.
(MonadMask m, MonadST m, MonadSTM m) =>
Session m h
-> Vector (WeakBlobRef m h) -> m (Vector SerialisedBlob)
Internal.retrieveBlobs Session m h
session Vector (WeakBlobRef m h)
weakBlobRefs
  Vector b -> m (Vector b)
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (Vector b -> m (Vector b)) -> Vector b -> m (Vector b)
forall a b. (a -> b) -> a -> b
$ SerialisedBlob -> b
forall v. SerialiseValue v => SerialisedBlob -> v
Internal.deserialiseBlob (SerialisedBlob -> b) -> Vector SerialisedBlob -> Vector b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Vector SerialisedBlob
serialisedBlobs
 where
  assertBlobRefHandleType :: (Int, BlobRef m b) -> m (Internal.WeakBlobRef m h)
  assertBlobRefHandleType :: (Int, BlobRef m b) -> m (WeakBlobRef m h)
assertBlobRefHandleType (Int
i, BlobRef (WeakBlobRef m h
weakBlobRef :: Internal.WeakBlobRef m h'))
    | Just h :~: h
Refl <- forall {k} (a :: k) (b :: k).
(Typeable a, Typeable b) =>
Maybe (a :~: b)
forall a b. (Typeable a, Typeable b) => Maybe (a :~: b)
eqT @h @h' = WeakBlobRef m h -> m (WeakBlobRef m h)
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure WeakBlobRef m h
WeakBlobRef m h
weakBlobRef
    | Bool
otherwise = BlobRefInvalidError -> m (WeakBlobRef m h)
forall e a. Exception e => e -> m a
forall (m :: * -> *) e a. (MonadThrow m, Exception e) => e -> m a
throwIO (BlobRefInvalidError -> m (WeakBlobRef m h))
-> BlobRefInvalidError -> m (WeakBlobRef m h)
forall a b. (a -> b) -> a -> b
$ Int -> BlobRefInvalidError
ErrBlobRefInvalid Int
i

--------------------------------------------------------------------------------
-- Cursors
--------------------------------------------------------------------------------

-- NOTE: 'Cursor' is defined in 'Database.LSMTree.Internal.Types'

{- |
Run an action with access to a cursor.

>>> :{
runExample $ \session table -> do
  LSMT.insert table 0 "Hello" Nothing
  LSMT.insert table 1 "World" Nothing
  LSMT.withCursor table $ \cursor -> do
    traverse_ print
      =<< LSMT.take 32 cursor
:}
Entry (Key 0) (Value "Hello")
Entry (Key 1) (Value "World")

The worst-case disk I\/O complexity of this operation depends on the merge policy of the table:

['LazyLevelling']:
    \(O(T \log_T \frac{n}{B})\).

This function is exception-safe for both synchronous and asynchronous exceptions.

It is recommended to use this function instead of 'newCursor' and 'closeCursor'.

Throws the following exceptions:

['SessionClosedError']:
    If the session is closed.
['TableClosedError']:
    If the table is closed.
-}
{-# SPECIALISE
  withCursor ::
    (ResolveValue v) =>
    Table IO k v b ->
    (Cursor IO k v b -> IO a) ->
    IO a
  #-}
withCursor ::
  forall m k v b a.
  (IOLike m) =>
  (ResolveValue v) =>
  Table m k v b ->
  (Cursor m k v b -> m a) ->
  m a
withCursor :: forall (m :: * -> *) k v b a.
(IOLike m, ResolveValue v) =>
Table m k v b -> (Cursor m k v b -> m a) -> m a
withCursor (Table Table m h
table) Cursor m k v b -> m a
action =
  ResolveSerialisedValue
-> OffsetKey -> Table m h -> (Cursor m h -> m a) -> m a
forall (m :: * -> *) h a.
(MonadMask m, MonadMVar m, MonadST m, MonadSTM m) =>
ResolveSerialisedValue
-> OffsetKey -> Table m h -> (Cursor m h -> m a) -> m a
Internal.withCursor (Proxy v -> ResolveSerialisedValue
forall v. ResolveValue v => Proxy v -> ResolveSerialisedValue
_getResolveSerialisedValue (forall t. Proxy t
forall {k} (t :: k). Proxy t
Proxy @v)) OffsetKey
Internal.NoOffsetKey Table m h
table (Cursor m k v b -> m a
action (Cursor m k v b -> m a)
-> (Cursor m h -> Cursor m k v b) -> Cursor m h -> m a
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Cursor m h -> Cursor m k v b
forall (m :: * -> *) k v b h.
Typeable h =>
Cursor m h -> Cursor m k v b
Cursor)

{- |
Variant of 'withCursor' that starts at a given key.

>>> :{
runExample $ \session table -> do
  LSMT.insert table 0 "Hello" Nothing
  LSMT.insert table 1 "World" Nothing
  LSMT.withCursorAtOffset table 1 $ \cursor -> do
    traverse_ print
      =<< LSMT.take 32 cursor
:}
Entry (Key 1) (Value "World")
-}
{-# SPECIALISE
  withCursorAtOffset ::
    (SerialiseKey k, ResolveValue v) =>
    Table IO k v b ->
    k ->
    (Cursor IO k v b -> IO a) ->
    IO a
  #-}
withCursorAtOffset ::
  forall m k v b a.
  (IOLike m) =>
  (SerialiseKey k, ResolveValue v) =>
  Table m k v b ->
  k ->
  (Cursor m k v b -> m a) ->
  m a
withCursorAtOffset :: forall (m :: * -> *) k v b a.
(IOLike m, SerialiseKey k, ResolveValue v) =>
Table m k v b -> k -> (Cursor m k v b -> m a) -> m a
withCursorAtOffset (Table Table m h
table) k
offsetKey Cursor m k v b -> m a
action =
  ResolveSerialisedValue
-> OffsetKey -> Table m h -> (Cursor m h -> m a) -> m a
forall (m :: * -> *) h a.
(MonadMask m, MonadMVar m, MonadST m, MonadSTM m) =>
ResolveSerialisedValue
-> OffsetKey -> Table m h -> (Cursor m h -> m a) -> m a
Internal.withCursor (Proxy v -> ResolveSerialisedValue
forall v. ResolveValue v => Proxy v -> ResolveSerialisedValue
_getResolveSerialisedValue (forall t. Proxy t
forall {k} (t :: k). Proxy t
Proxy @v)) (SerialisedKey -> OffsetKey
Internal.OffsetKey (SerialisedKey -> OffsetKey) -> SerialisedKey -> OffsetKey
forall a b. (a -> b) -> a -> b
$ k -> SerialisedKey
forall k. SerialiseKey k => k -> SerialisedKey
Internal.serialiseKey k
offsetKey) Table m h
table (Cursor m k v b -> m a
action (Cursor m k v b -> m a)
-> (Cursor m h -> Cursor m k v b) -> Cursor m h -> m a
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Cursor m h -> Cursor m k v b
forall (m :: * -> *) k v b h.
Typeable h =>
Cursor m h -> Cursor m k v b
Cursor)

{- |
Create a cursor for the given table.

>>> :{
runExample $ \session table -> do
  LSMT.insert table 0 "Hello" Nothing
  LSMT.insert table 1 "World" Nothing
  bracket (LSMT.newCursor table) LSMT.closeCursor $ \cursor -> do
    traverse_ print
      =<< LSMT.take 32 cursor
:}
Entry (Key 0) (Value "Hello")
Entry (Key 1) (Value "World")

The worst-case disk I\/O complexity of this operation depends on the merge policy of the table:

['LazyLevelling']:
    \(O(T \log_T \frac{n}{B})\).

__Warning:__ Cursors hold open resources and must be closed using 'closeCursor'.

Throws the following exceptions:

['SessionClosedError']:
    If the session is closed.
['TableClosedError']:
    If the table is closed.
-}
{-# SPECIALISE
  newCursor ::
    (ResolveValue v) =>
    Table IO k v b ->
    IO (Cursor IO k v b)
  #-}
newCursor ::
  forall m k v b.
  (IOLike m) =>
  (ResolveValue v) =>
  Table m k v b ->
  m (Cursor m k v b)
newCursor :: forall (m :: * -> *) k v b.
(IOLike m, ResolveValue v) =>
Table m k v b -> m (Cursor m k v b)
newCursor (Table Table m h
table) =
  Cursor m h -> Cursor m k v b
forall (m :: * -> *) k v b h.
Typeable h =>
Cursor m h -> Cursor m k v b
Cursor (Cursor m h -> Cursor m k v b)
-> m (Cursor m h) -> m (Cursor m k v b)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> ResolveSerialisedValue -> OffsetKey -> Table m h -> m (Cursor m h)
forall (m :: * -> *) h.
(MonadMask m, MonadMVar m, MonadST m, MonadSTM m) =>
ResolveSerialisedValue -> OffsetKey -> Table m h -> m (Cursor m h)
Internal.newCursor (Proxy v -> ResolveSerialisedValue
forall v. ResolveValue v => Proxy v -> ResolveSerialisedValue
_getResolveSerialisedValue (forall t. Proxy t
forall {k} (t :: k). Proxy t
Proxy @v)) OffsetKey
Internal.NoOffsetKey Table m h
table

{- |
Variant of 'newCursor' that starts at a given key.

>>> :{
runExample $ \session table -> do
  LSMT.insert table 0 "Hello" Nothing
  LSMT.insert table 1 "World" Nothing
  bracket (LSMT.newCursorAtOffset table 1) LSMT.closeCursor $ \cursor -> do
    traverse_ print
      =<< LSMT.take 32 cursor
:}
Entry (Key 1) (Value "World")
-}
{-# SPECIALISE
  newCursorAtOffset ::
    (SerialiseKey k, ResolveValue v) =>
    Table IO k v b ->
    k ->
    IO (Cursor IO k v b)
  #-}
newCursorAtOffset ::
  forall m k v b.
  (IOLike m) =>
  (SerialiseKey k, ResolveValue v) =>
  Table m k v b ->
  k ->
  m (Cursor m k v b)
newCursorAtOffset :: forall (m :: * -> *) k v b.
(IOLike m, SerialiseKey k, ResolveValue v) =>
Table m k v b -> k -> m (Cursor m k v b)
newCursorAtOffset (Table Table m h
table) k
offsetKey =
  Cursor m h -> Cursor m k v b
forall (m :: * -> *) k v b h.
Typeable h =>
Cursor m h -> Cursor m k v b
Cursor (Cursor m h -> Cursor m k v b)
-> m (Cursor m h) -> m (Cursor m k v b)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> ResolveSerialisedValue -> OffsetKey -> Table m h -> m (Cursor m h)
forall (m :: * -> *) h.
(MonadMask m, MonadMVar m, MonadST m, MonadSTM m) =>
ResolveSerialisedValue -> OffsetKey -> Table m h -> m (Cursor m h)
Internal.newCursor (Proxy v -> ResolveSerialisedValue
forall v. ResolveValue v => Proxy v -> ResolveSerialisedValue
_getResolveSerialisedValue (forall t. Proxy t
forall {k} (t :: k). Proxy t
Proxy @v)) (SerialisedKey -> OffsetKey
Internal.OffsetKey (SerialisedKey -> OffsetKey) -> SerialisedKey -> OffsetKey
forall a b. (a -> b) -> a -> b
$ k -> SerialisedKey
forall k. SerialiseKey k => k -> SerialisedKey
Internal.serialiseKey k
offsetKey) Table m h
table

{- |
Close a cursor.

The worst-case disk I\/O complexity of this operation depends on the merge policy of the table:

['LazyLevelling']:
    \(O(T \log_T \frac{n}{B})\).

Closing is idempotent, i.e., closing a closed cursor does nothing.
All other operations on a closed cursor will throw an exception.
-}
{-# SPECIALISE
  closeCursor ::
    Cursor IO k v b ->
    IO ()
  #-}
closeCursor ::
  forall m k v b.
  (IOLike m) =>
  Cursor m k v b ->
  m ()
closeCursor :: forall (m :: * -> *) k v b. IOLike m => Cursor m k v b -> m ()
closeCursor (Cursor Cursor m h
cursor) =
  Cursor m h -> m ()
forall (m :: * -> *) h.
(MonadMask m, MonadMVar m, MonadSTM m, PrimMonad m) =>
Cursor m h -> m ()
Internal.closeCursor Cursor m h
cursor

{- |
Read the next table entry from the cursor.

>>> :{
runExample $ \session table -> do
  LSMT.insert table 0 "Hello" Nothing
  LSMT.insert table 1 "World" Nothing
  LSMT.withCursor table $ \cursor -> do
    print =<< LSMT.next cursor
    print =<< LSMT.next cursor
    print =<< LSMT.next cursor
:}
Just (Entry (Key 0) (Value "Hello"))
Just (Entry (Key 1) (Value "World"))
Nothing

The worst-case disk I\/O complexity of this operation is \(O(\frac{1}{P})\).

Throws the following exceptions:

['SessionClosedError']:
    If the session is closed.
['CursorClosedError']:
    If the cursor is closed.
-}
{-# SPECIALISE
  next ::
    (SerialiseKey k, SerialiseValue v, ResolveValue v) =>
    Cursor IO k v b ->
    IO (Maybe (Entry k v (BlobRef IO b)))
  #-}
next ::
  forall m k v b.
  (IOLike m) =>
  (SerialiseKey k, SerialiseValue v, ResolveValue v) =>
  Cursor m k v b ->
  m (Maybe (Entry k v (BlobRef m b)))
next :: forall (m :: * -> *) k v b.
(IOLike m, SerialiseKey k, SerialiseValue v, ResolveValue v) =>
Cursor m k v b -> m (Maybe (Entry k v (BlobRef m b)))
next Cursor m k v b
iterator = do
  -- TODO: implement this function in terms of 'readEntry'
  Vector (Entry k v (BlobRef m b))
entries <- Int -> Cursor m k v b -> m (Vector (Entry k v (BlobRef m b)))
forall (m :: * -> *) k v b.
(IOLike m, SerialiseKey k, SerialiseValue v, ResolveValue v) =>
Int -> Cursor m k v b -> m (Vector (Entry k v (BlobRef m b)))
take Int
1 Cursor m k v b
iterator
  Maybe (Entry k v (BlobRef m b))
-> m (Maybe (Entry k v (BlobRef m b)))
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (Maybe (Entry k v (BlobRef m b))
 -> m (Maybe (Entry k v (BlobRef m b))))
-> Maybe (Entry k v (BlobRef m b))
-> m (Maybe (Entry k v (BlobRef m b)))
forall a b. (a -> b) -> a -> b
$ (Entry k v (BlobRef m b), Vector (Entry k v (BlobRef m b)))
-> Entry k v (BlobRef m b)
forall a b. (a, b) -> a
fst ((Entry k v (BlobRef m b), Vector (Entry k v (BlobRef m b)))
 -> Entry k v (BlobRef m b))
-> Maybe
     (Entry k v (BlobRef m b), Vector (Entry k v (BlobRef m b)))
-> Maybe (Entry k v (BlobRef m b))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Vector (Entry k v (BlobRef m b))
-> Maybe
     (Entry k v (BlobRef m b), Vector (Entry k v (BlobRef m b)))
forall a. Vector a -> Maybe (a, Vector a)
V.uncons Vector (Entry k v (BlobRef m b))
entries

{- |
Read the next batch of table entries from the cursor.

>>> :{
runExample $ \session table -> do
  LSMT.insert table 0 "Hello" Nothing
  LSMT.insert table 1 "World" Nothing
  LSMT.withCursor table $ \cursor -> do
    traverse_ print
      =<< LSMT.take 32 cursor
:}
Entry (Key 0) (Value "Hello")
Entry (Key 1) (Value "World")

The worst-case disk I\/O complexity of this operation is \(O(\frac{b}{P})\),
where the variable \(b\) refers to the length of the /output/ vector,
which is /at most/ equal to the given number.
In practice, the length of the output vector is only less than the given number
once the cursor reaches the end of the table.

The following property holds:

prop> take n cursor = catMaybes <$> replicateM n (next cursor)

Throws the following exceptions:

['SessionClosedError']:
    If the session is closed.
['CursorClosedError']:
    If the cursor is closed.
-}
{-# SPECIALISE
  take ::
    (SerialiseKey k, SerialiseValue v, ResolveValue v) =>
    Int ->
    Cursor IO k v b ->
    IO (Vector (Entry k v (BlobRef IO b)))
  #-}
take ::
  forall m k v b.
  (IOLike m) =>
  (SerialiseKey k, SerialiseValue v, ResolveValue v) =>
  Int ->
  Cursor m k v b ->
  m (Vector (Entry k v (BlobRef m b)))
take :: forall (m :: * -> *) k v b.
(IOLike m, SerialiseKey k, SerialiseValue v, ResolveValue v) =>
Int -> Cursor m k v b -> m (Vector (Entry k v (BlobRef m b)))
take Int
n (Cursor Cursor m h
cursor :: Cursor m k v b) =
  ResolveSerialisedValue
-> Int
-> Cursor m h
-> (SerialisedKey
    -> SerialisedValue
    -> Maybe (WeakBlobRef m h)
    -> Entry k v (BlobRef m b))
-> m (Vector (Entry k v (BlobRef m b)))
forall (m :: * -> *) h res.
(MonadMask m, MonadMVar m, MonadST m, MonadSTM m) =>
ResolveSerialisedValue
-> Int
-> Cursor m h
-> (SerialisedKey
    -> SerialisedValue -> Maybe (WeakBlobRef m h) -> res)
-> m (Vector res)
Internal.readCursor (Proxy v -> ResolveSerialisedValue
forall v. ResolveValue v => Proxy v -> ResolveSerialisedValue
_getResolveSerialisedValue (forall t. Proxy t
forall {k} (t :: k). Proxy t
Proxy @v)) Int
n Cursor m h
cursor ((SerialisedKey
  -> SerialisedValue
  -> Maybe (WeakBlobRef m h)
  -> Entry k v (BlobRef m b))
 -> m (Vector (Entry k v (BlobRef m b))))
-> (SerialisedKey
    -> SerialisedValue
    -> Maybe (WeakBlobRef m h)
    -> Entry k v (BlobRef m b))
-> m (Vector (Entry k v (BlobRef m b)))
forall a b. (a -> b) -> a -> b
$ \ !SerialisedKey
k !SerialisedValue
v -> \case
    Just !WeakBlobRef m h
b -> k -> v -> BlobRef m b -> Entry k v (BlobRef m b)
forall k v b. k -> v -> b -> Entry k v b
EntryWithBlob (SerialisedKey -> k
forall k. SerialiseKey k => SerialisedKey -> k
Internal.deserialiseKey SerialisedKey
k) (SerialisedValue -> v
forall v. SerialiseValue v => SerialisedValue -> v
Internal.deserialiseValue SerialisedValue
v) (WeakBlobRef m h -> BlobRef m b
forall (m :: * -> *) b h.
Typeable h =>
WeakBlobRef m h -> BlobRef m b
BlobRef WeakBlobRef m h
b)
    Maybe (WeakBlobRef m h)
Nothing -> k -> v -> Entry k v (BlobRef m b)
forall k v b. k -> v -> Entry k v b
Entry (SerialisedKey -> k
forall k. SerialiseKey k => SerialisedKey -> k
Internal.deserialiseKey SerialisedKey
k) (SerialisedValue -> v
forall v. SerialiseValue v => SerialisedValue -> v
Internal.deserialiseValue SerialisedValue
v)

{- |
Variant of 'take' that accepts an additional predicate to determine whether or not to continue reading.

>>> :{
runExample $ \session table -> do
  LSMT.insert table 0 "Hello" Nothing
  LSMT.insert table 1 "World" Nothing
  LSMT.withCursor table $ \cursor -> do
    traverse_ print
      =<< LSMT.takeWhile 32 (<1) cursor
:}
Entry (Key 0) (Value "Hello")

The worst-case disk I\/O complexity of this operation is \(O(\frac{b}{P})\),
where the variable \(b\) refers to the length of the /output/ vector,
which is /at most/ equal to the given number.
In practice, the length of the output vector is only less than the given number
when the predicate returns false or the cursor reaches the end of the table.

The following properties hold:

prop> takeWhile n (const True) cursor = take n cursor
prop> takeWhile n (const False) cursor = pure empty

Throws the following exceptions:

['SessionClosedError']:
    If the session is closed.
['CursorClosedError']:
    If the cursor is closed.
-}
{-# SPECIALISE
  takeWhile ::
    (SerialiseKey k, SerialiseValue v, ResolveValue v) =>
    Int ->
    (k -> Bool) ->
    Cursor IO k v b ->
    IO (Vector (Entry k v (BlobRef IO b)))
  #-}
takeWhile ::
  forall m k v b.
  (IOLike m) =>
  (SerialiseKey k, SerialiseValue v, ResolveValue v) =>
  Int ->
  (k -> Bool) ->
  Cursor m k v b ->
  m (Vector (Entry k v (BlobRef m b)))
takeWhile :: forall (m :: * -> *) k v b.
(IOLike m, SerialiseKey k, SerialiseValue v, ResolveValue v) =>
Int
-> (k -> Bool)
-> Cursor m k v b
-> m (Vector (Entry k v (BlobRef m b)))
takeWhile Int
n k -> Bool
p (Cursor Cursor m h
cursor :: Cursor m k v b) =
  -- TODO: implement this function using a variant of 'readCursorWhile' that does not take the maximum batch size
  ResolveSerialisedValue
-> (SerialisedKey -> Bool)
-> Int
-> Cursor m h
-> (SerialisedKey
    -> SerialisedValue
    -> Maybe (WeakBlobRef m h)
    -> Entry k v (BlobRef m b))
-> m (Vector (Entry k v (BlobRef m b)))
forall (m :: * -> *) h res.
(MonadMask m, MonadMVar m, MonadST m, MonadSTM m) =>
ResolveSerialisedValue
-> (SerialisedKey -> Bool)
-> Int
-> Cursor m h
-> (SerialisedKey
    -> SerialisedValue -> Maybe (WeakBlobRef m h) -> res)
-> m (Vector res)
Internal.readCursorWhile (Proxy v -> ResolveSerialisedValue
forall v. ResolveValue v => Proxy v -> ResolveSerialisedValue
_getResolveSerialisedValue (forall t. Proxy t
forall {k} (t :: k). Proxy t
Proxy @v)) (k -> Bool
p (k -> Bool) -> (SerialisedKey -> k) -> SerialisedKey -> Bool
forall b c a. (b -> c) -> (a -> b) -> a -> c
. SerialisedKey -> k
forall k. SerialiseKey k => SerialisedKey -> k
Internal.deserialiseKey) Int
n Cursor m h
cursor ((SerialisedKey
  -> SerialisedValue
  -> Maybe (WeakBlobRef m h)
  -> Entry k v (BlobRef m b))
 -> m (Vector (Entry k v (BlobRef m b))))
-> (SerialisedKey
    -> SerialisedValue
    -> Maybe (WeakBlobRef m h)
    -> Entry k v (BlobRef m b))
-> m (Vector (Entry k v (BlobRef m b)))
forall a b. (a -> b) -> a -> b
$  \ !SerialisedKey
k !SerialisedValue
v -> \case
    Just !WeakBlobRef m h
b -> k -> v -> BlobRef m b -> Entry k v (BlobRef m b)
forall k v b. k -> v -> b -> Entry k v b
EntryWithBlob (SerialisedKey -> k
forall k. SerialiseKey k => SerialisedKey -> k
Internal.deserialiseKey SerialisedKey
k) (SerialisedValue -> v
forall v. SerialiseValue v => SerialisedValue -> v
Internal.deserialiseValue SerialisedValue
v) (WeakBlobRef m h -> BlobRef m b
forall (m :: * -> *) b h.
Typeable h =>
WeakBlobRef m h -> BlobRef m b
BlobRef WeakBlobRef m h
b)
    Maybe (WeakBlobRef m h)
Nothing -> k -> v -> Entry k v (BlobRef m b)
forall k v b. k -> v -> Entry k v b
Entry (SerialisedKey -> k
forall k. SerialiseKey k => SerialisedKey -> k
Internal.deserialiseKey SerialisedKey
k) (SerialisedValue -> v
forall v. SerialiseValue v => SerialisedValue -> v
Internal.deserialiseValue SerialisedValue
v)

--------------------------------------------------------------------------------
-- Snapshots
--------------------------------------------------------------------------------

{- |
Save the current state of the table to disk as a snapshot under the given
snapshot name. This is the /only/ mechanism that persists a table. Each snapshot
must have a unique name, which may be used to restore the table from that snapshot
using 'openTableFromSnapshot'.
Saving a snapshot /does not/ close the table.

Saving a snapshot is /relatively/ cheap when compared to opening a snapshot.
However, it is not so cheap that one should use it after every operation.

>>> :{
runExample $ \session table -> do
  LSMT.insert table 0 "Hello" Nothing
  LSMT.insert table 1 "World" Nothing
  LSMT.saveSnapshot "example" "Key Value Blob" table
:}

The worst-case disk I\/O complexity of this operation depends on the merge policy of the table:

['LazyLevelling']:
    \(O(T \log_T \frac{n}{B})\).

Throws the following exceptions:

['SessionClosedError']:
    If the session is closed.
['TableClosedError']:
    If the table is closed.
['SnapshotExistsError']:
    If a snapshot with the same name already exists.
-}
{-# SPECIALISE
  saveSnapshot ::
    SnapshotName ->
    SnapshotLabel ->
    Table IO k v b ->
    IO ()
  #-}
saveSnapshot ::
  forall m k v b.
  (IOLike m) =>
  SnapshotName ->
  SnapshotLabel ->
  Table m k v b ->
  m ()
saveSnapshot :: forall (m :: * -> *) k v b.
IOLike m =>
SnapshotName -> SnapshotLabel -> Table m k v b -> m ()
saveSnapshot SnapshotName
snapName SnapshotLabel
snapLabel (Table Table m h
table) =
  SnapshotName -> SnapshotLabel -> Table m h -> m ()
forall (m :: * -> *) h.
(MonadMask m, MonadMVar m, MonadST m, MonadSTM m) =>
SnapshotName -> SnapshotLabel -> Table m h -> m ()
Internal.saveSnapshot SnapshotName
snapName SnapshotLabel
snapLabel Table m h
table

{- |
Run an action with access to a table from a snapshot.

>>> :{
runExample $ \session table -> do
  -- Save snapshot
  LSMT.insert table 0 "Hello" Nothing
  LSMT.insert table 1 "World" Nothing
  LSMT.saveSnapshot "example" "Key Value Blob" table
  -- Open snapshot
  LSMT.withTableFromSnapshot @_ @Key @Value @Blob session "example" "Key Value Blob" $ \table' -> do
      LSMT.withCursor table' $ \cursor ->
        traverse_ print
          =<< LSMT.take 32 cursor
:}
Entry (Key 0) (Value "Hello")
Entry (Key 1) (Value "World")

The worst-case disk I\/O complexity of this operation is \(O(\frac{n}{P})\).

This function is exception-safe for both synchronous and asynchronous exceptions.

It is recommended to use this function instead of 'openTableFromSnapshot' and 'closeTable'.

Throws the following exceptions:

['SessionClosedError']:
    If the session is closed.
['TableClosedError']:
    If the table is closed.
['SnapshotDoesNotExistError']
    If no snapshot with the given name exists.
['SnapshotCorruptedError']:
    If the snapshot data is corrupted.
['SnapshotNotCompatibleError']:
    If the snapshot has a different label or is a different table type.
-}
{-# SPECIALISE
  withTableFromSnapshot ::
    (ResolveValue v) =>
    Session IO ->
    SnapshotName ->
    SnapshotLabel ->
    (Table IO k v b -> IO a) ->
    IO a
  #-}
withTableFromSnapshot ::
  forall m k v b a.
  (IOLike m) =>
  (ResolveValue v) =>
  Session m ->
  SnapshotName ->
  SnapshotLabel ->
  (Table m k v b -> m a) ->
  m a
withTableFromSnapshot :: forall (m :: * -> *) k v b a.
(IOLike m, ResolveValue v) =>
Session m
-> SnapshotName -> SnapshotLabel -> (Table m k v b -> m a) -> m a
withTableFromSnapshot Session m
session SnapshotName
snapName SnapshotLabel
snapLabel =
  m (Table m k v b)
-> (Table m k v b -> m ()) -> (Table m k v b -> m a) -> m a
forall a b c. m a -> (a -> m b) -> (a -> m c) -> m c
forall (m :: * -> *) a b c.
MonadThrow m =>
m a -> (a -> m b) -> (a -> m c) -> m c
bracket (Session m -> SnapshotName -> SnapshotLabel -> m (Table m k v b)
forall (m :: * -> *) k v b.
(IOLike m, ResolveValue v) =>
Session m -> SnapshotName -> SnapshotLabel -> m (Table m k v b)
openTableFromSnapshot Session m
session SnapshotName
snapName SnapshotLabel
snapLabel) Table m k v b -> m ()
forall (m :: * -> *) k v b. IOLike m => Table m k v b -> m ()
closeTable

{- |
Variant of 'withTableFromSnapshot' that accepts [table configuration overrides](#g:table_configuration_overrides).
-}
{-# SPECIALISE
  withTableFromSnapshotWith ::
    forall k v b a.
    (ResolveValue v) =>
    OverrideDiskCachePolicy ->
    Session IO ->
    SnapshotName ->
    SnapshotLabel ->
    (Table IO k v b -> IO a) ->
    IO a
  #-}
withTableFromSnapshotWith ::
  forall m k v b a.
  (IOLike m) =>
  (ResolveValue v) =>
  OverrideDiskCachePolicy ->
  Session m ->
  SnapshotName ->
  SnapshotLabel ->
  (Table m k v b -> m a) ->
  m a
withTableFromSnapshotWith :: forall (m :: * -> *) k v b a.
(IOLike m, ResolveValue v) =>
OverrideDiskCachePolicy
-> Session m
-> SnapshotName
-> SnapshotLabel
-> (Table m k v b -> m a)
-> m a
withTableFromSnapshotWith OverrideDiskCachePolicy
tableConfigOverride Session m
session SnapshotName
snapName SnapshotLabel
snapLabel =
  m (Table m k v b)
-> (Table m k v b -> m ()) -> (Table m k v b -> m a) -> m a
forall a b c. m a -> (a -> m b) -> (a -> m c) -> m c
forall (m :: * -> *) a b c.
MonadThrow m =>
m a -> (a -> m b) -> (a -> m c) -> m c
bracket (OverrideDiskCachePolicy
-> Session m -> SnapshotName -> SnapshotLabel -> m (Table m k v b)
forall (m :: * -> *) k v b.
(IOLike m, ResolveValue v) =>
OverrideDiskCachePolicy
-> Session m -> SnapshotName -> SnapshotLabel -> m (Table m k v b)
openTableFromSnapshotWith OverrideDiskCachePolicy
tableConfigOverride Session m
session SnapshotName
snapName SnapshotLabel
snapLabel) Table m k v b -> m ()
forall (m :: * -> *) k v b. IOLike m => Table m k v b -> m ()
closeTable

{- |
Open a table from a named snapshot.

>>> :{
runExample $ \session table -> do
  -- Save snapshot
  LSMT.insert table 0 "Hello" Nothing
  LSMT.insert table 1 "World" Nothing
  LSMT.saveSnapshot "example" "Key Value Blob" table
  -- Open snapshot
  bracket
    (LSMT.openTableFromSnapshot @_ @Key @Value @Blob session "example" "Key Value Blob")
    LSMT.closeTable $ \table' -> do
      LSMT.withCursor table' $ \cursor ->
        traverse_ print
          =<< LSMT.take 32 cursor
:}
Entry (Key 0) (Value "Hello")
Entry (Key 1) (Value "World")

The worst-case disk I\/O complexity of this operation is \(O(\frac{n}{P})\).

__Warning:__ The new table must be independently closed using 'closeTable'.

Throws the following exceptions:

['SessionClosedError']:
    If the session is closed.
['TableClosedError']:
    If the table is closed.
['SnapshotDoesNotExistError']
    If no snapshot with the given name exists.
['SnapshotCorruptedError']:
    If the snapshot data is corrupted.
['SnapshotNotCompatibleError']:
    If the snapshot has a different label or is a different table type.
-}
{-# SPECIALISE
  openTableFromSnapshot ::
    forall k v b.
    (ResolveValue v) =>
    Session IO ->
    SnapshotName ->
    SnapshotLabel ->
    IO (Table IO k v b)
  #-}
openTableFromSnapshot ::
  forall m k v b.
  (IOLike m) =>
  (ResolveValue v) =>
  Session m ->
  SnapshotName ->
  SnapshotLabel ->
  m (Table m k v b)
openTableFromSnapshot :: forall (m :: * -> *) k v b.
(IOLike m, ResolveValue v) =>
Session m -> SnapshotName -> SnapshotLabel -> m (Table m k v b)
openTableFromSnapshot Session m
session SnapshotName
snapName SnapshotLabel
snapLabel =
  OverrideDiskCachePolicy
-> Session m -> SnapshotName -> SnapshotLabel -> m (Table m k v b)
forall (m :: * -> *) k v b.
(IOLike m, ResolveValue v) =>
OverrideDiskCachePolicy
-> Session m -> SnapshotName -> SnapshotLabel -> m (Table m k v b)
openTableFromSnapshotWith OverrideDiskCachePolicy
NoOverrideDiskCachePolicy Session m
session SnapshotName
snapName SnapshotLabel
snapLabel

{- |
Variant of 'openTableFromSnapshot' that accepts [table configuration overrides](#g:table_configuration_overrides).
-}
{-# SPECIALISE
  openTableFromSnapshotWith ::
    forall k v b.
    (ResolveValue v) =>
    OverrideDiskCachePolicy ->
    Session IO ->
    SnapshotName ->
    SnapshotLabel ->
    IO (Table IO k v b)
  #-}
openTableFromSnapshotWith ::
  forall m k v b.
  (IOLike m) =>
  (ResolveValue v) =>
  OverrideDiskCachePolicy ->
  Session m ->
  SnapshotName ->
  SnapshotLabel ->
  m (Table m k v b)
openTableFromSnapshotWith :: forall (m :: * -> *) k v b.
(IOLike m, ResolveValue v) =>
OverrideDiskCachePolicy
-> Session m -> SnapshotName -> SnapshotLabel -> m (Table m k v b)
openTableFromSnapshotWith OverrideDiskCachePolicy
tableConfigOverride (Session Session m h
session) SnapshotName
snapName SnapshotLabel
snapLabel =
  Table m h -> Table m k v b
forall (m :: * -> *) k v b h.
Typeable h =>
Table m h -> Table m k v b
Table (Table m h -> Table m k v b) -> m (Table m h) -> m (Table m k v b)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> OverrideDiskCachePolicy
-> Session m h
-> SnapshotName
-> SnapshotLabel
-> ResolveSerialisedValue
-> m (Table m h)
forall (m :: * -> *) h.
(MonadMask m, MonadMVar m, MonadST m, MonadSTM m) =>
OverrideDiskCachePolicy
-> Session m h
-> SnapshotName
-> SnapshotLabel
-> ResolveSerialisedValue
-> m (Table m h)
Internal.openTableFromSnapshot OverrideDiskCachePolicy
tableConfigOverride Session m h
session SnapshotName
snapName SnapshotLabel
snapLabel (Proxy v -> ResolveSerialisedValue
forall v. ResolveValue v => Proxy v -> ResolveSerialisedValue
_getResolveSerialisedValue (forall t. Proxy t
forall {k} (t :: k). Proxy t
Proxy @v))

{- |
Delete the named snapshot.

>>> :{
runExample $ \session table -> do
  -- Save snapshot
  LSMT.insert table 0 "Hello" Nothing
  LSMT.insert table 1 "World" Nothing
  LSMT.saveSnapshot "example" "Key Value Blob" table
  -- Delete snapshot
  LSMT.deleteSnapshot session "example"
:}

The worst-case disk I\/O complexity of this operation depends on the merge policy of the table:

['LazyLevelling']:
    \(O(T \log_T \frac{n}{B})\).

Throws the following exceptions:

['SessionClosedError']:
    If the session is closed.
['SnapshotDoesNotExistError']:
    If no snapshot with the given name exists.
-}
{-# SPECIALISE
  deleteSnapshot ::
    Session IO ->
    SnapshotName ->
    IO ()
  #-}
deleteSnapshot ::
  forall m.
  (IOLike m) =>
  Session m ->
  SnapshotName ->
  m ()
deleteSnapshot :: forall (m :: * -> *). IOLike m => Session m -> SnapshotName -> m ()
deleteSnapshot (Session Session m h
session) =
  Session m h -> SnapshotName -> m ()
forall (m :: * -> *) h.
(MonadMask m, MonadSTM m) =>
Session m h -> SnapshotName -> m ()
Internal.deleteSnapshot Session m h
session

{- |
Check if the named snapshot exists.

>>> :{
runExample $ \session table -> do
  -- Save snapshot
  LSMT.insert table 0 "Hello" Nothing
  LSMT.insert table 1 "World" Nothing
  LSMT.saveSnapshot "example" "Key Value Blob" table
  -- Check snapshots
  print =<< doesSnapshotExist session "example"
  print =<< doesSnapshotExist session "this_snapshot_does_not_exist"
:}
True
False

The worst-case disk I\/O complexity of this operation is \(O(1)\).

Throws the following exceptions:

['SessionClosedError']:
    If the session is closed.
-}
{-# SPECIALISE
  doesSnapshotExist ::
    Session IO ->
    SnapshotName ->
    IO Bool
  #-}
doesSnapshotExist ::
  forall m.
  (IOLike m) =>
  Session m ->
  SnapshotName ->
  m Bool
doesSnapshotExist :: forall (m :: * -> *).
IOLike m =>
Session m -> SnapshotName -> m Bool
doesSnapshotExist (Session Session m h
session) =
  Session m h -> SnapshotName -> m Bool
forall (m :: * -> *) h.
(MonadMask m, MonadSTM m) =>
Session m h -> SnapshotName -> m Bool
Internal.doesSnapshotExist Session m h
session

{- |
List the names of all snapshots.

>>> :{
runExample $ \session table -> do
  -- Save snapshot
  LSMT.insert table 0 "Hello" Nothing
  LSMT.insert table 1 "World" Nothing
  LSMT.saveSnapshot "example" "Key Value Blob" table
  -- List snapshots
  traverse_ print
    =<< listSnapshots session
:}
"example"

The worst-case disk I\/O complexity of this operation is \(O(s)\),
where \(s\) refers to the number of snapshots in the session.

Throws the following exceptions:

['SessionClosedError']:
    If the session is closed.
-}
{-# SPECIALISE
  listSnapshots ::
    Session IO ->
    IO [SnapshotName]
  #-}
listSnapshots ::
  forall m.
  (IOLike m) =>
  Session m ->
  m [SnapshotName]
listSnapshots :: forall (m :: * -> *). IOLike m => Session m -> m [SnapshotName]
listSnapshots (Session Session m h
session) =
  Session m h -> m [SnapshotName]
forall (m :: * -> *) h.
(MonadMask m, MonadSTM m) =>
Session m h -> m [SnapshotName]
Internal.listSnapshots Session m h
session

-- | Internal helper. Get 'resolveSerialised' at type 'ResolveSerialisedValue'.
_getResolveSerialisedValue ::
  forall v.
  (ResolveValue v) =>
  Proxy v ->
  ResolveSerialisedValue
_getResolveSerialisedValue :: forall v. ResolveValue v => Proxy v -> ResolveSerialisedValue
_getResolveSerialisedValue = (RawBytes -> RawBytes -> RawBytes) -> ResolveSerialisedValue
forall a b. Coercible a b => a -> b
coerce ((RawBytes -> RawBytes -> RawBytes) -> ResolveSerialisedValue)
-> (Proxy v -> RawBytes -> RawBytes -> RawBytes)
-> Proxy v
-> ResolveSerialisedValue
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Proxy v -> RawBytes -> RawBytes -> RawBytes
forall v.
ResolveValue v =>
Proxy v -> RawBytes -> RawBytes -> RawBytes
resolveSerialised